From 9dc24eb18fd6740301f52cc8bd212f33225ba8da Mon Sep 17 00:00:00 2001 From: sumangala Date: Wed, 15 Sep 2021 22:18:11 +0530 Subject: [PATCH 01/77] config to req headers --- .../hadoop/fs/azurebfs/AbfsConfiguration.java | 27 ++++++ .../fs/azurebfs/AzureBlobFileSystem.java | 6 ++ .../fs/azurebfs/AzureBlobFileSystemStore.java | 19 +++-- .../azurebfs/constants/ConfigurationKeys.java | 3 + .../constants/HttpHeaderConfigurations.java | 1 + .../extensions/EncryptionContextProvider.java | 39 +++++++++ .../fs/azurebfs/services/AbfsClient.java | 82 +++++++++++++------ .../fs/azurebfs/ITestCustomerProvidedKey.java | 35 +++++--- 8 files changed, 170 insertions(+), 42 deletions(-) create mode 100644 hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/extensions/EncryptionContextProvider.java diff --git a/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/AbfsConfiguration.java b/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/AbfsConfiguration.java index 50cc57447f92b..debea20372c86 100644 --- a/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/AbfsConfiguration.java +++ b/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/AbfsConfiguration.java @@ -21,6 +21,7 @@ import java.io.IOException; import java.lang.reflect.Field; +import org.apache.hadoop.fs.azurebfs.extensions.EncryptionContextProvider; import org.apache.hadoop.thirdparty.com.google.common.annotations.VisibleForTesting; import org.apache.hadoop.thirdparty.com.google.common.base.Preconditions; @@ -906,6 +907,32 @@ public SASTokenProvider getSASTokenProvider() throws AzureBlobFileSystemExceptio } } + public EncryptionContextProvider initializeEncryptionContextProvider() { + try { + String configKey = FS_AZURE_ENCRYPTION_CONTEXT_PROVIDER_TYPE; + Class encryptionContextClass = + getAccountSpecificClass(configKey, null, EncryptionContextProvider.class); + if (encryptionContextClass == null) { + encryptionContextClass = getAccountAgnosticClass(configKey, null, + EncryptionContextProvider.class); + } + Preconditions.checkArgument(encryptionContextClass != null, + String.format("The configuration value for %s is invalid.", configKey)); + + EncryptionContextProvider encryptionContextProvider = + ReflectionUtils.newInstance(encryptionContextClass, rawConfig); + Preconditions.checkArgument(encryptionContextProvider != null, + String.format("Failed to initialize %s", encryptionContextClass)); + + LOG.trace("Initializing {}", encryptionContextClass.getName()); + LOG.trace("{} init complete", encryptionContextClass.getName()); + return encryptionContextProvider; + } catch (Exception e) { + throw new IllegalArgumentException("Unable to load encryption context provider class: ", e); + } + + } + public int getReadAheadRange() { return this.readAheadRange; } diff --git a/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/AzureBlobFileSystem.java b/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/AzureBlobFileSystem.java index 91274289f5415..fd7bece697cc2 100644 --- a/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/AzureBlobFileSystem.java +++ b/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/AzureBlobFileSystem.java @@ -42,6 +42,7 @@ import java.util.concurrent.Executors; import java.util.concurrent.Future; +import org.apache.commons.lang3.StringUtils; import org.apache.hadoop.io.IOUtils; import org.apache.hadoop.thirdparty.com.google.common.annotations.VisibleForTesting; import org.apache.hadoop.thirdparty.com.google.common.base.Preconditions; @@ -170,6 +171,11 @@ public void initialize(URI uri, Configuration configuration) LOG.debug("Created DelegationTokenManager {}", delegationTokenManager); } } +// // Initialize encryption context. +// // For fileSystemName, extract containerName from uri. +// abfsConfiguration.initializeEncryptionContextProvider() +// .initialize(configuration, abfsConfiguration.getAccountName(), +// StringUtils.substringBefore(uri.getAuthority(), "@")); AbfsClientThrottlingIntercept.initializeSingleton(abfsConfiguration.isAutoThrottlingEnabled()); diff --git a/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/AzureBlobFileSystemStore.java b/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/AzureBlobFileSystemStore.java index de6f676bab55a..c3750d00f9cef 100644 --- a/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/AzureBlobFileSystemStore.java +++ b/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/AzureBlobFileSystemStore.java @@ -52,6 +52,8 @@ import java.util.WeakHashMap; import java.util.concurrent.ExecutionException; +import org.apache.commons.lang3.StringUtils; +import org.apache.hadoop.fs.azurebfs.extensions.EncryptionContextProvider; import org.apache.hadoop.thirdparty.com.google.common.annotations.VisibleForTesting; import org.apache.hadoop.thirdparty.com.google.common.base.Preconditions; import org.apache.hadoop.thirdparty.com.google.common.base.Strings; @@ -453,7 +455,7 @@ public Hashtable getPathStatus(final Path path, public void setPathProperties(final Path path, final Hashtable properties, TracingContext tracingContext) - throws AzureBlobFileSystemException { + throws IOException { try (AbfsPerfInfo perfInfo = startTracking("setPathProperties", "setPathProperties")){ LOG.debug("setFilesystemProperties for filesystem: {} path: {} with properties: {}", client.getFileSystem(), @@ -813,8 +815,8 @@ public void breakLease(final Path path, final TracingContext tracingContext) thr client.breakLease(getRelativePath(path), tracingContext); } - public void rename(final Path source, final Path destination, TracingContext tracingContext) throws - AzureBlobFileSystemException { + public void rename(final Path source, final Path destination, TracingContext tracingContext) + throws IOException { final Instant startAggregate = abfsPerfTracker.getLatencyInstant(); long countAggregate = 0; boolean shouldContinue; @@ -1526,16 +1528,23 @@ private void initializeClient(URI uri, String fileSystemName, abfsConfiguration.getRawConfiguration()); } + // Initialize encryption context + EncryptionContextProvider encryptionContextProvider = + abfsConfiguration.initializeEncryptionContextProvider(); + encryptionContextProvider.initialize( + abfsConfiguration.getRawConfiguration(), accountName, fileSystemName); + LOG.trace("Initializing AbfsClient for {}", baseUrl); if (tokenProvider != null) { this.client = new AbfsClient(baseUrl, creds, abfsConfiguration, - tokenProvider, + tokenProvider, encryptionContextProvider, populateAbfsClientContext()); } else { this.client = new AbfsClient(baseUrl, creds, abfsConfiguration, - sasTokenProvider, + sasTokenProvider, encryptionContextProvider, populateAbfsClientContext()); } + LOG.trace("AbfsClient init complete"); } diff --git a/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/constants/ConfigurationKeys.java b/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/constants/ConfigurationKeys.java index 4a2c5951bd53d..7231442bea491 100644 --- a/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/constants/ConfigurationKeys.java +++ b/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/constants/ConfigurationKeys.java @@ -159,6 +159,9 @@ public final class ConfigurationKeys { public static final String FS_AZURE_ENABLE_ABFS_LIST_ITERATOR = "fs.azure.enable.abfslistiterator"; /** Server side encryption key */ public static final String FS_AZURE_CLIENT_PROVIDED_ENCRYPTION_KEY = "fs.azure.client-provided-encryption-key"; + /** Custom EncryptionContextProvider type */ + public static final String FS_AZURE_ENCRYPTION_CONTEXT_PROVIDER_TYPE = "fs.azure.encryption-context.provider.type"; + public static final String FS_AZURE_ENCRYPTION_MODE = "fs.azure.encryption.mode"; /** End point of ABFS account: {@value}. */ public static final String AZURE_ABFS_ENDPOINT = "fs.azure.abfs.endpoint"; diff --git a/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/constants/HttpHeaderConfigurations.java b/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/constants/HttpHeaderConfigurations.java index d4065ac2836d0..312677e32da2a 100644 --- a/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/constants/HttpHeaderConfigurations.java +++ b/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/constants/HttpHeaderConfigurations.java @@ -65,6 +65,7 @@ public final class HttpHeaderConfigurations { public static final String X_MS_ENCRYPTION_ALGORITHM = "x-ms-encryption-algorithm"; public static final String X_MS_REQUEST_SERVER_ENCRYPTED = "x-ms-request-server-encrypted"; public static final String X_MS_SERVER_ENCRYPTED = "x-ms-server-encrypted"; + public static final String X_MS_ENCRYPTION_CONTEXT = "x-ms-encryption-context"; public static final String X_MS_LEASE_ACTION = "x-ms-lease-action"; public static final String X_MS_LEASE_DURATION = "x-ms-lease-duration"; public static final String X_MS_LEASE_ID = "x-ms-lease-id"; diff --git a/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/extensions/EncryptionContextProvider.java b/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/extensions/EncryptionContextProvider.java new file mode 100644 index 0000000000000..3751e923793a8 --- /dev/null +++ b/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/extensions/EncryptionContextProvider.java @@ -0,0 +1,39 @@ +package org.apache.hadoop.fs.azurebfs.extensions; + +import org.apache.avro.util.ByteBufferInputStream; +import org.apache.hadoop.conf.Configuration; + +import java.io.IOException; + +public interface EncryptionContextProvider extends AutoCloseable { + /** + * Initialize instance + * + * @param configuration rawConfig instance + * @param accountName Account Name (with domain) + * @throws IOException error in initialization + */ + void initialize(Configuration configuration, String accountName, String fileSystem) throws IOException; + + /** + * Fetch encryption context for a given path + * + * @param path file path from filesystem root + * @return encryptionContext string + * @throws IOException + */ + ByteBufferInputStream getEncryptionContext(String path) throws IOException; + + /** + * Fetch encryption key in-exchange for encryption context + * + * @param path file path from filesystem root + * @param encryptionContext encryptionContext fetched from server + * @return Encryption key + * @throws IOException + */ + ByteBufferInputStream getEncryptionKey(String path, String encryptionContext) throws IOException; + + @Override + void close() throws IOException; +} diff --git a/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/services/AbfsClient.java b/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/services/AbfsClient.java index 27206959ba533..ad305f5001c11 100644 --- a/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/services/AbfsClient.java +++ b/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/services/AbfsClient.java @@ -38,6 +38,7 @@ import java.util.concurrent.ThreadFactory; import java.util.concurrent.TimeUnit; +import org.apache.hadoop.fs.azurebfs.extensions.EncryptionContextProvider; import org.apache.hadoop.thirdparty.com.google.common.annotations.VisibleForTesting; import org.apache.hadoop.thirdparty.com.google.common.base.Strings; import org.apache.hadoop.thirdparty.com.google.common.util.concurrent.FutureCallback; @@ -89,7 +90,7 @@ public class AbfsClient implements Closeable { private final AbfsConfiguration abfsConfiguration; private final String userAgent; private final AbfsPerfTracker abfsPerfTracker; - private final String clientProvidedEncryptionKey; + private String clientProvidedEncryptionKey; private final String clientProvidedEncryptionKeySHA; private final String accountName; @@ -97,13 +98,15 @@ public class AbfsClient implements Closeable { private AccessTokenProvider tokenProvider; private SASTokenProvider sasTokenProvider; private final AbfsCounters abfsCounters; + private final EncryptionContextProvider encryptionContextProvider; private final ListeningScheduledExecutorService executorService; - private AbfsClient(final URL baseUrl, final SharedKeyCredentials sharedKeyCredentials, - final AbfsConfiguration abfsConfiguration, - final AbfsClientContext abfsClientContext) - throws IOException { + private AbfsClient(final URL baseUrl, + final SharedKeyCredentials sharedKeyCredentials, + final AbfsConfiguration abfsConfiguration, + final EncryptionContextProvider encryptionContextProvider, + final AbfsClientContext abfsClientContext) throws IOException { this.baseUrl = baseUrl; this.sharedKeyCredentials = sharedKeyCredentials; String baseUrlString = baseUrl.toString(); @@ -112,6 +115,7 @@ private AbfsClient(final URL baseUrl, final SharedKeyCredentials sharedKeyCreden this.retryPolicy = abfsClientContext.getExponentialRetryPolicy(); this.accountName = abfsConfiguration.getAccountName().substring(0, abfsConfiguration.getAccountName().indexOf(AbfsHttpConstants.DOT)); this.authType = abfsConfiguration.getAuthType(accountName); + this.encryptionContextProvider = encryptionContextProvider; String encryptionKey = this.abfsConfiguration .getClientProvidedEncryptionKey(); @@ -152,18 +156,22 @@ private AbfsClient(final URL baseUrl, final SharedKeyCredentials sharedKeyCreden public AbfsClient(final URL baseUrl, final SharedKeyCredentials sharedKeyCredentials, final AbfsConfiguration abfsConfiguration, final AccessTokenProvider tokenProvider, + final EncryptionContextProvider encryptionContextProvider, final AbfsClientContext abfsClientContext) throws IOException { - this(baseUrl, sharedKeyCredentials, abfsConfiguration, abfsClientContext); + this(baseUrl, sharedKeyCredentials, abfsConfiguration, + encryptionContextProvider, abfsClientContext); this.tokenProvider = tokenProvider; } public AbfsClient(final URL baseUrl, final SharedKeyCredentials sharedKeyCredentials, final AbfsConfiguration abfsConfiguration, final SASTokenProvider sasTokenProvider, + final EncryptionContextProvider encryptionContextProvider, final AbfsClientContext abfsClientContext) throws IOException { - this(baseUrl, sharedKeyCredentials, abfsConfiguration, abfsClientContext); + this(baseUrl, sharedKeyCredentials, abfsConfiguration, + encryptionContextProvider, abfsClientContext); this.sasTokenProvider = sasTokenProvider; } @@ -184,6 +192,10 @@ private String getBase64EncodedString(byte[] bytes) { return Base64.getEncoder().encodeToString(bytes); } + public void setClientProvidedEncryptionKey(String key) { + clientProvidedEncryptionKey = key; + } + @Override public void close() throws IOException { if (tokenProvider instanceof Closeable) { @@ -221,7 +233,16 @@ List createDefaultHeaders() { } private void addCustomerProvidedKeyHeaders( - final List requestHeaders) { + final List requestHeaders, String path, TracingContext tracingContext) + throws IOException { + String encryptionContext = + getPathStatus(path, false, tracingContext).getResult().getResponseHeader(X_MS_ENCRYPTION_CONTEXT); + addCustomerProvidedKeyHeaders(requestHeaders, path, encryptionContext); + } + + private void addCustomerProvidedKeyHeaders( + List requestHeaders, String path, String encryptionContext) + throws IOException { if (clientProvidedEncryptionKey != null) { requestHeaders.add( new AbfsHttpHeader(X_MS_ENCRYPTION_KEY, clientProvidedEncryptionKey)); @@ -229,6 +250,13 @@ private void addCustomerProvidedKeyHeaders( clientProvidedEncryptionKeySHA)); requestHeaders.add(new AbfsHttpHeader(X_MS_ENCRYPTION_ALGORITHM, SERVER_SIDE_ENCRYPTION_ALGORITHM)); + } else if (encryptionContextProvider != null) { + requestHeaders.add( + new AbfsHttpHeader(X_MS_ENCRYPTION_CONTEXT, encryptionContext)); + requestHeaders.add(new AbfsHttpHeader(X_MS_ENCRYPTION_KEY, + encryptionContextProvider.getEncryptionKey(path, encryptionContext) + .toString())); + } } @@ -281,8 +309,9 @@ public AbfsRestOperation setFilesystemProperties(final String properties, Tracin public AbfsRestOperation listPath(final String relativePath, final boolean recursive, final int listMaxResults, final String continuation, TracingContext tracingContext) - throws AzureBlobFileSystemException { + throws IOException { final List requestHeaders = createDefaultHeaders(); + addCustomerProvidedKeyHeaders(requestHeaders, relativePath, tracingContext); final AbfsUriQueryBuilder abfsUriQueryBuilder = createDefaultUriQueryBuilder(); abfsUriQueryBuilder.addQuery(QUERY_PARAM_RESOURCE, FILESYSTEM); @@ -341,10 +370,13 @@ public AbfsRestOperation deleteFilesystem(TracingContext tracingContext) throws public AbfsRestOperation createPath(final String path, final boolean isFile, final boolean overwrite, final String permission, final String umask, final boolean isAppendBlob, final String eTag, - TracingContext tracingContext) throws AzureBlobFileSystemException { + TracingContext tracingContext) + throws IOException { final List requestHeaders = createDefaultHeaders(); if (isFile) { - addCustomerProvidedKeyHeaders(requestHeaders); + String encryptionContext = encryptionContextProvider.getEncryptionContext( + path).toString(); + addCustomerProvidedKeyHeaders(requestHeaders, path, encryptionContext); } if (!overwrite) { requestHeaders.add(new AbfsHttpHeader(IF_NONE_MATCH, AbfsHttpConstants.STAR)); @@ -477,7 +509,7 @@ public AbfsRestOperation breakLease(final String path, public AbfsRestOperation renamePath(String source, final String destination, final String continuation, TracingContext tracingContext) - throws AzureBlobFileSystemException { + throws IOException { final List requestHeaders = createDefaultHeaders(); String encodedRenameSource = urlEncode(FORWARD_SLASH + this.getFileSystem() + source); @@ -540,7 +572,7 @@ public AbfsRestOperation renameIdempotencyCheckOp( final Instant renameRequestStartTime, final AbfsRestOperation op, final String destination, - TracingContext tracingContext) throws AzureBlobFileSystemException { + TracingContext tracingContext) throws IOException { if ((op.isARetriedRequest()) && (op.getResult().getStatusCode() == HttpURLConnection.HTTP_NOT_FOUND)) { // Server has returned HTTP 404, which means rename source no longer @@ -571,9 +603,9 @@ public AbfsRestOperation renameIdempotencyCheckOp( public AbfsRestOperation append(final String path, final byte[] buffer, AppendRequestParameters reqParams, final String cachedSasToken, TracingContext tracingContext) - throws AzureBlobFileSystemException { + throws IOException { final List requestHeaders = createDefaultHeaders(); - addCustomerProvidedKeyHeaders(requestHeaders); + addCustomerProvidedKeyHeaders(requestHeaders, path, tracingContext); // JDK7 does not support PATCH, so to workaround the issue we will use // PUT and specify the real method in the X-Http-Method-Override header. requestHeaders.add(new AbfsHttpHeader(X_HTTP_METHOD_OVERRIDE, @@ -640,7 +672,8 @@ && appendSuccessCheckOp(op, path, // Hence, we pass/succeed the appendblob append call // in case we are doing a retry after checking the length of the file public boolean appendSuccessCheckOp(AbfsRestOperation op, final String path, - final long length, TracingContext tracingContext) throws AzureBlobFileSystemException { + final long length, TracingContext tracingContext) + throws IOException { if ((op.isARetriedRequest()) && (op.getResult().getStatusCode() == HttpURLConnection.HTTP_BAD_REQUEST)) { final AbfsRestOperation destStatusOp = getPathStatus(path, false, tracingContext); @@ -659,9 +692,9 @@ public boolean appendSuccessCheckOp(AbfsRestOperation op, final String path, public AbfsRestOperation flush(final String path, final long position, boolean retainUncommittedData, boolean isClose, final String cachedSasToken, final String leaseId, - TracingContext tracingContext) throws AzureBlobFileSystemException { + TracingContext tracingContext) throws IOException { final List requestHeaders = createDefaultHeaders(); - addCustomerProvidedKeyHeaders(requestHeaders); + addCustomerProvidedKeyHeaders(requestHeaders, path, tracingContext); // JDK7 does not support PATCH, so to workaround the issue we will use // PUT and specify the real method in the X-Http-Method-Override header. requestHeaders.add(new AbfsHttpHeader(X_HTTP_METHOD_OVERRIDE, @@ -692,9 +725,9 @@ public AbfsRestOperation flush(final String path, final long position, public AbfsRestOperation setPathProperties(final String path, final String properties, TracingContext tracingContext) - throws AzureBlobFileSystemException { + throws IOException { final List requestHeaders = createDefaultHeaders(); - addCustomerProvidedKeyHeaders(requestHeaders); + addCustomerProvidedKeyHeaders(requestHeaders, path, tracingContext); // JDK7 does not support PATCH, so to workaround the issue we will use // PUT and specify the real method in the X-Http-Method-Override header. requestHeaders.add(new AbfsHttpHeader(X_HTTP_METHOD_OVERRIDE, @@ -718,7 +751,8 @@ public AbfsRestOperation setPathProperties(final String path, final String prope } public AbfsRestOperation getPathStatus(final String path, final boolean includeProperties, - TracingContext tracingContext) throws AzureBlobFileSystemException { + TracingContext tracingContext) + throws IOException { final List requestHeaders = createDefaultHeaders(); final AbfsUriQueryBuilder abfsUriQueryBuilder = createDefaultUriQueryBuilder(); @@ -730,7 +764,7 @@ public AbfsRestOperation getPathStatus(final String path, final boolean includeP abfsUriQueryBuilder.addQuery(HttpQueryParams.QUERY_PARAM_ACTION, AbfsHttpConstants.GET_STATUS); operation = SASTokenProvider.GET_STATUS_OPERATION; } else { - addCustomerProvidedKeyHeaders(requestHeaders); + addCustomerProvidedKeyHeaders(requestHeaders, path, tracingContext); } abfsUriQueryBuilder.addQuery(HttpQueryParams.QUERY_PARAM_UPN, String.valueOf(abfsConfiguration.isUpnUsed())); appendSASTokenToQuery(path, operation, abfsUriQueryBuilder); @@ -748,9 +782,9 @@ public AbfsRestOperation getPathStatus(final String path, final boolean includeP public AbfsRestOperation read(final String path, final long position, final byte[] buffer, final int bufferOffset, final int bufferLength, final String eTag, String cachedSasToken, - TracingContext tracingContext) throws AzureBlobFileSystemException { + TracingContext tracingContext) throws IOException { final List requestHeaders = createDefaultHeaders(); - addCustomerProvidedKeyHeaders(requestHeaders); + addCustomerProvidedKeyHeaders(requestHeaders, path, tracingContext); requestHeaders.add(new AbfsHttpHeader(RANGE, String.format("bytes=%d-%d", position, position + bufferLength - 1))); requestHeaders.add(new AbfsHttpHeader(IF_MATCH, eTag)); diff --git a/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/ITestCustomerProvidedKey.java b/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/ITestCustomerProvidedKey.java index b7b1a3bba2db6..6f5066012ba2e 100644 --- a/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/ITestCustomerProvidedKey.java +++ b/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/ITestCustomerProvidedKey.java @@ -35,6 +35,7 @@ import java.util.Random; import java.util.UUID; +import org.apache.commons.lang3.StringUtils; import org.apache.hadoop.fs.azurebfs.utils.TracingContext; import org.apache.hadoop.fs.contract.ContractTestUtils; import org.apache.hadoop.thirdparty.com.google.common.base.Preconditions; @@ -278,6 +279,12 @@ public void testAppendWithoutCPK() throws Exception { @Test public void testSetGetXAttr() throws Exception { final AzureBlobFileSystem fs = getAbfs(true); + System.out.println(StringUtils.substringBefore(fs.getUri().getAuthority(), + "@") +" \n "+ + fs.getUri().getAuthority()+" \n "+ + fs.getUri().getAuthority()+"\n "+ + fs.getUri().getHost() + " \n " + fs.getUri().getPort() + + fs.getUri().getFragment() + " \n " + fs.getUri().parseServerAuthority()); final String fileName = path(methodName.getMethodName()).toString(); createFileAndGetContent(fs, fileName, FILE_SIZE); @@ -411,7 +418,7 @@ public void testListPathWithoutCPK() throws Exception { private void testListPath(final boolean isWithCPK) throws Exception { final AzureBlobFileSystem fs = getAbfs(isWithCPK); - final Path testPath = path("/" + methodName.getMethodName()); + final Path testPath = new Path("/" + methodName.getMethodName()); String testDirName = testPath.toString(); fs.mkdirs(testPath); createFileAndGetContent(fs, testDirName + "/aaa", FILE_SIZE); @@ -429,6 +436,7 @@ private void testListPath(final boolean isWithCPK) throws Exception { "different-1234567890123456789012"); AzureBlobFileSystem fs2 = (AzureBlobFileSystem) FileSystem.newInstance(conf); AbfsClient abfsClient2 = fs2.getAbfsClient(); + abfsClient2.setClientProvidedEncryptionKey(null); TracingContext tracingContext = getTestTracingContext(fs, false); abfsRestOperation = abfsClient2.listPath(testDirName, false, INT_50, null, tracingContext); @@ -471,8 +479,7 @@ public void testCreatePathWithoutCPK() throws Exception { private void testCreatePath(final boolean isWithCPK) throws Exception { final AzureBlobFileSystem fs = getAbfs(isWithCPK); - final String testFileName = path("/" + methodName.getMethodName()) - .toString(); + final String testFileName = new Path("/" + methodName.getMethodName()).toString(); createFileAndGetContent(fs, testFileName, FILE_SIZE); AbfsClient abfsClient = fs.getAbfsClient(); @@ -515,7 +522,7 @@ public void testRenamePathWithoutCPK() throws Exception { private void testRenamePath(final boolean isWithCPK) throws Exception { final AzureBlobFileSystem fs = getAbfs(isWithCPK); - final String testFileName = path("/" + methodName.getMethodName()) + final String testFileName = new Path("/" + methodName.getMethodName()) .toString(); createFileAndGetContent(fs, testFileName, FILE_SIZE); @@ -551,7 +558,7 @@ public void testFlushWithoutCPK() throws Exception { private void testFlush(final boolean isWithCPK) throws Exception { final AzureBlobFileSystem fs = getAbfs(isWithCPK); - final String testFileName = path("/" + methodName.getMethodName()) + final String testFileName = new Path("/" + methodName.getMethodName()) .toString(); fs.create(new Path(testFileName)).close(); AbfsClient abfsClient = fs.getAbfsClient(); @@ -612,7 +619,7 @@ public void testSetPathPropertiesWithoutCPK() throws Exception { private void testSetPathProperties(final boolean isWithCPK) throws Exception { final AzureBlobFileSystem fs = getAbfs(isWithCPK); - final String testFileName = path("/" + methodName.getMethodName()) + final String testFileName = new Path("/" + methodName.getMethodName()) .toString(); createFileAndGetContent(fs, testFileName, FILE_SIZE); @@ -643,7 +650,7 @@ public void testGetPathStatusFileWithoutCPK() throws Exception { private void testGetPathStatusFile(final boolean isWithCPK) throws Exception { final AzureBlobFileSystem fs = getAbfs(isWithCPK); - final String testFileName = path("/" + methodName.getMethodName()) + final String testFileName = new Path("/" + methodName.getMethodName()) .toString(); createFileAndGetContent(fs, testFileName, FILE_SIZE); @@ -659,7 +666,9 @@ private void testGetPathStatusFile(final boolean isWithCPK) throws Exception { assertResponseHeader(abfsRestOperation, false, X_MS_REQUEST_SERVER_ENCRYPTED, ""); - abfsRestOperation = abfsClient.getPathStatus(testFileName, true, tracingContext); + abfsClient.setClientProvidedEncryptionKey(null); + abfsRestOperation = abfsClient.getPathStatus(testFileName, false, + tracingContext); assertCPKHeaders(abfsRestOperation, isWithCPK); assertResponseHeader(abfsRestOperation, isWithCPK, X_MS_ENCRYPTION_KEY_SHA256, getCPKSha(fs)); @@ -681,7 +690,7 @@ public void testDeletePathWithoutCPK() throws Exception { private void testDeletePath(final boolean isWithCPK) throws Exception { final AzureBlobFileSystem fs = getAbfs(isWithCPK); - final String testFileName = path("/" + methodName.getMethodName()) + final String testFileName = new Path("/" + methodName.getMethodName()) .toString(); createFileAndGetContent(fs, testFileName, FILE_SIZE); @@ -712,7 +721,7 @@ public void testSetPermissionWithoutCPK() throws Exception { private void testSetPermission(final boolean isWithCPK) throws Exception { final AzureBlobFileSystem fs = getAbfs(isWithCPK); - final String testFileName = path("/" + methodName.getMethodName()) + final String testFileName = new Path("/" + methodName.getMethodName()) .toString(); Assume.assumeTrue(fs.getIsNamespaceEnabled(getTestTracingContext(fs, false))); createFileAndGetContent(fs, testFileName, FILE_SIZE); @@ -738,7 +747,7 @@ public void testSetAclWithoutCPK() throws Exception { private void testSetAcl(final boolean isWithCPK) throws Exception { final AzureBlobFileSystem fs = getAbfs(isWithCPK); - final String testFileName = path("/" + methodName.getMethodName()) + final String testFileName = new Path("/" + methodName.getMethodName()) .toString(); TracingContext tracingContext = getTestTracingContext(fs, false); Assume.assumeTrue(fs.getIsNamespaceEnabled(tracingContext)); @@ -768,7 +777,7 @@ public void testGetAclWithoutCPK() throws Exception { private void testGetAcl(final boolean isWithCPK) throws Exception { final AzureBlobFileSystem fs = getAbfs(isWithCPK); - final String testFileName = path("/" + methodName.getMethodName()) + final String testFileName = new Path("/" + methodName.getMethodName()) .toString(); TracingContext tracingContext = getTestTracingContext(fs, false); Assume.assumeTrue(fs.getIsNamespaceEnabled(tracingContext)); @@ -799,7 +808,7 @@ private void testCheckAccess(final boolean isWithCPK) throws Exception { getAuthType() == AuthType.OAuth); final AzureBlobFileSystem fs = getAbfs(isWithCPK); - final String testFileName = path("/" + methodName.getMethodName()) + final String testFileName = new Path("/" + methodName.getMethodName()) .toString(); fs.create(new Path(testFileName)).close(); AbfsClient abfsClient = fs.getAbfsClient(); From bd579f2e8df490ea91cc92b6c712ff76a4ac222c Mon Sep 17 00:00:00 2001 From: sumangala Date: Thu, 16 Sep 2021 17:25:33 +0530 Subject: [PATCH 02/77] cache keys, refactor client encr into 1 fn, outstream --- .../hadoop/fs/azurebfs/AbfsConfiguration.java | 1 + .../fs/azurebfs/AzureBlobFileSystem.java | 6 -- .../fs/azurebfs/AzureBlobFileSystemStore.java | 31 ++++-- .../constants/HttpHeaderConfigurations.java | 1 + .../extensions/EncryptionContextProvider.java | 5 +- .../fs/azurebfs/services/AbfsClient.java | 94 +++++++++++-------- .../azurebfs/services/AbfsHttpOperation.java | 13 +++ .../azurebfs/services/AbfsOutputStream.java | 17 ++-- .../services/AbfsOutputStreamContext.java | 14 +++ .../fs/azurebfs/utils/EncryptionType.java | 13 +++ .../fs/azurebfs/ITestCustomerProvidedKey.java | 8 -- .../MockEncryptionContextProvider.java | 47 ++++++++++ 12 files changed, 182 insertions(+), 68 deletions(-) create mode 100644 hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/utils/EncryptionType.java create mode 100644 hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/extensions/MockEncryptionContextProvider.java diff --git a/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/AbfsConfiguration.java b/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/AbfsConfiguration.java index debea20372c86..cf240e73edf91 100644 --- a/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/AbfsConfiguration.java +++ b/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/AbfsConfiguration.java @@ -908,6 +908,7 @@ public SASTokenProvider getSASTokenProvider() throws AzureBlobFileSystemExceptio } public EncryptionContextProvider initializeEncryptionContextProvider() { + try { String configKey = FS_AZURE_ENCRYPTION_CONTEXT_PROVIDER_TYPE; Class encryptionContextClass = diff --git a/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/AzureBlobFileSystem.java b/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/AzureBlobFileSystem.java index fd7bece697cc2..91274289f5415 100644 --- a/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/AzureBlobFileSystem.java +++ b/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/AzureBlobFileSystem.java @@ -42,7 +42,6 @@ import java.util.concurrent.Executors; import java.util.concurrent.Future; -import org.apache.commons.lang3.StringUtils; import org.apache.hadoop.io.IOUtils; import org.apache.hadoop.thirdparty.com.google.common.annotations.VisibleForTesting; import org.apache.hadoop.thirdparty.com.google.common.base.Preconditions; @@ -171,11 +170,6 @@ public void initialize(URI uri, Configuration configuration) LOG.debug("Created DelegationTokenManager {}", delegationTokenManager); } } -// // Initialize encryption context. -// // For fileSystemName, extract containerName from uri. -// abfsConfiguration.initializeEncryptionContextProvider() -// .initialize(configuration, abfsConfiguration.getAccountName(), -// StringUtils.substringBefore(uri.getAuthority(), "@")); AbfsClientThrottlingIntercept.initializeSingleton(abfsConfiguration.isAutoThrottlingEnabled()); diff --git a/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/AzureBlobFileSystemStore.java b/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/AzureBlobFileSystemStore.java index c3750d00f9cef..1f2f810010d01 100644 --- a/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/AzureBlobFileSystemStore.java +++ b/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/AzureBlobFileSystemStore.java @@ -54,6 +54,7 @@ import org.apache.commons.lang3.StringUtils; import org.apache.hadoop.fs.azurebfs.extensions.EncryptionContextProvider; +import org.apache.hadoop.fs.azurebfs.utils.EncryptionType; import org.apache.hadoop.thirdparty.com.google.common.annotations.VisibleForTesting; import org.apache.hadoop.thirdparty.com.google.common.base.Preconditions; import org.apache.hadoop.thirdparty.com.google.common.base.Strings; @@ -432,7 +433,7 @@ public void setFilesystemProperties( } public Hashtable getPathStatus(final Path path, - TracingContext tracingContext) throws AzureBlobFileSystemException { + TracingContext tracingContext) throws IOException { try (AbfsPerfInfo perfInfo = startTracking("getPathStatus", "getPathStatus")){ LOG.debug("getPathStatus for filesystem: {} path: {}", client.getFileSystem(), @@ -500,7 +501,7 @@ public void deleteFilesystem(TracingContext tracingContext) public OutputStream createFile(final Path path, final FileSystem.Statistics statistics, final boolean overwrite, final FsPermission permission, final FsPermission umask, - TracingContext tracingContext) throws AzureBlobFileSystemException { + TracingContext tracingContext) throws IOException { try (AbfsPerfInfo perfInfo = startTracking("createFile", "createPath")) { boolean isNamespaceEnabled = getIsNamespaceEnabled(tracingContext); LOG.debug("createFile filesystem: {} path: {} overwrite: {} permission: {} umask: {} isNamespaceEnabled: {}", @@ -555,7 +556,8 @@ public OutputStream createFile(final Path path, statistics, relativePath, 0, - populateAbfsOutputStreamContext(isAppendBlob, lease), + populateAbfsOutputStreamContext(isAppendBlob, lease, + op.getResult().getEncryptionKeyHeaders()), tracingContext); } } @@ -576,7 +578,7 @@ private AbfsRestOperation conditionalCreateOverwriteFile(final String relativePa final String permission, final String umask, final boolean isAppendBlob, - TracingContext tracingContext) throws AzureBlobFileSystemException { + TracingContext tracingContext) throws IOException { AbfsRestOperation op; try { @@ -631,7 +633,7 @@ private AbfsRestOperation conditionalCreateOverwriteFile(final String relativePa } private AbfsOutputStreamContext populateAbfsOutputStreamContext(boolean isAppendBlob, - AbfsLease lease) { + AbfsLease lease, HashMap encryptionHeaders) { int bufferSize = abfsConfiguration.getWriteBufferSize(); if (isAppendBlob && bufferSize > FileSystemConfigurations.APPENDBLOB_MAX_WRITE_BUFFER_SIZE) { bufferSize = FileSystemConfigurations.APPENDBLOB_MAX_WRITE_BUFFER_SIZE; @@ -646,12 +648,13 @@ private AbfsOutputStreamContext populateAbfsOutputStreamContext(boolean isAppend .withWriteMaxConcurrentRequestCount(abfsConfiguration.getWriteMaxConcurrentRequestCount()) .withMaxWriteRequestsToQueue(abfsConfiguration.getMaxWriteRequestsToQueue()) .withLease(lease) + .withEncryptionHeaders(encryptionHeaders) .build(); } public void createDirectory(final Path path, final FsPermission permission, final FsPermission umask, TracingContext tracingContext) - throws AzureBlobFileSystemException { + throws IOException { try (AbfsPerfInfo perfInfo = startTracking("createDirectory", "createPath")) { boolean isNamespaceEnabled = getIsNamespaceEnabled(tracingContext); LOG.debug("createDirectory filesystem: {} path: {} permission: {} umask: {} isNamespaceEnabled: {}", @@ -757,7 +760,7 @@ private AbfsInputStreamContext populateAbfsInputStreamContext( public OutputStream openFileForWrite(final Path path, final FileSystem.Statistics statistics, final boolean overwrite, - TracingContext tracingContext) throws AzureBlobFileSystemException { + TracingContext tracingContext) throws IOException { try (AbfsPerfInfo perfInfo = startTracking("openFileForWrite", "getPathStatus")) { LOG.debug("openFileForWrite filesystem: {} path: {} overwrite: {}", client.getFileSystem(), @@ -797,7 +800,8 @@ public OutputStream openFileForWrite(final Path path, statistics, relativePath, offset, - populateAbfsOutputStreamContext(isAppendBlob, lease), + populateAbfsOutputStreamContext(isAppendBlob, lease, + op.getResult().getEncryptionKeyHeaders()), tracingContext); } } @@ -1531,6 +1535,17 @@ private void initializeClient(URI uri, String fileSystemName, // Initialize encryption context EncryptionContextProvider encryptionContextProvider = abfsConfiguration.initializeEncryptionContextProvider(); + if (encryptionContextProvider != null) { + if (abfsConfiguration.getClientProvidedEncryptionKey() != null) { + throw new IOException( + "Both global key and encryption context are set, only one allowed"); + } + client.setEncryptionType(EncryptionType.ENCRYPTION_CONTEXT); + } else if (abfsConfiguration.getClientProvidedEncryptionKey() != null) { + client.setEncryptionType(EncryptionType.GLOBAL_KEY); + } else { + client.setEncryptionType(EncryptionType.NONE); + } encryptionContextProvider.initialize( abfsConfiguration.getRawConfiguration(), accountName, fileSystemName); diff --git a/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/constants/HttpHeaderConfigurations.java b/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/constants/HttpHeaderConfigurations.java index 312677e32da2a..2f4e705fdd87e 100644 --- a/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/constants/HttpHeaderConfigurations.java +++ b/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/constants/HttpHeaderConfigurations.java @@ -66,6 +66,7 @@ public final class HttpHeaderConfigurations { public static final String X_MS_REQUEST_SERVER_ENCRYPTED = "x-ms-request-server-encrypted"; public static final String X_MS_SERVER_ENCRYPTED = "x-ms-server-encrypted"; public static final String X_MS_ENCRYPTION_CONTEXT = "x-ms-encryption-context"; + public static final String X_MS_SERVER_ENCRYPTION_CONTEXT = "x-ms-server-encryption-context"; public static final String X_MS_LEASE_ACTION = "x-ms-lease-action"; public static final String X_MS_LEASE_DURATION = "x-ms-lease-duration"; public static final String X_MS_LEASE_ID = "x-ms-lease-id"; diff --git a/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/extensions/EncryptionContextProvider.java b/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/extensions/EncryptionContextProvider.java index 3751e923793a8..cbc832bdd1b5a 100644 --- a/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/extensions/EncryptionContextProvider.java +++ b/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/extensions/EncryptionContextProvider.java @@ -3,6 +3,7 @@ import org.apache.avro.util.ByteBufferInputStream; import org.apache.hadoop.conf.Configuration; +import java.io.ByteArrayInputStream; import java.io.IOException; public interface EncryptionContextProvider extends AutoCloseable { @@ -22,7 +23,7 @@ public interface EncryptionContextProvider extends AutoCloseable { * @return encryptionContext string * @throws IOException */ - ByteBufferInputStream getEncryptionContext(String path) throws IOException; + ByteArrayInputStream getEncryptionContext(String path) throws IOException; /** * Fetch encryption key in-exchange for encryption context @@ -32,7 +33,7 @@ public interface EncryptionContextProvider extends AutoCloseable { * @return Encryption key * @throws IOException */ - ByteBufferInputStream getEncryptionKey(String path, String encryptionContext) throws IOException; + ByteArrayInputStream getEncryptionKey(String path, String encryptionContext) throws IOException; @Override void close() throws IOException; diff --git a/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/services/AbfsClient.java b/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/services/AbfsClient.java index ad305f5001c11..c93cb1ad8f833 100644 --- a/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/services/AbfsClient.java +++ b/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/services/AbfsClient.java @@ -31,6 +31,7 @@ import java.time.Instant; import java.util.ArrayList; import java.util.Base64; +import java.util.HashMap; import java.util.List; import java.util.Locale; import java.util.UUID; @@ -39,6 +40,7 @@ import java.util.concurrent.TimeUnit; import org.apache.hadoop.fs.azurebfs.extensions.EncryptionContextProvider; +import org.apache.hadoop.fs.azurebfs.utils.EncryptionType; import org.apache.hadoop.thirdparty.com.google.common.annotations.VisibleForTesting; import org.apache.hadoop.thirdparty.com.google.common.base.Strings; import org.apache.hadoop.thirdparty.com.google.common.util.concurrent.FutureCallback; @@ -99,6 +101,7 @@ public class AbfsClient implements Closeable { private SASTokenProvider sasTokenProvider; private final AbfsCounters abfsCounters; private final EncryptionContextProvider encryptionContextProvider; + private EncryptionType encryptionType; private final ListeningScheduledExecutorService executorService; @@ -220,6 +223,10 @@ SharedKeyCredentials getSharedKeyCredentials() { return sharedKeyCredentials; } + public void setEncryptionType(EncryptionType encryptionType) { + this.encryptionType = encryptionType; + } + List createDefaultHeaders() { final List requestHeaders = new ArrayList(); requestHeaders.add(new AbfsHttpHeader(X_MS_VERSION, xMsVersion)); @@ -232,32 +239,39 @@ List createDefaultHeaders() { return requestHeaders; } - private void addCustomerProvidedKeyHeaders( - final List requestHeaders, String path, TracingContext tracingContext) - throws IOException { - String encryptionContext = - getPathStatus(path, false, tracingContext).getResult().getResponseHeader(X_MS_ENCRYPTION_CONTEXT); - addCustomerProvidedKeyHeaders(requestHeaders, path, encryptionContext); - } - - private void addCustomerProvidedKeyHeaders( - List requestHeaders, String path, String encryptionContext) - throws IOException { - if (clientProvidedEncryptionKey != null) { - requestHeaders.add( - new AbfsHttpHeader(X_MS_ENCRYPTION_KEY, clientProvidedEncryptionKey)); - requestHeaders.add(new AbfsHttpHeader(X_MS_ENCRYPTION_KEY_SHA256, - clientProvidedEncryptionKeySHA)); - requestHeaders.add(new AbfsHttpHeader(X_MS_ENCRYPTION_ALGORITHM, - SERVER_SIDE_ENCRYPTION_ALGORITHM)); - } else if (encryptionContextProvider != null) { - requestHeaders.add( - new AbfsHttpHeader(X_MS_ENCRYPTION_CONTEXT, encryptionContext)); - requestHeaders.add(new AbfsHttpHeader(X_MS_ENCRYPTION_KEY, - encryptionContextProvider.getEncryptionKey(path, encryptionContext) - .toString())); - + private void addEncryptionKeyRequestHeaders( + List requestHeaders, String path, + boolean isCreateFileRequest, HashMap encryptionHeaders, + TracingContext tracingContext) throws IOException { + String key, encodedKey, encodedKeySHA256, encryptionContext; + switch (encryptionType) { + case GLOBAL_KEY: // common key applicable to all files in the account + encodedKey = clientProvidedEncryptionKey; + encodedKeySHA256 = clientProvidedEncryptionKeySHA; + break; + case ENCRYPTION_CONTEXT: // keys retrieved from EncryptionContextProvider + if (encryptionHeaders != null) { + // use cached encryption keys from input/output streams + encodedKey = encryptionHeaders.get(X_MS_ENCRYPTION_KEY); + encodedKeySHA256 = encryptionHeaders.get(X_MS_ENCRYPTION_KEY_SHA256); + } else { + // get new context if create file request, else fetch via GetPathStatus + encryptionContext = isCreateFileRequest + ? encryptionContextProvider.getEncryptionContext(path).toString() + : getPathStatus(path, false, tracingContext).getResult() + .getResponseHeader(X_MS_SERVER_ENCRYPTION_CONTEXT); + key = encryptionContextProvider.getEncryptionKey(path, + encryptionContext).toString(); + encodedKey = getBase64EncodedString(key); + encodedKeySHA256 = getBase64EncodedString(getSHA256Hash(key)); + } + break; + default: return; // no client-provided encryption keys } + requestHeaders.add(new AbfsHttpHeader(X_MS_ENCRYPTION_KEY, encodedKey)); + requestHeaders.add(new AbfsHttpHeader(X_MS_ENCRYPTION_KEY_SHA256, encodedKeySHA256)); + requestHeaders.add(new AbfsHttpHeader(X_MS_ENCRYPTION_ALGORITHM, + SERVER_SIDE_ENCRYPTION_ALGORITHM)); } AbfsUriQueryBuilder createDefaultUriQueryBuilder() { @@ -311,7 +325,6 @@ public AbfsRestOperation listPath(final String relativePath, final boolean recur final String continuation, TracingContext tracingContext) throws IOException { final List requestHeaders = createDefaultHeaders(); - addCustomerProvidedKeyHeaders(requestHeaders, relativePath, tracingContext); final AbfsUriQueryBuilder abfsUriQueryBuilder = createDefaultUriQueryBuilder(); abfsUriQueryBuilder.addQuery(QUERY_PARAM_RESOURCE, FILESYSTEM); @@ -374,9 +387,7 @@ public AbfsRestOperation createPath(final String path, final boolean isFile, fin throws IOException { final List requestHeaders = createDefaultHeaders(); if (isFile) { - String encryptionContext = encryptionContextProvider.getEncryptionContext( - path).toString(); - addCustomerProvidedKeyHeaders(requestHeaders, path, encryptionContext); + addEncryptionKeyRequestHeaders(requestHeaders, path, true, null, null); } if (!overwrite) { requestHeaders.add(new AbfsHttpHeader(IF_NONE_MATCH, AbfsHttpConstants.STAR)); @@ -602,10 +613,12 @@ public AbfsRestOperation renameIdempotencyCheckOp( } public AbfsRestOperation append(final String path, final byte[] buffer, - AppendRequestParameters reqParams, final String cachedSasToken, TracingContext tracingContext) + AppendRequestParameters reqParams, final String cachedSasToken, + HashMap encryptionHeaders, TracingContext tracingContext) throws IOException { final List requestHeaders = createDefaultHeaders(); - addCustomerProvidedKeyHeaders(requestHeaders, path, tracingContext); + addEncryptionKeyRequestHeaders(requestHeaders, path, false, + encryptionHeaders, tracingContext); // JDK7 does not support PATCH, so to workaround the issue we will use // PUT and specify the real method in the X-Http-Method-Override header. requestHeaders.add(new AbfsHttpHeader(X_HTTP_METHOD_OVERRIDE, @@ -691,10 +704,11 @@ public boolean appendSuccessCheckOp(AbfsRestOperation op, final String path, public AbfsRestOperation flush(final String path, final long position, boolean retainUncommittedData, boolean isClose, - final String cachedSasToken, final String leaseId, - TracingContext tracingContext) throws IOException { + final String cachedSasToken, final String leaseId, HashMap encryptionHeaders, TracingContext tracingContext) throws IOException { final List requestHeaders = createDefaultHeaders(); - addCustomerProvidedKeyHeaders(requestHeaders, path, tracingContext); + addEncryptionKeyRequestHeaders(requestHeaders, path, false, + encryptionHeaders, tracingContext); // JDK7 does not support PATCH, so to workaround the issue we will use // PUT and specify the real method in the X-Http-Method-Override header. requestHeaders.add(new AbfsHttpHeader(X_HTTP_METHOD_OVERRIDE, @@ -727,7 +741,8 @@ public AbfsRestOperation setPathProperties(final String path, final String prope TracingContext tracingContext) throws IOException { final List requestHeaders = createDefaultHeaders(); - addCustomerProvidedKeyHeaders(requestHeaders, path, tracingContext); + addEncryptionKeyRequestHeaders(requestHeaders, path, false, + null, tracingContext); // JDK7 does not support PATCH, so to workaround the issue we will use // PUT and specify the real method in the X-Http-Method-Override header. requestHeaders.add(new AbfsHttpHeader(X_HTTP_METHOD_OVERRIDE, @@ -764,7 +779,8 @@ public AbfsRestOperation getPathStatus(final String path, final boolean includeP abfsUriQueryBuilder.addQuery(HttpQueryParams.QUERY_PARAM_ACTION, AbfsHttpConstants.GET_STATUS); operation = SASTokenProvider.GET_STATUS_OPERATION; } else { - addCustomerProvidedKeyHeaders(requestHeaders, path, tracingContext); + addEncryptionKeyRequestHeaders(requestHeaders, path, false, null, + tracingContext); } abfsUriQueryBuilder.addQuery(HttpQueryParams.QUERY_PARAM_UPN, String.valueOf(abfsConfiguration.isUpnUsed())); appendSASTokenToQuery(path, operation, abfsUriQueryBuilder); @@ -781,10 +797,12 @@ public AbfsRestOperation getPathStatus(final String path, final boolean includeP } public AbfsRestOperation read(final String path, final long position, final byte[] buffer, final int bufferOffset, - final int bufferLength, final String eTag, String cachedSasToken, + final int bufferLength, final String eTag, + String cachedSasToken, HashMap encryptionHeaders, TracingContext tracingContext) throws IOException { final List requestHeaders = createDefaultHeaders(); - addCustomerProvidedKeyHeaders(requestHeaders, path, tracingContext); + addEncryptionKeyRequestHeaders(requestHeaders, path, false, + encryptionHeaders, tracingContext); requestHeaders.add(new AbfsHttpHeader(RANGE, String.format("bytes=%d-%d", position, position + bufferLength - 1))); requestHeaders.add(new AbfsHttpHeader(IF_MATCH, eTag)); diff --git a/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/services/AbfsHttpOperation.java b/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/services/AbfsHttpOperation.java index 6e13d4cec6737..5ca3d5c4f811f 100644 --- a/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/services/AbfsHttpOperation.java +++ b/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/services/AbfsHttpOperation.java @@ -23,6 +23,7 @@ import java.io.OutputStream; import java.net.HttpURLConnection; import java.net.URL; +import java.util.HashMap; import java.util.List; import javax.net.ssl.HttpsURLConnection; @@ -43,6 +44,9 @@ import org.apache.hadoop.fs.azurebfs.contracts.services.AbfsPerfLoggable; import org.apache.hadoop.fs.azurebfs.contracts.services.ListResultSchema; +import static org.apache.hadoop.fs.azurebfs.constants.HttpHeaderConfigurations.X_MS_ENCRYPTION_KEY; +import static org.apache.hadoop.fs.azurebfs.constants.HttpHeaderConfigurations.X_MS_ENCRYPTION_KEY_SHA256; + /** * Represents an HTTP operation. */ @@ -139,6 +143,15 @@ public String getClientRequestId() { .getRequestProperty(HttpHeaderConfigurations.X_MS_CLIENT_REQUEST_ID); } + public HashMap getEncryptionKeyHeaders() { + HashMap encryptionHeaders = new HashMap<>(); + encryptionHeaders.put(X_MS_ENCRYPTION_KEY, + this.connection.getRequestProperty(X_MS_ENCRYPTION_KEY)); + encryptionHeaders.put(X_MS_ENCRYPTION_KEY_SHA256, + this.connection.getRequestProperty(X_MS_ENCRYPTION_KEY_SHA256)); + return encryptionHeaders; + } + public String getExpectedAppendPos() { return expectedAppendPos; } diff --git a/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/services/AbfsOutputStream.java b/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/services/AbfsOutputStream.java index 91b068a78c93f..994f8c3c4afdc 100644 --- a/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/services/AbfsOutputStream.java +++ b/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/services/AbfsOutputStream.java @@ -24,6 +24,7 @@ import java.io.OutputStream; import java.net.HttpURLConnection; import java.nio.ByteBuffer; +import java.util.HashMap; import java.util.concurrent.ConcurrentLinkedDeque; import java.util.concurrent.LinkedBlockingQueue; import java.util.concurrent.ExecutorCompletionService; @@ -93,6 +94,7 @@ public class AbfsOutputStream extends OutputStream implements Syncable, private ConcurrentLinkedDeque writeOperations; private final ThreadPoolExecutor threadExecutor; private final ExecutorCompletionService completionService; + private HashMap encryptionHeaders; // SAS tokens can be re-used until they expire private CachedSASToken cachedSasToken; @@ -145,6 +147,7 @@ public AbfsOutputStream( this.numOfAppendsToServerSinceLastFlush = 0; this.writeOperations = new ConcurrentLinkedDeque<>(); this.outputStreamStatistics = abfsOutputStreamContext.getStreamStatistics(); + this.encryptionHeaders = abfsOutputStreamContext.getEncryptionHeaders(); if (this.isAppendBlob) { this.maxConcurrentRequestCount = 1; @@ -412,9 +415,9 @@ private void writeAppendBlobCurrentBufferToService() throws IOException { "writeCurrentBufferToService", "append")) { AppendRequestParameters reqParams = new AppendRequestParameters(offset, 0, bytesLength, APPEND_MODE, true, leaseId); - AbfsRestOperation op = client - .append(path, bytes, reqParams, cachedSasToken.get(), - new TracingContext(tracingContext)); + AbfsRestOperation op = client.append(path, bytes, reqParams, + cachedSasToken.get(), encryptionHeaders, + new TracingContext(tracingContext)); cachedSasToken.update(op.getSasToken()); if (outputStreamStatistics != null) { outputStreamStatistics.uploadSuccessful(bytesLength); @@ -487,7 +490,8 @@ private synchronized void writeCurrentBufferToService(boolean isFlush, boolean i AppendRequestParameters reqParams = new AppendRequestParameters( offset, 0, bytesLength, mode, false, leaseId); AbfsRestOperation op = client.append(path, bytes, reqParams, - cachedSasToken.get(), new TracingContext(tracingContext)); + cachedSasToken.get(), encryptionHeaders, + new TracingContext(tracingContext)); cachedSasToken.update(op.getSasToken()); perfInfo.registerResult(op.getResult()); byteBufferPool.putBuffer(ByteBuffer.wrap(bytes)); @@ -553,8 +557,9 @@ private synchronized void flushWrittenBytesToServiceInternal(final long offset, AbfsPerfTracker tracker = client.getAbfsPerfTracker(); try (AbfsPerfInfo perfInfo = new AbfsPerfInfo(tracker, "flushWrittenBytesToServiceInternal", "flush")) { - AbfsRestOperation op = client.flush(path, offset, retainUncommitedData, isClose, - cachedSasToken.get(), leaseId, new TracingContext(tracingContext)); + AbfsRestOperation op = client.flush(path, offset, retainUncommitedData, + isClose, cachedSasToken.get(), leaseId, encryptionHeaders, + new TracingContext(tracingContext)); cachedSasToken.update(op.getSasToken()); perfInfo.registerResult(op.getResult()).registerSuccess(true); } catch (AzureBlobFileSystemException ex) { diff --git a/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/services/AbfsOutputStreamContext.java b/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/services/AbfsOutputStreamContext.java index 48f6f54081053..87d2006c6b2b8 100644 --- a/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/services/AbfsOutputStreamContext.java +++ b/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/services/AbfsOutputStreamContext.java @@ -18,6 +18,8 @@ package org.apache.hadoop.fs.azurebfs.services; +import java.util.HashMap; + /** * Class to hold extra output stream configs. */ @@ -41,6 +43,8 @@ public class AbfsOutputStreamContext extends AbfsStreamContext { private AbfsLease lease; + private HashMap encryptionHeaders; + public AbfsOutputStreamContext(final long sasTokenRenewPeriodForStreamsInSeconds) { super(sasTokenRenewPeriodForStreamsInSeconds); } @@ -101,6 +105,12 @@ public AbfsOutputStreamContext withLease(final AbfsLease lease) { return this; } + public AbfsOutputStreamContext withEncryptionHeaders( + final HashMap encryptionHeaders) { + this.encryptionHeaders = encryptionHeaders; + return this; + } + public int getWriteBufferSize() { return writeBufferSize; } @@ -143,4 +153,8 @@ public String getLeaseId() { } return this.lease.getLeaseID(); } + + public HashMap getEncryptionHeaders() { + return encryptionHeaders; + } } diff --git a/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/utils/EncryptionType.java b/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/utils/EncryptionType.java new file mode 100644 index 0000000000000..4624f2ef1c2a2 --- /dev/null +++ b/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/utils/EncryptionType.java @@ -0,0 +1,13 @@ +package org.apache.hadoop.fs.azurebfs.utils; + +/** + * Enum EncryptionType to represent the level of encryption applied: + * GLOBAL_KEY for encrypting all files with the same client-provided key + * ENCRYPTION_CONTEXT uses client-provided implementation to generate keys + * NONE encryption handled entirely at server, no client + */ +public enum EncryptionType { + GLOBAL_KEY, // encrypt all files with the same client-provided key + ENCRYPTION_CONTEXT, // uses client-provided implementation to generate keys + NONE // encryption handled entirely at server +} diff --git a/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/ITestCustomerProvidedKey.java b/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/ITestCustomerProvidedKey.java index 6f5066012ba2e..e3e6360b5eaed 100644 --- a/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/ITestCustomerProvidedKey.java +++ b/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/ITestCustomerProvidedKey.java @@ -279,12 +279,6 @@ public void testAppendWithoutCPK() throws Exception { @Test public void testSetGetXAttr() throws Exception { final AzureBlobFileSystem fs = getAbfs(true); - System.out.println(StringUtils.substringBefore(fs.getUri().getAuthority(), - "@") +" \n "+ - fs.getUri().getAuthority()+" \n "+ - fs.getUri().getAuthority()+"\n "+ - fs.getUri().getHost() + " \n " + fs.getUri().getPort() + - fs.getUri().getFragment() + " \n " + fs.getUri().parseServerAuthority()); final String fileName = path(methodName.getMethodName()).toString(); createFileAndGetContent(fs, fileName, FILE_SIZE); @@ -436,7 +430,6 @@ private void testListPath(final boolean isWithCPK) throws Exception { "different-1234567890123456789012"); AzureBlobFileSystem fs2 = (AzureBlobFileSystem) FileSystem.newInstance(conf); AbfsClient abfsClient2 = fs2.getAbfsClient(); - abfsClient2.setClientProvidedEncryptionKey(null); TracingContext tracingContext = getTestTracingContext(fs, false); abfsRestOperation = abfsClient2.listPath(testDirName, false, INT_50, null, tracingContext); @@ -666,7 +659,6 @@ private void testGetPathStatusFile(final boolean isWithCPK) throws Exception { assertResponseHeader(abfsRestOperation, false, X_MS_REQUEST_SERVER_ENCRYPTED, ""); - abfsClient.setClientProvidedEncryptionKey(null); abfsRestOperation = abfsClient.getPathStatus(testFileName, false, tracingContext); assertCPKHeaders(abfsRestOperation, isWithCPK); diff --git a/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/extensions/MockEncryptionContextProvider.java b/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/extensions/MockEncryptionContextProvider.java new file mode 100644 index 0000000000000..55cd24694f577 --- /dev/null +++ b/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/extensions/MockEncryptionContextProvider.java @@ -0,0 +1,47 @@ +package org.apache.hadoop.fs.azurebfs.extensions; + +import org.apache.avro.util.ByteBufferInputStream; +import org.apache.hadoop.conf.Configuration; + +import java.io.ByteArrayInputStream; +import java.io.IOException; +import java.nio.charset.StandardCharsets; +import java.util.HashMap; +import java.util.UUID; + +public class MockEncryptionContextProvider implements EncryptionContextProvider { + String dummyKey = UUID.randomUUID().toString(); + HashMap pathToContextMap = new HashMap<>(); + HashMap contextToKeyMap = new HashMap<>(); + @Override + public void initialize(Configuration configuration, String accountName, + String fileSystem) throws IOException { + + } + + @Override + public ByteArrayInputStream getEncryptionContext(String path) + throws IOException { + String newContext = UUID.randomUUID().toString(); + pathToContextMap.put(path, newContext); + // String key = UUID.randomUUID().toString(); + String key = dummyKey; // replace with above once server supports + contextToKeyMap.put(newContext, key); + return new ByteArrayInputStream(newContext.getBytes((StandardCharsets.UTF_8))); + } + + @Override + public ByteArrayInputStream getEncryptionKey(String path, + String encryptionContext) throws IOException { + if (encryptionContext.equals(pathToContextMap.get(path))) { + throw new IOException("encryption context does not match path"); + } + return new ByteArrayInputStream(contextToKeyMap.get(encryptionContext) + .getBytes(StandardCharsets.UTF_8)); + } + + @Override + public void close() throws IOException { + + } +} From 59d171f33df91c654c8ccbcfe0ddb375af4b8ee7 Mon Sep 17 00:00:00 2001 From: sumangala Date: Thu, 16 Sep 2021 23:04:54 +0530 Subject: [PATCH 03/77] use existing getpathstatus, cache at stream open --- .../fs/azurebfs/AzureBlobFileSystemStore.java | 69 +++++++++++-------- .../constants/HttpHeaderConfigurations.java | 1 - .../fs/azurebfs/services/AbfsClient.java | 63 ++++++++++------- .../azurebfs/services/AbfsHttpOperation.java | 9 --- .../fs/azurebfs/services/AbfsInputStream.java | 6 +- .../services/AbfsInputStreamContext.java | 14 ++++ .../azurebfs/services/AbfsOutputStream.java | 2 +- .../azurebfs/AbstractAbfsIntegrationTest.java | 2 +- 8 files changed, 102 insertions(+), 64 deletions(-) diff --git a/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/AzureBlobFileSystemStore.java b/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/AzureBlobFileSystemStore.java index 1f2f810010d01..7f9aad723de23 100644 --- a/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/AzureBlobFileSystemStore.java +++ b/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/AzureBlobFileSystemStore.java @@ -52,7 +52,6 @@ import java.util.WeakHashMap; import java.util.concurrent.ExecutionException; -import org.apache.commons.lang3.StringUtils; import org.apache.hadoop.fs.azurebfs.extensions.EncryptionContextProvider; import org.apache.hadoop.fs.azurebfs.utils.EncryptionType; import org.apache.hadoop.thirdparty.com.google.common.annotations.VisibleForTesting; @@ -528,12 +527,14 @@ public OutputStream createFile(final Path path, } AbfsRestOperation op; + HashMap encryptionHeaders = new HashMap<>(); if (triggerConditionalCreateOverwrite) { op = conditionalCreateOverwriteFile(relativePath, statistics, isNamespaceEnabled ? getOctalNotation(permission) : null, isNamespaceEnabled ? getOctalNotation(umask) : null, isAppendBlob, + encryptionHeaders, tracingContext ); @@ -544,6 +545,7 @@ public OutputStream createFile(final Path path, isNamespaceEnabled ? getOctalNotation(umask) : null, isAppendBlob, null, + encryptionHeaders, tracingContext); } @@ -556,8 +558,7 @@ public OutputStream createFile(final Path path, statistics, relativePath, 0, - populateAbfsOutputStreamContext(isAppendBlob, lease, - op.getResult().getEncryptionKeyHeaders()), + populateAbfsOutputStreamContext(isAppendBlob, lease, encryptionHeaders), tracingContext); } } @@ -578,6 +579,7 @@ private AbfsRestOperation conditionalCreateOverwriteFile(final String relativePa final String permission, final String umask, final boolean isAppendBlob, + HashMap encryptionHeaders, TracingContext tracingContext) throws IOException { AbfsRestOperation op; @@ -586,7 +588,7 @@ private AbfsRestOperation conditionalCreateOverwriteFile(final String relativePa // avoided for cases when no pre-existing file is present (major portion // of create file traffic falls into the case of no pre-existing file). op = client.createPath(relativePath, true, false, permission, umask, - isAppendBlob, null, tracingContext); + isAppendBlob, null, encryptionHeaders, tracingContext); } catch (AbfsRestOperationException e) { if (e.getStatusCode() == HttpURLConnection.HTTP_CONFLICT) { @@ -611,7 +613,7 @@ private AbfsRestOperation conditionalCreateOverwriteFile(final String relativePa try { // overwrite only if eTag matches with the file properties fetched befpre op = client.createPath(relativePath, true, true, permission, umask, - isAppendBlob, eTag, tracingContext); + isAppendBlob, eTag, encryptionHeaders, tracingContext); } catch (AbfsRestOperationException ex) { if (ex.getStatusCode() == HttpURLConnection.HTTP_PRECON_FAILED) { // Is a parallel access case, as file with eTag was just queried @@ -669,8 +671,8 @@ public void createDirectory(final Path path, final FsPermission permission, final AbfsRestOperation op = client.createPath(getRelativePath(path), false, overwrite, isNamespaceEnabled ? getOctalNotation(permission) : null, - isNamespaceEnabled ? getOctalNotation(umask) : null, false, null, - tracingContext); + isNamespaceEnabled ? getOctalNotation(umask) : null, false, + null, null, tracingContext); perfInfo.registerResult(op.getResult()).registerSuccess(true); } } @@ -695,8 +697,10 @@ public AbfsInputStream openFileForRead(Path path, .orElse(null); String relativePath = getRelativePath(path); String resourceType, eTag; + String encryptionContext = null; long contentLength; - if (fileStatus instanceof VersionedFileStatus) { + if (fileStatus instanceof VersionedFileStatus + && client.getEncryptionType() != EncryptionType.ENCRYPTION_CONTEXT) { path = path.makeQualified(this.uri, path); Preconditions.checkArgument(fileStatus.getPath().equals(path), String.format( @@ -707,9 +711,8 @@ public AbfsInputStream openFileForRead(Path path, eTag = ((VersionedFileStatus) fileStatus).getVersion(); } else { if (fileStatus != null) { - LOG.warn( - "Fallback to getPathStatus REST call as provided filestatus " - + "is not of type VersionedFileStatus"); + LOG.debug("Fallback to getPathStatus REST call as provided fileStatus " + + "is not of type VersionedFileStatus, or file is encrypted"); } AbfsHttpOperation op = client.getPathStatus(relativePath, false, tracingContext).getResult(); @@ -718,6 +721,8 @@ public AbfsInputStream openFileForRead(Path path, contentLength = Long.parseLong( op.getResponseHeader(HttpHeaderConfigurations.CONTENT_LENGTH)); eTag = op.getResponseHeader(HttpHeaderConfigurations.ETAG); + encryptionContext = op.getResponseHeader( + HttpHeaderConfigurations.X_MS_PROPERTIES); } if (parseIsDirectory(resourceType)) { @@ -733,13 +738,14 @@ public AbfsInputStream openFileForRead(Path path, // Add statistics for InputStream return new AbfsInputStream(client, statistics, relativePath, contentLength, populateAbfsInputStreamContext( - parameters.map(OpenFileParameters::getOptions)), + parameters.map(OpenFileParameters::getOptions), + client.getEncryptionHeaders(path.toString(), encryptionContext)), eTag, tracingContext); } } private AbfsInputStreamContext populateAbfsInputStreamContext( - Optional options) { + Optional options, HashMap encryptionHeaders) { boolean bufferedPreadDisabled = options .map(c -> c.getBoolean(FS_AZURE_BUFFERED_PREAD_DISABLE, false)) .orElse(false); @@ -755,6 +761,7 @@ private AbfsInputStreamContext populateAbfsInputStreamContext( abfsConfiguration.shouldReadBufferSizeAlways()) .withReadAheadBlockSize(abfsConfiguration.getReadAheadBlockSize()) .withBufferedPreadDisabled(bufferedPreadDisabled) + .withEncryptionHeaders(encryptionHeaders) .build(); } @@ -794,6 +801,8 @@ public OutputStream openFileForWrite(final Path path, } AbfsLease lease = maybeCreateLease(relativePath, tracingContext); + String encryptionContext = op.getResult() + .getResponseHeader(HttpHeaderConfigurations.X_MS_PROPERTIES); return new AbfsOutputStream( client, @@ -801,7 +810,7 @@ public OutputStream openFileForWrite(final Path path, relativePath, offset, populateAbfsOutputStreamContext(isAppendBlob, lease, - op.getResult().getEncryptionKeyHeaders()), + client.getEncryptionHeaders(path.toString(), encryptionContext)), tracingContext); } } @@ -1532,22 +1541,25 @@ private void initializeClient(URI uri, String fileSystemName, abfsConfiguration.getRawConfiguration()); } - // Initialize encryption context - EncryptionContextProvider encryptionContextProvider = - abfsConfiguration.initializeEncryptionContextProvider(); - if (encryptionContextProvider != null) { - if (abfsConfiguration.getClientProvidedEncryptionKey() != null) { - throw new IOException( - "Both global key and encryption context are set, only one allowed"); + // Encryption setup + EncryptionType encryptionType = EncryptionType.NONE; + EncryptionContextProvider encryptionContextProvider = null; + if (isSecure) { + encryptionContextProvider = + abfsConfiguration.initializeEncryptionContextProvider(); + if (encryptionContextProvider != null) { + if (abfsConfiguration.getClientProvidedEncryptionKey() != null) { + throw new IOException( + "Both global key and encryption context are set, only one allowed"); + } + encryptionContextProvider.initialize( + abfsConfiguration.getRawConfiguration(), accountName, + fileSystemName); + encryptionType = EncryptionType.ENCRYPTION_CONTEXT; + } else if (abfsConfiguration.getClientProvidedEncryptionKey() != null) { + encryptionType = EncryptionType.GLOBAL_KEY; } - client.setEncryptionType(EncryptionType.ENCRYPTION_CONTEXT); - } else if (abfsConfiguration.getClientProvidedEncryptionKey() != null) { - client.setEncryptionType(EncryptionType.GLOBAL_KEY); - } else { - client.setEncryptionType(EncryptionType.NONE); } - encryptionContextProvider.initialize( - abfsConfiguration.getRawConfiguration(), accountName, fileSystemName); LOG.trace("Initializing AbfsClient for {}", baseUrl); if (tokenProvider != null) { @@ -1559,6 +1571,7 @@ private void initializeClient(URI uri, String fileSystemName, sasTokenProvider, encryptionContextProvider, populateAbfsClientContext()); } + client.setEncryptionType(encryptionType); LOG.trace("AbfsClient init complete"); } diff --git a/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/constants/HttpHeaderConfigurations.java b/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/constants/HttpHeaderConfigurations.java index 2f4e705fdd87e..312677e32da2a 100644 --- a/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/constants/HttpHeaderConfigurations.java +++ b/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/constants/HttpHeaderConfigurations.java @@ -66,7 +66,6 @@ public final class HttpHeaderConfigurations { public static final String X_MS_REQUEST_SERVER_ENCRYPTED = "x-ms-request-server-encrypted"; public static final String X_MS_SERVER_ENCRYPTED = "x-ms-server-encrypted"; public static final String X_MS_ENCRYPTION_CONTEXT = "x-ms-encryption-context"; - public static final String X_MS_SERVER_ENCRYPTION_CONTEXT = "x-ms-server-encryption-context"; public static final String X_MS_LEASE_ACTION = "x-ms-lease-action"; public static final String X_MS_LEASE_DURATION = "x-ms-lease-duration"; public static final String X_MS_LEASE_ID = "x-ms-lease-id"; diff --git a/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/services/AbfsClient.java b/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/services/AbfsClient.java index c93cb1ad8f833..2122e24ee3ed7 100644 --- a/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/services/AbfsClient.java +++ b/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/services/AbfsClient.java @@ -226,6 +226,9 @@ SharedKeyCredentials getSharedKeyCredentials() { public void setEncryptionType(EncryptionType encryptionType) { this.encryptionType = encryptionType; } + public EncryptionType getEncryptionType() { + return encryptionType; + } List createDefaultHeaders() { final List requestHeaders = new ArrayList(); @@ -242,30 +245,32 @@ List createDefaultHeaders() { private void addEncryptionKeyRequestHeaders( List requestHeaders, String path, boolean isCreateFileRequest, HashMap encryptionHeaders, - TracingContext tracingContext) throws IOException { - String key, encodedKey, encodedKeySHA256, encryptionContext; + TracingContext tracingContext) + throws IOException { + String encodedKey, encodedKeySHA256, encryptionContext; switch (encryptionType) { - case GLOBAL_KEY: // common key applicable to all files in the account + case GLOBAL_KEY: encodedKey = clientProvidedEncryptionKey; encodedKeySHA256 = clientProvidedEncryptionKeySHA; break; - case ENCRYPTION_CONTEXT: // keys retrieved from EncryptionContextProvider - if (encryptionHeaders != null) { - // use cached encryption keys from input/output streams - encodedKey = encryptionHeaders.get(X_MS_ENCRYPTION_KEY); - encodedKeySHA256 = encryptionHeaders.get(X_MS_ENCRYPTION_KEY_SHA256); - } else { - // get new context if create file request, else fetch via GetPathStatus - encryptionContext = isCreateFileRequest - ? encryptionContextProvider.getEncryptionContext(path).toString() - : getPathStatus(path, false, tracingContext).getResult() - .getResponseHeader(X_MS_SERVER_ENCRYPTION_CONTEXT); - key = encryptionContextProvider.getEncryptionKey(path, - encryptionContext).toString(); - encodedKey = getBase64EncodedString(key); - encodedKeySHA256 = getBase64EncodedString(getSHA256Hash(key)); + case ENCRYPTION_CONTEXT: + if (isCreateFileRequest) { + encryptionContext = encryptionContextProvider.getEncryptionContext(path) + .toString(); // get new context for create file request + requestHeaders.add(new AbfsHttpHeader(X_MS_ENCRYPTION_CONTEXT, + encryptionContext)); + encryptionHeaders = getEncryptionHeaders(path, encryptionContext); + + } else if (encryptionHeaders == null) { + encryptionContext = getPathStatus(path, false, + tracingContext).getResult().getResponseHeader(X_MS_PROPERTIES); + encryptionHeaders = getEncryptionHeaders(path, encryptionContext); } + // use cached encryption keys from input/output streams + encodedKey = encryptionHeaders.get(X_MS_ENCRYPTION_KEY); + encodedKeySHA256 = encryptionHeaders.get(X_MS_ENCRYPTION_KEY_SHA256); break; + default: return; // no client-provided encryption keys } requestHeaders.add(new AbfsHttpHeader(X_MS_ENCRYPTION_KEY, encodedKey)); @@ -274,6 +279,17 @@ private void addEncryptionKeyRequestHeaders( SERVER_SIDE_ENCRYPTION_ALGORITHM)); } + public HashMap getEncryptionHeaders(String path, + String encryptionContext) throws IOException { + String key = encryptionContextProvider.getEncryptionKey(path, + encryptionContext).toString(); + HashMap encryptionHeaders = new HashMap<>(); + encryptionHeaders.put(X_MS_ENCRYPTION_KEY, getBase64EncodedString(key)); + encryptionHeaders.put(X_MS_ENCRYPTION_KEY_SHA256, + getBase64EncodedString(getSHA256Hash(key))); + return encryptionHeaders; + } + AbfsUriQueryBuilder createDefaultUriQueryBuilder() { final AbfsUriQueryBuilder abfsUriQueryBuilder = new AbfsUriQueryBuilder(); abfsUriQueryBuilder.addQuery(QUERY_PARAM_TIMEOUT, DEFAULT_TIMEOUT); @@ -380,14 +396,15 @@ public AbfsRestOperation deleteFilesystem(TracingContext tracingContext) throws return op; } - public AbfsRestOperation createPath(final String path, final boolean isFile, final boolean overwrite, - final String permission, final String umask, - final boolean isAppendBlob, final String eTag, - TracingContext tracingContext) + public AbfsRestOperation createPath(final String path, final boolean isFile, + final boolean overwrite, final String permission, final String umask, + final boolean isAppendBlob, final String eTag, + HashMap encryptionHeaders, TracingContext tracingContext) throws IOException { final List requestHeaders = createDefaultHeaders(); if (isFile) { - addEncryptionKeyRequestHeaders(requestHeaders, path, true, null, null); + addEncryptionKeyRequestHeaders(requestHeaders, path, true, encryptionHeaders, + tracingContext); } if (!overwrite) { requestHeaders.add(new AbfsHttpHeader(IF_NONE_MATCH, AbfsHttpConstants.STAR)); diff --git a/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/services/AbfsHttpOperation.java b/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/services/AbfsHttpOperation.java index 5ca3d5c4f811f..445b1e0314072 100644 --- a/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/services/AbfsHttpOperation.java +++ b/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/services/AbfsHttpOperation.java @@ -143,15 +143,6 @@ public String getClientRequestId() { .getRequestProperty(HttpHeaderConfigurations.X_MS_CLIENT_REQUEST_ID); } - public HashMap getEncryptionKeyHeaders() { - HashMap encryptionHeaders = new HashMap<>(); - encryptionHeaders.put(X_MS_ENCRYPTION_KEY, - this.connection.getRequestProperty(X_MS_ENCRYPTION_KEY)); - encryptionHeaders.put(X_MS_ENCRYPTION_KEY_SHA256, - this.connection.getRequestProperty(X_MS_ENCRYPTION_KEY_SHA256)); - return encryptionHeaders; - } - public String getExpectedAppendPos() { return expectedAppendPos; } diff --git a/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/services/AbfsInputStream.java b/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/services/AbfsInputStream.java index 7033ae9a4a039..f1588b7eed46f 100644 --- a/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/services/AbfsInputStream.java +++ b/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/services/AbfsInputStream.java @@ -22,6 +22,7 @@ import java.io.FileNotFoundException; import java.io.IOException; import java.net.HttpURLConnection; +import java.util.HashMap; import java.util.UUID; import org.apache.commons.lang3.StringUtils; @@ -97,6 +98,7 @@ public class AbfsInputStream extends FSInputStream implements CanUnbuffer, // of valid bytes in buffer) private boolean closed = false; private TracingContext tracingContext; + private final HashMap encryptionHeaders; // Optimisations modify the pointer fields. // For better resilience the following fields are used to save the @@ -151,6 +153,7 @@ public AbfsInputStream( this.tracingContext.setStreamID(inputStreamId); this.context = abfsInputStreamContext; readAheadBlockSize = abfsInputStreamContext.getReadAheadBlockSize(); + encryptionHeaders = abfsInputStreamContext.getEncryptionHeaders(); // Propagate the config values to ReadBufferManager so that the first instance // to initialize can set the readAheadBlockSize @@ -542,7 +545,8 @@ int readRemote(long position, byte[] b, int offset, int length, TracingContext t } LOG.trace("Trigger client.read for path={} position={} offset={} length={}", path, position, offset, length); op = client.read(path, position, b, offset, length, - tolerateOobAppends ? "*" : eTag, cachedSasToken.get(), tracingContext); + tolerateOobAppends ? "*" : eTag, cachedSasToken.get(), + encryptionHeaders, tracingContext); cachedSasToken.update(op.getSasToken()); LOG.debug("issuing HTTP GET request params position = {} b.length = {} " + "offset = {} length = {}", position, b.length, offset, length); diff --git a/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/services/AbfsInputStreamContext.java b/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/services/AbfsInputStreamContext.java index 55f01bf15bcf7..70f0a401c543a 100644 --- a/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/services/AbfsInputStreamContext.java +++ b/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/services/AbfsInputStreamContext.java @@ -22,6 +22,8 @@ import org.slf4j.LoggerFactory; import org.apache.hadoop.thirdparty.com.google.common.base.Preconditions; +import java.util.HashMap; + /** * Class to hold extra input stream configs. */ @@ -49,6 +51,8 @@ public class AbfsInputStreamContext extends AbfsStreamContext { private boolean bufferedPreadDisabled; + private HashMap encryptionHeaders; + public AbfsInputStreamContext(final long sasTokenRenewPeriodForStreamsInSeconds) { super(sasTokenRenewPeriodForStreamsInSeconds); } @@ -114,6 +118,12 @@ public AbfsInputStreamContext withBufferedPreadDisabled( return this; } + public AbfsInputStreamContext withEncryptionHeaders( + HashMap encryptionHeaders) { + this.encryptionHeaders = encryptionHeaders; + return this; + } + public AbfsInputStreamContext build() { if (readBufferSize > readAheadBlockSize) { LOG.debug( @@ -168,4 +178,8 @@ public int getReadAheadBlockSize() { public boolean isBufferedPreadDisabled() { return bufferedPreadDisabled; } + + public HashMap getEncryptionHeaders() { + return encryptionHeaders; + } } diff --git a/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/services/AbfsOutputStream.java b/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/services/AbfsOutputStream.java index 994f8c3c4afdc..8c654eef9fa8d 100644 --- a/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/services/AbfsOutputStream.java +++ b/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/services/AbfsOutputStream.java @@ -94,7 +94,7 @@ public class AbfsOutputStream extends OutputStream implements Syncable, private ConcurrentLinkedDeque writeOperations; private final ThreadPoolExecutor threadExecutor; private final ExecutorCompletionService completionService; - private HashMap encryptionHeaders; + private final HashMap encryptionHeaders; // SAS tokens can be re-used until they expire private CachedSASToken cachedSasToken; diff --git a/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/AbstractAbfsIntegrationTest.java b/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/AbstractAbfsIntegrationTest.java index d4e44c37f632a..56d553819feae 100644 --- a/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/AbstractAbfsIntegrationTest.java +++ b/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/AbstractAbfsIntegrationTest.java @@ -488,7 +488,7 @@ protected AbfsDelegationTokenManager getDelegationTokenManager() */ protected AbfsOutputStream createAbfsOutputStreamWithFlushEnabled( AzureBlobFileSystem fs, - Path path) throws AzureBlobFileSystemException { + Path path) throws IOException { AzureBlobFileSystemStore abfss = fs.getAbfsStore(); abfss.getAbfsConfiguration().setDisableOutputStreamFlush(false); From 538820e5e7ea2807f6939f48fea62a269399fd89 Mon Sep 17 00:00:00 2001 From: sumangala Date: Mon, 20 Sep 2021 08:24:48 +0530 Subject: [PATCH 04/77] main code change --- .../fs/azurebfs/AzureBlobFileSystemStore.java | 19 ++++++++----- .../azurebfs/constants/ConfigurationKeys.java | 2 +- .../extensions/EncryptionContextProvider.java | 19 ++++++++++++- .../fs/azurebfs/services/AbfsClient.java | 2 +- .../azurebfs/services/AbfsHttpOperation.java | 4 --- .../services/AbfsInputStreamContext.java | 2 +- .../fs/azurebfs/utils/EncryptionType.java | 27 ++++++++++++++----- 7 files changed, 55 insertions(+), 20 deletions(-) diff --git a/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/AzureBlobFileSystemStore.java b/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/AzureBlobFileSystemStore.java index 7f9aad723de23..27e4362aca384 100644 --- a/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/AzureBlobFileSystemStore.java +++ b/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/AzureBlobFileSystemStore.java @@ -696,9 +696,9 @@ public AbfsInputStream openFileForRead(Path path, FileStatus fileStatus = parameters.map(OpenFileParameters::getStatus) .orElse(null); String relativePath = getRelativePath(path); - String resourceType, eTag; - String encryptionContext = null; + String resourceType, eTag, encryptionContext; long contentLength; + HashMap encryptionHeaders = null; if (fileStatus instanceof VersionedFileStatus && client.getEncryptionType() != EncryptionType.ENCRYPTION_CONTEXT) { path = path.makeQualified(this.uri, path); @@ -721,8 +721,16 @@ public AbfsInputStream openFileForRead(Path path, contentLength = Long.parseLong( op.getResponseHeader(HttpHeaderConfigurations.CONTENT_LENGTH)); eTag = op.getResponseHeader(HttpHeaderConfigurations.ETAG); - encryptionContext = op.getResponseHeader( - HttpHeaderConfigurations.X_MS_PROPERTIES); + if (client.getEncryptionType() == EncryptionType.ENCRYPTION_CONTEXT) { + encryptionContext = op.getResponseHeader( + HttpHeaderConfigurations.X_MS_PROPERTIES); + if (encryptionContext == null) { + throw new IOException( + "Encryption context missing from server response"); + } + encryptionHeaders = client.getEncryptionHeaders(path.toString(), + encryptionContext); + } } if (parseIsDirectory(resourceType)) { @@ -738,8 +746,7 @@ public AbfsInputStream openFileForRead(Path path, // Add statistics for InputStream return new AbfsInputStream(client, statistics, relativePath, contentLength, populateAbfsInputStreamContext( - parameters.map(OpenFileParameters::getOptions), - client.getEncryptionHeaders(path.toString(), encryptionContext)), + parameters.map(OpenFileParameters::getOptions), encryptionHeaders), eTag, tracingContext); } } diff --git a/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/constants/ConfigurationKeys.java b/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/constants/ConfigurationKeys.java index 7231442bea491..668e91dd3fc56 100644 --- a/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/constants/ConfigurationKeys.java +++ b/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/constants/ConfigurationKeys.java @@ -160,7 +160,7 @@ public final class ConfigurationKeys { /** Server side encryption key */ public static final String FS_AZURE_CLIENT_PROVIDED_ENCRYPTION_KEY = "fs.azure.client-provided-encryption-key"; /** Custom EncryptionContextProvider type */ - public static final String FS_AZURE_ENCRYPTION_CONTEXT_PROVIDER_TYPE = "fs.azure.encryption-context.provider.type"; + public static final String FS_AZURE_ENCRYPTION_CONTEXT_PROVIDER_TYPE = "fs.azure.encryption.context.provider.type"; public static final String FS_AZURE_ENCRYPTION_MODE = "fs.azure.encryption.mode"; /** End point of ABFS account: {@value}. */ diff --git a/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/extensions/EncryptionContextProvider.java b/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/extensions/EncryptionContextProvider.java index cbc832bdd1b5a..ae84e9257a932 100644 --- a/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/extensions/EncryptionContextProvider.java +++ b/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/extensions/EncryptionContextProvider.java @@ -1,6 +1,23 @@ +/** + * 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.fs.azurebfs.extensions; -import org.apache.avro.util.ByteBufferInputStream; import org.apache.hadoop.conf.Configuration; import java.io.ByteArrayInputStream; diff --git a/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/services/AbfsClient.java b/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/services/AbfsClient.java index 2122e24ee3ed7..93b806a9aa4ec 100644 --- a/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/services/AbfsClient.java +++ b/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/services/AbfsClient.java @@ -261,7 +261,7 @@ private void addEncryptionKeyRequestHeaders( encryptionContext)); encryptionHeaders = getEncryptionHeaders(path, encryptionContext); - } else if (encryptionHeaders == null) { + } else if (encryptionHeaders == null || encryptionHeaders.isEmpty()) { encryptionContext = getPathStatus(path, false, tracingContext).getResult().getResponseHeader(X_MS_PROPERTIES); encryptionHeaders = getEncryptionHeaders(path, encryptionContext); diff --git a/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/services/AbfsHttpOperation.java b/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/services/AbfsHttpOperation.java index 445b1e0314072..6e13d4cec6737 100644 --- a/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/services/AbfsHttpOperation.java +++ b/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/services/AbfsHttpOperation.java @@ -23,7 +23,6 @@ import java.io.OutputStream; import java.net.HttpURLConnection; import java.net.URL; -import java.util.HashMap; import java.util.List; import javax.net.ssl.HttpsURLConnection; @@ -44,9 +43,6 @@ import org.apache.hadoop.fs.azurebfs.contracts.services.AbfsPerfLoggable; import org.apache.hadoop.fs.azurebfs.contracts.services.ListResultSchema; -import static org.apache.hadoop.fs.azurebfs.constants.HttpHeaderConfigurations.X_MS_ENCRYPTION_KEY; -import static org.apache.hadoop.fs.azurebfs.constants.HttpHeaderConfigurations.X_MS_ENCRYPTION_KEY_SHA256; - /** * Represents an HTTP operation. */ diff --git a/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/services/AbfsInputStreamContext.java b/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/services/AbfsInputStreamContext.java index 70f0a401c543a..4c676f52914f0 100644 --- a/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/services/AbfsInputStreamContext.java +++ b/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/services/AbfsInputStreamContext.java @@ -51,7 +51,7 @@ public class AbfsInputStreamContext extends AbfsStreamContext { private boolean bufferedPreadDisabled; - private HashMap encryptionHeaders; + private HashMap encryptionHeaders = null; public AbfsInputStreamContext(final long sasTokenRenewPeriodForStreamsInSeconds) { super(sasTokenRenewPeriodForStreamsInSeconds); diff --git a/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/utils/EncryptionType.java b/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/utils/EncryptionType.java index 4624f2ef1c2a2..f7a06e1487c17 100644 --- a/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/utils/EncryptionType.java +++ b/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/utils/EncryptionType.java @@ -1,13 +1,28 @@ +/** + * 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.fs.azurebfs.utils; /** - * Enum EncryptionType to represent the level of encryption applied: - * GLOBAL_KEY for encrypting all files with the same client-provided key - * ENCRYPTION_CONTEXT uses client-provided implementation to generate keys - * NONE encryption handled entirely at server, no client + * Enum EncryptionType to represent the level of encryption applied */ public enum EncryptionType { - GLOBAL_KEY, // encrypt all files with the same client-provided key + GLOBAL_KEY, // encrypt all files with the same client-provided key ENCRYPTION_CONTEXT, // uses client-provided implementation to generate keys - NONE // encryption handled entirely at server + NONE // encryption handled entirely at server } From 09c0fbb594737fbe4de2d823f893f2519a13b4c8 Mon Sep 17 00:00:00 2001 From: sumangala Date: Mon, 20 Sep 2021 09:04:18 +0530 Subject: [PATCH 05/77] other tests syntax fix --- .../ITestAzureBlobFileSystemCreate.java | 4 +- .../fs/azurebfs/ITestCustomerProvidedKey.java | 26 +++--- .../fs/azurebfs/TestTracingContext.java | 2 +- .../fs/azurebfs/services/TestAbfsClient.java | 3 +- .../services/TestAbfsInputStream.java | 21 +++-- .../services/TestAbfsOutputStream.java | 88 +++++++++---------- 6 files changed, 72 insertions(+), 72 deletions(-) diff --git a/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/ITestAzureBlobFileSystemCreate.java b/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/ITestAzureBlobFileSystemCreate.java index 2f23ac5c5c708..ca0782ec20092 100644 --- a/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/ITestAzureBlobFileSystemCreate.java +++ b/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/ITestAzureBlobFileSystemCreate.java @@ -400,7 +400,7 @@ public void testNegativeScenariosForCreateOverwriteDisabled() .createPath(any(String.class), eq(true), eq(false), isNamespaceEnabled ? any(String.class) : eq(null), isNamespaceEnabled ? any(String.class) : eq(null), - any(boolean.class), eq(null), any(TracingContext.class)); + any(boolean.class), eq(null), any(), any(TracingContext.class)); doThrow(fileNotFoundResponseEx) // Scn1: GFS fails with Http404 .doThrow(serverErrorResponseEx) // Scn2: GFS fails with Http500 @@ -418,7 +418,7 @@ public void testNegativeScenariosForCreateOverwriteDisabled() .createPath(any(String.class), eq(true), eq(true), isNamespaceEnabled ? any(String.class) : eq(null), isNamespaceEnabled ? any(String.class) : eq(null), - any(boolean.class), eq(null), any(TracingContext.class)); + any(boolean.class), eq(null), any(), any(TracingContext.class)); // Scn1: GFS fails with Http404 // Sequence of events expected: diff --git a/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/ITestCustomerProvidedKey.java b/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/ITestCustomerProvidedKey.java index e3e6360b5eaed..dbd4d0c32b333 100644 --- a/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/ITestCustomerProvidedKey.java +++ b/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/ITestCustomerProvidedKey.java @@ -121,7 +121,7 @@ public void testReadWithCPK() throws Exception { final String eTag = op.getResult() .getResponseHeader(HttpHeaderConfigurations.ETAG); AbfsRestOperation abfsRestOperation = abfsClient - .read(fileName, 0, buffer, 0, length, eTag, null, tracingContext); + .read(fileName, 0, buffer, 0, length, eTag, null, null, tracingContext); assertCPKHeaders(abfsRestOperation, true); assertResponseHeader(abfsRestOperation, true, X_MS_ENCRYPTION_KEY_SHA256, getCPKSha(fs)); @@ -171,7 +171,7 @@ public void testReadWithoutCPK() throws Exception { final String eTag = op.getResult() .getResponseHeader(HttpHeaderConfigurations.ETAG); AbfsRestOperation abfsRestOperation = abfsClient - .read(fileName, 0, buffer, 0, length, eTag, null, tracingContext); + .read(fileName, 0, buffer, 0, length, eTag, null, null, tracingContext); assertCPKHeaders(abfsRestOperation, false); assertResponseHeader(abfsRestOperation, false, X_MS_ENCRYPTION_KEY_SHA256, getCPKSha(fs)); @@ -190,7 +190,7 @@ public void testReadWithoutCPK() throws Exception { AbfsClient abfsClient2 = fs2.getAbfsClient()) { LambdaTestUtils.intercept(IOException.class, () -> { abfsClient2.read(fileName, 0, buffer, 0, length, eTag, null, - getTestTracingContext(fs, false)); + null, getTestTracingContext(fs, false)); }); } } @@ -208,7 +208,7 @@ public void testAppendWithCPK() throws Exception { byte[] buffer = getRandomBytesArray(5); AbfsClient abfsClient = fs.getAbfsClient(); AbfsRestOperation abfsRestOperation = abfsClient - .append(fileName, buffer, appendRequestParameters, null, getTestTracingContext(fs, false)); + .append(fileName, buffer, appendRequestParameters, null, null, getTestTracingContext(fs, false)); assertCPKHeaders(abfsRestOperation, true); assertResponseHeader(abfsRestOperation, true, X_MS_ENCRYPTION_KEY_SHA256, getCPKSha(fs)); @@ -225,7 +225,7 @@ public void testAppendWithCPK() throws Exception { AbfsClient abfsClient2 = fs2.getAbfsClient()) { LambdaTestUtils.intercept(IOException.class, () -> { abfsClient2.append(fileName, buffer, appendRequestParameters, null, - getTestTracingContext(fs, false)); + null, getTestTracingContext(fs, false)); }); } @@ -235,7 +235,7 @@ public void testAppendWithCPK() throws Exception { .get(conf); AbfsClient abfsClient3 = fs3.getAbfsClient()) { LambdaTestUtils.intercept(IOException.class, () -> { abfsClient3.append(fileName, buffer, appendRequestParameters, null, - getTestTracingContext(fs, false)); + null, getTestTracingContext(fs, false)); }); } } @@ -253,7 +253,7 @@ public void testAppendWithoutCPK() throws Exception { byte[] buffer = getRandomBytesArray(5); AbfsClient abfsClient = fs.getAbfsClient(); AbfsRestOperation abfsRestOperation = abfsClient - .append(fileName, buffer, appendRequestParameters, null, + .append(fileName, buffer, appendRequestParameters, null, null, getTestTracingContext(fs, false)); assertCPKHeaders(abfsRestOperation, false); assertResponseHeader(abfsRestOperation, false, X_MS_ENCRYPTION_KEY_SHA256, @@ -271,7 +271,7 @@ public void testAppendWithoutCPK() throws Exception { AbfsClient abfsClient2 = fs2.getAbfsClient()) { LambdaTestUtils.intercept(IOException.class, () -> { abfsClient2.append(fileName, buffer, appendRequestParameters, null, - getTestTracingContext(fs, false)); + null, getTestTracingContext(fs, false)); }); } } @@ -432,7 +432,7 @@ private void testListPath(final boolean isWithCPK) throws Exception { AbfsClient abfsClient2 = fs2.getAbfsClient(); TracingContext tracingContext = getTestTracingContext(fs, false); abfsRestOperation = abfsClient2.listPath(testDirName, false, INT_50, - null, tracingContext); + null, null); assertListstatus(fs, abfsRestOperation, testPath); if (isWithCPK) { @@ -486,7 +486,7 @@ private void testCreatePath(final boolean isWithCPK) throws Exception { .createPath(testFileName, true, true, isNamespaceEnabled ? getOctalNotation(permission) : null, isNamespaceEnabled ? getOctalNotation(umask) : null, false, null, - tracingContext); + null, tracingContext); assertCPKHeaders(abfsRestOperation, isWithCPK); assertResponseHeader(abfsRestOperation, isWithCPK, X_MS_ENCRYPTION_KEY_SHA256, getCPKSha(fs)); @@ -572,7 +572,7 @@ private void testFlush(final boolean isWithCPK) throws Exception { AbfsClient abfsClient2 = fs2.getAbfsClient()) { LambdaTestUtils.intercept(IOException.class, () -> { abfsClient2.flush(testFileName, 0, false, false, null, null, - getTestTracingContext(fs, false)); + null, getTestTracingContext(fs, false)); }); } @@ -583,7 +583,7 @@ private void testFlush(final boolean isWithCPK) throws Exception { .get(conf); AbfsClient abfsClient3 = fs3.getAbfsClient()) { LambdaTestUtils.intercept(IOException.class, () -> { abfsClient3.flush(testFileName, 0, false, false, null, null, - getTestTracingContext(fs, false)); + null, getTestTracingContext(fs, false)); }); } } @@ -591,7 +591,7 @@ private void testFlush(final boolean isWithCPK) throws Exception { // With correct CPK AbfsRestOperation abfsRestOperation = abfsClient .flush(testFileName, 0, false, false, null, null, - getTestTracingContext(fs, false)); + null, getTestTracingContext(fs, false)); assertCPKHeaders(abfsRestOperation, isWithCPK); assertResponseHeader(abfsRestOperation, isWithCPK, X_MS_ENCRYPTION_KEY_SHA256, expectedCPKSha); diff --git a/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/TestTracingContext.java b/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/TestTracingContext.java index 006004850d0df..39a9d996e2d69 100644 --- a/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/TestTracingContext.java +++ b/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/TestTracingContext.java @@ -108,7 +108,7 @@ public void checkCorrelationConfigValidation(String clientCorrelationId, //request should not fail for invalid clientCorrelationID AbfsRestOperation op = fs.getAbfsClient() .createPath(path, false, true, permission, umask, false, null, - tracingContext); + null, tracingContext); int statusCode = op.getResult().getStatusCode(); Assertions.assertThat(statusCode).describedAs("Request should not fail") diff --git a/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/services/TestAbfsClient.java b/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/services/TestAbfsClient.java index a725bf3175a5c..68b481d447b57 100644 --- a/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/services/TestAbfsClient.java +++ b/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/services/TestAbfsClient.java @@ -105,7 +105,7 @@ private String getUserAgentString(AbfsConfiguration config, boolean includeSSLProvider) throws IOException { AbfsClientContext abfsClientContext = new AbfsClientContextBuilder().build(); AbfsClient client = new AbfsClient(new URL("https://azure.com"), null, - config, (AccessTokenProvider) null, abfsClientContext); + config, (AccessTokenProvider) null, null, abfsClientContext); String sslProviderName = null; if (includeSSLProvider) { sslProviderName = DelegatingSSLSocketFactory.getDefaultFactory() @@ -276,6 +276,7 @@ public static AbfsClient createTestClientFromCurrentContext( (currentAuthType == AuthType.OAuth ? abfsConfig.getTokenProvider() : null), + null, abfsClientContext); return testClient; diff --git a/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/services/TestAbfsInputStream.java b/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/services/TestAbfsInputStream.java index b5ae9b737842d..c50c192add7f0 100644 --- a/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/services/TestAbfsInputStream.java +++ b/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/services/TestAbfsInputStream.java @@ -39,7 +39,6 @@ import org.apache.hadoop.fs.azurebfs.AbstractAbfsIntegrationTest; import org.apache.hadoop.fs.azurebfs.AzureBlobFileSystem; import org.apache.hadoop.fs.azurebfs.AzureBlobFileSystemStore; -import org.apache.hadoop.fs.azurebfs.contracts.exceptions.AzureBlobFileSystemException; import org.apache.hadoop.fs.azurebfs.contracts.exceptions.TimeoutException; import org.apache.hadoop.fs.azurebfs.contracts.services.ReadBufferStatus; import org.apache.hadoop.fs.azurebfs.utils.TestCachedSASToken; @@ -162,14 +161,14 @@ private void queueReadAheads(AbfsInputStream inputStream) { inputStream.getTracingContext()); } - private void verifyReadCallCount(AbfsClient client, int count) throws - AzureBlobFileSystemException, InterruptedException { + private void verifyReadCallCount(AbfsClient client, int count) + throws IOException, InterruptedException { // ReadAhead threads are triggered asynchronously. // Wait a second before verifying the number of total calls. Thread.sleep(1000); verify(client, times(count)).read(any(String.class), any(Long.class), any(byte[].class), any(Integer.class), any(Integer.class), - any(String.class), any(String.class), any(TracingContext.class)); + any(String.class), any(String.class), any(), any(TracingContext.class)); } private void checkEvictedStatus(AbfsInputStream inputStream, int position, boolean expectedToThrowException) @@ -324,7 +323,7 @@ public void testFailedReadAhead() throws Exception { .when(client) .read(any(String.class), any(Long.class), any(byte[].class), any(Integer.class), any(Integer.class), any(String.class), - any(String.class), any(TracingContext.class)); + any(String.class), any(), any(TracingContext.class)); AbfsInputStream inputStream = getAbfsInputStream(client, "testFailedReadAhead.txt"); @@ -358,7 +357,7 @@ public void testFailedReadAheadEviction() throws Exception { .when(client) .read(any(String.class), any(Long.class), any(byte[].class), any(Integer.class), any(Integer.class), any(String.class), - any(String.class), any(TracingContext.class)); + any(String.class), any(), any(TracingContext.class)); AbfsInputStream inputStream = getAbfsInputStream(client, "testFailedReadAheadEviction.txt"); @@ -403,7 +402,7 @@ public void testOlderReadAheadFailure() throws Exception { .when(client) .read(any(String.class), any(Long.class), any(byte[].class), any(Integer.class), any(Integer.class), any(String.class), - any(String.class), any(TracingContext.class)); + any(String.class), any(), any(TracingContext.class)); AbfsInputStream inputStream = getAbfsInputStream(client, "testOlderReadAheadFailure.txt"); @@ -457,7 +456,7 @@ public void testSuccessfulReadAhead() throws Exception { .when(client) .read(any(String.class), any(Long.class), any(byte[].class), any(Integer.class), any(Integer.class), any(String.class), - any(String.class), any(TracingContext.class)); + any(String.class), any(), any(TracingContext.class)); AbfsInputStream inputStream = getAbfsInputStream(client, "testSuccessfulReadAhead.txt"); int beforeReadCompletedListSize = ReadBufferManager.getBufferManager().getCompletedReadListSize(); @@ -515,7 +514,7 @@ public void testReadAheadManagerForFailedReadAhead() throws Exception { .when(client) .read(any(String.class), any(Long.class), any(byte[].class), any(Integer.class), any(Integer.class), any(String.class), - any(String.class), any(TracingContext.class)); + any(String.class), any(), any(TracingContext.class)); AbfsInputStream inputStream = getAbfsInputStream(client, "testReadAheadManagerForFailedReadAhead.txt"); @@ -568,7 +567,7 @@ public void testReadAheadManagerForOlderReadAheadFailure() throws Exception { .when(client) .read(any(String.class), any(Long.class), any(byte[].class), any(Integer.class), any(Integer.class), any(String.class), - any(String.class), any(TracingContext.class)); + any(String.class), any(), any(TracingContext.class)); AbfsInputStream inputStream = getAbfsInputStream(client, "testReadAheadManagerForOlderReadAheadFailure.txt"); @@ -622,7 +621,7 @@ public void testReadAheadManagerForSuccessfulReadAhead() throws Exception { .when(client) .read(any(String.class), any(Long.class), any(byte[].class), any(Integer.class), any(Integer.class), any(String.class), - any(String.class), any(TracingContext.class)); + any(String.class), any(), any(TracingContext.class)); AbfsInputStream inputStream = getAbfsInputStream(client, "testSuccessfulReadAhead.txt"); diff --git a/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/services/TestAbfsOutputStream.java b/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/services/TestAbfsOutputStream.java index f01c81b74eeed..8e33015289c52 100644 --- a/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/services/TestAbfsOutputStream.java +++ b/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/services/TestAbfsOutputStream.java @@ -90,10 +90,11 @@ public void verifyShortWriteRequest() throws Exception { AbfsPerfTracker tracker = new AbfsPerfTracker("test", accountName1, abfsConf); when(client.getAbfsPerfTracker()).thenReturn(tracker); when(client.append(anyString(), any(byte[].class), - any(AppendRequestParameters.class), any(), any(TracingContext.class))) + any(AppendRequestParameters.class), any(), + any(), any(TracingContext.class))) .thenReturn(op); when(client.flush(anyString(), anyLong(), anyBoolean(), anyBoolean(), any(), - isNull(), any(TracingContext.class))).thenReturn(op); + isNull(), any(), any(TracingContext.class))).thenReturn(op); AbfsOutputStream out = new AbfsOutputStream(client, null, PATH, 0, populateAbfsOutputStreamContext(BUFFER_SIZE, true, false, false), @@ -119,13 +120,13 @@ public void verifyShortWriteRequest() throws Exception { WRITE_SIZE, 0, 2 * WRITE_SIZE, APPEND_MODE, false, null); verify(client, times(1)).append( - eq(PATH), any(byte[].class), refEq(firstReqParameters), any(), + eq(PATH), any(byte[].class), refEq(firstReqParameters), any(), any(), any(TracingContext.class)); verify(client, times(1)).append( - eq(PATH), any(byte[].class), refEq(secondReqParameters), any(), any(TracingContext.class)); + eq(PATH), any(byte[].class), refEq(secondReqParameters), any(), any(), any(TracingContext.class)); // confirm there were only 2 invocations in all verify(client, times(2)).append( - eq(PATH), any(byte[].class), any(), any(), any(TracingContext.class)); + eq(PATH), any(byte[].class), any(), any(), any(), any(TracingContext.class)); } /** @@ -146,8 +147,8 @@ public void verifyWriteRequest() throws Exception { TracingHeaderFormat.ALL_ID_FORMAT, null); when(client.getAbfsPerfTracker()).thenReturn(tracker); - when(client.append(anyString(), any(byte[].class), any(AppendRequestParameters.class), any(), any(TracingContext.class))).thenReturn(op); - when(client.flush(anyString(), anyLong(), anyBoolean(), anyBoolean(), any(), isNull(), any(TracingContext.class))).thenReturn(op); + when(client.append(anyString(), any(byte[].class), any(AppendRequestParameters.class), any(), any(), any(TracingContext.class))).thenReturn(op); + when(client.flush(anyString(), anyLong(), anyBoolean(), anyBoolean(), any(), isNull(), any(), any(TracingContext.class))).thenReturn(op); AbfsOutputStream out = new AbfsOutputStream(client, null, PATH, 0, populateAbfsOutputStreamContext(BUFFER_SIZE, true, false, false), @@ -165,16 +166,13 @@ public void verifyWriteRequest() throws Exception { AppendRequestParameters secondReqParameters = new AppendRequestParameters( BUFFER_SIZE, 0, 5*WRITE_SIZE-BUFFER_SIZE, APPEND_MODE, false, null); - verify(client, times(1)).append( - eq(PATH), any(byte[].class), refEq(firstReqParameters), any(), - any(TracingContext.class)); - verify(client, times(1)).append( - eq(PATH), any(byte[].class), refEq(secondReqParameters), any(), - any(TracingContext.class)); + verify(client, times(1)).append(eq(PATH), any(byte[].class), + refEq(firstReqParameters), any(), any(), any(TracingContext.class)); + verify(client, times(1)).append(eq(PATH), any(byte[].class), + refEq(secondReqParameters), any(), any(), any(TracingContext.class)); // confirm there were only 2 invocations in all - verify(client, times(2)).append( - eq(PATH), any(byte[].class), any(), any(), - any(TracingContext.class)); + verify(client, times(2)).append(eq(PATH), any(byte[].class), any(), any(), + any(), any(TracingContext.class)); ArgumentCaptor acFlushPath = ArgumentCaptor.forClass(String.class); ArgumentCaptor acFlushPosition = ArgumentCaptor.forClass(Long.class); @@ -185,7 +183,8 @@ public void verifyWriteRequest() throws Exception { ArgumentCaptor acFlushSASToken = ArgumentCaptor.forClass(String.class); verify(client, times(1)).flush(acFlushPath.capture(), acFlushPosition.capture(), acFlushRetainUnCommittedData.capture(), acFlushClose.capture(), - acFlushSASToken.capture(), isNull(), acTracingContext.capture()); + acFlushSASToken.capture(), isNull(), isNull(), + acTracingContext.capture()); assertThat(Arrays.asList(PATH)).describedAs("path").isEqualTo(acFlushPath.getAllValues()); assertThat(Arrays.asList(Long.valueOf(5*WRITE_SIZE))).describedAs("position").isEqualTo(acFlushPosition.getAllValues()); assertThat(Arrays.asList(false)).describedAs("RetainUnCommittedData flag").isEqualTo(acFlushRetainUnCommittedData.getAllValues()); @@ -211,8 +210,8 @@ public void verifyWriteRequestOfBufferSizeAndClose() throws Exception { FSOperationType.WRITE, abfsConf.getTracingHeaderFormat(), null); when(client.getAbfsPerfTracker()).thenReturn(tracker); - when(client.append(anyString(), any(byte[].class), any(AppendRequestParameters.class), any(), any(TracingContext.class))).thenReturn(op); - when(client.flush(anyString(), anyLong(), anyBoolean(), anyBoolean(), any(), isNull(), any(TracingContext.class))).thenReturn(op); + when(client.append(anyString(), any(byte[].class), any(AppendRequestParameters.class), any(), any(), any(TracingContext.class))).thenReturn(op); + when(client.flush(anyString(), anyLong(), anyBoolean(), anyBoolean(), any(), isNull(), any(), any(TracingContext.class))).thenReturn(op); when(op.getSasToken()).thenReturn("testToken"); when(op.getResult()).thenReturn(httpOp); @@ -233,12 +232,12 @@ public void verifyWriteRequestOfBufferSizeAndClose() throws Exception { BUFFER_SIZE, 0, BUFFER_SIZE, APPEND_MODE, false, null); verify(client, times(1)).append( - eq(PATH), any(byte[].class), refEq(firstReqParameters), any(), any(TracingContext.class)); + eq(PATH), any(byte[].class), refEq(firstReqParameters), any(), any(), any(TracingContext.class)); verify(client, times(1)).append( - eq(PATH), any(byte[].class), refEq(secondReqParameters), any(), any(TracingContext.class)); + eq(PATH), any(byte[].class), refEq(secondReqParameters), any(), any(), any(TracingContext.class)); // confirm there were only 2 invocations in all verify(client, times(2)).append( - eq(PATH), any(byte[].class), any(), any(), any(TracingContext.class)); + eq(PATH), any(byte[].class), any(), any(), any(), any(TracingContext.class)); ArgumentCaptor acFlushPath = ArgumentCaptor.forClass(String.class); ArgumentCaptor acFlushPosition = ArgumentCaptor.forClass(Long.class); @@ -249,7 +248,8 @@ public void verifyWriteRequestOfBufferSizeAndClose() throws Exception { ArgumentCaptor acFlushSASToken = ArgumentCaptor.forClass(String.class); verify(client, times(1)).flush(acFlushPath.capture(), acFlushPosition.capture(), acFlushRetainUnCommittedData.capture(), acFlushClose.capture(), - acFlushSASToken.capture(), isNull(), acTracingContext.capture()); + acFlushSASToken.capture(), isNull(), isNull(), + acTracingContext.capture()); assertThat(Arrays.asList(PATH)).describedAs("path").isEqualTo(acFlushPath.getAllValues()); assertThat(Arrays.asList(Long.valueOf(2*BUFFER_SIZE))).describedAs("position").isEqualTo(acFlushPosition.getAllValues()); assertThat(Arrays.asList(false)).describedAs("RetainUnCommittedData flag").isEqualTo(acFlushRetainUnCommittedData.getAllValues()); @@ -273,10 +273,10 @@ public void verifyWriteRequestOfBufferSize() throws Exception { when(client.getAbfsPerfTracker()).thenReturn(tracker); when(client.append(anyString(), any(byte[].class), - any(AppendRequestParameters.class), any(), any(TracingContext.class))) + any(AppendRequestParameters.class), any(), any(), any(TracingContext.class))) .thenReturn(op); when(client.flush(anyString(), anyLong(), anyBoolean(), anyBoolean(), - any(), isNull(), any(TracingContext.class))).thenReturn(op); + any(), isNull(), any(), any(TracingContext.class))).thenReturn(op); when(op.getSasToken()).thenReturn("testToken"); when(op.getResult()).thenReturn(httpOp); @@ -299,12 +299,12 @@ public void verifyWriteRequestOfBufferSize() throws Exception { BUFFER_SIZE, 0, BUFFER_SIZE, APPEND_MODE, false, null); verify(client, times(1)).append( - eq(PATH), any(byte[].class), refEq(firstReqParameters), any(), any(TracingContext.class)); + eq(PATH), any(byte[].class), refEq(firstReqParameters), any(), any(), any(TracingContext.class)); verify(client, times(1)).append( - eq(PATH), any(byte[].class), refEq(secondReqParameters), any(), any(TracingContext.class)); + eq(PATH), any(byte[].class), refEq(secondReqParameters), any(), any(), any(TracingContext.class)); // confirm there were only 2 invocations in all verify(client, times(2)).append( - eq(PATH), any(byte[].class), any(), any(), any(TracingContext.class)); + eq(PATH), any(byte[].class), any(), any(), any(), any(TracingContext.class)); } /** @@ -323,10 +323,10 @@ public void verifyWriteRequestOfBufferSizeWithAppendBlob() throws Exception { when(client.getAbfsPerfTracker()).thenReturn(tracker); when(client.append(anyString(), any(byte[].class), - any(AppendRequestParameters.class), any(), any(TracingContext.class))) + any(AppendRequestParameters.class), any(), any(), any(TracingContext.class))) .thenReturn(op); when(client.flush(anyString(), anyLong(), anyBoolean(), anyBoolean(), any(), - isNull(), any(TracingContext.class))).thenReturn(op); + isNull(), any(), any(TracingContext.class))).thenReturn(op); AbfsOutputStream out = new AbfsOutputStream(client, null, PATH, 0, populateAbfsOutputStreamContext(BUFFER_SIZE, true, false, true), @@ -347,12 +347,12 @@ public void verifyWriteRequestOfBufferSizeWithAppendBlob() throws Exception { BUFFER_SIZE, 0, BUFFER_SIZE, APPEND_MODE, true, null); verify(client, times(1)).append( - eq(PATH), any(byte[].class), refEq(firstReqParameters), any(), any(TracingContext.class)); + eq(PATH), any(byte[].class), refEq(firstReqParameters), any(), any(), any(TracingContext.class)); verify(client, times(1)).append( - eq(PATH), any(byte[].class), refEq(secondReqParameters), any(), any(TracingContext.class)); + eq(PATH), any(byte[].class), refEq(secondReqParameters), any(), any(), any(TracingContext.class)); // confirm there were only 2 invocations in all verify(client, times(2)).append( - eq(PATH), any(byte[].class), any(), any(), any(TracingContext.class)); + eq(PATH), any(byte[].class), any(), any(), any(), any(TracingContext.class)); } /** @@ -375,10 +375,10 @@ public void verifyWriteRequestOfBufferSizeAndHFlush() throws Exception { when(client.getAbfsPerfTracker()).thenReturn(tracker); when(client.append(anyString(), any(byte[].class), - any(AppendRequestParameters.class), any(), any(TracingContext.class))) + any(AppendRequestParameters.class), any(), any(), any(TracingContext.class))) .thenReturn(op); when(client.flush(anyString(), anyLong(), anyBoolean(), anyBoolean(), any(), - isNull(), any(TracingContext.class))).thenReturn(op); + isNull(), any(), any(TracingContext.class))).thenReturn(op); AbfsOutputStream out = new AbfsOutputStream(client, null, PATH, 0, populateAbfsOutputStreamContext(BUFFER_SIZE, true, false, false), new TracingContext(abfsConf.getClientCorrelationId(), "test-fs-id", @@ -398,12 +398,12 @@ public void verifyWriteRequestOfBufferSizeAndHFlush() throws Exception { BUFFER_SIZE, 0, BUFFER_SIZE, APPEND_MODE, false, null); verify(client, times(1)).append( - eq(PATH), any(byte[].class), refEq(firstReqParameters), any(), any(TracingContext.class)); + eq(PATH), any(byte[].class), refEq(firstReqParameters), any(), any(), any(TracingContext.class)); verify(client, times(1)).append( - eq(PATH), any(byte[].class), refEq(secondReqParameters), any(), any(TracingContext.class)); + eq(PATH), any(byte[].class), refEq(secondReqParameters), any(), any(), any(TracingContext.class)); // confirm there were only 2 invocations in all verify(client, times(2)).append( - eq(PATH), any(byte[].class), any(), any(), any(TracingContext.class)); + eq(PATH), any(byte[].class), any(), any(), any(), any(TracingContext.class)); ArgumentCaptor acFlushPath = ArgumentCaptor.forClass(String.class); ArgumentCaptor acFlushPosition = ArgumentCaptor.forClass(Long.class); @@ -414,7 +414,7 @@ public void verifyWriteRequestOfBufferSizeAndHFlush() throws Exception { ArgumentCaptor acFlushSASToken = ArgumentCaptor.forClass(String.class); verify(client, times(1)).flush(acFlushPath.capture(), acFlushPosition.capture(), acFlushRetainUnCommittedData.capture(), acFlushClose.capture(), - acFlushSASToken.capture(), isNull(), acTracingContext.capture()); + acFlushSASToken.capture(), isNull(), isNull(), acTracingContext.capture()); assertThat(Arrays.asList(PATH)).describedAs("path").isEqualTo(acFlushPath.getAllValues()); assertThat(Arrays.asList(Long.valueOf(2*BUFFER_SIZE))).describedAs("position").isEqualTo(acFlushPosition.getAllValues()); assertThat(Arrays.asList(false)).describedAs("RetainUnCommittedData flag").isEqualTo(acFlushRetainUnCommittedData.getAllValues()); @@ -436,10 +436,10 @@ public void verifyWriteRequestOfBufferSizeAndFlush() throws Exception { AbfsPerfTracker tracker = new AbfsPerfTracker("test", accountName1, abfsConf); when(client.getAbfsPerfTracker()).thenReturn(tracker); when(client.append(anyString(), any(byte[].class), - any(AppendRequestParameters.class), any(), any(TracingContext.class))) + any(AppendRequestParameters.class), any(), any(), any(TracingContext.class))) .thenReturn(op); when(client.flush(anyString(), anyLong(), anyBoolean(), anyBoolean(), any(), - isNull(), any(TracingContext.class))).thenReturn(op); + isNull(), any(), any(TracingContext.class))).thenReturn(op); AbfsOutputStream out = new AbfsOutputStream(client, null, PATH, 0, populateAbfsOutputStreamContext(BUFFER_SIZE, true, false, false), @@ -462,11 +462,11 @@ public void verifyWriteRequestOfBufferSizeAndFlush() throws Exception { BUFFER_SIZE, 0, BUFFER_SIZE, APPEND_MODE, false, null); verify(client, times(1)).append( - eq(PATH), any(byte[].class), refEq(firstReqParameters), any(), any(TracingContext.class)); + eq(PATH), any(byte[].class), refEq(firstReqParameters), any(), any(), any(TracingContext.class)); verify(client, times(1)).append( - eq(PATH), any(byte[].class), refEq(secondReqParameters), any(), any(TracingContext.class)); + eq(PATH), any(byte[].class), refEq(secondReqParameters), any(), any(), any(TracingContext.class)); // confirm there were only 2 invocations in all verify(client, times(2)).append( - eq(PATH), any(byte[].class), any(), any(), any(TracingContext.class)); + eq(PATH), any(byte[].class), any(), any(), any(), any(TracingContext.class)); } } From 10adf42e700b993f3858386e33255dac10110811 Mon Sep 17 00:00:00 2001 From: sumangala Date: Mon, 20 Sep 2021 11:00:12 +0530 Subject: [PATCH 06/77] encryption test --- .../hadoop/fs/azurebfs/AbfsConfiguration.java | 3 ++ .../fs/azurebfs/services/AbfsClient.java | 19 +++++---- .../azurebfs/services/AbfsOutputStream.java | 5 +++ .../fs/azurebfs/ITestEncryptionContext.java | 39 +++++++++++++++++++ .../MockEncryptionContextProvider.java | 3 +- 5 files changed, 57 insertions(+), 12 deletions(-) create mode 100644 hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/ITestEncryptionContext.java diff --git a/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/AbfsConfiguration.java b/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/AbfsConfiguration.java index cf240e73edf91..d73750624e097 100644 --- a/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/AbfsConfiguration.java +++ b/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/AbfsConfiguration.java @@ -911,6 +911,9 @@ public EncryptionContextProvider initializeEncryptionContextProvider() { try { String configKey = FS_AZURE_ENCRYPTION_CONTEXT_PROVIDER_TYPE; + if (get(configKey) == null) { + return null; + } Class encryptionContextClass = getAccountSpecificClass(configKey, null, EncryptionContextProvider.class); if (encryptionContextClass == null) { diff --git a/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/services/AbfsClient.java b/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/services/AbfsClient.java index 93b806a9aa4ec..81cd8d964fab9 100644 --- a/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/services/AbfsClient.java +++ b/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/services/AbfsClient.java @@ -195,10 +195,6 @@ private String getBase64EncodedString(byte[] bytes) { return Base64.getEncoder().encodeToString(bytes); } - public void setClientProvidedEncryptionKey(String key) { - clientProvidedEncryptionKey = key; - } - @Override public void close() throws IOException { if (tokenProvider instanceof Closeable) { @@ -255,10 +251,13 @@ private void addEncryptionKeyRequestHeaders( break; case ENCRYPTION_CONTEXT: if (isCreateFileRequest) { - encryptionContext = encryptionContextProvider.getEncryptionContext(path) - .toString(); // get new context for create file request - requestHeaders.add(new AbfsHttpHeader(X_MS_ENCRYPTION_CONTEXT, - encryptionContext)); +// encryptionContext = new String((encryptionContextProvider.getEncryptionContext(path)).readAllBytes(), StandardCharsets.UTF_8); // get new context for create file + encryptionContext = + org.apache.commons.io.IOUtils.toString(encryptionContextProvider.getEncryptionContext(path), + StandardCharsets.UTF_8.name()); + // request +// requestHeaders.add(new AbfsHttpHeader(X_MS_ENCRYPTION_CONTEXT, +// encryptionContext)); encryptionHeaders = getEncryptionHeaders(path, encryptionContext); } else if (encryptionHeaders == null || encryptionHeaders.isEmpty()) { @@ -281,8 +280,8 @@ private void addEncryptionKeyRequestHeaders( public HashMap getEncryptionHeaders(String path, String encryptionContext) throws IOException { - String key = encryptionContextProvider.getEncryptionKey(path, - encryptionContext).toString(); + String key = org.apache.commons.io.IOUtils.toString(encryptionContextProvider.getEncryptionKey(path, + encryptionContext), StandardCharsets.UTF_8.name()); HashMap encryptionHeaders = new HashMap<>(); encryptionHeaders.put(X_MS_ENCRYPTION_KEY, getBase64EncodedString(key)); encryptionHeaders.put(X_MS_ENCRYPTION_KEY_SHA256, diff --git a/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/services/AbfsOutputStream.java b/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/services/AbfsOutputStream.java index 8c654eef9fa8d..47c2f69f32d93 100644 --- a/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/services/AbfsOutputStream.java +++ b/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/services/AbfsOutputStream.java @@ -679,6 +679,11 @@ public IOStatistics getIOStatistics() { return ioStatistics; } + @VisibleForTesting + public boolean isEncryptionHeadersCached() { + return encryptionHeaders != null; + } + @VisibleForTesting public boolean isLeaseFreed() { if (lease == null) { diff --git a/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/ITestEncryptionContext.java b/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/ITestEncryptionContext.java new file mode 100644 index 0000000000000..efaf269f58922 --- /dev/null +++ b/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/ITestEncryptionContext.java @@ -0,0 +1,39 @@ +package org.apache.hadoop.fs.azurebfs; + +import org.apache.hadoop.conf.Configuration; +import org.apache.hadoop.fs.FSDataOutputStream; +import org.apache.hadoop.fs.Path; +import org.apache.hadoop.fs.azurebfs.extensions.MockDelegationSASTokenProvider; +import org.apache.hadoop.fs.azurebfs.extensions.MockEncryptionContextProvider; +import org.apache.hadoop.fs.azurebfs.services.AbfsOutputStream; +import org.assertj.core.api.Assertions; +import org.junit.Assume; +import org.junit.Test; + +import static org.apache.hadoop.fs.azurebfs.constants.ConfigurationKeys.FS_AZURE_ENCRYPTION_CONTEXT_PROVIDER_TYPE; +import static org.apache.hadoop.fs.azurebfs.constants.ConfigurationKeys.FS_AZURE_SAS_TOKEN_PROVIDER_TYPE; + +public class ITestEncryptionContext extends AbstractAbfsIntegrationTest { + public ITestEncryptionContext() throws Exception { + String encryptionContextProvider = getRawConfiguration().get( + FS_AZURE_ENCRYPTION_CONTEXT_PROVIDER_TYPE); + Assume.assumeTrue(MockEncryptionContextProvider.class.getCanonicalName() + .equals(encryptionContextProvider)); + } + + private AzureBlobFileSystem getAbfsEncrypted() throws Exception { + Configuration configuration = getRawConfiguration(); + configuration.set(FS_AZURE_ENCRYPTION_CONTEXT_PROVIDER_TYPE, "MockEncryptionContextProvider"); + return getFileSystem(configuration); + } + + @Test + public void testCreateFile() throws Exception { + AzureBlobFileSystem fs = getFileSystem(); +// AzureBlobFileSystem fs = getAbfsEncrypted(); + Path testPath = path("createTest"); + FSDataOutputStream out = fs.create(testPath); + Assertions.assertThat(((AbfsOutputStream) out.getWrappedStream()) + .isEncryptionHeadersCached()).isTrue(); + } +} diff --git a/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/extensions/MockEncryptionContextProvider.java b/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/extensions/MockEncryptionContextProvider.java index 55cd24694f577..11c0022a2b3e1 100644 --- a/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/extensions/MockEncryptionContextProvider.java +++ b/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/extensions/MockEncryptionContextProvider.java @@ -1,6 +1,5 @@ package org.apache.hadoop.fs.azurebfs.extensions; -import org.apache.avro.util.ByteBufferInputStream; import org.apache.hadoop.conf.Configuration; import java.io.ByteArrayInputStream; @@ -33,7 +32,7 @@ public ByteArrayInputStream getEncryptionContext(String path) @Override public ByteArrayInputStream getEncryptionKey(String path, String encryptionContext) throws IOException { - if (encryptionContext.equals(pathToContextMap.get(path))) { + if (!encryptionContext.equals(pathToContextMap.get(path))) { throw new IOException("encryption context does not match path"); } return new ByteArrayInputStream(contextToKeyMap.get(encryptionContext) From 84df4a2cbe2da039f42e52beeb2ef8fc84cfa6a5 Mon Sep 17 00:00:00 2001 From: sumangala Date: Tue, 21 Sep 2021 18:17:14 +0530 Subject: [PATCH 07/77] encryption adapter --- .../fs/azurebfs/AzureBlobFileSystemStore.java | 32 +++++---- .../extensions/EncryptionContextProvider.java | 12 ++-- .../azurebfs/security/EncryptionAdapter.java | 69 ++++++++++++++++++ .../fs/azurebfs/services/AbfsClient.java | 72 ++++++++++--------- .../fs/azurebfs/services/AbfsInputStream.java | 8 +-- .../services/AbfsInputStreamContext.java | 13 ++-- .../azurebfs/services/AbfsOutputStream.java | 13 ++-- .../services/AbfsOutputStreamContext.java | 14 ++-- .../MockEncryptionContextProvider.java | 38 ++++++++-- 9 files changed, 187 insertions(+), 84 deletions(-) create mode 100644 hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/security/EncryptionAdapter.java diff --git a/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/AzureBlobFileSystemStore.java b/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/AzureBlobFileSystemStore.java index 27e4362aca384..fd8cb5a139402 100644 --- a/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/AzureBlobFileSystemStore.java +++ b/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/AzureBlobFileSystemStore.java @@ -53,6 +53,7 @@ import java.util.concurrent.ExecutionException; import org.apache.hadoop.fs.azurebfs.extensions.EncryptionContextProvider; +import org.apache.hadoop.fs.azurebfs.security.EncryptionAdapter; import org.apache.hadoop.fs.azurebfs.utils.EncryptionType; import org.apache.hadoop.thirdparty.com.google.common.annotations.VisibleForTesting; import org.apache.hadoop.thirdparty.com.google.common.base.Preconditions; @@ -526,15 +527,16 @@ public OutputStream createFile(final Path path, triggerConditionalCreateOverwrite = true; } + EncryptionAdapter encryptionAdapter = new EncryptionAdapter( + client.getEncryptionContextProvider(), path.toString()); AbfsRestOperation op; - HashMap encryptionHeaders = new HashMap<>(); if (triggerConditionalCreateOverwrite) { op = conditionalCreateOverwriteFile(relativePath, statistics, isNamespaceEnabled ? getOctalNotation(permission) : null, isNamespaceEnabled ? getOctalNotation(umask) : null, isAppendBlob, - encryptionHeaders, + encryptionAdapter, tracingContext ); @@ -545,7 +547,7 @@ public OutputStream createFile(final Path path, isNamespaceEnabled ? getOctalNotation(umask) : null, isAppendBlob, null, - encryptionHeaders, + encryptionAdapter, tracingContext); } @@ -558,7 +560,7 @@ public OutputStream createFile(final Path path, statistics, relativePath, 0, - populateAbfsOutputStreamContext(isAppendBlob, lease, encryptionHeaders), + populateAbfsOutputStreamContext(isAppendBlob, lease, encryptionAdapter), tracingContext); } } @@ -579,7 +581,7 @@ private AbfsRestOperation conditionalCreateOverwriteFile(final String relativePa final String permission, final String umask, final boolean isAppendBlob, - HashMap encryptionHeaders, + EncryptionAdapter encryptionAdapter, TracingContext tracingContext) throws IOException { AbfsRestOperation op; @@ -588,7 +590,7 @@ private AbfsRestOperation conditionalCreateOverwriteFile(final String relativePa // avoided for cases when no pre-existing file is present (major portion // of create file traffic falls into the case of no pre-existing file). op = client.createPath(relativePath, true, false, permission, umask, - isAppendBlob, null, encryptionHeaders, tracingContext); + isAppendBlob, null, encryptionAdapter, tracingContext); } catch (AbfsRestOperationException e) { if (e.getStatusCode() == HttpURLConnection.HTTP_CONFLICT) { @@ -613,7 +615,7 @@ private AbfsRestOperation conditionalCreateOverwriteFile(final String relativePa try { // overwrite only if eTag matches with the file properties fetched befpre op = client.createPath(relativePath, true, true, permission, umask, - isAppendBlob, eTag, encryptionHeaders, tracingContext); + isAppendBlob, eTag, encryptionAdapter, tracingContext); } catch (AbfsRestOperationException ex) { if (ex.getStatusCode() == HttpURLConnection.HTTP_PRECON_FAILED) { // Is a parallel access case, as file with eTag was just queried @@ -635,7 +637,7 @@ private AbfsRestOperation conditionalCreateOverwriteFile(final String relativePa } private AbfsOutputStreamContext populateAbfsOutputStreamContext(boolean isAppendBlob, - AbfsLease lease, HashMap encryptionHeaders) { + AbfsLease lease, EncryptionAdapter encryptionAdapter) { int bufferSize = abfsConfiguration.getWriteBufferSize(); if (isAppendBlob && bufferSize > FileSystemConfigurations.APPENDBLOB_MAX_WRITE_BUFFER_SIZE) { bufferSize = FileSystemConfigurations.APPENDBLOB_MAX_WRITE_BUFFER_SIZE; @@ -650,7 +652,7 @@ private AbfsOutputStreamContext populateAbfsOutputStreamContext(boolean isAppend .withWriteMaxConcurrentRequestCount(abfsConfiguration.getWriteMaxConcurrentRequestCount()) .withMaxWriteRequestsToQueue(abfsConfiguration.getMaxWriteRequestsToQueue()) .withLease(lease) - .withEncryptionHeaders(encryptionHeaders) + .withEncryptionAdapter(encryptionAdapter) .build(); } @@ -698,7 +700,7 @@ public AbfsInputStream openFileForRead(Path path, String relativePath = getRelativePath(path); String resourceType, eTag, encryptionContext; long contentLength; - HashMap encryptionHeaders = null; + EncryptionAdapter encryptionAdapter = null; if (fileStatus instanceof VersionedFileStatus && client.getEncryptionType() != EncryptionType.ENCRYPTION_CONTEXT) { path = path.makeQualified(this.uri, path); @@ -728,7 +730,7 @@ public AbfsInputStream openFileForRead(Path path, throw new IOException( "Encryption context missing from server response"); } - encryptionHeaders = client.getEncryptionHeaders(path.toString(), + encryptionAdapter = client.getEncryptionAdapter(path.toString(), encryptionContext); } } @@ -746,13 +748,13 @@ public AbfsInputStream openFileForRead(Path path, // Add statistics for InputStream return new AbfsInputStream(client, statistics, relativePath, contentLength, populateAbfsInputStreamContext( - parameters.map(OpenFileParameters::getOptions), encryptionHeaders), + parameters.map(OpenFileParameters::getOptions), encryptionAdapter), eTag, tracingContext); } } private AbfsInputStreamContext populateAbfsInputStreamContext( - Optional options, HashMap encryptionHeaders) { + Optional options, EncryptionAdapter encryptionAdapter) { boolean bufferedPreadDisabled = options .map(c -> c.getBoolean(FS_AZURE_BUFFERED_PREAD_DISABLE, false)) .orElse(false); @@ -768,7 +770,7 @@ private AbfsInputStreamContext populateAbfsInputStreamContext( abfsConfiguration.shouldReadBufferSizeAlways()) .withReadAheadBlockSize(abfsConfiguration.getReadAheadBlockSize()) .withBufferedPreadDisabled(bufferedPreadDisabled) - .withEncryptionHeaders(encryptionHeaders) + .withEncryptionHeaders(encryptionAdapter) .build(); } @@ -817,7 +819,7 @@ public OutputStream openFileForWrite(final Path path, relativePath, offset, populateAbfsOutputStreamContext(isAppendBlob, lease, - client.getEncryptionHeaders(path.toString(), encryptionContext)), + client.getEncryptionAdapter(path.toString(), encryptionContext)), tracingContext); } } diff --git a/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/extensions/EncryptionContextProvider.java b/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/extensions/EncryptionContextProvider.java index ae84e9257a932..55344e86b3de4 100644 --- a/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/extensions/EncryptionContextProvider.java +++ b/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/extensions/EncryptionContextProvider.java @@ -20,10 +20,11 @@ import org.apache.hadoop.conf.Configuration; -import java.io.ByteArrayInputStream; +import javax.crypto.SecretKey; +import javax.security.auth.Destroyable; import java.io.IOException; -public interface EncryptionContextProvider extends AutoCloseable { +public interface EncryptionContextProvider extends Destroyable { /** * Initialize instance * @@ -40,7 +41,7 @@ public interface EncryptionContextProvider extends AutoCloseable { * @return encryptionContext string * @throws IOException */ - ByteArrayInputStream getEncryptionContext(String path) throws IOException; + String getEncryptionContext(String path) throws IOException; /** * Fetch encryption key in-exchange for encryption context @@ -50,8 +51,7 @@ public interface EncryptionContextProvider extends AutoCloseable { * @return Encryption key * @throws IOException */ - ByteArrayInputStream getEncryptionKey(String path, String encryptionContext) throws IOException; + SecretKey getEncryptionKey(String path, String encryptionContext) throws IOException; - @Override - void close() throws IOException; + void destroy(); } diff --git a/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/security/EncryptionAdapter.java b/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/security/EncryptionAdapter.java new file mode 100644 index 0000000000000..ddad9c68182b1 --- /dev/null +++ b/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/security/EncryptionAdapter.java @@ -0,0 +1,69 @@ +package org.apache.hadoop.fs.azurebfs.security; + +import org.apache.hadoop.fs.azurebfs.extensions.EncryptionContextProvider; +import org.apache.hadoop.thirdparty.com.google.common.base.Preconditions; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + +import javax.crypto.SecretKey; +import javax.security.auth.DestroyFailedException; +import javax.security.auth.Destroyable; +import java.io.IOException; + +public class EncryptionAdapter implements Destroyable { + String path; + String encryptionContext; + SecretKey encryptionKey; + EncryptionContextProvider provider; + byte[] encodedKey = null; + byte[] encodedKeySHA = null; + private static final Logger LOG = + LoggerFactory.getLogger(EncryptionAdapter.class); + + public EncryptionAdapter(EncryptionContextProvider provider, String path, + String encryptionContext) throws IOException { + this.provider = provider; + this.path = path; + Preconditions.checkNotNull(encryptionContext, + "Encryption context should not be null."); + this.encryptionContext = encryptionContext; + this.encryptionKey = provider.getEncryptionKey(path, encryptionContext); + } + + public EncryptionAdapter(EncryptionContextProvider provider, String path) + throws IOException { + this(provider, path, provider.getEncryptionContext(path)); + } + + public SecretKey getEncryptionKey() throws IOException { + return encryptionKey; + } + + public String getEncryptionContext(String path) throws IOException { + if (encryptionContext != null) { + return encryptionContext; + } + return provider.getEncryptionContext(path); + } + + public byte[] getEncodedKey() { + return encodedKey; + } + + public void setEncodedKey(byte[] encodedKey) { + this.encodedKey = encodedKey; + } + + public byte[] getEncodedKeySHA() { + return encodedKeySHA; + } + + public void setEncodedKeySHA(byte[] encodedKeySHA) { + this.encodedKeySHA = encodedKeySHA; + } + + public void destroy() throws DestroyFailedException { + encryptionKey.destroy(); + provider.destroy(); + } +} diff --git a/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/services/AbfsClient.java b/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/services/AbfsClient.java index 81cd8d964fab9..2554add32e69a 100644 --- a/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/services/AbfsClient.java +++ b/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/services/AbfsClient.java @@ -31,7 +31,6 @@ import java.time.Instant; import java.util.ArrayList; import java.util.Base64; -import java.util.HashMap; import java.util.List; import java.util.Locale; import java.util.UUID; @@ -40,6 +39,7 @@ import java.util.concurrent.TimeUnit; import org.apache.hadoop.fs.azurebfs.extensions.EncryptionContextProvider; +import org.apache.hadoop.fs.azurebfs.security.EncryptionAdapter; import org.apache.hadoop.fs.azurebfs.utils.EncryptionType; import org.apache.hadoop.thirdparty.com.google.common.annotations.VisibleForTesting; import org.apache.hadoop.thirdparty.com.google.common.base.Strings; @@ -71,6 +71,8 @@ import org.apache.hadoop.security.ssl.DelegatingSSLSocketFactory; import org.apache.hadoop.util.concurrent.HadoopExecutors; +import javax.crypto.SecretKey; + import static org.apache.hadoop.fs.azurebfs.constants.AbfsHttpConstants.*; import static org.apache.hadoop.fs.azurebfs.constants.FileSystemConfigurations.DEFAULT_DELETE_CONSIDERED_IDEMPOTENT; import static org.apache.hadoop.fs.azurebfs.constants.FileSystemConfigurations.SERVER_SIDE_ENCRYPTION_ALGORITHM; @@ -240,7 +242,7 @@ List createDefaultHeaders() { private void addEncryptionKeyRequestHeaders( List requestHeaders, String path, - boolean isCreateFileRequest, HashMap encryptionHeaders, + boolean isCreateFileRequest, EncryptionAdapter encryptionAdapter, TracingContext tracingContext) throws IOException { String encodedKey, encodedKeySHA256, encryptionContext; @@ -251,23 +253,22 @@ private void addEncryptionKeyRequestHeaders( break; case ENCRYPTION_CONTEXT: if (isCreateFileRequest) { -// encryptionContext = new String((encryptionContextProvider.getEncryptionContext(path)).readAllBytes(), StandardCharsets.UTF_8); // get new context for create file - encryptionContext = - org.apache.commons.io.IOUtils.toString(encryptionContextProvider.getEncryptionContext(path), - StandardCharsets.UTF_8.name()); - // request -// requestHeaders.add(new AbfsHttpHeader(X_MS_ENCRYPTION_CONTEXT, -// encryptionContext)); - encryptionHeaders = getEncryptionHeaders(path, encryptionContext); - - } else if (encryptionHeaders == null || encryptionHeaders.isEmpty()) { + // get new context for create file request + encryptionContext = encryptionAdapter.getEncryptionContext(path); + requestHeaders.add(new AbfsHttpHeader(X_MS_ENCRYPTION_CONTEXT, + encryptionContext)); + + } else if (encryptionAdapter == null) { encryptionContext = getPathStatus(path, false, tracingContext).getResult().getResponseHeader(X_MS_PROPERTIES); - encryptionHeaders = getEncryptionHeaders(path, encryptionContext); + encryptionAdapter = new EncryptionAdapter(encryptionContextProvider, + path, encryptionContext); } // use cached encryption keys from input/output streams - encodedKey = encryptionHeaders.get(X_MS_ENCRYPTION_KEY); - encodedKeySHA256 = encryptionHeaders.get(X_MS_ENCRYPTION_KEY_SHA256); + encodedKey = org.apache.commons.io.IOUtils.toString( + encryptionAdapter.getEncodedKey(), StandardCharsets.UTF_8.name()); + encodedKeySHA256 = org.apache.commons.io.IOUtils.toString( + encryptionAdapter.getEncodedKeySHA(), StandardCharsets.UTF_8.name()); break; default: return; // no client-provided encryption keys @@ -278,15 +279,17 @@ private void addEncryptionKeyRequestHeaders( SERVER_SIDE_ENCRYPTION_ALGORITHM)); } - public HashMap getEncryptionHeaders(String path, + public EncryptionAdapter getEncryptionAdapter(String path, String encryptionContext) throws IOException { - String key = org.apache.commons.io.IOUtils.toString(encryptionContextProvider.getEncryptionKey(path, - encryptionContext), StandardCharsets.UTF_8.name()); - HashMap encryptionHeaders = new HashMap<>(); - encryptionHeaders.put(X_MS_ENCRYPTION_KEY, getBase64EncodedString(key)); - encryptionHeaders.put(X_MS_ENCRYPTION_KEY_SHA256, - getBase64EncodedString(getSHA256Hash(key))); - return encryptionHeaders; + EncryptionAdapter encryptionAdapter = + new EncryptionAdapter(encryptionContextProvider, path, encryptionContext); + SecretKey key = encryptionAdapter.getEncryptionKey(); + encryptionAdapter.setEncodedKey(getBase64EncodedString(key.getEncoded()) + .getBytes(StandardCharsets.UTF_8)); + encryptionAdapter.setEncodedKeySHA(getBase64EncodedString(getSHA256Hash( + org.apache.commons.io.IOUtils.toString(key.getEncoded(), + StandardCharsets.UTF_8.name()))).getBytes(StandardCharsets.UTF_8)); + return encryptionAdapter; } AbfsUriQueryBuilder createDefaultUriQueryBuilder() { @@ -398,11 +401,11 @@ public AbfsRestOperation deleteFilesystem(TracingContext tracingContext) throws public AbfsRestOperation createPath(final String path, final boolean isFile, final boolean overwrite, final String permission, final String umask, final boolean isAppendBlob, final String eTag, - HashMap encryptionHeaders, TracingContext tracingContext) + EncryptionAdapter encryptionAdapter, TracingContext tracingContext) throws IOException { final List requestHeaders = createDefaultHeaders(); if (isFile) { - addEncryptionKeyRequestHeaders(requestHeaders, path, true, encryptionHeaders, + addEncryptionKeyRequestHeaders(requestHeaders, path, true, encryptionAdapter, tracingContext); } if (!overwrite) { @@ -630,11 +633,11 @@ public AbfsRestOperation renameIdempotencyCheckOp( public AbfsRestOperation append(final String path, final byte[] buffer, AppendRequestParameters reqParams, final String cachedSasToken, - HashMap encryptionHeaders, TracingContext tracingContext) + EncryptionAdapter encryptionAdapter, TracingContext tracingContext) throws IOException { final List requestHeaders = createDefaultHeaders(); addEncryptionKeyRequestHeaders(requestHeaders, path, false, - encryptionHeaders, tracingContext); + encryptionAdapter, tracingContext); // JDK7 does not support PATCH, so to workaround the issue we will use // PUT and specify the real method in the X-Http-Method-Override header. requestHeaders.add(new AbfsHttpHeader(X_HTTP_METHOD_OVERRIDE, @@ -720,11 +723,12 @@ public boolean appendSuccessCheckOp(AbfsRestOperation op, final String path, public AbfsRestOperation flush(final String path, final long position, boolean retainUncommittedData, boolean isClose, - final String cachedSasToken, final String leaseId, HashMap encryptionHeaders, TracingContext tracingContext) throws IOException { + final String cachedSasToken, final String leaseId, + EncryptionAdapter encryptionAdapter, TracingContext tracingContext) + throws IOException { final List requestHeaders = createDefaultHeaders(); addEncryptionKeyRequestHeaders(requestHeaders, path, false, - encryptionHeaders, tracingContext); + encryptionAdapter, tracingContext); // JDK7 does not support PATCH, so to workaround the issue we will use // PUT and specify the real method in the X-Http-Method-Override header. requestHeaders.add(new AbfsHttpHeader(X_HTTP_METHOD_OVERRIDE, @@ -814,11 +818,11 @@ public AbfsRestOperation getPathStatus(final String path, final boolean includeP public AbfsRestOperation read(final String path, final long position, final byte[] buffer, final int bufferOffset, final int bufferLength, final String eTag, - String cachedSasToken, HashMap encryptionHeaders, + String cachedSasToken, EncryptionAdapter encryptionAdapter, TracingContext tracingContext) throws IOException { final List requestHeaders = createDefaultHeaders(); addEncryptionKeyRequestHeaders(requestHeaders, path, false, - encryptionHeaders, tracingContext); + encryptionAdapter, tracingContext); requestHeaders.add(new AbfsHttpHeader(RANGE, String.format("bytes=%d-%d", position, position + bufferLength - 1))); requestHeaders.add(new AbfsHttpHeader(IF_MATCH, eTag)); @@ -1180,6 +1184,10 @@ public AuthType getAuthType() { return authType; } + public EncryptionContextProvider getEncryptionContextProvider() { + return encryptionContextProvider; + } + @VisibleForTesting String initializeUserAgent(final AbfsConfiguration abfsConfiguration, final String sslProviderName) { diff --git a/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/services/AbfsInputStream.java b/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/services/AbfsInputStream.java index f1588b7eed46f..aeb578db89dbc 100644 --- a/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/services/AbfsInputStream.java +++ b/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/services/AbfsInputStream.java @@ -22,10 +22,10 @@ import java.io.FileNotFoundException; import java.io.IOException; import java.net.HttpURLConnection; -import java.util.HashMap; import java.util.UUID; import org.apache.commons.lang3.StringUtils; +import org.apache.hadoop.fs.azurebfs.security.EncryptionAdapter; import org.apache.hadoop.thirdparty.com.google.common.annotations.VisibleForTesting; import org.apache.hadoop.thirdparty.com.google.common.base.Preconditions; @@ -98,7 +98,7 @@ public class AbfsInputStream extends FSInputStream implements CanUnbuffer, // of valid bytes in buffer) private boolean closed = false; private TracingContext tracingContext; - private final HashMap encryptionHeaders; + private final EncryptionAdapter encryptionAdapter; // Optimisations modify the pointer fields. // For better resilience the following fields are used to save the @@ -153,7 +153,7 @@ public AbfsInputStream( this.tracingContext.setStreamID(inputStreamId); this.context = abfsInputStreamContext; readAheadBlockSize = abfsInputStreamContext.getReadAheadBlockSize(); - encryptionHeaders = abfsInputStreamContext.getEncryptionHeaders(); + encryptionAdapter = abfsInputStreamContext.getEncryptionAdapter(); // Propagate the config values to ReadBufferManager so that the first instance // to initialize can set the readAheadBlockSize @@ -546,7 +546,7 @@ int readRemote(long position, byte[] b, int offset, int length, TracingContext t LOG.trace("Trigger client.read for path={} position={} offset={} length={}", path, position, offset, length); op = client.read(path, position, b, offset, length, tolerateOobAppends ? "*" : eTag, cachedSasToken.get(), - encryptionHeaders, tracingContext); + encryptionAdapter, tracingContext); cachedSasToken.update(op.getSasToken()); LOG.debug("issuing HTTP GET request params position = {} b.length = {} " + "offset = {} length = {}", position, b.length, offset, length); diff --git a/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/services/AbfsInputStreamContext.java b/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/services/AbfsInputStreamContext.java index 4c676f52914f0..8397457a97513 100644 --- a/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/services/AbfsInputStreamContext.java +++ b/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/services/AbfsInputStreamContext.java @@ -18,12 +18,11 @@ package org.apache.hadoop.fs.azurebfs.services; +import org.apache.hadoop.fs.azurebfs.security.EncryptionAdapter; import org.slf4j.Logger; import org.slf4j.LoggerFactory; import org.apache.hadoop.thirdparty.com.google.common.base.Preconditions; -import java.util.HashMap; - /** * Class to hold extra input stream configs. */ @@ -51,7 +50,7 @@ public class AbfsInputStreamContext extends AbfsStreamContext { private boolean bufferedPreadDisabled; - private HashMap encryptionHeaders = null; + private EncryptionAdapter encryptionAdapter = null; public AbfsInputStreamContext(final long sasTokenRenewPeriodForStreamsInSeconds) { super(sasTokenRenewPeriodForStreamsInSeconds); @@ -119,8 +118,8 @@ public AbfsInputStreamContext withBufferedPreadDisabled( } public AbfsInputStreamContext withEncryptionHeaders( - HashMap encryptionHeaders) { - this.encryptionHeaders = encryptionHeaders; + EncryptionAdapter encryptionAdapter) { + this.encryptionAdapter = encryptionAdapter; return this; } @@ -179,7 +178,7 @@ public boolean isBufferedPreadDisabled() { return bufferedPreadDisabled; } - public HashMap getEncryptionHeaders() { - return encryptionHeaders; + public EncryptionAdapter getEncryptionAdapter() { + return encryptionAdapter; } } diff --git a/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/services/AbfsOutputStream.java b/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/services/AbfsOutputStream.java index 47c2f69f32d93..05060a84cf973 100644 --- a/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/services/AbfsOutputStream.java +++ b/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/services/AbfsOutputStream.java @@ -33,6 +33,7 @@ import java.util.concurrent.TimeUnit; import java.util.UUID; +import org.apache.hadoop.fs.azurebfs.security.EncryptionAdapter; import org.apache.hadoop.thirdparty.com.google.common.annotations.VisibleForTesting; import org.apache.hadoop.thirdparty.com.google.common.base.Preconditions; @@ -94,7 +95,7 @@ public class AbfsOutputStream extends OutputStream implements Syncable, private ConcurrentLinkedDeque writeOperations; private final ThreadPoolExecutor threadExecutor; private final ExecutorCompletionService completionService; - private final HashMap encryptionHeaders; + private final EncryptionAdapter encryptionAdapter; // SAS tokens can be re-used until they expire private CachedSASToken cachedSasToken; @@ -147,7 +148,7 @@ public AbfsOutputStream( this.numOfAppendsToServerSinceLastFlush = 0; this.writeOperations = new ConcurrentLinkedDeque<>(); this.outputStreamStatistics = abfsOutputStreamContext.getStreamStatistics(); - this.encryptionHeaders = abfsOutputStreamContext.getEncryptionHeaders(); + this.encryptionAdapter = abfsOutputStreamContext.getEncryptionAdapter(); if (this.isAppendBlob) { this.maxConcurrentRequestCount = 1; @@ -416,7 +417,7 @@ private void writeAppendBlobCurrentBufferToService() throws IOException { AppendRequestParameters reqParams = new AppendRequestParameters(offset, 0, bytesLength, APPEND_MODE, true, leaseId); AbfsRestOperation op = client.append(path, bytes, reqParams, - cachedSasToken.get(), encryptionHeaders, + cachedSasToken.get(), encryptionAdapter, new TracingContext(tracingContext)); cachedSasToken.update(op.getSasToken()); if (outputStreamStatistics != null) { @@ -490,7 +491,7 @@ private synchronized void writeCurrentBufferToService(boolean isFlush, boolean i AppendRequestParameters reqParams = new AppendRequestParameters( offset, 0, bytesLength, mode, false, leaseId); AbfsRestOperation op = client.append(path, bytes, reqParams, - cachedSasToken.get(), encryptionHeaders, + cachedSasToken.get(), encryptionAdapter, new TracingContext(tracingContext)); cachedSasToken.update(op.getSasToken()); perfInfo.registerResult(op.getResult()); @@ -558,7 +559,7 @@ private synchronized void flushWrittenBytesToServiceInternal(final long offset, try (AbfsPerfInfo perfInfo = new AbfsPerfInfo(tracker, "flushWrittenBytesToServiceInternal", "flush")) { AbfsRestOperation op = client.flush(path, offset, retainUncommitedData, - isClose, cachedSasToken.get(), leaseId, encryptionHeaders, + isClose, cachedSasToken.get(), leaseId, encryptionAdapter, new TracingContext(tracingContext)); cachedSasToken.update(op.getSasToken()); perfInfo.registerResult(op.getResult()).registerSuccess(true); @@ -681,7 +682,7 @@ public IOStatistics getIOStatistics() { @VisibleForTesting public boolean isEncryptionHeadersCached() { - return encryptionHeaders != null; + return encryptionAdapter != null; } @VisibleForTesting diff --git a/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/services/AbfsOutputStreamContext.java b/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/services/AbfsOutputStreamContext.java index 87d2006c6b2b8..72df892dc7f4e 100644 --- a/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/services/AbfsOutputStreamContext.java +++ b/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/services/AbfsOutputStreamContext.java @@ -18,7 +18,7 @@ package org.apache.hadoop.fs.azurebfs.services; -import java.util.HashMap; +import org.apache.hadoop.fs.azurebfs.security.EncryptionAdapter; /** * Class to hold extra output stream configs. @@ -43,7 +43,7 @@ public class AbfsOutputStreamContext extends AbfsStreamContext { private AbfsLease lease; - private HashMap encryptionHeaders; + private EncryptionAdapter encryptionAdapter; public AbfsOutputStreamContext(final long sasTokenRenewPeriodForStreamsInSeconds) { super(sasTokenRenewPeriodForStreamsInSeconds); @@ -105,9 +105,9 @@ public AbfsOutputStreamContext withLease(final AbfsLease lease) { return this; } - public AbfsOutputStreamContext withEncryptionHeaders( - final HashMap encryptionHeaders) { - this.encryptionHeaders = encryptionHeaders; + public AbfsOutputStreamContext withEncryptionAdapter( + final EncryptionAdapter encryptionAdapter) { + this.encryptionAdapter = encryptionAdapter; return this; } @@ -154,7 +154,7 @@ public String getLeaseId() { return this.lease.getLeaseID(); } - public HashMap getEncryptionHeaders() { - return encryptionHeaders; + public EncryptionAdapter getEncryptionAdapter() { + return encryptionAdapter; } } diff --git a/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/extensions/MockEncryptionContextProvider.java b/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/extensions/MockEncryptionContextProvider.java index 11c0022a2b3e1..869808762d3dc 100644 --- a/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/extensions/MockEncryptionContextProvider.java +++ b/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/extensions/MockEncryptionContextProvider.java @@ -2,7 +2,7 @@ import org.apache.hadoop.conf.Configuration; -import java.io.ByteArrayInputStream; +import javax.crypto.SecretKey; import java.io.IOException; import java.nio.charset.StandardCharsets; import java.util.HashMap; @@ -19,28 +19,52 @@ public void initialize(Configuration configuration, String accountName, } @Override - public ByteArrayInputStream getEncryptionContext(String path) + public String getEncryptionContext(String path) throws IOException { String newContext = UUID.randomUUID().toString(); pathToContextMap.put(path, newContext); // String key = UUID.randomUUID().toString(); String key = dummyKey; // replace with above once server supports contextToKeyMap.put(newContext, key); - return new ByteArrayInputStream(newContext.getBytes((StandardCharsets.UTF_8))); + return newContext; } @Override - public ByteArrayInputStream getEncryptionKey(String path, + public SecretKey getEncryptionKey(String path, String encryptionContext) throws IOException { if (!encryptionContext.equals(pathToContextMap.get(path))) { throw new IOException("encryption context does not match path"); } - return new ByteArrayInputStream(contextToKeyMap.get(encryptionContext) - .getBytes(StandardCharsets.UTF_8)); + return new Key(encryptionContext); } @Override - public void close() throws IOException { + public void destroy() { } + + class Key implements SecretKey { + + private final byte[] key; + Key(String encryptionContext) { + key = + contextToKeyMap.get(encryptionContext).getBytes(StandardCharsets.UTF_8); + } + @Override + public String getAlgorithm() { + return null; + } + + @Override + public String getFormat() { + return null; + } + + @Override + public byte[] getEncoded() { + return key; + } + } } + + From 7365d82d1b622093ec1e70b844e4053f2d0c80f3 Mon Sep 17 00:00:00 2001 From: sumangala Date: Tue, 21 Sep 2021 23:03:11 +0530 Subject: [PATCH 08/77] move key fetch/compute to client --- .../fs/azurebfs/AzureBlobFileSystemStore.java | 11 ++++--- .../azurebfs/security/EncryptionAdapter.java | 20 ++++++------- .../fs/azurebfs/services/AbfsClient.java | 30 +++++++++---------- .../services/AbfsInputStreamContext.java | 2 +- .../azurebfs/services/AbfsOutputStream.java | 2 +- .../fs/azurebfs/ITestEncryptionContext.java | 18 +++++++++-- 6 files changed, 50 insertions(+), 33 deletions(-) diff --git a/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/AzureBlobFileSystemStore.java b/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/AzureBlobFileSystemStore.java index fd8cb5a139402..f9f4660882000 100644 --- a/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/AzureBlobFileSystemStore.java +++ b/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/AzureBlobFileSystemStore.java @@ -730,7 +730,8 @@ public AbfsInputStream openFileForRead(Path path, throw new IOException( "Encryption context missing from server response"); } - encryptionAdapter = client.getEncryptionAdapter(path.toString(), + encryptionAdapter = new EncryptionAdapter( + client.getEncryptionContextProvider(), path.toString(), encryptionContext); } } @@ -770,7 +771,7 @@ private AbfsInputStreamContext populateAbfsInputStreamContext( abfsConfiguration.shouldReadBufferSizeAlways()) .withReadAheadBlockSize(abfsConfiguration.getReadAheadBlockSize()) .withBufferedPreadDisabled(bufferedPreadDisabled) - .withEncryptionHeaders(encryptionAdapter) + .withEncryptionAdapter(encryptionAdapter) .build(); } @@ -812,14 +813,16 @@ public OutputStream openFileForWrite(final Path path, AbfsLease lease = maybeCreateLease(relativePath, tracingContext); String encryptionContext = op.getResult() .getResponseHeader(HttpHeaderConfigurations.X_MS_PROPERTIES); + EncryptionAdapter encryptionAdapter = new EncryptionAdapter( + client.getEncryptionContextProvider(), path.toString(), + encryptionContext); return new AbfsOutputStream( client, statistics, relativePath, offset, - populateAbfsOutputStreamContext(isAppendBlob, lease, - client.getEncryptionAdapter(path.toString(), encryptionContext)), + populateAbfsOutputStreamContext(isAppendBlob, lease, encryptionAdapter), tracingContext); } } diff --git a/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/security/EncryptionAdapter.java b/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/security/EncryptionAdapter.java index ddad9c68182b1..c2ca22e507236 100644 --- a/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/security/EncryptionAdapter.java +++ b/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/security/EncryptionAdapter.java @@ -22,28 +22,28 @@ public class EncryptionAdapter implements Destroyable { public EncryptionAdapter(EncryptionContextProvider provider, String path, String encryptionContext) throws IOException { - this.provider = provider; - this.path = path; + this(provider, path); Preconditions.checkNotNull(encryptionContext, "Encryption context should not be null."); this.encryptionContext = encryptionContext; - this.encryptionKey = provider.getEncryptionKey(path, encryptionContext); } public EncryptionAdapter(EncryptionContextProvider provider, String path) throws IOException { - this(provider, path, provider.getEncryptionContext(path)); + this.provider = provider; + this.path = path; } public SecretKey getEncryptionKey() throws IOException { - return encryptionKey; + if (encryptionKey != null) { + return encryptionKey; + } + return provider.getEncryptionKey(path, encryptionContext); } - public String getEncryptionContext(String path) throws IOException { - if (encryptionContext != null) { - return encryptionContext; - } - return provider.getEncryptionContext(path); + public String fetchEncryptionContext() throws IOException { + encryptionContext = provider.getEncryptionContext(path); + return encryptionContext; } public byte[] getEncodedKey() { diff --git a/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/services/AbfsClient.java b/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/services/AbfsClient.java index 2554add32e69a..e83cc1584dc15 100644 --- a/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/services/AbfsClient.java +++ b/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/services/AbfsClient.java @@ -38,6 +38,7 @@ import java.util.concurrent.ThreadFactory; import java.util.concurrent.TimeUnit; +import org.apache.commons.io.IOUtils; import org.apache.hadoop.fs.azurebfs.extensions.EncryptionContextProvider; import org.apache.hadoop.fs.azurebfs.security.EncryptionAdapter; import org.apache.hadoop.fs.azurebfs.utils.EncryptionType; @@ -67,7 +68,6 @@ import org.apache.hadoop.fs.azurebfs.oauth2.AccessTokenProvider; import org.apache.hadoop.fs.azurebfs.utils.DateTimeUtils; import org.apache.hadoop.fs.azurebfs.utils.TracingContext; -import org.apache.hadoop.io.IOUtils; import org.apache.hadoop.security.ssl.DelegatingSSLSocketFactory; import org.apache.hadoop.util.concurrent.HadoopExecutors; @@ -200,7 +200,8 @@ private String getBase64EncodedString(byte[] bytes) { @Override public void close() throws IOException { if (tokenProvider instanceof Closeable) { - IOUtils.cleanupWithLogger(LOG, (Closeable) tokenProvider); + org.apache.hadoop.io.IOUtils.cleanupWithLogger(LOG, + (Closeable) tokenProvider); } HadoopExecutors.shutdown(executorService, LOG, 0, TimeUnit.SECONDS); } @@ -254,7 +255,8 @@ private void addEncryptionKeyRequestHeaders( case ENCRYPTION_CONTEXT: if (isCreateFileRequest) { // get new context for create file request - encryptionContext = encryptionAdapter.getEncryptionContext(path); + encryptionContext = encryptionAdapter.fetchEncryptionContext(); + computeKeys(encryptionAdapter); requestHeaders.add(new AbfsHttpHeader(X_MS_ENCRYPTION_CONTEXT, encryptionContext)); @@ -263,12 +265,13 @@ private void addEncryptionKeyRequestHeaders( tracingContext).getResult().getResponseHeader(X_MS_PROPERTIES); encryptionAdapter = new EncryptionAdapter(encryptionContextProvider, path, encryptionContext); + computeKeys(encryptionAdapter); } // use cached encryption keys from input/output streams - encodedKey = org.apache.commons.io.IOUtils.toString( - encryptionAdapter.getEncodedKey(), StandardCharsets.UTF_8.name()); - encodedKeySHA256 = org.apache.commons.io.IOUtils.toString( - encryptionAdapter.getEncodedKeySHA(), StandardCharsets.UTF_8.name()); + encodedKey = IOUtils.toString(encryptionAdapter.getEncodedKey(), + StandardCharsets.UTF_8.name()); + encodedKeySHA256 = IOUtils.toString(encryptionAdapter.getEncodedKeySHA(), + StandardCharsets.UTF_8.name()); break; default: return; // no client-provided encryption keys @@ -279,17 +282,14 @@ private void addEncryptionKeyRequestHeaders( SERVER_SIDE_ENCRYPTION_ALGORITHM)); } - public EncryptionAdapter getEncryptionAdapter(String path, - String encryptionContext) throws IOException { - EncryptionAdapter encryptionAdapter = - new EncryptionAdapter(encryptionContextProvider, path, encryptionContext); + public void computeKeys(EncryptionAdapter encryptionAdapter) throws IOException { SecretKey key = encryptionAdapter.getEncryptionKey(); - encryptionAdapter.setEncodedKey(getBase64EncodedString(key.getEncoded()) - .getBytes(StandardCharsets.UTF_8)); + encryptionAdapter.setEncodedKey( + getBase64EncodedString(key.getEncoded()).getBytes( + StandardCharsets.UTF_8)); encryptionAdapter.setEncodedKeySHA(getBase64EncodedString(getSHA256Hash( - org.apache.commons.io.IOUtils.toString(key.getEncoded(), + IOUtils.toString(key.getEncoded(), StandardCharsets.UTF_8.name()))).getBytes(StandardCharsets.UTF_8)); - return encryptionAdapter; } AbfsUriQueryBuilder createDefaultUriQueryBuilder() { diff --git a/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/services/AbfsInputStreamContext.java b/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/services/AbfsInputStreamContext.java index 8397457a97513..e4fad93650645 100644 --- a/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/services/AbfsInputStreamContext.java +++ b/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/services/AbfsInputStreamContext.java @@ -117,7 +117,7 @@ public AbfsInputStreamContext withBufferedPreadDisabled( return this; } - public AbfsInputStreamContext withEncryptionHeaders( + public AbfsInputStreamContext withEncryptionAdapter( EncryptionAdapter encryptionAdapter) { this.encryptionAdapter = encryptionAdapter; return this; diff --git a/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/services/AbfsOutputStream.java b/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/services/AbfsOutputStream.java index 05060a84cf973..6731464f97d46 100644 --- a/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/services/AbfsOutputStream.java +++ b/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/services/AbfsOutputStream.java @@ -681,7 +681,7 @@ public IOStatistics getIOStatistics() { } @VisibleForTesting - public boolean isEncryptionHeadersCached() { + public boolean isEncryptionAdapterCached() { return encryptionAdapter != null; } diff --git a/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/ITestEncryptionContext.java b/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/ITestEncryptionContext.java index efaf269f58922..0ae05f408d462 100644 --- a/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/ITestEncryptionContext.java +++ b/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/ITestEncryptionContext.java @@ -6,12 +6,15 @@ import org.apache.hadoop.fs.azurebfs.extensions.MockDelegationSASTokenProvider; import org.apache.hadoop.fs.azurebfs.extensions.MockEncryptionContextProvider; import org.apache.hadoop.fs.azurebfs.services.AbfsOutputStream; +import org.apache.hadoop.test.LambdaTestUtils; import org.assertj.core.api.Assertions; import org.junit.Assume; import org.junit.Test; +import java.io.IOException; + import static org.apache.hadoop.fs.azurebfs.constants.ConfigurationKeys.FS_AZURE_ENCRYPTION_CONTEXT_PROVIDER_TYPE; -import static org.apache.hadoop.fs.azurebfs.constants.ConfigurationKeys.FS_AZURE_SAS_TOKEN_PROVIDER_TYPE; +import static org.assertj.core.api.ErrorCollector.intercept; public class ITestEncryptionContext extends AbstractAbfsIntegrationTest { public ITestEncryptionContext() throws Exception { @@ -34,6 +37,17 @@ public void testCreateFile() throws Exception { Path testPath = path("createTest"); FSDataOutputStream out = fs.create(testPath); Assertions.assertThat(((AbfsOutputStream) out.getWrappedStream()) - .isEncryptionHeadersCached()).isTrue(); + .isEncryptionAdapterCached()).isTrue(); + + Configuration conf = getRawConfiguration(); + conf.unset(FS_AZURE_ENCRYPTION_CONTEXT_PROVIDER_TYPE); + fs = getFileSystem(conf); + AzureBlobFileSystem finalFs = fs; + LambdaTestUtils.intercept(IOException.class, () -> finalFs.append(testPath)); + LambdaTestUtils.intercept(IOException.class, () -> finalFs.open(testPath)); + LambdaTestUtils.intercept(IOException.class, () -> finalFs.append(testPath)); + LambdaTestUtils.intercept(IOException.class, + () -> finalFs.setXAttr(testPath, "newAttr", new byte[]{1})); + } } From ffb4764de3292d3e4861496315e77f92e763690e Mon Sep 17 00:00:00 2001 From: sumangala Date: Thu, 23 Sep 2021 00:13:32 +0530 Subject: [PATCH 09/77] enc-cxt - SecretKey, fetch key object from provider --- .../hadoop/fs/azurebfs/AbfsConfiguration.java | 2 +- .../fs/azurebfs/AzureBlobFileSystem.java | 5 ++- .../fs/azurebfs/AzureBlobFileSystemStore.java | 43 ++++++++++++------- .../azurebfs/constants/ConfigurationKeys.java | 2 +- .../extensions/EncryptionContextProvider.java | 6 ++- .../azurebfs/security/EncryptionAdapter.java | 6 +-- .../fs/azurebfs/services/AbfsClient.java | 39 ++++++++++------- .../fs/azurebfs/services/AbfsInputStream.java | 42 ++++++++++++------ .../fs/azurebfs/ITestCustomerProvidedKey.java | 40 ++++++++--------- .../MockEncryptionContextProvider.java | 22 +++++----- 10 files changed, 125 insertions(+), 82 deletions(-) diff --git a/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/AbfsConfiguration.java b/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/AbfsConfiguration.java index d73750624e097..14c3af2bb09bf 100644 --- a/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/AbfsConfiguration.java +++ b/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/AbfsConfiguration.java @@ -1040,7 +1040,7 @@ public boolean enableAbfsListIterator() { } public String getClientProvidedEncryptionKey() { - String accSpecEncKey = accountConf(FS_AZURE_CLIENT_PROVIDED_ENCRYPTION_KEY); + String accSpecEncKey = accountConf(FS_AZURE_ENCRYPTION_CLIENT_PROVIDED_KEY); return rawConfig.get(accSpecEncKey, null); } diff --git a/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/AzureBlobFileSystem.java b/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/AzureBlobFileSystem.java index 91274289f5415..c2fc0ee8f14c6 100644 --- a/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/AzureBlobFileSystem.java +++ b/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/AzureBlobFileSystem.java @@ -98,6 +98,8 @@ import org.apache.hadoop.util.LambdaUtils; import org.apache.hadoop.util.Progressable; +import javax.security.auth.DestroyFailedException; + import static org.apache.hadoop.fs.CommonConfigurationKeys.IOSTATISTICS_LOGGING_LEVEL; import static org.apache.hadoop.fs.CommonConfigurationKeys.IOSTATISTICS_LOGGING_LEVEL_DEFAULT; import static org.apache.hadoop.fs.azurebfs.AbfsStatistic.*; @@ -208,7 +210,8 @@ public FSDataInputStream open(final Path path, final int bufferSize) throws IOEx } private FSDataInputStream open(final Path path, - final Optional parameters) throws IOException { + final Optional parameters) + throws IOException, DestroyFailedException { statIncrement(CALL_OPEN); Path qualifiedPath = makeQualified(path); diff --git a/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/AzureBlobFileSystemStore.java b/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/AzureBlobFileSystemStore.java index f9f4660882000..8be933d427325 100644 --- a/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/AzureBlobFileSystemStore.java +++ b/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/AzureBlobFileSystemStore.java @@ -127,6 +127,9 @@ import org.apache.hadoop.security.UserGroupInformation; import org.apache.http.client.utils.URIBuilder; +import javax.crypto.SecretKey; +import javax.security.auth.DestroyFailedException; + import static org.apache.hadoop.fs.azurebfs.constants.AbfsHttpConstants.CHAR_EQUALS; import static org.apache.hadoop.fs.azurebfs.constants.AbfsHttpConstants.CHAR_FORWARD_SLASH; import static org.apache.hadoop.fs.azurebfs.constants.AbfsHttpConstants.CHAR_HYPHEN; @@ -433,7 +436,8 @@ public void setFilesystemProperties( } public Hashtable getPathStatus(final Path path, - TracingContext tracingContext) throws IOException { + TracingContext tracingContext) throws IOException, + DestroyFailedException { try (AbfsPerfInfo perfInfo = startTracking("getPathStatus", "getPathStatus")){ LOG.debug("getPathStatus for filesystem: {} path: {}", client.getFileSystem(), @@ -456,7 +460,7 @@ public Hashtable getPathStatus(final Path path, public void setPathProperties(final Path path, final Hashtable properties, TracingContext tracingContext) - throws IOException { + throws IOException, DestroyFailedException { try (AbfsPerfInfo perfInfo = startTracking("setPathProperties", "setPathProperties")){ LOG.debug("setFilesystemProperties for filesystem: {} path: {} with properties: {}", client.getFileSystem(), @@ -501,7 +505,8 @@ public void deleteFilesystem(TracingContext tracingContext) public OutputStream createFile(final Path path, final FileSystem.Statistics statistics, final boolean overwrite, final FsPermission permission, final FsPermission umask, - TracingContext tracingContext) throws IOException { + TracingContext tracingContext) throws IOException, + DestroyFailedException { try (AbfsPerfInfo perfInfo = startTracking("createFile", "createPath")) { boolean isNamespaceEnabled = getIsNamespaceEnabled(tracingContext); LOG.debug("createFile filesystem: {} path: {} overwrite: {} permission: {} umask: {} isNamespaceEnabled: {}", @@ -582,7 +587,8 @@ private AbfsRestOperation conditionalCreateOverwriteFile(final String relativePa final String umask, final boolean isAppendBlob, EncryptionAdapter encryptionAdapter, - TracingContext tracingContext) throws IOException { + TracingContext tracingContext) throws IOException, + DestroyFailedException { AbfsRestOperation op; try { @@ -658,7 +664,7 @@ private AbfsOutputStreamContext populateAbfsOutputStreamContext(boolean isAppend public void createDirectory(final Path path, final FsPermission permission, final FsPermission umask, TracingContext tracingContext) - throws IOException { + throws IOException, DestroyFailedException { try (AbfsPerfInfo perfInfo = startTracking("createDirectory", "createPath")) { boolean isNamespaceEnabled = getIsNamespaceEnabled(tracingContext); LOG.debug("createDirectory filesystem: {} path: {} permission: {} umask: {} isNamespaceEnabled: {}", @@ -681,7 +687,7 @@ public void createDirectory(final Path path, final FsPermission permission, public AbfsInputStream openFileForRead(final Path path, final FileSystem.Statistics statistics, TracingContext tracingContext) - throws IOException { + throws IOException, DestroyFailedException { return openFileForRead(path, Optional.empty(), statistics, tracingContext); } @@ -689,7 +695,7 @@ public AbfsInputStream openFileForRead(final Path path, public AbfsInputStream openFileForRead(Path path, final Optional parameters, final FileSystem.Statistics statistics, TracingContext tracingContext) - throws IOException { + throws IOException, DestroyFailedException { try (AbfsPerfInfo perfInfo = startTracking("openFileForRead", "getPathStatus")) { LOG.debug("openFileForRead filesystem: {} path: {}", @@ -698,7 +704,8 @@ public AbfsInputStream openFileForRead(Path path, FileStatus fileStatus = parameters.map(OpenFileParameters::getStatus) .orElse(null); String relativePath = getRelativePath(path); - String resourceType, eTag, encryptionContext; + String resourceType, eTag; + SecretKey encryptionContext; long contentLength; EncryptionAdapter encryptionAdapter = null; if (fileStatus instanceof VersionedFileStatus @@ -724,8 +731,10 @@ public AbfsInputStream openFileForRead(Path path, op.getResponseHeader(HttpHeaderConfigurations.CONTENT_LENGTH)); eTag = op.getResponseHeader(HttpHeaderConfigurations.ETAG); if (client.getEncryptionType() == EncryptionType.ENCRYPTION_CONTEXT) { - encryptionContext = op.getResponseHeader( - HttpHeaderConfigurations.X_MS_PROPERTIES); + encryptionContext = + client.getEncryptionContextProvider().getSecretKeyFromBytes(op.getResponseHeader( + HttpHeaderConfigurations.X_MS_PROPERTIES).getBytes( + StandardCharsets.UTF_8)); if (encryptionContext == null) { throw new IOException( "Encryption context missing from server response"); @@ -777,7 +786,8 @@ private AbfsInputStreamContext populateAbfsInputStreamContext( public OutputStream openFileForWrite(final Path path, final FileSystem.Statistics statistics, final boolean overwrite, - TracingContext tracingContext) throws IOException { + TracingContext tracingContext) throws IOException, + DestroyFailedException { try (AbfsPerfInfo perfInfo = startTracking("openFileForWrite", "getPathStatus")) { LOG.debug("openFileForWrite filesystem: {} path: {} overwrite: {}", client.getFileSystem(), @@ -811,8 +821,10 @@ public OutputStream openFileForWrite(final Path path, } AbfsLease lease = maybeCreateLease(relativePath, tracingContext); - String encryptionContext = op.getResult() - .getResponseHeader(HttpHeaderConfigurations.X_MS_PROPERTIES); + SecretKey encryptionContext = + client.getEncryptionContextProvider().getSecretKeyFromBytes(op.getResult() + .getResponseHeader(HttpHeaderConfigurations.X_MS_PROPERTIES).getBytes( + StandardCharsets.UTF_8)); EncryptionAdapter encryptionAdapter = new EncryptionAdapter( client.getEncryptionContextProvider(), path.toString(), encryptionContext); @@ -841,7 +853,7 @@ public void breakLease(final Path path, final TracingContext tracingContext) thr } public void rename(final Path source, final Path destination, TracingContext tracingContext) - throws IOException { + throws IOException, DestroyFailedException { final Instant startAggregate = abfsPerfTracker.getLatencyInstant(); long countAggregate = 0; boolean shouldContinue; @@ -912,7 +924,8 @@ public void delete(final Path path, final boolean recursive, } public FileStatus getFileStatus(final Path path, - TracingContext tracingContext) throws IOException { + TracingContext tracingContext) throws IOException, + DestroyFailedException { try (AbfsPerfInfo perfInfo = startTracking("getFileStatus", "undetermined")) { boolean isNamespaceEnabled = getIsNamespaceEnabled(tracingContext); LOG.debug("getFileStatus filesystem: {} path: {} isNamespaceEnabled: {}", diff --git a/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/constants/ConfigurationKeys.java b/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/constants/ConfigurationKeys.java index 668e91dd3fc56..1d1059c4556e3 100644 --- a/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/constants/ConfigurationKeys.java +++ b/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/constants/ConfigurationKeys.java @@ -158,7 +158,7 @@ public final class ConfigurationKeys { /** Setting this true will make the driver use it's own RemoteIterator implementation */ public static final String FS_AZURE_ENABLE_ABFS_LIST_ITERATOR = "fs.azure.enable.abfslistiterator"; /** Server side encryption key */ - public static final String FS_AZURE_CLIENT_PROVIDED_ENCRYPTION_KEY = "fs.azure.client-provided-encryption-key"; + public static final String FS_AZURE_ENCRYPTION_CLIENT_PROVIDED_KEY = "fs.azure.encryption.client-provided-key"; /** Custom EncryptionContextProvider type */ public static final String FS_AZURE_ENCRYPTION_CONTEXT_PROVIDER_TYPE = "fs.azure.encryption.context.provider.type"; public static final String FS_AZURE_ENCRYPTION_MODE = "fs.azure.encryption.mode"; diff --git a/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/extensions/EncryptionContextProvider.java b/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/extensions/EncryptionContextProvider.java index 55344e86b3de4..93c2f2dc08fa5 100644 --- a/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/extensions/EncryptionContextProvider.java +++ b/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/extensions/EncryptionContextProvider.java @@ -41,7 +41,7 @@ public interface EncryptionContextProvider extends Destroyable { * @return encryptionContext string * @throws IOException */ - String getEncryptionContext(String path) throws IOException; + SecretKey getEncryptionContext(String path) throws IOException; /** * Fetch encryption key in-exchange for encryption context @@ -51,7 +51,9 @@ public interface EncryptionContextProvider extends Destroyable { * @return Encryption key * @throws IOException */ - SecretKey getEncryptionKey(String path, String encryptionContext) throws IOException; + SecretKey getEncryptionKey(String path, SecretKey encryptionContext) throws IOException; + + SecretKey getSecretKeyFromBytes(byte[] secret); void destroy(); } diff --git a/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/security/EncryptionAdapter.java b/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/security/EncryptionAdapter.java index c2ca22e507236..b22f8ed3504f6 100644 --- a/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/security/EncryptionAdapter.java +++ b/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/security/EncryptionAdapter.java @@ -12,7 +12,7 @@ public class EncryptionAdapter implements Destroyable { String path; - String encryptionContext; + SecretKey encryptionContext; SecretKey encryptionKey; EncryptionContextProvider provider; byte[] encodedKey = null; @@ -21,7 +21,7 @@ public class EncryptionAdapter implements Destroyable { LoggerFactory.getLogger(EncryptionAdapter.class); public EncryptionAdapter(EncryptionContextProvider provider, String path, - String encryptionContext) throws IOException { + SecretKey encryptionContext) throws IOException { this(provider, path); Preconditions.checkNotNull(encryptionContext, "Encryption context should not be null."); @@ -41,7 +41,7 @@ public SecretKey getEncryptionKey() throws IOException { return provider.getEncryptionKey(path, encryptionContext); } - public String fetchEncryptionContext() throws IOException { + public SecretKey fetchEncryptionContext() throws IOException { encryptionContext = provider.getEncryptionContext(path); return encryptionContext; } diff --git a/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/services/AbfsClient.java b/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/services/AbfsClient.java index e83cc1584dc15..f13e1f399f6ab 100644 --- a/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/services/AbfsClient.java +++ b/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/services/AbfsClient.java @@ -72,6 +72,7 @@ import org.apache.hadoop.util.concurrent.HadoopExecutors; import javax.crypto.SecretKey; +import javax.security.auth.DestroyFailedException; import static org.apache.hadoop.fs.azurebfs.constants.AbfsHttpConstants.*; import static org.apache.hadoop.fs.azurebfs.constants.FileSystemConfigurations.DEFAULT_DELETE_CONSIDERED_IDEMPOTENT; @@ -245,8 +246,8 @@ private void addEncryptionKeyRequestHeaders( List requestHeaders, String path, boolean isCreateFileRequest, EncryptionAdapter encryptionAdapter, TracingContext tracingContext) - throws IOException { - String encodedKey, encodedKeySHA256, encryptionContext; + throws IOException, DestroyFailedException { + String encodedKey, encodedKeySHA256; switch (encryptionType) { case GLOBAL_KEY: encodedKey = clientProvidedEncryptionKey; @@ -255,17 +256,21 @@ private void addEncryptionKeyRequestHeaders( case ENCRYPTION_CONTEXT: if (isCreateFileRequest) { // get new context for create file request - encryptionContext = encryptionAdapter.fetchEncryptionContext(); + SecretKey encryptionContext = + encryptionAdapter.fetchEncryptionContext(); computeKeys(encryptionAdapter); requestHeaders.add(new AbfsHttpHeader(X_MS_ENCRYPTION_CONTEXT, - encryptionContext)); + IOUtils.toString(encryptionContext.getEncoded(), + StandardCharsets.UTF_8.name()))); + encryptionContext.destroy(); } else if (encryptionAdapter == null) { - encryptionContext = getPathStatus(path, false, - tracingContext).getResult().getResponseHeader(X_MS_PROPERTIES); + SecretKey encryptionContext = encryptionContextProvider.getSecretKeyFromBytes( + getPathStatus(path, false, tracingContext).getResult() + .getResponseHeader(X_MS_PROPERTIES) + .getBytes(StandardCharsets.UTF_8)); encryptionAdapter = new EncryptionAdapter(encryptionContextProvider, path, encryptionContext); - computeKeys(encryptionAdapter); } // use cached encryption keys from input/output streams encodedKey = IOUtils.toString(encryptionAdapter.getEncodedKey(), @@ -402,7 +407,7 @@ public AbfsRestOperation createPath(final String path, final boolean isFile, final boolean overwrite, final String permission, final String umask, final boolean isAppendBlob, final String eTag, EncryptionAdapter encryptionAdapter, TracingContext tracingContext) - throws IOException { + throws IOException, DestroyFailedException { final List requestHeaders = createDefaultHeaders(); if (isFile) { addEncryptionKeyRequestHeaders(requestHeaders, path, true, encryptionAdapter, @@ -539,7 +544,7 @@ public AbfsRestOperation breakLease(final String path, public AbfsRestOperation renamePath(String source, final String destination, final String continuation, TracingContext tracingContext) - throws IOException { + throws IOException, DestroyFailedException { final List requestHeaders = createDefaultHeaders(); String encodedRenameSource = urlEncode(FORWARD_SLASH + this.getFileSystem() + source); @@ -602,7 +607,8 @@ public AbfsRestOperation renameIdempotencyCheckOp( final Instant renameRequestStartTime, final AbfsRestOperation op, final String destination, - TracingContext tracingContext) throws IOException { + TracingContext tracingContext) throws IOException, + DestroyFailedException { if ((op.isARetriedRequest()) && (op.getResult().getStatusCode() == HttpURLConnection.HTTP_NOT_FOUND)) { // Server has returned HTTP 404, which means rename source no longer @@ -634,7 +640,7 @@ public AbfsRestOperation renameIdempotencyCheckOp( public AbfsRestOperation append(final String path, final byte[] buffer, AppendRequestParameters reqParams, final String cachedSasToken, EncryptionAdapter encryptionAdapter, TracingContext tracingContext) - throws IOException { + throws IOException, DestroyFailedException { final List requestHeaders = createDefaultHeaders(); addEncryptionKeyRequestHeaders(requestHeaders, path, false, encryptionAdapter, tracingContext); @@ -705,7 +711,7 @@ && appendSuccessCheckOp(op, path, // in case we are doing a retry after checking the length of the file public boolean appendSuccessCheckOp(AbfsRestOperation op, final String path, final long length, TracingContext tracingContext) - throws IOException { + throws IOException, DestroyFailedException { if ((op.isARetriedRequest()) && (op.getResult().getStatusCode() == HttpURLConnection.HTTP_BAD_REQUEST)) { final AbfsRestOperation destStatusOp = getPathStatus(path, false, tracingContext); @@ -725,7 +731,7 @@ public AbfsRestOperation flush(final String path, final long position, boolean retainUncommittedData, boolean isClose, final String cachedSasToken, final String leaseId, EncryptionAdapter encryptionAdapter, TracingContext tracingContext) - throws IOException { + throws IOException, DestroyFailedException { final List requestHeaders = createDefaultHeaders(); addEncryptionKeyRequestHeaders(requestHeaders, path, false, encryptionAdapter, tracingContext); @@ -759,7 +765,7 @@ public AbfsRestOperation flush(final String path, final long position, public AbfsRestOperation setPathProperties(final String path, final String properties, TracingContext tracingContext) - throws IOException { + throws IOException, DestroyFailedException { final List requestHeaders = createDefaultHeaders(); addEncryptionKeyRequestHeaders(requestHeaders, path, false, null, tracingContext); @@ -787,7 +793,7 @@ public AbfsRestOperation setPathProperties(final String path, final String prope public AbfsRestOperation getPathStatus(final String path, final boolean includeProperties, TracingContext tracingContext) - throws IOException { + throws IOException, DestroyFailedException { final List requestHeaders = createDefaultHeaders(); final AbfsUriQueryBuilder abfsUriQueryBuilder = createDefaultUriQueryBuilder(); @@ -819,7 +825,8 @@ public AbfsRestOperation getPathStatus(final String path, final boolean includeP public AbfsRestOperation read(final String path, final long position, final byte[] buffer, final int bufferOffset, final int bufferLength, final String eTag, String cachedSasToken, EncryptionAdapter encryptionAdapter, - TracingContext tracingContext) throws IOException { + TracingContext tracingContext) throws IOException, + DestroyFailedException { final List requestHeaders = createDefaultHeaders(); addEncryptionKeyRequestHeaders(requestHeaders, path, false, encryptionAdapter, tracingContext); diff --git a/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/services/AbfsInputStream.java b/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/services/AbfsInputStream.java index aeb578db89dbc..959a37028a7ca 100644 --- a/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/services/AbfsInputStream.java +++ b/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/services/AbfsInputStream.java @@ -46,6 +46,8 @@ import org.apache.hadoop.fs.statistics.IOStatistics; import org.apache.hadoop.fs.statistics.IOStatisticsSource; +import javax.security.auth.DestroyFailedException; + import static java.lang.Math.max; import static java.lang.Math.min; @@ -196,7 +198,12 @@ public int read(long position, byte[] buffer, int offset, int length) if (streamStatistics != null) { streamStatistics.readOperationStarted(); } - int bytesRead = readRemote(position, buffer, offset, length, tracingContext); + int bytesRead = 0; + try { + bytesRead = readRemote(position, buffer, offset, length, tracingContext); + } catch (DestroyFailedException e) { + LOG.debug(e.getMessage()); + } if (statistics != null) { statistics.incrementBytesRead(bytesRead); } @@ -258,12 +265,17 @@ public synchronized int read(final byte[] b, final int off, final int len) throw limit = 0; bCursor = 0; } - if (shouldReadFully()) { - lastReadBytes = readFileCompletely(b, currentOff, currentLen); - } else if (shouldReadLastBlock()) { - lastReadBytes = readLastBlock(b, currentOff, currentLen); - } else { - lastReadBytes = readOneBlock(b, currentOff, currentLen); + try { + if (shouldReadFully()) { + lastReadBytes = readFileCompletely(b, currentOff, currentLen); + } else if (shouldReadLastBlock()) { + lastReadBytes = readLastBlock(b, currentOff, currentLen); + } else { + lastReadBytes = readOneBlock(b, currentOff, currentLen); + } + } catch (DestroyFailedException e) { + LOG.debug(e.getMessage()); + throw new IOException(e); } if (lastReadBytes > 0) { currentOff += lastReadBytes; @@ -288,7 +300,8 @@ private boolean shouldReadLastBlock() { && this.fCursor >= footerStart; } - private int readOneBlock(final byte[] b, final int off, final int len) throws IOException { + private int readOneBlock(final byte[] b, final int off, final int len) + throws IOException, DestroyFailedException { if (len == 0) { return 0; } @@ -341,7 +354,7 @@ private int readOneBlock(final byte[] b, final int off, final int len) throws IO } private int readFileCompletely(final byte[] b, final int off, final int len) - throws IOException { + throws IOException, DestroyFailedException { if (len == 0) { return 0; } @@ -356,7 +369,7 @@ private int readFileCompletely(final byte[] b, final int off, final int len) } private int readLastBlock(final byte[] b, final int off, final int len) - throws IOException { + throws IOException, DestroyFailedException { if (len == 0) { return 0; } @@ -375,7 +388,8 @@ private int readLastBlock(final byte[] b, final int off, final int len) } private int optimisedRead(final byte[] b, final int off, final int len, - final long readFrom, final long actualLen) throws IOException { + final long readFrom, final long actualLen) + throws IOException, DestroyFailedException { fCursor = readFrom; int totalBytesRead = 0; int lastBytesRead = 0; @@ -469,7 +483,8 @@ private int copyToUserBuffer(byte[] b, int off, int len){ } private int readInternal(final long position, final byte[] b, final int offset, final int length, - final boolean bypassReadAhead) throws IOException { + final boolean bypassReadAhead) + throws IOException, DestroyFailedException { if (readAheadEnabled && !bypassReadAhead) { // try reading from read-ahead if (offset != 0) { @@ -518,7 +533,8 @@ private int readInternal(final long position, final byte[] b, final int offset, } } - int readRemote(long position, byte[] b, int offset, int length, TracingContext tracingContext) throws IOException { + int readRemote(long position, byte[] b, int offset, int length, TracingContext tracingContext) + throws IOException, DestroyFailedException { if (position < 0) { throw new IllegalArgumentException("attempting to read from negative offset"); } diff --git a/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/ITestCustomerProvidedKey.java b/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/ITestCustomerProvidedKey.java index dbd4d0c32b333..b6500aef597df 100644 --- a/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/ITestCustomerProvidedKey.java +++ b/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/ITestCustomerProvidedKey.java @@ -70,7 +70,7 @@ import org.apache.hadoop.test.LambdaTestUtils; import static org.apache.hadoop.fs.azurebfs.constants.ConfigurationKeys.AZURE_CREATE_REMOTE_FILESYSTEM_DURING_INITIALIZATION; -import static org.apache.hadoop.fs.azurebfs.constants.ConfigurationKeys.FS_AZURE_CLIENT_PROVIDED_ENCRYPTION_KEY; +import static org.apache.hadoop.fs.azurebfs.constants.ConfigurationKeys.FS_AZURE_ENCRYPTION_CLIENT_PROVIDED_KEY; import static org.apache.hadoop.fs.azurebfs.constants.FileSystemConfigurations.ONE_MB; import static org.apache.hadoop.fs.azurebfs.constants.HttpHeaderConfigurations.X_MS_ENCRYPTION_ALGORITHM; import static org.apache.hadoop.fs.azurebfs.constants.HttpHeaderConfigurations.X_MS_ENCRYPTION_KEY; @@ -133,7 +133,7 @@ public void testReadWithCPK() throws Exception { // Trying to read with different CPK headers Configuration conf = fs.getConf(); String accountName = conf.get(FS_AZURE_ABFS_ACCOUNT_NAME); - conf.set(FS_AZURE_CLIENT_PROVIDED_ENCRYPTION_KEY + "." + accountName, + conf.set(FS_AZURE_ENCRYPTION_CLIENT_PROVIDED_KEY + "." + accountName, "different-1234567890123456789012"); try (AzureBlobFileSystem fs2 = (AzureBlobFileSystem) FileSystem.newInstance(conf); FSDataInputStream iStream = fs2.open(new Path(fileName))) { @@ -145,7 +145,7 @@ public void testReadWithCPK() throws Exception { } // Trying to read with no CPK headers - conf.unset(FS_AZURE_CLIENT_PROVIDED_ENCRYPTION_KEY + "." + accountName); + conf.unset(FS_AZURE_ENCRYPTION_CLIENT_PROVIDED_KEY + "." + accountName); try (AzureBlobFileSystem fs3 = (AzureBlobFileSystem) FileSystem .get(conf); FSDataInputStream iStream = fs3.open(new Path(fileName))) { int len = 8 * ONE_MB; @@ -183,7 +183,7 @@ public void testReadWithoutCPK() throws Exception { // Trying to read with CPK headers Configuration conf = fs.getConf(); String accountName = conf.get(FS_AZURE_ABFS_ACCOUNT_NAME); - conf.set(FS_AZURE_CLIENT_PROVIDED_ENCRYPTION_KEY + "." + accountName, + conf.set(FS_AZURE_ENCRYPTION_CLIENT_PROVIDED_KEY + "." + accountName, "12345678901234567890123456789012"); try (AzureBlobFileSystem fs2 = (AzureBlobFileSystem) FileSystem.newInstance(conf); @@ -219,7 +219,7 @@ public void testAppendWithCPK() throws Exception { // Trying to append with different CPK headers Configuration conf = fs.getConf(); String accountName = conf.get(FS_AZURE_ABFS_ACCOUNT_NAME); - conf.set(FS_AZURE_CLIENT_PROVIDED_ENCRYPTION_KEY + "." + accountName, + conf.set(FS_AZURE_ENCRYPTION_CLIENT_PROVIDED_KEY + "." + accountName, "different-1234567890123456789012"); try (AzureBlobFileSystem fs2 = (AzureBlobFileSystem) FileSystem.newInstance(conf); AbfsClient abfsClient2 = fs2.getAbfsClient()) { @@ -230,7 +230,7 @@ public void testAppendWithCPK() throws Exception { } // Trying to append with no CPK headers - conf.unset(FS_AZURE_CLIENT_PROVIDED_ENCRYPTION_KEY + "." + accountName); + conf.unset(FS_AZURE_ENCRYPTION_CLIENT_PROVIDED_KEY + "." + accountName); try (AzureBlobFileSystem fs3 = (AzureBlobFileSystem) FileSystem .get(conf); AbfsClient abfsClient3 = fs3.getAbfsClient()) { LambdaTestUtils.intercept(IOException.class, () -> { @@ -265,7 +265,7 @@ public void testAppendWithoutCPK() throws Exception { // Trying to append with CPK headers Configuration conf = fs.getConf(); String accountName = conf.get(FS_AZURE_ABFS_ACCOUNT_NAME); - conf.set(FS_AZURE_CLIENT_PROVIDED_ENCRYPTION_KEY + "." + accountName, + conf.set(FS_AZURE_ENCRYPTION_CLIENT_PROVIDED_KEY + "." + accountName, "12345678901234567890123456789012"); try (AzureBlobFileSystem fs2 = (AzureBlobFileSystem) FileSystem.newInstance(conf); AbfsClient abfsClient2 = fs2.getAbfsClient()) { @@ -337,7 +337,7 @@ public void testCopyBetweenAccounts() throws Exception { conf.unset(FS_AZURE_ABFS_ACCOUNT_NAME); conf.set(FS_AZURE_ABFS_ACCOUNT_NAME, accountName); conf.set(FS_AZURE_ACCOUNT_KEY + "." + accountName, accountKey); - conf.set(FS_AZURE_CLIENT_PROVIDED_ENCRYPTION_KEY + "." + accountName, + conf.set(FS_AZURE_ENCRYPTION_CLIENT_PROVIDED_KEY + "." + accountName, "123456789012345678901234567890ab"); conf.set("fs.defaultFS", "abfs://" + fileSystemName + "@" + accountName); AzureBlobFileSystem fs2 = (AzureBlobFileSystem) FileSystem.newInstance(conf); @@ -359,8 +359,8 @@ public void testCopyBetweenAccounts() throws Exception { } // Trying to read fs2DestFilePath with different CPK headers - conf.unset(FS_AZURE_CLIENT_PROVIDED_ENCRYPTION_KEY + "." + accountName); - conf.set(FS_AZURE_CLIENT_PROVIDED_ENCRYPTION_KEY + "." + accountName, + conf.unset(FS_AZURE_ENCRYPTION_CLIENT_PROVIDED_KEY + "." + accountName); + conf.set(FS_AZURE_ENCRYPTION_CLIENT_PROVIDED_KEY + "." + accountName, "different-1234567890123456789012"); try (AzureBlobFileSystem fs3 = (AzureBlobFileSystem) FileSystem .get(conf); FSDataInputStream iStream = fs3.open(fs2DestFilePath)) { @@ -372,7 +372,7 @@ public void testCopyBetweenAccounts() throws Exception { } // Trying to read fs2DestFilePath with no CPK headers - conf.unset(FS_AZURE_CLIENT_PROVIDED_ENCRYPTION_KEY + "." + accountName); + conf.unset(FS_AZURE_ENCRYPTION_CLIENT_PROVIDED_KEY + "." + accountName); try (AzureBlobFileSystem fs4 = (AzureBlobFileSystem) FileSystem .get(conf); FSDataInputStream iStream = fs4.open(fs2DestFilePath)) { int length = 8 * ONE_MB; @@ -426,7 +426,7 @@ private void testListPath(final boolean isWithCPK) throws Exception { // Trying with different CPK headers Configuration conf = fs.getConf(); String accountName = conf.get(FS_AZURE_ABFS_ACCOUNT_NAME); - conf.set(FS_AZURE_CLIENT_PROVIDED_ENCRYPTION_KEY + "." + accountName, + conf.set(FS_AZURE_ENCRYPTION_CLIENT_PROVIDED_KEY + "." + accountName, "different-1234567890123456789012"); AzureBlobFileSystem fs2 = (AzureBlobFileSystem) FileSystem.newInstance(conf); AbfsClient abfsClient2 = fs2.getAbfsClient(); @@ -437,7 +437,7 @@ private void testListPath(final boolean isWithCPK) throws Exception { if (isWithCPK) { // Trying with no CPK headers - conf.unset(FS_AZURE_CLIENT_PROVIDED_ENCRYPTION_KEY + "." + accountName); + conf.unset(FS_AZURE_ENCRYPTION_CLIENT_PROVIDED_KEY + "." + accountName); AzureBlobFileSystem fs3 = (AzureBlobFileSystem) FileSystem.get(conf); AbfsClient abfsClient3 = fs3.getAbfsClient(); abfsRestOperation = abfsClient3 @@ -566,7 +566,7 @@ private void testFlush(final boolean isWithCPK) throws Exception { // Trying to read with different CPK headers Configuration conf = fs.getConf(); String accountName = conf.get(FS_AZURE_ABFS_ACCOUNT_NAME); - conf.set(FS_AZURE_CLIENT_PROVIDED_ENCRYPTION_KEY + "." + accountName, + conf.set(FS_AZURE_ENCRYPTION_CLIENT_PROVIDED_KEY + "." + accountName, "different-1234567890123456789012"); try (AzureBlobFileSystem fs2 = (AzureBlobFileSystem) FileSystem.newInstance(conf); AbfsClient abfsClient2 = fs2.getAbfsClient()) { @@ -578,7 +578,7 @@ private void testFlush(final boolean isWithCPK) throws Exception { // Trying to read with no CPK headers if (isWithCPK) { - conf.unset(FS_AZURE_CLIENT_PROVIDED_ENCRYPTION_KEY + "." + accountName); + conf.unset(FS_AZURE_ENCRYPTION_CLIENT_PROVIDED_KEY + "." + accountName); try (AzureBlobFileSystem fs3 = (AzureBlobFileSystem) FileSystem .get(conf); AbfsClient abfsClient3 = fs3.getAbfsClient()) { LambdaTestUtils.intercept(IOException.class, () -> { @@ -878,7 +878,7 @@ private String getCPKSha(final AzureBlobFileSystem abfs) throws IOException { Configuration conf = abfs.getConf(); String accountName = conf.get(FS_AZURE_ABFS_ACCOUNT_NAME); String encryptionKey = conf - .get(FS_AZURE_CLIENT_PROVIDED_ENCRYPTION_KEY + "." + accountName); + .get(FS_AZURE_ENCRYPTION_CLIENT_PROVIDED_KEY + "." + accountName); if (encryptionKey == null || encryptionKey.isEmpty()) { return ""; } @@ -945,11 +945,11 @@ private AzureBlobFileSystem getAbfs(boolean withCPK, String cpk) throws IOException { Configuration conf = getRawConfiguration(); if (withCPK) { - conf.set(FS_AZURE_CLIENT_PROVIDED_ENCRYPTION_KEY + "." + getAccountName(), + conf.set(FS_AZURE_ENCRYPTION_CLIENT_PROVIDED_KEY + "." + getAccountName(), cpk); } else { conf.unset( - FS_AZURE_CLIENT_PROVIDED_ENCRYPTION_KEY + "." + getAccountName()); + FS_AZURE_ENCRYPTION_CLIENT_PROVIDED_KEY + "." + getAccountName()); } return (AzureBlobFileSystem) FileSystem.newInstance(conf); } @@ -960,13 +960,13 @@ private AzureBlobFileSystem getSameFSWithWrongCPK( Configuration conf = abfsConf.getRawConfiguration(); String accountName = conf.get(FS_AZURE_ABFS_ACCOUNT_NAME); String cpk = conf - .get(FS_AZURE_CLIENT_PROVIDED_ENCRYPTION_KEY + "." + accountName); + .get(FS_AZURE_ENCRYPTION_CLIENT_PROVIDED_KEY + "." + accountName); if (cpk == null || cpk.isEmpty()) { cpk = "01234567890123456789012345678912"; } cpk = "different-" + cpk; String differentCpk = cpk.substring(0, ENCRYPTION_KEY_LEN - 1); - conf.set(FS_AZURE_CLIENT_PROVIDED_ENCRYPTION_KEY + "." + accountName, + conf.set(FS_AZURE_ENCRYPTION_CLIENT_PROVIDED_KEY + "." + accountName, differentCpk); conf.set("fs.defaultFS", "abfs://" + getFileSystemName() + "@" + accountName); diff --git a/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/extensions/MockEncryptionContextProvider.java b/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/extensions/MockEncryptionContextProvider.java index 869808762d3dc..8ef4328e378d1 100644 --- a/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/extensions/MockEncryptionContextProvider.java +++ b/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/extensions/MockEncryptionContextProvider.java @@ -10,8 +10,8 @@ public class MockEncryptionContextProvider implements EncryptionContextProvider { String dummyKey = UUID.randomUUID().toString(); - HashMap pathToContextMap = new HashMap<>(); - HashMap contextToKeyMap = new HashMap<>(); + HashMap pathToContextMap = new HashMap<>(); + HashMap contextToKeyMap = new HashMap<>(); @Override public void initialize(Configuration configuration, String accountName, String fileSystem) throws IOException { @@ -19,23 +19,25 @@ public void initialize(Configuration configuration, String accountName, } @Override - public String getEncryptionContext(String path) + public SecretKey getEncryptionContext(String path) throws IOException { - String newContext = UUID.randomUUID().toString(); + SecretKey newContext = + new Key(UUID.randomUUID().toString().getBytes(StandardCharsets.UTF_8)); pathToContextMap.put(path, newContext); // String key = UUID.randomUUID().toString(); - String key = dummyKey; // replace with above once server supports + SecretKey key = new Key(dummyKey.getBytes(StandardCharsets.UTF_8)); + // replace with above once server supports contextToKeyMap.put(newContext, key); return newContext; } @Override public SecretKey getEncryptionKey(String path, - String encryptionContext) throws IOException { + SecretKey encryptionContext) throws IOException { if (!encryptionContext.equals(pathToContextMap.get(path))) { throw new IOException("encryption context does not match path"); } - return new Key(encryptionContext); + return contextToKeyMap.get(encryptionContext); } @Override @@ -46,9 +48,9 @@ public void destroy() { class Key implements SecretKey { private final byte[] key; - Key(String encryptionContext) { - key = - contextToKeyMap.get(encryptionContext).getBytes(StandardCharsets.UTF_8); + + Key(byte[] secret) { + key = secret; } @Override public String getAlgorithm() { From 31f4c0f814fa86badd7863c1869d99a58fd3b623 Mon Sep 17 00:00:00 2001 From: sumangala Date: Fri, 1 Oct 2021 18:12:59 +0530 Subject: [PATCH 10/77] refactor, move cpk to adapter, abfsKey cxt --- .../fs/azurebfs/AzureBlobFileSystem.java | 4 +- .../fs/azurebfs/AzureBlobFileSystemStore.java | 44 +++----- .../extensions/EncryptionContextProvider.java | 2 - .../azurebfs/security/EncryptionAdapter.java | 73 ++++++++++-- .../fs/azurebfs/services/AbfsClient.java | 105 ++++++------------ .../fs/azurebfs/services/AbfsInputStream.java | 8 +- .../azurebfs/services/AbfsOutputStream.java | 2 + .../azurebfs/AbstractAbfsIntegrationTest.java | 2 + 8 files changed, 120 insertions(+), 120 deletions(-) diff --git a/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/AzureBlobFileSystem.java b/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/AzureBlobFileSystem.java index c2fc0ee8f14c6..ec0b7088ef9ab 100644 --- a/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/AzureBlobFileSystem.java +++ b/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/AzureBlobFileSystem.java @@ -206,12 +206,12 @@ public void registerListener(Listener listener1) { public FSDataInputStream open(final Path path, final int bufferSize) throws IOException { LOG.debug("AzureBlobFileSystem.open path: {} bufferSize: {}", path, bufferSize); // bufferSize is unused. - return open(path, Optional.empty()); + return open(path, Optional.empty()); } private FSDataInputStream open(final Path path, final Optional parameters) - throws IOException, DestroyFailedException { + throws IOException { statIncrement(CALL_OPEN); Path qualifiedPath = makeQualified(path); diff --git a/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/AzureBlobFileSystemStore.java b/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/AzureBlobFileSystemStore.java index 8be933d427325..816eeae549eb4 100644 --- a/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/AzureBlobFileSystemStore.java +++ b/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/AzureBlobFileSystemStore.java @@ -127,7 +127,6 @@ import org.apache.hadoop.security.UserGroupInformation; import org.apache.http.client.utils.URIBuilder; -import javax.crypto.SecretKey; import javax.security.auth.DestroyFailedException; import static org.apache.hadoop.fs.azurebfs.constants.AbfsHttpConstants.CHAR_EQUALS; @@ -436,8 +435,7 @@ public void setFilesystemProperties( } public Hashtable getPathStatus(final Path path, - TracingContext tracingContext) throws IOException, - DestroyFailedException { + TracingContext tracingContext) throws IOException { try (AbfsPerfInfo perfInfo = startTracking("getPathStatus", "getPathStatus")){ LOG.debug("getPathStatus for filesystem: {} path: {}", client.getFileSystem(), @@ -460,7 +458,7 @@ public Hashtable getPathStatus(final Path path, public void setPathProperties(final Path path, final Hashtable properties, TracingContext tracingContext) - throws IOException, DestroyFailedException { + throws IOException { try (AbfsPerfInfo perfInfo = startTracking("setPathProperties", "setPathProperties")){ LOG.debug("setFilesystemProperties for filesystem: {} path: {} with properties: {}", client.getFileSystem(), @@ -505,8 +503,7 @@ public void deleteFilesystem(TracingContext tracingContext) public OutputStream createFile(final Path path, final FileSystem.Statistics statistics, final boolean overwrite, final FsPermission permission, final FsPermission umask, - TracingContext tracingContext) throws IOException, - DestroyFailedException { + TracingContext tracingContext) throws IOException { try (AbfsPerfInfo perfInfo = startTracking("createFile", "createPath")) { boolean isNamespaceEnabled = getIsNamespaceEnabled(tracingContext); LOG.debug("createFile filesystem: {} path: {} overwrite: {} permission: {} umask: {} isNamespaceEnabled: {}", @@ -587,8 +584,7 @@ private AbfsRestOperation conditionalCreateOverwriteFile(final String relativePa final String umask, final boolean isAppendBlob, EncryptionAdapter encryptionAdapter, - TracingContext tracingContext) throws IOException, - DestroyFailedException { + TracingContext tracingContext) throws IOException { AbfsRestOperation op; try { @@ -664,7 +660,7 @@ private AbfsOutputStreamContext populateAbfsOutputStreamContext(boolean isAppend public void createDirectory(final Path path, final FsPermission permission, final FsPermission umask, TracingContext tracingContext) - throws IOException, DestroyFailedException { + throws IOException { try (AbfsPerfInfo perfInfo = startTracking("createDirectory", "createPath")) { boolean isNamespaceEnabled = getIsNamespaceEnabled(tracingContext); LOG.debug("createDirectory filesystem: {} path: {} permission: {} umask: {} isNamespaceEnabled: {}", @@ -687,7 +683,7 @@ public void createDirectory(final Path path, final FsPermission permission, public AbfsInputStream openFileForRead(final Path path, final FileSystem.Statistics statistics, TracingContext tracingContext) - throws IOException, DestroyFailedException { + throws IOException { return openFileForRead(path, Optional.empty(), statistics, tracingContext); } @@ -695,7 +691,7 @@ public AbfsInputStream openFileForRead(final Path path, public AbfsInputStream openFileForRead(Path path, final Optional parameters, final FileSystem.Statistics statistics, TracingContext tracingContext) - throws IOException, DestroyFailedException { + throws IOException { try (AbfsPerfInfo perfInfo = startTracking("openFileForRead", "getPathStatus")) { LOG.debug("openFileForRead filesystem: {} path: {}", @@ -705,7 +701,6 @@ public AbfsInputStream openFileForRead(Path path, .orElse(null); String relativePath = getRelativePath(path); String resourceType, eTag; - SecretKey encryptionContext; long contentLength; EncryptionAdapter encryptionAdapter = null; if (fileStatus instanceof VersionedFileStatus @@ -731,17 +726,10 @@ public AbfsInputStream openFileForRead(Path path, op.getResponseHeader(HttpHeaderConfigurations.CONTENT_LENGTH)); eTag = op.getResponseHeader(HttpHeaderConfigurations.ETAG); if (client.getEncryptionType() == EncryptionType.ENCRYPTION_CONTEXT) { - encryptionContext = - client.getEncryptionContextProvider().getSecretKeyFromBytes(op.getResponseHeader( - HttpHeaderConfigurations.X_MS_PROPERTIES).getBytes( - StandardCharsets.UTF_8)); - if (encryptionContext == null) { - throw new IOException( - "Encryption context missing from server response"); - } encryptionAdapter = new EncryptionAdapter( client.getEncryptionContextProvider(), path.toString(), - encryptionContext); + op.getResponseHeader(HttpHeaderConfigurations.X_MS_PROPERTIES) + .getBytes(StandardCharsets.UTF_8)); } } @@ -786,8 +774,7 @@ private AbfsInputStreamContext populateAbfsInputStreamContext( public OutputStream openFileForWrite(final Path path, final FileSystem.Statistics statistics, final boolean overwrite, - TracingContext tracingContext) throws IOException, - DestroyFailedException { + TracingContext tracingContext) throws IOException { try (AbfsPerfInfo perfInfo = startTracking("openFileForWrite", "getPathStatus")) { LOG.debug("openFileForWrite filesystem: {} path: {} overwrite: {}", client.getFileSystem(), @@ -821,10 +808,8 @@ public OutputStream openFileForWrite(final Path path, } AbfsLease lease = maybeCreateLease(relativePath, tracingContext); - SecretKey encryptionContext = - client.getEncryptionContextProvider().getSecretKeyFromBytes(op.getResult() - .getResponseHeader(HttpHeaderConfigurations.X_MS_PROPERTIES).getBytes( - StandardCharsets.UTF_8)); + byte[] encryptionContext = op.getResult().getResponseHeader(HttpHeaderConfigurations.X_MS_PROPERTIES) + .getBytes(StandardCharsets.UTF_8); EncryptionAdapter encryptionAdapter = new EncryptionAdapter( client.getEncryptionContextProvider(), path.toString(), encryptionContext); @@ -853,7 +838,7 @@ public void breakLease(final Path path, final TracingContext tracingContext) thr } public void rename(final Path source, final Path destination, TracingContext tracingContext) - throws IOException, DestroyFailedException { + throws IOException { final Instant startAggregate = abfsPerfTracker.getLatencyInstant(); long countAggregate = 0; boolean shouldContinue; @@ -924,8 +909,7 @@ public void delete(final Path path, final boolean recursive, } public FileStatus getFileStatus(final Path path, - TracingContext tracingContext) throws IOException, - DestroyFailedException { + TracingContext tracingContext) throws IOException { try (AbfsPerfInfo perfInfo = startTracking("getFileStatus", "undetermined")) { boolean isNamespaceEnabled = getIsNamespaceEnabled(tracingContext); LOG.debug("getFileStatus filesystem: {} path: {} isNamespaceEnabled: {}", diff --git a/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/extensions/EncryptionContextProvider.java b/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/extensions/EncryptionContextProvider.java index 93c2f2dc08fa5..24d25a58e3302 100644 --- a/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/extensions/EncryptionContextProvider.java +++ b/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/extensions/EncryptionContextProvider.java @@ -53,7 +53,5 @@ public interface EncryptionContextProvider extends Destroyable { */ SecretKey getEncryptionKey(String path, SecretKey encryptionContext) throws IOException; - SecretKey getSecretKeyFromBytes(byte[] secret); - void destroy(); } diff --git a/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/security/EncryptionAdapter.java b/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/security/EncryptionAdapter.java index b22f8ed3504f6..c6677599fef66 100644 --- a/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/security/EncryptionAdapter.java +++ b/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/security/EncryptionAdapter.java @@ -1,6 +1,6 @@ package org.apache.hadoop.fs.azurebfs.security; -import org.apache.hadoop.fs.azurebfs.extensions.EncryptionContextProvider; +import org.apache.commons.io.IOUtils; import org.apache.hadoop.thirdparty.com.google.common.base.Preconditions; import org.slf4j.Logger; import org.slf4j.LoggerFactory; @@ -9,6 +9,13 @@ import javax.security.auth.DestroyFailedException; import javax.security.auth.Destroyable; import java.io.IOException; +import java.nio.charset.StandardCharsets; +import java.security.MessageDigest; +import java.security.NoSuchAlgorithmException; +import java.util.Arrays; +import java.util.Base64; + +import org.apache.hadoop.fs.azurebfs.extensions.EncryptionContextProvider; public class EncryptionAdapter implements Destroyable { String path; @@ -21,11 +28,11 @@ public class EncryptionAdapter implements Destroyable { LoggerFactory.getLogger(EncryptionAdapter.class); public EncryptionAdapter(EncryptionContextProvider provider, String path, - SecretKey encryptionContext) throws IOException { + byte[] encryptionContext) throws IOException { this(provider, path); Preconditions.checkNotNull(encryptionContext, "Encryption context should not be null."); - this.encryptionContext = encryptionContext; + this.encryptionContext = new ABFSSecretKey(encryptionContext); } public EncryptionAdapter(EncryptionContextProvider provider, String path) @@ -41,8 +48,14 @@ public SecretKey getEncryptionKey() throws IOException { return provider.getEncryptionKey(path, encryptionContext); } - public SecretKey fetchEncryptionContext() throws IOException { + public SecretKey fetchEncryptionContextAndComputeKeys() throws IOException { encryptionContext = provider.getEncryptionContext(path); + SecretKey key = getEncryptionKey(); + encodedKey = getBase64EncodedString(key.getEncoded()).getBytes( + StandardCharsets.UTF_8); + encodedKeySHA = getBase64EncodedString(getSHA256Hash( + IOUtils.toString(key.getEncoded(), + StandardCharsets.UTF_8.name()))).getBytes(StandardCharsets.UTF_8); return encryptionContext; } @@ -50,20 +63,56 @@ public byte[] getEncodedKey() { return encodedKey; } - public void setEncodedKey(byte[] encodedKey) { - this.encodedKey = encodedKey; - } - public byte[] getEncodedKeySHA() { return encodedKeySHA; } - public void setEncodedKeySHA(byte[] encodedKeySHA) { - this.encodedKeySHA = encodedKeySHA; - } - public void destroy() throws DestroyFailedException { encryptionKey.destroy(); provider.destroy(); } + + public class ABFSSecretKey implements SecretKey { + final byte[] secret; + public ABFSSecretKey(byte[] secret) { + this.secret = secret; + } + + @Override + public String getAlgorithm() { + return null; + } + + @Override + public String getFormat() { + return null; + } + + @Override + public byte[] getEncoded() { + return secret; + } + + @Override + public void destroy() { + Arrays.fill(secret, (byte) 0); + } + } + + public static byte[] getSHA256Hash(String key) throws IOException { + try { + final MessageDigest digester = MessageDigest.getInstance("SHA-256"); + return digester.digest(key.getBytes(StandardCharsets.UTF_8)); + } catch (NoSuchAlgorithmException e) { + throw new IOException(e); + } + } + + public static String getBase64EncodedString(String key) { + return getBase64EncodedString(key.getBytes(StandardCharsets.UTF_8)); + } + + public static String getBase64EncodedString(byte[] bytes) { + return Base64.getEncoder().encodeToString(bytes); + } } diff --git a/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/services/AbfsClient.java b/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/services/AbfsClient.java index f13e1f399f6ab..4d76658763678 100644 --- a/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/services/AbfsClient.java +++ b/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/services/AbfsClient.java @@ -26,11 +26,8 @@ import java.net.URL; import java.net.URLEncoder; import java.nio.charset.StandardCharsets; -import java.security.MessageDigest; -import java.security.NoSuchAlgorithmException; import java.time.Instant; import java.util.ArrayList; -import java.util.Base64; import java.util.List; import java.util.Locale; import java.util.UUID; @@ -126,9 +123,9 @@ private AbfsClient(final URL baseUrl, String encryptionKey = this.abfsConfiguration .getClientProvidedEncryptionKey(); if (encryptionKey != null) { - this.clientProvidedEncryptionKey = getBase64EncodedString(encryptionKey); - this.clientProvidedEncryptionKeySHA = getBase64EncodedString( - getSHA256Hash(encryptionKey)); + this.clientProvidedEncryptionKey = EncryptionAdapter.getBase64EncodedString(encryptionKey); + this.clientProvidedEncryptionKeySHA = EncryptionAdapter.getBase64EncodedString( + EncryptionAdapter.getSHA256Hash(encryptionKey)); } else { this.clientProvidedEncryptionKey = null; this.clientProvidedEncryptionKeySHA = null; @@ -181,23 +178,6 @@ public AbfsClient(final URL baseUrl, final SharedKeyCredentials sharedKeyCredent this.sasTokenProvider = sasTokenProvider; } - private byte[] getSHA256Hash(String key) throws IOException { - try { - final MessageDigest digester = MessageDigest.getInstance("SHA-256"); - return digester.digest(key.getBytes(StandardCharsets.UTF_8)); - } catch (NoSuchAlgorithmException e) { - throw new IOException(e); - } - } - - private String getBase64EncodedString(String key) { - return getBase64EncodedString(key.getBytes(StandardCharsets.UTF_8)); - } - - private String getBase64EncodedString(byte[] bytes) { - return Base64.getEncoder().encodeToString(bytes); - } - @Override public void close() throws IOException { if (tokenProvider instanceof Closeable) { @@ -242,37 +222,38 @@ List createDefaultHeaders() { return requestHeaders; } - private void addEncryptionKeyRequestHeaders( - List requestHeaders, String path, - boolean isCreateFileRequest, EncryptionAdapter encryptionAdapter, - TracingContext tracingContext) - throws IOException, DestroyFailedException { + private void addEncryptionKeyRequestHeaders(String path, + List requestHeaders, boolean isCreateFileRequest, + EncryptionAdapter encryptionAdapter, TracingContext tracingContext) + throws IOException { String encodedKey, encodedKeySHA256; switch (encryptionType) { case GLOBAL_KEY: encodedKey = clientProvidedEncryptionKey; encodedKeySHA256 = clientProvidedEncryptionKeySHA; break; + case ENCRYPTION_CONTEXT: if (isCreateFileRequest) { // get new context for create file request SecretKey encryptionContext = - encryptionAdapter.fetchEncryptionContext(); - computeKeys(encryptionAdapter); + encryptionAdapter.fetchEncryptionContextAndComputeKeys(); requestHeaders.add(new AbfsHttpHeader(X_MS_ENCRYPTION_CONTEXT, IOUtils.toString(encryptionContext.getEncoded(), StandardCharsets.UTF_8.name()))); - encryptionContext.destroy(); - + try { + encryptionContext.destroy(); + } catch (DestroyFailedException e) { + throw new IOException(e.getMessage()); + } } else if (encryptionAdapter == null) { - SecretKey encryptionContext = encryptionContextProvider.getSecretKeyFromBytes( - getPathStatus(path, false, tracingContext).getResult() - .getResponseHeader(X_MS_PROPERTIES) - .getBytes(StandardCharsets.UTF_8)); + // get encryption context from GetPathStatus response header encryptionAdapter = new EncryptionAdapter(encryptionContextProvider, - path, encryptionContext); + path, getPathStatus(path, false, tracingContext).getResult() + .getResponseHeader(X_MS_PROPERTIES) + .getBytes(StandardCharsets.UTF_8)); } - // use cached encryption keys from input/output streams + // else use cached encryption keys from input/output streams encodedKey = IOUtils.toString(encryptionAdapter.getEncodedKey(), StandardCharsets.UTF_8.name()); encodedKeySHA256 = IOUtils.toString(encryptionAdapter.getEncodedKeySHA(), @@ -287,16 +268,6 @@ private void addEncryptionKeyRequestHeaders( SERVER_SIDE_ENCRYPTION_ALGORITHM)); } - public void computeKeys(EncryptionAdapter encryptionAdapter) throws IOException { - SecretKey key = encryptionAdapter.getEncryptionKey(); - encryptionAdapter.setEncodedKey( - getBase64EncodedString(key.getEncoded()).getBytes( - StandardCharsets.UTF_8)); - encryptionAdapter.setEncodedKeySHA(getBase64EncodedString(getSHA256Hash( - IOUtils.toString(key.getEncoded(), - StandardCharsets.UTF_8.name()))).getBytes(StandardCharsets.UTF_8)); - } - AbfsUriQueryBuilder createDefaultUriQueryBuilder() { final AbfsUriQueryBuilder abfsUriQueryBuilder = new AbfsUriQueryBuilder(); abfsUriQueryBuilder.addQuery(QUERY_PARAM_TIMEOUT, DEFAULT_TIMEOUT); @@ -407,11 +378,11 @@ public AbfsRestOperation createPath(final String path, final boolean isFile, final boolean overwrite, final String permission, final String umask, final boolean isAppendBlob, final String eTag, EncryptionAdapter encryptionAdapter, TracingContext tracingContext) - throws IOException, DestroyFailedException { + throws IOException { final List requestHeaders = createDefaultHeaders(); if (isFile) { - addEncryptionKeyRequestHeaders(requestHeaders, path, true, encryptionAdapter, - tracingContext); + addEncryptionKeyRequestHeaders(path, requestHeaders, true, + encryptionAdapter, tracingContext); } if (!overwrite) { requestHeaders.add(new AbfsHttpHeader(IF_NONE_MATCH, AbfsHttpConstants.STAR)); @@ -544,7 +515,7 @@ public AbfsRestOperation breakLease(final String path, public AbfsRestOperation renamePath(String source, final String destination, final String continuation, TracingContext tracingContext) - throws IOException, DestroyFailedException { + throws IOException { final List requestHeaders = createDefaultHeaders(); String encodedRenameSource = urlEncode(FORWARD_SLASH + this.getFileSystem() + source); @@ -607,8 +578,7 @@ public AbfsRestOperation renameIdempotencyCheckOp( final Instant renameRequestStartTime, final AbfsRestOperation op, final String destination, - TracingContext tracingContext) throws IOException, - DestroyFailedException { + TracingContext tracingContext) throws IOException { if ((op.isARetriedRequest()) && (op.getResult().getStatusCode() == HttpURLConnection.HTTP_NOT_FOUND)) { // Server has returned HTTP 404, which means rename source no longer @@ -640,9 +610,9 @@ public AbfsRestOperation renameIdempotencyCheckOp( public AbfsRestOperation append(final String path, final byte[] buffer, AppendRequestParameters reqParams, final String cachedSasToken, EncryptionAdapter encryptionAdapter, TracingContext tracingContext) - throws IOException, DestroyFailedException { + throws IOException { final List requestHeaders = createDefaultHeaders(); - addEncryptionKeyRequestHeaders(requestHeaders, path, false, + addEncryptionKeyRequestHeaders(path, requestHeaders, false, encryptionAdapter, tracingContext); // JDK7 does not support PATCH, so to workaround the issue we will use // PUT and specify the real method in the X-Http-Method-Override header. @@ -711,7 +681,7 @@ && appendSuccessCheckOp(op, path, // in case we are doing a retry after checking the length of the file public boolean appendSuccessCheckOp(AbfsRestOperation op, final String path, final long length, TracingContext tracingContext) - throws IOException, DestroyFailedException { + throws IOException { if ((op.isARetriedRequest()) && (op.getResult().getStatusCode() == HttpURLConnection.HTTP_BAD_REQUEST)) { final AbfsRestOperation destStatusOp = getPathStatus(path, false, tracingContext); @@ -731,9 +701,9 @@ public AbfsRestOperation flush(final String path, final long position, boolean retainUncommittedData, boolean isClose, final String cachedSasToken, final String leaseId, EncryptionAdapter encryptionAdapter, TracingContext tracingContext) - throws IOException, DestroyFailedException { + throws IOException { final List requestHeaders = createDefaultHeaders(); - addEncryptionKeyRequestHeaders(requestHeaders, path, false, + addEncryptionKeyRequestHeaders(path, requestHeaders, false, encryptionAdapter, tracingContext); // JDK7 does not support PATCH, so to workaround the issue we will use // PUT and specify the real method in the X-Http-Method-Override header. @@ -765,9 +735,9 @@ public AbfsRestOperation flush(final String path, final long position, public AbfsRestOperation setPathProperties(final String path, final String properties, TracingContext tracingContext) - throws IOException, DestroyFailedException { + throws IOException { final List requestHeaders = createDefaultHeaders(); - addEncryptionKeyRequestHeaders(requestHeaders, path, false, + addEncryptionKeyRequestHeaders(path, requestHeaders, false, null, tracingContext); // JDK7 does not support PATCH, so to workaround the issue we will use // PUT and specify the real method in the X-Http-Method-Override header. @@ -791,9 +761,9 @@ public AbfsRestOperation setPathProperties(final String path, final String prope return op; } - public AbfsRestOperation getPathStatus(final String path, final boolean includeProperties, - TracingContext tracingContext) - throws IOException, DestroyFailedException { + public AbfsRestOperation getPathStatus(final String path, + final boolean includeProperties, TracingContext tracingContext) + throws IOException { final List requestHeaders = createDefaultHeaders(); final AbfsUriQueryBuilder abfsUriQueryBuilder = createDefaultUriQueryBuilder(); @@ -805,7 +775,7 @@ public AbfsRestOperation getPathStatus(final String path, final boolean includeP abfsUriQueryBuilder.addQuery(HttpQueryParams.QUERY_PARAM_ACTION, AbfsHttpConstants.GET_STATUS); operation = SASTokenProvider.GET_STATUS_OPERATION; } else { - addEncryptionKeyRequestHeaders(requestHeaders, path, false, null, + addEncryptionKeyRequestHeaders(path, requestHeaders, false, null, tracingContext); } abfsUriQueryBuilder.addQuery(HttpQueryParams.QUERY_PARAM_UPN, String.valueOf(abfsConfiguration.isUpnUsed())); @@ -825,10 +795,9 @@ public AbfsRestOperation getPathStatus(final String path, final boolean includeP public AbfsRestOperation read(final String path, final long position, final byte[] buffer, final int bufferOffset, final int bufferLength, final String eTag, String cachedSasToken, EncryptionAdapter encryptionAdapter, - TracingContext tracingContext) throws IOException, - DestroyFailedException { + TracingContext tracingContext) throws IOException { final List requestHeaders = createDefaultHeaders(); - addEncryptionKeyRequestHeaders(requestHeaders, path, false, + addEncryptionKeyRequestHeaders(path, requestHeaders, false, encryptionAdapter, tracingContext); requestHeaders.add(new AbfsHttpHeader(RANGE, String.format("bytes=%d-%d", position, position + bufferLength - 1))); diff --git a/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/services/AbfsInputStream.java b/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/services/AbfsInputStream.java index 959a37028a7ca..0d2b797de41cb 100644 --- a/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/services/AbfsInputStream.java +++ b/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/services/AbfsInputStream.java @@ -199,11 +199,7 @@ public int read(long position, byte[] buffer, int offset, int length) streamStatistics.readOperationStarted(); } int bytesRead = 0; - try { - bytesRead = readRemote(position, buffer, offset, length, tracingContext); - } catch (DestroyFailedException e) { - LOG.debug(e.getMessage()); - } + bytesRead = readRemote(position, buffer, offset, length, tracingContext); if (statistics != null) { statistics.incrementBytesRead(bytesRead); } @@ -534,7 +530,7 @@ private int readInternal(final long position, final byte[] b, final int offset, } int readRemote(long position, byte[] b, int offset, int length, TracingContext tracingContext) - throws IOException, DestroyFailedException { + throws IOException { if (position < 0) { throw new IllegalArgumentException("attempting to read from negative offset"); } diff --git a/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/services/AbfsOutputStream.java b/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/services/AbfsOutputStream.java index 6731464f97d46..8ef692dfbdd6d 100644 --- a/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/services/AbfsOutputStream.java +++ b/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/services/AbfsOutputStream.java @@ -58,6 +58,8 @@ import org.apache.hadoop.fs.StreamCapabilities; import org.apache.hadoop.fs.Syncable; +import javax.security.auth.DestroyFailedException; + import static org.apache.hadoop.fs.azurebfs.constants.FileSystemConfigurations.STREAM_ID_LEN; import static org.apache.hadoop.fs.azurebfs.services.AbfsErrors.ERR_WRITE_WITHOUT_LEASE; import static org.apache.hadoop.fs.impl.StoreImplementationUtils.isProbeForSyncable; diff --git a/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/AbstractAbfsIntegrationTest.java b/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/AbstractAbfsIntegrationTest.java index 56d553819feae..e8a013626e112 100644 --- a/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/AbstractAbfsIntegrationTest.java +++ b/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/AbstractAbfsIntegrationTest.java @@ -54,6 +54,8 @@ import org.apache.hadoop.fs.permission.FsPermission; import org.apache.hadoop.io.IOUtils; +import javax.security.auth.DestroyFailedException; + import static org.apache.hadoop.fs.azure.AzureBlobStorageTestAccount.WASB_ACCOUNT_NAME_DOMAIN_SUFFIX; import static org.apache.hadoop.fs.azurebfs.constants.ConfigurationKeys.*; import static org.apache.hadoop.fs.azurebfs.contracts.services.AzureServiceErrorCode.FILE_SYSTEM_NOT_FOUND; From af6cd1d257e2fb490c05175327139fcb788925db Mon Sep 17 00:00:00 2001 From: sumangala Date: Mon, 4 Oct 2021 18:52:58 +0530 Subject: [PATCH 11/77] partial bug fix --- .../fs/azurebfs/AzureBlobFileSystemStore.java | 5 ++- .../azurebfs/security/EncryptionAdapter.java | 39 +++++++++++++------ .../fs/azurebfs/services/AbfsClient.java | 28 +++++++------ .../fs/azurebfs/ITestCustomerProvidedKey.java | 5 ++- .../MockEncryptionContextProvider.java | 36 ++++++++++++++--- 5 files changed, 80 insertions(+), 33 deletions(-) diff --git a/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/AzureBlobFileSystemStore.java b/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/AzureBlobFileSystemStore.java index 816eeae549eb4..09c1cea7160b9 100644 --- a/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/AzureBlobFileSystemStore.java +++ b/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/AzureBlobFileSystemStore.java @@ -728,8 +728,9 @@ public AbfsInputStream openFileForRead(Path path, if (client.getEncryptionType() == EncryptionType.ENCRYPTION_CONTEXT) { encryptionAdapter = new EncryptionAdapter( client.getEncryptionContextProvider(), path.toString(), - op.getResponseHeader(HttpHeaderConfigurations.X_MS_PROPERTIES) - .getBytes(StandardCharsets.UTF_8)); + "context".getBytes(StandardCharsets.UTF_8)); +// op.getResponseHeader(HttpHeaderConfigurations.X_MS_PROPERTIES) +// .getBytes(StandardCharsets.UTF_8)); } } diff --git a/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/security/EncryptionAdapter.java b/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/security/EncryptionAdapter.java index c6677599fef66..d6377b32573c1 100644 --- a/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/security/EncryptionAdapter.java +++ b/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/security/EncryptionAdapter.java @@ -1,6 +1,5 @@ package org.apache.hadoop.fs.azurebfs.security; -import org.apache.commons.io.IOUtils; import org.apache.hadoop.thirdparty.com.google.common.base.Preconditions; import org.slf4j.Logger; import org.slf4j.LoggerFactory; @@ -22,8 +21,8 @@ public class EncryptionAdapter implements Destroyable { SecretKey encryptionContext; SecretKey encryptionKey; EncryptionContextProvider provider; - byte[] encodedKey = null; - byte[] encodedKeySHA = null; + String encodedKey = null; + String encodedKeySHA = null; private static final Logger LOG = LoggerFactory.getLogger(EncryptionAdapter.class); @@ -50,20 +49,38 @@ public SecretKey getEncryptionKey() throws IOException { public SecretKey fetchEncryptionContextAndComputeKeys() throws IOException { encryptionContext = provider.getEncryptionContext(path); - SecretKey key = getEncryptionKey(); - encodedKey = getBase64EncodedString(key.getEncoded()).getBytes( - StandardCharsets.UTF_8); - encodedKeySHA = getBase64EncodedString(getSHA256Hash( - IOUtils.toString(key.getEncoded(), - StandardCharsets.UTF_8.name()))).getBytes(StandardCharsets.UTF_8); + computeKeys(); return encryptionContext; } - public byte[] getEncodedKey() { + public void computeKeys() throws IOException { + SecretKey key = getEncryptionKey(); + System.out.println("key "+ new String(key.getEncoded(), + StandardCharsets.UTF_8)); + // encodedKey = getBase64EncodedString(key.getEncoded()).getBytes( + // StandardCharsets.UTF_8.name()); + // encodedKey = getBase64EncodedString(new String(key.getEncoded(), StandardCharsets.UTF_8)).getBytes( + // StandardCharsets.UTF_8.name()); //this is in byte [] form + encodedKey = getBase64EncodedString(new String(key.getEncoded(), + StandardCharsets.UTF_8)); + // encodedKeySHA = getBase64EncodedString(getSHA256Hash( + // IOUtils.toString(key.getEncoded(), + // StandardCharsets.UTF_8.name()))).getBytes(StandardCharsets.UTF_8); + encodedKeySHA = getBase64EncodedString(getSHA256Hash(new String(key.getEncoded(), + StandardCharsets.UTF_8))); + } + + public String getEncodedKey() throws IOException { + if (encodedKey == null) { + computeKeys(); + } return encodedKey; } - public byte[] getEncodedKeySHA() { + public String getEncodedKeySHA() throws IOException { + if (encodedKeySHA == null) { + computeKeys(); + } return encodedKeySHA; } diff --git a/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/services/AbfsClient.java b/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/services/AbfsClient.java index 4d76658763678..34880f8930648 100644 --- a/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/services/AbfsClient.java +++ b/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/services/AbfsClient.java @@ -238,14 +238,14 @@ private void addEncryptionKeyRequestHeaders(String path, // get new context for create file request SecretKey encryptionContext = encryptionAdapter.fetchEncryptionContextAndComputeKeys(); - requestHeaders.add(new AbfsHttpHeader(X_MS_ENCRYPTION_CONTEXT, - IOUtils.toString(encryptionContext.getEncoded(), - StandardCharsets.UTF_8.name()))); - try { - encryptionContext.destroy(); - } catch (DestroyFailedException e) { - throw new IOException(e.getMessage()); - } +// requestHeaders.add(new AbfsHttpHeader(X_MS_ENCRYPTION_CONTEXT, +// IOUtils.toString(encryptionContext.getEncoded(), +// StandardCharsets.UTF_8.name()))); +// try { +// encryptionContext.destroy(); +// } catch (DestroyFailedException e) { +// throw new IOException(e.getMessage()); +// } } else if (encryptionAdapter == null) { // get encryption context from GetPathStatus response header encryptionAdapter = new EncryptionAdapter(encryptionContextProvider, @@ -254,10 +254,14 @@ path, getPathStatus(path, false, tracingContext).getResult() .getBytes(StandardCharsets.UTF_8)); } // else use cached encryption keys from input/output streams - encodedKey = IOUtils.toString(encryptionAdapter.getEncodedKey(), - StandardCharsets.UTF_8.name()); - encodedKeySHA256 = IOUtils.toString(encryptionAdapter.getEncodedKeySHA(), - StandardCharsets.UTF_8.name()); +// encodedKey = IOUtils.toString(encryptionAdapter.getEncodedKey(), +// StandardCharsets.UTF_8.name()); + encodedKey = encryptionAdapter.getEncodedKey(); + System.out.println("1 " + encodedKey); +// encodedKeySHA256 = IOUtils.toString(encryptionAdapter.getEncodedKeySHA(), +// StandardCharsets.UTF_8.name()); + encodedKeySHA256 = encryptionAdapter.getEncodedKeySHA(); + System.out.println("2 " + encodedKeySHA256); break; default: return; // no client-provided encryption keys diff --git a/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/ITestCustomerProvidedKey.java b/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/ITestCustomerProvidedKey.java index b6500aef597df..a8bc99a53631a 100644 --- a/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/ITestCustomerProvidedKey.java +++ b/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/ITestCustomerProvidedKey.java @@ -103,7 +103,7 @@ public class ITestCustomerProvidedKey extends AbstractAbfsIntegrationTest { public ITestCustomerProvidedKey() throws Exception { boolean isCPKTestsEnabled = getConfiguration() .getBoolean(FS_AZURE_TEST_CPK_ENABLED, false); - Assume.assumeTrue(isCPKTestsEnabled); +// Assume.assumeTrue(isCPKTestsEnabled); } @Test @@ -198,7 +198,8 @@ public void testReadWithoutCPK() throws Exception { @Test public void testAppendWithCPK() throws Exception { final AzureBlobFileSystem fs = getAbfs(true); - final String fileName = path("/" + methodName.getMethodName()).toString(); + final String fileName = + new Path("/" + methodName.getMethodName()).toString(); createFileAndGetContent(fs, fileName, FILE_SIZE); // Trying to append with correct CPK headers diff --git a/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/extensions/MockEncryptionContextProvider.java b/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/extensions/MockEncryptionContextProvider.java index 8ef4328e378d1..fc2f6d0ef9fb6 100644 --- a/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/extensions/MockEncryptionContextProvider.java +++ b/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/extensions/MockEncryptionContextProvider.java @@ -5,38 +5,51 @@ import javax.crypto.SecretKey; import java.io.IOException; import java.nio.charset.StandardCharsets; +import java.util.Arrays; import java.util.HashMap; import java.util.UUID; public class MockEncryptionContextProvider implements EncryptionContextProvider { - String dummyKey = UUID.randomUUID().toString(); + String dummyKey = "12345678901234567890123456789012";//UUID.randomUUID().toString(); HashMap pathToContextMap = new HashMap<>(); HashMap contextToKeyMap = new HashMap<>(); @Override public void initialize(Configuration configuration, String accountName, String fileSystem) throws IOException { - + System.out.println("key for this session is " + dummyKey); } @Override public SecretKey getEncryptionContext(String path) throws IOException { - SecretKey newContext = - new Key(UUID.randomUUID().toString().getBytes(StandardCharsets.UTF_8)); + SecretKey newContext = new Key("context".getBytes(StandardCharsets.UTF_8)); +// new Key(UUID.randomUUID().toString().getBytes(StandardCharsets.UTF_8)); pathToContextMap.put(path, newContext); // String key = UUID.randomUUID().toString(); SecretKey key = new Key(dummyKey.getBytes(StandardCharsets.UTF_8)); // replace with above once server supports contextToKeyMap.put(newContext, key); + for (SecretKey k : contextToKeyMap.keySet()) { + System.out.println(new String(k.getEncoded(), StandardCharsets.UTF_8) + + " .. " + new String(contextToKeyMap.get(k).getEncoded(), + StandardCharsets.UTF_8)); + } return newContext; } @Override public SecretKey getEncryptionKey(String path, SecretKey encryptionContext) throws IOException { - if (!encryptionContext.equals(pathToContextMap.get(path))) { - throw new IOException("encryption context does not match path"); +// if (!encryptionContext.equals(pathToContextMap.get(path))) { +// throw new IOException("encryption context does not match path"); +// } + for (SecretKey k : contextToKeyMap.keySet()) { + System.out.println(new String(k.getEncoded(), StandardCharsets.UTF_8) + + " .. " + new String(contextToKeyMap.get(k).getEncoded(), + StandardCharsets.UTF_8)); } + System.out.println(new String(encryptionContext.getEncoded())); + System.out.println(contextToKeyMap.containsKey(encryptionContext)); return contextToKeyMap.get(encryptionContext); } @@ -66,6 +79,17 @@ public String getFormat() { public byte[] getEncoded() { return key; } + + @Override + public void destroy() { + Arrays.fill(key, (byte)0); + } + + @Override + public boolean equals(Object key) { + SecretKey k = (SecretKey) key; + return new String(k.getEncoded()).equals(new String(this.key)); + } } } From 2ad36d3fb57512c95780cc148795e1952389520c Mon Sep 17 00:00:00 2001 From: sumangala Date: Tue, 5 Oct 2021 17:55:26 +0530 Subject: [PATCH 12/77] fix mockProv map, clean up --- .../fs/azurebfs/AzureBlobFileSystemStore.java | 5 +- .../azurebfs/security/EncryptionAdapter.java | 27 ++++--- .../fs/azurebfs/services/AbfsClient.java | 23 +++--- .../fs/azurebfs/services/AbfsInputStream.java | 29 +++----- .../azurebfs/services/AbfsOutputStream.java | 2 - .../fs/azurebfs/ITestCustomerProvidedKey.java | 7 +- .../MockEncryptionContextProvider.java | 71 ++++++++++--------- 7 files changed, 81 insertions(+), 83 deletions(-) diff --git a/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/AzureBlobFileSystemStore.java b/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/AzureBlobFileSystemStore.java index 09c1cea7160b9..816eeae549eb4 100644 --- a/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/AzureBlobFileSystemStore.java +++ b/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/AzureBlobFileSystemStore.java @@ -728,9 +728,8 @@ public AbfsInputStream openFileForRead(Path path, if (client.getEncryptionType() == EncryptionType.ENCRYPTION_CONTEXT) { encryptionAdapter = new EncryptionAdapter( client.getEncryptionContextProvider(), path.toString(), - "context".getBytes(StandardCharsets.UTF_8)); -// op.getResponseHeader(HttpHeaderConfigurations.X_MS_PROPERTIES) -// .getBytes(StandardCharsets.UTF_8)); + op.getResponseHeader(HttpHeaderConfigurations.X_MS_PROPERTIES) + .getBytes(StandardCharsets.UTF_8)); } } diff --git a/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/security/EncryptionAdapter.java b/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/security/EncryptionAdapter.java index d6377b32573c1..32f5be5cec468 100644 --- a/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/security/EncryptionAdapter.java +++ b/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/security/EncryptionAdapter.java @@ -1,3 +1,21 @@ +/** + * 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.fs.azurebfs.security; import org.apache.hadoop.thirdparty.com.google.common.base.Preconditions; @@ -55,17 +73,8 @@ public SecretKey fetchEncryptionContextAndComputeKeys() throws IOException { public void computeKeys() throws IOException { SecretKey key = getEncryptionKey(); - System.out.println("key "+ new String(key.getEncoded(), - StandardCharsets.UTF_8)); - // encodedKey = getBase64EncodedString(key.getEncoded()).getBytes( - // StandardCharsets.UTF_8.name()); - // encodedKey = getBase64EncodedString(new String(key.getEncoded(), StandardCharsets.UTF_8)).getBytes( - // StandardCharsets.UTF_8.name()); //this is in byte [] form encodedKey = getBase64EncodedString(new String(key.getEncoded(), StandardCharsets.UTF_8)); - // encodedKeySHA = getBase64EncodedString(getSHA256Hash( - // IOUtils.toString(key.getEncoded(), - // StandardCharsets.UTF_8.name()))).getBytes(StandardCharsets.UTF_8); encodedKeySHA = getBase64EncodedString(getSHA256Hash(new String(key.getEncoded(), StandardCharsets.UTF_8))); } diff --git a/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/services/AbfsClient.java b/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/services/AbfsClient.java index 34880f8930648..d1e3c3bf4b5af 100644 --- a/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/services/AbfsClient.java +++ b/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/services/AbfsClient.java @@ -238,14 +238,15 @@ private void addEncryptionKeyRequestHeaders(String path, // get new context for create file request SecretKey encryptionContext = encryptionAdapter.fetchEncryptionContextAndComputeKeys(); -// requestHeaders.add(new AbfsHttpHeader(X_MS_ENCRYPTION_CONTEXT, -// IOUtils.toString(encryptionContext.getEncoded(), -// StandardCharsets.UTF_8.name()))); -// try { -// encryptionContext.destroy(); -// } catch (DestroyFailedException e) { -// throw new IOException(e.getMessage()); -// } + requestHeaders.add(new AbfsHttpHeader(X_MS_ENCRYPTION_CONTEXT, + new String(encryptionContext.getEncoded(), + StandardCharsets.UTF_8))); + try { + encryptionContext.destroy(); + } catch (DestroyFailedException e) { + throw new IOException( + "Could not destroy encryptionContext: " + e.getMessage()); + } } else if (encryptionAdapter == null) { // get encryption context from GetPathStatus response header encryptionAdapter = new EncryptionAdapter(encryptionContextProvider, @@ -254,14 +255,8 @@ path, getPathStatus(path, false, tracingContext).getResult() .getBytes(StandardCharsets.UTF_8)); } // else use cached encryption keys from input/output streams -// encodedKey = IOUtils.toString(encryptionAdapter.getEncodedKey(), -// StandardCharsets.UTF_8.name()); encodedKey = encryptionAdapter.getEncodedKey(); - System.out.println("1 " + encodedKey); -// encodedKeySHA256 = IOUtils.toString(encryptionAdapter.getEncodedKeySHA(), -// StandardCharsets.UTF_8.name()); encodedKeySHA256 = encryptionAdapter.getEncodedKeySHA(); - System.out.println("2 " + encodedKeySHA256); break; default: return; // no client-provided encryption keys diff --git a/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/services/AbfsInputStream.java b/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/services/AbfsInputStream.java index 0d2b797de41cb..58b8d689eabf0 100644 --- a/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/services/AbfsInputStream.java +++ b/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/services/AbfsInputStream.java @@ -46,8 +46,6 @@ import org.apache.hadoop.fs.statistics.IOStatistics; import org.apache.hadoop.fs.statistics.IOStatisticsSource; -import javax.security.auth.DestroyFailedException; - import static java.lang.Math.max; import static java.lang.Math.min; @@ -261,17 +259,12 @@ public synchronized int read(final byte[] b, final int off, final int len) throw limit = 0; bCursor = 0; } - try { - if (shouldReadFully()) { - lastReadBytes = readFileCompletely(b, currentOff, currentLen); - } else if (shouldReadLastBlock()) { - lastReadBytes = readLastBlock(b, currentOff, currentLen); - } else { - lastReadBytes = readOneBlock(b, currentOff, currentLen); - } - } catch (DestroyFailedException e) { - LOG.debug(e.getMessage()); - throw new IOException(e); + if (shouldReadFully()) { + lastReadBytes = readFileCompletely(b, currentOff, currentLen); + } else if (shouldReadLastBlock()) { + lastReadBytes = readLastBlock(b, currentOff, currentLen); + } else { + lastReadBytes = readOneBlock(b, currentOff, currentLen); } if (lastReadBytes > 0) { currentOff += lastReadBytes; @@ -297,7 +290,7 @@ private boolean shouldReadLastBlock() { } private int readOneBlock(final byte[] b, final int off, final int len) - throws IOException, DestroyFailedException { + throws IOException { if (len == 0) { return 0; } @@ -350,7 +343,7 @@ private int readOneBlock(final byte[] b, final int off, final int len) } private int readFileCompletely(final byte[] b, final int off, final int len) - throws IOException, DestroyFailedException { + throws IOException { if (len == 0) { return 0; } @@ -365,7 +358,7 @@ private int readFileCompletely(final byte[] b, final int off, final int len) } private int readLastBlock(final byte[] b, final int off, final int len) - throws IOException, DestroyFailedException { + throws IOException { if (len == 0) { return 0; } @@ -385,7 +378,7 @@ private int readLastBlock(final byte[] b, final int off, final int len) private int optimisedRead(final byte[] b, final int off, final int len, final long readFrom, final long actualLen) - throws IOException, DestroyFailedException { + throws IOException { fCursor = readFrom; int totalBytesRead = 0; int lastBytesRead = 0; @@ -480,7 +473,7 @@ private int copyToUserBuffer(byte[] b, int off, int len){ private int readInternal(final long position, final byte[] b, final int offset, final int length, final boolean bypassReadAhead) - throws IOException, DestroyFailedException { + throws IOException { if (readAheadEnabled && !bypassReadAhead) { // try reading from read-ahead if (offset != 0) { diff --git a/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/services/AbfsOutputStream.java b/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/services/AbfsOutputStream.java index 8ef692dfbdd6d..6731464f97d46 100644 --- a/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/services/AbfsOutputStream.java +++ b/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/services/AbfsOutputStream.java @@ -58,8 +58,6 @@ import org.apache.hadoop.fs.StreamCapabilities; import org.apache.hadoop.fs.Syncable; -import javax.security.auth.DestroyFailedException; - import static org.apache.hadoop.fs.azurebfs.constants.FileSystemConfigurations.STREAM_ID_LEN; import static org.apache.hadoop.fs.azurebfs.services.AbfsErrors.ERR_WRITE_WITHOUT_LEASE; import static org.apache.hadoop.fs.impl.StoreImplementationUtils.isProbeForSyncable; diff --git a/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/ITestCustomerProvidedKey.java b/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/ITestCustomerProvidedKey.java index a8bc99a53631a..d063eaa78423c 100644 --- a/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/ITestCustomerProvidedKey.java +++ b/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/ITestCustomerProvidedKey.java @@ -36,6 +36,7 @@ import java.util.UUID; import org.apache.commons.lang3.StringUtils; +import org.apache.hadoop.fs.azurebfs.security.EncryptionAdapter; import org.apache.hadoop.fs.azurebfs.utils.TracingContext; import org.apache.hadoop.fs.contract.ContractTestUtils; import org.apache.hadoop.thirdparty.com.google.common.base.Preconditions; @@ -244,7 +245,7 @@ public void testAppendWithCPK() throws Exception { @Test public void testAppendWithoutCPK() throws Exception { final AzureBlobFileSystem fs = getAbfs(false); - final String fileName = path("/" + methodName.getMethodName()).toString(); + final String fileName = new Path("/" + methodName.getMethodName()).toString(); createFileAndGetContent(fs, fileName, FILE_SIZE); // Trying to append without CPK headers @@ -487,7 +488,9 @@ private void testCreatePath(final boolean isWithCPK) throws Exception { .createPath(testFileName, true, true, isNamespaceEnabled ? getOctalNotation(permission) : null, isNamespaceEnabled ? getOctalNotation(umask) : null, false, null, - null, tracingContext); + new EncryptionAdapter(abfsClient.getEncryptionContextProvider(), + testFileName, "context".getBytes(StandardCharsets.UTF_8)), + tracingContext); assertCPKHeaders(abfsRestOperation, isWithCPK); assertResponseHeader(abfsRestOperation, isWithCPK, X_MS_ENCRYPTION_KEY_SHA256, getCPKSha(fs)); diff --git a/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/extensions/MockEncryptionContextProvider.java b/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/extensions/MockEncryptionContextProvider.java index fc2f6d0ef9fb6..11880ef482c59 100644 --- a/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/extensions/MockEncryptionContextProvider.java +++ b/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/extensions/MockEncryptionContextProvider.java @@ -1,61 +1,68 @@ -package org.apache.hadoop.fs.azurebfs.extensions; +/** + * 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. + */ -import org.apache.hadoop.conf.Configuration; +package org.apache.hadoop.fs.azurebfs.extensions; import javax.crypto.SecretKey; import java.io.IOException; import java.nio.charset.StandardCharsets; import java.util.Arrays; import java.util.HashMap; -import java.util.UUID; + +import org.apache.hadoop.conf.Configuration; public class MockEncryptionContextProvider implements EncryptionContextProvider { - String dummyKey = "12345678901234567890123456789012";//UUID.randomUUID().toString(); - HashMap pathToContextMap = new HashMap<>(); - HashMap contextToKeyMap = new HashMap<>(); + String dummyKey = "12345678901234567890123456789012"; + HashMap pathToContextMap = new HashMap<>(); + HashMap contextToKeyMap = new HashMap<>(); @Override public void initialize(Configuration configuration, String accountName, String fileSystem) throws IOException { - System.out.println("key for this session is " + dummyKey); } @Override public SecretKey getEncryptionContext(String path) throws IOException { - SecretKey newContext = new Key("context".getBytes(StandardCharsets.UTF_8)); -// new Key(UUID.randomUUID().toString().getBytes(StandardCharsets.UTF_8)); + String newContext = "context"; pathToContextMap.put(path, newContext); - // String key = UUID.randomUUID().toString(); - SecretKey key = new Key(dummyKey.getBytes(StandardCharsets.UTF_8)); - // replace with above once server supports + // String key = UUID.randomUUID().toString(); + Key key = new Key(dummyKey.getBytes(StandardCharsets.UTF_8)); + // replace dummyKey with key above once server supports contextToKeyMap.put(newContext, key); - for (SecretKey k : contextToKeyMap.keySet()) { - System.out.println(new String(k.getEncoded(), StandardCharsets.UTF_8) - + " .. " + new String(contextToKeyMap.get(k).getEncoded(), - StandardCharsets.UTF_8)); - } - return newContext; + return new Key(newContext.getBytes(StandardCharsets.UTF_8)); } @Override public SecretKey getEncryptionKey(String path, SecretKey encryptionContext) throws IOException { -// if (!encryptionContext.equals(pathToContextMap.get(path))) { -// throw new IOException("encryption context does not match path"); -// } - for (SecretKey k : contextToKeyMap.keySet()) { - System.out.println(new String(k.getEncoded(), StandardCharsets.UTF_8) - + " .. " + new String(contextToKeyMap.get(k).getEncoded(), - StandardCharsets.UTF_8)); + if (!new String(encryptionContext.getEncoded()).equals(pathToContextMap.get(path))) { + throw new IOException("encryption context does not match path"); } - System.out.println(new String(encryptionContext.getEncoded())); - System.out.println(contextToKeyMap.containsKey(encryptionContext)); - return contextToKeyMap.get(encryptionContext); + return contextToKeyMap.get(new String(encryptionContext.getEncoded())); } @Override public void destroy() { - + pathToContextMap = null; + for (Key encryptionKey : contextToKeyMap.values()) { + encryptionKey.destroy(); + } + contextToKeyMap = null; } class Key implements SecretKey { @@ -84,12 +91,6 @@ public byte[] getEncoded() { public void destroy() { Arrays.fill(key, (byte)0); } - - @Override - public boolean equals(Object key) { - SecretKey k = (SecretKey) key; - return new String(k.getEncoded()).equals(new String(this.key)); - } } } From 0e19fd49e9972c266847648f7faaef0cecce6e89 Mon Sep 17 00:00:00 2001 From: sumangala Date: Mon, 11 Oct 2021 10:24:45 +0530 Subject: [PATCH 13/77] tests, formatting, fix cpk test path --- .../fs/azurebfs/AzureBlobFileSystem.java | 7 +- .../fs/azurebfs/AzureBlobFileSystemStore.java | 13 +- .../azurebfs/constants/ConfigurationKeys.java | 1 - .../fs/azurebfs/services/AbfsClient.java | 8 +- .../fs/azurebfs/services/AbfsHttpHeader.java | 10 + .../fs/azurebfs/services/AbfsInputStream.java | 15 +- .../services/AbfsInputStreamContext.java | 3 +- .../azurebfs/services/AbfsOutputStream.java | 6 - .../azurebfs/AbstractAbfsIntegrationTest.java | 2 - .../fs/azurebfs/ITestCustomerProvidedKey.java | 12 +- .../fs/azurebfs/ITestEncryptionContext.java | 173 +++++++++++++++--- .../fs/azurebfs/services/TestAbfsClient.java | 2 +- 12 files changed, 182 insertions(+), 70 deletions(-) diff --git a/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/AzureBlobFileSystem.java b/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/AzureBlobFileSystem.java index ec0b7088ef9ab..91274289f5415 100644 --- a/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/AzureBlobFileSystem.java +++ b/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/AzureBlobFileSystem.java @@ -98,8 +98,6 @@ import org.apache.hadoop.util.LambdaUtils; import org.apache.hadoop.util.Progressable; -import javax.security.auth.DestroyFailedException; - import static org.apache.hadoop.fs.CommonConfigurationKeys.IOSTATISTICS_LOGGING_LEVEL; import static org.apache.hadoop.fs.CommonConfigurationKeys.IOSTATISTICS_LOGGING_LEVEL_DEFAULT; import static org.apache.hadoop.fs.azurebfs.AbfsStatistic.*; @@ -206,12 +204,11 @@ public void registerListener(Listener listener1) { public FSDataInputStream open(final Path path, final int bufferSize) throws IOException { LOG.debug("AzureBlobFileSystem.open path: {} bufferSize: {}", path, bufferSize); // bufferSize is unused. - return open(path, Optional.empty()); + return open(path, Optional.empty()); } private FSDataInputStream open(final Path path, - final Optional parameters) - throws IOException { + final Optional parameters) throws IOException { statIncrement(CALL_OPEN); Path qualifiedPath = makeQualified(path); diff --git a/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/AzureBlobFileSystemStore.java b/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/AzureBlobFileSystemStore.java index 816eeae549eb4..c77bdff44a307 100644 --- a/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/AzureBlobFileSystemStore.java +++ b/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/AzureBlobFileSystemStore.java @@ -127,8 +127,6 @@ import org.apache.hadoop.security.UserGroupInformation; import org.apache.http.client.utils.URIBuilder; -import javax.security.auth.DestroyFailedException; - import static org.apache.hadoop.fs.azurebfs.constants.AbfsHttpConstants.CHAR_EQUALS; import static org.apache.hadoop.fs.azurebfs.constants.AbfsHttpConstants.CHAR_FORWARD_SLASH; import static org.apache.hadoop.fs.azurebfs.constants.AbfsHttpConstants.CHAR_HYPHEN; @@ -530,7 +528,7 @@ public OutputStream createFile(final Path path, } EncryptionAdapter encryptionAdapter = new EncryptionAdapter( - client.getEncryptionContextProvider(), path.toString()); + client.getEncryptionContextProvider(), getRelativePath(path)); AbfsRestOperation op; if (triggerConditionalCreateOverwrite) { op = conditionalCreateOverwriteFile(relativePath, @@ -727,8 +725,9 @@ public AbfsInputStream openFileForRead(Path path, eTag = op.getResponseHeader(HttpHeaderConfigurations.ETAG); if (client.getEncryptionType() == EncryptionType.ENCRYPTION_CONTEXT) { encryptionAdapter = new EncryptionAdapter( - client.getEncryptionContextProvider(), path.toString(), - op.getResponseHeader(HttpHeaderConfigurations.X_MS_PROPERTIES) + client.getEncryptionContextProvider(), getRelativePath(path), +// op.getResponseHeader(HttpHeaderConfigurations.X_MS_PROPERTIES) + "context" .getBytes(StandardCharsets.UTF_8)); } } @@ -811,7 +810,7 @@ public OutputStream openFileForWrite(final Path path, byte[] encryptionContext = op.getResult().getResponseHeader(HttpHeaderConfigurations.X_MS_PROPERTIES) .getBytes(StandardCharsets.UTF_8); EncryptionAdapter encryptionAdapter = new EncryptionAdapter( - client.getEncryptionContextProvider(), path.toString(), + client.getEncryptionContextProvider(), getRelativePath(path), encryptionContext); return new AbfsOutputStream( @@ -1604,7 +1603,7 @@ private String getOctalNotation(FsPermission fsPermission) { return String.format(AbfsHttpConstants.PERMISSION_FORMAT, fsPermission.toOctal()); } - private String getRelativePath(final Path path) { + public String getRelativePath(final Path path) { Preconditions.checkNotNull(path, "path"); return path.toUri().getPath(); } diff --git a/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/constants/ConfigurationKeys.java b/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/constants/ConfigurationKeys.java index 1d1059c4556e3..1175f2e42f349 100644 --- a/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/constants/ConfigurationKeys.java +++ b/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/constants/ConfigurationKeys.java @@ -161,7 +161,6 @@ public final class ConfigurationKeys { public static final String FS_AZURE_ENCRYPTION_CLIENT_PROVIDED_KEY = "fs.azure.encryption.client-provided-key"; /** Custom EncryptionContextProvider type */ public static final String FS_AZURE_ENCRYPTION_CONTEXT_PROVIDER_TYPE = "fs.azure.encryption.context.provider.type"; - public static final String FS_AZURE_ENCRYPTION_MODE = "fs.azure.encryption.mode"; /** End point of ABFS account: {@value}. */ public static final String AZURE_ABFS_ENDPOINT = "fs.azure.abfs.endpoint"; diff --git a/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/services/AbfsClient.java b/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/services/AbfsClient.java index d1e3c3bf4b5af..6e0c94dae332d 100644 --- a/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/services/AbfsClient.java +++ b/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/services/AbfsClient.java @@ -36,6 +36,7 @@ import java.util.concurrent.TimeUnit; import org.apache.commons.io.IOUtils; +import org.apache.hadoop.fs.Path; import org.apache.hadoop.fs.azurebfs.extensions.EncryptionContextProvider; import org.apache.hadoop.fs.azurebfs.security.EncryptionAdapter; import org.apache.hadoop.fs.azurebfs.utils.EncryptionType; @@ -250,9 +251,10 @@ private void addEncryptionKeyRequestHeaders(String path, } else if (encryptionAdapter == null) { // get encryption context from GetPathStatus response header encryptionAdapter = new EncryptionAdapter(encryptionContextProvider, - path, getPathStatus(path, false, tracingContext).getResult() - .getResponseHeader(X_MS_PROPERTIES) - .getBytes(StandardCharsets.UTF_8)); + new Path(path).toUri().getPath(), + getPathStatus(path, false, tracingContext).getResult() + .getResponseHeader(X_MS_PROPERTIES) + .getBytes(StandardCharsets.UTF_8)); } // else use cached encryption keys from input/output streams encodedKey = encryptionAdapter.getEncodedKey(); diff --git a/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/services/AbfsHttpHeader.java b/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/services/AbfsHttpHeader.java index 0067b7554608b..f1bbcbccc64cd 100644 --- a/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/services/AbfsHttpHeader.java +++ b/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/services/AbfsHttpHeader.java @@ -18,6 +18,8 @@ package org.apache.hadoop.fs.azurebfs.services; +import org.apache.hadoop.thirdparty.com.google.common.base.Preconditions; + /** * The Http Request / Response Headers for Rest AbfsClient. */ @@ -37,4 +39,12 @@ public String getName() { public String getValue() { return value; } + + @Override + public boolean equals(Object header) { + Preconditions.checkArgument(header instanceof AbfsHttpHeader); + AbfsHttpHeader httpHeader = (AbfsHttpHeader) header; + return httpHeader.getName().equals(name) && httpHeader.getValue() + .equals(value); + } } diff --git a/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/services/AbfsInputStream.java b/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/services/AbfsInputStream.java index 58b8d689eabf0..aeb578db89dbc 100644 --- a/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/services/AbfsInputStream.java +++ b/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/services/AbfsInputStream.java @@ -196,8 +196,7 @@ public int read(long position, byte[] buffer, int offset, int length) if (streamStatistics != null) { streamStatistics.readOperationStarted(); } - int bytesRead = 0; - bytesRead = readRemote(position, buffer, offset, length, tracingContext); + int bytesRead = readRemote(position, buffer, offset, length, tracingContext); if (statistics != null) { statistics.incrementBytesRead(bytesRead); } @@ -289,8 +288,7 @@ private boolean shouldReadLastBlock() { && this.fCursor >= footerStart; } - private int readOneBlock(final byte[] b, final int off, final int len) - throws IOException { + private int readOneBlock(final byte[] b, final int off, final int len) throws IOException { if (len == 0) { return 0; } @@ -377,8 +375,7 @@ private int readLastBlock(final byte[] b, final int off, final int len) } private int optimisedRead(final byte[] b, final int off, final int len, - final long readFrom, final long actualLen) - throws IOException { + final long readFrom, final long actualLen) throws IOException { fCursor = readFrom; int totalBytesRead = 0; int lastBytesRead = 0; @@ -472,8 +469,7 @@ private int copyToUserBuffer(byte[] b, int off, int len){ } private int readInternal(final long position, final byte[] b, final int offset, final int length, - final boolean bypassReadAhead) - throws IOException { + final boolean bypassReadAhead) throws IOException { if (readAheadEnabled && !bypassReadAhead) { // try reading from read-ahead if (offset != 0) { @@ -522,8 +518,7 @@ private int readInternal(final long position, final byte[] b, final int offset, } } - int readRemote(long position, byte[] b, int offset, int length, TracingContext tracingContext) - throws IOException { + int readRemote(long position, byte[] b, int offset, int length, TracingContext tracingContext) throws IOException { if (position < 0) { throw new IllegalArgumentException("attempting to read from negative offset"); } diff --git a/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/services/AbfsInputStreamContext.java b/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/services/AbfsInputStreamContext.java index e4fad93650645..ded3d3ff07ea8 100644 --- a/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/services/AbfsInputStreamContext.java +++ b/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/services/AbfsInputStreamContext.java @@ -18,11 +18,12 @@ package org.apache.hadoop.fs.azurebfs.services; -import org.apache.hadoop.fs.azurebfs.security.EncryptionAdapter; import org.slf4j.Logger; import org.slf4j.LoggerFactory; import org.apache.hadoop.thirdparty.com.google.common.base.Preconditions; +import org.apache.hadoop.fs.azurebfs.security.EncryptionAdapter; + /** * Class to hold extra input stream configs. */ diff --git a/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/services/AbfsOutputStream.java b/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/services/AbfsOutputStream.java index 6731464f97d46..ffec73b59b797 100644 --- a/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/services/AbfsOutputStream.java +++ b/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/services/AbfsOutputStream.java @@ -24,7 +24,6 @@ import java.io.OutputStream; import java.net.HttpURLConnection; import java.nio.ByteBuffer; -import java.util.HashMap; import java.util.concurrent.ConcurrentLinkedDeque; import java.util.concurrent.LinkedBlockingQueue; import java.util.concurrent.ExecutorCompletionService; @@ -680,11 +679,6 @@ public IOStatistics getIOStatistics() { return ioStatistics; } - @VisibleForTesting - public boolean isEncryptionAdapterCached() { - return encryptionAdapter != null; - } - @VisibleForTesting public boolean isLeaseFreed() { if (lease == null) { diff --git a/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/AbstractAbfsIntegrationTest.java b/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/AbstractAbfsIntegrationTest.java index e8a013626e112..56d553819feae 100644 --- a/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/AbstractAbfsIntegrationTest.java +++ b/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/AbstractAbfsIntegrationTest.java @@ -54,8 +54,6 @@ import org.apache.hadoop.fs.permission.FsPermission; import org.apache.hadoop.io.IOUtils; -import javax.security.auth.DestroyFailedException; - import static org.apache.hadoop.fs.azure.AzureBlobStorageTestAccount.WASB_ACCOUNT_NAME_DOMAIN_SUFFIX; import static org.apache.hadoop.fs.azurebfs.constants.ConfigurationKeys.*; import static org.apache.hadoop.fs.azurebfs.contracts.services.AzureServiceErrorCode.FILE_SYSTEM_NOT_FOUND; diff --git a/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/ITestCustomerProvidedKey.java b/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/ITestCustomerProvidedKey.java index d063eaa78423c..35077fbb106b6 100644 --- a/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/ITestCustomerProvidedKey.java +++ b/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/ITestCustomerProvidedKey.java @@ -72,6 +72,7 @@ import static org.apache.hadoop.fs.azurebfs.constants.ConfigurationKeys.AZURE_CREATE_REMOTE_FILESYSTEM_DURING_INITIALIZATION; import static org.apache.hadoop.fs.azurebfs.constants.ConfigurationKeys.FS_AZURE_ENCRYPTION_CLIENT_PROVIDED_KEY; +import static org.apache.hadoop.fs.azurebfs.constants.ConfigurationKeys.FS_AZURE_ENCRYPTION_CONTEXT_PROVIDER_TYPE; import static org.apache.hadoop.fs.azurebfs.constants.FileSystemConfigurations.ONE_MB; import static org.apache.hadoop.fs.azurebfs.constants.HttpHeaderConfigurations.X_MS_ENCRYPTION_ALGORITHM; import static org.apache.hadoop.fs.azurebfs.constants.HttpHeaderConfigurations.X_MS_ENCRYPTION_KEY; @@ -104,13 +105,16 @@ public class ITestCustomerProvidedKey extends AbstractAbfsIntegrationTest { public ITestCustomerProvidedKey() throws Exception { boolean isCPKTestsEnabled = getConfiguration() .getBoolean(FS_AZURE_TEST_CPK_ENABLED, false); -// Assume.assumeTrue(isCPKTestsEnabled); + Assume.assumeTrue(isCPKTestsEnabled); + Assume.assumeTrue( + getConfiguration().get(FS_AZURE_ENCRYPTION_CONTEXT_PROVIDER_TYPE) + == null); } @Test public void testReadWithCPK() throws Exception { final AzureBlobFileSystem fs = getAbfs(true); - String fileName = path("/" + methodName.getMethodName()).toString(); + String fileName = new Path("/" + methodName.getMethodName()).toString(); createFileAndGetContent(fs, fileName, FILE_SIZE); AbfsClient abfsClient = fs.getAbfsClient(); @@ -160,7 +164,7 @@ public void testReadWithCPK() throws Exception { @Test public void testReadWithoutCPK() throws Exception { final AzureBlobFileSystem fs = getAbfs(false); - String fileName = path("/" + methodName.getMethodName()).toString(); + String fileName = new Path("/" + methodName.getMethodName()).toString(); createFileAndGetContent(fs, fileName, FILE_SIZE); AbfsClient abfsClient = fs.getAbfsClient(); @@ -281,7 +285,7 @@ public void testAppendWithoutCPK() throws Exception { @Test public void testSetGetXAttr() throws Exception { final AzureBlobFileSystem fs = getAbfs(true); - final String fileName = path(methodName.getMethodName()).toString(); + final String fileName = new Path(methodName.getMethodName()).toString(); createFileAndGetContent(fs, fileName, FILE_SIZE); String valSent = "testValue"; diff --git a/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/ITestEncryptionContext.java b/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/ITestEncryptionContext.java index 0ae05f408d462..d934256299aa9 100644 --- a/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/ITestEncryptionContext.java +++ b/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/ITestEncryptionContext.java @@ -1,53 +1,166 @@ +/** + * 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.fs.azurebfs; +import java.io.IOException; +import java.nio.charset.StandardCharsets; +import java.util.Collections; + +import org.assertj.core.api.Assertions; +import org.junit.Test; + import org.apache.hadoop.conf.Configuration; +import org.apache.hadoop.fs.FSDataInputStream; import org.apache.hadoop.fs.FSDataOutputStream; +import org.apache.hadoop.fs.FileSystem; import org.apache.hadoop.fs.Path; -import org.apache.hadoop.fs.azurebfs.extensions.MockDelegationSASTokenProvider; import org.apache.hadoop.fs.azurebfs.extensions.MockEncryptionContextProvider; import org.apache.hadoop.fs.azurebfs.services.AbfsOutputStream; +import org.apache.hadoop.fs.azurebfs.utils.EncryptionType; +import org.apache.hadoop.fs.permission.FsAction; +import org.apache.hadoop.fs.permission.FsPermission; import org.apache.hadoop.test.LambdaTestUtils; -import org.assertj.core.api.Assertions; -import org.junit.Assume; -import org.junit.Test; - -import java.io.IOException; +import static org.apache.hadoop.fs.azurebfs.constants.ConfigurationKeys.FS_AZURE_ENCRYPTION_CLIENT_PROVIDED_KEY; import static org.apache.hadoop.fs.azurebfs.constants.ConfigurationKeys.FS_AZURE_ENCRYPTION_CONTEXT_PROVIDER_TYPE; -import static org.assertj.core.api.ErrorCollector.intercept; +import static org.apache.hadoop.fs.azurebfs.utils.AclTestHelpers.aclEntry; +import static org.apache.hadoop.fs.permission.AclEntryScope.ACCESS; +import static org.apache.hadoop.fs.permission.AclEntryType.USER; public class ITestEncryptionContext extends AbstractAbfsIntegrationTest { public ITestEncryptionContext() throws Exception { - String encryptionContextProvider = getRawConfiguration().get( - FS_AZURE_ENCRYPTION_CONTEXT_PROVIDER_TYPE); - Assume.assumeTrue(MockEncryptionContextProvider.class.getCanonicalName() - .equals(encryptionContextProvider)); } - private AzureBlobFileSystem getAbfsEncrypted() throws Exception { + private AzureBlobFileSystem getEncryptionEnabledFS() throws Exception { Configuration configuration = getRawConfiguration(); - configuration.set(FS_AZURE_ENCRYPTION_CONTEXT_PROVIDER_TYPE, "MockEncryptionContextProvider"); - return getFileSystem(configuration); + configuration.set(FS_AZURE_ENCRYPTION_CONTEXT_PROVIDER_TYPE, + MockEncryptionContextProvider.class.getCanonicalName()); + configuration.unset( + FS_AZURE_ENCRYPTION_CLIENT_PROVIDED_KEY + "." + getAccountName()); + return (AzureBlobFileSystem) FileSystem.newInstance(configuration); } - @Test - public void testCreateFile() throws Exception { - AzureBlobFileSystem fs = getFileSystem(); -// AzureBlobFileSystem fs = getAbfsEncrypted(); - Path testPath = path("createTest"); - FSDataOutputStream out = fs.create(testPath); - Assertions.assertThat(((AbfsOutputStream) out.getWrappedStream()) - .isEncryptionAdapterCached()).isTrue(); - + private AzureBlobFileSystem getEncryptionDisabledFS() throws IOException { Configuration conf = getRawConfiguration(); conf.unset(FS_AZURE_ENCRYPTION_CONTEXT_PROVIDER_TYPE); - fs = getFileSystem(conf); - AzureBlobFileSystem finalFs = fs; - LambdaTestUtils.intercept(IOException.class, () -> finalFs.append(testPath)); - LambdaTestUtils.intercept(IOException.class, () -> finalFs.open(testPath)); - LambdaTestUtils.intercept(IOException.class, () -> finalFs.append(testPath)); + return (AzureBlobFileSystem) FileSystem.newInstance(conf); + } + + @Test + public void testCreateEncryptedFile() throws Exception { + AzureBlobFileSystem fs = getEncryptionEnabledFS(); + Path testPath = path("/testFile"); + createEncryptedFile(fs, testPath); + } + + @Test + public void testOpenReadWrite() throws Exception { + AzureBlobFileSystem fs = getEncryptionEnabledFS(); + Path testPath = path("/testFile"); + createEncryptedFile(fs, testPath).close(); + + try (FSDataOutputStream out = fs.append(testPath)) { + out.write("bytes".getBytes()); + out.flush(); + } + try (FSDataInputStream in = fs.open(testPath)) { + byte[] buffer = new byte[7]; + Assertions.assertThat(in.read(buffer)) + .describedAs("Incorrect read length").isEqualTo(5); + } + + AzureBlobFileSystem fs1 = getEncryptionDisabledFS(); + LambdaTestUtils.intercept(IOException.class, () -> { + try (FSDataOutputStream out = fs1.append(testPath)) { + out.write("bytes".getBytes(StandardCharsets.UTF_8)); + } + }); + LambdaTestUtils.intercept(IOException.class, () -> { + try (FSDataInputStream in = fs1.open(testPath)) { + in.read(new byte[5]); + } + }); + } + + @Test + public void testGetSetPathProperties() throws Exception { + AzureBlobFileSystem fs = getEncryptionEnabledFS(); + Path testPath = path("/testFile"); + createEncryptedFile(fs, testPath); + Assertions.assertThat(fs.getFileStatus(testPath)).isNotNull(); + byte[] value = "value".getBytes(StandardCharsets.UTF_8); + fs.setXAttr(testPath, "attribute", value); + assertArrayEquals(value, fs.getXAttr(testPath, "attribute")); + + AzureBlobFileSystem fs1 = getEncryptionDisabledFS(); + Assertions.assertThat(fs1.getFileStatus(testPath)).isNotNull(); LambdaTestUtils.intercept(IOException.class, - () -> finalFs.setXAttr(testPath, "newAttr", new byte[]{1})); + () -> fs1.setXAttr(testPath, "attribute", value)); + LambdaTestUtils.intercept(IOException.class, + () -> fs1.getXAttr(testPath, "attribute")); + } + @Test + public void testOtherApiWithoutEncryptionHeaders() throws Exception { + testOtherApi(getEncryptionDisabledFS()); + } + + @Test + public void testOtherApiWithEncryptionHeaders() throws Exception { + testOtherApi(getEncryptionEnabledFS()); + } + + private void testOtherApi(AzureBlobFileSystem fs) + throws Exception { + Path testPath = path("/testFile"); + createEncryptedFile(getEncryptionEnabledFS(), testPath); + Assertions.assertThat(fs.listStatus(testPath)) + .describedAs("ListStatus should succeed without encryption headers") + .isNotEmpty(); + fs.setAcl(testPath, + Collections.singletonList(aclEntry(ACCESS, USER, FsAction.ALL))); + fs.modifyAclEntries(testPath, + Collections.singletonList(aclEntry(ACCESS, USER, FsAction.EXECUTE))); + fs.setPermission(testPath, FsPermission.getDefault()); + Assertions.assertThat(fs.getAclStatus(testPath)) + .describedAs("GetAcl should succeed without encryption headers") + .isNotNull(); + fs.access(testPath, FsAction.EXECUTE); + Path renamedPath = new Path(testPath + "_2"); + Assertions.assertThat(fs.rename(testPath, renamedPath)) + .describedAs("Rename should succeed without encryption headers") + .isTrue(); + Assertions.assertThat(fs.delete(renamedPath, false)) + .describedAs("Delete should succeed without encryption headers") + .isTrue(); + } + + private AbfsOutputStream createEncryptedFile(AzureBlobFileSystem fs, + Path testPath) throws Exception { + FSDataOutputStream out = fs.create(testPath); + // verify file is encrypted by calling getPathStatus (with properties) + // without encryption headers in request + fs.getAbfsClient().setEncryptionType(EncryptionType.NONE); + LambdaTestUtils.intercept(IOException.class, () -> fs.getAbfsClient() + .getPathStatus(fs.getAbfsStore().getRelativePath(testPath), + true,getTestTracingContext(fs, false))); + fs.getAbfsClient().setEncryptionType(EncryptionType.ENCRYPTION_CONTEXT); + return (AbfsOutputStream) out.getWrappedStream(); } -} +} \ No newline at end of file diff --git a/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/services/TestAbfsClient.java b/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/services/TestAbfsClient.java index 68b481d447b57..154eb804c48af 100644 --- a/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/services/TestAbfsClient.java +++ b/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/services/TestAbfsClient.java @@ -276,7 +276,7 @@ public static AbfsClient createTestClientFromCurrentContext( (currentAuthType == AuthType.OAuth ? abfsConfig.getTokenProvider() : null), - null, + null, abfsClientContext); return testClient; From 4dc1601ece2066165578396c16d0b71542e6bf88 Mon Sep 17 00:00:00 2001 From: sumangala Date: Mon, 11 Oct 2021 10:42:12 +0530 Subject: [PATCH 14/77] cpk test minor change --- .../apache/hadoop/fs/azurebfs/ITestCustomerProvidedKey.java | 4 ++-- 1 file changed, 2 insertions(+), 2 deletions(-) diff --git a/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/ITestCustomerProvidedKey.java b/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/ITestCustomerProvidedKey.java index 35077fbb106b6..4c7956ab4f2e3 100644 --- a/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/ITestCustomerProvidedKey.java +++ b/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/ITestCustomerProvidedKey.java @@ -438,7 +438,7 @@ private void testListPath(final boolean isWithCPK) throws Exception { AbfsClient abfsClient2 = fs2.getAbfsClient(); TracingContext tracingContext = getTestTracingContext(fs, false); abfsRestOperation = abfsClient2.listPath(testDirName, false, INT_50, - null, null); + null, getTestTracingContext(fs2, true)); assertListstatus(fs, abfsRestOperation, testPath); if (isWithCPK) { @@ -667,7 +667,7 @@ private void testGetPathStatusFile(final boolean isWithCPK) throws Exception { assertResponseHeader(abfsRestOperation, false, X_MS_REQUEST_SERVER_ENCRYPTED, ""); - abfsRestOperation = abfsClient.getPathStatus(testFileName, false, + abfsRestOperation = abfsClient.getPathStatus(testFileName, true, tracingContext); assertCPKHeaders(abfsRestOperation, isWithCPK); assertResponseHeader(abfsRestOperation, isWithCPK, From 8357830578825b52cf93619f722754730b1227bc Mon Sep 17 00:00:00 2001 From: sumangala Date: Mon, 11 Oct 2021 17:23:00 +0530 Subject: [PATCH 15/77] checkstyle --- .../fs/azurebfs/security/EncryptionAdapter.java | 14 +++++++------- .../hadoop/fs/azurebfs/services/AbfsClient.java | 1 - .../fs/azurebfs/services/AbfsHttpHeader.java | 8 -------- .../fs/azurebfs/ITestCustomerProvidedKey.java | 1 - .../hadoop/fs/azurebfs/ITestEncryptionContext.java | 4 ++-- .../extensions/MockEncryptionContextProvider.java | 8 ++++---- 6 files changed, 13 insertions(+), 23 deletions(-) diff --git a/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/security/EncryptionAdapter.java b/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/security/EncryptionAdapter.java index 32f5be5cec468..b69cec3fa6d99 100644 --- a/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/security/EncryptionAdapter.java +++ b/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/security/EncryptionAdapter.java @@ -35,12 +35,12 @@ import org.apache.hadoop.fs.azurebfs.extensions.EncryptionContextProvider; public class EncryptionAdapter implements Destroyable { - String path; - SecretKey encryptionContext; - SecretKey encryptionKey; - EncryptionContextProvider provider; - String encodedKey = null; - String encodedKeySHA = null; + private final String path; + private SecretKey encryptionContext; + private SecretKey encryptionKey; + private final EncryptionContextProvider provider; + private String encodedKey = null; + private String encodedKeySHA = null; private static final Logger LOG = LoggerFactory.getLogger(EncryptionAdapter.class); @@ -99,7 +99,7 @@ public void destroy() throws DestroyFailedException { } public class ABFSSecretKey implements SecretKey { - final byte[] secret; + private final byte[] secret; public ABFSSecretKey(byte[] secret) { this.secret = secret; } diff --git a/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/services/AbfsClient.java b/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/services/AbfsClient.java index b54d74fea2667..0533d78b659d8 100644 --- a/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/services/AbfsClient.java +++ b/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/services/AbfsClient.java @@ -35,7 +35,6 @@ import java.util.concurrent.ThreadFactory; import java.util.concurrent.TimeUnit; -import org.apache.commons.io.IOUtils; import org.apache.hadoop.fs.Path; import org.apache.hadoop.fs.azurebfs.extensions.EncryptionContextProvider; import org.apache.hadoop.fs.azurebfs.security.EncryptionAdapter; diff --git a/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/services/AbfsHttpHeader.java b/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/services/AbfsHttpHeader.java index f1bbcbccc64cd..1176fd5cd34b7 100644 --- a/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/services/AbfsHttpHeader.java +++ b/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/services/AbfsHttpHeader.java @@ -39,12 +39,4 @@ public String getName() { public String getValue() { return value; } - - @Override - public boolean equals(Object header) { - Preconditions.checkArgument(header instanceof AbfsHttpHeader); - AbfsHttpHeader httpHeader = (AbfsHttpHeader) header; - return httpHeader.getName().equals(name) && httpHeader.getValue() - .equals(value); - } } diff --git a/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/ITestCustomerProvidedKey.java b/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/ITestCustomerProvidedKey.java index 4c7956ab4f2e3..a32e5a33061d8 100644 --- a/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/ITestCustomerProvidedKey.java +++ b/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/ITestCustomerProvidedKey.java @@ -35,7 +35,6 @@ import java.util.Random; import java.util.UUID; -import org.apache.commons.lang3.StringUtils; import org.apache.hadoop.fs.azurebfs.security.EncryptionAdapter; import org.apache.hadoop.fs.azurebfs.utils.TracingContext; import org.apache.hadoop.fs.contract.ContractTestUtils; diff --git a/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/ITestEncryptionContext.java b/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/ITestEncryptionContext.java index d934256299aa9..26e882d56675e 100644 --- a/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/ITestEncryptionContext.java +++ b/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/ITestEncryptionContext.java @@ -158,8 +158,8 @@ private AbfsOutputStream createEncryptedFile(AzureBlobFileSystem fs, // without encryption headers in request fs.getAbfsClient().setEncryptionType(EncryptionType.NONE); LambdaTestUtils.intercept(IOException.class, () -> fs.getAbfsClient() - .getPathStatus(fs.getAbfsStore().getRelativePath(testPath), - true,getTestTracingContext(fs, false))); + .getPathStatus(fs.getAbfsStore().getRelativePath(testPath), true, + getTestTracingContext(fs, false))); fs.getAbfsClient().setEncryptionType(EncryptionType.ENCRYPTION_CONTEXT); return (AbfsOutputStream) out.getWrappedStream(); } diff --git a/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/extensions/MockEncryptionContextProvider.java b/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/extensions/MockEncryptionContextProvider.java index 11880ef482c59..aad0b33e5af04 100644 --- a/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/extensions/MockEncryptionContextProvider.java +++ b/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/extensions/MockEncryptionContextProvider.java @@ -27,9 +27,9 @@ import org.apache.hadoop.conf.Configuration; public class MockEncryptionContextProvider implements EncryptionContextProvider { - String dummyKey = "12345678901234567890123456789012"; - HashMap pathToContextMap = new HashMap<>(); - HashMap contextToKeyMap = new HashMap<>(); + private String dummyKey = "12345678901234567890123456789012"; + private HashMap pathToContextMap = new HashMap<>(); + private HashMap contextToKeyMap = new HashMap<>(); @Override public void initialize(Configuration configuration, String accountName, String fileSystem) throws IOException { @@ -89,7 +89,7 @@ public byte[] getEncoded() { @Override public void destroy() { - Arrays.fill(key, (byte)0); + Arrays.fill(key, (byte) 0); } } } From 8035ed737a68953fb53e6328172cd7d55cb2aff0 Mon Sep 17 00:00:00 2001 From: sumangala Date: Mon, 11 Oct 2021 21:21:36 +0530 Subject: [PATCH 16/77] javadoc --- .../azurebfs/extensions/EncryptionContextProvider.java | 9 +++++---- .../hadoop/fs/azurebfs/services/AbfsHttpHeader.java | 2 -- 2 files changed, 5 insertions(+), 6 deletions(-) diff --git a/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/extensions/EncryptionContextProvider.java b/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/extensions/EncryptionContextProvider.java index 24d25a58e3302..cce6c902615a7 100644 --- a/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/extensions/EncryptionContextProvider.java +++ b/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/extensions/EncryptionContextProvider.java @@ -18,18 +18,19 @@ package org.apache.hadoop.fs.azurebfs.extensions; -import org.apache.hadoop.conf.Configuration; - import javax.crypto.SecretKey; import javax.security.auth.Destroyable; import java.io.IOException; +import org.apache.hadoop.conf.Configuration; + public interface EncryptionContextProvider extends Destroyable { /** * Initialize instance * * @param configuration rawConfig instance * @param accountName Account Name (with domain) + * @param fileSystem container name * @throws IOException error in initialization */ void initialize(Configuration configuration, String accountName, String fileSystem) throws IOException; @@ -39,7 +40,7 @@ public interface EncryptionContextProvider extends Destroyable { * * @param path file path from filesystem root * @return encryptionContext string - * @throws IOException + * @throws IOException error in fetching encryption context */ SecretKey getEncryptionContext(String path) throws IOException; @@ -49,7 +50,7 @@ public interface EncryptionContextProvider extends Destroyable { * @param path file path from filesystem root * @param encryptionContext encryptionContext fetched from server * @return Encryption key - * @throws IOException + * @throws IOException error in fetching encryption key */ SecretKey getEncryptionKey(String path, SecretKey encryptionContext) throws IOException; diff --git a/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/services/AbfsHttpHeader.java b/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/services/AbfsHttpHeader.java index 1176fd5cd34b7..0067b7554608b 100644 --- a/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/services/AbfsHttpHeader.java +++ b/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/services/AbfsHttpHeader.java @@ -18,8 +18,6 @@ package org.apache.hadoop.fs.azurebfs.services; -import org.apache.hadoop.thirdparty.com.google.common.base.Preconditions; - /** * The Http Request / Response Headers for Rest AbfsClient. */ From 3bf75c318cba253de00f526c17528c3d85c15c2e Mon Sep 17 00:00:00 2001 From: sumangala Date: Thu, 14 Oct 2021 11:14:49 +0530 Subject: [PATCH 17/77] add parameterized test --- .../fs/azurebfs/AzureBlobFileSystemStore.java | 6 +- .../azurebfs/security/EncryptionAdapter.java | 17 +- .../fs/azurebfs/services/AbfsClient.java | 9 +- .../fs/azurebfs/ITestEncryptionContext.java | 273 ++++++++++++++++-- 4 files changed, 273 insertions(+), 32 deletions(-) diff --git a/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/AzureBlobFileSystemStore.java b/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/AzureBlobFileSystemStore.java index 528716c6db5f3..07f3b91a7a9f6 100644 --- a/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/AzureBlobFileSystemStore.java +++ b/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/AzureBlobFileSystemStore.java @@ -790,8 +790,7 @@ public AbfsInputStream openFileForRead(Path path, if (client.getEncryptionType() == EncryptionType.ENCRYPTION_CONTEXT) { encryptionAdapter = new EncryptionAdapter( client.getEncryptionContextProvider(), getRelativePath(path), -// op.getResponseHeader(HttpHeaderConfigurations.X_MS_PROPERTIES) - "context" + op.getResponseHeader(HttpHeaderConfigurations.X_MS_ENCRYPTION_CONTEXT) .getBytes(StandardCharsets.UTF_8)); } } @@ -871,7 +870,8 @@ public OutputStream openFileForWrite(final Path path, } AbfsLease lease = maybeCreateLease(relativePath, tracingContext); - byte[] encryptionContext = op.getResult().getResponseHeader(HttpHeaderConfigurations.X_MS_PROPERTIES) + byte[] encryptionContext = "context" +// op.getResult().getResponseHeader(HttpHeaderConfigurations.X_MS_ENCRYPTION_CONTEXT) .getBytes(StandardCharsets.UTF_8); EncryptionAdapter encryptionAdapter = new EncryptionAdapter( client.getEncryptionContextProvider(), getRelativePath(path), diff --git a/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/security/EncryptionAdapter.java b/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/security/EncryptionAdapter.java index b69cec3fa6d99..4a062f66874e1 100644 --- a/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/security/EncryptionAdapter.java +++ b/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/security/EncryptionAdapter.java @@ -18,19 +18,19 @@ package org.apache.hadoop.fs.azurebfs.security; -import org.apache.hadoop.thirdparty.com.google.common.base.Preconditions; -import org.slf4j.Logger; -import org.slf4j.LoggerFactory; - -import javax.crypto.SecretKey; -import javax.security.auth.DestroyFailedException; -import javax.security.auth.Destroyable; import java.io.IOException; import java.nio.charset.StandardCharsets; import java.security.MessageDigest; import java.security.NoSuchAlgorithmException; import java.util.Arrays; import java.util.Base64; +import javax.crypto.SecretKey; +import javax.security.auth.DestroyFailedException; +import javax.security.auth.Destroyable; + +import org.apache.hadoop.thirdparty.com.google.common.base.Preconditions; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; import org.apache.hadoop.fs.azurebfs.extensions.EncryptionContextProvider; @@ -62,7 +62,8 @@ public SecretKey getEncryptionKey() throws IOException { if (encryptionKey != null) { return encryptionKey; } - return provider.getEncryptionKey(path, encryptionContext); + encryptionKey = provider.getEncryptionKey(path, encryptionContext); + return encryptionKey; } public SecretKey fetchEncryptionContextAndComputeKeys() throws IOException { diff --git a/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/services/AbfsClient.java b/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/services/AbfsClient.java index 0533d78b659d8..2c55b36e656f5 100644 --- a/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/services/AbfsClient.java +++ b/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/services/AbfsClient.java @@ -101,7 +101,7 @@ public class AbfsClient implements Closeable { private AccessTokenProvider tokenProvider; private SASTokenProvider sasTokenProvider; private final AbfsCounters abfsCounters; - private final EncryptionContextProvider encryptionContextProvider; + private EncryptionContextProvider encryptionContextProvider; private EncryptionType encryptionType; private final ListeningScheduledExecutorService executorService; @@ -253,7 +253,7 @@ private void addEncryptionKeyRequestHeaders(String path, encryptionAdapter = new EncryptionAdapter(encryptionContextProvider, new Path(path).toUri().getPath(), getPathStatus(path, false, tracingContext).getResult() - .getResponseHeader(X_MS_PROPERTIES) + .getResponseHeader(X_MS_ENCRYPTION_CONTEXT) .getBytes(StandardCharsets.UTF_8)); } // else use cached encryption keys from input/output streams @@ -1251,6 +1251,11 @@ public SASTokenProvider getSasTokenProvider() { return this.sasTokenProvider; } + @VisibleForTesting + public void setEncryptionContextProvider(EncryptionContextProvider provider) { + encryptionContextProvider = provider; + } + /** * Getter for abfsCounters from AbfsClient. * @return AbfsCounters instance. diff --git a/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/ITestEncryptionContext.java b/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/ITestEncryptionContext.java index 26e882d56675e..b6d9e2d4e9fa5 100644 --- a/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/ITestEncryptionContext.java +++ b/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/ITestEncryptionContext.java @@ -20,8 +20,17 @@ import java.io.IOException; import java.nio.charset.StandardCharsets; +import java.util.Arrays; import java.util.Collections; +import org.apache.hadoop.fs.azurebfs.constants.FSOperationType; +import org.apache.hadoop.fs.azurebfs.extensions.EncryptionContextProvider; +import org.apache.hadoop.fs.azurebfs.security.EncryptionAdapter; +import org.apache.hadoop.fs.azurebfs.services.AbfsClient; +import org.apache.hadoop.fs.azurebfs.services.AbfsHttpOperation; +import org.apache.hadoop.fs.azurebfs.services.AbfsRestOperation; +import org.apache.hadoop.fs.permission.AclEntry; +import org.apache.hadoop.util.Lists; import org.assertj.core.api.Assertions; import org.junit.Test; @@ -31,23 +40,191 @@ import org.apache.hadoop.fs.FileSystem; import org.apache.hadoop.fs.Path; import org.apache.hadoop.fs.azurebfs.extensions.MockEncryptionContextProvider; -import org.apache.hadoop.fs.azurebfs.services.AbfsOutputStream; import org.apache.hadoop.fs.azurebfs.utils.EncryptionType; import org.apache.hadoop.fs.permission.FsAction; import org.apache.hadoop.fs.permission.FsPermission; import org.apache.hadoop.test.LambdaTestUtils; +import org.junit.runner.RunWith; +import org.junit.runners.Parameterized; +import org.junit.runners.Parameterized.Parameter; import static org.apache.hadoop.fs.azurebfs.constants.ConfigurationKeys.FS_AZURE_ENCRYPTION_CLIENT_PROVIDED_KEY; import static org.apache.hadoop.fs.azurebfs.constants.ConfigurationKeys.FS_AZURE_ENCRYPTION_CONTEXT_PROVIDER_TYPE; +import static org.apache.hadoop.fs.azurebfs.constants.HttpHeaderConfigurations.X_MS_ENCRYPTION_KEY_SHA256; +import static org.apache.hadoop.fs.azurebfs.constants.HttpHeaderConfigurations.X_MS_REQUEST_SERVER_ENCRYPTED; +import static org.apache.hadoop.fs.azurebfs.constants.HttpHeaderConfigurations.X_MS_SERVER_ENCRYPTED; import static org.apache.hadoop.fs.azurebfs.utils.AclTestHelpers.aclEntry; +import static org.apache.hadoop.fs.azurebfs.utils.EncryptionType.ENCRYPTION_CONTEXT; +import static org.apache.hadoop.fs.azurebfs.utils.EncryptionType.GLOBAL_KEY; +import static org.apache.hadoop.fs.azurebfs.utils.EncryptionType.NONE; import static org.apache.hadoop.fs.permission.AclEntryScope.ACCESS; import static org.apache.hadoop.fs.permission.AclEntryType.USER; +import static org.apache.hadoop.fs.permission.FsAction.ALL; +@RunWith(Parameterized.class) public class ITestEncryptionContext extends AbstractAbfsIntegrationTest { + private final String cpk = "12345678901234567890123456789012"; + private final String cpkSHAEncoded = EncryptionAdapter.getBase64EncodedString( + EncryptionAdapter.getSHA256Hash(cpk)); + + @Parameterized.Parameter + public EncryptionType fileEncryptionType; + + @Parameterized.Parameter(1) + public EncryptionType requestEncryptionType; + + @Parameterized.Parameter(2) + public FSOperationType operation; + + @Parameterized.Parameter(3) + public boolean responseHeaderServerEnc; + + @Parameterized.Parameter(4) + public boolean responseHeaderReqServerEnc; + + @Parameterized.Parameter(5) + public boolean isExceptionCase; + + @Parameterized.Parameter(6) + public boolean isCpkResponseHdrExpected; + + + @Parameterized.Parameters(name = "{2}") + public static Iterable params() { + return Arrays.asList(new Object[][] { + {ENCRYPTION_CONTEXT, ENCRYPTION_CONTEXT, FSOperationType.READ, true, + false, false, true}, + {ENCRYPTION_CONTEXT, ENCRYPTION_CONTEXT, FSOperationType.WRITE, false, + true, false, true}, + {ENCRYPTION_CONTEXT, ENCRYPTION_CONTEXT, FSOperationType.SET_ACL, false, + false, false, false}, + {ENCRYPTION_CONTEXT, ENCRYPTION_CONTEXT, FSOperationType.GET_ATTR, false, + false, false, false}, + {ENCRYPTION_CONTEXT, ENCRYPTION_CONTEXT, FSOperationType.LISTSTATUS, false, + false, false, false}, + {ENCRYPTION_CONTEXT, NONE, FSOperationType.WRITE, false, false, true, + false}, + {ENCRYPTION_CONTEXT, NONE, FSOperationType.GET_ATTR, false, false, true, + false}, + {GLOBAL_KEY, GLOBAL_KEY, FSOperationType.READ, true, false, false, + true}, + {GLOBAL_KEY, GLOBAL_KEY, FSOperationType.WRITE, false, true, false, + true}, + {GLOBAL_KEY, GLOBAL_KEY, FSOperationType.SET_ACL, false, false, false, + false}, + {GLOBAL_KEY, GLOBAL_KEY, FSOperationType.LISTSTATUS, false, false, false, + false}, + {GLOBAL_KEY, GLOBAL_KEY, FSOperationType.RENAME, false, false, false, + false}, + {GLOBAL_KEY, GLOBAL_KEY, FSOperationType.DELETE, false, false, false, + false}, + {GLOBAL_KEY, GLOBAL_KEY, FSOperationType.GET_ATTR, false, false, false, + false}, + {GLOBAL_KEY, GLOBAL_KEY, FSOperationType.SET_ATTR, false, false, false, + false}, + {GLOBAL_KEY, NONE, FSOperationType.READ, true, false, true, + true}, + {GLOBAL_KEY, NONE, FSOperationType.WRITE, false, true, true, + true}, + {GLOBAL_KEY, NONE, FSOperationType.SET_ACL, false, false, false, + false}, + {GLOBAL_KEY, NONE, FSOperationType.RENAME, true, false, true, + true}, + {GLOBAL_KEY, NONE, FSOperationType.LISTSTATUS, false, true, true, + true}, + {GLOBAL_KEY, NONE, FSOperationType.DELETE, false, false, false, + false}, + }); + } + public ITestEncryptionContext() throws Exception { + super(); } - private AzureBlobFileSystem getEncryptionEnabledFS() throws Exception { + @Test + public void runTest() throws Exception { + AzureBlobFileSystem fs = getFS(); + Path testPath = path("/testFile"); + EncryptionContextProvider ecp = createEncryptedFile(testPath); + AbfsRestOperation op = callOperation(fs, testPath, ecp); + if (op == null) { + return; + } + AbfsHttpOperation httpOp = op.getResult(); + if (isCpkResponseHdrExpected) { + Assertions.assertThat(httpOp.getResponseHeader(X_MS_ENCRYPTION_KEY_SHA256)) + .isEqualTo(cpkSHAEncoded); + + if (operation == FSOperationType.READ) { + Assertions.assertThat(httpOp.getResponseHeader(X_MS_SERVER_ENCRYPTED)).isEqualTo("true"); + } else if (operation == FSOperationType.WRITE || operation == FSOperationType.APPEND) { + Assertions.assertThat(httpOp.getResponseHeader(X_MS_REQUEST_SERVER_ENCRYPTED)) + .isEqualTo("true"); + } + } + } + + private AbfsRestOperation callOperation(AzureBlobFileSystem fs, + Path testPath, EncryptionContextProvider ecp) + throws Exception { + AbfsClient client = fs.getAbfsClient(); + client.setEncryptionContextProvider(ecp); + if (isExceptionCase) { + LambdaTestUtils.intercept(IOException.class, () -> { + switch (operation) { + case WRITE: try (FSDataOutputStream out = fs.append(testPath)) { + out.write("bytes".getBytes()); + } + break; + case READ: try (FSDataInputStream in = fs.open(testPath)) { + in.read(new byte[5]); + } + break; + case SET_ATTR: + fs.setXAttr(testPath, "attribute", "value".getBytes()); break; + case GET_ATTR: + fs.getXAttr(testPath, "attribute"); break; + } + }); + return null; + } else { + EncryptionAdapter encryptionAdapter = null; + if (fileEncryptionType == ENCRYPTION_CONTEXT) { + encryptionAdapter = new EncryptionAdapter( +// client.getEncryptionContextProvider(), + ecp, + fs.getAbfsStore().getRelativePath(testPath), + "context".getBytes(StandardCharsets.UTF_8)); + } + String path = fs.getAbfsStore().getRelativePath(testPath); + switch (operation) { + case READ: + return client.read(path, 0, + new byte[5], + 0, 5, null, null, encryptionAdapter, + getTestTracingContext(fs, true)); + case WRITE: return client.flush(path, 3, false, false, null, null, + encryptionAdapter, getTestTracingContext(fs, false)); + case SET_ACL: return client.setAcl(path, AclEntry.aclSpecToString( + Lists.newArrayList(aclEntry(ACCESS, USER, ALL))), + getTestTracingContext(fs, false)); + case LISTSTATUS: return client.listPath(path, false, 5, null, + getTestTracingContext(fs, true)); + case RENAME: return client.renamePath(path, new Path(path + "_2").toString(), + null, + getTestTracingContext(fs, true)); +// return client.renamePath(new Path(path + "_2").toString(), path, null, +// getTestTracingContext(fs, true)); + case DELETE: return client.deletePath(path, false, null, + getTestTracingContext(fs, false)); + case GET_ATTR: return client.getPathStatus(path, true, + getTestTracingContext(fs, false)); + default: throw new NoSuchFieldException(); + } + } + } + + private AzureBlobFileSystem getECProviderEnabledFS() throws Exception { Configuration configuration = getRawConfiguration(); configuration.set(FS_AZURE_ENCRYPTION_CONTEXT_PROVIDER_TYPE, MockEncryptionContextProvider.class.getCanonicalName()); @@ -56,24 +233,77 @@ private AzureBlobFileSystem getEncryptionEnabledFS() throws Exception { return (AzureBlobFileSystem) FileSystem.newInstance(configuration); } + private AzureBlobFileSystem getCPKenabledFS() throws IOException { + Configuration conf = getRawConfiguration(); + conf.set(FS_AZURE_ENCRYPTION_CLIENT_PROVIDED_KEY + "." + getAccountName(), + cpk); + conf.unset(FS_AZURE_ENCRYPTION_CONTEXT_PROVIDER_TYPE); + return (AzureBlobFileSystem) FileSystem.newInstance(conf); + } + private AzureBlobFileSystem getEncryptionDisabledFS() throws IOException { Configuration conf = getRawConfiguration(); conf.unset(FS_AZURE_ENCRYPTION_CONTEXT_PROVIDER_TYPE); return (AzureBlobFileSystem) FileSystem.newInstance(conf); } + private AzureBlobFileSystem getFS() throws Exception { + AzureBlobFileSystem fs; + if (requestEncryptionType == ENCRYPTION_CONTEXT) { + fs = getECProviderEnabledFS(); + } else if (requestEncryptionType == GLOBAL_KEY) { + fs = getCPKenabledFS(); + } else { + fs = getEncryptionDisabledFS(); + } + return fs; + } + @Test public void testCreateEncryptedFile() throws Exception { - AzureBlobFileSystem fs = getEncryptionEnabledFS(); Path testPath = path("/testFile"); - createEncryptedFile(fs, testPath); + createEncryptedFile(testPath); + } + + @Test + public void testRead() throws Exception { + AzureBlobFileSystem fs = getFS(); + Path testPath = path("/testFile"); + AbfsClient client = fs.getAbfsClient(); + EncryptionAdapter encryptionAdapter = null; + if (fileEncryptionType == ENCRYPTION_CONTEXT) { + encryptionAdapter = new EncryptionAdapter( + client.getEncryptionContextProvider(), + fs.getAbfsStore().getRelativePath(testPath), + "context".getBytes(StandardCharsets.UTF_8)); + } + AbfsRestOperation op = client.read( + fs.getAbfsStore().getRelativePath(testPath), 0, new byte[5], 0, 5, null, + null, encryptionAdapter, getTestTracingContext(fs, true)); + checkEncryptionResponseHeaders(op.getResult(), FSOperationType.READ); + } + + @Test + public void testWrite() throws Exception { + AzureBlobFileSystem fs = getECProviderEnabledFS(); + } + + private void checkEncryptionResponseHeaders(AbfsHttpOperation op, + FSOperationType type) { + Assertions.assertThat(op.getResponseHeader(X_MS_ENCRYPTION_KEY_SHA256)).isEqualTo(cpkSHAEncoded); + if (type == FSOperationType.READ) { + Assertions.assertThat(op.getResponseHeader(X_MS_SERVER_ENCRYPTED)).isEqualTo("true"); + } else if (type == FSOperationType.WRITE || type == FSOperationType.APPEND) { + Assertions.assertThat(op.getResponseHeader(X_MS_REQUEST_SERVER_ENCRYPTED)) + .isEqualTo("true"); + } } @Test public void testOpenReadWrite() throws Exception { - AzureBlobFileSystem fs = getEncryptionEnabledFS(); + AzureBlobFileSystem fs = getECProviderEnabledFS(); Path testPath = path("/testFile"); - createEncryptedFile(fs, testPath).close(); + createEncryptedFile(testPath); try (FSDataOutputStream out = fs.append(testPath)) { out.write("bytes".getBytes()); @@ -100,9 +330,9 @@ public void testOpenReadWrite() throws Exception { @Test public void testGetSetPathProperties() throws Exception { - AzureBlobFileSystem fs = getEncryptionEnabledFS(); + AzureBlobFileSystem fs = getECProviderEnabledFS(); Path testPath = path("/testFile"); - createEncryptedFile(fs, testPath); + createEncryptedFile(testPath); Assertions.assertThat(fs.getFileStatus(testPath)).isNotNull(); byte[] value = "value".getBytes(StandardCharsets.UTF_8); fs.setXAttr(testPath, "attribute", value); @@ -123,13 +353,13 @@ public void testOtherApiWithoutEncryptionHeaders() throws Exception { @Test public void testOtherApiWithEncryptionHeaders() throws Exception { - testOtherApi(getEncryptionEnabledFS()); + testOtherApi(getECProviderEnabledFS()); } private void testOtherApi(AzureBlobFileSystem fs) throws Exception { Path testPath = path("/testFile"); - createEncryptedFile(getEncryptionEnabledFS(), testPath); + createEncryptedFile(testPath); Assertions.assertThat(fs.listStatus(testPath)) .describedAs("ListStatus should succeed without encryption headers") .isNotEmpty(); @@ -151,16 +381,21 @@ private void testOtherApi(AzureBlobFileSystem fs) .isTrue(); } - private AbfsOutputStream createEncryptedFile(AzureBlobFileSystem fs, - Path testPath) throws Exception { - FSDataOutputStream out = fs.create(testPath); + private EncryptionContextProvider createEncryptedFile(Path testPath) throws Exception { + AzureBlobFileSystem fs = fileEncryptionType == ENCRYPTION_CONTEXT? + getECProviderEnabledFS() : getCPKenabledFS(); + try (FSDataOutputStream out = fs.create(testPath)) { + out.write("123".getBytes()); + } // verify file is encrypted by calling getPathStatus (with properties) // without encryption headers in request - fs.getAbfsClient().setEncryptionType(EncryptionType.NONE); - LambdaTestUtils.intercept(IOException.class, () -> fs.getAbfsClient() - .getPathStatus(fs.getAbfsStore().getRelativePath(testPath), true, - getTestTracingContext(fs, false))); - fs.getAbfsClient().setEncryptionType(EncryptionType.ENCRYPTION_CONTEXT); - return (AbfsOutputStream) out.getWrappedStream(); + if (fileEncryptionType != EncryptionType.NONE) { + fs.getAbfsClient().setEncryptionType(EncryptionType.NONE); + LambdaTestUtils.intercept(IOException.class, () -> fs.getAbfsClient() + .getPathStatus(fs.getAbfsStore().getRelativePath(testPath), true, + getTestTracingContext(fs, false))); + fs.getAbfsClient().setEncryptionType(fileEncryptionType); + } + return fs.getAbfsClient().getEncryptionContextProvider(); } } \ No newline at end of file From fdc3149423e7a8e9b1d377cd48b6f6131a7fe668 Mon Sep 17 00:00:00 2001 From: sumangala Date: Thu, 14 Oct 2021 20:07:07 +0530 Subject: [PATCH 18/77] complete tests --- .../fs/azurebfs/AzureBlobFileSystemStore.java | 9 +- ...ontext.java => ITestCustomEncryption.java} | 357 +++++++----------- .../MockEncryptionContextProvider.java | 11 + 3 files changed, 147 insertions(+), 230 deletions(-) rename hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/{ITestEncryptionContext.java => ITestCustomEncryption.java} (52%) diff --git a/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/AzureBlobFileSystemStore.java b/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/AzureBlobFileSystemStore.java index 07f3b91a7a9f6..10bb27514c01d 100644 --- a/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/AzureBlobFileSystemStore.java +++ b/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/AzureBlobFileSystemStore.java @@ -870,9 +870,9 @@ public OutputStream openFileForWrite(final Path path, } AbfsLease lease = maybeCreateLease(relativePath, tracingContext); - byte[] encryptionContext = "context" -// op.getResult().getResponseHeader(HttpHeaderConfigurations.X_MS_ENCRYPTION_CONTEXT) - .getBytes(StandardCharsets.UTF_8); + byte[] encryptionContext = op.getResult() + .getResponseHeader(HttpHeaderConfigurations.X_MS_ENCRYPTION_CONTEXT) + .getBytes(StandardCharsets.UTF_8); EncryptionAdapter encryptionAdapter = new EncryptionAdapter( client.getEncryptionContextProvider(), getRelativePath(path), encryptionContext); @@ -1688,7 +1688,8 @@ private boolean parseIsDirectory(final String resourceType) { && resourceType.equalsIgnoreCase(AbfsHttpConstants.DIRECTORY); } - private String convertXmsPropertiesToCommaSeparatedString(final Hashtable properties) throws + public String convertXmsPropertiesToCommaSeparatedString(final Hashtable properties) throws CharacterCodingException { StringBuilder commaSeparatedProperties = new StringBuilder(); diff --git a/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/ITestEncryptionContext.java b/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/ITestCustomEncryption.java similarity index 52% rename from hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/ITestEncryptionContext.java rename to hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/ITestCustomEncryption.java index b6d9e2d4e9fa5..cb795fad0b6bf 100644 --- a/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/ITestEncryptionContext.java +++ b/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/ITestCustomEncryption.java @@ -21,38 +21,38 @@ import java.io.IOException; import java.nio.charset.StandardCharsets; import java.util.Arrays; -import java.util.Collections; +import java.util.Hashtable; -import org.apache.hadoop.fs.azurebfs.constants.FSOperationType; -import org.apache.hadoop.fs.azurebfs.extensions.EncryptionContextProvider; -import org.apache.hadoop.fs.azurebfs.security.EncryptionAdapter; -import org.apache.hadoop.fs.azurebfs.services.AbfsClient; -import org.apache.hadoop.fs.azurebfs.services.AbfsHttpOperation; -import org.apache.hadoop.fs.azurebfs.services.AbfsRestOperation; -import org.apache.hadoop.fs.permission.AclEntry; -import org.apache.hadoop.util.Lists; +import org.apache.hadoop.fs.permission.FsPermission; import org.assertj.core.api.Assertions; import org.junit.Test; +import org.junit.runner.RunWith; +import org.junit.runners.Parameterized; import org.apache.hadoop.conf.Configuration; import org.apache.hadoop.fs.FSDataInputStream; import org.apache.hadoop.fs.FSDataOutputStream; import org.apache.hadoop.fs.FileSystem; import org.apache.hadoop.fs.Path; +import org.apache.hadoop.fs.azurebfs.constants.FSOperationType; +import org.apache.hadoop.fs.azurebfs.contracts.services.AppendRequestParameters; +import org.apache.hadoop.fs.azurebfs.extensions.EncryptionContextProvider; import org.apache.hadoop.fs.azurebfs.extensions.MockEncryptionContextProvider; +import org.apache.hadoop.fs.azurebfs.security.EncryptionAdapter; +import org.apache.hadoop.fs.azurebfs.services.AbfsClient; +import org.apache.hadoop.fs.azurebfs.services.AbfsHttpOperation; +import org.apache.hadoop.fs.azurebfs.services.AbfsRestOperation; import org.apache.hadoop.fs.azurebfs.utils.EncryptionType; -import org.apache.hadoop.fs.permission.FsAction; -import org.apache.hadoop.fs.permission.FsPermission; +import org.apache.hadoop.fs.permission.AclEntry; import org.apache.hadoop.test.LambdaTestUtils; -import org.junit.runner.RunWith; -import org.junit.runners.Parameterized; -import org.junit.runners.Parameterized.Parameter; +import org.apache.hadoop.util.Lists; import static org.apache.hadoop.fs.azurebfs.constants.ConfigurationKeys.FS_AZURE_ENCRYPTION_CLIENT_PROVIDED_KEY; import static org.apache.hadoop.fs.azurebfs.constants.ConfigurationKeys.FS_AZURE_ENCRYPTION_CONTEXT_PROVIDER_TYPE; import static org.apache.hadoop.fs.azurebfs.constants.HttpHeaderConfigurations.X_MS_ENCRYPTION_KEY_SHA256; import static org.apache.hadoop.fs.azurebfs.constants.HttpHeaderConfigurations.X_MS_REQUEST_SERVER_ENCRYPTED; import static org.apache.hadoop.fs.azurebfs.constants.HttpHeaderConfigurations.X_MS_SERVER_ENCRYPTED; +import static org.apache.hadoop.fs.azurebfs.contracts.services.AppendRequestParameters.Mode.APPEND_MODE; import static org.apache.hadoop.fs.azurebfs.utils.AclTestHelpers.aclEntry; import static org.apache.hadoop.fs.azurebfs.utils.EncryptionType.ENCRYPTION_CONTEXT; import static org.apache.hadoop.fs.azurebfs.utils.EncryptionType.GLOBAL_KEY; @@ -62,14 +62,16 @@ import static org.apache.hadoop.fs.permission.FsAction.ALL; @RunWith(Parameterized.class) -public class ITestEncryptionContext extends AbstractAbfsIntegrationTest { +public class ITestCustomEncryption extends AbstractAbfsIntegrationTest { private final String cpk = "12345678901234567890123456789012"; private final String cpkSHAEncoded = EncryptionAdapter.getBase64EncodedString( EncryptionAdapter.getSHA256Hash(cpk)); + // Encryption type used by filesystem while creating file @Parameterized.Parameter public EncryptionType fileEncryptionType; + // Encryption type used by filesystem to call different operations @Parameterized.Parameter(1) public EncryptionType requestEncryptionType; @@ -89,81 +91,98 @@ public class ITestEncryptionContext extends AbstractAbfsIntegrationTest { public boolean isCpkResponseHdrExpected; - @Parameterized.Parameters(name = "{2}") + @Parameterized.Parameters(name = "{0} mode, {2}") public static Iterable params() { return Arrays.asList(new Object[][] { - {ENCRYPTION_CONTEXT, ENCRYPTION_CONTEXT, FSOperationType.READ, true, - false, false, true}, - {ENCRYPTION_CONTEXT, ENCRYPTION_CONTEXT, FSOperationType.WRITE, false, - true, false, true}, - {ENCRYPTION_CONTEXT, ENCRYPTION_CONTEXT, FSOperationType.SET_ACL, false, - false, false, false}, - {ENCRYPTION_CONTEXT, ENCRYPTION_CONTEXT, FSOperationType.GET_ATTR, false, - false, false, false}, - {ENCRYPTION_CONTEXT, ENCRYPTION_CONTEXT, FSOperationType.LISTSTATUS, false, - false, false, false}, - {ENCRYPTION_CONTEXT, NONE, FSOperationType.WRITE, false, false, true, - false}, - {ENCRYPTION_CONTEXT, NONE, FSOperationType.GET_ATTR, false, false, true, - false}, - {GLOBAL_KEY, GLOBAL_KEY, FSOperationType.READ, true, false, false, - true}, - {GLOBAL_KEY, GLOBAL_KEY, FSOperationType.WRITE, false, true, false, - true}, - {GLOBAL_KEY, GLOBAL_KEY, FSOperationType.SET_ACL, false, false, false, - false}, - {GLOBAL_KEY, GLOBAL_KEY, FSOperationType.LISTSTATUS, false, false, false, - false}, - {GLOBAL_KEY, GLOBAL_KEY, FSOperationType.RENAME, false, false, false, - false}, - {GLOBAL_KEY, GLOBAL_KEY, FSOperationType.DELETE, false, false, false, - false}, - {GLOBAL_KEY, GLOBAL_KEY, FSOperationType.GET_ATTR, false, false, false, - false}, - {GLOBAL_KEY, GLOBAL_KEY, FSOperationType.SET_ATTR, false, false, false, - false}, - {GLOBAL_KEY, NONE, FSOperationType.READ, true, false, true, - true}, - {GLOBAL_KEY, NONE, FSOperationType.WRITE, false, true, true, - true}, - {GLOBAL_KEY, NONE, FSOperationType.SET_ACL, false, false, false, - false}, - {GLOBAL_KEY, NONE, FSOperationType.RENAME, true, false, true, - true}, - {GLOBAL_KEY, NONE, FSOperationType.LISTSTATUS, false, true, true, - true}, - {GLOBAL_KEY, NONE, FSOperationType.DELETE, false, false, false, - false}, + {ENCRYPTION_CONTEXT, ENCRYPTION_CONTEXT, FSOperationType.READ, true, false, false, true}, + {ENCRYPTION_CONTEXT, ENCRYPTION_CONTEXT, FSOperationType.WRITE, false, true, false, true}, + {ENCRYPTION_CONTEXT, ENCRYPTION_CONTEXT, FSOperationType.APPEND, false, true, false, true}, + {ENCRYPTION_CONTEXT, ENCRYPTION_CONTEXT, FSOperationType.SET_ACL, false, false, false, false}, + {ENCRYPTION_CONTEXT, ENCRYPTION_CONTEXT, FSOperationType.GET_ATTR, true, false, false, true}, + {ENCRYPTION_CONTEXT, ENCRYPTION_CONTEXT, FSOperationType.SET_ATTR, false, true, false, true}, + {ENCRYPTION_CONTEXT, ENCRYPTION_CONTEXT, FSOperationType.LISTSTATUS, false, false, false, false}, + {ENCRYPTION_CONTEXT, ENCRYPTION_CONTEXT, FSOperationType.RENAME, false, false, false, false}, + {ENCRYPTION_CONTEXT, ENCRYPTION_CONTEXT, FSOperationType.DELETE, false, false, false, false}, + + {ENCRYPTION_CONTEXT, NONE, FSOperationType.WRITE, false, false, true, false}, + {ENCRYPTION_CONTEXT, NONE, FSOperationType.GET_ATTR, true, false, true, false}, + {ENCRYPTION_CONTEXT, NONE, FSOperationType.READ, false, false, true, false}, + {ENCRYPTION_CONTEXT, NONE, FSOperationType.SET_ATTR, false, true, true, false}, + {ENCRYPTION_CONTEXT, NONE, FSOperationType.RENAME, false, false, false, false}, + {ENCRYPTION_CONTEXT, NONE, FSOperationType.LISTSTATUS, false, false, false, false}, + {ENCRYPTION_CONTEXT, NONE, FSOperationType.DELETE, false, false, false, false}, + {ENCRYPTION_CONTEXT, NONE, FSOperationType.SET_ACL, false, false, false, false}, + {ENCRYPTION_CONTEXT, NONE, FSOperationType.SET_PERMISSION, false, false, false, false}, + + {GLOBAL_KEY, GLOBAL_KEY, FSOperationType.READ, true, false, false, true}, + {GLOBAL_KEY, GLOBAL_KEY, FSOperationType.WRITE, false, true, false, true}, + {GLOBAL_KEY, GLOBAL_KEY, FSOperationType.APPEND, false, true, false, true}, + {GLOBAL_KEY, GLOBAL_KEY, FSOperationType.SET_ACL, false, false, false, false}, + {GLOBAL_KEY, GLOBAL_KEY, FSOperationType.LISTSTATUS, false, false, false, false}, + {GLOBAL_KEY, GLOBAL_KEY, FSOperationType.RENAME, false, false, false, false}, + {GLOBAL_KEY, GLOBAL_KEY, FSOperationType.DELETE, false, false, false, false}, + {GLOBAL_KEY, GLOBAL_KEY, FSOperationType.GET_ATTR, true, false, false, true}, + {GLOBAL_KEY, GLOBAL_KEY, FSOperationType.SET_ATTR, false, true, false, true}, + + {GLOBAL_KEY, NONE, FSOperationType.READ, true, false, true, true}, + {GLOBAL_KEY, NONE, FSOperationType.WRITE, false, true, true, true}, + {GLOBAL_KEY, NONE, FSOperationType.SET_ATTR, false, false, true, true}, + {GLOBAL_KEY, NONE, FSOperationType.SET_ACL, false, false, false, false}, + {GLOBAL_KEY, NONE, FSOperationType.RENAME, false, false, false, false}, + {GLOBAL_KEY, NONE, FSOperationType.LISTSTATUS, false, false, false, false}, + {GLOBAL_KEY, NONE, FSOperationType.DELETE, false, false, false, false}, + {GLOBAL_KEY, NONE, FSOperationType.SET_PERMISSION, false, false, false, false}, }); } - public ITestEncryptionContext() throws Exception { + public ITestCustomEncryption() throws Exception { super(); } + @Test - public void runTest() throws Exception { + public void testCustomEncryptionCombinations() throws Exception { AzureBlobFileSystem fs = getFS(); Path testPath = path("/testFile"); - EncryptionContextProvider ecp = createEncryptedFile(testPath); - AbfsRestOperation op = callOperation(fs, testPath, ecp); + String relativePath = fs.getAbfsStore().getRelativePath(testPath); + MockEncryptionContextProvider ecp = + (MockEncryptionContextProvider) createEncryptedFile(testPath); + AbfsRestOperation op = callOperation(fs, new Path(relativePath), ecp); if (op == null) { return; } AbfsHttpOperation httpOp = op.getResult(); if (isCpkResponseHdrExpected) { - Assertions.assertThat(httpOp.getResponseHeader(X_MS_ENCRYPTION_KEY_SHA256)) - .isEqualTo(cpkSHAEncoded); - - if (operation == FSOperationType.READ) { - Assertions.assertThat(httpOp.getResponseHeader(X_MS_SERVER_ENCRYPTED)).isEqualTo("true"); - } else if (operation == FSOperationType.WRITE || operation == FSOperationType.APPEND) { - Assertions.assertThat(httpOp.getResponseHeader(X_MS_REQUEST_SERVER_ENCRYPTED)) - .isEqualTo("true"); + if (requestEncryptionType == ENCRYPTION_CONTEXT) { + String encryptionContext = ecp.getEncryptionContextForTest(relativePath); + String expectedKeySHA = EncryptionAdapter.getBase64EncodedString( + EncryptionAdapter.getSHA256Hash(ecp.getEncryptionKeyForTest(encryptionContext))); + Assertions.assertThat(httpOp.getResponseHeader(X_MS_ENCRYPTION_KEY_SHA256)) + .isEqualTo(expectedKeySHA); + } else { // GLOBAL_KEY + Assertions.assertThat(httpOp.getResponseHeader(X_MS_ENCRYPTION_KEY_SHA256)) + .isEqualTo(cpkSHAEncoded); } + } else { + Assertions.assertThat(httpOp.getResponseHeader(X_MS_ENCRYPTION_KEY_SHA256)) + .isEqualTo(null); } + Assertions.assertThat(httpOp.getResponseHeader(X_MS_SERVER_ENCRYPTED)) + .isEqualTo(responseHeaderServerEnc? "true" : null); + Assertions.assertThat(httpOp.getResponseHeader(X_MS_REQUEST_SERVER_ENCRYPTED)) + .isEqualTo(responseHeaderReqServerEnc? "true" : null); } + /** + * Executes a given operation at the AbfsClient level and returns + * AbfsRestOperation instance to verify response headers. Asserts excetion + * for combinations that should not succeed. + * @param fs AzureBlobFileSystem instance + * @param testPath path of file + * @param ecp EncryptionContextProvider instance to support AbfsClient methods + * @return Rest op or null depending on whether the request is allowed + * @throws Exception error + */ private AbfsRestOperation callOperation(AzureBlobFileSystem fs, Path testPath, EncryptionContextProvider ecp) throws Exception { @@ -180,45 +199,49 @@ private AbfsRestOperation callOperation(AzureBlobFileSystem fs, in.read(new byte[5]); } break; - case SET_ATTR: - fs.setXAttr(testPath, "attribute", "value".getBytes()); break; - case GET_ATTR: - fs.getXAttr(testPath, "attribute"); break; + case SET_ATTR: fs.setXAttr(testPath, "attribute", "value".getBytes()); + break; + case GET_ATTR: fs.getXAttr(testPath, "attribute"); + default: throw new NoSuchFieldException(); } }); return null; } else { EncryptionAdapter encryptionAdapter = null; if (fileEncryptionType == ENCRYPTION_CONTEXT) { - encryptionAdapter = new EncryptionAdapter( -// client.getEncryptionContextProvider(), - ecp, + encryptionAdapter = new EncryptionAdapter(ecp, fs.getAbfsStore().getRelativePath(testPath), "context".getBytes(StandardCharsets.UTF_8)); } - String path = fs.getAbfsStore().getRelativePath(testPath); + String path = testPath.toString(); switch (operation) { - case READ: - return client.read(path, 0, - new byte[5], - 0, 5, null, null, encryptionAdapter, - getTestTracingContext(fs, true)); - case WRITE: return client.flush(path, 3, false, false, null, null, - encryptionAdapter, getTestTracingContext(fs, false)); + case READ: return client.read(path, 0, new byte[5], 0, 5, null, + null, encryptionAdapter, getTestTracingContext(fs, true)); + case WRITE: return client.flush(path, 3, false, false, null, + null, encryptionAdapter, getTestTracingContext(fs, false)); + case APPEND: return client.append(path, "val".getBytes(), + new AppendRequestParameters(3, 0, 3, APPEND_MODE, false, null), + null, encryptionAdapter, getTestTracingContext(fs, false)); case SET_ACL: return client.setAcl(path, AclEntry.aclSpecToString( Lists.newArrayList(aclEntry(ACCESS, USER, ALL))), getTestTracingContext(fs, false)); case LISTSTATUS: return client.listPath(path, false, 5, null, getTestTracingContext(fs, true)); case RENAME: return client.renamePath(path, new Path(path + "_2").toString(), - null, - getTestTracingContext(fs, true)); -// return client.renamePath(new Path(path + "_2").toString(), path, null, -// getTestTracingContext(fs, true)); + null, getTestTracingContext(fs, true)); case DELETE: return client.deletePath(path, false, null, getTestTracingContext(fs, false)); case GET_ATTR: return client.getPathStatus(path, true, getTestTracingContext(fs, false)); + case SET_ATTR: + Hashtable properties = new Hashtable<>(); + properties.put("key", "{ value: valueTest }"); + return client.setPathProperties(path, fs.getAbfsStore() + .convertXmsPropertiesToCommaSeparatedString(properties), + getTestTracingContext(fs, false)); + case SET_PERMISSION: + return client.setPermission(path, FsPermission.getDefault().toString(), + getTestTracingContext(fs, false)); default: throw new NoSuchFieldException(); } } @@ -241,150 +264,32 @@ private AzureBlobFileSystem getCPKenabledFS() throws IOException { return (AzureBlobFileSystem) FileSystem.newInstance(conf); } - private AzureBlobFileSystem getEncryptionDisabledFS() throws IOException { - Configuration conf = getRawConfiguration(); - conf.unset(FS_AZURE_ENCRYPTION_CONTEXT_PROVIDER_TYPE); - return (AzureBlobFileSystem) FileSystem.newInstance(conf); - } - private AzureBlobFileSystem getFS() throws Exception { - AzureBlobFileSystem fs; + if (getFileSystem().getAbfsClient().getEncryptionType() == requestEncryptionType) { + return getFileSystem(); + } if (requestEncryptionType == ENCRYPTION_CONTEXT) { - fs = getECProviderEnabledFS(); + return getECProviderEnabledFS(); } else if (requestEncryptionType == GLOBAL_KEY) { - fs = getCPKenabledFS(); + return getCPKenabledFS(); } else { - fs = getEncryptionDisabledFS(); + Configuration conf = getRawConfiguration(); + conf.unset(FS_AZURE_ENCRYPTION_CONTEXT_PROVIDER_TYPE); + return (AzureBlobFileSystem) FileSystem.newInstance(conf); } - return fs; - } - - @Test - public void testCreateEncryptedFile() throws Exception { - Path testPath = path("/testFile"); - createEncryptedFile(testPath); - } - - @Test - public void testRead() throws Exception { - AzureBlobFileSystem fs = getFS(); - Path testPath = path("/testFile"); - AbfsClient client = fs.getAbfsClient(); - EncryptionAdapter encryptionAdapter = null; - if (fileEncryptionType == ENCRYPTION_CONTEXT) { - encryptionAdapter = new EncryptionAdapter( - client.getEncryptionContextProvider(), - fs.getAbfsStore().getRelativePath(testPath), - "context".getBytes(StandardCharsets.UTF_8)); - } - AbfsRestOperation op = client.read( - fs.getAbfsStore().getRelativePath(testPath), 0, new byte[5], 0, 5, null, - null, encryptionAdapter, getTestTracingContext(fs, true)); - checkEncryptionResponseHeaders(op.getResult(), FSOperationType.READ); - } - - @Test - public void testWrite() throws Exception { - AzureBlobFileSystem fs = getECProviderEnabledFS(); - } - - private void checkEncryptionResponseHeaders(AbfsHttpOperation op, - FSOperationType type) { - Assertions.assertThat(op.getResponseHeader(X_MS_ENCRYPTION_KEY_SHA256)).isEqualTo(cpkSHAEncoded); - if (type == FSOperationType.READ) { - Assertions.assertThat(op.getResponseHeader(X_MS_SERVER_ENCRYPTED)).isEqualTo("true"); - } else if (type == FSOperationType.WRITE || type == FSOperationType.APPEND) { - Assertions.assertThat(op.getResponseHeader(X_MS_REQUEST_SERVER_ENCRYPTED)) - .isEqualTo("true"); - } - } - - @Test - public void testOpenReadWrite() throws Exception { - AzureBlobFileSystem fs = getECProviderEnabledFS(); - Path testPath = path("/testFile"); - createEncryptedFile(testPath); - - try (FSDataOutputStream out = fs.append(testPath)) { - out.write("bytes".getBytes()); - out.flush(); - } - try (FSDataInputStream in = fs.open(testPath)) { - byte[] buffer = new byte[7]; - Assertions.assertThat(in.read(buffer)) - .describedAs("Incorrect read length").isEqualTo(5); - } - - AzureBlobFileSystem fs1 = getEncryptionDisabledFS(); - LambdaTestUtils.intercept(IOException.class, () -> { - try (FSDataOutputStream out = fs1.append(testPath)) { - out.write("bytes".getBytes(StandardCharsets.UTF_8)); - } - }); - LambdaTestUtils.intercept(IOException.class, () -> { - try (FSDataInputStream in = fs1.open(testPath)) { - in.read(new byte[5]); - } - }); - } - - @Test - public void testGetSetPathProperties() throws Exception { - AzureBlobFileSystem fs = getECProviderEnabledFS(); - Path testPath = path("/testFile"); - createEncryptedFile(testPath); - Assertions.assertThat(fs.getFileStatus(testPath)).isNotNull(); - byte[] value = "value".getBytes(StandardCharsets.UTF_8); - fs.setXAttr(testPath, "attribute", value); - assertArrayEquals(value, fs.getXAttr(testPath, "attribute")); - - AzureBlobFileSystem fs1 = getEncryptionDisabledFS(); - Assertions.assertThat(fs1.getFileStatus(testPath)).isNotNull(); - LambdaTestUtils.intercept(IOException.class, - () -> fs1.setXAttr(testPath, "attribute", value)); - LambdaTestUtils.intercept(IOException.class, - () -> fs1.getXAttr(testPath, "attribute")); - } - - @Test - public void testOtherApiWithoutEncryptionHeaders() throws Exception { - testOtherApi(getEncryptionDisabledFS()); - } - - @Test - public void testOtherApiWithEncryptionHeaders() throws Exception { - testOtherApi(getECProviderEnabledFS()); - } - - private void testOtherApi(AzureBlobFileSystem fs) - throws Exception { - Path testPath = path("/testFile"); - createEncryptedFile(testPath); - Assertions.assertThat(fs.listStatus(testPath)) - .describedAs("ListStatus should succeed without encryption headers") - .isNotEmpty(); - fs.setAcl(testPath, - Collections.singletonList(aclEntry(ACCESS, USER, FsAction.ALL))); - fs.modifyAclEntries(testPath, - Collections.singletonList(aclEntry(ACCESS, USER, FsAction.EXECUTE))); - fs.setPermission(testPath, FsPermission.getDefault()); - Assertions.assertThat(fs.getAclStatus(testPath)) - .describedAs("GetAcl should succeed without encryption headers") - .isNotNull(); - fs.access(testPath, FsAction.EXECUTE); - Path renamedPath = new Path(testPath + "_2"); - Assertions.assertThat(fs.rename(testPath, renamedPath)) - .describedAs("Rename should succeed without encryption headers") - .isTrue(); - Assertions.assertThat(fs.delete(renamedPath, false)) - .describedAs("Delete should succeed without encryption headers") - .isTrue(); } private EncryptionContextProvider createEncryptedFile(Path testPath) throws Exception { - AzureBlobFileSystem fs = fileEncryptionType == ENCRYPTION_CONTEXT? - getECProviderEnabledFS() : getCPKenabledFS(); - try (FSDataOutputStream out = fs.create(testPath)) { + AzureBlobFileSystem fs; + if (getFileSystem().getAbfsClient().getEncryptionType() == fileEncryptionType) { + fs = getFileSystem(); + } else { + fs = fileEncryptionType == ENCRYPTION_CONTEXT + ? getECProviderEnabledFS() + : getCPKenabledFS(); + } + String relativePath = fs.getAbfsStore().getRelativePath(testPath); + try (FSDataOutputStream out = fs.create(new Path(relativePath))) { out.write("123".getBytes()); } // verify file is encrypted by calling getPathStatus (with properties) @@ -398,4 +303,4 @@ private EncryptionContextProvider createEncryptedFile(Path testPath) throws Exce } return fs.getAbfsClient().getEncryptionContextProvider(); } -} \ No newline at end of file +} diff --git a/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/extensions/MockEncryptionContextProvider.java b/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/extensions/MockEncryptionContextProvider.java index aad0b33e5af04..0eb0e4099936b 100644 --- a/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/extensions/MockEncryptionContextProvider.java +++ b/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/extensions/MockEncryptionContextProvider.java @@ -25,6 +25,7 @@ import java.util.HashMap; import org.apache.hadoop.conf.Configuration; +import org.apache.hadoop.thirdparty.com.google.common.annotations.VisibleForTesting; public class MockEncryptionContextProvider implements EncryptionContextProvider { private String dummyKey = "12345678901234567890123456789012"; @@ -92,6 +93,16 @@ public void destroy() { Arrays.fill(key, (byte) 0); } } + + @VisibleForTesting + public String getEncryptionKeyForTest(String encryptionContext) { + return new String(contextToKeyMap.get(encryptionContext).getEncoded()); + } + + @VisibleForTesting + public String getEncryptionContextForTest(String path) { + return pathToContextMap.get(path); + } } From a3559a3319cb8820b6501fc51cc243c7c372d9e8 Mon Sep 17 00:00:00 2001 From: sumangala Date: Wed, 10 Nov 2021 08:13:35 +0530 Subject: [PATCH 19/77] address review comments --- .../hadoop/fs/azurebfs/AbfsConfiguration.java | 2 +- .../fs/azurebfs/AzureBlobFileSystemStore.java | 2 +- .../azurebfs/security/EncryptionAdapter.java | 6 +++-- .../fs/azurebfs/services/AbfsClient.java | 27 +++++++++++++------ 4 files changed, 25 insertions(+), 12 deletions(-) diff --git a/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/AbfsConfiguration.java b/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/AbfsConfiguration.java index 14c3af2bb09bf..45f830a396c75 100644 --- a/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/AbfsConfiguration.java +++ b/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/AbfsConfiguration.java @@ -907,7 +907,7 @@ public SASTokenProvider getSASTokenProvider() throws AzureBlobFileSystemExceptio } } - public EncryptionContextProvider initializeEncryptionContextProvider() { + public EncryptionContextProvider createEncryptionContextProvider() { try { String configKey = FS_AZURE_ENCRYPTION_CONTEXT_PROVIDER_TYPE; diff --git a/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/AzureBlobFileSystemStore.java b/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/AzureBlobFileSystemStore.java index 10bb27514c01d..e038cc70e3f67 100644 --- a/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/AzureBlobFileSystemStore.java +++ b/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/AzureBlobFileSystemStore.java @@ -1621,7 +1621,7 @@ private void initializeClient(URI uri, String fileSystemName, EncryptionContextProvider encryptionContextProvider = null; if (isSecure) { encryptionContextProvider = - abfsConfiguration.initializeEncryptionContextProvider(); + abfsConfiguration.createEncryptionContextProvider(); if (encryptionContextProvider != null) { if (abfsConfiguration.getClientProvidedEncryptionKey() != null) { throw new IOException( diff --git a/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/security/EncryptionAdapter.java b/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/security/EncryptionAdapter.java index 4a062f66874e1..60d2a9dfd2bbe 100644 --- a/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/security/EncryptionAdapter.java +++ b/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/security/EncryptionAdapter.java @@ -66,14 +66,16 @@ public SecretKey getEncryptionKey() throws IOException { return encryptionKey; } - public SecretKey fetchEncryptionContextAndComputeKeys() throws IOException { + public SecretKey createEncryptionContext() throws IOException { encryptionContext = provider.getEncryptionContext(path); - computeKeys(); + Preconditions.checkNotNull(encryptionContext, + "Encryption context should not be null."); return encryptionContext; } public void computeKeys() throws IOException { SecretKey key = getEncryptionKey(); + Preconditions.checkNotNull(key, "Encryption key should not be null."); encodedKey = getBase64EncodedString(new String(key.getEncoded(), StandardCharsets.UTF_8)); encodedKeySHA = getBase64EncodedString(getSHA256Hash(new String(key.getEncoded(), diff --git a/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/services/AbfsClient.java b/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/services/AbfsClient.java index 2c55b36e656f5..6ca3c7d06a0ed 100644 --- a/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/services/AbfsClient.java +++ b/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/services/AbfsClient.java @@ -238,10 +238,10 @@ private void addEncryptionKeyRequestHeaders(String path, if (isCreateFileRequest) { // get new context for create file request SecretKey encryptionContext = - encryptionAdapter.fetchEncryptionContextAndComputeKeys(); + encryptionAdapter.createEncryptionContext(); + encryptionAdapter.computeKeys(); requestHeaders.add(new AbfsHttpHeader(X_MS_ENCRYPTION_CONTEXT, - new String(encryptionContext.getEncoded(), - StandardCharsets.UTF_8))); + new String(encryptionContext.getEncoded(), StandardCharsets.UTF_8))); try { encryptionContext.destroy(); } catch (DestroyFailedException e) { @@ -250,11 +250,22 @@ private void addEncryptionKeyRequestHeaders(String path, } } else if (encryptionAdapter == null) { // get encryption context from GetPathStatus response header - encryptionAdapter = new EncryptionAdapter(encryptionContextProvider, - new Path(path).toUri().getPath(), - getPathStatus(path, false, tracingContext).getResult() - .getResponseHeader(X_MS_ENCRYPTION_CONTEXT) - .getBytes(StandardCharsets.UTF_8)); + byte[] encryptionContext; + try { + encryptionContext = getPathStatus(path, false, tracingContext) + .getResult().getResponseHeader(X_MS_ENCRYPTION_CONTEXT) + .getBytes(StandardCharsets.UTF_8); + } catch (IOException e) { + LOG.debug("GetPathStatus call to retrieve encryptionContext failed."); + throw e; + } + try { + encryptionAdapter = new EncryptionAdapter(encryptionContextProvider, + new Path(path).toUri().getPath(), encryptionContext); + } catch (IOException e) { + LOG.debug("Could not initialize EncryptionAdapter"); + throw e; + } } // else use cached encryption keys from input/output streams encodedKey = encryptionAdapter.getEncodedKey(); From 99cf2bd8fd973cd3ace1b591999ff180dde4c9b9 Mon Sep 17 00:00:00 2001 From: sumangala Date: Fri, 19 Nov 2021 09:29:20 +0530 Subject: [PATCH 20/77] test to use random context and key strings --- .../fs/azurebfs/AzureBlobFileSystemStore.java | 29 ++++++++++++------- .../fs/azurebfs/services/AbfsClient.java | 7 +++-- .../fs/azurebfs/ITestCustomEncryption.java | 3 +- .../MockEncryptionContextProvider.java | 10 +++---- 4 files changed, 30 insertions(+), 19 deletions(-) diff --git a/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/AzureBlobFileSystemStore.java b/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/AzureBlobFileSystemStore.java index e038cc70e3f67..6383403a2342a 100644 --- a/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/AzureBlobFileSystemStore.java +++ b/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/AzureBlobFileSystemStore.java @@ -788,10 +788,16 @@ public AbfsInputStream openFileForRead(Path path, op.getResponseHeader(HttpHeaderConfigurations.CONTENT_LENGTH)); eTag = op.getResponseHeader(HttpHeaderConfigurations.ETAG); if (client.getEncryptionType() == EncryptionType.ENCRYPTION_CONTEXT) { - encryptionAdapter = new EncryptionAdapter( - client.getEncryptionContextProvider(), getRelativePath(path), - op.getResponseHeader(HttpHeaderConfigurations.X_MS_ENCRYPTION_CONTEXT) - .getBytes(StandardCharsets.UTF_8)); + try { + encryptionAdapter = new EncryptionAdapter( + client.getEncryptionContextProvider(), getRelativePath(path), + op.getResponseHeader(HttpHeaderConfigurations.X_MS_ENCRYPTION_CONTEXT) + .getBytes(StandardCharsets.UTF_8)); + } catch (NullPointerException ex) { + LOG.debug("EncryptionContext missing in GetPathStatus response"); + throw new IOException( + "EncryptionContext not present in GetPathStatus response headers", ex); + } } } @@ -870,12 +876,15 @@ public OutputStream openFileForWrite(final Path path, } AbfsLease lease = maybeCreateLease(relativePath, tracingContext); - byte[] encryptionContext = op.getResult() - .getResponseHeader(HttpHeaderConfigurations.X_MS_ENCRYPTION_CONTEXT) - .getBytes(StandardCharsets.UTF_8); - EncryptionAdapter encryptionAdapter = new EncryptionAdapter( - client.getEncryptionContextProvider(), getRelativePath(path), - encryptionContext); + EncryptionAdapter encryptionAdapter = null; + if (client.getEncryptionType() == EncryptionType.ENCRYPTION_CONTEXT) { + byte[] encryptionContext = op.getResult() + .getResponseHeader(HttpHeaderConfigurations.X_MS_ENCRYPTION_CONTEXT) + .getBytes(StandardCharsets.UTF_8); + encryptionAdapter = new EncryptionAdapter( + client.getEncryptionContextProvider(), getRelativePath(path), + encryptionContext); + } return new AbfsOutputStream( populateAbfsOutputStreamContext( diff --git a/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/services/AbfsClient.java b/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/services/AbfsClient.java index 6ca3c7d06a0ed..ae95e474c8832 100644 --- a/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/services/AbfsClient.java +++ b/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/services/AbfsClient.java @@ -255,9 +255,10 @@ private void addEncryptionKeyRequestHeaders(String path, encryptionContext = getPathStatus(path, false, tracingContext) .getResult().getResponseHeader(X_MS_ENCRYPTION_CONTEXT) .getBytes(StandardCharsets.UTF_8); - } catch (IOException e) { - LOG.debug("GetPathStatus call to retrieve encryptionContext failed."); - throw e; + } catch (NullPointerException e) { + LOG.debug("EncryptionContext not present in GetPathStatus response."); + throw new IOException( + "EncryptionContext not present in GetPathStatus response", e); } try { encryptionAdapter = new EncryptionAdapter(encryptionContextProvider, diff --git a/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/ITestCustomEncryption.java b/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/ITestCustomEncryption.java index cb795fad0b6bf..38bc0a6d7bbac 100644 --- a/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/ITestCustomEncryption.java +++ b/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/ITestCustomEncryption.java @@ -211,7 +211,8 @@ private AbfsRestOperation callOperation(AzureBlobFileSystem fs, if (fileEncryptionType == ENCRYPTION_CONTEXT) { encryptionAdapter = new EncryptionAdapter(ecp, fs.getAbfsStore().getRelativePath(testPath), - "context".getBytes(StandardCharsets.UTF_8)); + ((MockEncryptionContextProvider) ecp).getEncryptionContextForTest(testPath.toString()) + .getBytes(StandardCharsets.UTF_8)); } String path = testPath.toString(); switch (operation) { diff --git a/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/extensions/MockEncryptionContextProvider.java b/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/extensions/MockEncryptionContextProvider.java index 0eb0e4099936b..1024bdcec4c3a 100644 --- a/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/extensions/MockEncryptionContextProvider.java +++ b/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/extensions/MockEncryptionContextProvider.java @@ -23,12 +23,13 @@ import java.nio.charset.StandardCharsets; import java.util.Arrays; import java.util.HashMap; +import java.util.UUID; +import org.apache.commons.lang3.RandomStringUtils; import org.apache.hadoop.conf.Configuration; import org.apache.hadoop.thirdparty.com.google.common.annotations.VisibleForTesting; public class MockEncryptionContextProvider implements EncryptionContextProvider { - private String dummyKey = "12345678901234567890123456789012"; private HashMap pathToContextMap = new HashMap<>(); private HashMap contextToKeyMap = new HashMap<>(); @Override @@ -39,11 +40,10 @@ public void initialize(Configuration configuration, String accountName, @Override public SecretKey getEncryptionContext(String path) throws IOException { - String newContext = "context"; + String newContext = UUID.randomUUID().toString(); pathToContextMap.put(path, newContext); - // String key = UUID.randomUUID().toString(); - Key key = new Key(dummyKey.getBytes(StandardCharsets.UTF_8)); - // replace dummyKey with key above once server supports + String keyString = RandomStringUtils.random(32, true, true); + Key key = new Key(keyString.getBytes(StandardCharsets.UTF_8)); contextToKeyMap.put(newContext, key); return new Key(newContext.getBytes(StandardCharsets.UTF_8)); } From 04a73e092253b0247644c74adf51a678d64ec013 Mon Sep 17 00:00:00 2001 From: sumangala Date: Mon, 22 Nov 2021 17:11:17 +0530 Subject: [PATCH 21/77] grp createpath args, pre-computed key configs, x-ms-version --- .../hadoop/fs/azurebfs/AbfsConfiguration.java | 9 +- .../fs/azurebfs/AzureBlobFileSystemStore.java | 69 +- .../azurebfs/constants/ConfigurationKeys.java | 8 +- .../azurebfs/security/EncryptionAdapter.java | 5 +- .../fs/azurebfs/services/AbfsClient.java | 54 +- .../ITestAzureBlobFileSystemCreate.java | 11 +- .../fs/azurebfs/ITestCustomEncryption.java | 18 +- .../fs/azurebfs/ITestCustomerProvidedKey.java | 985 ------------------ .../fs/azurebfs/TestTracingContext.java | 13 +- .../constants/TestConfigurationKeys.java | 1 + .../MockEncryptionContextProvider.java | 4 +- 11 files changed, 117 insertions(+), 1060 deletions(-) delete mode 100644 hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/ITestCustomerProvidedKey.java diff --git a/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/AbfsConfiguration.java b/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/AbfsConfiguration.java index fc43d8a6ff68c..7c433a638f2c3 100644 --- a/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/AbfsConfiguration.java +++ b/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/AbfsConfiguration.java @@ -1039,8 +1039,13 @@ public boolean enableAbfsListIterator() { return this.enableAbfsListIterator; } - public String getClientProvidedEncryptionKey() { - String accSpecEncKey = accountConf(FS_AZURE_ENCRYPTION_CLIENT_PROVIDED_KEY); + public String getEncodedClientProvidedEncryptionKey() { + String accSpecEncKey = accountConf(FS_AZURE_ENCRYPTION_ENCODED_CLIENT_PROVIDED_KEY); + return rawConfig.get(accSpecEncKey, null); + } + + public String getEncodedClientProvidedEncryptionKeySHA() { + String accSpecEncKey = accountConf(FS_AZURE_ENCRYPTION_ENCODED_CLIENT_PROVIDED_KEY_SHA); return rawConfig.get(accSpecEncKey, null); } diff --git a/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/AzureBlobFileSystemStore.java b/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/AzureBlobFileSystemStore.java index ed2aa92df19fc..dd79c944662b5 100644 --- a/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/AzureBlobFileSystemStore.java +++ b/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/AzureBlobFileSystemStore.java @@ -563,8 +563,7 @@ public OutputStream createFile(final Path path, if (triggerConditionalCreateOverwrite) { op = conditionalCreateOverwriteFile(relativePath, statistics, - isNamespaceEnabled ? getOctalNotation(permission) : null, - isNamespaceEnabled ? getOctalNotation(umask) : null, + new Permissions(isNamespaceEnabled, permission, umask), isAppendBlob, encryptionAdapter, tracingContext @@ -573,8 +572,7 @@ public OutputStream createFile(final Path path, } else { op = client.createPath(relativePath, true, overwrite, - isNamespaceEnabled ? getOctalNotation(permission) : null, - isNamespaceEnabled ? getOctalNotation(umask) : null, + new Permissions(isNamespaceEnabled, permission, umask), isAppendBlob, null, encryptionAdapter, @@ -603,16 +601,14 @@ public OutputStream createFile(final Path path, * only if there is match for eTag of existing file. * @param relativePath * @param statistics - * @param permission - * @param umask + * @param permissions contains permission and umask * @param isAppendBlob * @return * @throws AzureBlobFileSystemException */ private AbfsRestOperation conditionalCreateOverwriteFile(final String relativePath, final FileSystem.Statistics statistics, - final String permission, - final String umask, + Permissions permissions, final boolean isAppendBlob, EncryptionAdapter encryptionAdapter, TracingContext tracingContext) throws IOException { @@ -622,7 +618,7 @@ private AbfsRestOperation conditionalCreateOverwriteFile(final String relativePa // Trigger a create with overwrite=false first so that eTag fetch can be // avoided for cases when no pre-existing file is present (major portion // of create file traffic falls into the case of no pre-existing file). - op = client.createPath(relativePath, true, false, permission, umask, + op = client.createPath(relativePath, true, false, permissions, isAppendBlob, null, encryptionAdapter, tracingContext); } catch (AbfsRestOperationException e) { @@ -647,7 +643,7 @@ private AbfsRestOperation conditionalCreateOverwriteFile(final String relativePa try { // overwrite only if eTag matches with the file properties fetched befpre - op = client.createPath(relativePath, true, true, permission, umask, + op = client.createPath(relativePath, true, true, permissions, isAppendBlob, eTag, encryptionAdapter, tracingContext); } catch (AbfsRestOperationException ex) { if (ex.getStatusCode() == HttpURLConnection.HTTP_PRECON_FAILED) { @@ -734,11 +730,10 @@ public void createDirectory(final Path path, final FsPermission permission, boolean overwrite = !isNamespaceEnabled || abfsConfiguration.isEnabledMkdirOverwrite(); + Permissions permissions = new Permissions(isNamespaceEnabled, + permission, umask); final AbfsRestOperation op = client.createPath(getRelativePath(path), - false, overwrite, - isNamespaceEnabled ? getOctalNotation(permission) : null, - isNamespaceEnabled ? getOctalNotation(umask) : null, false, - null, null, tracingContext); + false, overwrite, permissions, false, null, null, tracingContext); perfInfo.registerResult(op.getResult()).registerSuccess(true); } } @@ -1632,7 +1627,7 @@ private void initializeClient(URI uri, String fileSystemName, encryptionContextProvider = abfsConfiguration.createEncryptionContextProvider(); if (encryptionContextProvider != null) { - if (abfsConfiguration.getClientProvidedEncryptionKey() != null) { + if (abfsConfiguration.getEncodedClientProvidedEncryptionKey() != null) { throw new IOException( "Both global key and encryption context are set, only one allowed"); } @@ -1640,8 +1635,13 @@ private void initializeClient(URI uri, String fileSystemName, abfsConfiguration.getRawConfiguration(), accountName, fileSystemName); encryptionType = EncryptionType.ENCRYPTION_CONTEXT; - } else if (abfsConfiguration.getClientProvidedEncryptionKey() != null) { - encryptionType = EncryptionType.GLOBAL_KEY; + } else if (abfsConfiguration.getEncodedClientProvidedEncryptionKey() != null) { + if (abfsConfiguration.getEncodedClientProvidedEncryptionKeySHA() != null) { + encryptionType = EncryptionType.GLOBAL_KEY; + } else { + throw new IOException( + "Encoded SHA256 hash must be provided for global encryption"); + } } } @@ -1655,7 +1655,6 @@ private void initializeClient(URI uri, String fileSystemName, sasTokenProvider, encryptionContextProvider, populateAbfsClientContext()); } - client.setEncryptionType(encryptionType); LOG.trace("AbfsClient init complete"); } @@ -1674,11 +1673,6 @@ private AbfsClientContext populateAbfsClientContext() { .build(); } - private String getOctalNotation(FsPermission fsPermission) { - Preconditions.checkNotNull(fsPermission, "fsPermission"); - return String.format(AbfsHttpConstants.PERMISSION_FORMAT, fsPermission.toOctal()); - } - public String getRelativePath(final Path path) { Preconditions.checkNotNull(path, "path"); return path.toUri().getPath(); @@ -1866,6 +1860,35 @@ public String toString() { } } + public static class Permissions { + private final String permission; + private final String umask; + + Permissions(boolean isNamespaceEnabled, FsPermission permission, + FsPermission umask) { + if (isNamespaceEnabled) { + this.permission = getOctalNotation(permission); + this.umask = getOctalNotation(umask); + } else { + this.permission = null; + this.umask = null; + } + } + + private String getOctalNotation(FsPermission fsPermission) { + Preconditions.checkNotNull(fsPermission, "fsPermission"); + return String.format(AbfsHttpConstants.PERMISSION_FORMAT, fsPermission.toOctal()); + } + + public String getPermission() { + return permission; + } + + public String getUmask() { + return umask; + } + } + /** * A builder class for AzureBlobFileSystemStore. */ diff --git a/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/constants/ConfigurationKeys.java b/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/constants/ConfigurationKeys.java index 4c46fada2a7d2..4c608d9312f8a 100644 --- a/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/constants/ConfigurationKeys.java +++ b/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/constants/ConfigurationKeys.java @@ -188,8 +188,12 @@ public final class ConfigurationKeys { public static final String AZURE_KEY_ACCOUNT_SHELLKEYPROVIDER_SCRIPT = "fs.azure.shellkeyprovider.script"; /** Setting this true will make the driver use it's own RemoteIterator implementation */ public static final String FS_AZURE_ENABLE_ABFS_LIST_ITERATOR = "fs.azure.enable.abfslistiterator"; - /** Server side encryption key */ - public static final String FS_AZURE_ENCRYPTION_CLIENT_PROVIDED_KEY = "fs.azure.encryption.client-provided-key"; + /** Server side encryption key encoded in Base6format */ + public static final String FS_AZURE_ENCRYPTION_ENCODED_CLIENT_PROVIDED_KEY = + "fs.azure.encryption.encoded.client-provided-key"; + /** SHA256 hash of encryption key encoded in Base6format */ + public static final String FS_AZURE_ENCRYPTION_ENCODED_CLIENT_PROVIDED_KEY_SHA = + "fs.azure.encryption.encoded.client-provided-key-sha"; /** Custom EncryptionContextProvider type */ public static final String FS_AZURE_ENCRYPTION_CONTEXT_PROVIDER_TYPE = "fs.azure.encryption.context.provider.type"; diff --git a/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/security/EncryptionAdapter.java b/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/security/EncryptionAdapter.java index 60d2a9dfd2bbe..575660c85e9a6 100644 --- a/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/security/EncryptionAdapter.java +++ b/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/security/EncryptionAdapter.java @@ -76,8 +76,9 @@ public SecretKey createEncryptionContext() throws IOException { public void computeKeys() throws IOException { SecretKey key = getEncryptionKey(); Preconditions.checkNotNull(key, "Encryption key should not be null."); - encodedKey = getBase64EncodedString(new String(key.getEncoded(), - StandardCharsets.UTF_8)); +// encodedKey = getBase64EncodedString(new String(key.getEncoded(), +// StandardCharsets.UTF_8)); + encodedKey = getBase64EncodedString(key.getEncoded()); encodedKeySHA = getBase64EncodedString(getSHA256Hash(new String(key.getEncoded(), StandardCharsets.UTF_8))); } diff --git a/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/services/AbfsClient.java b/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/services/AbfsClient.java index a06d8cf139961..3e45439caaf7f 100644 --- a/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/services/AbfsClient.java +++ b/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/services/AbfsClient.java @@ -26,9 +26,6 @@ import java.net.URL; import java.net.URLEncoder; import java.nio.charset.StandardCharsets; -import java.time.Instant; -import java.security.MessageDigest; -import java.security.NoSuchAlgorithmException; import java.util.ArrayList; import java.util.List; import java.util.Locale; @@ -39,6 +36,7 @@ import org.apache.hadoop.classification.VisibleForTesting; import org.apache.hadoop.fs.Path; +import org.apache.hadoop.fs.azurebfs.AzureBlobFileSystemStore.Permissions; import org.apache.hadoop.fs.azurebfs.extensions.EncryptionContextProvider; import org.apache.hadoop.fs.azurebfs.security.EncryptionAdapter; import org.apache.hadoop.fs.azurebfs.utils.EncryptionType; @@ -88,22 +86,22 @@ public class AbfsClient implements Closeable { private final URL baseUrl; private final SharedKeyCredentials sharedKeyCredentials; - private final String xMsVersion = "2021-04-10";//"2019-12-12"; + private String xMsVersion = "2019-12-12"; private final ExponentialRetryPolicy retryPolicy; private final String filesystem; private final AbfsConfiguration abfsConfiguration; private final String userAgent; private final AbfsPerfTracker abfsPerfTracker; - private String clientProvidedEncryptionKey; - private final String clientProvidedEncryptionKeySHA; + private String clientProvidedEncryptionKey = null; + private String clientProvidedEncryptionKeySHA = null; private final String accountName; private final AuthType authType; private AccessTokenProvider tokenProvider; private SASTokenProvider sasTokenProvider; private final AbfsCounters abfsCounters; - private EncryptionContextProvider encryptionContextProvider; - private EncryptionType encryptionType; + private EncryptionContextProvider encryptionContextProvider = null; + private EncryptionType encryptionType = EncryptionType.NONE; private final ListeningScheduledExecutorService executorService; @@ -115,22 +113,23 @@ private AbfsClient(final URL baseUrl, this.baseUrl = baseUrl; this.sharedKeyCredentials = sharedKeyCredentials; String baseUrlString = baseUrl.toString(); - this.filesystem = baseUrlString.substring(baseUrlString.lastIndexOf(FORWARD_SLASH) + 1); + this.filesystem = baseUrlString.substring( + baseUrlString.lastIndexOf(FORWARD_SLASH) + 1); this.abfsConfiguration = abfsConfiguration; this.retryPolicy = abfsClientContext.getExponentialRetryPolicy(); - this.accountName = abfsConfiguration.getAccountName().substring(0, abfsConfiguration.getAccountName().indexOf(AbfsHttpConstants.DOT)); + this.accountName = abfsConfiguration.getAccountName().substring(0, + abfsConfiguration.getAccountName().indexOf(AbfsHttpConstants.DOT)); this.authType = abfsConfiguration.getAuthType(accountName); - this.encryptionContextProvider = encryptionContextProvider; - - String encryptionKey = this.abfsConfiguration - .getClientProvidedEncryptionKey(); - if (encryptionKey != null) { - this.clientProvidedEncryptionKey = EncryptionAdapter.getBase64EncodedString(encryptionKey); - this.clientProvidedEncryptionKeySHA = EncryptionAdapter.getBase64EncodedString( - EncryptionAdapter.getSHA256Hash(encryptionKey)); - } else { - this.clientProvidedEncryptionKey = null; - this.clientProvidedEncryptionKeySHA = null; + + if (encryptionContextProvider != null) { + this.encryptionContextProvider = encryptionContextProvider; + xMsVersion = "2021-04-10"; // will be default once server change deployed + encryptionType = EncryptionType.ENCRYPTION_CONTEXT; + } else if ((clientProvidedEncryptionKey = + abfsConfiguration.getEncodedClientProvidedEncryptionKey()) != null) { + this.clientProvidedEncryptionKeySHA = + abfsConfiguration.getEncodedClientProvidedEncryptionKeySHA(); + encryptionType = EncryptionType.GLOBAL_KEY; } String sslProviderName = null; @@ -389,7 +388,7 @@ public AbfsRestOperation deleteFilesystem(TracingContext tracingContext) throws } public AbfsRestOperation createPath(final String path, final boolean isFile, - final boolean overwrite, final String permission, final String umask, + final boolean overwrite, final Permissions permissions, final boolean isAppendBlob, final String eTag, EncryptionAdapter encryptionAdapter, TracingContext tracingContext) throws IOException { @@ -402,12 +401,15 @@ public AbfsRestOperation createPath(final String path, final boolean isFile, requestHeaders.add(new AbfsHttpHeader(IF_NONE_MATCH, AbfsHttpConstants.STAR)); } - if (permission != null && !permission.isEmpty()) { - requestHeaders.add(new AbfsHttpHeader(HttpHeaderConfigurations.X_MS_PERMISSIONS, permission)); + if (permissions.getPermission() != null && !permissions.getPermission().isEmpty()) { + requestHeaders.add( + new AbfsHttpHeader(HttpHeaderConfigurations.X_MS_PERMISSIONS, + permissions.getPermission())); } - if (umask != null && !umask.isEmpty()) { - requestHeaders.add(new AbfsHttpHeader(HttpHeaderConfigurations.X_MS_UMASK, umask)); + if (permissions.getUmask() != null && !permissions.getUmask().isEmpty()) { + requestHeaders.add(new AbfsHttpHeader(HttpHeaderConfigurations.X_MS_UMASK, + permissions.getUmask())); } if (eTag != null && !eTag.isEmpty()) { diff --git a/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/ITestAzureBlobFileSystemCreate.java b/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/ITestAzureBlobFileSystemCreate.java index ca0782ec20092..44e6a0f08d0dd 100644 --- a/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/ITestAzureBlobFileSystemCreate.java +++ b/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/ITestAzureBlobFileSystemCreate.java @@ -25,6 +25,7 @@ import java.util.EnumSet; import java.util.UUID; +import org.apache.hadoop.fs.azurebfs.AzureBlobFileSystemStore.Permissions; import org.junit.Test; import org.apache.hadoop.conf.Configuration; @@ -398,9 +399,8 @@ public void testNegativeScenariosForCreateOverwriteDisabled() serverErrorResponseEx) // Scn5: create overwrite=false fails with Http500 .when(mockClient) .createPath(any(String.class), eq(true), eq(false), - isNamespaceEnabled ? any(String.class) : eq(null), - isNamespaceEnabled ? any(String.class) : eq(null), - any(boolean.class), eq(null), any(), any(TracingContext.class)); + any(Permissions.class), any(boolean.class), eq(null), any(), + any(TracingContext.class)); doThrow(fileNotFoundResponseEx) // Scn1: GFS fails with Http404 .doThrow(serverErrorResponseEx) // Scn2: GFS fails with Http500 @@ -416,9 +416,8 @@ public void testNegativeScenariosForCreateOverwriteDisabled() serverErrorResponseEx) // Scn4: create overwrite=true fails with Http500 .when(mockClient) .createPath(any(String.class), eq(true), eq(true), - isNamespaceEnabled ? any(String.class) : eq(null), - isNamespaceEnabled ? any(String.class) : eq(null), - any(boolean.class), eq(null), any(), any(TracingContext.class)); + any(Permissions.class), any(boolean.class), eq(null), any(), + any(TracingContext.class)); // Scn1: GFS fails with Http404 // Sequence of events expected: diff --git a/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/ITestCustomEncryption.java b/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/ITestCustomEncryption.java index 38bc0a6d7bbac..dd8c7a1a8eb0f 100644 --- a/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/ITestCustomEncryption.java +++ b/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/ITestCustomEncryption.java @@ -47,8 +47,9 @@ import org.apache.hadoop.test.LambdaTestUtils; import org.apache.hadoop.util.Lists; -import static org.apache.hadoop.fs.azurebfs.constants.ConfigurationKeys.FS_AZURE_ENCRYPTION_CLIENT_PROVIDED_KEY; import static org.apache.hadoop.fs.azurebfs.constants.ConfigurationKeys.FS_AZURE_ENCRYPTION_CONTEXT_PROVIDER_TYPE; +import static org.apache.hadoop.fs.azurebfs.constants.ConfigurationKeys.FS_AZURE_ENCRYPTION_ENCODED_CLIENT_PROVIDED_KEY; +import static org.apache.hadoop.fs.azurebfs.constants.ConfigurationKeys.FS_AZURE_ENCRYPTION_ENCODED_CLIENT_PROVIDED_KEY_SHA; import static org.apache.hadoop.fs.azurebfs.constants.HttpHeaderConfigurations.X_MS_ENCRYPTION_KEY_SHA256; import static org.apache.hadoop.fs.azurebfs.constants.HttpHeaderConfigurations.X_MS_REQUEST_SERVER_ENCRYPTED; import static org.apache.hadoop.fs.azurebfs.constants.HttpHeaderConfigurations.X_MS_SERVER_ENCRYPTED; @@ -252,15 +253,22 @@ private AzureBlobFileSystem getECProviderEnabledFS() throws Exception { Configuration configuration = getRawConfiguration(); configuration.set(FS_AZURE_ENCRYPTION_CONTEXT_PROVIDER_TYPE, MockEncryptionContextProvider.class.getCanonicalName()); - configuration.unset( - FS_AZURE_ENCRYPTION_CLIENT_PROVIDED_KEY + "." + getAccountName()); + configuration.unset(FS_AZURE_ENCRYPTION_ENCODED_CLIENT_PROVIDED_KEY + "." + + getAccountName()); + configuration.unset(FS_AZURE_ENCRYPTION_ENCODED_CLIENT_PROVIDED_KEY_SHA + "." + + getAccountName()); return (AzureBlobFileSystem) FileSystem.newInstance(configuration); } private AzureBlobFileSystem getCPKenabledFS() throws IOException { Configuration conf = getRawConfiguration(); - conf.set(FS_AZURE_ENCRYPTION_CLIENT_PROVIDED_KEY + "." + getAccountName(), - cpk); + String cpkEncoded = EncryptionAdapter.getBase64EncodedString(cpk); + String cpkEncodedSHA = EncryptionAdapter.getBase64EncodedString( + EncryptionAdapter.getSHA256Hash(cpk)); + conf.set(FS_AZURE_ENCRYPTION_ENCODED_CLIENT_PROVIDED_KEY + "." + + getAccountName(), cpkEncoded); + conf.set(FS_AZURE_ENCRYPTION_ENCODED_CLIENT_PROVIDED_KEY_SHA + "." + + getAccountName(), cpkEncodedSHA); conf.unset(FS_AZURE_ENCRYPTION_CONTEXT_PROVIDER_TYPE); return (AzureBlobFileSystem) FileSystem.newInstance(conf); } diff --git a/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/ITestCustomerProvidedKey.java b/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/ITestCustomerProvidedKey.java deleted file mode 100644 index a32e5a33061d8..0000000000000 --- a/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/ITestCustomerProvidedKey.java +++ /dev/null @@ -1,985 +0,0 @@ -/** - * 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 ("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.fs.azurebfs; - -import java.io.FileNotFoundException; -import java.io.IOException; -import java.nio.CharBuffer; -import java.nio.charset.CharacterCodingException; -import java.nio.charset.Charset; -import java.nio.charset.CharsetEncoder; -import java.nio.charset.StandardCharsets; -import java.security.MessageDigest; -import java.security.NoSuchAlgorithmException; -import java.util.EnumSet; -import java.util.Hashtable; -import java.util.List; -import java.util.Map; -import java.util.Optional; -import java.util.Random; -import java.util.UUID; - -import org.apache.hadoop.fs.azurebfs.security.EncryptionAdapter; -import org.apache.hadoop.fs.azurebfs.utils.TracingContext; -import org.apache.hadoop.fs.contract.ContractTestUtils; -import org.apache.hadoop.thirdparty.com.google.common.base.Preconditions; -import org.apache.hadoop.util.Lists; -import org.assertj.core.api.Assertions; -import org.junit.Assume; -import org.junit.Test; -import org.slf4j.Logger; -import org.slf4j.LoggerFactory; - -import org.apache.hadoop.conf.Configuration; -import org.apache.hadoop.fs.FSDataInputStream; -import org.apache.hadoop.fs.FSDataOutputStream; -import org.apache.hadoop.fs.FileStatus; -import org.apache.hadoop.fs.FileSystem; -import org.apache.hadoop.fs.Path; -import org.apache.hadoop.fs.XAttrSetFlag; -import org.apache.hadoop.fs.azurebfs.constants.AbfsHttpConstants; -import org.apache.hadoop.fs.azurebfs.constants.HttpHeaderConfigurations; -import org.apache.hadoop.fs.azurebfs.contracts.services.AppendRequestParameters; -import org.apache.hadoop.fs.azurebfs.contracts.services.AppendRequestParameters.Mode; -import org.apache.hadoop.fs.azurebfs.services.AbfsAclHelper; -import org.apache.hadoop.fs.azurebfs.services.AbfsClient; -import org.apache.hadoop.fs.azurebfs.services.AbfsHttpHeader; -import org.apache.hadoop.fs.azurebfs.services.AbfsHttpOperation; -import org.apache.hadoop.fs.azurebfs.services.AbfsRestOperation; -import org.apache.hadoop.fs.azurebfs.services.AuthType; -import org.apache.hadoop.fs.azurebfs.utils.Base64; -import org.apache.hadoop.fs.permission.AclEntry; -import org.apache.hadoop.fs.permission.FsAction; -import org.apache.hadoop.fs.permission.FsPermission; -import org.apache.hadoop.test.LambdaTestUtils; - -import static org.apache.hadoop.fs.azurebfs.constants.ConfigurationKeys.AZURE_CREATE_REMOTE_FILESYSTEM_DURING_INITIALIZATION; -import static org.apache.hadoop.fs.azurebfs.constants.ConfigurationKeys.FS_AZURE_ENCRYPTION_CLIENT_PROVIDED_KEY; -import static org.apache.hadoop.fs.azurebfs.constants.ConfigurationKeys.FS_AZURE_ENCRYPTION_CONTEXT_PROVIDER_TYPE; -import static org.apache.hadoop.fs.azurebfs.constants.FileSystemConfigurations.ONE_MB; -import static org.apache.hadoop.fs.azurebfs.constants.HttpHeaderConfigurations.X_MS_ENCRYPTION_ALGORITHM; -import static org.apache.hadoop.fs.azurebfs.constants.HttpHeaderConfigurations.X_MS_ENCRYPTION_KEY; -import static org.apache.hadoop.fs.azurebfs.constants.HttpHeaderConfigurations.X_MS_ENCRYPTION_KEY_SHA256; -import static org.apache.hadoop.fs.azurebfs.constants.HttpHeaderConfigurations.X_MS_REQUEST_SERVER_ENCRYPTED; -import static org.apache.hadoop.fs.azurebfs.constants.HttpHeaderConfigurations.X_MS_SERVER_ENCRYPTED; -import static org.apache.hadoop.fs.azurebfs.constants.TestConfigurationKeys.FS_AZURE_ABFS_ACCOUNT_NAME; -import static org.apache.hadoop.fs.azurebfs.constants.TestConfigurationKeys.FS_AZURE_ACCOUNT_KEY; -import static org.apache.hadoop.fs.azurebfs.constants.TestConfigurationKeys.FS_AZURE_TEST_CPK_ENABLED; -import static org.apache.hadoop.fs.azurebfs.constants.TestConfigurationKeys.FS_AZURE_TEST_CPK_ENABLED_SECONDARY_ACCOUNT; -import static org.apache.hadoop.fs.azurebfs.constants.TestConfigurationKeys.FS_AZURE_TEST_CPK_ENABLED_SECONDARY_ACCOUNT_KEY; -import static org.apache.hadoop.fs.azurebfs.constants.TestConfigurationKeys.FS_AZURE_TEST_NAMESPACE_ENABLED_ACCOUNT; -import static org.apache.hadoop.fs.azurebfs.constants.TestConfigurationKeys.TEST_CONFIGURATION_FILE_NAME; -import static org.apache.hadoop.fs.azurebfs.utils.AclTestHelpers.aclEntry; -import static org.apache.hadoop.fs.permission.AclEntryScope.ACCESS; -import static org.apache.hadoop.fs.permission.AclEntryType.USER; -import static org.apache.hadoop.fs.permission.FsAction.ALL; - -public class ITestCustomerProvidedKey extends AbstractAbfsIntegrationTest { - private static final Logger LOG = LoggerFactory - .getLogger(ITestCustomerProvidedKey.class); - - private static final String XMS_PROPERTIES_ENCODING = "ISO-8859-1"; - private static final int INT_512 = 512; - private static final int INT_50 = 50; - private static final int ENCRYPTION_KEY_LEN = 32; - private static final int FILE_SIZE = 10 * ONE_MB; - private static final int FILE_SIZE_FOR_COPY_BETWEEN_ACCOUNTS = 24 * ONE_MB; - - public ITestCustomerProvidedKey() throws Exception { - boolean isCPKTestsEnabled = getConfiguration() - .getBoolean(FS_AZURE_TEST_CPK_ENABLED, false); - Assume.assumeTrue(isCPKTestsEnabled); - Assume.assumeTrue( - getConfiguration().get(FS_AZURE_ENCRYPTION_CONTEXT_PROVIDER_TYPE) - == null); - } - - @Test - public void testReadWithCPK() throws Exception { - final AzureBlobFileSystem fs = getAbfs(true); - String fileName = new Path("/" + methodName.getMethodName()).toString(); - createFileAndGetContent(fs, fileName, FILE_SIZE); - - AbfsClient abfsClient = fs.getAbfsClient(); - int length = FILE_SIZE; - byte[] buffer = new byte[length]; - TracingContext tracingContext = getTestTracingContext(fs, false); - final AbfsRestOperation op = abfsClient.getPathStatus(fileName, false, - tracingContext); - final String eTag = op.getResult() - .getResponseHeader(HttpHeaderConfigurations.ETAG); - AbfsRestOperation abfsRestOperation = abfsClient - .read(fileName, 0, buffer, 0, length, eTag, null, null, tracingContext); - assertCPKHeaders(abfsRestOperation, true); - assertResponseHeader(abfsRestOperation, true, X_MS_ENCRYPTION_KEY_SHA256, - getCPKSha(fs)); - assertResponseHeader(abfsRestOperation, true, X_MS_SERVER_ENCRYPTED, - "true"); - assertResponseHeader(abfsRestOperation, false, - X_MS_REQUEST_SERVER_ENCRYPTED, ""); - - // Trying to read with different CPK headers - Configuration conf = fs.getConf(); - String accountName = conf.get(FS_AZURE_ABFS_ACCOUNT_NAME); - conf.set(FS_AZURE_ENCRYPTION_CLIENT_PROVIDED_KEY + "." + accountName, - "different-1234567890123456789012"); - try (AzureBlobFileSystem fs2 = (AzureBlobFileSystem) FileSystem.newInstance(conf); - FSDataInputStream iStream = fs2.open(new Path(fileName))) { - int len = 8 * ONE_MB; - byte[] b = new byte[len]; - LambdaTestUtils.intercept(IOException.class, () -> { - iStream.read(b, 0, len); - }); - } - - // Trying to read with no CPK headers - conf.unset(FS_AZURE_ENCRYPTION_CLIENT_PROVIDED_KEY + "." + accountName); - try (AzureBlobFileSystem fs3 = (AzureBlobFileSystem) FileSystem - .get(conf); FSDataInputStream iStream = fs3.open(new Path(fileName))) { - int len = 8 * ONE_MB; - byte[] b = new byte[len]; - LambdaTestUtils.intercept(IOException.class, () -> { - iStream.read(b, 0, len); - }); - } - } - - @Test - public void testReadWithoutCPK() throws Exception { - final AzureBlobFileSystem fs = getAbfs(false); - String fileName = new Path("/" + methodName.getMethodName()).toString(); - createFileAndGetContent(fs, fileName, FILE_SIZE); - - AbfsClient abfsClient = fs.getAbfsClient(); - int length = INT_512; - byte[] buffer = new byte[length * 4]; - TracingContext tracingContext = getTestTracingContext(fs, false); - final AbfsRestOperation op = abfsClient - .getPathStatus(fileName, false, tracingContext); - final String eTag = op.getResult() - .getResponseHeader(HttpHeaderConfigurations.ETAG); - AbfsRestOperation abfsRestOperation = abfsClient - .read(fileName, 0, buffer, 0, length, eTag, null, null, tracingContext); - assertCPKHeaders(abfsRestOperation, false); - assertResponseHeader(abfsRestOperation, false, X_MS_ENCRYPTION_KEY_SHA256, - getCPKSha(fs)); - assertResponseHeader(abfsRestOperation, true, X_MS_SERVER_ENCRYPTED, - "true"); - assertResponseHeader(abfsRestOperation, false, - X_MS_REQUEST_SERVER_ENCRYPTED, ""); - - // Trying to read with CPK headers - Configuration conf = fs.getConf(); - String accountName = conf.get(FS_AZURE_ABFS_ACCOUNT_NAME); - conf.set(FS_AZURE_ENCRYPTION_CLIENT_PROVIDED_KEY + "." + accountName, - "12345678901234567890123456789012"); - - try (AzureBlobFileSystem fs2 = (AzureBlobFileSystem) FileSystem.newInstance(conf); - AbfsClient abfsClient2 = fs2.getAbfsClient()) { - LambdaTestUtils.intercept(IOException.class, () -> { - abfsClient2.read(fileName, 0, buffer, 0, length, eTag, null, - null, getTestTracingContext(fs, false)); - }); - } - } - - @Test - public void testAppendWithCPK() throws Exception { - final AzureBlobFileSystem fs = getAbfs(true); - final String fileName = - new Path("/" + methodName.getMethodName()).toString(); - createFileAndGetContent(fs, fileName, FILE_SIZE); - - // Trying to append with correct CPK headers - AppendRequestParameters appendRequestParameters = - new AppendRequestParameters( - 0, 0, 5, Mode.APPEND_MODE, false, null); - byte[] buffer = getRandomBytesArray(5); - AbfsClient abfsClient = fs.getAbfsClient(); - AbfsRestOperation abfsRestOperation = abfsClient - .append(fileName, buffer, appendRequestParameters, null, null, getTestTracingContext(fs, false)); - assertCPKHeaders(abfsRestOperation, true); - assertResponseHeader(abfsRestOperation, true, X_MS_ENCRYPTION_KEY_SHA256, - getCPKSha(fs)); - assertResponseHeader(abfsRestOperation, false, X_MS_SERVER_ENCRYPTED, ""); - assertResponseHeader(abfsRestOperation, true, X_MS_REQUEST_SERVER_ENCRYPTED, - "true"); - - // Trying to append with different CPK headers - Configuration conf = fs.getConf(); - String accountName = conf.get(FS_AZURE_ABFS_ACCOUNT_NAME); - conf.set(FS_AZURE_ENCRYPTION_CLIENT_PROVIDED_KEY + "." + accountName, - "different-1234567890123456789012"); - try (AzureBlobFileSystem fs2 = (AzureBlobFileSystem) FileSystem.newInstance(conf); - AbfsClient abfsClient2 = fs2.getAbfsClient()) { - LambdaTestUtils.intercept(IOException.class, () -> { - abfsClient2.append(fileName, buffer, appendRequestParameters, null, - null, getTestTracingContext(fs, false)); - }); - } - - // Trying to append with no CPK headers - conf.unset(FS_AZURE_ENCRYPTION_CLIENT_PROVIDED_KEY + "." + accountName); - try (AzureBlobFileSystem fs3 = (AzureBlobFileSystem) FileSystem - .get(conf); AbfsClient abfsClient3 = fs3.getAbfsClient()) { - LambdaTestUtils.intercept(IOException.class, () -> { - abfsClient3.append(fileName, buffer, appendRequestParameters, null, - null, getTestTracingContext(fs, false)); - }); - } - } - - @Test - public void testAppendWithoutCPK() throws Exception { - final AzureBlobFileSystem fs = getAbfs(false); - final String fileName = new Path("/" + methodName.getMethodName()).toString(); - createFileAndGetContent(fs, fileName, FILE_SIZE); - - // Trying to append without CPK headers - AppendRequestParameters appendRequestParameters = - new AppendRequestParameters( - 0, 0, 5, Mode.APPEND_MODE, false, null); - byte[] buffer = getRandomBytesArray(5); - AbfsClient abfsClient = fs.getAbfsClient(); - AbfsRestOperation abfsRestOperation = abfsClient - .append(fileName, buffer, appendRequestParameters, null, null, - getTestTracingContext(fs, false)); - assertCPKHeaders(abfsRestOperation, false); - assertResponseHeader(abfsRestOperation, false, X_MS_ENCRYPTION_KEY_SHA256, - ""); - assertResponseHeader(abfsRestOperation, false, X_MS_SERVER_ENCRYPTED, ""); - assertResponseHeader(abfsRestOperation, true, X_MS_REQUEST_SERVER_ENCRYPTED, - "true"); - - // Trying to append with CPK headers - Configuration conf = fs.getConf(); - String accountName = conf.get(FS_AZURE_ABFS_ACCOUNT_NAME); - conf.set(FS_AZURE_ENCRYPTION_CLIENT_PROVIDED_KEY + "." + accountName, - "12345678901234567890123456789012"); - try (AzureBlobFileSystem fs2 = (AzureBlobFileSystem) FileSystem.newInstance(conf); - AbfsClient abfsClient2 = fs2.getAbfsClient()) { - LambdaTestUtils.intercept(IOException.class, () -> { - abfsClient2.append(fileName, buffer, appendRequestParameters, null, - null, getTestTracingContext(fs, false)); - }); - } - } - - @Test - public void testSetGetXAttr() throws Exception { - final AzureBlobFileSystem fs = getAbfs(true); - final String fileName = new Path(methodName.getMethodName()).toString(); - createFileAndGetContent(fs, fileName, FILE_SIZE); - - String valSent = "testValue"; - String attrName = "testXAttr"; - - // set get and verify - fs.setXAttr(new Path(fileName), attrName, - valSent.getBytes(StandardCharsets.UTF_8), - EnumSet.of(XAttrSetFlag.CREATE)); - byte[] valBytes = fs.getXAttr(new Path(fileName), attrName); - String valRecieved = new String(valBytes); - assertEquals(valSent, valRecieved); - - // set new value get and verify - valSent = "new value"; - fs.setXAttr(new Path(fileName), attrName, - valSent.getBytes(StandardCharsets.UTF_8), - EnumSet.of(XAttrSetFlag.REPLACE)); - valBytes = fs.getXAttr(new Path(fileName), attrName); - valRecieved = new String(valBytes); - assertEquals(valSent, valRecieved); - - // Read without CPK header - LambdaTestUtils.intercept(IOException.class, () -> { - getAbfs(false).getXAttr(new Path(fileName), attrName); - }); - - // Wrong CPK - LambdaTestUtils.intercept(IOException.class, () -> { - getSameFSWithWrongCPK(fs).getXAttr(new Path(fileName), attrName); - }); - } - - @Test - public void testCopyBetweenAccounts() throws Exception { - String accountName = getRawConfiguration() - .get(FS_AZURE_TEST_CPK_ENABLED_SECONDARY_ACCOUNT); - String accountKey = getRawConfiguration() - .get(FS_AZURE_TEST_CPK_ENABLED_SECONDARY_ACCOUNT_KEY); - Assume.assumeTrue(accountName != null && !accountName.isEmpty()); - Assume.assumeTrue(accountKey != null && !accountKey.isEmpty()); - String fileSystemName = "cpkfs"; - - // Create fs1 and a file with CPK - AzureBlobFileSystem fs1 = getAbfs(true); - int fileSize = FILE_SIZE_FOR_COPY_BETWEEN_ACCOUNTS; - byte[] fileContent = getRandomBytesArray(fileSize); - Path testFilePath = createFileWithContent(fs1, - String.format("fs1-file%s.txt", UUID.randomUUID()), fileContent); - - // Create fs2 with different CPK - Configuration conf = new Configuration(); - conf.addResource(TEST_CONFIGURATION_FILE_NAME); - conf.setBoolean(AZURE_CREATE_REMOTE_FILESYSTEM_DURING_INITIALIZATION, true); - conf.unset(FS_AZURE_ABFS_ACCOUNT_NAME); - conf.set(FS_AZURE_ABFS_ACCOUNT_NAME, accountName); - conf.set(FS_AZURE_ACCOUNT_KEY + "." + accountName, accountKey); - conf.set(FS_AZURE_ENCRYPTION_CLIENT_PROVIDED_KEY + "." + accountName, - "123456789012345678901234567890ab"); - conf.set("fs.defaultFS", "abfs://" + fileSystemName + "@" + accountName); - AzureBlobFileSystem fs2 = (AzureBlobFileSystem) FileSystem.newInstance(conf); - - // Read from fs1 and write to fs2, fs1 and fs2 are having different CPK - Path fs2DestFilePath = new Path( - String.format("fs2-dest-file%s.txt", UUID.randomUUID())); - FSDataOutputStream ops = fs2.create(fs2DestFilePath); - try (FSDataInputStream iStream = fs1.open(testFilePath)) { - long totalBytesRead = 0; - do { - int length = 8 * ONE_MB; - byte[] buffer = new byte[length]; - int bytesRead = iStream.read(buffer, 0, length); - totalBytesRead += bytesRead; - ops.write(buffer); - } while (totalBytesRead < fileContent.length); - ops.close(); - } - - // Trying to read fs2DestFilePath with different CPK headers - conf.unset(FS_AZURE_ENCRYPTION_CLIENT_PROVIDED_KEY + "." + accountName); - conf.set(FS_AZURE_ENCRYPTION_CLIENT_PROVIDED_KEY + "." + accountName, - "different-1234567890123456789012"); - try (AzureBlobFileSystem fs3 = (AzureBlobFileSystem) FileSystem - .get(conf); FSDataInputStream iStream = fs3.open(fs2DestFilePath)) { - int length = 8 * ONE_MB; - byte[] buffer = new byte[length]; - LambdaTestUtils.intercept(IOException.class, () -> { - iStream.read(buffer, 0, length); - }); - } - - // Trying to read fs2DestFilePath with no CPK headers - conf.unset(FS_AZURE_ENCRYPTION_CLIENT_PROVIDED_KEY + "." + accountName); - try (AzureBlobFileSystem fs4 = (AzureBlobFileSystem) FileSystem - .get(conf); FSDataInputStream iStream = fs4.open(fs2DestFilePath)) { - int length = 8 * ONE_MB; - byte[] buffer = new byte[length]; - LambdaTestUtils.intercept(IOException.class, () -> { - iStream.read(buffer, 0, length); - }); - } - - // Read fs2DestFilePath and verify the content with the initial random - // bytes created and wrote into the source file at fs1 - try (FSDataInputStream iStream = fs2.open(fs2DestFilePath)) { - long totalBytesRead = 0; - int pos = 0; - do { - int length = 8 * ONE_MB; - byte[] buffer = new byte[length]; - int bytesRead = iStream.read(buffer, 0, length); - totalBytesRead += bytesRead; - for (int i = 0; i < bytesRead; i++) { - assertEquals(fileContent[pos + i], buffer[i]); - } - pos = pos + bytesRead; - } while (totalBytesRead < fileContent.length); - } - } - - @Test - public void testListPathWithCPK() throws Exception { - testListPath(true); - } - - @Test - public void testListPathWithoutCPK() throws Exception { - testListPath(false); - } - - private void testListPath(final boolean isWithCPK) throws Exception { - final AzureBlobFileSystem fs = getAbfs(isWithCPK); - final Path testPath = new Path("/" + methodName.getMethodName()); - String testDirName = testPath.toString(); - fs.mkdirs(testPath); - createFileAndGetContent(fs, testDirName + "/aaa", FILE_SIZE); - createFileAndGetContent(fs, testDirName + "/bbb", FILE_SIZE); - AbfsClient abfsClient = fs.getAbfsClient(); - AbfsRestOperation abfsRestOperation = abfsClient - .listPath(testDirName, false, INT_50, null, - getTestTracingContext(fs, false)); - assertListstatus(fs, abfsRestOperation, testPath); - - // Trying with different CPK headers - Configuration conf = fs.getConf(); - String accountName = conf.get(FS_AZURE_ABFS_ACCOUNT_NAME); - conf.set(FS_AZURE_ENCRYPTION_CLIENT_PROVIDED_KEY + "." + accountName, - "different-1234567890123456789012"); - AzureBlobFileSystem fs2 = (AzureBlobFileSystem) FileSystem.newInstance(conf); - AbfsClient abfsClient2 = fs2.getAbfsClient(); - TracingContext tracingContext = getTestTracingContext(fs, false); - abfsRestOperation = abfsClient2.listPath(testDirName, false, INT_50, - null, getTestTracingContext(fs2, true)); - assertListstatus(fs, abfsRestOperation, testPath); - - if (isWithCPK) { - // Trying with no CPK headers - conf.unset(FS_AZURE_ENCRYPTION_CLIENT_PROVIDED_KEY + "." + accountName); - AzureBlobFileSystem fs3 = (AzureBlobFileSystem) FileSystem.get(conf); - AbfsClient abfsClient3 = fs3.getAbfsClient(); - abfsRestOperation = abfsClient3 - .listPath(testDirName, false, INT_50, null, tracingContext); - assertListstatus(fs, abfsRestOperation, testPath); - } - } - - private void assertListstatus(AzureBlobFileSystem fs, - AbfsRestOperation abfsRestOperation, Path testPath) throws IOException { - assertCPKHeaders(abfsRestOperation, false); - assertNoCPKResponseHeadersPresent(abfsRestOperation); - - FileStatus[] listStatuses = fs.listStatus(testPath); - Assertions.assertThat(listStatuses.length) - .describedAs("listStatuses should have 2 entries").isEqualTo(2); - - listStatuses = getSameFSWithWrongCPK(fs).listStatus(testPath); - Assertions.assertThat(listStatuses.length) - .describedAs("listStatuses should have 2 entries").isEqualTo(2); - } - - @Test - public void testCreatePathWithCPK() throws Exception { - testCreatePath(true); - } - - @Test - public void testCreatePathWithoutCPK() throws Exception { - testCreatePath(false); - } - - private void testCreatePath(final boolean isWithCPK) throws Exception { - final AzureBlobFileSystem fs = getAbfs(isWithCPK); - final String testFileName = new Path("/" + methodName.getMethodName()).toString(); - createFileAndGetContent(fs, testFileName, FILE_SIZE); - - AbfsClient abfsClient = fs.getAbfsClient(); - FsPermission permission = new FsPermission(FsAction.EXECUTE, - FsAction.EXECUTE, FsAction.EXECUTE); - FsPermission umask = new FsPermission(FsAction.NONE, FsAction.NONE, - FsAction.NONE); - TracingContext tracingContext = getTestTracingContext(fs, false); - boolean isNamespaceEnabled = fs.getIsNamespaceEnabled(tracingContext); - AbfsRestOperation abfsRestOperation = abfsClient - .createPath(testFileName, true, true, - isNamespaceEnabled ? getOctalNotation(permission) : null, - isNamespaceEnabled ? getOctalNotation(umask) : null, false, null, - new EncryptionAdapter(abfsClient.getEncryptionContextProvider(), - testFileName, "context".getBytes(StandardCharsets.UTF_8)), - tracingContext); - assertCPKHeaders(abfsRestOperation, isWithCPK); - assertResponseHeader(abfsRestOperation, isWithCPK, - X_MS_ENCRYPTION_KEY_SHA256, getCPKSha(fs)); - assertResponseHeader(abfsRestOperation, false, X_MS_SERVER_ENCRYPTED, ""); - assertResponseHeader(abfsRestOperation, true, X_MS_REQUEST_SERVER_ENCRYPTED, - "true"); - - FileStatus[] listStatuses = fs.listStatus(new Path(testFileName)); - Assertions.assertThat(listStatuses.length) - .describedAs("listStatuses should have 1 entry").isEqualTo(1); - - listStatuses = getSameFSWithWrongCPK(fs).listStatus(new Path(testFileName)); - Assertions.assertThat(listStatuses.length) - .describedAs("listStatuses should have 1 entry").isEqualTo(1); - } - - @Test - public void testRenamePathWithCPK() throws Exception { - testRenamePath(true); - } - - @Test - public void testRenamePathWithoutCPK() throws Exception { - testRenamePath(false); - } - - private void testRenamePath(final boolean isWithCPK) throws Exception { - final AzureBlobFileSystem fs = getAbfs(isWithCPK); - final String testFileName = new Path("/" + methodName.getMethodName()) - .toString(); - createFileAndGetContent(fs, testFileName, FILE_SIZE); - - FileStatus fileStatusBeforeRename = fs - .getFileStatus(new Path(testFileName)); - - String newName = "/newName"; - AbfsClient abfsClient = fs.getAbfsClient(); - AbfsRestOperation abfsRestOperation = abfsClient - .renamePath(testFileName, newName, null, - getTestTracingContext(fs, false)); - assertCPKHeaders(abfsRestOperation, false); - assertNoCPKResponseHeadersPresent(abfsRestOperation); - - LambdaTestUtils.intercept(FileNotFoundException.class, - (() -> fs.getFileStatus(new Path(testFileName)))); - - FileStatus fileStatusAfterRename = fs.getFileStatus(new Path(newName)); - Assertions.assertThat(fileStatusAfterRename.getLen()) - .describedAs("File size has to be same before and after rename") - .isEqualTo(fileStatusBeforeRename.getLen()); - } - - @Test - public void testFlushWithCPK() throws Exception { - testFlush(true); - } - - @Test - public void testFlushWithoutCPK() throws Exception { - testFlush(false); - } - - private void testFlush(final boolean isWithCPK) throws Exception { - final AzureBlobFileSystem fs = getAbfs(isWithCPK); - final String testFileName = new Path("/" + methodName.getMethodName()) - .toString(); - fs.create(new Path(testFileName)).close(); - AbfsClient abfsClient = fs.getAbfsClient(); - String expectedCPKSha = getCPKSha(fs); - - byte[] fileContent = getRandomBytesArray(FILE_SIZE); - Path testFilePath = new Path(testFileName + "1"); - try (FSDataOutputStream oStream = fs.create(testFilePath)) { - oStream.write(fileContent); - } - - // Trying to read with different CPK headers - Configuration conf = fs.getConf(); - String accountName = conf.get(FS_AZURE_ABFS_ACCOUNT_NAME); - conf.set(FS_AZURE_ENCRYPTION_CLIENT_PROVIDED_KEY + "." + accountName, - "different-1234567890123456789012"); - try (AzureBlobFileSystem fs2 = (AzureBlobFileSystem) FileSystem.newInstance(conf); - AbfsClient abfsClient2 = fs2.getAbfsClient()) { - LambdaTestUtils.intercept(IOException.class, () -> { - abfsClient2.flush(testFileName, 0, false, false, null, null, - null, getTestTracingContext(fs, false)); - }); - } - - // Trying to read with no CPK headers - if (isWithCPK) { - conf.unset(FS_AZURE_ENCRYPTION_CLIENT_PROVIDED_KEY + "." + accountName); - try (AzureBlobFileSystem fs3 = (AzureBlobFileSystem) FileSystem - .get(conf); AbfsClient abfsClient3 = fs3.getAbfsClient()) { - LambdaTestUtils.intercept(IOException.class, () -> { - abfsClient3.flush(testFileName, 0, false, false, null, null, - null, getTestTracingContext(fs, false)); - }); - } - } - - // With correct CPK - AbfsRestOperation abfsRestOperation = abfsClient - .flush(testFileName, 0, false, false, null, null, - null, getTestTracingContext(fs, false)); - assertCPKHeaders(abfsRestOperation, isWithCPK); - assertResponseHeader(abfsRestOperation, isWithCPK, - X_MS_ENCRYPTION_KEY_SHA256, expectedCPKSha); - assertResponseHeader(abfsRestOperation, false, X_MS_SERVER_ENCRYPTED, ""); - assertResponseHeader(abfsRestOperation, true, X_MS_REQUEST_SERVER_ENCRYPTED, - isWithCPK + ""); - } - - @Test - public void testSetPathPropertiesWithCPK() throws Exception { - testSetPathProperties(true); - } - - @Test - public void testSetPathPropertiesWithoutCPK() throws Exception { - testSetPathProperties(false); - } - - private void testSetPathProperties(final boolean isWithCPK) throws Exception { - final AzureBlobFileSystem fs = getAbfs(isWithCPK); - final String testFileName = new Path("/" + methodName.getMethodName()) - .toString(); - createFileAndGetContent(fs, testFileName, FILE_SIZE); - - AbfsClient abfsClient = fs.getAbfsClient(); - final Hashtable properties = new Hashtable<>(); - properties.put("key", "val"); - AbfsRestOperation abfsRestOperation = abfsClient - .setPathProperties(testFileName, - convertXmsPropertiesToCommaSeparatedString(properties), - getTestTracingContext(fs, false)); - assertCPKHeaders(abfsRestOperation, isWithCPK); - assertResponseHeader(abfsRestOperation, isWithCPK, - X_MS_ENCRYPTION_KEY_SHA256, getCPKSha(fs)); - assertResponseHeader(abfsRestOperation, false, X_MS_SERVER_ENCRYPTED, ""); - assertResponseHeader(abfsRestOperation, true, X_MS_REQUEST_SERVER_ENCRYPTED, - "true"); - } - - @Test - public void testGetPathStatusFileWithCPK() throws Exception { - testGetPathStatusFile(true); - } - - @Test - public void testGetPathStatusFileWithoutCPK() throws Exception { - testGetPathStatusFile(false); - } - - private void testGetPathStatusFile(final boolean isWithCPK) throws Exception { - final AzureBlobFileSystem fs = getAbfs(isWithCPK); - final String testFileName = new Path("/" + methodName.getMethodName()) - .toString(); - createFileAndGetContent(fs, testFileName, FILE_SIZE); - - AbfsClient abfsClient = fs.getAbfsClient(); - TracingContext tracingContext = getTestTracingContext(fs, false); - AbfsRestOperation abfsRestOperation = abfsClient - .getPathStatus(testFileName, false, tracingContext); - assertCPKHeaders(abfsRestOperation, false); - assertResponseHeader(abfsRestOperation, isWithCPK, - X_MS_ENCRYPTION_KEY_SHA256, getCPKSha(fs)); - assertResponseHeader(abfsRestOperation, true, X_MS_SERVER_ENCRYPTED, - "true"); - assertResponseHeader(abfsRestOperation, false, - X_MS_REQUEST_SERVER_ENCRYPTED, ""); - - abfsRestOperation = abfsClient.getPathStatus(testFileName, true, - tracingContext); - assertCPKHeaders(abfsRestOperation, isWithCPK); - assertResponseHeader(abfsRestOperation, isWithCPK, - X_MS_ENCRYPTION_KEY_SHA256, getCPKSha(fs)); - assertResponseHeader(abfsRestOperation, true, X_MS_SERVER_ENCRYPTED, - "true"); - assertResponseHeader(abfsRestOperation, false, - X_MS_REQUEST_SERVER_ENCRYPTED, ""); - } - - @Test - public void testDeletePathWithCPK() throws Exception { - testDeletePath(false); - } - - @Test - public void testDeletePathWithoutCPK() throws Exception { - testDeletePath(false); - } - - private void testDeletePath(final boolean isWithCPK) throws Exception { - final AzureBlobFileSystem fs = getAbfs(isWithCPK); - final String testFileName = new Path("/" + methodName.getMethodName()) - .toString(); - createFileAndGetContent(fs, testFileName, FILE_SIZE); - - FileStatus[] listStatuses = fs.listStatus(new Path(testFileName)); - Assertions.assertThat(listStatuses.length) - .describedAs("listStatuses should have 1 entry").isEqualTo(1); - - AbfsClient abfsClient = fs.getAbfsClient(); - AbfsRestOperation abfsRestOperation = abfsClient - .deletePath(testFileName, false, null, - getTestTracingContext(fs, false)); - assertCPKHeaders(abfsRestOperation, false); - assertNoCPKResponseHeadersPresent(abfsRestOperation); - - Assertions.assertThatThrownBy(() -> fs.listStatus(new Path(testFileName))) - .isInstanceOf(FileNotFoundException.class); - } - - @Test - public void testSetPermissionWithCPK() throws Exception { - testSetPermission(true); - } - - @Test - public void testSetPermissionWithoutCPK() throws Exception { - testSetPermission(false); - } - - private void testSetPermission(final boolean isWithCPK) throws Exception { - final AzureBlobFileSystem fs = getAbfs(isWithCPK); - final String testFileName = new Path("/" + methodName.getMethodName()) - .toString(); - Assume.assumeTrue(fs.getIsNamespaceEnabled(getTestTracingContext(fs, false))); - createFileAndGetContent(fs, testFileName, FILE_SIZE); - AbfsClient abfsClient = fs.getAbfsClient(); - FsPermission permission = new FsPermission(FsAction.EXECUTE, - FsAction.EXECUTE, FsAction.EXECUTE); - AbfsRestOperation abfsRestOperation = abfsClient - .setPermission(testFileName, permission.toString(), - getTestTracingContext(fs, false)); - assertCPKHeaders(abfsRestOperation, false); - assertNoCPKResponseHeadersPresent(abfsRestOperation); - } - - @Test - public void testSetAclWithCPK() throws Exception { - testSetAcl(true); - } - - @Test - public void testSetAclWithoutCPK() throws Exception { - testSetAcl(false); - } - - private void testSetAcl(final boolean isWithCPK) throws Exception { - final AzureBlobFileSystem fs = getAbfs(isWithCPK); - final String testFileName = new Path("/" + methodName.getMethodName()) - .toString(); - TracingContext tracingContext = getTestTracingContext(fs, false); - Assume.assumeTrue(fs.getIsNamespaceEnabled(tracingContext)); - createFileAndGetContent(fs, testFileName, FILE_SIZE); - AbfsClient abfsClient = fs.getAbfsClient(); - - List aclSpec = Lists.newArrayList(aclEntry(ACCESS, USER, ALL)); - final Map aclEntries = AbfsAclHelper - .deserializeAclSpec(AclEntry.aclSpecToString(aclSpec)); - - AbfsRestOperation abfsRestOperation = abfsClient - .setAcl(testFileName, AbfsAclHelper.serializeAclSpec(aclEntries), - tracingContext); - assertCPKHeaders(abfsRestOperation, false); - assertNoCPKResponseHeadersPresent(abfsRestOperation); - } - - @Test - public void testGetAclWithCPK() throws Exception { - testGetAcl(true); - } - - @Test - public void testGetAclWithoutCPK() throws Exception { - testGetAcl(false); - } - - private void testGetAcl(final boolean isWithCPK) throws Exception { - final AzureBlobFileSystem fs = getAbfs(isWithCPK); - final String testFileName = new Path("/" + methodName.getMethodName()) - .toString(); - TracingContext tracingContext = getTestTracingContext(fs, false); - Assume.assumeTrue(fs.getIsNamespaceEnabled(tracingContext)); - createFileAndGetContent(fs, testFileName, FILE_SIZE); - AbfsClient abfsClient = fs.getAbfsClient(); - AbfsRestOperation abfsRestOperation = - abfsClient.getAclStatus(testFileName, tracingContext); - assertCPKHeaders(abfsRestOperation, false); - assertNoCPKResponseHeadersPresent(abfsRestOperation); - } - - @Test - public void testCheckAccessWithCPK() throws Exception { - testCheckAccess(true); - } - - @Test - public void testCheckAccessWithoutCPK() throws Exception { - testCheckAccess(false); - } - - private void testCheckAccess(final boolean isWithCPK) throws Exception { - boolean isHNSEnabled = getConfiguration() - .getBoolean(FS_AZURE_TEST_NAMESPACE_ENABLED_ACCOUNT, false); - Assume.assumeTrue(FS_AZURE_TEST_NAMESPACE_ENABLED_ACCOUNT + " is false", - isHNSEnabled); - Assume.assumeTrue("AuthType has to be OAuth", - getAuthType() == AuthType.OAuth); - - final AzureBlobFileSystem fs = getAbfs(isWithCPK); - final String testFileName = new Path("/" + methodName.getMethodName()) - .toString(); - fs.create(new Path(testFileName)).close(); - AbfsClient abfsClient = fs.getAbfsClient(); - AbfsRestOperation abfsRestOperation = abfsClient - .checkAccess(testFileName, "rwx", getTestTracingContext(fs, false)); - assertCPKHeaders(abfsRestOperation, false); - assertNoCPKResponseHeadersPresent(abfsRestOperation); - } - - private byte[] createFileAndGetContent(AzureBlobFileSystem fs, - String fileName, int fileSize) throws IOException { - byte[] fileContent = getRandomBytesArray(fileSize); - Path testFilePath = createFileWithContent(fs, fileName, fileContent); - ContractTestUtils.verifyFileContents(fs, testFilePath, fileContent); - return fileContent; - } - - private void assertCPKHeaders(AbfsRestOperation abfsRestOperation, - boolean isCPKHeaderExpected) { - assertHeader(abfsRestOperation, X_MS_ENCRYPTION_KEY, isCPKHeaderExpected); - assertHeader(abfsRestOperation, X_MS_ENCRYPTION_KEY_SHA256, - isCPKHeaderExpected); - assertHeader(abfsRestOperation, X_MS_ENCRYPTION_ALGORITHM, - isCPKHeaderExpected); - } - - private void assertNoCPKResponseHeadersPresent( - AbfsRestOperation abfsRestOperation) { - assertResponseHeader(abfsRestOperation, false, X_MS_SERVER_ENCRYPTED, ""); - assertResponseHeader(abfsRestOperation, false, - X_MS_REQUEST_SERVER_ENCRYPTED, ""); - assertResponseHeader(abfsRestOperation, false, X_MS_ENCRYPTION_KEY_SHA256, - ""); - } - - private void assertResponseHeader(AbfsRestOperation abfsRestOperation, - boolean isHeaderExpected, String headerName, String expectedValue) { - final AbfsHttpOperation result = abfsRestOperation.getResult(); - final String value = result.getResponseHeader(headerName); - if (isHeaderExpected) { - Assertions.assertThat(value).isEqualTo(expectedValue); - } else { - Assertions.assertThat(value).isNull(); - } - } - - private void assertHeader(AbfsRestOperation abfsRestOperation, - String headerName, boolean isCPKHeaderExpected) { - assertTrue(abfsRestOperation != null); - Optional header = abfsRestOperation.getRequestHeaders() - .stream().filter(abfsHttpHeader -> abfsHttpHeader.getName() - .equalsIgnoreCase(headerName)).findFirst(); - String desc; - if (isCPKHeaderExpected) { - desc = - "CPK header " + headerName + " is expected, but the same is absent."; - } else { - desc = "CPK header " + headerName - + " is not expected, but the same is present."; - } - Assertions.assertThat(header.isPresent()).describedAs(desc) - .isEqualTo(isCPKHeaderExpected); - } - - private byte[] getSHA256Hash(String key) throws IOException { - try { - final MessageDigest digester = MessageDigest.getInstance("SHA-256"); - return digester.digest(key.getBytes(StandardCharsets.UTF_8)); - } catch (NoSuchAlgorithmException e) { - throw new IOException(e); - } - } - - private String getCPKSha(final AzureBlobFileSystem abfs) throws IOException { - Configuration conf = abfs.getConf(); - String accountName = conf.get(FS_AZURE_ABFS_ACCOUNT_NAME); - String encryptionKey = conf - .get(FS_AZURE_ENCRYPTION_CLIENT_PROVIDED_KEY + "." + accountName); - if (encryptionKey == null || encryptionKey.isEmpty()) { - return ""; - } - return getBase64EncodedString(getSHA256Hash(encryptionKey)); - } - - private String getBase64EncodedString(byte[] bytes) { - return java.util.Base64.getEncoder().encodeToString(bytes); - } - - private Path createFileWithContent(FileSystem fs, String fileName, - byte[] fileContent) throws IOException { - Path testFilePath = new Path(fileName); - try (FSDataOutputStream oStream = fs.create(testFilePath)) { - oStream.write(fileContent); - oStream.flush(); - } - return testFilePath; - } - - private String convertXmsPropertiesToCommaSeparatedString( - final Hashtable properties) - throws CharacterCodingException { - StringBuilder commaSeparatedProperties = new StringBuilder(); - final CharsetEncoder encoder = Charset.forName(XMS_PROPERTIES_ENCODING) - .newEncoder(); - for (Map.Entry propertyEntry : properties.entrySet()) { - String key = propertyEntry.getKey(); - String value = propertyEntry.getValue(); - Boolean canEncodeValue = encoder.canEncode(value); - if (!canEncodeValue) { - throw new CharacterCodingException(); - } - String encodedPropertyValue = Base64 - .encode(encoder.encode(CharBuffer.wrap(value)).array()); - commaSeparatedProperties.append(key).append(AbfsHttpConstants.EQUAL) - .append(encodedPropertyValue); - commaSeparatedProperties.append(AbfsHttpConstants.COMMA); - } - if (commaSeparatedProperties.length() != 0) { - commaSeparatedProperties - .deleteCharAt(commaSeparatedProperties.length() - 1); - } - return commaSeparatedProperties.toString(); - } - - private String getOctalNotation(FsPermission fsPermission) { - Preconditions.checkNotNull(fsPermission, "fsPermission"); - return String - .format(AbfsHttpConstants.PERMISSION_FORMAT, fsPermission.toOctal()); - } - - private byte[] getRandomBytesArray(int length) { - final byte[] b = new byte[length]; - new Random().nextBytes(b); - return b; - } - - private AzureBlobFileSystem getAbfs(boolean withCPK) throws IOException { - return getAbfs(withCPK, "12345678901234567890123456789012"); - } - - private AzureBlobFileSystem getAbfs(boolean withCPK, String cpk) - throws IOException { - Configuration conf = getRawConfiguration(); - if (withCPK) { - conf.set(FS_AZURE_ENCRYPTION_CLIENT_PROVIDED_KEY + "." + getAccountName(), - cpk); - } else { - conf.unset( - FS_AZURE_ENCRYPTION_CLIENT_PROVIDED_KEY + "." + getAccountName()); - } - return (AzureBlobFileSystem) FileSystem.newInstance(conf); - } - - private AzureBlobFileSystem getSameFSWithWrongCPK( - final AzureBlobFileSystem fs) throws IOException { - AbfsConfiguration abfsConf = fs.getAbfsStore().getAbfsConfiguration(); - Configuration conf = abfsConf.getRawConfiguration(); - String accountName = conf.get(FS_AZURE_ABFS_ACCOUNT_NAME); - String cpk = conf - .get(FS_AZURE_ENCRYPTION_CLIENT_PROVIDED_KEY + "." + accountName); - if (cpk == null || cpk.isEmpty()) { - cpk = "01234567890123456789012345678912"; - } - cpk = "different-" + cpk; - String differentCpk = cpk.substring(0, ENCRYPTION_KEY_LEN - 1); - conf.set(FS_AZURE_ENCRYPTION_CLIENT_PROVIDED_KEY + "." + accountName, - differentCpk); - conf.set("fs.defaultFS", - "abfs://" + getFileSystemName() + "@" + accountName); - AzureBlobFileSystem sameFSWithDifferentCPK = - (AzureBlobFileSystem) FileSystem.newInstance(conf); - return sameFSWithDifferentCPK; - } - -} diff --git a/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/TestTracingContext.java b/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/TestTracingContext.java index 39a9d996e2d69..2b354928cee2a 100644 --- a/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/TestTracingContext.java +++ b/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/TestTracingContext.java @@ -26,6 +26,7 @@ import org.apache.hadoop.conf.Configuration; import org.apache.hadoop.fs.FileSystem; +import org.apache.hadoop.fs.azurebfs.AzureBlobFileSystemStore.Permissions; import org.assertj.core.api.Assertions; import org.junit.Assume; import org.junit.AssumptionViolatedException; @@ -98,17 +99,13 @@ public void checkCorrelationConfigValidation(String clientCorrelationId, TracingHeaderFormat.ALL_ID_FORMAT, null); boolean isNamespaceEnabled = fs.getIsNamespaceEnabled(tracingContext); String path = getRelativePath(new Path("/testDir")); - String permission = isNamespaceEnabled - ? getOctalNotation(FsPermission.getDirDefault()) - : null; - String umask = isNamespaceEnabled - ? getOctalNotation(FsPermission.getUMask(fs.getConf())) - : null; + Permissions permissions = new Permissions(isNamespaceEnabled, + FsPermission.getDefault(), FsPermission.getUMask(fs.getConf())); //request should not fail for invalid clientCorrelationID AbfsRestOperation op = fs.getAbfsClient() - .createPath(path, false, true, permission, umask, false, null, - null, tracingContext); + .createPath(path, false, true, permissions, false, null, null, + tracingContext); int statusCode = op.getResult().getStatusCode(); Assertions.assertThat(statusCode).describedAs("Request should not fail") diff --git a/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/constants/TestConfigurationKeys.java b/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/constants/TestConfigurationKeys.java index 565eb38c4f70a..fef435a94cc88 100644 --- a/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/constants/TestConfigurationKeys.java +++ b/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/constants/TestConfigurationKeys.java @@ -61,6 +61,7 @@ public final class TestConfigurationKeys { public static final String TEST_CONFIGURATION_FILE_NAME = "azure-test.xml"; public static final String TEST_CONTAINER_PREFIX = "abfs-testcontainer-"; public static final int TEST_TIMEOUT = 15 * 60 * 1000; + public static final int ENCRYPTION_KEY_LEN = 32; private TestConfigurationKeys() {} } diff --git a/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/extensions/MockEncryptionContextProvider.java b/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/extensions/MockEncryptionContextProvider.java index e34889d892cd3..e539a01043e8a 100644 --- a/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/extensions/MockEncryptionContextProvider.java +++ b/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/extensions/MockEncryptionContextProvider.java @@ -29,6 +29,8 @@ import org.apache.hadoop.conf.Configuration; import org.apache.hadoop.classification.VisibleForTesting; +import static org.apache.hadoop.fs.azurebfs.constants.TestConfigurationKeys.ENCRYPTION_KEY_LEN; + public class MockEncryptionContextProvider implements EncryptionContextProvider { private HashMap pathToContextMap = new HashMap<>(); private HashMap contextToKeyMap = new HashMap<>(); @@ -42,7 +44,7 @@ public SecretKey getEncryptionContext(String path) throws IOException { String newContext = UUID.randomUUID().toString(); pathToContextMap.put(path, newContext); - String keyString = RandomStringUtils.random(32, true, true); + String keyString = RandomStringUtils.random(ENCRYPTION_KEY_LEN, true, true); Key key = new Key(keyString.getBytes(StandardCharsets.UTF_8)); contextToKeyMap.put(newContext, key); return new Key(newContext.getBytes(StandardCharsets.UTF_8)); From d96e08d4c0cf4add69c7845c9c4aa785689f8343 Mon Sep 17 00:00:00 2001 From: sumangala Date: Tue, 23 Nov 2021 15:59:51 +0530 Subject: [PATCH 22/77] allow account specific ECP only --- .../hadoop/fs/azurebfs/AbfsConfiguration.java | 29 ++++++++++++------- .../fs/azurebfs/AzureBlobFileSystemStore.java | 3 -- .../azurebfs/security/EncryptionAdapter.java | 2 -- .../fs/azurebfs/services/AbfsClient.java | 6 ++-- .../fs/azurebfs/ITestCustomEncryption.java | 11 ++++--- 5 files changed, 25 insertions(+), 26 deletions(-) diff --git a/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/AbfsConfiguration.java b/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/AbfsConfiguration.java index 7c433a638f2c3..2a730f5fc9605 100644 --- a/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/AbfsConfiguration.java +++ b/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/AbfsConfiguration.java @@ -307,6 +307,9 @@ public class AbfsConfiguration{ FS_AZURE_ENABLE_ABFS_LIST_ITERATOR, DefaultValue = DEFAULT_ENABLE_ABFS_LIST_ITERATOR) private boolean enableAbfsListIterator; + private String clientProvidedEncryptionKey; + private String clientProvidedEncryptionKeySHA; + public AbfsConfiguration(final Configuration rawConfig, String accountName) throws IllegalAccessException, InvalidConfigurationValueException, IOException { this.rawConfig = ProviderUtils.excludeIncompatibleCredentialProviders( @@ -916,19 +919,15 @@ public EncryptionContextProvider createEncryptionContextProvider() { } Class encryptionContextClass = getAccountSpecificClass(configKey, null, EncryptionContextProvider.class); - if (encryptionContextClass == null) { - encryptionContextClass = getAccountAgnosticClass(configKey, null, - EncryptionContextProvider.class); - } - Preconditions.checkArgument(encryptionContextClass != null, - String.format("The configuration value for %s is invalid.", configKey)); + Preconditions.checkArgument(encryptionContextClass != null, String.format( + "The configuration value for %s is invalid, or config key is not account-specific", + configKey)); EncryptionContextProvider encryptionContextProvider = ReflectionUtils.newInstance(encryptionContextClass, rawConfig); Preconditions.checkArgument(encryptionContextProvider != null, String.format("Failed to initialize %s", encryptionContextClass)); - LOG.trace("Initializing {}", encryptionContextClass.getName()); LOG.trace("{} init complete", encryptionContextClass.getName()); return encryptionContextProvider; } catch (Exception e) { @@ -1040,13 +1039,21 @@ public boolean enableAbfsListIterator() { } public String getEncodedClientProvidedEncryptionKey() { - String accSpecEncKey = accountConf(FS_AZURE_ENCRYPTION_ENCODED_CLIENT_PROVIDED_KEY); - return rawConfig.get(accSpecEncKey, null); + if (clientProvidedEncryptionKey == null) { + String accSpecEncKey = accountConf( + FS_AZURE_ENCRYPTION_ENCODED_CLIENT_PROVIDED_KEY); + clientProvidedEncryptionKey = rawConfig.get(accSpecEncKey, null); + } + return clientProvidedEncryptionKey; } public String getEncodedClientProvidedEncryptionKeySHA() { - String accSpecEncKey = accountConf(FS_AZURE_ENCRYPTION_ENCODED_CLIENT_PROVIDED_KEY_SHA); - return rawConfig.get(accSpecEncKey, null); + if (clientProvidedEncryptionKeySHA == null) { + String accSpecEncKey = accountConf( + FS_AZURE_ENCRYPTION_ENCODED_CLIENT_PROVIDED_KEY_SHA); + clientProvidedEncryptionKeySHA = rawConfig.get(accSpecEncKey, null); + } + return clientProvidedEncryptionKeySHA; } @VisibleForTesting diff --git a/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/AzureBlobFileSystemStore.java b/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/AzureBlobFileSystemStore.java index dd79c944662b5..55682885a89c1 100644 --- a/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/AzureBlobFileSystemStore.java +++ b/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/AzureBlobFileSystemStore.java @@ -1621,7 +1621,6 @@ private void initializeClient(URI uri, String fileSystemName, } // Encryption setup - EncryptionType encryptionType = EncryptionType.NONE; EncryptionContextProvider encryptionContextProvider = null; if (isSecure) { encryptionContextProvider = @@ -1634,10 +1633,8 @@ private void initializeClient(URI uri, String fileSystemName, encryptionContextProvider.initialize( abfsConfiguration.getRawConfiguration(), accountName, fileSystemName); - encryptionType = EncryptionType.ENCRYPTION_CONTEXT; } else if (abfsConfiguration.getEncodedClientProvidedEncryptionKey() != null) { if (abfsConfiguration.getEncodedClientProvidedEncryptionKeySHA() != null) { - encryptionType = EncryptionType.GLOBAL_KEY; } else { throw new IOException( "Encoded SHA256 hash must be provided for global encryption"); diff --git a/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/security/EncryptionAdapter.java b/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/security/EncryptionAdapter.java index 575660c85e9a6..4e979869d538a 100644 --- a/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/security/EncryptionAdapter.java +++ b/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/security/EncryptionAdapter.java @@ -76,8 +76,6 @@ public SecretKey createEncryptionContext() throws IOException { public void computeKeys() throws IOException { SecretKey key = getEncryptionKey(); Preconditions.checkNotNull(key, "Encryption key should not be null."); -// encodedKey = getBase64EncodedString(new String(key.getEncoded(), -// StandardCharsets.UTF_8)); encodedKey = getBase64EncodedString(key.getEncoded()); encodedKeySHA = getBase64EncodedString(getSHA256Hash(new String(key.getEncoded(), StandardCharsets.UTF_8))); diff --git a/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/services/AbfsClient.java b/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/services/AbfsClient.java index 3e45439caaf7f..ccb0f4940a43c 100644 --- a/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/services/AbfsClient.java +++ b/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/services/AbfsClient.java @@ -113,12 +113,10 @@ private AbfsClient(final URL baseUrl, this.baseUrl = baseUrl; this.sharedKeyCredentials = sharedKeyCredentials; String baseUrlString = baseUrl.toString(); - this.filesystem = baseUrlString.substring( - baseUrlString.lastIndexOf(FORWARD_SLASH) + 1); + this.filesystem = baseUrlString.substring(baseUrlString.lastIndexOf(FORWARD_SLASH) + 1); this.abfsConfiguration = abfsConfiguration; this.retryPolicy = abfsClientContext.getExponentialRetryPolicy(); - this.accountName = abfsConfiguration.getAccountName().substring(0, - abfsConfiguration.getAccountName().indexOf(AbfsHttpConstants.DOT)); + this.accountName = abfsConfiguration.getAccountName().substring(0, abfsConfiguration.getAccountName().indexOf(AbfsHttpConstants.DOT)); this.authType = abfsConfiguration.getAuthType(accountName); if (encryptionContextProvider != null) { diff --git a/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/ITestCustomEncryption.java b/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/ITestCustomEncryption.java index dd8c7a1a8eb0f..234969a7a85e2 100644 --- a/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/ITestCustomEncryption.java +++ b/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/ITestCustomEncryption.java @@ -140,7 +140,6 @@ public ITestCustomEncryption() throws Exception { super(); } - @Test public void testCustomEncryptionCombinations() throws Exception { AzureBlobFileSystem fs = getFS(); @@ -251,8 +250,8 @@ private AbfsRestOperation callOperation(AzureBlobFileSystem fs, private AzureBlobFileSystem getECProviderEnabledFS() throws Exception { Configuration configuration = getRawConfiguration(); - configuration.set(FS_AZURE_ENCRYPTION_CONTEXT_PROVIDER_TYPE, - MockEncryptionContextProvider.class.getCanonicalName()); + configuration.set(FS_AZURE_ENCRYPTION_CONTEXT_PROVIDER_TYPE + "." + + getAccountName(), MockEncryptionContextProvider.class.getCanonicalName()); configuration.unset(FS_AZURE_ENCRYPTION_ENCODED_CLIENT_PROVIDED_KEY + "." + getAccountName()); configuration.unset(FS_AZURE_ENCRYPTION_ENCODED_CLIENT_PROVIDED_KEY_SHA + "." @@ -260,7 +259,7 @@ private AzureBlobFileSystem getECProviderEnabledFS() throws Exception { return (AzureBlobFileSystem) FileSystem.newInstance(configuration); } - private AzureBlobFileSystem getCPKenabledFS() throws IOException { + private AzureBlobFileSystem getCPKEnabledFS() throws IOException { Configuration conf = getRawConfiguration(); String cpkEncoded = EncryptionAdapter.getBase64EncodedString(cpk); String cpkEncodedSHA = EncryptionAdapter.getBase64EncodedString( @@ -280,7 +279,7 @@ private AzureBlobFileSystem getFS() throws Exception { if (requestEncryptionType == ENCRYPTION_CONTEXT) { return getECProviderEnabledFS(); } else if (requestEncryptionType == GLOBAL_KEY) { - return getCPKenabledFS(); + return getCPKEnabledFS(); } else { Configuration conf = getRawConfiguration(); conf.unset(FS_AZURE_ENCRYPTION_CONTEXT_PROVIDER_TYPE); @@ -295,7 +294,7 @@ private EncryptionContextProvider createEncryptedFile(Path testPath) throws Exce } else { fs = fileEncryptionType == ENCRYPTION_CONTEXT ? getECProviderEnabledFS() - : getCPKenabledFS(); + : getCPKEnabledFS(); } String relativePath = fs.getAbfsStore().getRelativePath(testPath); try (FSDataOutputStream out = fs.create(new Path(relativePath))) { From fb7545438556e1da4a7cae5070ccc9676dca1dbc Mon Sep 17 00:00:00 2001 From: sumangala Date: Tue, 30 Nov 2021 09:34:13 +0530 Subject: [PATCH 23/77] encryption context encoding + checkstyle --- .../hadoop/fs/azurebfs/security/EncryptionAdapter.java | 2 +- .../apache/hadoop/fs/azurebfs/services/AbfsClient.java | 8 +++++--- .../apache/hadoop/fs/azurebfs/ITestCustomEncryption.java | 4 +++- .../extensions/MockEncryptionContextProvider.java | 7 +++++-- 4 files changed, 14 insertions(+), 7 deletions(-) diff --git a/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/security/EncryptionAdapter.java b/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/security/EncryptionAdapter.java index 4e979869d538a..a0087e137c75c 100644 --- a/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/security/EncryptionAdapter.java +++ b/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/security/EncryptionAdapter.java @@ -49,7 +49,7 @@ public EncryptionAdapter(EncryptionContextProvider provider, String path, this(provider, path); Preconditions.checkNotNull(encryptionContext, "Encryption context should not be null."); - this.encryptionContext = new ABFSSecretKey(encryptionContext); + this.encryptionContext = new ABFSSecretKey(Base64.getDecoder().decode(encryptionContext)); } public EncryptionAdapter(EncryptionContextProvider provider, String path) diff --git a/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/services/AbfsClient.java b/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/services/AbfsClient.java index ccb0f4940a43c..e17341a2da0de 100644 --- a/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/services/AbfsClient.java +++ b/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/services/AbfsClient.java @@ -27,6 +27,7 @@ import java.net.URLEncoder; import java.nio.charset.StandardCharsets; import java.util.ArrayList; +import java.util.Base64; import java.util.List; import java.util.Locale; import java.util.UUID; @@ -123,8 +124,9 @@ private AbfsClient(final URL baseUrl, this.encryptionContextProvider = encryptionContextProvider; xMsVersion = "2021-04-10"; // will be default once server change deployed encryptionType = EncryptionType.ENCRYPTION_CONTEXT; - } else if ((clientProvidedEncryptionKey = - abfsConfiguration.getEncodedClientProvidedEncryptionKey()) != null) { + } else if (abfsConfiguration.getEncodedClientProvidedEncryptionKey() != null) { + clientProvidedEncryptionKey = + abfsConfiguration.getEncodedClientProvidedEncryptionKey(); this.clientProvidedEncryptionKeySHA = abfsConfiguration.getEncodedClientProvidedEncryptionKeySHA(); encryptionType = EncryptionType.GLOBAL_KEY; @@ -239,7 +241,7 @@ private void addEncryptionKeyRequestHeaders(String path, encryptionAdapter.createEncryptionContext(); encryptionAdapter.computeKeys(); requestHeaders.add(new AbfsHttpHeader(X_MS_ENCRYPTION_CONTEXT, - new String(encryptionContext.getEncoded(), StandardCharsets.UTF_8))); + Base64.getEncoder().encodeToString(encryptionContext.getEncoded()))); try { encryptionContext.destroy(); } catch (DestroyFailedException e) { diff --git a/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/ITestCustomEncryption.java b/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/ITestCustomEncryption.java index 234969a7a85e2..99e4e9c6ac847 100644 --- a/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/ITestCustomEncryption.java +++ b/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/ITestCustomEncryption.java @@ -21,6 +21,7 @@ import java.io.IOException; import java.nio.charset.StandardCharsets; import java.util.Arrays; +import java.util.Base64; import java.util.Hashtable; import org.apache.hadoop.fs.permission.FsPermission; @@ -211,8 +212,9 @@ private AbfsRestOperation callOperation(AzureBlobFileSystem fs, if (fileEncryptionType == ENCRYPTION_CONTEXT) { encryptionAdapter = new EncryptionAdapter(ecp, fs.getAbfsStore().getRelativePath(testPath), + Base64.getEncoder().encode( ((MockEncryptionContextProvider) ecp).getEncryptionContextForTest(testPath.toString()) - .getBytes(StandardCharsets.UTF_8)); + .getBytes(StandardCharsets.UTF_8))); } String path = testPath.toString(); switch (operation) { diff --git a/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/extensions/MockEncryptionContextProvider.java b/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/extensions/MockEncryptionContextProvider.java index e539a01043e8a..fbdb40f44c8b3 100644 --- a/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/extensions/MockEncryptionContextProvider.java +++ b/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/extensions/MockEncryptionContextProvider.java @@ -22,6 +22,7 @@ import java.io.IOException; import java.nio.charset.StandardCharsets; import java.util.Arrays; +import java.util.Base64; import java.util.HashMap; import java.util.UUID; @@ -53,10 +54,12 @@ public SecretKey getEncryptionContext(String path) @Override public SecretKey getEncryptionKey(String path, SecretKey encryptionContext) throws IOException { - if (!new String(encryptionContext.getEncoded()).equals(pathToContextMap.get(path))) { + String encryptionContextString = + new String(encryptionContext.getEncoded(), StandardCharsets.UTF_8); + if (!encryptionContextString.equals(pathToContextMap.get(path))) { throw new IOException("encryption context does not match path"); } - return contextToKeyMap.get(new String(encryptionContext.getEncoded())); + return contextToKeyMap.get(encryptionContextString); } @Override From 3caeb7be7203fd2e744704328a808923e042aafd Mon Sep 17 00:00:00 2001 From: sumangala Date: Thu, 2 Dec 2021 16:24:00 +0530 Subject: [PATCH 24/77] generalize key -> String to byte array, add doc for cpk options --- .../azurebfs/constants/ConfigurationKeys.java | 2 +- .../azurebfs/security/EncryptionAdapter.java | 11 ++----- .../hadoop-azure/src/site/markdown/abfs.md | 32 +++++++++++++++++++ ...on.java => ITestAbfsCustomEncryption.java} | 17 ++++++---- .../MockEncryptionContextProvider.java | 12 +++---- 5 files changed, 53 insertions(+), 21 deletions(-) rename hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/{ITestCustomEncryption.java => ITestAbfsCustomEncryption.java} (96%) diff --git a/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/constants/ConfigurationKeys.java b/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/constants/ConfigurationKeys.java index 4c608d9312f8a..e41a6453f903d 100644 --- a/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/constants/ConfigurationKeys.java +++ b/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/constants/ConfigurationKeys.java @@ -191,7 +191,7 @@ public final class ConfigurationKeys { /** Server side encryption key encoded in Base6format */ public static final String FS_AZURE_ENCRYPTION_ENCODED_CLIENT_PROVIDED_KEY = "fs.azure.encryption.encoded.client-provided-key"; - /** SHA256 hash of encryption key encoded in Base6format */ + /** SHA256 hash of encryption key encoded in Base64format */ public static final String FS_AZURE_ENCRYPTION_ENCODED_CLIENT_PROVIDED_KEY_SHA = "fs.azure.encryption.encoded.client-provided-key-sha"; /** Custom EncryptionContextProvider type */ diff --git a/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/security/EncryptionAdapter.java b/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/security/EncryptionAdapter.java index a0087e137c75c..86b5a1691c4f5 100644 --- a/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/security/EncryptionAdapter.java +++ b/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/security/EncryptionAdapter.java @@ -77,8 +77,7 @@ public void computeKeys() throws IOException { SecretKey key = getEncryptionKey(); Preconditions.checkNotNull(key, "Encryption key should not be null."); encodedKey = getBase64EncodedString(key.getEncoded()); - encodedKeySHA = getBase64EncodedString(getSHA256Hash(new String(key.getEncoded(), - StandardCharsets.UTF_8))); + encodedKeySHA = getBase64EncodedString(getSHA256Hash(key.getEncoded())); } public String getEncodedKey() throws IOException { @@ -127,19 +126,15 @@ public void destroy() { } } - public static byte[] getSHA256Hash(String key) throws IOException { + public static byte[] getSHA256Hash(byte[] key) throws IOException { try { final MessageDigest digester = MessageDigest.getInstance("SHA-256"); - return digester.digest(key.getBytes(StandardCharsets.UTF_8)); + return digester.digest(key); } catch (NoSuchAlgorithmException e) { throw new IOException(e); } } - public static String getBase64EncodedString(String key) { - return getBase64EncodedString(key.getBytes(StandardCharsets.UTF_8)); - } - public static String getBase64EncodedString(byte[] bytes) { return Base64.getEncoder().encodeToString(bytes); } diff --git a/hadoop-tools/hadoop-azure/src/site/markdown/abfs.md b/hadoop-tools/hadoop-azure/src/site/markdown/abfs.md index dfb7f3f42a5cf..4f6c230c01f3a 100644 --- a/hadoop-tools/hadoop-azure/src/site/markdown/abfs.md +++ b/hadoop-tools/hadoop-azure/src/site/markdown/abfs.md @@ -890,6 +890,38 @@ specified SSL channel mode. Value should be of the enum DelegatingSSLSocketFactory.SSLChannelMode. The default value will be DelegatingSSLSocketFactory.SSLChannelMode.Default. +### Encryption Options +Only one of the following two options can be configured. If config values of +both types are set, ABFS driver will throw an exception. If using the global +key type, ensure both pre-computed values are provided. + +#### Customer-Provided Global Key +A global encryption key can be configured by providing the following +pre-computed values. The key will be applied to any new files created post +setting the configuration, and will be required in the requests to read ro +modify the contents of the files. + +`fs.azure.encryption.encoded.client-provided-key`: The Base64 encoded version +of the 256-bit encryption key. + +`fs.azure.encryption.encoded.client-provided-key-sha`: The Base64 encoded +version of the SHA256 has of the 256-bit encryption key + +#### Encryption Context Provider + +ABFS driver supports an interface called `EncryptionContextProvider` that +can be used as a plugin for clients to provide custom implementations for +the encryption framework. This framework allows for an `encryptionContext` +and an `encryptionKey` to be generated by the EncryptionContextProvider for +a file to be created. The server keeps track of the encryptionContext for +each file. To perform subsequent operations such as read on the encrypted file, +ABFS driver will fetch the corresponding encryption key from the +EncryptionContextProvider implementation by providing the encryptionContext +string retrieved from a GetFileStatus request to the server. + +`fs.azure.encryption.context.provider.type`: The canonical name of the class +implementing EncryptionContextProvider. + ### Server Options When the config `fs.azure.io.read.tolerate.concurrent.append` is made true, the If-Match header sent to the server for read calls will be set as * otherwise the diff --git a/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/ITestCustomEncryption.java b/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/ITestAbfsCustomEncryption.java similarity index 96% rename from hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/ITestCustomEncryption.java rename to hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/ITestAbfsCustomEncryption.java index 99e4e9c6ac847..055a2fcb77158 100644 --- a/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/ITestCustomEncryption.java +++ b/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/ITestAbfsCustomEncryption.java @@ -23,6 +23,7 @@ import java.util.Arrays; import java.util.Base64; import java.util.Hashtable; +import java.util.Random; import org.apache.hadoop.fs.permission.FsPermission; import org.assertj.core.api.Assertions; @@ -54,6 +55,7 @@ import static org.apache.hadoop.fs.azurebfs.constants.HttpHeaderConfigurations.X_MS_ENCRYPTION_KEY_SHA256; import static org.apache.hadoop.fs.azurebfs.constants.HttpHeaderConfigurations.X_MS_REQUEST_SERVER_ENCRYPTED; import static org.apache.hadoop.fs.azurebfs.constants.HttpHeaderConfigurations.X_MS_SERVER_ENCRYPTED; +import static org.apache.hadoop.fs.azurebfs.constants.TestConfigurationKeys.ENCRYPTION_KEY_LEN; import static org.apache.hadoop.fs.azurebfs.contracts.services.AppendRequestParameters.Mode.APPEND_MODE; import static org.apache.hadoop.fs.azurebfs.utils.AclTestHelpers.aclEntry; import static org.apache.hadoop.fs.azurebfs.utils.EncryptionType.ENCRYPTION_CONTEXT; @@ -64,10 +66,9 @@ import static org.apache.hadoop.fs.permission.FsAction.ALL; @RunWith(Parameterized.class) -public class ITestCustomEncryption extends AbstractAbfsIntegrationTest { - private final String cpk = "12345678901234567890123456789012"; - private final String cpkSHAEncoded = EncryptionAdapter.getBase64EncodedString( - EncryptionAdapter.getSHA256Hash(cpk)); +public class ITestAbfsCustomEncryption extends AbstractAbfsIntegrationTest { + private final byte[] cpk = new byte[ENCRYPTION_KEY_LEN]; + private final String cpkSHAEncoded; // Encryption type used by filesystem while creating file @Parameterized.Parameter @@ -137,8 +138,11 @@ public static Iterable params() { }); } - public ITestCustomEncryption() throws Exception { + public ITestAbfsCustomEncryption() throws Exception { super(); + new Random().nextBytes(cpk); + cpkSHAEncoded = EncryptionAdapter.getBase64EncodedString( + EncryptionAdapter.getSHA256Hash(cpk)); } @Test @@ -157,7 +161,8 @@ public void testCustomEncryptionCombinations() throws Exception { if (requestEncryptionType == ENCRYPTION_CONTEXT) { String encryptionContext = ecp.getEncryptionContextForTest(relativePath); String expectedKeySHA = EncryptionAdapter.getBase64EncodedString( - EncryptionAdapter.getSHA256Hash(ecp.getEncryptionKeyForTest(encryptionContext))); + EncryptionAdapter.getSHA256Hash( + ecp.getEncryptionKeyForTest(encryptionContext))); Assertions.assertThat(httpOp.getResponseHeader(X_MS_ENCRYPTION_KEY_SHA256)) .isEqualTo(expectedKeySHA); } else { // GLOBAL_KEY diff --git a/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/extensions/MockEncryptionContextProvider.java b/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/extensions/MockEncryptionContextProvider.java index fbdb40f44c8b3..74ff51eb7c9f0 100644 --- a/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/extensions/MockEncryptionContextProvider.java +++ b/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/extensions/MockEncryptionContextProvider.java @@ -22,11 +22,10 @@ import java.io.IOException; import java.nio.charset.StandardCharsets; import java.util.Arrays; -import java.util.Base64; import java.util.HashMap; +import java.util.Random; import java.util.UUID; -import org.apache.commons.lang3.RandomStringUtils; import org.apache.hadoop.conf.Configuration; import org.apache.hadoop.classification.VisibleForTesting; @@ -45,8 +44,9 @@ public SecretKey getEncryptionContext(String path) throws IOException { String newContext = UUID.randomUUID().toString(); pathToContextMap.put(path, newContext); - String keyString = RandomStringUtils.random(ENCRYPTION_KEY_LEN, true, true); - Key key = new Key(keyString.getBytes(StandardCharsets.UTF_8)); + byte[] newKey = new byte[ENCRYPTION_KEY_LEN]; + new Random().nextBytes(newKey); + Key key = new Key(newKey); contextToKeyMap.put(newContext, key); return new Key(newContext.getBytes(StandardCharsets.UTF_8)); } @@ -100,8 +100,8 @@ public void destroy() { } @VisibleForTesting - public String getEncryptionKeyForTest(String encryptionContext) { - return new String(contextToKeyMap.get(encryptionContext).getEncoded()); + public byte[] getEncryptionKeyForTest(String encryptionContext) { + return contextToKeyMap.get(encryptionContext).getEncoded(); } @VisibleForTesting From b840da31a0459703562c4a8a65ad354ef0459c77 Mon Sep 17 00:00:00 2001 From: sumangala Date: Mon, 3 Jan 2022 04:32:16 +0530 Subject: [PATCH 25/77] yetus fix --- .../hadoop/fs/azurebfs/security/EncryptionAdapter.java | 5 +---- 1 file changed, 1 insertion(+), 4 deletions(-) diff --git a/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/security/EncryptionAdapter.java b/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/security/EncryptionAdapter.java index 86b5a1691c4f5..7b818d90669a5 100644 --- a/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/security/EncryptionAdapter.java +++ b/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/security/EncryptionAdapter.java @@ -19,7 +19,6 @@ package org.apache.hadoop.fs.azurebfs.security; import java.io.IOException; -import java.nio.charset.StandardCharsets; import java.security.MessageDigest; import java.security.NoSuchAlgorithmException; import java.util.Arrays; @@ -28,7 +27,7 @@ import javax.security.auth.DestroyFailedException; import javax.security.auth.Destroyable; -import org.apache.hadoop.thirdparty.com.google.common.base.Preconditions; +import org.apache.hadoop.util.Preconditions; import org.slf4j.Logger; import org.slf4j.LoggerFactory; @@ -41,8 +40,6 @@ public class EncryptionAdapter implements Destroyable { private final EncryptionContextProvider provider; private String encodedKey = null; private String encodedKeySHA = null; - private static final Logger LOG = - LoggerFactory.getLogger(EncryptionAdapter.class); public EncryptionAdapter(EncryptionContextProvider provider, String path, byte[] encryptionContext) throws IOException { From 72b5f2f1112fef242400b0f50c675c303aa640ab Mon Sep 17 00:00:00 2001 From: sumangala Date: Mon, 3 Jan 2022 09:00:21 +0530 Subject: [PATCH 26/77] yetus blanks --- .../azurebfs/security/EncryptionAdapter.java | 16 ++++----- .../hadoop-azure/src/site/markdown/abfs.md | 34 +++++++++---------- 2 files changed, 24 insertions(+), 26 deletions(-) diff --git a/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/security/EncryptionAdapter.java b/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/security/EncryptionAdapter.java index 7b818d90669a5..4a3e517de02dd 100644 --- a/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/security/EncryptionAdapter.java +++ b/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/security/EncryptionAdapter.java @@ -28,8 +28,6 @@ import javax.security.auth.Destroyable; import org.apache.hadoop.util.Preconditions; -import org.slf4j.Logger; -import org.slf4j.LoggerFactory; import org.apache.hadoop.fs.azurebfs.extensions.EncryptionContextProvider; @@ -46,7 +44,7 @@ public EncryptionAdapter(EncryptionContextProvider provider, String path, this(provider, path); Preconditions.checkNotNull(encryptionContext, "Encryption context should not be null."); - this.encryptionContext = new ABFSSecretKey(Base64.getDecoder().decode(encryptionContext)); + this.encryptionContext = new ABFSKey(Base64.getDecoder().decode(encryptionContext)); } public EncryptionAdapter(EncryptionContextProvider provider, String path) @@ -96,10 +94,10 @@ public void destroy() throws DestroyFailedException { provider.destroy(); } - public class ABFSSecretKey implements SecretKey { - private final byte[] secret; - public ABFSSecretKey(byte[] secret) { - this.secret = secret; + public class ABFSKey implements SecretKey { + private final byte[] bytes; + public ABFSKey(byte[] bytes) { + this.bytes = bytes; } @Override @@ -114,12 +112,12 @@ public String getFormat() { @Override public byte[] getEncoded() { - return secret; + return bytes; } @Override public void destroy() { - Arrays.fill(secret, (byte) 0); + Arrays.fill(bytes, (byte) 0); } } diff --git a/hadoop-tools/hadoop-azure/src/site/markdown/abfs.md b/hadoop-tools/hadoop-azure/src/site/markdown/abfs.md index 4f6c230c01f3a..8b985d26b353e 100644 --- a/hadoop-tools/hadoop-azure/src/site/markdown/abfs.md +++ b/hadoop-tools/hadoop-azure/src/site/markdown/abfs.md @@ -891,35 +891,35 @@ DelegatingSSLSocketFactory.SSLChannelMode. The default value will be DelegatingSSLSocketFactory.SSLChannelMode.Default. ### Encryption Options -Only one of the following two options can be configured. If config values of -both types are set, ABFS driver will throw an exception. If using the global +Only one of the following two options can be configured. If config values of +both types are set, ABFS driver will throw an exception. If using the global key type, ensure both pre-computed values are provided. #### Customer-Provided Global Key -A global encryption key can be configured by providing the following -pre-computed values. The key will be applied to any new files created post -setting the configuration, and will be required in the requests to read ro +A global encryption key can be configured by providing the following +pre-computed values. The key will be applied to any new files created post +setting the configuration, and will be required in the requests to read ro modify the contents of the files. -`fs.azure.encryption.encoded.client-provided-key`: The Base64 encoded version +`fs.azure.encryption.encoded.client-provided-key`: The Base64 encoded version of the 256-bit encryption key. -`fs.azure.encryption.encoded.client-provided-key-sha`: The Base64 encoded -version of the SHA256 has of the 256-bit encryption key +`fs.azure.encryption.encoded.client-provided-key-sha`: The Base64 encoded +version of the SHA256 has of the 256-bit encryption key. #### Encryption Context Provider -ABFS driver supports an interface called `EncryptionContextProvider` that -can be used as a plugin for clients to provide custom implementations for -the encryption framework. This framework allows for an `encryptionContext` -and an `encryptionKey` to be generated by the EncryptionContextProvider for -a file to be created. The server keeps track of the encryptionContext for -each file. To perform subsequent operations such as read on the encrypted file, -ABFS driver will fetch the corresponding encryption key from the -EncryptionContextProvider implementation by providing the encryptionContext +ABFS driver supports an interface called `EncryptionContextProvider` that +can be used as a plugin for clients to provide custom implementations for +the encryption framework. This framework allows for an `encryptionContext` +and an `encryptionKey` to be generated by the EncryptionContextProvider for +a file to be created. The server keeps track of the encryptionContext for +each file. To perform subsequent operations such as read on the encrypted file, +ABFS driver will fetch the corresponding encryption key from the +EncryptionContextProvider implementation by providing the encryptionContext string retrieved from a GetFileStatus request to the server. -`fs.azure.encryption.context.provider.type`: The canonical name of the class +`fs.azure.encryption.context.provider.type`: The canonical name of the class implementing EncryptionContextProvider. ### Server Options From 1721d435d6e67191e20da178e4bad043b2dfc0e1 Mon Sep 17 00:00:00 2001 From: sumangala Date: Wed, 11 May 2022 14:45:48 +0530 Subject: [PATCH 27/77] merge --- .../azurebfs/ITestAbfsCustomEncryption.java | 2 +- .../fs/azurebfs/ITestCustomerProvidedKey.java | 978 ------------------ 2 files changed, 1 insertion(+), 979 deletions(-) delete mode 100644 hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/ITestCustomerProvidedKey.java diff --git a/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/ITestAbfsCustomEncryption.java b/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/ITestAbfsCustomEncryption.java index 055a2fcb77158..a914c432df267 100644 --- a/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/ITestAbfsCustomEncryption.java +++ b/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/ITestAbfsCustomEncryption.java @@ -236,7 +236,7 @@ private AbfsRestOperation callOperation(AzureBlobFileSystem fs, case LISTSTATUS: return client.listPath(path, false, 5, null, getTestTracingContext(fs, true)); case RENAME: return client.renamePath(path, new Path(path + "_2").toString(), - null, getTestTracingContext(fs, true)); + null, getTestTracingContext(fs, true), null).getLeft(); case DELETE: return client.deletePath(path, false, null, getTestTracingContext(fs, false)); case GET_ATTR: return client.getPathStatus(path, true, diff --git a/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/ITestCustomerProvidedKey.java b/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/ITestCustomerProvidedKey.java deleted file mode 100644 index 02260310bb813..0000000000000 --- a/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/ITestCustomerProvidedKey.java +++ /dev/null @@ -1,978 +0,0 @@ -/** - * 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 ("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.fs.azurebfs; - -import java.io.FileNotFoundException; -import java.io.IOException; -import java.nio.CharBuffer; -import java.nio.charset.CharacterCodingException; -import java.nio.charset.Charset; -import java.nio.charset.CharsetEncoder; -import java.nio.charset.StandardCharsets; -import java.security.MessageDigest; -import java.security.NoSuchAlgorithmException; -import java.util.EnumSet; -import java.util.Hashtable; -import java.util.List; -import java.util.Map; -import java.util.Optional; -import java.util.Random; -import java.util.UUID; - -import org.apache.hadoop.fs.azurebfs.utils.TracingContext; -import org.apache.hadoop.fs.contract.ContractTestUtils; -import org.apache.hadoop.util.Preconditions; -import org.apache.hadoop.util.Lists; -import org.assertj.core.api.Assertions; -import org.junit.Assume; -import org.junit.Test; -import org.slf4j.Logger; -import org.slf4j.LoggerFactory; - -import org.apache.hadoop.conf.Configuration; -import org.apache.hadoop.fs.FSDataInputStream; -import org.apache.hadoop.fs.FSDataOutputStream; -import org.apache.hadoop.fs.FileStatus; -import org.apache.hadoop.fs.FileSystem; -import org.apache.hadoop.fs.Path; -import org.apache.hadoop.fs.XAttrSetFlag; -import org.apache.hadoop.fs.azurebfs.constants.AbfsHttpConstants; -import org.apache.hadoop.fs.azurebfs.constants.HttpHeaderConfigurations; -import org.apache.hadoop.fs.azurebfs.contracts.services.AppendRequestParameters; -import org.apache.hadoop.fs.azurebfs.contracts.services.AppendRequestParameters.Mode; -import org.apache.hadoop.fs.azurebfs.services.AbfsAclHelper; -import org.apache.hadoop.fs.azurebfs.services.AbfsClient; -import org.apache.hadoop.fs.azurebfs.services.AbfsHttpHeader; -import org.apache.hadoop.fs.azurebfs.services.AbfsHttpOperation; -import org.apache.hadoop.fs.azurebfs.services.AbfsRestOperation; -import org.apache.hadoop.fs.azurebfs.services.AuthType; -import org.apache.hadoop.fs.azurebfs.utils.Base64; -import org.apache.hadoop.fs.permission.AclEntry; -import org.apache.hadoop.fs.permission.FsAction; -import org.apache.hadoop.fs.permission.FsPermission; -import org.apache.hadoop.test.LambdaTestUtils; - -import static org.apache.hadoop.fs.azurebfs.constants.ConfigurationKeys.AZURE_CREATE_REMOTE_FILESYSTEM_DURING_INITIALIZATION; -import static org.apache.hadoop.fs.azurebfs.constants.ConfigurationKeys.FS_AZURE_CLIENT_PROVIDED_ENCRYPTION_KEY; -import static org.apache.hadoop.fs.azurebfs.constants.FileSystemConfigurations.ONE_MB; -import static org.apache.hadoop.fs.azurebfs.constants.HttpHeaderConfigurations.X_MS_ENCRYPTION_ALGORITHM; -import static org.apache.hadoop.fs.azurebfs.constants.HttpHeaderConfigurations.X_MS_ENCRYPTION_KEY; -import static org.apache.hadoop.fs.azurebfs.constants.HttpHeaderConfigurations.X_MS_ENCRYPTION_KEY_SHA256; -import static org.apache.hadoop.fs.azurebfs.constants.HttpHeaderConfigurations.X_MS_REQUEST_SERVER_ENCRYPTED; -import static org.apache.hadoop.fs.azurebfs.constants.HttpHeaderConfigurations.X_MS_SERVER_ENCRYPTED; -import static org.apache.hadoop.fs.azurebfs.constants.TestConfigurationKeys.FS_AZURE_ABFS_ACCOUNT_NAME; -import static org.apache.hadoop.fs.azurebfs.constants.TestConfigurationKeys.FS_AZURE_ACCOUNT_KEY; -import static org.apache.hadoop.fs.azurebfs.constants.TestConfigurationKeys.FS_AZURE_TEST_CPK_ENABLED; -import static org.apache.hadoop.fs.azurebfs.constants.TestConfigurationKeys.FS_AZURE_TEST_CPK_ENABLED_SECONDARY_ACCOUNT; -import static org.apache.hadoop.fs.azurebfs.constants.TestConfigurationKeys.FS_AZURE_TEST_CPK_ENABLED_SECONDARY_ACCOUNT_KEY; -import static org.apache.hadoop.fs.azurebfs.constants.TestConfigurationKeys.FS_AZURE_TEST_NAMESPACE_ENABLED_ACCOUNT; -import static org.apache.hadoop.fs.azurebfs.constants.TestConfigurationKeys.TEST_CONFIGURATION_FILE_NAME; -import static org.apache.hadoop.fs.azurebfs.utils.AclTestHelpers.aclEntry; -import static org.apache.hadoop.fs.permission.AclEntryScope.ACCESS; -import static org.apache.hadoop.fs.permission.AclEntryType.USER; -import static org.apache.hadoop.fs.permission.FsAction.ALL; - -public class ITestCustomerProvidedKey extends AbstractAbfsIntegrationTest { - private static final Logger LOG = LoggerFactory - .getLogger(ITestCustomerProvidedKey.class); - - private static final String XMS_PROPERTIES_ENCODING = "ISO-8859-1"; - private static final int INT_512 = 512; - private static final int INT_50 = 50; - private static final int ENCRYPTION_KEY_LEN = 32; - private static final int FILE_SIZE = 10 * ONE_MB; - private static final int FILE_SIZE_FOR_COPY_BETWEEN_ACCOUNTS = 24 * ONE_MB; - - public ITestCustomerProvidedKey() throws Exception { - boolean isCPKTestsEnabled = getConfiguration() - .getBoolean(FS_AZURE_TEST_CPK_ENABLED, false); - Assume.assumeTrue(isCPKTestsEnabled); - } - - @Test - public void testReadWithCPK() throws Exception { - final AzureBlobFileSystem fs = getAbfs(true); - String fileName = path("/" + methodName.getMethodName()).toString(); - createFileAndGetContent(fs, fileName, FILE_SIZE); - - AbfsClient abfsClient = fs.getAbfsClient(); - int length = FILE_SIZE; - byte[] buffer = new byte[length]; - TracingContext tracingContext = getTestTracingContext(fs, false); - final AbfsRestOperation op = abfsClient.getPathStatus(fileName, false, - tracingContext); - final String eTag = op.getResult() - .getResponseHeader(HttpHeaderConfigurations.ETAG); - AbfsRestOperation abfsRestOperation = abfsClient - .read(fileName, 0, buffer, 0, length, eTag, null, tracingContext); - assertCPKHeaders(abfsRestOperation, true); - assertResponseHeader(abfsRestOperation, true, X_MS_ENCRYPTION_KEY_SHA256, - getCPKSha(fs)); - assertResponseHeader(abfsRestOperation, true, X_MS_SERVER_ENCRYPTED, - "true"); - assertResponseHeader(abfsRestOperation, false, - X_MS_REQUEST_SERVER_ENCRYPTED, ""); - - // Trying to read with different CPK headers - Configuration conf = fs.getConf(); - String accountName = conf.get(FS_AZURE_ABFS_ACCOUNT_NAME); - conf.set(FS_AZURE_CLIENT_PROVIDED_ENCRYPTION_KEY + "." + accountName, - "different-1234567890123456789012"); - try (AzureBlobFileSystem fs2 = (AzureBlobFileSystem) FileSystem.newInstance(conf); - FSDataInputStream iStream = fs2.open(new Path(fileName))) { - int len = 8 * ONE_MB; - byte[] b = new byte[len]; - LambdaTestUtils.intercept(IOException.class, () -> { - iStream.read(b, 0, len); - }); - } - - // Trying to read with no CPK headers - conf.unset(FS_AZURE_CLIENT_PROVIDED_ENCRYPTION_KEY + "." + accountName); - try (AzureBlobFileSystem fs3 = (AzureBlobFileSystem) FileSystem - .get(conf); FSDataInputStream iStream = fs3.open(new Path(fileName))) { - int len = 8 * ONE_MB; - byte[] b = new byte[len]; - LambdaTestUtils.intercept(IOException.class, () -> { - iStream.read(b, 0, len); - }); - } - } - - @Test - public void testReadWithoutCPK() throws Exception { - final AzureBlobFileSystem fs = getAbfs(false); - String fileName = path("/" + methodName.getMethodName()).toString(); - createFileAndGetContent(fs, fileName, FILE_SIZE); - - AbfsClient abfsClient = fs.getAbfsClient(); - int length = INT_512; - byte[] buffer = new byte[length * 4]; - TracingContext tracingContext = getTestTracingContext(fs, false); - final AbfsRestOperation op = abfsClient - .getPathStatus(fileName, false, tracingContext); - final String eTag = op.getResult() - .getResponseHeader(HttpHeaderConfigurations.ETAG); - AbfsRestOperation abfsRestOperation = abfsClient - .read(fileName, 0, buffer, 0, length, eTag, null, tracingContext); - assertCPKHeaders(abfsRestOperation, false); - assertResponseHeader(abfsRestOperation, false, X_MS_ENCRYPTION_KEY_SHA256, - getCPKSha(fs)); - assertResponseHeader(abfsRestOperation, true, X_MS_SERVER_ENCRYPTED, - "true"); - assertResponseHeader(abfsRestOperation, false, - X_MS_REQUEST_SERVER_ENCRYPTED, ""); - - // Trying to read with CPK headers - Configuration conf = fs.getConf(); - String accountName = conf.get(FS_AZURE_ABFS_ACCOUNT_NAME); - conf.set(FS_AZURE_CLIENT_PROVIDED_ENCRYPTION_KEY + "." + accountName, - "12345678901234567890123456789012"); - - try (AzureBlobFileSystem fs2 = (AzureBlobFileSystem) FileSystem.newInstance(conf); - AbfsClient abfsClient2 = fs2.getAbfsClient()) { - LambdaTestUtils.intercept(IOException.class, () -> { - abfsClient2.read(fileName, 0, buffer, 0, length, eTag, null, - getTestTracingContext(fs, false)); - }); - } - } - - @Test - public void testAppendWithCPK() throws Exception { - final AzureBlobFileSystem fs = getAbfs(true); - final String fileName = path("/" + methodName.getMethodName()).toString(); - createFileAndGetContent(fs, fileName, FILE_SIZE); - - // Trying to append with correct CPK headers - AppendRequestParameters appendRequestParameters = - new AppendRequestParameters( - 0, 0, 5, Mode.APPEND_MODE, false, null); - byte[] buffer = getRandomBytesArray(5); - AbfsClient abfsClient = fs.getAbfsClient(); - AbfsRestOperation abfsRestOperation = abfsClient - .append(fileName, buffer, appendRequestParameters, null, getTestTracingContext(fs, false)); - assertCPKHeaders(abfsRestOperation, true); - assertResponseHeader(abfsRestOperation, true, X_MS_ENCRYPTION_KEY_SHA256, - getCPKSha(fs)); - assertResponseHeader(abfsRestOperation, false, X_MS_SERVER_ENCRYPTED, ""); - assertResponseHeader(abfsRestOperation, true, X_MS_REQUEST_SERVER_ENCRYPTED, - "true"); - - // Trying to append with different CPK headers - Configuration conf = fs.getConf(); - String accountName = conf.get(FS_AZURE_ABFS_ACCOUNT_NAME); - conf.set(FS_AZURE_CLIENT_PROVIDED_ENCRYPTION_KEY + "." + accountName, - "different-1234567890123456789012"); - try (AzureBlobFileSystem fs2 = (AzureBlobFileSystem) FileSystem.newInstance(conf); - AbfsClient abfsClient2 = fs2.getAbfsClient()) { - LambdaTestUtils.intercept(IOException.class, () -> { - abfsClient2.append(fileName, buffer, appendRequestParameters, null, - getTestTracingContext(fs, false)); - }); - } - - // Trying to append with no CPK headers - conf.unset(FS_AZURE_CLIENT_PROVIDED_ENCRYPTION_KEY + "." + accountName); - try (AzureBlobFileSystem fs3 = (AzureBlobFileSystem) FileSystem - .get(conf); AbfsClient abfsClient3 = fs3.getAbfsClient()) { - LambdaTestUtils.intercept(IOException.class, () -> { - abfsClient3.append(fileName, buffer, appendRequestParameters, null, - getTestTracingContext(fs, false)); - }); - } - } - - @Test - public void testAppendWithoutCPK() throws Exception { - final AzureBlobFileSystem fs = getAbfs(false); - final String fileName = path("/" + methodName.getMethodName()).toString(); - createFileAndGetContent(fs, fileName, FILE_SIZE); - - // Trying to append without CPK headers - AppendRequestParameters appendRequestParameters = - new AppendRequestParameters( - 0, 0, 5, Mode.APPEND_MODE, false, null); - byte[] buffer = getRandomBytesArray(5); - AbfsClient abfsClient = fs.getAbfsClient(); - AbfsRestOperation abfsRestOperation = abfsClient - .append(fileName, buffer, appendRequestParameters, null, - getTestTracingContext(fs, false)); - assertCPKHeaders(abfsRestOperation, false); - assertResponseHeader(abfsRestOperation, false, X_MS_ENCRYPTION_KEY_SHA256, - ""); - assertResponseHeader(abfsRestOperation, false, X_MS_SERVER_ENCRYPTED, ""); - assertResponseHeader(abfsRestOperation, true, X_MS_REQUEST_SERVER_ENCRYPTED, - "true"); - - // Trying to append with CPK headers - Configuration conf = fs.getConf(); - String accountName = conf.get(FS_AZURE_ABFS_ACCOUNT_NAME); - conf.set(FS_AZURE_CLIENT_PROVIDED_ENCRYPTION_KEY + "." + accountName, - "12345678901234567890123456789012"); - try (AzureBlobFileSystem fs2 = (AzureBlobFileSystem) FileSystem.newInstance(conf); - AbfsClient abfsClient2 = fs2.getAbfsClient()) { - LambdaTestUtils.intercept(IOException.class, () -> { - abfsClient2.append(fileName, buffer, appendRequestParameters, null, - getTestTracingContext(fs, false)); - }); - } - } - - @Test - public void testSetGetXAttr() throws Exception { - final AzureBlobFileSystem fs = getAbfs(true); - final String fileName = path(methodName.getMethodName()).toString(); - createFileAndGetContent(fs, fileName, FILE_SIZE); - - String valSent = "testValue"; - String attrName = "testXAttr"; - - // set get and verify - fs.setXAttr(new Path(fileName), attrName, - valSent.getBytes(StandardCharsets.UTF_8), - EnumSet.of(XAttrSetFlag.CREATE)); - byte[] valBytes = fs.getXAttr(new Path(fileName), attrName); - String valRecieved = new String(valBytes); - assertEquals(valSent, valRecieved); - - // set new value get and verify - valSent = "new value"; - fs.setXAttr(new Path(fileName), attrName, - valSent.getBytes(StandardCharsets.UTF_8), - EnumSet.of(XAttrSetFlag.REPLACE)); - valBytes = fs.getXAttr(new Path(fileName), attrName); - valRecieved = new String(valBytes); - assertEquals(valSent, valRecieved); - - // Read without CPK header - LambdaTestUtils.intercept(IOException.class, () -> { - getAbfs(false).getXAttr(new Path(fileName), attrName); - }); - - // Wrong CPK - LambdaTestUtils.intercept(IOException.class, () -> { - getSameFSWithWrongCPK(fs).getXAttr(new Path(fileName), attrName); - }); - } - - @Test - public void testCopyBetweenAccounts() throws Exception { - String accountName = getRawConfiguration() - .get(FS_AZURE_TEST_CPK_ENABLED_SECONDARY_ACCOUNT); - String accountKey = getRawConfiguration() - .get(FS_AZURE_TEST_CPK_ENABLED_SECONDARY_ACCOUNT_KEY); - Assume.assumeTrue(accountName != null && !accountName.isEmpty()); - Assume.assumeTrue(accountKey != null && !accountKey.isEmpty()); - String fileSystemName = "cpkfs"; - - // Create fs1 and a file with CPK - AzureBlobFileSystem fs1 = getAbfs(true); - int fileSize = FILE_SIZE_FOR_COPY_BETWEEN_ACCOUNTS; - byte[] fileContent = getRandomBytesArray(fileSize); - Path testFilePath = createFileWithContent(fs1, - String.format("fs1-file%s.txt", UUID.randomUUID()), fileContent); - - // Create fs2 with different CPK - Configuration conf = new Configuration(); - conf.addResource(TEST_CONFIGURATION_FILE_NAME); - conf.setBoolean(AZURE_CREATE_REMOTE_FILESYSTEM_DURING_INITIALIZATION, true); - conf.unset(FS_AZURE_ABFS_ACCOUNT_NAME); - conf.set(FS_AZURE_ABFS_ACCOUNT_NAME, accountName); - conf.set(FS_AZURE_ACCOUNT_KEY + "." + accountName, accountKey); - conf.set(FS_AZURE_CLIENT_PROVIDED_ENCRYPTION_KEY + "." + accountName, - "123456789012345678901234567890ab"); - conf.set("fs.defaultFS", "abfs://" + fileSystemName + "@" + accountName); - AzureBlobFileSystem fs2 = (AzureBlobFileSystem) FileSystem.newInstance(conf); - - // Read from fs1 and write to fs2, fs1 and fs2 are having different CPK - Path fs2DestFilePath = new Path( - String.format("fs2-dest-file%s.txt", UUID.randomUUID())); - FSDataOutputStream ops = fs2.create(fs2DestFilePath); - try (FSDataInputStream iStream = fs1.open(testFilePath)) { - long totalBytesRead = 0; - do { - int length = 8 * ONE_MB; - byte[] buffer = new byte[length]; - int bytesRead = iStream.read(buffer, 0, length); - totalBytesRead += bytesRead; - ops.write(buffer); - } while (totalBytesRead < fileContent.length); - ops.close(); - } - - // Trying to read fs2DestFilePath with different CPK headers - conf.unset(FS_AZURE_CLIENT_PROVIDED_ENCRYPTION_KEY + "." + accountName); - conf.set(FS_AZURE_CLIENT_PROVIDED_ENCRYPTION_KEY + "." + accountName, - "different-1234567890123456789012"); - try (AzureBlobFileSystem fs3 = (AzureBlobFileSystem) FileSystem - .get(conf); FSDataInputStream iStream = fs3.open(fs2DestFilePath)) { - int length = 8 * ONE_MB; - byte[] buffer = new byte[length]; - LambdaTestUtils.intercept(IOException.class, () -> { - iStream.read(buffer, 0, length); - }); - } - - // Trying to read fs2DestFilePath with no CPK headers - conf.unset(FS_AZURE_CLIENT_PROVIDED_ENCRYPTION_KEY + "." + accountName); - try (AzureBlobFileSystem fs4 = (AzureBlobFileSystem) FileSystem - .get(conf); FSDataInputStream iStream = fs4.open(fs2DestFilePath)) { - int length = 8 * ONE_MB; - byte[] buffer = new byte[length]; - LambdaTestUtils.intercept(IOException.class, () -> { - iStream.read(buffer, 0, length); - }); - } - - // Read fs2DestFilePath and verify the content with the initial random - // bytes created and wrote into the source file at fs1 - try (FSDataInputStream iStream = fs2.open(fs2DestFilePath)) { - long totalBytesRead = 0; - int pos = 0; - do { - int length = 8 * ONE_MB; - byte[] buffer = new byte[length]; - int bytesRead = iStream.read(buffer, 0, length); - totalBytesRead += bytesRead; - for (int i = 0; i < bytesRead; i++) { - assertEquals(fileContent[pos + i], buffer[i]); - } - pos = pos + bytesRead; - } while (totalBytesRead < fileContent.length); - } - } - - @Test - public void testListPathWithCPK() throws Exception { - testListPath(true); - } - - @Test - public void testListPathWithoutCPK() throws Exception { - testListPath(false); - } - - private void testListPath(final boolean isWithCPK) throws Exception { - final AzureBlobFileSystem fs = getAbfs(isWithCPK); - final Path testPath = path("/" + methodName.getMethodName()); - String testDirName = testPath.toString(); - fs.mkdirs(testPath); - createFileAndGetContent(fs, testDirName + "/aaa", FILE_SIZE); - createFileAndGetContent(fs, testDirName + "/bbb", FILE_SIZE); - AbfsClient abfsClient = fs.getAbfsClient(); - AbfsRestOperation abfsRestOperation = abfsClient - .listPath(testDirName, false, INT_50, null, - getTestTracingContext(fs, false)); - assertListstatus(fs, abfsRestOperation, testPath); - - // Trying with different CPK headers - Configuration conf = fs.getConf(); - String accountName = conf.get(FS_AZURE_ABFS_ACCOUNT_NAME); - conf.set(FS_AZURE_CLIENT_PROVIDED_ENCRYPTION_KEY + "." + accountName, - "different-1234567890123456789012"); - AzureBlobFileSystem fs2 = (AzureBlobFileSystem) FileSystem.newInstance(conf); - AbfsClient abfsClient2 = fs2.getAbfsClient(); - TracingContext tracingContext = getTestTracingContext(fs, false); - abfsRestOperation = abfsClient2.listPath(testDirName, false, INT_50, - null, tracingContext); - assertListstatus(fs, abfsRestOperation, testPath); - - if (isWithCPK) { - // Trying with no CPK headers - conf.unset(FS_AZURE_CLIENT_PROVIDED_ENCRYPTION_KEY + "." + accountName); - AzureBlobFileSystem fs3 = (AzureBlobFileSystem) FileSystem.get(conf); - AbfsClient abfsClient3 = fs3.getAbfsClient(); - abfsRestOperation = abfsClient3 - .listPath(testDirName, false, INT_50, null, tracingContext); - assertListstatus(fs, abfsRestOperation, testPath); - } - } - - private void assertListstatus(AzureBlobFileSystem fs, - AbfsRestOperation abfsRestOperation, Path testPath) throws IOException { - assertCPKHeaders(abfsRestOperation, false); - assertNoCPKResponseHeadersPresent(abfsRestOperation); - - FileStatus[] listStatuses = fs.listStatus(testPath); - Assertions.assertThat(listStatuses.length) - .describedAs("listStatuses should have 2 entries").isEqualTo(2); - - listStatuses = getSameFSWithWrongCPK(fs).listStatus(testPath); - Assertions.assertThat(listStatuses.length) - .describedAs("listStatuses should have 2 entries").isEqualTo(2); - } - - @Test - public void testCreatePathWithCPK() throws Exception { - testCreatePath(true); - } - - @Test - public void testCreatePathWithoutCPK() throws Exception { - testCreatePath(false); - } - - private void testCreatePath(final boolean isWithCPK) throws Exception { - final AzureBlobFileSystem fs = getAbfs(isWithCPK); - final String testFileName = path("/" + methodName.getMethodName()) - .toString(); - createFileAndGetContent(fs, testFileName, FILE_SIZE); - - AbfsClient abfsClient = fs.getAbfsClient(); - FsPermission permission = new FsPermission(FsAction.EXECUTE, - FsAction.EXECUTE, FsAction.EXECUTE); - FsPermission umask = new FsPermission(FsAction.NONE, FsAction.NONE, - FsAction.NONE); - TracingContext tracingContext = getTestTracingContext(fs, false); - boolean isNamespaceEnabled = fs.getIsNamespaceEnabled(tracingContext); - AbfsRestOperation abfsRestOperation = abfsClient - .createPath(testFileName, true, true, - isNamespaceEnabled ? getOctalNotation(permission) : null, - isNamespaceEnabled ? getOctalNotation(umask) : null, false, null, - tracingContext); - assertCPKHeaders(abfsRestOperation, isWithCPK); - assertResponseHeader(abfsRestOperation, isWithCPK, - X_MS_ENCRYPTION_KEY_SHA256, getCPKSha(fs)); - assertResponseHeader(abfsRestOperation, false, X_MS_SERVER_ENCRYPTED, ""); - assertResponseHeader(abfsRestOperation, true, X_MS_REQUEST_SERVER_ENCRYPTED, - "true"); - - FileStatus[] listStatuses = fs.listStatus(new Path(testFileName)); - Assertions.assertThat(listStatuses.length) - .describedAs("listStatuses should have 1 entry").isEqualTo(1); - - listStatuses = getSameFSWithWrongCPK(fs).listStatus(new Path(testFileName)); - Assertions.assertThat(listStatuses.length) - .describedAs("listStatuses should have 1 entry").isEqualTo(1); - } - - @Test - public void testRenamePathWithCPK() throws Exception { - testRenamePath(true); - } - - @Test - public void testRenamePathWithoutCPK() throws Exception { - testRenamePath(false); - } - - private void testRenamePath(final boolean isWithCPK) throws Exception { - final AzureBlobFileSystem fs = getAbfs(isWithCPK); - final String testFileName = path("/" + methodName.getMethodName()) - .toString(); - createFileAndGetContent(fs, testFileName, FILE_SIZE); - - FileStatus fileStatusBeforeRename = fs - .getFileStatus(new Path(testFileName)); - - String newName = "/newName"; - AbfsClient abfsClient = fs.getAbfsClient(); - AbfsRestOperation abfsRestOperation = abfsClient - .renamePath(testFileName, newName, null, - getTestTracingContext(fs, false), null) - .getLeft(); - assertCPKHeaders(abfsRestOperation, false); - assertNoCPKResponseHeadersPresent(abfsRestOperation); - - LambdaTestUtils.intercept(FileNotFoundException.class, - (() -> fs.getFileStatus(new Path(testFileName)))); - - FileStatus fileStatusAfterRename = fs.getFileStatus(new Path(newName)); - Assertions.assertThat(fileStatusAfterRename.getLen()) - .describedAs("File size has to be same before and after rename") - .isEqualTo(fileStatusBeforeRename.getLen()); - } - - @Test - public void testFlushWithCPK() throws Exception { - testFlush(true); - } - - @Test - public void testFlushWithoutCPK() throws Exception { - testFlush(false); - } - - private void testFlush(final boolean isWithCPK) throws Exception { - final AzureBlobFileSystem fs = getAbfs(isWithCPK); - final String testFileName = path("/" + methodName.getMethodName()) - .toString(); - fs.create(new Path(testFileName)).close(); - AbfsClient abfsClient = fs.getAbfsClient(); - String expectedCPKSha = getCPKSha(fs); - - byte[] fileContent = getRandomBytesArray(FILE_SIZE); - Path testFilePath = new Path(testFileName + "1"); - try (FSDataOutputStream oStream = fs.create(testFilePath)) { - oStream.write(fileContent); - } - - // Trying to read with different CPK headers - Configuration conf = fs.getConf(); - String accountName = conf.get(FS_AZURE_ABFS_ACCOUNT_NAME); - conf.set(FS_AZURE_CLIENT_PROVIDED_ENCRYPTION_KEY + "." + accountName, - "different-1234567890123456789012"); - try (AzureBlobFileSystem fs2 = (AzureBlobFileSystem) FileSystem.newInstance(conf); - AbfsClient abfsClient2 = fs2.getAbfsClient()) { - LambdaTestUtils.intercept(IOException.class, () -> { - abfsClient2.flush(testFileName, 0, false, false, null, null, - getTestTracingContext(fs, false)); - }); - } - - // Trying to read with no CPK headers - if (isWithCPK) { - conf.unset(FS_AZURE_CLIENT_PROVIDED_ENCRYPTION_KEY + "." + accountName); - try (AzureBlobFileSystem fs3 = (AzureBlobFileSystem) FileSystem - .get(conf); AbfsClient abfsClient3 = fs3.getAbfsClient()) { - LambdaTestUtils.intercept(IOException.class, () -> { - abfsClient3.flush(testFileName, 0, false, false, null, null, - getTestTracingContext(fs, false)); - }); - } - } - - // With correct CPK - AbfsRestOperation abfsRestOperation = abfsClient - .flush(testFileName, 0, false, false, null, null, - getTestTracingContext(fs, false)); - assertCPKHeaders(abfsRestOperation, isWithCPK); - assertResponseHeader(abfsRestOperation, isWithCPK, - X_MS_ENCRYPTION_KEY_SHA256, expectedCPKSha); - assertResponseHeader(abfsRestOperation, false, X_MS_SERVER_ENCRYPTED, ""); - assertResponseHeader(abfsRestOperation, true, X_MS_REQUEST_SERVER_ENCRYPTED, - isWithCPK + ""); - } - - @Test - public void testSetPathPropertiesWithCPK() throws Exception { - testSetPathProperties(true); - } - - @Test - public void testSetPathPropertiesWithoutCPK() throws Exception { - testSetPathProperties(false); - } - - private void testSetPathProperties(final boolean isWithCPK) throws Exception { - final AzureBlobFileSystem fs = getAbfs(isWithCPK); - final String testFileName = path("/" + methodName.getMethodName()) - .toString(); - createFileAndGetContent(fs, testFileName, FILE_SIZE); - - AbfsClient abfsClient = fs.getAbfsClient(); - final Hashtable properties = new Hashtable<>(); - properties.put("key", "val"); - AbfsRestOperation abfsRestOperation = abfsClient - .setPathProperties(testFileName, - convertXmsPropertiesToCommaSeparatedString(properties), - getTestTracingContext(fs, false)); - assertCPKHeaders(abfsRestOperation, isWithCPK); - assertResponseHeader(abfsRestOperation, isWithCPK, - X_MS_ENCRYPTION_KEY_SHA256, getCPKSha(fs)); - assertResponseHeader(abfsRestOperation, false, X_MS_SERVER_ENCRYPTED, ""); - assertResponseHeader(abfsRestOperation, true, X_MS_REQUEST_SERVER_ENCRYPTED, - "true"); - } - - @Test - public void testGetPathStatusFileWithCPK() throws Exception { - testGetPathStatusFile(true); - } - - @Test - public void testGetPathStatusFileWithoutCPK() throws Exception { - testGetPathStatusFile(false); - } - - private void testGetPathStatusFile(final boolean isWithCPK) throws Exception { - final AzureBlobFileSystem fs = getAbfs(isWithCPK); - final String testFileName = path("/" + methodName.getMethodName()) - .toString(); - createFileAndGetContent(fs, testFileName, FILE_SIZE); - - AbfsClient abfsClient = fs.getAbfsClient(); - TracingContext tracingContext = getTestTracingContext(fs, false); - AbfsRestOperation abfsRestOperation = abfsClient - .getPathStatus(testFileName, false, tracingContext); - assertCPKHeaders(abfsRestOperation, false); - assertResponseHeader(abfsRestOperation, isWithCPK, - X_MS_ENCRYPTION_KEY_SHA256, getCPKSha(fs)); - assertResponseHeader(abfsRestOperation, true, X_MS_SERVER_ENCRYPTED, - "true"); - assertResponseHeader(abfsRestOperation, false, - X_MS_REQUEST_SERVER_ENCRYPTED, ""); - - abfsRestOperation = abfsClient.getPathStatus(testFileName, true, tracingContext); - assertCPKHeaders(abfsRestOperation, isWithCPK); - assertResponseHeader(abfsRestOperation, isWithCPK, - X_MS_ENCRYPTION_KEY_SHA256, getCPKSha(fs)); - assertResponseHeader(abfsRestOperation, true, X_MS_SERVER_ENCRYPTED, - "true"); - assertResponseHeader(abfsRestOperation, false, - X_MS_REQUEST_SERVER_ENCRYPTED, ""); - } - - @Test - public void testDeletePathWithCPK() throws Exception { - testDeletePath(false); - } - - @Test - public void testDeletePathWithoutCPK() throws Exception { - testDeletePath(false); - } - - private void testDeletePath(final boolean isWithCPK) throws Exception { - final AzureBlobFileSystem fs = getAbfs(isWithCPK); - final String testFileName = path("/" + methodName.getMethodName()) - .toString(); - createFileAndGetContent(fs, testFileName, FILE_SIZE); - - FileStatus[] listStatuses = fs.listStatus(new Path(testFileName)); - Assertions.assertThat(listStatuses.length) - .describedAs("listStatuses should have 1 entry").isEqualTo(1); - - AbfsClient abfsClient = fs.getAbfsClient(); - AbfsRestOperation abfsRestOperation = abfsClient - .deletePath(testFileName, false, null, - getTestTracingContext(fs, false)); - assertCPKHeaders(abfsRestOperation, false); - assertNoCPKResponseHeadersPresent(abfsRestOperation); - - Assertions.assertThatThrownBy(() -> fs.listStatus(new Path(testFileName))) - .isInstanceOf(FileNotFoundException.class); - } - - @Test - public void testSetPermissionWithCPK() throws Exception { - testSetPermission(true); - } - - @Test - public void testSetPermissionWithoutCPK() throws Exception { - testSetPermission(false); - } - - private void testSetPermission(final boolean isWithCPK) throws Exception { - final AzureBlobFileSystem fs = getAbfs(isWithCPK); - final String testFileName = path("/" + methodName.getMethodName()) - .toString(); - Assume.assumeTrue(fs.getIsNamespaceEnabled(getTestTracingContext(fs, false))); - createFileAndGetContent(fs, testFileName, FILE_SIZE); - AbfsClient abfsClient = fs.getAbfsClient(); - FsPermission permission = new FsPermission(FsAction.EXECUTE, - FsAction.EXECUTE, FsAction.EXECUTE); - AbfsRestOperation abfsRestOperation = abfsClient - .setPermission(testFileName, permission.toString(), - getTestTracingContext(fs, false)); - assertCPKHeaders(abfsRestOperation, false); - assertNoCPKResponseHeadersPresent(abfsRestOperation); - } - - @Test - public void testSetAclWithCPK() throws Exception { - testSetAcl(true); - } - - @Test - public void testSetAclWithoutCPK() throws Exception { - testSetAcl(false); - } - - private void testSetAcl(final boolean isWithCPK) throws Exception { - final AzureBlobFileSystem fs = getAbfs(isWithCPK); - final String testFileName = path("/" + methodName.getMethodName()) - .toString(); - TracingContext tracingContext = getTestTracingContext(fs, false); - Assume.assumeTrue(fs.getIsNamespaceEnabled(tracingContext)); - createFileAndGetContent(fs, testFileName, FILE_SIZE); - AbfsClient abfsClient = fs.getAbfsClient(); - - List aclSpec = Lists.newArrayList(aclEntry(ACCESS, USER, ALL)); - final Map aclEntries = AbfsAclHelper - .deserializeAclSpec(AclEntry.aclSpecToString(aclSpec)); - - AbfsRestOperation abfsRestOperation = abfsClient - .setAcl(testFileName, AbfsAclHelper.serializeAclSpec(aclEntries), - tracingContext); - assertCPKHeaders(abfsRestOperation, false); - assertNoCPKResponseHeadersPresent(abfsRestOperation); - } - - @Test - public void testGetAclWithCPK() throws Exception { - testGetAcl(true); - } - - @Test - public void testGetAclWithoutCPK() throws Exception { - testGetAcl(false); - } - - private void testGetAcl(final boolean isWithCPK) throws Exception { - final AzureBlobFileSystem fs = getAbfs(isWithCPK); - final String testFileName = path("/" + methodName.getMethodName()) - .toString(); - TracingContext tracingContext = getTestTracingContext(fs, false); - Assume.assumeTrue(fs.getIsNamespaceEnabled(tracingContext)); - createFileAndGetContent(fs, testFileName, FILE_SIZE); - AbfsClient abfsClient = fs.getAbfsClient(); - AbfsRestOperation abfsRestOperation = - abfsClient.getAclStatus(testFileName, tracingContext); - assertCPKHeaders(abfsRestOperation, false); - assertNoCPKResponseHeadersPresent(abfsRestOperation); - } - - @Test - public void testCheckAccessWithCPK() throws Exception { - testCheckAccess(true); - } - - @Test - public void testCheckAccessWithoutCPK() throws Exception { - testCheckAccess(false); - } - - private void testCheckAccess(final boolean isWithCPK) throws Exception { - boolean isHNSEnabled = getConfiguration() - .getBoolean(FS_AZURE_TEST_NAMESPACE_ENABLED_ACCOUNT, false); - Assume.assumeTrue(FS_AZURE_TEST_NAMESPACE_ENABLED_ACCOUNT + " is false", - isHNSEnabled); - Assume.assumeTrue("AuthType has to be OAuth", - getAuthType() == AuthType.OAuth); - - final AzureBlobFileSystem fs = getAbfs(isWithCPK); - final String testFileName = path("/" + methodName.getMethodName()) - .toString(); - fs.create(new Path(testFileName)).close(); - AbfsClient abfsClient = fs.getAbfsClient(); - AbfsRestOperation abfsRestOperation = abfsClient - .checkAccess(testFileName, "rwx", getTestTracingContext(fs, false)); - assertCPKHeaders(abfsRestOperation, false); - assertNoCPKResponseHeadersPresent(abfsRestOperation); - } - - private byte[] createFileAndGetContent(AzureBlobFileSystem fs, - String fileName, int fileSize) throws IOException { - byte[] fileContent = getRandomBytesArray(fileSize); - Path testFilePath = createFileWithContent(fs, fileName, fileContent); - ContractTestUtils.verifyFileContents(fs, testFilePath, fileContent); - return fileContent; - } - - private void assertCPKHeaders(AbfsRestOperation abfsRestOperation, - boolean isCPKHeaderExpected) { - assertHeader(abfsRestOperation, X_MS_ENCRYPTION_KEY, isCPKHeaderExpected); - assertHeader(abfsRestOperation, X_MS_ENCRYPTION_KEY_SHA256, - isCPKHeaderExpected); - assertHeader(abfsRestOperation, X_MS_ENCRYPTION_ALGORITHM, - isCPKHeaderExpected); - } - - private void assertNoCPKResponseHeadersPresent( - AbfsRestOperation abfsRestOperation) { - assertResponseHeader(abfsRestOperation, false, X_MS_SERVER_ENCRYPTED, ""); - assertResponseHeader(abfsRestOperation, false, - X_MS_REQUEST_SERVER_ENCRYPTED, ""); - assertResponseHeader(abfsRestOperation, false, X_MS_ENCRYPTION_KEY_SHA256, - ""); - } - - private void assertResponseHeader(AbfsRestOperation abfsRestOperation, - boolean isHeaderExpected, String headerName, String expectedValue) { - final AbfsHttpOperation result = abfsRestOperation.getResult(); - final String value = result.getResponseHeader(headerName); - if (isHeaderExpected) { - Assertions.assertThat(value).isEqualTo(expectedValue); - } else { - Assertions.assertThat(value).isNull(); - } - } - - private void assertHeader(AbfsRestOperation abfsRestOperation, - String headerName, boolean isCPKHeaderExpected) { - assertTrue(abfsRestOperation != null); - Optional header = abfsRestOperation.getRequestHeaders() - .stream().filter(abfsHttpHeader -> abfsHttpHeader.getName() - .equalsIgnoreCase(headerName)).findFirst(); - String desc; - if (isCPKHeaderExpected) { - desc = - "CPK header " + headerName + " is expected, but the same is absent."; - } else { - desc = "CPK header " + headerName - + " is not expected, but the same is present."; - } - Assertions.assertThat(header.isPresent()).describedAs(desc) - .isEqualTo(isCPKHeaderExpected); - } - - private byte[] getSHA256Hash(String key) throws IOException { - try { - final MessageDigest digester = MessageDigest.getInstance("SHA-256"); - return digester.digest(key.getBytes(StandardCharsets.UTF_8)); - } catch (NoSuchAlgorithmException e) { - throw new IOException(e); - } - } - - private String getCPKSha(final AzureBlobFileSystem abfs) throws IOException { - Configuration conf = abfs.getConf(); - String accountName = conf.get(FS_AZURE_ABFS_ACCOUNT_NAME); - String encryptionKey = conf - .get(FS_AZURE_CLIENT_PROVIDED_ENCRYPTION_KEY + "." + accountName); - if (encryptionKey == null || encryptionKey.isEmpty()) { - return ""; - } - return getBase64EncodedString(getSHA256Hash(encryptionKey)); - } - - private String getBase64EncodedString(byte[] bytes) { - return java.util.Base64.getEncoder().encodeToString(bytes); - } - - private Path createFileWithContent(FileSystem fs, String fileName, - byte[] fileContent) throws IOException { - Path testFilePath = new Path(fileName); - try (FSDataOutputStream oStream = fs.create(testFilePath)) { - oStream.write(fileContent); - oStream.flush(); - } - return testFilePath; - } - - private String convertXmsPropertiesToCommaSeparatedString( - final Hashtable properties) - throws CharacterCodingException { - StringBuilder commaSeparatedProperties = new StringBuilder(); - final CharsetEncoder encoder = Charset.forName(XMS_PROPERTIES_ENCODING) - .newEncoder(); - for (Map.Entry propertyEntry : properties.entrySet()) { - String key = propertyEntry.getKey(); - String value = propertyEntry.getValue(); - Boolean canEncodeValue = encoder.canEncode(value); - if (!canEncodeValue) { - throw new CharacterCodingException(); - } - String encodedPropertyValue = Base64 - .encode(encoder.encode(CharBuffer.wrap(value)).array()); - commaSeparatedProperties.append(key).append(AbfsHttpConstants.EQUAL) - .append(encodedPropertyValue); - commaSeparatedProperties.append(AbfsHttpConstants.COMMA); - } - if (commaSeparatedProperties.length() != 0) { - commaSeparatedProperties - .deleteCharAt(commaSeparatedProperties.length() - 1); - } - return commaSeparatedProperties.toString(); - } - - private String getOctalNotation(FsPermission fsPermission) { - Preconditions.checkNotNull(fsPermission, "fsPermission"); - return String - .format(AbfsHttpConstants.PERMISSION_FORMAT, fsPermission.toOctal()); - } - - private byte[] getRandomBytesArray(int length) { - final byte[] b = new byte[length]; - new Random().nextBytes(b); - return b; - } - - private AzureBlobFileSystem getAbfs(boolean withCPK) throws IOException { - return getAbfs(withCPK, "12345678901234567890123456789012"); - } - - private AzureBlobFileSystem getAbfs(boolean withCPK, String cpk) - throws IOException { - Configuration conf = getRawConfiguration(); - if (withCPK) { - conf.set(FS_AZURE_CLIENT_PROVIDED_ENCRYPTION_KEY + "." + getAccountName(), - cpk); - } else { - conf.unset( - FS_AZURE_CLIENT_PROVIDED_ENCRYPTION_KEY + "." + getAccountName()); - } - return (AzureBlobFileSystem) FileSystem.newInstance(conf); - } - - private AzureBlobFileSystem getSameFSWithWrongCPK( - final AzureBlobFileSystem fs) throws IOException { - AbfsConfiguration abfsConf = fs.getAbfsStore().getAbfsConfiguration(); - Configuration conf = abfsConf.getRawConfiguration(); - String accountName = conf.get(FS_AZURE_ABFS_ACCOUNT_NAME); - String cpk = conf - .get(FS_AZURE_CLIENT_PROVIDED_ENCRYPTION_KEY + "." + accountName); - if (cpk == null || cpk.isEmpty()) { - cpk = "01234567890123456789012345678912"; - } - cpk = "different-" + cpk; - String differentCpk = cpk.substring(0, ENCRYPTION_KEY_LEN - 1); - conf.set(FS_AZURE_CLIENT_PROVIDED_ENCRYPTION_KEY + "." + accountName, - differentCpk); - conf.set("fs.defaultFS", - "abfs://" + getFileSystemName() + "@" + accountName); - AzureBlobFileSystem sameFSWithDifferentCPK = - (AzureBlobFileSystem) FileSystem.newInstance(conf); - return sameFSWithDifferentCPK; - } - -} From 15a0ce808f027cb8f78db6d20087032362f0df28 Mon Sep 17 00:00:00 2001 From: sumangala Date: Fri, 13 May 2022 10:22:17 +0530 Subject: [PATCH 28/77] skip for nonHNS account --- .../hadoop/fs/azurebfs/ITestAbfsCustomEncryption.java | 7 ++++++- 1 file changed, 6 insertions(+), 1 deletion(-) diff --git a/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/ITestAbfsCustomEncryption.java b/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/ITestAbfsCustomEncryption.java index a914c432df267..61f1292396e9b 100644 --- a/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/ITestAbfsCustomEncryption.java +++ b/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/ITestAbfsCustomEncryption.java @@ -25,8 +25,8 @@ import java.util.Hashtable; import java.util.Random; -import org.apache.hadoop.fs.permission.FsPermission; import org.assertj.core.api.Assertions; +import org.junit.Assume; import org.junit.Test; import org.junit.runner.RunWith; import org.junit.runners.Parameterized; @@ -46,6 +46,7 @@ import org.apache.hadoop.fs.azurebfs.services.AbfsRestOperation; import org.apache.hadoop.fs.azurebfs.utils.EncryptionType; import org.apache.hadoop.fs.permission.AclEntry; +import org.apache.hadoop.fs.permission.FsPermission; import org.apache.hadoop.test.LambdaTestUtils; import org.apache.hadoop.util.Lists; @@ -56,6 +57,7 @@ import static org.apache.hadoop.fs.azurebfs.constants.HttpHeaderConfigurations.X_MS_REQUEST_SERVER_ENCRYPTED; import static org.apache.hadoop.fs.azurebfs.constants.HttpHeaderConfigurations.X_MS_SERVER_ENCRYPTED; import static org.apache.hadoop.fs.azurebfs.constants.TestConfigurationKeys.ENCRYPTION_KEY_LEN; +import static org.apache.hadoop.fs.azurebfs.constants.TestConfigurationKeys.FS_AZURE_TEST_NAMESPACE_ENABLED_ACCOUNT; import static org.apache.hadoop.fs.azurebfs.contracts.services.AppendRequestParameters.Mode.APPEND_MODE; import static org.apache.hadoop.fs.azurebfs.utils.AclTestHelpers.aclEntry; import static org.apache.hadoop.fs.azurebfs.utils.EncryptionType.ENCRYPTION_CONTEXT; @@ -140,6 +142,9 @@ public static Iterable params() { public ITestAbfsCustomEncryption() throws Exception { super(); + Assume.assumeTrue("Account should be HNS enabled for CPK", + getConfiguration().getBoolean(FS_AZURE_TEST_NAMESPACE_ENABLED_ACCOUNT, + false)); new Random().nextBytes(cpk); cpkSHAEncoded = EncryptionAdapter.getBase64EncodedString( EncryptionAdapter.getSHA256Hash(cpk)); From 2f2947cb6c793eb2830245c1a383011230b8968f Mon Sep 17 00:00:00 2001 From: sumangala Date: Fri, 13 May 2022 10:51:50 +0530 Subject: [PATCH 29/77] override final variable xMsVersion in mock to fix NPE --- .../org/apache/hadoop/fs/azurebfs/services/AbfsClient.java | 4 ++++ .../apache/hadoop/fs/azurebfs/services/TestAbfsClient.java | 4 ++++ 2 files changed, 8 insertions(+) diff --git a/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/services/AbfsClient.java b/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/services/AbfsClient.java index c741017bd69cf..abf9a73dd177f 100644 --- a/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/services/AbfsClient.java +++ b/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/services/AbfsClient.java @@ -1302,6 +1302,10 @@ protected AbfsCounters getAbfsCounters() { return abfsCounters; } + public String getxMsVersion() { + return xMsVersion; + } + public int getNumLeaseThreads() { return abfsConfiguration.getNumLeaseThreads(); } diff --git a/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/services/TestAbfsClient.java b/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/services/TestAbfsClient.java index 154eb804c48af..c1b1ee48f879b 100644 --- a/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/services/TestAbfsClient.java +++ b/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/services/TestAbfsClient.java @@ -316,6 +316,10 @@ public static AbfsClient getMockAbfsClient(AbfsClient baseAbfsClientInstance, client = TestAbfsClient.setAbfsClientField(client, "baseUrl", baseAbfsClientInstance.getBaseUrl()); + // override xMsVersion + client = TestAbfsClient.setAbfsClientField(client, "xMsVersion", + baseAbfsClientInstance.getxMsVersion()); + // override auth provider if (currentAuthType == AuthType.SharedKey) { client = TestAbfsClient.setAbfsClientField(client, "sharedKeyCredentials", From 16973d461d79a1516f8a0683cb4753c490e154d2 Mon Sep 17 00:00:00 2001 From: Pranav Saxena Date: Fri, 8 Jul 2022 10:30:24 +0530 Subject: [PATCH 30/77] merged pranavsaxena-microsoft:HADOOP-17912-backemerge --- .BUILDING.txt.swp | Bin 16384 -> 0 bytes LICENSE-binary | 4 +- dev-support/Jenkinsfile | 4 +- dev-support/bin/yetus-wrapper | 2 +- dev-support/docker/Dockerfile_debian_10 | 1 + .../docker/pkg-resolver/install-maven.sh | 2 +- dev-support/docker/pkg-resolver/packages.json | 1 - .../git_jira_fix_version_check.py | 7 +- dev-support/hadoop-vote.sh | 3 + .../hadoop-client-minicluster/pom.xml | 6 + .../hadoop-client-runtime/pom.xml | 7 + hadoop-client-modules/hadoop-client/pom.xml | 12 + .../hadoop-cos/pom.xml | 2 +- .../examples/RequestLoggerFilter.java | 1 + .../client/AuthenticatedURL.java | 3 + .../client/KerberosAuthenticator.java | 6 + .../server/AuthenticationFilter.java | 4 +- .../authentication/util/CertificateUtil.java | 1 - .../authentication/util/KerberosUtil.java | 2 +- .../authentication/KerberosTestUtils.java | 4 +- .../client/TestAuthenticatedURL.java | 38 + .../client/TestKerberosAuthenticator.java | 82 + .../server/TestAuthenticationFilter.java | 38 + .../TestJWTRedirectAuthenticationHandler.java | 1 - .../TestPseudoAuthenticationHandler.java | 1 - .../util/StringSignerSecretProvider.java | 1 - .../jdiff/Apache_Hadoop_Common_2.10.2.xml | 41055 ++++++++++++++++ .../jdiff/Apache_Hadoop_Common_3.3.3.xml | 39037 +++++++++++++++ hadoop-common-project/hadoop-common/pom.xml | 10 + .../apache/hadoop/conf/ConfigRedactor.java | 4 +- .../org/apache/hadoop/conf/Configuration.java | 54 +- .../org/apache/hadoop/conf/Configured.java | 4 +- .../apache/hadoop/conf/Reconfigurable.java | 6 + .../hadoop/conf/ReconfigurableBase.java | 3 + .../hadoop/conf/ReconfigurationException.java | 10 + .../conf/ReconfigurationTaskStatus.java | 4 +- .../org/apache/hadoop/crypto/CryptoCodec.java | 10 +- .../hadoop/crypto/CryptoInputStream.java | 2 +- .../hadoop/crypto/CryptoOutputStream.java | 2 +- .../hadoop/crypto/CryptoStreamUtils.java | 35 +- .../apache/hadoop/crypto/OpensslCipher.java | 23 +- .../apache/hadoop/crypto/key/KeyProvider.java | 48 +- .../key/KeyProviderCryptoExtension.java | 13 +- .../KeyProviderDelegationTokenExtension.java | 4 +- .../apache/hadoop/crypto/key/KeyShell.java | 4 +- .../kms/LoadBalancingKMSClientProvider.java | 7 +- .../hadoop/crypto/key/kms/ValueQueue.java | 12 +- .../apache/hadoop/fs/AbstractFileSystem.java | 277 +- .../org/apache/hadoop/fs/AvroFSInput.java | 13 +- .../hadoop/fs/BatchedRemoteIterator.java | 3 + .../org/apache/hadoop/fs/BlockLocation.java | 45 + .../hadoop/fs/BufferedFSInputStream.java | 27 +- .../org/apache/hadoop/fs/ByteBufferUtil.java | 6 + .../apache/hadoop/fs/CachingGetSpaceUsed.java | 9 + .../apache/hadoop/fs/ChecksumFileSystem.java | 253 +- .../java/org/apache/hadoop/fs/ChecksumFs.java | 37 +- .../fs/CommonConfigurationKeysPublic.java | 15 +- .../hadoop/fs/CompositeCrcFileChecksum.java | 8 +- .../org/apache/hadoop/fs/ContentSummary.java | 26 +- .../java/org/apache/hadoop/fs/CreateFlag.java | 2 + .../main/java/org/apache/hadoop/fs/DF.java | 10 +- .../hadoop/fs/DelegationTokenRenewer.java | 26 +- .../java/org/apache/hadoop/fs/FSBuilder.java | 45 +- .../apache/hadoop/fs/FSDataInputStream.java | 22 +- .../hadoop/fs/FSDataOutputStreamBuilder.java | 32 + .../org/apache/hadoop/fs/FSInputChecker.java | 10 +- .../org/apache/hadoop/fs/FSLinkResolver.java | 2 +- .../org/apache/hadoop/fs/FSOutputSummer.java | 8 + .../org/apache/hadoop/fs/FileChecksum.java | 25 +- .../org/apache/hadoop/fs/FileContext.java | 115 +- .../apache/hadoop/fs/FileEncryptionInfo.java | 3 + .../java/org/apache/hadoop/fs/FileRange.java | 67 + .../java/org/apache/hadoop/fs/FileStatus.java | 14 + .../java/org/apache/hadoop/fs/FileSystem.java | 170 +- .../hadoop/fs/FileSystemLinkResolver.java | 8 +- .../java/org/apache/hadoop/fs/FileUtil.java | 87 +- .../apache/hadoop/fs/FilterFileSystem.java | 4 +- .../java/org/apache/hadoop/fs/FsShell.java | 2 +- .../java/org/apache/hadoop/fs/FsStatus.java | 23 +- .../org/apache/hadoop/fs/GlobExpander.java | 4 +- .../hadoop/fs/GlobalStorageStatistics.java | 2 + .../org/apache/hadoop/fs/HarFileSystem.java | 8 +- .../java/org/apache/hadoop/fs/HardLink.java | 6 + .../apache/hadoop/fs/HasFileDescriptor.java | 2 +- .../apache/hadoop/fs/LocalDirAllocator.java | 31 +- .../org/apache/hadoop/fs/LocalFileSystem.java | 6 +- .../fs/MD5MD5CRC32CastagnoliFileChecksum.java | 8 +- .../hadoop/fs/MD5MD5CRC32FileChecksum.java | 13 +- .../fs/MD5MD5CRC32GzipFileChecksum.java | 8 +- .../apache/hadoop/fs/MultipartUploader.java | 3 +- .../hadoop/fs/MultipartUploaderBuilder.java | 19 +- .../java/org/apache/hadoop/fs/Options.java | 7 +- .../apache/hadoop/fs/PositionedReadable.java | 41 +- .../java/org/apache/hadoop/fs/QuotaUsage.java | 64 +- .../apache/hadoop/fs/RawLocalFileSystem.java | 117 +- .../java/org/apache/hadoop/fs/Seekable.java | 14 +- .../main/java/org/apache/hadoop/fs/Stat.java | 4 +- .../apache/hadoop/fs/StorageStatistics.java | 5 + .../apache/hadoop/fs/StreamCapabilities.java | 6 + .../main/java/org/apache/hadoop/fs/Trash.java | 43 +- .../org/apache/hadoop/fs/TrashPolicy.java | 16 +- .../apache/hadoop/fs/VectoredReadUtils.java | 292 + .../java/org/apache/hadoop/fs/XAttrCodec.java | 6 +- .../hadoop/fs/audit/AuditConstants.java | 5 + .../hadoop/fs/audit/CommonAuditContext.java | 8 +- .../hadoop/fs/impl/AbstractFSBuilderImpl.java | 2 + .../fs/impl/AbstractMultipartUploader.java | 2 +- .../hadoop/fs/impl/CombinedFileRange.java | 70 + .../apache/hadoop/fs/impl/FileRangeImpl.java | 74 + .../FutureDataInputStreamBuilderImpl.java | 5 + .../hadoop/fs/impl/FutureIOSupport.java | 2 + .../fs/impl/MultipartUploaderBuilderImpl.java | 3 + .../hadoop/fs/permission/AclStatus.java | 4 +- .../apache/hadoop/fs/permission/FsAction.java | 20 +- .../hadoop/fs/permission/FsCreateModes.java | 9 +- .../hadoop/fs/permission/FsPermission.java | 54 +- .../fs/permission/PermissionStatus.java | 39 +- .../hadoop/fs/sftp/SFTPConnectionPool.java | 4 +- .../apache/hadoop/fs/sftp/SFTPFileSystem.java | 6 +- .../org/apache/hadoop/fs/shell/Command.java | 24 +- .../apache/hadoop/fs/shell/CommandFormat.java | 2 +- .../fs/shell/CommandWithDestination.java | 3 + .../org/apache/hadoop/fs/shell/PathData.java | 3 +- .../hadoop/fs/shell/find/BaseExpression.java | 19 +- .../hadoop/fs/shell/find/Expression.java | 15 +- .../org/apache/hadoop/fs/shell/find/Find.java | 2 +- .../hadoop/fs/shell/find/FindOptions.java | 1 + .../apache/hadoop/fs/shell/find/Result.java | 21 +- .../fs/statistics/IOStatisticsSnapshot.java | 8 +- .../fs/statistics/IOStatisticsSupport.java | 1 + .../hadoop/fs/statistics/MeanStatistic.java | 1 + .../fs/statistics/StoreStatisticNames.java | 3 + .../statistics/impl/IOStatisticsBinding.java | 4 + .../apache/hadoop/fs/store/DataBlocks.java | 4 + .../fs/store/audit/AuditingFunctions.java | 2 + .../apache/hadoop/fs/viewfs/ConfigUtil.java | 74 +- .../apache/hadoop/fs/viewfs/Constants.java | 13 +- .../org/apache/hadoop/fs/viewfs/FsGetter.java | 9 + .../apache/hadoop/fs/viewfs/InodeTree.java | 233 +- .../fs/viewfs/MountTableConfigLoader.java | 1 + .../hadoop/fs/viewfs/ViewFileSystem.java | 124 +- .../viewfs/ViewFileSystemOverloadScheme.java | 10 +- .../hadoop/fs/viewfs/ViewFileSystemUtil.java | 5 +- .../org/apache/hadoop/fs/viewfs/ViewFs.java | 162 +- .../hadoop/ha/ActiveStandbyElector.java | 31 +- .../java/org/apache/hadoop/ha/HAAdmin.java | 3 + .../apache/hadoop/ha/HAServiceProtocol.java | 8 +- .../org/apache/hadoop/ha/HAServiceTarget.java | 8 +- .../org/apache/hadoop/ha/HealthMonitor.java | 3 + .../hadoop/ha/ZKFailoverController.java | 2 + .../org/apache/hadoop/http/HtmlQuoting.java | 1 + .../org/apache/hadoop/http/HttpServer2.java | 36 +- .../apache/hadoop/io/AbstractMapWritable.java | 22 +- .../java/org/apache/hadoop/io/ArrayFile.java | 68 +- .../hadoop/io/ArrayPrimitiveWritable.java | 4 +- .../apache/hadoop/io/BinaryComparable.java | 9 + .../org/apache/hadoop/io/BloomMapFile.java | 2 +- .../org/apache/hadoop/io/BooleanWritable.java | 9 +- .../io/BoundedByteArrayOutputStream.java | 14 +- .../org/apache/hadoop/io/ByteBufferPool.java | 5 + .../org/apache/hadoop/io/ByteWritable.java | 10 +- .../org/apache/hadoop/io/BytesWritable.java | 4 + .../apache/hadoop/io/CompressedWritable.java | 13 +- .../org/apache/hadoop/io/DataInputBuffer.java | 23 +- .../apache/hadoop/io/DataOutputBuffer.java | 32 +- .../hadoop/io/ElasticByteBufferPool.java | 4 +- .../org/apache/hadoop/io/EnumSetWritable.java | 15 +- .../org/apache/hadoop/io/FloatWritable.java | 10 +- .../org/apache/hadoop/io/GenericWritable.java | 4 +- .../java/org/apache/hadoop/io/IOUtils.java | 14 +- .../org/apache/hadoop/io/InputBuffer.java | 23 +- .../org/apache/hadoop/io/IntWritable.java | 10 +- .../org/apache/hadoop/io/LongWritable.java | 10 +- .../java/org/apache/hadoop/io/MD5Hash.java | 81 +- .../java/org/apache/hadoop/io/MapFile.java | 230 +- .../apache/hadoop/io/MultipleIOException.java | 11 +- .../org/apache/hadoop/io/NullWritable.java | 5 +- .../org/apache/hadoop/io/ObjectWritable.java | 61 +- .../org/apache/hadoop/io/OutputBuffer.java | 20 +- .../org/apache/hadoop/io/RawComparator.java | 2 +- .../org/apache/hadoop/io/ReadaheadPool.java | 2 +- .../org/apache/hadoop/io/SecureIOUtils.java | 26 +- .../org/apache/hadoop/io/SequenceFile.java | 328 +- .../java/org/apache/hadoop/io/SetFile.java | 75 +- .../org/apache/hadoop/io/ShortWritable.java | 7 +- .../main/java/org/apache/hadoop/io/Text.java | 77 +- .../main/java/org/apache/hadoop/io/UTF8.java | 48 +- .../org/apache/hadoop/io/VIntWritable.java | 7 +- .../org/apache/hadoop/io/VLongWritable.java | 7 +- .../apache/hadoop/io/VersionedWritable.java | 2 +- .../WeakReferencedElasticByteBufferPool.java | 155 + .../java/org/apache/hadoop/io/Writable.java | 4 +- .../apache/hadoop/io/WritableComparator.java | 120 +- .../apache/hadoop/io/WritableFactories.java | 26 +- .../org/apache/hadoop/io/WritableFactory.java | 2 +- .../org/apache/hadoop/io/WritableName.java | 32 +- .../org/apache/hadoop/io/WritableUtils.java | 34 +- .../apache/hadoop/io/compress/BZip2Codec.java | 8 +- .../io/compress/BlockDecompressorStream.java | 4 +- .../apache/hadoop/io/compress/CodecPool.java | 12 +- .../hadoop/io/compress/CompressionCodec.java | 8 +- .../io/compress/CompressionCodecFactory.java | 5 +- .../io/compress/CompressionInputStream.java | 8 +- .../io/compress/CompressionOutputStream.java | 4 +- .../apache/hadoop/io/compress/Compressor.java | 3 + .../hadoop/io/compress/Decompressor.java | 2 +- .../io/compress/DecompressorStream.java | 2 +- .../hadoop/io/compress/DefaultCodec.java | 1 - .../apache/hadoop/io/compress/Lz4Codec.java | 8 +- .../hadoop/io/compress/SnappyCodec.java | 8 +- .../compress/SplittableCompressionCodec.java | 2 + .../hadoop/io/compress/ZStandardCodec.java | 8 +- .../io/compress/bzip2/Bzip2Compressor.java | 1 + .../io/compress/bzip2/Bzip2Decompressor.java | 2 + .../io/compress/bzip2/CBZip2InputStream.java | 8 +- .../io/compress/bzip2/CBZip2OutputStream.java | 10 +- .../io/compress/lz4/Lz4Decompressor.java | 2 +- .../compress/snappy/SnappyDecompressor.java | 2 +- .../io/compress/zlib/ZlibCompressor.java | 1 + .../io/compress/zlib/ZlibDecompressor.java | 2 + .../hadoop/io/compress/zlib/ZlibFactory.java | 2 +- .../io/compress/zstd/ZStandardCompressor.java | 2 + .../compress/zstd/ZStandardDecompressor.java | 1 + .../hadoop/io/erasurecode/CodecUtil.java | 2 + .../io/erasurecode/ErasureCodeNative.java | 2 + .../io/erasurecode/codec/ErasureCodec.java | 1 - .../io/erasurecode/coder/ErasureCoder.java | 1 + .../erasurecode/coder/ErasureCodingStep.java | 5 +- .../io/erasurecode/coder/ErasureDecoder.java | 10 +- .../coder/ErasureDecodingStep.java | 6 +- .../io/erasurecode/coder/ErasureEncoder.java | 2 +- .../coder/ErasureEncodingStep.java | 6 +- .../coder/HHErasureCodingStep.java | 4 +- .../coder/HHXORErasureDecodingStep.java | 4 +- .../coder/HHXORErasureEncodingStep.java | 4 +- .../erasurecode/coder/XORErasureDecoder.java | 2 +- .../io/erasurecode/coder/util/HHUtil.java | 2 + .../io/erasurecode/grouper/BlockGrouper.java | 6 +- .../rawcoder/DecodingValidator.java | 4 +- .../rawcoder/RawErasureDecoder.java | 2 + .../rawcoder/RawErasureEncoder.java | 5 +- .../erasurecode/rawcoder/util/DumpUtil.java | 10 +- .../io/erasurecode/rawcoder/util/GF256.java | 10 +- .../rawcoder/util/GaloisField.java | 38 +- .../io/erasurecode/rawcoder/util/RSUtil.java | 15 + .../hadoop/io/file/tfile/ByteArray.java | 2 +- .../apache/hadoop/io/file/tfile/TFile.java | 95 +- .../apache/hadoop/io/file/tfile/Utils.java | 20 +- .../apache/hadoop/io/nativeio/NativeIO.java | 46 +- .../hadoop/io/retry/AsyncCallHandler.java | 12 +- .../apache/hadoop/io/retry/RetryPolicies.java | 38 +- .../apache/hadoop/io/retry/RetryProxy.java | 5 + .../apache/hadoop/io/retry/RetryUtils.java | 4 +- .../hadoop/io/serializer/Deserializer.java | 7 +- .../io/serializer/DeserializerComparator.java | 2 +- .../JavaSerializationComparator.java | 2 +- .../hadoop/io/serializer/Serialization.java | 7 +- .../io/serializer/SerializationFactory.java | 2 + .../hadoop/io/serializer/Serializer.java | 7 +- .../avro/AvroReflectSerialization.java | 2 +- .../io/serializer/avro/AvroSerialization.java | 6 + .../apache/hadoop/ipc/AlignmentContext.java | 2 +- .../apache/hadoop/ipc/CallQueueManager.java | 6 + .../java/org/apache/hadoop/ipc/Client.java | 28 +- .../org/apache/hadoop/ipc/ClientCache.java | 2 + .../java/org/apache/hadoop/ipc/ClientId.java | 12 +- .../hadoop/ipc/GenericRefreshProtocol.java | 6 +- .../org/apache/hadoop/ipc/ProtobufHelper.java | 4 +- .../apache/hadoop/ipc/ProtobufRpcEngine.java | 8 + .../apache/hadoop/ipc/ProtobufRpcEngine2.java | 6 + .../hadoop/ipc/ProtocolMetaInterface.java | 2 +- .../org/apache/hadoop/ipc/ProtocolProxy.java | 5 +- .../main/java/org/apache/hadoop/ipc/RPC.java | 180 +- .../hadoop/ipc/RefreshCallQueueProtocol.java | 2 +- .../apache/hadoop/ipc/RefreshRegistry.java | 1 + .../apache/hadoop/ipc/RemoteException.java | 5 +- .../org/apache/hadoop/ipc/RetryCache.java | 22 +- .../org/apache/hadoop/ipc/RpcClientUtil.java | 6 +- .../java/org/apache/hadoop/ipc/RpcEngine.java | 37 +- .../org/apache/hadoop/ipc/RpcScheduler.java | 9 +- .../apache/hadoop/ipc/RpcServerException.java | 4 +- .../java/org/apache/hadoop/ipc/Server.java | 138 +- .../apache/hadoop/ipc/VersionedProtocol.java | 1 + .../apache/hadoop/ipc/WritableRpcEngine.java | 50 +- .../DecayRpcSchedulerDetailedMetrics.java | 7 +- .../apache/hadoop/ipc/metrics/RpcMetrics.java | 10 + .../org/apache/hadoop/jmx/JMXJsonServlet.java | 11 +- .../java/org/apache/hadoop/log/LogLevel.java | 2 + .../hadoop/log/LogThrottlingHelper.java | 8 +- .../apache/hadoop/metrics2/MetricsSystem.java | 6 +- .../hadoop/metrics2/MetricsSystemMXBean.java | 10 +- .../metrics2/lib/MutableMetricsFactory.java | 4 +- .../hadoop/metrics2/lib/MutableRates.java | 5 +- .../lib/MutableRatesWithAggregation.java | 4 +- .../metrics2/lib/MutableRollingAverages.java | 4 +- .../hadoop/metrics2/lib/MutableStat.java | 2 +- .../apache/hadoop/metrics2/package-info.java | 10 +- .../metrics2/sink/PrometheusMetricsSink.java | 4 + .../sink/ganglia/AbstractGangliaSink.java | 5 +- .../metrics2/sink/ganglia/GangliaSink30.java | 2 +- .../metrics2/sink/ganglia/GangliaSink31.java | 2 +- .../apache/hadoop/metrics2/util/MBeans.java | 8 +- .../hadoop/metrics2/util/SampleQuantiles.java | 2 +- .../net/AbstractDNSToSwitchMapping.java | 2 +- .../main/java/org/apache/hadoop/net/DNS.java | 10 +- .../apache/hadoop/net/DNSToSwitchMapping.java | 2 + .../apache/hadoop/net/DomainNameResolver.java | 13 +- .../java/org/apache/hadoop/net/InnerNode.java | 5 +- .../org/apache/hadoop/net/InnerNodeImpl.java | 15 +- .../java/org/apache/hadoop/net/NetUtils.java | 60 +- .../apache/hadoop/net/NetworkTopology.java | 26 +- .../apache/hadoop/net/ScriptBasedMapping.java | 8 +- .../net/ScriptBasedMappingWithDependency.java | 5 +- .../apache/hadoop/net/SocketInputStream.java | 8 +- .../apache/hadoop/net/SocketOutputStream.java | 13 +- .../apache/hadoop/net/unix/DomainSocket.java | 13 +- .../security/CompositeGroupsMapping.java | 2 +- .../apache/hadoop/security/Credentials.java | 33 +- .../security/GroupMappingServiceProvider.java | 8 +- .../org/apache/hadoop/security/Groups.java | 4 +- .../hadoop/security/HadoopKerberosName.java | 4 +- .../security/IdMappingServiceProvider.java | 2 - .../org/apache/hadoop/security/KDiag.java | 5 +- .../apache/hadoop/security/KerberosInfo.java | 5 +- .../apache/hadoop/security/NetgroupCache.java | 2 +- .../hadoop/security/NullGroupsMapping.java | 2 +- .../apache/hadoop/security/ProviderUtils.java | 1 + .../security/RefreshUserMappingsProtocol.java | 4 +- .../hadoop/security/SaslInputStream.java | 2 +- .../security/SaslPropertiesResolver.java | 2 +- .../apache/hadoop/security/SaslRpcClient.java | 14 +- .../apache/hadoop/security/SaslRpcServer.java | 25 +- .../apache/hadoop/security/SecurityUtil.java | 14 +- .../hadoop/security/ShellBasedIdMapping.java | 9 +- .../security/ShellBasedUnixGroupsMapping.java | 2 +- .../ShellBasedUnixGroupsNetgroupMapping.java | 2 + .../hadoop/security/UserGroupInformation.java | 46 +- .../security/alias/CredentialProvider.java | 13 +- .../security/alias/CredentialShell.java | 6 +- .../security/authorize/AccessControlList.java | 5 +- .../authorize/ImpersonationProvider.java | 4 +- .../security/authorize/ProxyServers.java | 2 +- .../hadoop/security/authorize/ProxyUsers.java | 12 +- .../RefreshAuthorizationPolicyProtocol.java | 2 +- .../security/http/CrossOriginFilter.java | 1 - .../http/RestCsrfPreventionFilter.java | 4 +- .../ssl/ReloadingX509KeystoreManager.java | 4 +- .../ssl/ReloadingX509TrustManager.java | 1 - .../security/token/DelegationTokenIssuer.java | 13 + .../hadoop/security/token/DtFetcher.java | 23 +- .../security/token/DtFileOperations.java | 18 +- .../hadoop/security/token/DtUtilShell.java | 4 +- .../apache/hadoop/security/token/Token.java | 18 +- .../hadoop/security/token/TokenInfo.java | 6 +- .../hadoop/security/token/TokenRenewer.java | 40 +- .../AbstractDelegationTokenSecretManager.java | 110 +- .../web/DelegationTokenAuthenticatedURL.java | 6 + .../DelegationTokenAuthenticationFilter.java | 1 + .../DelegationTokenAuthenticationHandler.java | 2 +- .../web/DelegationTokenAuthenticator.java | 8 + .../web/HttpUserGroupInformation.java | 2 - .../hadoop/service/AbstractService.java | 2 +- .../hadoop/service/CompositeService.java | 2 +- .../hadoop/service/ServiceStateModel.java | 3 + .../launcher/AbstractLaunchableService.java | 2 + .../service/launcher/ServiceLauncher.java | 8 +- .../hadoop/service/launcher/package-info.java | 27 +- .../org/apache/hadoop/tools/CommandShell.java | 4 +- .../apache/hadoop/tools/GetGroupsBase.java | 6 +- .../hadoop/tools/GetUserMappingsProtocol.java | 2 +- .../org/apache/hadoop/tools/TableListing.java | 11 +- .../apache/hadoop/util/AsyncDiskService.java | 7 +- .../BlockingThreadPoolExecutorService.java | 1 + .../org/apache/hadoop/util/CrcComposer.java | 27 + .../java/org/apache/hadoop/util/CrcUtil.java | 36 + .../java/org/apache/hadoop/util/Daemon.java | 11 +- .../org/apache/hadoop/util/DataChecksum.java | 55 +- .../apache/hadoop/util/DirectBufferPool.java | 3 + .../org/apache/hadoop/util/DiskChecker.java | 16 +- .../hadoop/util/DiskValidatorFactory.java | 2 + .../apache/hadoop/util/FileBasedIPList.java | 2 +- .../org/apache/hadoop/util/FindClass.java | 1 - .../org/apache/hadoop/util/GcTimeMonitor.java | 35 +- .../hadoop/util/GenericOptionsParser.java | 38 +- .../org/apache/hadoop/util/GenericsUtil.java | 4 + .../apache/hadoop/util/HostsFileReader.java | 2 +- .../java/org/apache/hadoop/util/IPList.java | 2 +- .../org/apache/hadoop/util/IdGenerator.java | 5 +- .../apache/hadoop/util/IdentityHashStore.java | 11 + .../apache/hadoop/util/IndexedSortable.java | 7 + .../org/apache/hadoop/util/IndexedSorter.java | 8 + .../apache/hadoop/util/InstrumentedLock.java | 1 + .../hadoop/util/IntrusiveCollection.java | 23 + .../apache/hadoop/util/JsonSerialization.java | 9 +- .../apache/hadoop/util/JvmPauseMonitor.java | 3 + .../apache/hadoop/util/LightWeightCache.java | 7 +- .../apache/hadoop/util/LightWeightGSet.java | 19 +- .../hadoop/util/LightWeightResizableGSet.java | 2 + .../org/apache/hadoop/util/LineReader.java | 6 +- .../java/org/apache/hadoop/util/Lists.java | 30 +- .../org/apache/hadoop/util/MachineList.java | 6 +- .../apache/hadoop/util/NativeCodeLoader.java | 8 +- .../hadoop/util/NativeLibraryChecker.java | 3 +- .../apache/hadoop/util/OperationDuration.java | 4 +- .../java/org/apache/hadoop/util/Options.java | 2 +- .../apache/hadoop/util/PrintJarMainClass.java | 2 +- .../org/apache/hadoop/util/PriorityQueue.java | 36 +- .../org/apache/hadoop/util/ProgramDriver.java | 20 +- .../java/org/apache/hadoop/util/Progress.java | 49 +- .../org/apache/hadoop/util/ProtoUtil.java | 4 + .../org/apache/hadoop/util/QuickSort.java | 3 + .../apache/hadoop/util/ReflectionUtils.java | 16 +- .../java/org/apache/hadoop/util/RunJar.java | 7 +- .../apache/hadoop/util/SequentialNumber.java | 17 +- .../org/apache/hadoop/util/ServletUtil.java | 16 +- .../java/org/apache/hadoop/util/Sets.java | 64 +- .../java/org/apache/hadoop/util/Shell.java | 98 +- .../hadoop/util/ShutdownHookManager.java | 2 +- .../hadoop/util/ShutdownThreadsHelper.java | 8 +- .../org/apache/hadoop/util/StopWatch.java | 3 + .../apache/hadoop/util/StringInterner.java | 3 + .../org/apache/hadoop/util/StringUtils.java | 60 +- .../java/org/apache/hadoop/util/Time.java | 2 + .../java/org/apache/hadoop/util/Tool.java | 6 +- .../org/apache/hadoop/util/ToolRunner.java | 10 +- .../java/org/apache/hadoop/util/XMLUtils.java | 6 +- .../java/org/apache/hadoop/util/ZKUtil.java | 1 + .../org/apache/hadoop/util/bloom/Key.java | 4 +- .../hadoop/util/concurrent/AsyncGet.java | 8 +- .../hadoop/util/curator/ZKCuratorManager.java | 13 +- .../CloseableTaskPoolSubmitter.java | 2 +- .../functional/CommonCallableSupplier.java | 5 + .../util/functional/RemoteIterators.java | 20 +- .../hadoop/util/functional/package-info.java | 6 +- .../apache/hadoop/util/hash/JenkinsHash.java | 2 +- .../src/site/markdown/Metrics.md | 2 + .../markdown/filesystem/fsdatainputstream.md | 39 + .../filesystem/fsdatainputstreambuilder.md | 2 +- .../filesystem/fsdataoutputstreambuilder.md | 59 +- .../release/2.10.2/CHANGELOG.2.10.2.md | 277 + .../release/2.10.2/RELEASENOTES.2.10.2.md | 102 + .../markdown/release/3.3.3/CHANGELOG.3.3.3.md | 65 + .../release/3.3.3/RELEASENOTES.3.3.3.md | 48 + .../hadoop/cli/util/CommandExecutor.java | 1 - .../conf/TestCommonConfigurationFields.java | 10 +- .../conf/TestConfigurationDeprecation.java | 1 - .../crypto/TestCryptoStreamsForLocalFS.java | 1 - .../hadoop/crypto/key/TestValueQueue.java | 9 +- .../TestLoadBalancingKMSClientProvider.java | 35 +- .../crypto/random/TestOsSecureRandom.java | 2 +- .../apache/hadoop/fs/TestCommandFormat.java | 8 +- .../hadoop/fs/TestHarFileSystemBasics.java | 2 +- .../org/apache/hadoop/fs/TestListFiles.java | 2 +- .../java/org/apache/hadoop/fs/TestTrash.java | 2 +- .../hadoop/fs/TestVectoredReadUtils.java | 371 + .../contract/AbstractContractCreateTest.java | 19 +- .../contract/AbstractContractDeleteTest.java | 2 +- .../AbstractContractVectoredReadTest.java | 406 + .../hadoop/fs/contract/ContractTestUtils.java | 98 +- .../hadoop/fs/contract/ftp/FTPContract.java | 1 - .../TestLocalFSContractVectoredRead.java | 86 + ...awLocalContractUnderlyingFileBehavior.java | 2 +- .../TestRawLocalContractVectoredRead.java | 35 + .../TestRawlocalContractPathHandle.java | 1 - .../apache/hadoop/fs/shell/TestPathData.java | 1 - .../hadoop/fs/shell/TestTextCommand.java | 3 - .../fs/viewfs/TestNestedMountPoint.java | 365 + ...ileSystemWithAuthorityLocalFileSystem.java | 1 - .../hadoop/fs/viewfs/TestViewFsConfig.java | 1 + .../fs/viewfs/ViewFileSystemBaseTest.java | 142 + .../fs/viewfs/ViewFileSystemTestSetup.java | 1 - .../hadoop/http/TestHttpServerLifecycle.java | 2 - .../org/apache/hadoop/io/TestIOUtils.java | 2 +- ...reWeakReferencedElasticByteBufferPool.java | 97 + ...stWeakReferencedElasticByteBufferPool.java | 232 + .../apache/hadoop/io/compress/TestCodec.java | 1 - .../hadoop/io/compress/TestCodecPool.java | 4 +- .../TestBzip2CompressorDecompressor.java | 4 - .../zlib/TestZlibCompressorDecompressor.java | 1 - .../hadoop/io/file/tfile/TestCompression.java | 3 - .../apache/hadoop/ipc/MiniRPCBenchmark.java | 1 - .../java/org/apache/hadoop/ipc/TestRPC.java | 58 + .../java/org/apache/hadoop/net/TestDNS.java | 2 - .../apache/hadoop/net/TestTableMapping.java | 2 +- .../hadoop/security/TestKDiagNoKDC.java | 11 +- .../hadoop/security/TestUGIWithMiniKdc.java | 7 +- .../security/http/TestCrossOriginFilter.java | 3 - .../service/launcher/TestServiceLauncher.java | 2 - .../testservices/NoArgsAllowedService.java | 1 - .../apache/hadoop/test/GenericTestUtils.java | 6 +- .../org/apache/hadoop/test/MoreAsserts.java | 49 +- .../hadoop/test/MultithreadedTestUtil.java | 4 +- .../test/TestTimedOutTestsListener.java | 1 - .../apache/hadoop/tools/TestCommandShell.java | 1 - .../util/TestDiskCheckerWithDiskIo.java | 3 - .../org/apache/hadoop/util/TestShell.java | 4 +- .../crypto/key/kms/server/KMSAudit.java | 6 +- .../hadoop/crypto/key/kms/server/TestKMS.java | 12 +- .../crypto/key/kms/server/TestKMSAudit.java | 14 +- .../nfs/nfs3/response/FSINFO3Response.java | 1 - .../nfs/nfs3/response/FSSTAT3Response.java | 1 - .../nfs/nfs3/response/LINK3Response.java | 2 - .../nfs/nfs3/response/READDIR3Response.java | 1 - .../nfs3/response/READDIRPLUS3Response.java | 2 - .../nfs/nfs3/response/REMOVE3Response.java | 2 - .../nfs/nfs3/response/WRITE3Response.java | 2 - .../hadoop/oncrpc/SimpleTcpClientHandler.java | 3 - .../oncrpc/security/SysSecurityHandler.java | 1 - .../client/binding/RegistryPathUtils.java | 1 - .../client/binding/RegistryUtils.java | 1 - .../impl/zk/RegistryOperationsService.java | 1 - .../client/impl/zk/RegistrySecurity.java | 1 - hadoop-common-project/pom.xml | 1 - .../dev-support/findbugsExcludeFile.xml | 13 + .../hadoop-hdfs-client/pom.xml | 10 +- .../hdfs/client/HdfsClientConfigKeys.java | 8 + .../hdfs/server/protocol/OutlierMetrics.java | 90 + .../hdfs/server/protocol/SlowPeerReports.java | 8 +- .../hadoop/hdfs/web/WebHdfsFileSystem.java | 2 - ...fRefreshTokenBasedAccessTokenProvider.java | 66 +- .../CredentialBasedAccessTokenProvider.java | 69 +- .../hdfs/web/oauth2/OAuth2Constants.java | 2 +- .../TestConfiguredFailoverProxyProvider.java | 1 - .../fs/http/client/HttpFSFileSystem.java | 1 - .../TestHttpFSFileSystemLocalFileSystem.java | 2 - .../server/TestHttpFSAccessControlled.java | 1 - .../apache/hadoop/lib/server/TestServer.java | 1 - .../src/main/native/fuse-dfs/fuse_dfs.h | 1 - .../src/main/native/fuse-dfs/fuse_trash.c | 1 - .../main/native/libhdfs-tests/CMakeLists.txt | 2 + .../src/main/native/libhdfs/CMakeLists.txt | 2 + .../src/main/native/libhdfs/jni_helper.c | 2 +- .../native/libhdfspp/CMakeLists-gtest.txt.in | 33 - .../src/main/native/libhdfspp/CMakeLists.txt | 37 +- .../libhdfspp/lib/common/configuration.cc | 1 - .../lib/common/configuration_loader.cc | 1 - .../native/libhdfspp/lib/proto/CMakeLists.txt | 4 +- .../libhdfspp/lib/x-platform/CMakeLists.txt | 5 +- .../lib/x-platform/c-api/core/dirent.h | 68 + .../libhdfspp/lib/x-platform/c-api/dirent.cc | 92 + .../libhdfspp/lib/x-platform/c-api/dirent.h | 39 + .../lib/x-platform/c-api/extern/dirent.h} | 19 +- .../native/libhdfspp/lib/x-platform/dirent.cc | 38 + .../native/libhdfspp/lib/x-platform/dirent.h | 77 + .../native/libhdfspp/lib/x-platform/types.h | 4 +- .../libhdfspp/tests/tools/hdfs-chmod-mock.cc | 9 + .../libhdfspp/tests/tools/hdfs-tool-tests.h | 13 + .../libhdfspp/tests/x-platform/CMakeLists.txt | 12 + .../tests/x-platform/c-api/dirent_test.cc | 100 + .../tests/x-platform/c-api/dirent_test.h | 37 + .../libhdfspp/tests/x-platform/dirent_test.cc | 151 + .../libhdfspp/tests/x-platform/dirent_test.h | 78 + .../libhdfspp/tools/hdfs-chmod/hdfs-chmod.cc | 15 +- .../hadoop/hdfs/nfs/nfs3/Nfs3HttpServer.java | 1 - .../hadoop/hdfs/nfs/nfs3/TestNfs3Utils.java | 2 - .../hdfs/rbfbalance/RouterFedBalance.java | 4 +- .../federation/metrics/FederationMBean.java | 2 +- .../FederationRPCPerformanceMonitor.java | 2 +- .../server/federation/metrics/RBFMetrics.java | 2 +- .../resolver/MountTableResolver.java | 2 +- .../federation/router/ConnectionContext.java | 4 +- .../federation/router/ConnectionManager.java | 14 +- .../federation/router/FederationUtil.java | 2 +- .../router/MountTableRefresherThread.java | 4 +- .../federation/router/RBFConfigKeys.java | 2 +- .../federation/router/RemoteMethod.java | 2 +- .../federation/router/RouterAdminServer.java | 2 +- .../router/RouterClientProtocol.java | 8 +- .../router/RouterQuotaUpdateService.java | 2 +- .../federation/router/RouterRpcClient.java | 8 +- .../federation/router/RouterRpcMonitor.java | 2 +- .../federation/router/RouterRpcServer.java | 8 +- .../router/RouterWebHdfsMethods.java | 2 +- .../federation/store/CachedRecordStore.java | 6 +- .../federation/store/StateStoreService.java | 2 +- .../store/driver/StateStoreDriver.java | 2 +- .../GetRouterRegistrationsRequest.java | 2 +- .../pb/FederationProtocolPBTranslator.java | 2 +- .../hdfs/tools/federation/RouterAdmin.java | 6 +- .../federation/MiniRouterDFSCluster.java | 2 +- .../hdfs/server/federation/MockResolver.java | 2 +- .../resolver/TestNamenodeResolver.java | 2 +- .../server/federation/router/TestRouter.java | 2 +- .../federation/router/TestRouterAdminCLI.java | 4 +- .../router/TestRouterFaultTolerant.java | 2 +- .../TestRouterMountTableCacheRefresh.java | 6 +- .../router/TestRouterQuotaManager.java | 14 +- .../router/TestRouterRPCClientRetries.java | 2 +- ...MultipleDestinationMountTableResolver.java | 2 +- .../federation/router/TestRouterRpc.java | 43 + .../router/TestRouterRpcMultiDestination.java | 2 +- .../TestRouterRpcStoragePolicySatisfier.java | 1 - .../federation/router/TestRouterTrash.java | 38 + .../router/TestRouterUserMappings.java | 5 +- ...estZKDelegationTokenSecretManagerImpl.java | 4 +- .../store/TestStateStoreMembershipState.java | 2 +- .../driver/TestStateStoreDriverBase.java | 4 +- .../jdiff/Apache_Hadoop_HDFS_2.10.2.xml | 312 + .../jdiff/Apache_Hadoop_HDFS_3.3.3.xml | 835 + .../org/apache/hadoop/hdfs/DFSConfigKeys.java | 4 + .../java/org/apache/hadoop/hdfs/DFSUtil.java | 6 +- .../hadoop/hdfs/protocol/SnapshotInfo.java | 1 - .../hdfs/protocol/datatransfer/Receiver.java | 1 - .../hadoop/hdfs/protocolPB/PBHelper.java | 25 +- .../hdfs/qjournal/server/JNStorage.java | 4 - .../qjournal/server/JournalNodeRpcServer.java | 23 +- .../qjournal/server/JournalNodeSyncer.java | 6 +- .../hdfs/server/balancer/Dispatcher.java | 3 + .../server/balancer/NameNodeConnector.java | 14 +- .../server/blockmanagement/BlockManager.java | 18 +- .../DatanodeAdminBackoffMonitor.java | 16 +- .../DatanodeAdminDefaultMonitor.java | 17 +- .../DatanodeAdminMonitorBase.java | 12 + .../DatanodeAdminMonitorInterface.java | 1 + .../blockmanagement/DatanodeManager.java | 25 +- .../SlowPeerDisabledTracker.java | 7 +- .../blockmanagement/SlowPeerJsonReport.java | 84 + .../SlowPeerLatencyWithReportingNode.java | 124 + .../blockmanagement/SlowPeerTracker.java | 151 +- .../common/IncorrectVersionException.java | 1 - .../impl/TextFileRegionAliasMap.java | 1 - .../hdfs/server/datanode/BPOfferService.java | 5 +- .../hdfs/server/datanode/BPServiceActor.java | 16 +- .../server/datanode/BlockPoolManager.java | 4 +- .../hdfs/server/datanode/BlockReceiver.java | 11 + .../hdfs/server/datanode/DataXceiver.java | 3 + .../datanode/LocalReplicaInPipeline.java | 4 + .../server/datanode/ReplicaInPipeline.java | 5 + .../datanode/checker/AbstractFuture.java | 1 - .../impl/FsDatasetAsyncDiskService.java | 26 +- .../fsdataset/impl/FsDatasetImpl.java | 68 +- .../datanode/metrics/DataNodePeerMetrics.java | 9 +- .../datanode/metrics/OutlierDetector.java | 35 +- .../datanode/web/DatanodeHttpServer.java | 60 +- ...RestrictingAuthorizationFilterHandler.java | 4 +- .../datanode/web/SimpleHttpProxyHandler.java | 5 +- .../server/datanode/web/URLDispatcher.java | 2 +- .../datanode/web/webhdfs/WebHdfsHandler.java | 10 +- .../server/namenode/AclEntryStatusFormat.java | 2 - .../hdfs/server/namenode/FSNamesystem.java | 4 +- .../hdfs/server/namenode/JournalSet.java | 4 +- .../hadoop/hdfs/server/namenode/NameNode.java | 15 +- .../hdfs/server/namenode/SerialNumberMap.java | 1 - .../namenode/ha/RemoteNameNodeInfo.java | 1 - .../startupprogress/package-info.java | 3 +- .../sps/metrics/ExternalSPSBeanMetrics.java | 3 +- .../apache/hadoop/hdfs/tools/DFSAdmin.java | 80 +- .../org/apache/hadoop/hdfs/tools/DFSck.java | 13 +- .../offlineImageViewer/FSImageHandler.java | 6 +- .../tools/snapshot/LsSnapshottableDir.java | 1 - .../src/main/proto/DatanodeProtocol.proto | 3 + .../src/main/resources/hdfs-default.xml | 26 + .../org/apache/hadoop/TestGenericRefresh.java | 2 - .../hadoop/fs/TestResolveHdfsSymlink.java | 1 - ...TestSWebHdfsFileContextMainOperations.java | 1 - .../hadoop/fs/TestSymlinkHdfsFileSystem.java | 1 - .../apache/hadoop/hdfs/MiniDFSCluster.java | 4 +- .../apache/hadoop/hdfs/TestAbandonBlock.java | 4 +- ...TestClientProtocolForPipelineRecovery.java | 2 +- .../org/apache/hadoop/hdfs/TestDFSShell.java | 2 +- .../org/apache/hadoop/hdfs/TestDFSUtil.java | 6 +- .../apache/hadoop/hdfs/TestDecommission.java | 282 + .../apache/hadoop/hdfs/TestFileAppend3.java | 1 - .../apache/hadoop/hdfs/TestFileCreation.java | 4 +- .../hadoop/hdfs/TestHDFSPolicyProvider.java | 2 - .../org/apache/hadoop/hdfs/TestModTime.java | 2 - .../hdfs/TestRollingUpgradeDowngrade.java | 1 - .../org/apache/hadoop/hdfs/TestSeekBug.java | 1 - .../org/apache/hadoop/hdfs/TestSetTimes.java | 1 - .../hadoop/hdfs/TestSlowDatanodeReport.java | 31 +- .../hadoop/hdfs/TestSnapshotCommands.java | 2 +- .../hdfs/net/TestDFSNetworkTopology.java | 14 +- .../hadoop/hdfs/protocolPB/TestPBHelper.java | 9 +- .../qjournal/client/TestQJMWithFaults.java | 4 +- .../hdfs/qjournal/server/TestJournalNode.java | 26 + .../qjournal/server/TestJournalNodeSync.java | 4 + .../balancer/TestBalancerWithHANameNodes.java | 8 +- .../blockmanagement/TestDatanodeManager.java | 25 +- ...TestReplicationPolicyExcludeSlowNodes.java | 20 +- .../blockmanagement/TestSlowPeerTracker.java | 117 +- .../server/datanode/SimulatedFSDataset.java | 5 +- .../datanode/TestBpServiceActorScheduler.java | 13 + .../datanode/TestDataNodeInitStorage.java | 4 - .../server/datanode/TestDatanodeRegister.java | 4 +- .../server/datanode/TestRefreshNamenodes.java | 5 +- .../extdataset/ExternalReplicaInPipeline.java | 4 + .../TestCacheByPmemMappableBlockLoader.java | 3 + .../fsdataset/impl/TestDatanodeRestart.java | 5 - .../fsdataset/impl/TestPmemCacheRecovery.java | 3 + .../fsdataset/impl/TestSpaceReservation.java | 50 + .../fsdataset/impl/TestWriteToReplica.java | 6 +- ...estDataNodeOutlierDetectionViaMetrics.java | 3 +- ...RestrictingAuthorizationFilterHandler.java | 12 +- .../hdfs/server/namenode/FSImageTestUtil.java | 5 +- .../namenode/TestBlockUnderConstruction.java | 3 - .../TestCommitBlockWithInvalidGenStamp.java | 2 - .../server/namenode/TestFileContextXAttr.java | 2 - .../hdfs/server/namenode/TestFileLimit.java | 1 - .../hadoop/hdfs/server/namenode/TestFsck.java | 14 +- .../TestFsckWithMultipleNameNodes.java | 1 - .../namenode/TestNameNodeReconfigure.java | 54 + .../server/namenode/TestNameNodeRecovery.java | 8 +- .../server/namenode/TestSecondaryWebUi.java | 2 - .../server/namenode/TestSecureNameNode.java | 1 - .../TestSecureNameNodeWithExternalKdc.java | 1 - .../ha/TestConsistentReadsObserver.java | 3 + .../snapshot/TestXAttrWithSnapshot.java | 1 - .../TestStartupProgressMetrics.java | 2 - .../TestExternalStoragePolicySatisfier.java | 2 +- .../shortcircuit/TestShortCircuitCache.java | 1 - .../hadoop/hdfs/tools/TestDFSAdmin.java | 73 +- .../hadoop/hdfs/tools/TestDFSAdminWithHA.java | 200 +- .../TestOfflineImageViewerForAcl.java | 4 - .../security/TestRefreshUserMappings.java | 5 +- .../org/apache/hadoop/tools/TestJMXGet.java | 4 - .../src/test/resources/testHDFSConf.xml | 8 +- .../Apache_Hadoop_MapReduce_Common_3.3.3.xml | 113 + .../Apache_Hadoop_MapReduce_Core_3.3.3.xml | 28087 +++++++++++ ...pache_Hadoop_MapReduce_JobClient_3.3.3.xml | 16 + .../mapred/TaskAttemptListenerImpl.java | 15 +- .../org/apache/hadoop/mapred/YarnChild.java | 2 +- .../v2/app/launcher/ContainerLauncher.java | 1 - .../v2/app/rm/RMContainerRequestor.java | 10 +- .../mapreduce/v2/app/webapp/AttemptsPage.java | 1 - .../mapred/TestTaskAttemptListenerImpl.java | 57 +- .../hadoop/mapreduce/v2/app/TestFail.java | 2 - .../mapreduce/v2/app/TestMRAppMaster.java | 2 +- .../mapred/LocalDistributedCacheManager.java | 1 - .../v2/jobhistory/JHAdminConfig.java | 1 - .../TestLocalDistributedCacheManager.java | 204 +- .../mapred/TestLocalModeWithNewApis.java | 1 - .../hadoop/mapreduce/v2/util/TestMRApps.java | 1 - .../apache/hadoop/mapred/JobACLsManager.java | 1 - .../org/apache/hadoop/mapred/JobClient.java | 1 - .../hadoop/mapred/TaskUmbilicalProtocol.java | 2 - .../java/org/apache/hadoop/mapreduce/Job.java | 1 - .../apache/hadoop/mapreduce/MRJobConfig.java | 7 + .../org/apache/hadoop/mapreduce/Mapper.java | 1 - .../mapreduce/jobhistory/AMStartedEvent.java | 1 - .../jobhistory/TaskAttemptStartedEvent.java | 1 - .../mapreduce/lib/db/DBRecordReader.java | 11 - .../lib/db/DataDrivenDBInputFormat.java | 7 - .../lib/db/DataDrivenDBRecordReader.java | 21 +- .../hadoop/mapreduce/lib/db/DateSplitter.java | 2 - .../mapreduce/lib/db/MySQLDBRecordReader.java | 1 - .../lib/db/MySQLDataDrivenDBRecordReader.java | 1 - .../lib/db/OracleDataDrivenDBInputFormat.java | 17 - .../db/OracleDataDrivenDBRecordReader.java | 2 - .../hadoop/mapreduce/lib/db/TextSplitter.java | 1 - .../mapreduce/lib/output/MultipleOutputs.java | 2 - .../manifest/files/AbstractManifestData.java | 1 - .../lib/partition/TotalOrderPartitioner.java | 2 - .../mapreduce/task/reduce/MergeManager.java | 11 - .../hadoop/mapreduce/util/ProcessTree.java | 1 - .../src/main/resources/mapred-default.xml | 9 +- .../lib/jobcontrol/TestJobControl.java | 1 - .../task/reduce/TestEventFetcher.java | 1 - .../mapreduce/v2/hs/HistoryClientService.java | 1 - .../mapreduce/v2/hs/webapp/HsNavBlock.java | 1 - .../io/TestSequenceFileMergeProgress.java | 5 - .../hadoop/mapred/EmptyInputFormat.java | 2 - .../apache/hadoop/mapred/TestLazyOutput.java | 2 - .../hadoop/mapred/TestMiniMRClasspath.java | 2 - .../hadoop/mapred/ThreadedMapBenchmark.java | 1 - .../org/apache/hadoop/mapreduce/FailJob.java | 2 - .../mapreduce/TestMapReduceLazyOutput.java | 1 - .../lib/input/TestMultipleInputs.java | 1 - .../mapreduce/lib/join/TestJoinDatamerge.java | 3 +- .../partition/TestTotalOrderPartitioner.java | 1 - .../mapreduce/security/TestJHSSecurity.java | 20 +- .../token/delegation/TestDelegationToken.java | 1 - .../mapreduce/v2/TestNonExistentJob.java | 1 - .../src/test/java/testjar/ClassWordCount.java | 11 - .../NativeMapOutputCollectorDelegator.java | 1 - .../nativetask/buffer/TestBufferPushPull.java | 1 - .../nativetask/kvtest/TestInputFile.java | 1 - .../examples/terasort/TeraChecksum.java | 1 - .../hadoop/examples/terasort/TeraGen.java | 1 - .../examples/terasort/TeraValidate.java | 1 - hadoop-project-dist/pom.xml | 2 +- hadoop-project/pom.xml | 58 +- hadoop-tools/hadoop-aws/pom.xml | 33 + .../org/apache/hadoop/fs/s3a/Constants.java | 95 + .../hadoop/fs/s3a/S3ABlockOutputStream.java | 59 +- .../apache/hadoop/fs/s3a/S3AFileSystem.java | 432 +- .../apache/hadoop/fs/s3a/S3AInputStream.java | 391 +- .../hadoop/fs/s3a/S3AInstrumentation.java | 18 +- .../hadoop/fs/s3a/S3AReadOpContext.java | 20 +- .../org/apache/hadoop/fs/s3a/S3AUtils.java | 14 +- .../org/apache/hadoop/fs/s3a/Statistic.java | 13 + .../hadoop/fs/s3a/VectoredIOContext.java | 78 + .../hadoop/fs/s3a/WriteOperationHelper.java | 88 +- .../apache/hadoop/fs/s3a/WriteOperations.java | 35 +- .../hadoop/fs/s3a/api/RequestFactory.java | 17 +- .../fs/s3a/commit/AbstractS3ACommitter.java | 604 +- .../hadoop/fs/s3a/commit/CommitConstants.java | 34 +- .../s3a/commit/CommitterStatisticNames.java | 211 + .../commit/InternalCommitterConstants.java | 6 + .../fs/s3a/commit/MagicCommitIntegration.java | 22 +- .../apache/hadoop/fs/s3a/commit/Tasks.java | 423 - .../fs/s3a/commit/files/PendingSet.java | 44 +- .../commit/files/PersistentCommitData.java | 115 +- .../s3a/commit/files/SinglePendingCommit.java | 36 +- .../fs/s3a/commit/files/SuccessData.java | 119 +- .../fs/s3a/commit/files/package-info.java | 7 +- .../s3a/commit/impl/AuditContextUpdater.java | 90 + .../fs/s3a/commit/impl/CommitContext.java | 401 + .../commit/{ => impl}/CommitOperations.java | 247 +- .../commit/{ => impl}/CommitUtilsWithMR.java | 68 +- .../fs/s3a/commit/impl/package-info.java | 31 + .../s3a/commit/magic/MagicCommitTracker.java | 55 +- .../commit/magic/MagicS3GuardCommitter.java | 150 +- .../staging/DirectoryStagingCommitter.java | 8 +- .../staging/PartitionedStagingCommitter.java | 41 +- .../s3a/commit/staging/StagingCommitter.java | 134 +- .../hadoop/fs/s3a/impl/CreateFileBuilder.java | 263 + .../s3a/impl/GetContentSummaryOperation.java | 3 +- .../hadoop/fs/s3a/impl/InternalConstants.java | 10 + .../hadoop/fs/s3a/impl/MkdirOperation.java | 68 +- .../hadoop/fs/s3a/impl/PutObjectOptions.java | 104 + .../fs/s3a/impl/RequestFactoryImpl.java | 76 +- .../fs/s3a/impl/S3AMultipartUploader.java | 3 +- .../BlockOutputStreamStatistics.java | 3 +- .../s3a/statistics/CommitterStatistics.java | 8 + .../s3a/statistics/PutTrackerStatistics.java | 29 + .../impl/EmptyS3AStatisticsContext.java | 7 + .../markdown/tools/hadoop-aws/auditing.md | 11 +- .../markdown/tools/hadoop-aws/committers.md | 11 +- .../site/markdown/tools/hadoop-aws/index.md | 47 +- .../markdown/tools/hadoop-aws/performance.md | 30 + .../site/markdown/tools/hadoop-aws/testing.md | 12 + .../tools/hadoop-aws/troubleshooting_s3a.md | 27 +- .../s3a/ITestS3AContractVectoredRead.java | 159 + .../fs/s3a/ITestS3AFileOperationCost.java | 63 - .../hadoop/fs/s3a/ITestS3AMiscOperations.java | 16 +- .../hadoop/fs/s3a/ITestS3AStorageClass.java | 217 + .../hadoop/fs/s3a/MockS3AFileSystem.java | 30 +- .../hadoop/fs/s3a/MultipartTestUtils.java | 3 +- .../hadoop/fs/s3a/S3ATestConstants.java | 11 + .../apache/hadoop/fs/s3a/S3ATestUtils.java | 38 +- .../fs/s3a/TestS3ABlockOutputStream.java | 4 +- .../fs/s3a/TestS3AInputStreamRetry.java | 3 +- .../fs/s3a/audit/ITestAuditAccessChecks.java | 4 +- .../hadoop/fs/s3a/auth/ITestAssumeRole.java | 39 +- .../fs/s3a/commit/AbstractCommitITest.java | 101 +- .../s3a/commit/AbstractITCommitProtocol.java | 139 +- .../fs/s3a/commit/CommitterTestHelper.java | 148 + .../s3a/commit/ITestCommitOperationCost.java | 325 + .../fs/s3a/commit/ITestCommitOperations.java | 290 +- .../s3a/commit/ITestS3ACommitterFactory.java | 2 +- .../hadoop/fs/s3a/commit/TestTasks.java | 569 - .../magic/ITestMagicCommitProtocol.java | 27 +- .../magic/ITestS3AHugeMagicCommits.java | 29 +- .../staging/MockedStagingCommitter.java | 4 +- .../s3a/commit/staging/StagingTestBase.java | 39 +- .../staging/TestDirectoryCommitterScale.java | 50 +- .../commit/staging/TestStagingCommitter.java | 21 +- .../TestStagingDirectoryOutputCommitter.java | 47 +- .../TestStagingPartitionedFileListing.java | 4 +- .../TestStagingPartitionedJobCommit.java | 17 +- .../TestStagingPartitionedTaskCommit.java | 10 +- .../ITestStagingCommitProtocol.java | 2 +- .../commit/terasort/ITestTerasortOnS3A.java | 2 +- .../fs/s3a/impl/StubContextAccessor.java | 82 + .../fs/s3a/impl/TestCreateFileBuilder.java | 173 + .../fs/s3a/impl/TestRequestFactory.java | 11 +- .../s3a/performance/AbstractS3ACostTest.java | 24 +- .../s3a/performance/ITestCreateFileCost.java | 248 + .../s3a/performance/ITestS3ADeleteCost.java | 11 +- .../fs/s3a/performance/OperationCost.java | 5 +- .../performance/OperationCostValidator.java | 18 +- .../s3a/scale/AbstractSTestS3AHugeFiles.java | 32 + .../scale/ITestS3ADirectoryPerformance.java | 5 +- .../scale/ITestS3AHugeFilesStorageClass.java | 135 + .../ITestS3AMultipartUploadSizeLimits.java | 2 +- .../src/test/resources/log4j.properties | 2 +- hadoop-tools/hadoop-azure-datalake/pom.xml | 13 - hadoop-tools/hadoop-azure/.gitignore | 1 + .../hadoop/fs/azurebfs/AbfsCountersImpl.java | 6 +- .../hadoop/fs/azurebfs/AbfsStatistic.java | 11 +- .../fs/azurebfs/AzureBlobFileSystem.java | 2 +- .../fs/azurebfs/AzureBlobFileSystemStore.java | 30 +- .../fs/azurebfs/services/AbfsClient.java | 61 +- .../services/AbfsClientRenameResult.java | 61 + .../azurebfs/ITestAbfsCustomEncryption.java | 2 +- ...ITestAzureBlobFileSystemDelegationSAS.java | 6 +- .../ITestAzureBlobFileSystemRename.java | 31 + .../services/TestAbfsRenameRetryRecovery.java | 139 + hadoop-tools/hadoop-benchmark/pom.xml | 94 + .../src/main/assembly/uber.xml | 33 + .../src/main/findbugs/exclude.xml | 22 + .../benchmark/VectoredReadBenchmark.java | 245 + .../hadoop/benchmark}/package-info.java | 8 +- .../apache/hadoop/tools/DistCpOptions.java | 17 + .../apache/hadoop/tools/OptionsParser.java | 2 +- .../hadoop/tools/TestDistCpOptions.java | 4 +- .../contract/AbstractContractDistCpTest.java | 2 +- .../tools/mapred/TestCopyCommitter.java | 2 +- hadoop-tools/pom.xml | 1 + .../jdiff/Apache_Hadoop_YARN_API_2.10.2.xml | 22541 +++++++++ .../jdiff/Apache_Hadoop_YARN_API_3.3.3.xml | 26407 ++++++++++ .../Apache_Hadoop_YARN_Client_2.10.2.xml | 2832 ++ .../jdiff/Apache_Hadoop_YARN_Client_3.3.3.xml | 3067 ++ .../Apache_Hadoop_YARN_Common_2.10.2.xml | 2936 ++ .../jdiff/Apache_Hadoop_YARN_Common_3.3.3.xml | 3975 ++ ...Apache_Hadoop_YARN_Server_Common_3.3.3.xml | 1456 + .../GetLabelsToNodesResponse.java | 1 - .../GetNewReservationResponse.java | 1 - .../hadoop/yarn/api/records/Resource.java | 2 - .../api/records/timeline/TimelineHealth.java | 4 +- .../ApplicationAttemptNotFoundException.java | 1 - .../ApplicationNotFoundException.java | 1 - .../ContainerNotFoundException.java | 1 - .../EmbeddedSolrServerFactory.java | 1 - .../DistributedShellTimelinePlugin.java | 1 - .../yarn/service/monitor/probe/HttpProbe.java | 6 +- .../containerlaunch/TestAbstractLauncher.java | 1 - .../yarn/client/api/impl/AHSv2ClientImpl.java | 5 +- .../api/impl/AMRMTokenIdentifierForTest.java | 3 - .../hadoop-yarn/hadoop-yarn-common/pom.xml | 2 +- .../pb/GetLabelsToNodesResponsePBImpl.java | 2 - .../pb/GetNodesToLabelsResponsePBImpl.java | 3 - .../impl/pb/QueueConfigurationsPBImpl.java | 4 +- .../api/records/impl/pb/ResourcePBImpl.java | 1 - .../api/impl/TimelineReaderClientImpl.java | 16 +- .../AggregatedLogDeletionService.java | 59 +- .../LogAggregationFileControllerFactory.java | 199 +- .../ifile/IndexedFileAggregatedLogsBlock.java | 2 - .../LogAggregationIndexedFileController.java | 15 +- .../tfile/TFileAggregatedLogsBlock.java | 3 - .../pb/UpdateNodeResourceRequestPBImpl.java | 1 - .../security/ApplicationACLsManager.java | 1 - .../util/ResourceCalculatorProcessTree.java | 1 - .../yarn/webapp/GenericExceptionHandler.java | 4 +- .../hadoop/yarn/webapp/WebAppException.java | 10 + .../apache/hadoop/yarn/webapp/WebApps.java | 2 +- .../hadoop/yarn/webapp/view/JQueryUI.java | 10 +- .../jquery/jquery-ui-1.12.1.custom.min.js | 13 - .../jquery/jquery-ui-1.13.1.custom.min.js | 6 + .../hadoop/yarn/api/TestPBImplRecords.java | 8 + .../impl/TestTimelineReaderClientImpl.java | 20 +- .../LogAggregationTestUtils.java | 68 + .../TestAggregatedLogDeletionService.java | 731 +- ...stLogAggregationFileControllerFactory.java | 76 +- .../AggregatedLogDeletionServiceForTest.java | 72 + .../testutils/FileStatusUtils.java | 76 + .../testutils/LogAggregationTestcase.java | 444 + .../LogAggregationTestcaseBuilder.java | 172 + .../testutils/MockRMClientUtils.java | 74 + .../testutils/PathWithFileStatus.java | 45 + .../yarn/webapp/view/TestCommonViews.java | 1 - .../hadoop-yarn/hadoop-yarn-csi/pom.xml | 8 - .../webapp/NavBlock.java | 1 - .../timeline/TimelineStoreTestUtils.java | 1 - .../TestTimelineWebServicesWithSSL.java | 1 - .../yarn/server/metrics/package-info.java | 18 - .../containermanager/AuxServicesEvent.java | 1 - .../launcher/ContainerCleanup.java | 1 - .../launcher/ContainersLauncherEvent.java | 1 - .../localizer/ContainerLocalizer.java | 2 +- .../localizer/LocalResourceRequest.java | 1 - .../event/LocalizerResourceRequestEvent.java | 1 - .../sharedcache/SharedCacheUploader.java | 1 - .../loghandler/NonAggregatingLogHandler.java | 2 - .../NMLogAggregationStatusTracker.java | 1 - .../server/nodemanager/webapp/NavBlock.java | 1 - .../nodemanager/webapp/dao/AppInfo.java | 1 - .../TestContainerManagerWithLCE.java | 1 - .../nodemanager/TestDirectoryCollection.java | 2 - .../impl/pb/TestPBRecordImpl.java | 2 +- .../localizer/TestLocalResource.java | 1 - .../TestContainerSchedulerRecovery.java | 1 - .../nodemanager/webapp/TestNMWebFilter.java | 1 - .../resourcemanager/ClientRMService.java | 29 +- .../resourcemanager/ClusterMetrics.java | 2 - .../resourcemanager/DefaultAMSProcessor.java | 2 - .../resourcemanager/ResourceManager.java | 8 +- .../ahs/RMApplicationHistoryWriter.java | 1 - ...AbstractPreemptableResourceCalculator.java | 173 +- .../FifoIntraQueuePreemptionPlugin.java | 4 +- .../capacity/TempQueuePerPartition.java | 4 + .../nodelabels/NodeAttributesManagerImpl.java | 1 - .../DynamicResourceConfiguration.java | 1 - .../rmcontainer/RMContainer.java | 1 - .../rmnode/RMNodeUpdateContainerEvent.java | 1 - .../scheduler/AbstractResourceUsage.java | 5 +- .../scheduler/PartitionQueueMetrics.java | 1 + .../scheduler/QueueMetrics.java | 29 +- .../activities/ActivitiesManager.java | 3 - .../scheduler/capacity/AbstractCSQueue.java | 5 +- .../scheduler/capacity/AbstractLeafQueue.java | 8 +- .../scheduler/capacity/CSQueueUtils.java | 6 +- .../scheduler/capacity/CapacityScheduler.java | 7 +- .../scheduler/capacity/QueueCapacities.java | 9 - .../QueueManagementDynamicEditPolicy.java | 1 - .../scheduler/capacity/UsersManager.java | 4 +- .../allocator/ContainerAllocation.java | 3 - .../conf/LeveldbConfigurationStore.java | 4 +- .../capacity/conf/ZKConfigurationStore.java | 7 +- .../capacity/preemption/PreemptableQueue.java | 1 - .../scheduler/fifo/FifoAppAttempt.java | 3 - .../webapp/CapacitySchedulerPage.java | 21 +- .../resourcemanager/webapp/NavBlock.java | 1 - .../webapp/RMWebAppFilter.java | 1 - .../webapp/dao/ClusterInfo.java | 1 - .../webapp/dao/ResourcesInfo.java | 2 +- .../webapp/dao/UserMetricsInfo.java | 1 - .../resourcemanager/TestClientRMService.java | 4 +- .../resourcemanager/TestClientRMTokens.java | 18 +- ...cityPreemptionPolicyInterQueueWithDRF.java | 11 +- ...nalCapacityPreemptionPolicyIntraQueue.java | 11 +- ...reemptionPolicyIntraQueueFairOrdering.java | 10 +- ...tyPreemptionPolicyIntraQueueUserLimit.java | 33 +- .../recovery/RMStateStoreTestBase.java | 1 - .../recovery/TestFSRMStateStore.java | 2 +- .../TestCapacityOverTimePolicy.java | 2 +- .../attempt/TestAMLivelinessMonitor.java | 2 +- .../CapacitySchedulerPreemptionTestBase.java | 3 - .../CapacitySchedulerQueueHelpers.java | 59 + .../CapacitySchedulerTestUtilities.java | 11 + .../capacity/TestCapacityScheduler.java | 60 + .../capacity/TestCapacitySchedulerApps.java | 118 +- .../TestNodeLabelContainerAllocation.java | 1 - .../capacity/TestQueueCapacities.java | 1 - .../conf/TestZKConfigurationStore.java | 35 + ...riorityUtilizationQueueOrderingPolicy.java | 2 +- .../scheduler/fair/TestFSAppStarvation.java | 1 - .../TestVisitedResourceRequestTracker.java | 2 +- .../ClientToAMTokenIdentifierForTest.java | 3 - .../resourcemanager/webapp/TestNodesPage.java | 1 - .../webapp/TestRMWebServicesApps.java | 150 +- .../webapp/TestRMWebappAuthentication.java | 2 +- .../yarn/server/router/RouterMetrics.java | 482 +- .../clientrm/FederationClientInterceptor.java | 574 +- .../clientrm/RouterYarnClientUtils.java | 172 + .../yarn/server/router/TestRouterMetrics.java | 472 +- .../TestFederationClientInterceptor.java | 484 +- .../clientrm/TestRouterYarnClientUtils.java | 326 + .../TestableFederationClientInterceptor.java | 14 +- .../webapp/TestRouterWebServicesREST.java | 42 +- .../src/test/resources/capacity-scheduler.xml | 11 +- .../src/test/resources/yarn-site.xml | 4 + .../yarn/server/TestMiniYARNClusterForHA.java | 1 - .../timeline/EntityGroupPlugInForTest.java | 1 - .../yarn/server/timeline/TestLogInfo.java | 1 - .../DocumentStoreTimelineReaderImpl.java | 4 +- .../DocumentStoreTimelineWriterImpl.java | 8 +- .../storage/HBaseTimelineReaderImpl.java | 2 +- .../storage/HBaseTimelineWriterImpl.java | 14 + .../storage/apptoflow/AppToFlowTableRW.java | 2 +- .../collector/TimelineCollector.java | 85 +- .../storage/FileSystemTimelineReaderImpl.java | 2 +- .../storage/FileSystemTimelineWriterImpl.java | 15 + .../storage/NoOpTimelineWriterImpl.java | 7 + .../storage/TimelineWriter.java | 10 + .../collector/TestTimelineCollector.java | 66 +- .../server/webproxy/TestAppReportFetcher.java | 1 - .../webproxy/amfilter/TestAmFilter.java | 3 - .../hadoop-yarn/hadoop-yarn-ui/pom.xml | 30 +- .../src/main/webapp/package.json | 3 +- pom.xml | 7 + 1061 files changed, 195955 insertions(+), 6368 deletions(-) delete mode 100644 .BUILDING.txt.swp create mode 100644 hadoop-common-project/hadoop-common/dev-support/jdiff/Apache_Hadoop_Common_2.10.2.xml create mode 100644 hadoop-common-project/hadoop-common/dev-support/jdiff/Apache_Hadoop_Common_3.3.3.xml create mode 100644 hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/fs/FileRange.java create mode 100644 hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/fs/VectoredReadUtils.java create mode 100644 hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/fs/impl/CombinedFileRange.java create mode 100644 hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/fs/impl/FileRangeImpl.java create mode 100644 hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/io/WeakReferencedElasticByteBufferPool.java create mode 100644 hadoop-common-project/hadoop-common/src/site/markdown/release/2.10.2/CHANGELOG.2.10.2.md create mode 100644 hadoop-common-project/hadoop-common/src/site/markdown/release/2.10.2/RELEASENOTES.2.10.2.md create mode 100644 hadoop-common-project/hadoop-common/src/site/markdown/release/3.3.3/CHANGELOG.3.3.3.md create mode 100644 hadoop-common-project/hadoop-common/src/site/markdown/release/3.3.3/RELEASENOTES.3.3.3.md create mode 100644 hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/fs/TestVectoredReadUtils.java create mode 100644 hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/fs/contract/AbstractContractVectoredReadTest.java create mode 100644 hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/fs/contract/localfs/TestLocalFSContractVectoredRead.java create mode 100644 hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/fs/contract/rawlocal/TestRawLocalContractVectoredRead.java create mode 100644 hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/fs/viewfs/TestNestedMountPoint.java create mode 100644 hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/io/TestMoreWeakReferencedElasticByteBufferPool.java create mode 100644 hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/io/TestWeakReferencedElasticByteBufferPool.java create mode 100644 hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/hdfs/server/protocol/OutlierMetrics.java delete mode 100644 hadoop-hdfs-project/hadoop-hdfs-native-client/src/main/native/libhdfspp/CMakeLists-gtest.txt.in create mode 100644 hadoop-hdfs-project/hadoop-hdfs-native-client/src/main/native/libhdfspp/lib/x-platform/c-api/core/dirent.h create mode 100644 hadoop-hdfs-project/hadoop-hdfs-native-client/src/main/native/libhdfspp/lib/x-platform/c-api/dirent.cc create mode 100644 hadoop-hdfs-project/hadoop-hdfs-native-client/src/main/native/libhdfspp/lib/x-platform/c-api/dirent.h rename hadoop-hdfs-project/{hadoop-hdfs-rbf/src/main/java/org/apache/hadoop/hdfs/protocolPB/package-info.java => hadoop-hdfs-native-client/src/main/native/libhdfspp/lib/x-platform/c-api/extern/dirent.h} (69%) create mode 100644 hadoop-hdfs-project/hadoop-hdfs-native-client/src/main/native/libhdfspp/lib/x-platform/dirent.cc create mode 100644 hadoop-hdfs-project/hadoop-hdfs-native-client/src/main/native/libhdfspp/lib/x-platform/dirent.h create mode 100644 hadoop-hdfs-project/hadoop-hdfs-native-client/src/main/native/libhdfspp/tests/x-platform/c-api/dirent_test.cc create mode 100644 hadoop-hdfs-project/hadoop-hdfs-native-client/src/main/native/libhdfspp/tests/x-platform/c-api/dirent_test.h create mode 100644 hadoop-hdfs-project/hadoop-hdfs-native-client/src/main/native/libhdfspp/tests/x-platform/dirent_test.cc create mode 100644 hadoop-hdfs-project/hadoop-hdfs-native-client/src/main/native/libhdfspp/tests/x-platform/dirent_test.h create mode 100644 hadoop-hdfs-project/hadoop-hdfs/dev-support/jdiff/Apache_Hadoop_HDFS_2.10.2.xml create mode 100644 hadoop-hdfs-project/hadoop-hdfs/dev-support/jdiff/Apache_Hadoop_HDFS_3.3.3.xml create mode 100644 hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/blockmanagement/SlowPeerJsonReport.java create mode 100644 hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/blockmanagement/SlowPeerLatencyWithReportingNode.java create mode 100644 hadoop-mapreduce-project/dev-support/jdiff/Apache_Hadoop_MapReduce_Common_3.3.3.xml create mode 100644 hadoop-mapreduce-project/dev-support/jdiff/Apache_Hadoop_MapReduce_Core_3.3.3.xml create mode 100644 hadoop-mapreduce-project/dev-support/jdiff/Apache_Hadoop_MapReduce_JobClient_3.3.3.xml create mode 100644 hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/VectoredIOContext.java create mode 100644 hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/commit/CommitterStatisticNames.java delete mode 100644 hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/commit/Tasks.java create mode 100644 hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/commit/impl/AuditContextUpdater.java create mode 100644 hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/commit/impl/CommitContext.java rename hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/commit/{ => impl}/CommitOperations.java (82%) rename hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/commit/{ => impl}/CommitUtilsWithMR.java (74%) create mode 100644 hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/commit/impl/package-info.java create mode 100644 hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/impl/CreateFileBuilder.java create mode 100644 hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/impl/PutObjectOptions.java create mode 100644 hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/statistics/PutTrackerStatistics.java create mode 100644 hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/contract/s3a/ITestS3AContractVectoredRead.java create mode 100644 hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/ITestS3AStorageClass.java create mode 100644 hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/commit/CommitterTestHelper.java create mode 100644 hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/commit/ITestCommitOperationCost.java delete mode 100644 hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/commit/TestTasks.java create mode 100644 hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/impl/StubContextAccessor.java create mode 100644 hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/impl/TestCreateFileBuilder.java create mode 100644 hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/performance/ITestCreateFileCost.java create mode 100644 hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/scale/ITestS3AHugeFilesStorageClass.java create mode 100644 hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/services/AbfsClientRenameResult.java create mode 100644 hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/services/TestAbfsRenameRetryRecovery.java create mode 100644 hadoop-tools/hadoop-benchmark/pom.xml create mode 100644 hadoop-tools/hadoop-benchmark/src/main/assembly/uber.xml create mode 100644 hadoop-tools/hadoop-benchmark/src/main/findbugs/exclude.xml create mode 100644 hadoop-tools/hadoop-benchmark/src/main/java/org/apache/hadoop/benchmark/VectoredReadBenchmark.java rename {hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/hdfs/protocolPB => hadoop-tools/hadoop-benchmark/src/main/java/org/apache/hadoop/benchmark}/package-info.java (89%) create mode 100644 hadoop-yarn-project/hadoop-yarn/dev-support/jdiff/Apache_Hadoop_YARN_API_2.10.2.xml create mode 100644 hadoop-yarn-project/hadoop-yarn/dev-support/jdiff/Apache_Hadoop_YARN_API_3.3.3.xml create mode 100644 hadoop-yarn-project/hadoop-yarn/dev-support/jdiff/Apache_Hadoop_YARN_Client_2.10.2.xml create mode 100644 hadoop-yarn-project/hadoop-yarn/dev-support/jdiff/Apache_Hadoop_YARN_Client_3.3.3.xml create mode 100644 hadoop-yarn-project/hadoop-yarn/dev-support/jdiff/Apache_Hadoop_YARN_Common_2.10.2.xml create mode 100644 hadoop-yarn-project/hadoop-yarn/dev-support/jdiff/Apache_Hadoop_YARN_Common_3.3.3.xml create mode 100644 hadoop-yarn-project/hadoop-yarn/dev-support/jdiff/Apache_Hadoop_YARN_Server_Common_3.3.3.xml delete mode 100644 hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/main/resources/webapps/static/jquery/jquery-ui-1.12.1.custom.min.js create mode 100644 hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/main/resources/webapps/static/jquery/jquery-ui-1.13.1.custom.min.js create mode 100644 hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/test/java/org/apache/hadoop/yarn/logaggregation/LogAggregationTestUtils.java create mode 100644 hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/test/java/org/apache/hadoop/yarn/logaggregation/testutils/AggregatedLogDeletionServiceForTest.java create mode 100644 hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/test/java/org/apache/hadoop/yarn/logaggregation/testutils/FileStatusUtils.java create mode 100644 hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/test/java/org/apache/hadoop/yarn/logaggregation/testutils/LogAggregationTestcase.java create mode 100644 hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/test/java/org/apache/hadoop/yarn/logaggregation/testutils/LogAggregationTestcaseBuilder.java create mode 100644 hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/test/java/org/apache/hadoop/yarn/logaggregation/testutils/MockRMClientUtils.java create mode 100644 hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/test/java/org/apache/hadoop/yarn/logaggregation/testutils/PathWithFileStatus.java delete mode 100644 hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-common/src/main/java/org/apache/hadoop/yarn/server/metrics/package-info.java diff --git a/.BUILDING.txt.swp b/.BUILDING.txt.swp deleted file mode 100644 index 1fb0c25d0a5160f91646c273b130ff1d4363c74f..0000000000000000000000000000000000000000 GIT binary patch literal 0 HcmV?d00001 literal 16384 zcmeHN&5s;M6)yvU@DUOv95_BmWbHcJ-8-|!_O2Ab-nEmB_rvmd4Guz9O?S;q@Ah}~NkP#Ik{{RRGgx{;~-mfeMAzK2}BmL&1 zK3={0)qAh1UQd7i=IT0K2)hEWM}!!^`sMzM??k8i!&72p%bbh<)oWeGm$cpqt{EM4 z!%W9AJF&~xSYWiP%}z9uiPlAQX=`1Fm>byZ|*nP^-FGf1Wz_`Ck3H zT9;otR{!_9{@J?x+sEqvR@Yyy%ilRx|88A>t1drNE70ii*SbEg>%VcV{x5a?b9Mdi zj@5s#E`PB8{9--fMt)t~ezXj<473ci473ci473ci473ci473ci473dVUos#QA#P)4 zKjn)sp8q%Z|F<3%;!WTs;OoE(!1KTmxDK2F{`g5Dehz#G(7>a>YYz$01K#_D5U&H< zz$Op@fBHE30N)3$0}H@!KPJT6z^{QH0zUv=0$v2p18;v+h_`@m0$&171Ao9J{`bId zfIGlbz&YUEj|lNHum^krc=N+T`~>(h@D<=;;1>@F@jc+Xz_);1U>&#&JPtepyz(K8 z2lRkX0|Iyh34{S~3wQwdCGrP91AYX&0(=$t6*y!bU%UTF1)?WtypyDc3ZtFwmxEv- zJRevS3$BVwlTolnfu*2FBj*ac6h*_-jmmx)>v80J#X3XBfikxGjUSOwnUYr7DABul zrezXMAUN#ow+Oc7MDk!^Cv?(;!#fn{BUX^) zdy@gh+#Jew7Q^{Q+ta5hPL9+Zp{yaxcMY`}MuRj1AE2SL7@Z{W^GAf$!(i^q>#RI? zJD5*dB+{usW=@%E##o`Co*+e~HBmp!qhTBq<8VQ+NayC!*k%$u9n?JeBn^SX1;L>a zW#NLMauiG(9=SP$LHV?8MHPiyD*I*b%D`!z#Uq*Kfr*^5 z?oikF4{i0p?MkBptX`pvOO*{^k7J#Ku&0-oX)#;~=kF@Qn@gaqRk=%LMzUlJ%Lo38 zH;*jsvr^Tv&>j5j41+{XsMr}|w0w{b35C{XzLdX8n|1jIwAiQ&5v4{N=yz9T##jqh zecgYu>v^+1fpkna-%fUd3jyBWFSDlo3~izvJ-c$1I^hL`Nd@<0Cm|>o*s>_Jaq!J} zN115Ff3SJZE{T~d+R&C?66(%}^JnLHg(Ef%nb-CXSId(OUbl-#&@_%^PJKnCRY@pT zD_2YNA?>?pA>$o6RF(!9Ax3z}_bo{e8Y!I%7?Rd*^k_xd9jA*>1o!+Uz#C7-lsQdx zX=rujvM;EZAmw6KElm`;k#ux(Jw%U-IUVvhvN|hWs&i)5jZ(|8tDmVc;VX(7UtCM` za<7R>wNOcu)VLBxHD!{1YDOT?0MXVFn&xRK(~;aFIcMz1B* z-C(_ZV;Z4dU%@7fLW>C+qCiqo7Fil2mdtqqAPG*X)7fpD8Fo&ahBfU zW!<>aqp^nPvSI%x%71Qg0X=(49a0n{?&R1o_PNA10^O?ce!vAW>zFhCFbMOY&78kg z#)v-gLVHtR1Wjg z`55!{y*D@3Zb4n_1~QHj5G#vT^vvNI2*DnM?eoB)IXG6{%yVF);3ht<@((Z3n_TS+ z(9wqwA30%Bh!Zy|%N-XE6HLJ;I}bBlz)YbZQ)-F-3d&#Min1~~{Fvap!S09H!3%we zXLfH6p<(2y3hGFoPeMDY>Q<}pM75zbvA~nW+XxDl1k9>(-iXZ$OYgm@4rlB;FdwGy zkRmm~GoA2k*zuDcr9&(nVXLYw8*at_T%8R?tvDQaf=H7z%2XRmMGcNn*g3$BobsaZ z{)?!2c>R0?=2c-1WWiCuq#z5;d2v8qe@bJ-T6hwI@O{xh8SLt)V78%7$v$L>B~+$zWvh3YX0Gr_Z)zQH203k6SSmI@ zHM?&r-S8YiVcFMkx|t(G4zPjV#+WRV_j;Tzc+XbZv3B)2w~8`Bkrvok*^X;7VFmka zI?t>iP-CKIT6m&xT519!pL&-wiRsL+(U*DZrj3fTb4^ua-pp{i*`q~242NYid$LZ| z*&9cY95J`@&Wcj-0OsJu^#KGwjFQQ1|5tCsHEnP{SUNbrF-Lx&Il7rz-Y_8@nGY`sn%`xnKPAi^-8SJ=O<%6c(lvp6}jW zc4f5-;le%-n0IreJ40J}x<@^4W_-Z&7a?ztOZUaqFL}S>P;N@^b5-cC97^#0{|(&N zKZUzH-~Vxq_ODlQzb^m{{26sG16#lifZIL>w6B(dmVuUmmVuUmmVuUmmVuUmmVuUm zmVy6m237}D=2p2mpDsaaY8%O#Cc%K)c$^_E#yL}8e{FLe8JrA{>1_QFL0rKdT)N8SiRkkzm{uM+c(0Axa{ zyRuKkLmaZkjrvtCz8mhLr+;qRkdi`4WUqbZ#weuWa}d^_3b8yw-loY!)pDDW_N5-` NK{pR(eLvCz{{Xbw)c^nh diff --git a/LICENSE-binary b/LICENSE-binary index fe60ac3609c2c..499485263906a 100644 --- a/LICENSE-binary +++ b/LICENSE-binary @@ -231,7 +231,7 @@ com.github.stephenc.jcip:jcip-annotations:1.0-1 com.google:guice:4.0 com.google:guice-servlet:4.0 com.google.api.grpc:proto-google-common-protos:1.0.0 -com.google.code.gson:2.2.4 +com.google.code.gson:2.9.0 com.google.errorprone:error_prone_annotations:2.2.0 com.google.j2objc:j2objc-annotations:1.1 com.google.json-simple:json-simple:1.1.1 @@ -241,7 +241,7 @@ com.google.guava:guava:27.0-jre com.google.guava:listenablefuture:9999.0-empty-to-avoid-conflict-with-guava com.microsoft.azure:azure-storage:7.0.0 com.nimbusds:nimbus-jose-jwt:9.8.1 -com.squareup.okhttp:okhttp:2.7.5 +com.squareup.okhttp3:okhttp:4.9.3 com.squareup.okio:okio:1.6.0 com.zaxxer:HikariCP:4.0.3 commons-beanutils:commons-beanutils:1.9.3 diff --git a/dev-support/Jenkinsfile b/dev-support/Jenkinsfile index 0ec32e385d275..51225268b653a 100644 --- a/dev-support/Jenkinsfile +++ b/dev-support/Jenkinsfile @@ -47,7 +47,7 @@ pipeline { options { buildDiscarder(logRotator(numToKeepStr: '5')) - timeout (time: 24, unit: 'HOURS') + timeout (time: 48, unit: 'HOURS') timestamps() checkoutToSubdirectory('src') } @@ -55,7 +55,7 @@ pipeline { environment { YETUS='yetus' // Branch or tag name. Yetus release tags are 'rel/X.Y.Z' - YETUS_VERSION='f9ba0170a5787a5f4662d3769804fef0226a182f' + YETUS_VERSION='rel/0.14.0' } parameters { diff --git a/dev-support/bin/yetus-wrapper b/dev-support/bin/yetus-wrapper index 8532d1749701b..a93833767a201 100755 --- a/dev-support/bin/yetus-wrapper +++ b/dev-support/bin/yetus-wrapper @@ -77,7 +77,7 @@ WANTED="$1" shift ARGV=("$@") -HADOOP_YETUS_VERSION=${HADOOP_YETUS_VERSION:-0.13.0} +HADOOP_YETUS_VERSION=${HADOOP_YETUS_VERSION:-0.14.0} BIN=$(yetus_abs "${BASH_SOURCE-$0}") BINDIR=$(dirname "${BIN}") diff --git a/dev-support/docker/Dockerfile_debian_10 b/dev-support/docker/Dockerfile_debian_10 index 256f0d5786ab9..ec3de11035cee 100644 --- a/dev-support/docker/Dockerfile_debian_10 +++ b/dev-support/docker/Dockerfile_debian_10 @@ -82,6 +82,7 @@ ENV HADOOP_SKIP_YETUS_VERIFICATION true #### # Install packages #### +RUN pkg-resolver/install-cmake.sh debian:10 RUN pkg-resolver/install-spotbugs.sh debian:10 RUN pkg-resolver/install-boost.sh debian:10 RUN pkg-resolver/install-protobuf.sh debian:10 diff --git a/dev-support/docker/pkg-resolver/install-maven.sh b/dev-support/docker/pkg-resolver/install-maven.sh index f9ff961a190f9..d1d0dc97fe5e4 100644 --- a/dev-support/docker/pkg-resolver/install-maven.sh +++ b/dev-support/docker/pkg-resolver/install-maven.sh @@ -40,7 +40,7 @@ fi if [ "$version_to_install" == "3.6.3" ]; then mkdir -p /opt/maven /tmp/maven && - curl -L -s -S https://mirrors.estointernet.in/apache/maven/maven-3/3.6.3/binaries/apache-maven-3.6.3-bin.tar.gz \ + curl -L -s -S https://dlcdn.apache.org/maven/maven-3/3.6.3/binaries/apache-maven-3.6.3-bin.tar.gz \ -o /tmp/maven/apache-maven-3.6.3-bin.tar.gz && tar xzf /tmp/maven/apache-maven-3.6.3-bin.tar.gz --strip-components 1 -C /opt/maven else diff --git a/dev-support/docker/pkg-resolver/packages.json b/dev-support/docker/pkg-resolver/packages.json index afe8a7a32b107..2225517834601 100644 --- a/dev-support/docker/pkg-resolver/packages.json +++ b/dev-support/docker/pkg-resolver/packages.json @@ -62,7 +62,6 @@ "centos:8": "clang" }, "cmake": { - "debian:10": "cmake", "ubuntu:focal": "cmake", "ubuntu:focal::arch64": "cmake" }, diff --git a/dev-support/git-jira-validation/git_jira_fix_version_check.py b/dev-support/git-jira-validation/git_jira_fix_version_check.py index c2e12a13aae22..513cc8e25ffdb 100644 --- a/dev-support/git-jira-validation/git_jira_fix_version_check.py +++ b/dev-support/git-jira-validation/git_jira_fix_version_check.py @@ -71,10 +71,9 @@ print("Commit seems reverted. \t\t\t Commit: " + commit) continue ACTUAL_PROJECT_JIRA = None - for project_jira in project_jira_keys: - if project_jira in commit: - ACTUAL_PROJECT_JIRA = project_jira - break + matches = re.findall('|'.join(project_jira_keys), commit) + if matches: + ACTUAL_PROJECT_JIRA = matches[0] if not ACTUAL_PROJECT_JIRA: print("WARN: Jira not found. \t\t\t Commit: " + commit) continue diff --git a/dev-support/hadoop-vote.sh b/dev-support/hadoop-vote.sh index 3d381fb0b4be2..825e776fabf16 100755 --- a/dev-support/hadoop-vote.sh +++ b/dev-support/hadoop-vote.sh @@ -189,6 +189,8 @@ pushd "${OUTPUT_DIR}" download_and_import_keys download_release_candidate +pushd "${HADOOP_RC_VERSION}" + execute verify_signatures execute verify_checksums execute unzip_from_source @@ -196,6 +198,7 @@ execute rat_test execute build_from_source execute build_tar_from_source +popd popd print_when_exit diff --git a/hadoop-client-modules/hadoop-client-minicluster/pom.xml b/hadoop-client-modules/hadoop-client-minicluster/pom.xml index 06e36837a2098..4c8900dc2af0d 100644 --- a/hadoop-client-modules/hadoop-client-minicluster/pom.xml +++ b/hadoop-client-modules/hadoop-client-minicluster/pom.xml @@ -757,6 +757,12 @@ META-INF/versions/11/module-info.class + + com.google.code.gson:gson + + META-INF/versions/9/module-info.class + + diff --git a/hadoop-client-modules/hadoop-client-runtime/pom.xml b/hadoop-client-modules/hadoop-client-runtime/pom.xml index 35fbd7665fb26..98756c2439544 100644 --- a/hadoop-client-modules/hadoop-client-runtime/pom.xml +++ b/hadoop-client-modules/hadoop-client-runtime/pom.xml @@ -249,6 +249,13 @@ META-INF/versions/11/module-info.class + + com.google.code.gson:gson + + META-INF/versions/9/module-info.class + + + diff --git a/hadoop-client-modules/hadoop-client/pom.xml b/hadoop-client-modules/hadoop-client/pom.xml index b48a221bdf179..5299c9e8713df 100644 --- a/hadoop-client-modules/hadoop-client/pom.xml +++ b/hadoop-client-modules/hadoop-client/pom.xml @@ -114,6 +114,18 @@ org.eclipse.jetty jetty-server + + org.jetbrains.kotlin + kotlin-stdlib + + + org.jetbrains.kotlin + kotlin-stdlib-common + + + com.squareup.okhttp3 + okhttp + com.sun.jersey jersey-core diff --git a/hadoop-cloud-storage-project/hadoop-cos/pom.xml b/hadoop-cloud-storage-project/hadoop-cos/pom.xml index fa47e354c7998..ca7c4bf516cad 100644 --- a/hadoop-cloud-storage-project/hadoop-cos/pom.xml +++ b/hadoop-cloud-storage-project/hadoop-cos/pom.xml @@ -109,7 +109,7 @@ com.qcloud cos_api-bundle - 5.6.19 + 5.6.69 compile diff --git a/hadoop-common-project/hadoop-auth-examples/src/main/java/org/apache/hadoop/security/authentication/examples/RequestLoggerFilter.java b/hadoop-common-project/hadoop-auth-examples/src/main/java/org/apache/hadoop/security/authentication/examples/RequestLoggerFilter.java index 80489917e0857..e96e5062919a7 100644 --- a/hadoop-common-project/hadoop-auth-examples/src/main/java/org/apache/hadoop/security/authentication/examples/RequestLoggerFilter.java +++ b/hadoop-common-project/hadoop-auth-examples/src/main/java/org/apache/hadoop/security/authentication/examples/RequestLoggerFilter.java @@ -116,6 +116,7 @@ private List getHeaderValues(String name, boolean reset) { public void addCookie(Cookie cookie) { super.addCookie(cookie); List cookies = getHeaderValues("Set-Cookie", false); + cookies.addAll(getHeaderValues("set-cookie", false)); cookies.add(cookie.getName() + "=" + cookie.getValue()); } diff --git a/hadoop-common-project/hadoop-auth/src/main/java/org/apache/hadoop/security/authentication/client/AuthenticatedURL.java b/hadoop-common-project/hadoop-auth/src/main/java/org/apache/hadoop/security/authentication/client/AuthenticatedURL.java index 32f4edfbc5710..cb7d36368aa39 100644 --- a/hadoop-common-project/hadoop-auth/src/main/java/org/apache/hadoop/security/authentication/client/AuthenticatedURL.java +++ b/hadoop-common-project/hadoop-auth/src/main/java/org/apache/hadoop/security/authentication/client/AuthenticatedURL.java @@ -92,6 +92,9 @@ public synchronized Map> get(URI uri, @Override public void put(URI uri, Map> responseHeaders) { List headers = responseHeaders.get("Set-Cookie"); + if (headers == null) { + headers = responseHeaders.get("set-cookie"); + } if (headers != null) { for (String header : headers) { List cookies; diff --git a/hadoop-common-project/hadoop-auth/src/main/java/org/apache/hadoop/security/authentication/client/KerberosAuthenticator.java b/hadoop-common-project/hadoop-auth/src/main/java/org/apache/hadoop/security/authentication/client/KerberosAuthenticator.java index 06b63c1b9916c..30e65efe10cba 100644 --- a/hadoop-common-project/hadoop-auth/src/main/java/org/apache/hadoop/security/authentication/client/KerberosAuthenticator.java +++ b/hadoop-common-project/hadoop-auth/src/main/java/org/apache/hadoop/security/authentication/client/KerberosAuthenticator.java @@ -280,6 +280,9 @@ private boolean isNegotiate(HttpURLConnection conn) throws IOException { boolean negotiate = false; if (conn.getResponseCode() == HttpURLConnection.HTTP_UNAUTHORIZED) { String authHeader = conn.getHeaderField(WWW_AUTHENTICATE); + if (authHeader == null) { + authHeader = conn.getHeaderField(WWW_AUTHENTICATE.toLowerCase()); + } negotiate = authHeader != null && authHeader.trim().startsWith(NEGOTIATE); } return negotiate; @@ -388,6 +391,9 @@ private byte[] readToken(HttpURLConnection conn) int status = conn.getResponseCode(); if (status == HttpURLConnection.HTTP_OK || status == HttpURLConnection.HTTP_UNAUTHORIZED) { String authHeader = conn.getHeaderField(WWW_AUTHENTICATE); + if (authHeader == null) { + authHeader = conn.getHeaderField(WWW_AUTHENTICATE.toLowerCase()); + } if (authHeader == null || !authHeader.trim().startsWith(NEGOTIATE)) { throw new AuthenticationException("Invalid SPNEGO sequence, '" + WWW_AUTHENTICATE + "' header incorrect: " + authHeader); diff --git a/hadoop-common-project/hadoop-auth/src/main/java/org/apache/hadoop/security/authentication/server/AuthenticationFilter.java b/hadoop-common-project/hadoop-auth/src/main/java/org/apache/hadoop/security/authentication/server/AuthenticationFilter.java index 3658bd8b8ec01..7cc70c493c0f6 100644 --- a/hadoop-common-project/hadoop-auth/src/main/java/org/apache/hadoop/security/authentication/server/AuthenticationFilter.java +++ b/hadoop-common-project/hadoop-auth/src/main/java/org/apache/hadoop/security/authentication/server/AuthenticationFilter.java @@ -616,7 +616,9 @@ && getMaxInactiveInterval() > 0) { // present.. reset to 403 if not found.. if ((errCode == HttpServletResponse.SC_UNAUTHORIZED) && (!httpResponse.containsHeader( - KerberosAuthenticator.WWW_AUTHENTICATE))) { + KerberosAuthenticator.WWW_AUTHENTICATE) + && !httpResponse.containsHeader( + KerberosAuthenticator.WWW_AUTHENTICATE.toLowerCase()))) { errCode = HttpServletResponse.SC_FORBIDDEN; } // After Jetty 9.4.21, sendError() no longer allows a custom message. diff --git a/hadoop-common-project/hadoop-auth/src/main/java/org/apache/hadoop/security/authentication/util/CertificateUtil.java b/hadoop-common-project/hadoop-auth/src/main/java/org/apache/hadoop/security/authentication/util/CertificateUtil.java index cf17aca15ceac..f25602c67d4a3 100644 --- a/hadoop-common-project/hadoop-auth/src/main/java/org/apache/hadoop/security/authentication/util/CertificateUtil.java +++ b/hadoop-common-project/hadoop-auth/src/main/java/org/apache/hadoop/security/authentication/util/CertificateUtil.java @@ -18,7 +18,6 @@ package org.apache.hadoop.security.authentication.util; import java.io.ByteArrayInputStream; -import java.io.UnsupportedEncodingException; import java.nio.charset.StandardCharsets; import java.security.PublicKey; import java.security.cert.CertificateException; diff --git a/hadoop-common-project/hadoop-auth/src/main/java/org/apache/hadoop/security/authentication/util/KerberosUtil.java b/hadoop-common-project/hadoop-auth/src/main/java/org/apache/hadoop/security/authentication/util/KerberosUtil.java index fc6f957b9622e..5125be078d67b 100644 --- a/hadoop-common-project/hadoop-auth/src/main/java/org/apache/hadoop/security/authentication/util/KerberosUtil.java +++ b/hadoop-common-project/hadoop-auth/src/main/java/org/apache/hadoop/security/authentication/util/KerberosUtil.java @@ -236,7 +236,7 @@ public static final String getServicePrincipal(String service, */ static final String[] getPrincipalNames(String keytabFileName) throws IOException { Keytab keytab = Keytab.loadKeytab(new File(keytabFileName)); - Set principals = new HashSet(); + Set principals = new HashSet<>(); List entries = keytab.getPrincipals(); for (PrincipalName entry : entries) { principals.add(entry.getName().replace("\\", "/")); diff --git a/hadoop-common-project/hadoop-auth/src/test/java/org/apache/hadoop/security/authentication/KerberosTestUtils.java b/hadoop-common-project/hadoop-auth/src/test/java/org/apache/hadoop/security/authentication/KerberosTestUtils.java index 8fc08e2171f67..293871bcd0620 100644 --- a/hadoop-common-project/hadoop-auth/src/test/java/org/apache/hadoop/security/authentication/KerberosTestUtils.java +++ b/hadoop-common-project/hadoop-auth/src/test/java/org/apache/hadoop/security/authentication/KerberosTestUtils.java @@ -108,9 +108,9 @@ public AppConfigurationEntry[] getAppConfigurationEntry(String name) { public static T doAs(String principal, final Callable callable) throws Exception { LoginContext loginContext = null; try { - Set principals = new HashSet(); + Set principals = new HashSet<>(); principals.add(new KerberosPrincipal(KerberosTestUtils.getClientPrincipal())); - Subject subject = new Subject(false, principals, new HashSet(), new HashSet()); + Subject subject = new Subject(false, principals, new HashSet<>(), new HashSet<>()); loginContext = new LoginContext("", subject, null, new KerberosConfiguration(principal)); loginContext.login(); subject = loginContext.getSubject(); diff --git a/hadoop-common-project/hadoop-auth/src/test/java/org/apache/hadoop/security/authentication/client/TestAuthenticatedURL.java b/hadoop-common-project/hadoop-auth/src/test/java/org/apache/hadoop/security/authentication/client/TestAuthenticatedURL.java index b56fc65b25b45..5b3de5208f49a 100644 --- a/hadoop-common-project/hadoop-auth/src/test/java/org/apache/hadoop/security/authentication/client/TestAuthenticatedURL.java +++ b/hadoop-common-project/hadoop-auth/src/test/java/org/apache/hadoop/security/authentication/client/TestAuthenticatedURL.java @@ -89,6 +89,44 @@ public void testExtractTokenFail() throws Exception { } } + @Test + public void testExtractTokenCookieHeader() throws Exception { + HttpURLConnection conn = Mockito.mock(HttpURLConnection.class); + + Mockito.when(conn.getResponseCode()).thenReturn(HttpURLConnection.HTTP_OK); + + String tokenStr = "foo"; + Map> headers = new HashMap<>(); + List cookies = new ArrayList<>(); + cookies.add(AuthenticatedURL.AUTH_COOKIE + "=" + tokenStr); + headers.put("Set-Cookie", cookies); + Mockito.when(conn.getHeaderFields()).thenReturn(headers); + + AuthenticatedURL.Token token = new AuthenticatedURL.Token(); + AuthenticatedURL.extractToken(conn, token); + + Assert.assertTrue(token.isSet()); + } + + @Test + public void testExtractTokenLowerCaseCookieHeader() throws Exception { + HttpURLConnection conn = Mockito.mock(HttpURLConnection.class); + + Mockito.when(conn.getResponseCode()).thenReturn(HttpURLConnection.HTTP_OK); + + String tokenStr = "foo"; + Map> headers = new HashMap<>(); + List cookies = new ArrayList<>(); + cookies.add(AuthenticatedURL.AUTH_COOKIE + "=" + tokenStr); + headers.put("set-cookie", cookies); + Mockito.when(conn.getHeaderFields()).thenReturn(headers); + + AuthenticatedURL.Token token = new AuthenticatedURL.Token(); + AuthenticatedURL.extractToken(conn, token); + + Assert.assertTrue(token.isSet()); + } + @Test public void testConnectionConfigurator() throws Exception { HttpURLConnection conn = Mockito.mock(HttpURLConnection.class); diff --git a/hadoop-common-project/hadoop-auth/src/test/java/org/apache/hadoop/security/authentication/client/TestKerberosAuthenticator.java b/hadoop-common-project/hadoop-auth/src/test/java/org/apache/hadoop/security/authentication/client/TestKerberosAuthenticator.java index 177bcb4547e33..bc316ef8cb8b2 100644 --- a/hadoop-common-project/hadoop-auth/src/test/java/org/apache/hadoop/security/authentication/client/TestKerberosAuthenticator.java +++ b/hadoop-common-project/hadoop-auth/src/test/java/org/apache/hadoop/security/authentication/client/TestKerberosAuthenticator.java @@ -21,8 +21,13 @@ import static org.apache.hadoop.security.authentication.server.KerberosAuthenticationHandler.NAME_RULES; import java.io.IOException; +import java.lang.reflect.InvocationTargetException; +import java.lang.reflect.Method; import java.nio.charset.CharacterCodingException; import javax.security.sasl.AuthenticationException; + +import org.apache.commons.codec.binary.Base64; +import org.apache.commons.lang3.reflect.FieldUtils; import org.apache.hadoop.minikdc.KerberosSecurityTestcase; import org.apache.hadoop.security.authentication.KerberosTestUtils; import org.apache.hadoop.security.authentication.server.AuthenticationFilter; @@ -32,10 +37,12 @@ import org.junit.Assert; import org.junit.Before; import org.junit.Test; +import org.mockito.Mockito; import java.io.File; import java.net.HttpURLConnection; import java.net.URL; +import java.util.Arrays; import java.util.Properties; import java.util.concurrent.Callable; @@ -248,4 +255,79 @@ public void testWrapExceptionWithMessage() { Assert.assertTrue(ex.equals(ex2)); } + @Test(timeout = 60000) + public void testNegotiate() throws NoSuchMethodException, InvocationTargetException, + IllegalAccessException, IOException { + KerberosAuthenticator kerberosAuthenticator = new KerberosAuthenticator(); + + HttpURLConnection conn = Mockito.mock(HttpURLConnection.class); + Mockito.when(conn.getHeaderField(KerberosAuthenticator.WWW_AUTHENTICATE)). + thenReturn(KerberosAuthenticator.NEGOTIATE); + Mockito.when(conn.getResponseCode()).thenReturn(HttpURLConnection.HTTP_UNAUTHORIZED); + + Method method = KerberosAuthenticator.class.getDeclaredMethod("isNegotiate", + HttpURLConnection.class); + method.setAccessible(true); + + Assert.assertTrue((boolean)method.invoke(kerberosAuthenticator, conn)); + } + + @Test(timeout = 60000) + public void testNegotiateLowerCase() throws NoSuchMethodException, InvocationTargetException, + IllegalAccessException, IOException { + KerberosAuthenticator kerberosAuthenticator = new KerberosAuthenticator(); + + HttpURLConnection conn = Mockito.mock(HttpURLConnection.class); + Mockito.when(conn.getHeaderField("www-authenticate")) + .thenReturn(KerberosAuthenticator.NEGOTIATE); + Mockito.when(conn.getResponseCode()).thenReturn(HttpURLConnection.HTTP_UNAUTHORIZED); + + Method method = KerberosAuthenticator.class.getDeclaredMethod("isNegotiate", + HttpURLConnection.class); + method.setAccessible(true); + + Assert.assertTrue((boolean)method.invoke(kerberosAuthenticator, conn)); + } + + @Test(timeout = 60000) + public void testReadToken() throws NoSuchMethodException, IOException, IllegalAccessException, + InvocationTargetException { + KerberosAuthenticator kerberosAuthenticator = new KerberosAuthenticator(); + FieldUtils.writeField(kerberosAuthenticator, "base64", new Base64(), true); + + Base64 base64 = new Base64(); + + HttpURLConnection conn = Mockito.mock(HttpURLConnection.class); + Mockito.when(conn.getResponseCode()).thenReturn(HttpURLConnection.HTTP_UNAUTHORIZED); + Mockito.when(conn.getHeaderField(KerberosAuthenticator.WWW_AUTHENTICATE)) + .thenReturn(KerberosAuthenticator.NEGOTIATE + " " + + Arrays.toString(base64.encode("foobar".getBytes()))); + + Method method = KerberosAuthenticator.class.getDeclaredMethod("readToken", + HttpURLConnection.class); + method.setAccessible(true); + + method.invoke(kerberosAuthenticator, conn); // expecting this not to throw an exception + } + + @Test(timeout = 60000) + public void testReadTokenLowerCase() throws NoSuchMethodException, IOException, + IllegalAccessException, InvocationTargetException { + KerberosAuthenticator kerberosAuthenticator = new KerberosAuthenticator(); + FieldUtils.writeField(kerberosAuthenticator, "base64", new Base64(), true); + + Base64 base64 = new Base64(); + + HttpURLConnection conn = Mockito.mock(HttpURLConnection.class); + Mockito.when(conn.getResponseCode()).thenReturn(HttpURLConnection.HTTP_UNAUTHORIZED); + Mockito.when(conn.getHeaderField("www-authenticate")) + .thenReturn(KerberosAuthenticator.NEGOTIATE + + Arrays.toString(base64.encode("foobar".getBytes()))); + + Method method = KerberosAuthenticator.class.getDeclaredMethod("readToken", + HttpURLConnection.class); + method.setAccessible(true); + + method.invoke(kerberosAuthenticator, conn); // expecting this not to throw an exception + } } diff --git a/hadoop-common-project/hadoop-auth/src/test/java/org/apache/hadoop/security/authentication/server/TestAuthenticationFilter.java b/hadoop-common-project/hadoop-auth/src/test/java/org/apache/hadoop/security/authentication/server/TestAuthenticationFilter.java index 4f4a4521b2f0c..b0066abbdd3cf 100644 --- a/hadoop-common-project/hadoop-auth/src/test/java/org/apache/hadoop/security/authentication/server/TestAuthenticationFilter.java +++ b/hadoop-common-project/hadoop-auth/src/test/java/org/apache/hadoop/security/authentication/server/TestAuthenticationFilter.java @@ -574,6 +574,44 @@ public Object answer(InvocationOnMock invocation) throws Throwable { } } + @Test + public void testDoFilterNotAuthenticatedLowerCase() throws Exception { + AuthenticationFilter filter = new AuthenticationFilter(); + try { + FilterConfig config = Mockito.mock(FilterConfig.class); + Mockito.when(config.getInitParameter("management.operation.return")). + thenReturn("true"); + Mockito.when(config.getInitParameter(AuthenticationFilter.AUTH_TYPE)).thenReturn( + DummyAuthenticationHandler.class.getName()); + Mockito.when(config.getInitParameterNames()).thenReturn( + new Vector<>( + Arrays.asList(AuthenticationFilter.AUTH_TYPE, + "management.operation.return")).elements()); + getMockedServletContextWithStringSigner(config); + filter.init(config); + + HttpServletRequest request = Mockito.mock(HttpServletRequest.class); + Mockito.when(request.getRequestURL()).thenReturn(new StringBuffer("http://foo:8080/bar")); + + HttpServletResponse response = Mockito.mock(HttpServletResponse.class); + + FilterChain chain = Mockito.mock(FilterChain.class); + + Mockito.doAnswer((Answer) invocation -> { + Assert.fail(); + return null; + }).when(chain).doFilter(any(), any()); + + Mockito.when(response.containsHeader("www-authenticate")).thenReturn(true); + filter.doFilter(request, response, chain); + + Mockito.verify(response).sendError( + HttpServletResponse.SC_UNAUTHORIZED, "Authentication required"); + } finally { + filter.destroy(); + } + } + private void _testDoFilterAuthentication(boolean withDomainPath, boolean invalidToken, boolean expired) throws Exception { diff --git a/hadoop-common-project/hadoop-auth/src/test/java/org/apache/hadoop/security/authentication/server/TestJWTRedirectAuthenticationHandler.java b/hadoop-common-project/hadoop-auth/src/test/java/org/apache/hadoop/security/authentication/server/TestJWTRedirectAuthenticationHandler.java index 5a2db9ba6fd97..7587bca2012d0 100644 --- a/hadoop-common-project/hadoop-auth/src/test/java/org/apache/hadoop/security/authentication/server/TestJWTRedirectAuthenticationHandler.java +++ b/hadoop-common-project/hadoop-auth/src/test/java/org/apache/hadoop/security/authentication/server/TestJWTRedirectAuthenticationHandler.java @@ -25,7 +25,6 @@ import java.util.List; import java.util.ArrayList; import java.util.Properties; -import java.util.Vector; import java.util.Date; import javax.servlet.ServletException; diff --git a/hadoop-common-project/hadoop-auth/src/test/java/org/apache/hadoop/security/authentication/server/TestPseudoAuthenticationHandler.java b/hadoop-common-project/hadoop-auth/src/test/java/org/apache/hadoop/security/authentication/server/TestPseudoAuthenticationHandler.java index b52915d9cc4ac..ac6221d642a6f 100644 --- a/hadoop-common-project/hadoop-auth/src/test/java/org/apache/hadoop/security/authentication/server/TestPseudoAuthenticationHandler.java +++ b/hadoop-common-project/hadoop-auth/src/test/java/org/apache/hadoop/security/authentication/server/TestPseudoAuthenticationHandler.java @@ -13,7 +13,6 @@ */ package org.apache.hadoop.security.authentication.server; -import org.apache.hadoop.security.authentication.client.AuthenticationException; import org.apache.hadoop.security.authentication.client.PseudoAuthenticator; import org.junit.Assert; import org.junit.Test; diff --git a/hadoop-common-project/hadoop-auth/src/test/java/org/apache/hadoop/security/authentication/util/StringSignerSecretProvider.java b/hadoop-common-project/hadoop-auth/src/test/java/org/apache/hadoop/security/authentication/util/StringSignerSecretProvider.java index 5582c923ae0e7..ed6b1aeccc7c2 100644 --- a/hadoop-common-project/hadoop-auth/src/test/java/org/apache/hadoop/security/authentication/util/StringSignerSecretProvider.java +++ b/hadoop-common-project/hadoop-auth/src/test/java/org/apache/hadoop/security/authentication/util/StringSignerSecretProvider.java @@ -18,7 +18,6 @@ import javax.servlet.ServletContext; import org.apache.hadoop.classification.VisibleForTesting; -import org.apache.hadoop.classification.InterfaceAudience; import org.apache.hadoop.classification.InterfaceStability; import org.apache.hadoop.security.authentication.server.AuthenticationFilter; diff --git a/hadoop-common-project/hadoop-common/dev-support/jdiff/Apache_Hadoop_Common_2.10.2.xml b/hadoop-common-project/hadoop-common/dev-support/jdiff/Apache_Hadoop_Common_2.10.2.xml new file mode 100644 index 0000000000000..7ddea60d5f724 --- /dev/null +++ b/hadoop-common-project/hadoop-common/dev-support/jdiff/Apache_Hadoop_Common_2.10.2.xml @@ -0,0 +1,41055 @@ + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + UnsupportedOperationException + + If a key is deprecated in favor of multiple keys, they are all treated as + aliases of each other, and setting any one of them resets all the others + to the new value. + + If you have multiple deprecation entries to add, it is more efficient to + use #addDeprecations(DeprecationDelta[] deltas) instead. + + @param key + @param newKeys + @param customMessage + @deprecated use {@link #addDeprecation(String key, String newKey, + String customMessage)} instead]]> + + + + + + + + UnsupportedOperationException + + If you have multiple deprecation entries to add, it is more efficient to + use #addDeprecations(DeprecationDelta[] deltas) instead. + + @param key + @param newKey + @param customMessage]]> + + + + + + + UnsupportedOperationException + + If a key is deprecated in favor of multiple keys, they are all treated as + aliases of each other, and setting any one of them resets all the others + to the new value. + + If you have multiple deprecation entries to add, it is more efficient to + use #addDeprecations(DeprecationDelta[] deltas) instead. + + @param key Key that is to be deprecated + @param newKeys list of keys that take up the values of deprecated key + @deprecated use {@link #addDeprecation(String key, String newKey)} instead]]> + + + + + + + UnsupportedOperationException + + If you have multiple deprecation entries to add, it is more efficient to + use #addDeprecations(DeprecationDelta[] deltas) instead. + + @param key Key that is to be deprecated + @param newKey key that takes up the value of deprecated key]]> + + + + + + key is deprecated. + + @param key the parameter which is to be checked for deprecation + @return true if the key is deprecated and + false otherwise.]]> + + + + + + + + + + + + + + + + + + + + + + + + + + + + final. + + @param name resource to be added, the classpath is examined for a file + with that name.]]> + + + + + + + + + + final. + + @param url url of the resource to be added, the local filesystem is + examined directly to find the resource, without referring to + the classpath.]]> + + + + + + + + + + final. + + @param file file-path of resource to be added, the local filesystem is + examined directly to find the resource, without referring to + the classpath.]]> + + + + + + + + + + final. + + WARNING: The contents of the InputStream will be cached, by this method. + So use this sparingly because it does increase the memory consumption. + + @param in InputStream to deserialize the object from. In will be read from + when a get or set is called next. After it is read the stream will be + closed.]]> + + + + + + + + + + + final. + + @param in InputStream to deserialize the object from. + @param name the name of the resource because InputStream.toString is not + very descriptive some times.]]> + + + + + + + + + + + final. + + @param conf Configuration object from which to load properties]]> + + + + + + + + + + + name property, null if + no such property exists. If the key is deprecated, it returns the value of + the first key which replaces the deprecated key and is not null. + + Values are processed for variable expansion + before being returned. + + @param name the property name, will be trimmed before get value. + @return the value of the name or its replacing property, + or null if no such property exists.]]> + + + + + + + + + + + + + + + name property, but only for + names which have no valid value, usually non-existent or commented + out in XML. + + @param name the property name + @return true if the property name exists without value]]> + + + + + + name property as a trimmed String, + null if no such property exists. + If the key is deprecated, it returns the value of + the first key which replaces the deprecated key and is not null + + Values are processed for variable expansion + before being returned. + + @param name the property name. + @return the value of the name or its replacing property, + or null if no such property exists.]]> + + + + + + + name property as a trimmed String, + defaultValue if no such property exists. + See @{Configuration#getTrimmed} for more details. + + @param name the property name. + @param defaultValue the property default value. + @return the value of the name or defaultValue + if it is not set.]]> + + + + + + name property, without doing + variable expansion.If the key is + deprecated, it returns the value of the first key which replaces + the deprecated key and is not null. + + @param name the property name. + @return the value of the name property or + its replacing property and null if no such property exists.]]> + + + + + + + value of the name property. If + name is deprecated or there is a deprecated name associated to it, + it sets the value to both names. Name will be trimmed before put into + configuration. + + @param name property name. + @param value property value.]]> + + + + + + + + value of the name property. If + name is deprecated, it also sets the value to + the keys that replace the deprecated key. Name will be trimmed before put + into configuration. + + @param name property name. + @param value property value. + @param source the place that this configuration value came from + (For debugging). + @throws IllegalArgumentException when the value or name is null.]]> + + + + + + + + + + + + + + + + + + + + name. If the key is deprecated, + it returns the value of the first key which replaces the deprecated key + and is not null. + If no such property exists, + then defaultValue is returned. + + @param name property name, will be trimmed before get value. + @param defaultValue default value. + @return property value, or defaultValue if the property + doesn't exist.]]> + + + + + + + name property as an int. + + If no such property exists, the provided default value is returned, + or if the specified value is not a valid int, + then an error is thrown. + + @param name property name. + @param defaultValue default value. + @throws NumberFormatException when the value is invalid + @return property value as an int, + or defaultValue.]]> + + + + + + name property as a set of comma-delimited + int values. + + If no such property exists, an empty array is returned. + + @param name property name + @return property value interpreted as an array of comma-delimited + int values]]> + + + + + + + name property to an int. + + @param name property name. + @param value int value of the property.]]> + + + + + + + name property as a long. + If no such property exists, the provided default value is returned, + or if the specified value is not a valid long, + then an error is thrown. + + @param name property name. + @param defaultValue default value. + @throws NumberFormatException when the value is invalid + @return property value as a long, + or defaultValue.]]> + + + + + + + name property as a long or + human readable format. If no such property exists, the provided default + value is returned, or if the specified value is not a valid + long or human readable format, then an error is thrown. You + can use the following suffix (case insensitive): k(kilo), m(mega), g(giga), + t(tera), p(peta), e(exa) + + @param name property name. + @param defaultValue default value. + @throws NumberFormatException when the value is invalid + @return property value as a long, + or defaultValue.]]> + + + + + + + name property to a long. + + @param name property name. + @param value long value of the property.]]> + + + + + + + name property as a float. + If no such property exists, the provided default value is returned, + or if the specified value is not a valid float, + then an error is thrown. + + @param name property name. + @param defaultValue default value. + @throws NumberFormatException when the value is invalid + @return property value as a float, + or defaultValue.]]> + + + + + + + name property to a float. + + @param name property name. + @param value property value.]]> + + + + + + + name property as a double. + If no such property exists, the provided default value is returned, + or if the specified value is not a valid double, + then an error is thrown. + + @param name property name. + @param defaultValue default value. + @throws NumberFormatException when the value is invalid + @return property value as a double, + or defaultValue.]]> + + + + + + + name property to a double. + + @param name property name. + @param value property value.]]> + + + + + + + name property as a boolean. + If no such property is specified, or if the specified value is not a valid + boolean, then defaultValue is returned. + + @param name property name. + @param defaultValue default value. + @return property value as a boolean, + or defaultValue.]]> + + + + + + + name property to a boolean. + + @param name property name. + @param value boolean value of the property.]]> + + + + + + + + + + + + + + name property to the given type. This + is equivalent to set(<name>, value.toString()). + @param name property name + @param value new value]]> + + + + + + + + + + + + + + + name to the given time duration. This + is equivalent to set(<name>, value + <time suffix>). + @param name Property name + @param value Time duration + @param unit Unit of time]]> + + + + + + + + + + + + + + + + + + + + + + + + + + + + name property as a Pattern. + If no such property is specified, or if the specified value is not a valid + Pattern, then DefaultValue is returned. + Note that the returned value is NOT trimmed by this method. + + @param name property name + @param defaultValue default value + @return property value as a compiled Pattern, or defaultValue]]> + + + + + + + Pattern. + If the pattern is passed as null, sets the empty pattern which results in + further calls to getPattern(...) returning the default value. + + @param name property name + @param pattern new value]]> + + + + + + + + + + + + + + + + + + + name property as + a collection of Strings. + If no such property is specified then empty collection is returned. +

+ This is an optimized version of {@link #getStrings(String)} + + @param name property name. + @return property value as a collection of Strings.]]> + + + + + + name property as + an array of Strings. + If no such property is specified then null is returned. + + @param name property name. + @return property value as an array of Strings, + or null.]]> + + + + + + + name property as + an array of Strings. + If no such property is specified then default value is returned. + + @param name property name. + @param defaultValue The default value + @return property value as an array of Strings, + or default value.]]> + + + + + + name property as + a collection of Strings, trimmed of the leading and trailing whitespace. + If no such property is specified then empty Collection is returned. + + @param name property name. + @return property value as a collection of Strings, or empty Collection]]> + + + + + + name property as + an array of Strings, trimmed of the leading and trailing whitespace. + If no such property is specified then an empty array is returned. + + @param name property name. + @return property value as an array of trimmed Strings, + or empty array.]]> + + + + + + + name property as + an array of Strings, trimmed of the leading and trailing whitespace. + If no such property is specified then default value is returned. + + @param name property name. + @param defaultValue The default value + @return property value as an array of trimmed Strings, + or default value.]]> + + + + + + + name property as + as comma delimited values. + + @param name property name. + @param values The values]]> + + + + + + + + + + + + + + + + + + + + + + + + + + + + + hostProperty as a + InetSocketAddress. If hostProperty is + null, addressProperty will be used. This + is useful for cases where we want to differentiate between host + bind address and address clients should use to establish connection. + + @param hostProperty bind host property name. + @param addressProperty address property name. + @param defaultAddressValue the default value + @param defaultPort the default port + @return InetSocketAddress]]> + + + + + + + + name property as a + InetSocketAddress. + @param name property name. + @param defaultAddress the default value + @param defaultPort the default port + @return InetSocketAddress]]> + + + + + + + name property as + a host:port.]]> + + + + + + + + + name property as a host:port. The wildcard + address is replaced with the local host's address. If the host and address + properties are configured the host component of the address will be combined + with the port component of the addr to generate the address. This is to allow + optional control over which host name is used in multi-home bind-host + cases where a host can have multiple names + @param hostProperty the bind-host configuration name + @param addressProperty the service address configuration name + @param defaultAddressValue the service default address configuration value + @param addr InetSocketAddress of the service listener + @return InetSocketAddress for clients to connect]]> + + + + + + + name property as a host:port. The wildcard + address is replaced with the local host's address. + @param name property name. + @param addr InetSocketAddress of a listener to store in the given property + @return InetSocketAddress for clients to connect]]> + + + + + + + + + + + + + + + + + + + + name property + as an array of Class. + The value of the property specifies a list of comma separated class names. + If no such property is specified, then defaultValue is + returned. + + @param name the property name. + @param defaultValue default value. + @return property value as a Class[], + or defaultValue.]]> + + + + + + + name property as a Class. + If no such property is specified, then defaultValue is + returned. + + @param name the class name. + @param defaultValue default value. + @return property value as a Class, + or defaultValue.]]> + + + + + + + + name property as a Class + implementing the interface specified by xface. + + If no such property is specified, then defaultValue is + returned. + + An exception is thrown if the returned class does not implement the named + interface. + + @param name the class name. + @param defaultValue default value. + @param xface the interface implemented by the named class. + @return property value as a Class, + or defaultValue.]]> + + + + + + + name property as a List + of objects implementing the interface specified by xface. + + An exception is thrown if any of the classes does not exist, or if it does + not implement the named interface. + + @param name the property name. + @param xface the interface implemented by the classes named by + name. + @return a List of objects implementing xface.]]> + + + + + + + + name property to the name of a + theClass implementing the given interface xface. + + An exception is thrown if theClass does not implement the + interface xface. + + @param name property name. + @param theClass property value. + @param xface the interface implemented by the named class.]]> + + + + + + + + dirsProp with + the given path. If dirsProp contains multiple directories, + then one is chosen based on path's hash code. If the selected + directory does not exist, an attempt is made to create it. + + @param dirsProp directory in which to locate the file. + @param path file-path. + @return local file under the directory with the given path.]]> + + + + + + + + dirsProp with + the given path. If dirsProp contains multiple directories, + then one is chosen based on path's hash code. If the selected + directory does not exist, an attempt is made to create it. + + @param dirsProp directory in which to locate the file. + @param path file-path. + @return local file under the directory with the given path.]]> + + + + + + + + + + + + name. + + @param name configuration resource name. + @return an input stream attached to the resource.]]> + + + + + + name. + + @param name configuration resource name. + @return a reader attached to the resource.]]> + + + + + + + + + + + + + + + + + + + + + + String + key-value pairs in the configuration. + + @return an iterator over the entries.]]> + + + + + + + + + + + + + + + + + + + + + + + + + + + When property name is not empty and the property exists in the + configuration, this method writes the property and its attributes + to the {@link Writer}. + +

+ +

  • + When property name is null or empty, this method writes all the + configuration properties and their attributes to the {@link Writer}. +
  • +

    + +

  • + When property name is not empty but the property doesn't exist in + the configuration, this method throws an {@link IllegalArgumentException}. +
  • +

    + @param out the writer to write to.]]> + + + + + + + + + + When propertyName is not empty, and the property exists + in the configuration, the format of the output would be, +

    +  {
    +    "property": {
    +      "key" : "key1",
    +      "value" : "value1",
    +      "isFinal" : "key1.isFinal",
    +      "resource" : "key1.resource"
    +    }
    +  }
    +  
    + + +
  • + When propertyName is null or empty, it behaves same as + {@link #dumpConfiguration(Configuration, Writer)}, the + output would be, +
    +  { "properties" :
    +      [ { key : "key1",
    +          value : "value1",
    +          isFinal : "key1.isFinal",
    +          resource : "key1.resource" },
    +        { key : "key2",
    +          value : "value2",
    +          isFinal : "ke2.isFinal",
    +          resource : "key2.resource" }
    +       ]
    +   }
    +  
    +
  • + +
  • + When propertyName is not empty, and the property is not + found in the configuration, this method will throw an + {@link IllegalArgumentException}. +
  • +

    + @param config the configuration + @param propertyName property name + @param out the Writer to write to + @throws IOException + @throws IllegalArgumentException when property name is not + empty and the property is not found in configuration]]> + + + + + + + + + { "properties" : + [ { key : "key1", + value : "value1", + isFinal : "key1.isFinal", + resource : "key1.resource" }, + { key : "key2", + value : "value2", + isFinal : "ke2.isFinal", + resource : "key2.resource" } + ] + } + + + It does not output the properties of the configuration object which + is loaded from an input stream. +

    + + @param config the configuration + @param out the Writer to write to + @throws IOException]]> + + + + + + + + + + + + + + + + + + + true to set quiet-mode on, false + to turn it off.]]> + + + + + + + + + + + + + + + + + + + + + with matching keys]]> + + + + + + + + + + + + Resources + +

    Configurations are specified by resources. A resource contains a set of + name/value pairs as XML data. Each resource is named by either a + String or by a {@link Path}. If named by a String, + then the classpath is examined for a file with that name. If named by a + Path, then the local filesystem is examined directly, without + referring to the classpath. + +

    Unless explicitly turned off, Hadoop by default specifies two + resources, loaded in-order from the classpath:

      +
    1. + + core-default.xml: Read-only defaults for hadoop.
    2. +
    3. core-site.xml: Site-specific configuration for a given hadoop + installation.
    4. +
    + Applications may add additional resources, which are loaded + subsequent to these resources in the order they are added. + +

    Final Parameters

    + +

    Configuration parameters may be declared final. + Once a resource declares a value final, no subsequently-loaded + resource can alter that value. + For example, one might define a final parameter with: +

    +  <property>
    +    <name>dfs.hosts.include</name>
    +    <value>/etc/hadoop/conf/hosts.include</value>
    +    <final>true</final>
    +  </property>
    + + Administrators typically define parameters as final in + core-site.xml for values that user applications may not alter. + +

    Variable Expansion

    + +

    Value strings are first processed for variable expansion. The + available properties are:

      +
    1. Other properties defined in this Configuration; and, if a name is + undefined here,
    2. +
    3. Properties in {@link System#getProperties()}.
    4. +
    + +

    For example, if a configuration resource contains the following property + definitions: +

    +  <property>
    +    <name>basedir</name>
    +    <value>/user/${user.name}</value>
    +  </property>
    +  
    +  <property>
    +    <name>tempdir</name>
    +    <value>${basedir}/tmp</value>
    +  </property>
    + + When conf.get("tempdir") is called, then ${basedir} + will be resolved to another property in this Configuration, while + ${user.name} would then ordinarily be resolved to the value + of the System property with that name. + By default, warnings will be given to any deprecated configuration + parameters and these are suppressible by configuring + log4j.logger.org.apache.hadoop.conf.Configuration.deprecation in + log4j.properties file.]]> +
    +
    + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + +
    + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + This implementation generates the key material and calls the + {@link #createKey(String, byte[], Options)} method. + + @param name the base name of the key + @param options the options for the new key. + @return the version name of the first version of the key. + @throws IOException + @throws NoSuchAlgorithmException]]> + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + This implementation generates the key material and calls the + {@link #rollNewVersion(String, byte[])} method. + + @param name the basename of the key + @return the name of the new version of the key + @throws IOException]]> + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + KeyProvider implementations must be thread safe.]]> + + + + + + + + + + + + + + + + + + + + + + NULL if + a provider for the specified URI scheme could not be found. + @throws IOException thrown if the provider failed to initialize.]]> + + + + + + + + + + + + + + + + + + + + + uri has syntax error]]> + + + + + + + + + + + + + + + + uri is + not found]]> + + + + + + + + + + + + + + + + + + + + + + + uri + determines a configuration property name, + fs.AbstractFileSystem.scheme.impl whose value names the + AbstractFileSystem class. + + The entire URI and conf is passed to the AbstractFileSystem factory method. + + @param uri for the file system to be created. + @param conf which is passed to the file system impl. + + @return file system for the given URI. + + @throws UnsupportedFileSystemException if the file system for + uri is not supported.]]> + + + + + + + + + + + + default port;]]> + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + In some FileSystem implementations such as HDFS metadata + synchronization is essential to guarantee consistency of read requests + particularly in HA setting. + @throws IOException + @throws UnsupportedOperationException]]> + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + describing modifications + @throws IOException if an ACL could not be modified]]> + + + + + + + + describing entries to remove + @throws IOException if an ACL could not be modified]]> + + + + + + + + + + + + + + + + + + + + + + describing modifications, must include entries + for user, group, and others for compatibility with permission bits. + @throws IOException if an ACL could not be modified]]> + + + + + + + which returns each AclStatus + @throws IOException if an ACL could not be read]]> + + + + + + + + + + Refer to the HDFS extended attributes user documentation for details. + + @param path Path to modify + @param name xattr name. + @param value xattr value. + @throws IOException]]> + + + + + + + + + + + Refer to the HDFS extended attributes user documentation for details. + + @param path Path to modify + @param name xattr name. + @param value xattr value. + @param flag xattr set flag + @throws IOException]]> + + + + + + + + + Refer to the HDFS extended attributes user documentation for details. + + @param path Path to get extended attribute + @param name xattr name. + @return byte[] xattr value. + @throws IOException]]> + + + + + + + + Refer to the HDFS extended attributes user documentation for details. + + @param path Path to get extended attributes + @return Map describing the XAttrs of the file or directory + @throws IOException]]> + + + + + + + + + Refer to the HDFS extended attributes user documentation for details. + + @param path Path to get extended attributes + @param names XAttr names. + @return Map describing the XAttrs of the file or directory + @throws IOException]]> + + + + + + + + Refer to the HDFS extended attributes user documentation for details. + + @param path Path to get extended attributes + @return Map describing the XAttrs of the file or directory + @throws IOException]]> + + + + + + + + + Refer to the HDFS extended attributes user documentation for details. + + @param path Path to remove extended attribute + @param name xattr name + @throws IOException]]> + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + After a successful call, {@code buf.position()} will be advanced by the + number of bytes read and {@code buf.limit()} will be unchanged. +

    + In the case of an exception, the state of the buffer (the contents of the + buffer, the {@code buf.position()}, the {@code buf.limit()}, etc.) is + undefined, and callers should be prepared to recover from this eventuality. +

    + Callers should use {@link StreamCapabilities#hasCapability(String)} with + {@link StreamCapabilities#PREADBYTEBUFFER} to check if the underlying + stream supports this interface, otherwise they might get a + {@link UnsupportedOperationException}. +

    + Implementations should treat 0-length requests as legitimate, and must not + signal an error upon their receipt. + + @param position position within file + @param buf the ByteBuffer to receive the results of the read operation. + @return the number of bytes read, possibly zero, or -1 if reached + end-of-stream + @throws IOException if there is some error performing the read]]> + + + + + + + + + + + + + + + After a successful call, buf.position() will be advanced by the number + of bytes read and buf.limit() should be unchanged. +

    + In the case of an exception, the values of buf.position() and buf.limit() + are undefined, and callers should be prepared to recover from this + eventuality. +

    + Many implementations will throw {@link UnsupportedOperationException}, so + callers that are not confident in support for this method from the + underlying filesystem should be prepared to handle that exception. +

    + Implementations should treat 0-length requests as legitimate, and must not + signal an error upon their receipt. + + @param buf + the ByteBuffer to receive the results of the read operation. + @return the number of bytes read, possibly zero, or -1 if + reach end-of-stream + @throws IOException + if there is some error performing the read]]> + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + setReplication of FileSystem + @param src file name + @param replication new replication + @throws IOException + @return true if successful; + false if file does not exist or is a directory]]> + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + core-default.xml]]> + + + + + + + + + + + core-default.xml]]> + + + + + + + + + + + core-default.xml]]> + + + + + + + + + + + core-default.xml]]> + + + + + + + + + + + core-default.xml]]> + + + + + + + + + + + core-default.xml]]> + + + + + + core-default.xml]]> + + + + + + + + + + + core-default.xml]]> + + + + + + + + + + + core-default.xml]]> + + + + + + core-default.xml]]> + + + + + + core-default.xml]]> + + + + + + core-default.xml]]> + + + + + + + + core-default.xml]]> + + + + + + + + + + + + + + + + + + + + + core-default.xml]]> + + + + + + + + + + + core-default.xml]]> + + + + + + core-default.xml]]> + + + + + + core-default.xml]]> + + + + + + core-default.xml]]> + + + + + + + + + + + core-default.xml]]> + + + + + + + + + + + core-default.xml]]> + + + + + + + + + + + core-default.xml]]> + + + + + + + + + + + + + + + + core-default.xml]]> + + + + + + + + + + + core-default.xml]]> + + + + + + + + + + + core-default.xml]]> + + + + + + + + + + + core-default.xml]]> + + + + + + + + + + + core-default.xml]]> + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + core-default.xml]]> + + + + + + + + + + + core-default.xml]]> + + + + + + + + + + + core-default.xml]]> + + + + + + core-default.xml]]> + + + + + + + + + + + core-default.xml]]> + + + + + + + + + + + core-default.xml]]> + + + + + + + + + + + + + + + + + + + + + + + core-default.xml]]> + + + + + + + + + + + core-default.xml]]> + + + + + + + + + + + core-default.xml]]> + + + + + + + + + + + core-default.xml]]> + + + + + + + + + + + core-default.xml]]> + + + + + + + + + + + core-default.xml]]> + + + + + + + + + + + + + + + + + + + + + core-default.xml]]> + + + + + + + + + + + core-default.xml]]> + + + + + + + + + + + core-default.xml]]> + + + + + + + + + + + core-default.xml]]> + + + + + + + + + + + core-default.xml]]> + + + + + + + + + + + + + + + + + + core-default.xml]]> + + + + + + + + core-default.xml]]> + + + + + + core-default.xml]]> + + + + + + + + + + + core-default.xml]]> + + + + + + core-default.xml]]> + + + + + + core-default.xml]]> + + + + + + core-default.xml]]> + + + + + + core-default.xml]]> + + + + + + core-default.xml]]> + + + + + + + + core-default.xml]]> + + + + + + core-default.xml]]> + + + + + + core-default.xml]]> + + + + + + core-default.xml]]> + + + + + + core-default.xml]]> + + + + + + + + + + + core-default.xml]]> + + + + + + + + + + + core-default.xml]]> + + + + + + core-default.xml]]> + + + + + + core-default.xml]]> + + + + + + core-default.xml]]> + + + + + + core-default.xml]]> + + + + + + core-default.xml]]> + + + + + + core-default.xml]]> + + + + + + core-default.xml]]> + + + + + + + + + + + core-default.xml]]> + + + + + + + + + + + + + + + core-default.xml]]> + + + + + + + + + + + + + + + + + core-default.xml]]> + + + + + + + + core-default.xml]]> + + + + + + core-default.xml]]> + + + + + + core-default.xml]]> + + + + + + + + + + + + + + + + core-default.xml]]> + + + + + + core-default.xml]]> + + + + + + + + + + + core-default.xml]]> + + + + + + + + + + + core-default.xml]]> + + + + + + + + + + + core-default.xml]]> + + + + + + + + + + + core-default.xml]]> + + + + + + + + + + + core-default.xml]]> + + + + + + + + + + + core-default.xml]]> + + + + + + + + + + + + + core-default.xml]]> + + + + + + + + + + + core-default.xml]]> + + + + + + + + + + + core-default.xml]]> + + + + + + + + + + + core-default.xml]]> + + + + + + core-default.xml]]> + + + + + + + + core-default.xml]]> + + + + + + + + core-default.xml]]> + + + + + + + + core-default.xml]]> + + + + + + + + + + + core-default.xml]]> + + + + + + core-default.xml]]> + + + + + + + + core-default.xml]]> + + + + + + core-default.xml]]> + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + EnumSet.of(CreateFlag.CREATE, CreateFlag.APPEND) + +

    + + Use the CreateFlag as follows: +

      +
    1. CREATE - to create a file if it does not exist, + else throw FileAlreadyExists.
    2. +
    3. APPEND - to append to a file if it exists, + else throw FileNotFoundException.
    4. +
    5. OVERWRITE - to truncate a file if it exists, + else throw FileNotFoundException.
    6. +
    7. CREATE|APPEND - to create a file if it does not exist, + else append to an existing file.
    8. +
    9. CREATE|OVERWRITE - to create a file if it does not exist, + else overwrite an existing file.
    10. +
    11. SYNC_BLOCK - to force closed blocks to the disk device. + In addition {@link Syncable#hsync()} should be called after each write, + if true synchronous behavior is required.
    12. +
    13. LAZY_PERSIST - Create the block on transient storage (RAM) if + available.
    14. +
    15. APPEND_NEWBLOCK - Append data to a new block instead of end of the last + partial block.
    16. +
    + + Following combinations are not valid and will result in + {@link HadoopIllegalArgumentException}: +
      +
    1. APPEND|OVERWRITE
    2. +
    3. CREATE|APPEND|OVERWRITE
    4. +
    ]]> +
    + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + absOrFqPath is not supported. + @throws IOException If the file system for absOrFqPath could + not be instantiated.]]> + +
    + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + defaultFsUri is not supported]]> + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + NewWdir can be one of: +
      +
    • relative path: "foo/bar";
    • +
    • absolute without scheme: "/foo/bar"
    • +
    • fully qualified with scheme: "xx://auth/foo/bar"
    • +
    +
    + Illegal WDs: +
      +
    • relative with scheme: "xx:foo/bar"
    • +
    • non existent directory
    • +
    ]]> +
    +
    + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + f does not exist + @throws AccessControlException if access denied + @throws IOException If an IO Error occurred + + Exceptions applicable to file systems accessed over RPC: + @throws RpcClientException If an exception occurred in the RPC client + @throws RpcServerException If an exception occurred in the RPC server + @throws UnexpectedServerException If server implementation throws + undeclared exception to RPC server + + RuntimeExceptions: + @throws InvalidPathException If path f is not valid]]> + + + + + + + + + + + + + + + + + + + + +
  • Progress - to report progress on the operation - default null +
  • Permission - umask is applied against permisssion: default is + FsPermissions:getDefault() + +
  • CreateParent - create missing parent path; default is to not + to create parents +
  • The defaults for the following are SS defaults of the file + server implementing the target path. Not all parameters make sense + for all kinds of file system - eg. localFS ignores Blocksize, + replication, checksum +
      +
    • BufferSize - buffersize used in FSDataOutputStream +
    • Blocksize - block size for file blocks +
    • ReplicationFactor - replication for blocks +
    • ChecksumParam - Checksum parameters. server default is used + if not specified. +
    + + + @return {@link FSDataOutputStream} for created file + + @throws AccessControlException If access is denied + @throws FileAlreadyExistsException If file f already exists + @throws FileNotFoundException If parent of f does not exist + and createParent is false + @throws ParentNotDirectoryException If parent of f is not a + directory. + @throws UnsupportedFileSystemException If file system for f is + not supported + @throws IOException If an I/O error occurred + + Exceptions applicable to file systems accessed over RPC: + @throws RpcClientException If an exception occurred in the RPC client + @throws RpcServerException If an exception occurred in the RPC server + @throws UnexpectedServerException If server implementation throws + undeclared exception to RPC server + + RuntimeExceptions: + @throws InvalidPathException If path f is not valid]]> + + + + + + + + + + + + + + dir already + exists + @throws FileNotFoundException If parent of dir does not exist + and createParent is false + @throws ParentNotDirectoryException If parent of dir is not a + directory + @throws UnsupportedFileSystemException If file system for dir + is not supported + @throws IOException If an I/O error occurred + + Exceptions applicable to file systems accessed over RPC: + @throws RpcClientException If an exception occurred in the RPC client + @throws UnexpectedServerException If server implementation throws + undeclared exception to RPC server + + RuntimeExceptions: + @throws InvalidPathException If path dir is not valid]]> + + + + + + + + + + + f does not exist + @throws UnsupportedFileSystemException If file system for f is + not supported + @throws IOException If an I/O error occurred + + Exceptions applicable to file systems accessed over RPC: + @throws RpcClientException If an exception occurred in the RPC client + @throws RpcServerException If an exception occurred in the RPC server + @throws UnexpectedServerException If server implementation throws + undeclared exception to RPC server + + RuntimeExceptions: + @throws InvalidPathException If path f is invalid]]> + + + + + + + + + + f does not exist + @throws UnsupportedFileSystemException If file system for f + is not supported + @throws IOException If an I/O error occurred + + Exceptions applicable to file systems accessed over RPC: + @throws RpcClientException If an exception occurred in the RPC client + @throws RpcServerException If an exception occurred in the RPC server + @throws UnexpectedServerException If server implementation throws + undeclared exception to RPC server]]> + + + + + + + + + + + f does not exist + @throws UnsupportedFileSystemException If file system for f is + not supported + @throws IOException If an I/O error occurred + + Exceptions applicable to file systems accessed over RPC: + @throws RpcClientException If an exception occurred in the RPC client + @throws RpcServerException If an exception occurred in the RPC server + @throws UnexpectedServerException If server implementation throws + undeclared exception to RPC server]]> + + + + + + + + + + + +
  • Fails if path is a directory. +
  • Fails if path does not exist. +
  • Fails if path is not closed. +
  • Fails if new size is greater than current size. + + @param f The path to the file to be truncated + @param newLength The size the file is to be truncated to + + @return true if the file has been truncated to the desired + newLength and is immediately available to be reused for + write operations such as append, or + false if a background process of adjusting the length of + the last block has been started, and clients should wait for it to + complete before proceeding with further file updates. + + @throws AccessControlException If access is denied + @throws FileNotFoundException If file f does not exist + @throws UnsupportedFileSystemException If file system for f is + not supported + @throws IOException If an I/O error occurred + + Exceptions applicable to file systems accessed over RPC: + @throws RpcClientException If an exception occurred in the RPC client + @throws RpcServerException If an exception occurred in the RPC server + @throws UnexpectedServerException If server implementation throws + undeclared exception to RPC server]]> + + + + + + + + + + f does not exist + @throws IOException If an I/O error occurred + + Exceptions applicable to file systems accessed over RPC: + @throws RpcClientException If an exception occurred in the RPC client + @throws RpcServerException If an exception occurred in the RPC server + @throws UnexpectedServerException If server implementation throws + undeclared exception to RPC server]]> + + + + + + + + + + + + + + +
  • Fails if src is a file and dst is a directory. +
  • Fails if src is a directory and dst is a file. +
  • Fails if the parent of dst does not exist or is a file. + +

    + If OVERWRITE option is not passed as an argument, rename fails if the dst + already exists. +

    + If OVERWRITE option is passed as an argument, rename overwrites the dst if + it is a file or an empty directory. Rename fails if dst is a non-empty + directory. +

    + Note that atomicity of rename is dependent on the file system + implementation. Please refer to the file system documentation for details +

    + + @param src path to be renamed + @param dst new path after rename + + @throws AccessControlException If access is denied + @throws FileAlreadyExistsException If dst already exists and + options has {@link Options.Rename#OVERWRITE} + option false. + @throws FileNotFoundException If src does not exist + @throws ParentNotDirectoryException If parent of dst is not a + directory + @throws UnsupportedFileSystemException If file system for src + and dst is not supported + @throws IOException If an I/O error occurred + + Exceptions applicable to file systems accessed over RPC: + @throws RpcClientException If an exception occurred in the RPC client + @throws RpcServerException If an exception occurred in the RPC server + @throws UnexpectedServerException If server implementation throws + undeclared exception to RPC server]]> + + + + + + + + + + + f does not exist + @throws UnsupportedFileSystemException If file system for f + is not supported + @throws IOException If an I/O error occurred + + Exceptions applicable to file systems accessed over RPC: + @throws RpcClientException If an exception occurred in the RPC client + @throws RpcServerException If an exception occurred in the RPC server + @throws UnexpectedServerException If server implementation throws + undeclared exception to RPC server]]> + + + + + + + + + + + + f does not exist + @throws UnsupportedFileSystemException If file system for f is + not supported + @throws IOException If an I/O error occurred + + Exceptions applicable to file systems accessed over RPC: + @throws RpcClientException If an exception occurred in the RPC client + @throws RpcServerException If an exception occurred in the RPC server + @throws UnexpectedServerException If server implementation throws + undeclared exception to RPC server + + RuntimeExceptions: + @throws HadoopIllegalArgumentException If username or + groupname is invalid.]]> + + + + + + + + + + + + f does not exist + @throws UnsupportedFileSystemException If file system for f is + not supported + @throws IOException If an I/O error occurred + + Exceptions applicable to file systems accessed over RPC: + @throws RpcClientException If an exception occurred in the RPC client + @throws RpcServerException If an exception occurred in the RPC server + @throws UnexpectedServerException If server implementation throws + undeclared exception to RPC server]]> + + + + + + + + + f does not exist + @throws IOException If an I/O error occurred + + Exceptions applicable to file systems accessed over RPC: + @throws RpcClientException If an exception occurred in the RPC client + @throws RpcServerException If an exception occurred in the RPC server + @throws UnexpectedServerException If server implementation throws + undeclared exception to RPC server]]> + + + + + + + + + + + f does not exist + @throws UnsupportedFileSystemException If file system for f is + not supported + @throws IOException If an I/O error occurred + + Exceptions applicable to file systems accessed over RPC: + @throws RpcClientException If an exception occurred in the RPC client + @throws RpcServerException If an exception occurred in the RPC server + @throws UnexpectedServerException If server implementation throws + undeclared exception to RPC server]]> + + + + + + + + + + f does not exist + @throws UnsupportedFileSystemException If file system for f is + not supported + @throws IOException If an I/O error occurred + + Exceptions applicable to file systems accessed over RPC: + @throws RpcClientException If an exception occurred in the RPC client + @throws RpcServerException If an exception occurred in the RPC server + @throws UnexpectedServerException If server implementation throws + undeclared exception to RPC server]]> + + + + + + + + + + + + + + + + + f does not exist + @throws UnsupportedFileSystemException If file system for f is + not supported + @throws IOException If an I/O error occurred]]> + + + + + + + + + + f does not exist + @throws UnsupportedFileSystemException If file system for f is + not supported + @throws IOException If the given path does not refer to a symlink + or an I/O error occurred]]> + + + + + + + + + + f does not exist + @throws UnsupportedFileSystemException If file system for f is + not supported + @throws IOException If an I/O error occurred + + Exceptions applicable to file systems accessed over RPC: + @throws RpcClientException If an exception occurred in the RPC client + @throws RpcServerException If an exception occurred in the RPC server + @throws UnexpectedServerException If server implementation throws + undeclared exception to RPC server]]> + + + + + + + + + + + + + + + Given a path referring to a symlink of form: + + <---X---> + fs://host/A/B/link + <-----Y-----> + + In this path X is the scheme and authority that identify the file system, + and Y is the path leading up to the final path component "link". If Y is + a symlink itself then let Y' be the target of Y and X' be the scheme and + authority of Y'. Symlink targets may: + + 1. Fully qualified URIs + + fs://hostX/A/B/file Resolved according to the target file system. + + 2. Partially qualified URIs (eg scheme but no host) + + fs:///A/B/file Resolved according to the target file system. Eg resolving + a symlink to hdfs:///A results in an exception because + HDFS URIs must be fully qualified, while a symlink to + file:///A will not since Hadoop's local file systems + require partially qualified URIs. + + 3. Relative paths + + path Resolves to [Y'][path]. Eg if Y resolves to hdfs://host/A and path + is "../B/file" then [Y'][path] is hdfs://host/B/file + + 4. Absolute paths + + path Resolves to [X'][path]. Eg if Y resolves hdfs://host/A/B and path + is "/file" then [X][path] is hdfs://host/file + + + @param target the target of the symbolic link + @param link the path to be created that points to target + @param createParent if true then missing parent dirs are created if + false then parent must exist + + + @throws AccessControlException If access is denied + @throws FileAlreadyExistsException If file linkcode> already exists + @throws FileNotFoundException If target does not exist + @throws ParentNotDirectoryException If parent of link is not a + directory. + @throws UnsupportedFileSystemException If file system for + target or link is not supported + @throws IOException If an I/O error occurred]]> + + + + + + + + + + f does not exist + @throws UnsupportedFileSystemException If file system for f is + not supported + @throws IOException If an I/O error occurred + + Exceptions applicable to file systems accessed over RPC: + @throws RpcClientException If an exception occurred in the RPC client + @throws RpcServerException If an exception occurred in the RPC server + @throws UnexpectedServerException If server implementation throws + undeclared exception to RPC server]]> + + + + + + + + + + + + + + + + + f does not exist + @throws UnsupportedFileSystemException If file system for f is + not supported + @throws IOException If an I/O error occurred + + Exceptions applicable to file systems accessed over RPC: + @throws RpcClientException If an exception occurred in the RPC client + @throws RpcServerException If an exception occurred in the RPC server + @throws UnexpectedServerException If server implementation throws + undeclared exception to RPC server]]> + + + + + + + + f is + not supported + @throws IOException If an I/O error occurred + + Exceptions applicable to file systems accessed over RPC: + @throws RpcClientException If an exception occurred in the RPC client + @throws RpcServerException If an exception occurred in the RPC server + @throws UnexpectedServerException If server implementation throws + undeclared exception to RPC server]]> + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + describing modifications + @throws IOException if an ACL could not be modified]]> + + + + + + + + describing entries to remove + @throws IOException if an ACL could not be modified]]> + + + + + + + + + + + + + + + + + + + + + + describing modifications, must include entries + for user, group, and others for compatibility with permission bits. + @throws IOException if an ACL could not be modified]]> + + + + + + + which returns each AclStatus + @throws IOException if an ACL could not be read]]> + + + + + + + + + + Refer to the HDFS extended attributes user documentation for details. + + @param path Path to modify + @param name xattr name. + @param value xattr value. + @throws IOException]]> + + + + + + + + + + + Refer to the HDFS extended attributes user documentation for details. + + @param path Path to modify + @param name xattr name. + @param value xattr value. + @param flag xattr set flag + @throws IOException]]> + + + + + + + + + Refer to the HDFS extended attributes user documentation for details. + + @param path Path to get extended attribute + @param name xattr name. + @return byte[] xattr value. + @throws IOException]]> + + + + + + + + Refer to the HDFS extended attributes user documentation for details. + + @param path Path to get extended attributes + @return Map describing the XAttrs of the file or directory + @throws IOException]]> + + + + + + + + + Refer to the HDFS extended attributes user documentation for details. + + @param path Path to get extended attributes + @param names XAttr names. + @return Map describing the XAttrs of the file or directory + @throws IOException]]> + + + + + + + + + Refer to the HDFS extended attributes user documentation for details. + + @param path Path to remove extended attribute + @param name xattr name + @throws IOException]]> + + + + + + + + Refer to the HDFS extended attributes user documentation for details. + + @param path Path to get extended attributes + @return List of the XAttr names of the file or directory + @throws IOException]]> + + + + + + + Exceptions applicable to file systems accessed over RPC: + @throws RpcClientException If an exception occurred in the RPC client + @throws RpcServerException If an exception occurred in the RPC server + @throws UnexpectedServerException If server implementation throws + undeclared exception to RPC server]]> + + + + + + + + Exceptions applicable to file systems accessed over RPC: + @throws RpcClientException If an exception occurred in the RPC client + @throws RpcServerException If an exception occurred in the RPC server + @throws UnexpectedServerException If server implementation throws + undeclared exception to RPC server]]> + + + + + + + + + Exceptions applicable to file systems accessed over RPC: + @throws RpcClientException If an exception occurred in the RPC client + @throws RpcServerException If an exception occurred in the RPC server + @throws UnexpectedServerException If server implementation throws + undeclared exception to RPC server]]> + + + + + + + + Exceptions applicable to file systems accessed over RPC: + @throws RpcClientException If an exception occurred in the RPC client + @throws RpcServerException If an exception occurred in the RPC server + @throws UnexpectedServerException If server implementation throws + undeclared exception to RPC server]]> + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + *** Path Names *** +

    + + The Hadoop file system supports a URI name space and URI names. + It offers a forest of file systems that can be referenced using fully + qualified URIs. + Two common Hadoop file systems implementations are +

      +
    • the local file system: file:///path +
    • the hdfs file system hdfs://nnAddress:nnPort/path +
    + + While URI names are very flexible, it requires knowing the name or address + of the server. For convenience one often wants to access the default system + in one's environment without knowing its name/address. This has an + additional benefit that it allows one to change one's default fs + (e.g. admin moves application from cluster1 to cluster2). +

    + + To facilitate this, Hadoop supports a notion of a default file system. + The user can set his default file system, although this is + typically set up for you in your environment via your default config. + A default file system implies a default scheme and authority; slash-relative + names (such as /for/bar) are resolved relative to that default FS. + Similarly a user can also have working-directory-relative names (i.e. names + not starting with a slash). While the working directory is generally in the + same default FS, the wd can be in a different FS. +

    + Hence Hadoop path names can be one of: +

      +
    • fully qualified URI: scheme://authority/path +
    • slash relative names: /path relative to the default file system +
    • wd-relative names: path relative to the working dir +
    + Relative paths with scheme (scheme:foo/bar) are illegal. + +

    + ****The Role of the FileContext and configuration defaults**** +

    + The FileContext provides file namespace context for resolving file names; + it also contains the umask for permissions, In that sense it is like the + per-process file-related state in Unix system. + These two properties +

      +
    • default file system i.e your slash) +
    • umask +
    + in general, are obtained from the default configuration file + in your environment, (@see {@link Configuration}). + + No other configuration parameters are obtained from the default config as + far as the file context layer is concerned. All file system instances + (i.e. deployments of file systems) have default properties; we call these + server side (SS) defaults. Operation like create allow one to select many + properties: either pass them in as explicit parameters or use + the SS properties. +

    + The file system related SS defaults are +

      +
    • the home directory (default is "/user/userName") +
    • the initial wd (only for local fs) +
    • replication factor +
    • block size +
    • buffer size +
    • encryptDataTransfer +
    • checksum option. (checksumType and bytesPerChecksum) +
    + +

    + *** Usage Model for the FileContext class *** +

    + Example 1: use the default config read from the $HADOOP_CONFIG/core.xml. + Unspecified values come from core-defaults.xml in the release jar. +

      +
    • myFContext = FileContext.getFileContext(); // uses the default config + // which has your default FS +
    • myFContext.create(path, ...); +
    • myFContext.setWorkingDir(path) +
    • myFContext.open (path, ...); +
    + Example 2: Get a FileContext with a specific URI as the default FS +
      +
    • myFContext = FileContext.getFileContext(URI) +
    • myFContext.create(path, ...); + ... +
    + Example 3: FileContext with local file system as the default +
      +
    • myFContext = FileContext.getLocalFSFileContext() +
    • myFContext.create(path, ...); +
    • ... +
    + Example 4: Use a specific config, ignoring $HADOOP_CONFIG + Generally you should not need use a config unless you are doing +
      +
    • configX = someConfigSomeOnePassedToYou. +
    • myFContext = getFileContext(configX); // configX is not changed, + // is passed down +
    • myFContext.create(path, ...); +
    • ... +
    ]]> + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + This implementation throws an UnsupportedOperationException. + + @return the protocol scheme for this FileSystem. + @throws UnsupportedOperationException if the operation is unsupported + (default).]]> + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + +
  • + If the configuration has the property + {@code "fs.$SCHEME.impl.disable.cache"} set to true, + a new instance will be created, initialized with the supplied URI and + configuration, then returned without being cached. +
  • +
  • + If the there is a cached FS instance matching the same URI, it will + be returned. +
  • +
  • + Otherwise: a new FS instance will be created, initialized with the + configuration and URI, cached and returned to the caller. +
  • + + @throws IOException if the FileSystem cannot be instantiated.]]> +
    +
    + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + if f == null : + result = null + elif f.getLen() <= start: + result = [] + else result = [ locations(FS, b) for b in blocks(FS, p, s, s+l)] + + This call is most helpful with and distributed filesystem + where the hostnames of machines that contain blocks of the given file + can be determined. + + The default implementation returns an array containing one element: +
    + BlockLocation( { "localhost:50010" },  { "localhost" }, 0, file.getLen())
    + 
    > + + @param file FilesStatus to get data from + @param start offset into the given file + @param len length for which to get locations for + @throws IOException IO failure]]> +
    +
    + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + Important: the default implementation is not atomic + @param f path to use for create + @throws IOException IO failure]]> + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + +
  • Fails if src is a file and dst is a directory.
  • +
  • Fails if src is a directory and dst is a file.
  • +
  • Fails if the parent of dst does not exist or is a file.
  • + +

    + If OVERWRITE option is not passed as an argument, rename fails + if the dst already exists. +

    + If OVERWRITE option is passed as an argument, rename overwrites + the dst if it is a file or an empty directory. Rename fails if dst is + a non-empty directory. +

    + Note that atomicity of rename is dependent on the file system + implementation. Please refer to the file system documentation for + details. This default implementation is non atomic. +

    + This method is deprecated since it is a temporary method added to + support the transition from FileSystem to FileContext for user + applications. + + @param src path to be renamed + @param dst new path after rename + @throws FileNotFoundException src path does not exist, or the parent + path of dst does not exist. + @throws FileAlreadyExistsException dest path exists and is a file + @throws ParentNotDirectoryException if the parent path of dest is not + a directory + @throws IOException on failure]]> + + + + + + + + +

  • Fails if path is a directory.
  • +
  • Fails if path does not exist.
  • +
  • Fails if path is not closed.
  • +
  • Fails if new size is greater than current size.
  • + + @param f The path to the file to be truncated + @param newLength The size the file is to be truncated to + + @return true if the file has been truncated to the desired + newLength and is immediately available to be reused for + write operations such as append, or + false if a background process of adjusting the length of + the last block has been started, and clients should wait for it to + complete before proceeding with further file updates. + @throws IOException IO failure + @throws UnsupportedOperationException if the operation is unsupported + (default).]]> +
    +
    + + + + + + + + + + + + + + + + + + + + +
  • Clean shutdown of the JVM cannot be guaranteed.
  • +
  • The time to shut down a FileSystem will depends on the number of + files to delete. For filesystems where the cost of checking + for the existence of a file/directory and the actual delete operation + (for example: object stores) is high, the time to shutdown the JVM can be + significantly extended by over-use of this feature.
  • +
  • Connectivity problems with a remote filesystem may delay shutdown + further, and may cause the files to not be deleted.
  • + + @param f the path to delete. + @return true if deleteOnExit is successful, otherwise false. + @throws IOException IO failure]]> +
    +
    + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + Does not guarantee to return the List of files/directories status in a + sorted order. + @param f given path + @return the statuses of the files/directories in the given patch + @throws FileNotFoundException when the path does not exist + @throws IOException see specific implementation]]> + + + + + + + + + + + + + + + + + Does not guarantee to return the List of files/directories status in a + sorted order. + + @param f + a path name + @param filter + the user-supplied path filter + @return an array of FileStatus objects for the files under the given path + after applying the filter + @throws FileNotFoundException when the path does not exist + @throws IOException see specific implementation]]> + + + + + + + + + Does not guarantee to return the List of files/directories status in a + sorted order. + + @param files + a list of paths + @return a list of statuses for the files under the given paths after + applying the filter default Path filter + @throws FileNotFoundException when the path does not exist + @throws IOException see specific implementation]]> + + + + + + + + + + Does not guarantee to return the List of files/directories status in a + sorted order. + + @param files + a list of paths + @param filter + the user-supplied path filter + @return a list of statuses for the files under the given paths after + applying the filter + @throws FileNotFoundException when the path does not exist + @throws IOException see specific implementation]]> + + + + + + + Return all the files that match filePattern and are not checksum + files. Results are sorted by their names. + +

    + A filename pattern is composed of regular characters and + special pattern matching characters, which are: + +

    +
    +
    +

    +

    ? +
    Matches any single character. + +

    +

    * +
    Matches zero or more characters. + +

    +

    [abc] +
    Matches a single character from character set + {a,b,c}. + +

    +

    [a-b] +
    Matches a single character from the character range + {a...b}. Note that character a must be + lexicographically less than or equal to character b. + +

    +

    [^a] +
    Matches a single character that is not from character set or range + {a}. Note that the ^ character must occur + immediately to the right of the opening bracket. + +

    +

    \c +
    Removes (escapes) any special meaning of character c. + +

    +

    {ab,cd} +
    Matches a string from the string set {ab, cd} + +

    +

    {ab,c{de,fh}} +
    Matches a string from the string set {ab, cde, cfh} + +
    +
    +
    + + @param pathPattern a regular expression specifying a pth pattern + + @return an array of paths that match the path pattern + @throws IOException IO failure]]> +
    +
    + + + + + + + + + + + + + + f does not exist + @throws IOException If an I/O error occurred]]> + + + + + + + + + f does not exist + @throws IOException if any I/O error occurred]]> + + + + + + + + p does not exist + @throws IOException if any I/O error occurred]]> + + + + + + + + + + If the path is a directory, + if recursive is false, returns files in the directory; + if recursive is true, return files in the subtree rooted at the path. + If the path is a file, return the file's status and block locations. + + @param f is the path + @param recursive if the subdirectories need to be traversed recursively + + @return an iterator that traverses statuses of the files + + @throws FileNotFoundException when the path does not exist; + @throws IOException see specific implementation]]> + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + undefined. + @throws IOException IO failure]]> + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + In some FileSystem implementations such as HDFS metadata + synchronization is essential to guarantee consistency of read requests + particularly in HA setting. + @throws IOException + @throws UnsupportedOperationException]]> + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + describing modifications + @throws IOException if an ACL could not be modified + @throws UnsupportedOperationException if the operation is unsupported + (default outcome).]]> + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + Refer to the HDFS extended attributes user documentation for details. + + @param path Path to modify + @param name xattr name. + @param value xattr value. + @throws IOException IO failure + @throws UnsupportedOperationException if the operation is unsupported + (default outcome).]]> + + + + + + + + + + + Refer to the HDFS extended attributes user documentation for details. + + @param path Path to modify + @param name xattr name. + @param value xattr value. + @param flag xattr set flag + @throws IOException IO failure + @throws UnsupportedOperationException if the operation is unsupported + (default outcome).]]> + + + + + + + + + Refer to the HDFS extended attributes user documentation for details. + + @param path Path to get extended attribute + @param name xattr name. + @return byte[] xattr value. + @throws IOException IO failure + @throws UnsupportedOperationException if the operation is unsupported + (default outcome).]]> + + + + + + + + Refer to the HDFS extended attributes user documentation for details. + + @param path Path to get extended attributes + @return Map describing the XAttrs of the file or directory + @throws IOException IO failure + @throws UnsupportedOperationException if the operation is unsupported + (default outcome).]]> + + + + + + + + + Refer to the HDFS extended attributes user documentation for details. + + @param path Path to get extended attributes + @param names XAttr names. + @return Map describing the XAttrs of the file or directory + @throws IOException IO failure + @throws UnsupportedOperationException if the operation is unsupported + (default outcome).]]> + + + + + + + + Refer to the HDFS extended attributes user documentation for details. + + @param path Path to get extended attributes + @return List of the XAttr names of the file or directory + @throws IOException IO failure + @throws UnsupportedOperationException if the operation is unsupported + (default outcome).]]> + + + + + + + + + Refer to the HDFS extended attributes user documentation for details. + + @param path Path to remove extended attribute + @param name xattr name + @throws IOException IO failure + @throws UnsupportedOperationException if the operation is unsupported + (default outcome).]]> + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + This is a default method which is intended to be overridden by + subclasses. The default implementation returns an empty storage statistics + object.

    + + @return The StorageStatistics for this FileSystem instance. + Will never be null.]]> +
    +
    + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + All user code that may potentially use the Hadoop Distributed + File System should be written to use a FileSystem object or its + successor, {@link FileContext}. + +

    + The local implementation is {@link LocalFileSystem} and distributed + implementation is DistributedFileSystem. There are other implementations + for object stores and (outside the Apache Hadoop codebase), + third party filesystems. +

    + Notes +

      +
    1. The behaviour of the filesystem is + + specified in the Hadoop documentation. + However, the normative specification of the behavior of this class is + actually HDFS: if HDFS does not behave the way these Javadocs or + the specification in the Hadoop documentations define, assume that + the documentation is incorrect. +
    2. +
    3. The term {@code FileSystem} refers to an instance of this class.
    4. +
    5. The acronym "FS" is used as an abbreviation of FileSystem.
    6. +
    7. The term {@code filesystem} refers to the distributed/local filesystem + itself, rather than the class used to interact with it.
    8. +
    9. The term "file" refers to a file in the remote filesystem, + rather than instances of {@code java.io.File}.
    10. +
    ]]> +
    + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + caller's environment variables to use + for expansion + @return String[] with absolute path to new jar in position 0 and + unexpanded wild card entry path in position 1 + @throws IOException if there is an I/O error while writing the jar file]]> + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + FilterFileSystem contains + some other file system, which it uses as + its basic file system, possibly transforming + the data along the way or providing additional + functionality. The class FilterFileSystem + itself simply overrides all methods of + FileSystem with versions that + pass all requests to the contained file + system. Subclasses of FilterFileSystem + may further override some of these methods + and may also provide additional methods + and fields.]]> + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + -1 + if there is no more data because the end of the stream has been + reached]]> + + + + + + + + + + length bytes have been read. + + @param position position in the input stream to seek + @param buffer buffer into which data is read + @param offset offset into the buffer in which data is written + @param length the number of bytes to read + @throws IOException IO problems + @throws EOFException If the end of stream is reached while reading. + If an exception is thrown an undetermined number + of bytes in the buffer may have been written.]]> + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + path is invalid]]> + + + + + + + + + + + + + + + + + + + + + + + @return file]]> + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + and the scheme is null, and the authority + is null. + + @return whether the path is absolute and the URI has no scheme nor + authority parts]]> + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + true if and only if pathname + should be included]]> + + + + + + + + + + + + + + Warning: Not all filesystems satisfy the thread-safety requirement. + @param position position within file + @param buffer destination buffer + @param offset offset in the buffer + @param length number of bytes to read + @return actual number of bytes read; -1 means "none" + @throws IOException IO problems.]]> + + + + + + + + + + Warning: Not all filesystems satisfy the thread-safety requirement. + @param position position within file + @param buffer destination buffer + @param offset offset in the buffer + @param length number of bytes to read + @throws IOException IO problems. + @throws EOFException the end of the data was reached before + the read operation completed]]> + + + + + + + + Warning: Not all filesystems satisfy the thread-safety requirement. + @param position position within file + @param buffer destination buffer + @throws IOException IO problems. + @throws EOFException the end of the data was reached before + the read operation completed]]> + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + <----15----> <----15----> <----15----> <-------18-------> + QUOTA REMAINING_QUATA SPACE_QUOTA SPACE_QUOTA_REM FILE_NAME]]> + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + XAttr is byte[], this class is to + covert byte[] to some kind of string representation or convert back. + String representation is convenient for display and input. For example + display in screen as shell response and json response, input as http + or shell parameter.]]> + + + + + + + + + + + + + + + + + + + + + +
    + + + + + + + + + + + + + + + + + + + + + + + + + @return ftp]]> + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + A {@link FileSystem} backed by an FTP client provided by Apache Commons Net. +

    ]]> +
    +
    + +
    + + + + + + + + + + + + + + + + + + (cause==null ? null : cause.toString()) (which + typically contains the class and detail message of cause). + @param cause the cause (which is saved for later retrieval by the + {@link #getCause()} method). (A null value is + permitted, and indicates that the cause is nonexistent or + unknown.)]]> + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + But for removeAcl operation it will be false. i.e. AclSpec should + not contain permissions.
    + Example: "user:foo,group:bar" + @return Returns list of {@link AclEntry} parsed]]> +
    +
    + + + + + + The expected format of ACL entries in the string parameter is the same + format produced by the {@link #toStringStable()} method. + + @param aclStr + String representation of an ACL.
    + Example: "user:foo:rw-" + @param includePermission + for setAcl operations this will be true. i.e. Acl should include + permissions.
    + But for removeAcl operation it will be false. i.e. Acl should not + contain permissions.
    + Example: "user:foo,group:bar,mask::" + @return Returns an {@link AclEntry} object]]> +
    +
    + + + + + + + + + +
    + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + unmodifiable ordered list of all ACL entries]]> + + + + + + + + + + + + + + + + + + + + + + + + + + + Recommended to use this API ONLY if client communicates with the old + NameNode, needs to pass the Permission for the path to get effective + permission, else use {@link AclStatus#getEffectivePermission(AclEntry)}. + @param entry AclEntry to get the effective action + @param permArg Permission for the path. However if the client is NOT + communicating with old namenode, then this argument will not have + any preference. + @return Returns the effective permission for the entry. + @throws IllegalArgumentException If the client communicating with old + namenode and permission is not passed as an argument.]]> + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + mode is invalid]]> + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + +
    + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + viewfs]]> + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + +
  • + If the trash root for path p is in the same mount point as path p, + and one of: +
      +
    1. The mount point isn't at the top of the target fs.
    2. +
    3. The resolved path of path is root (in fallback FS).
    4. +
    5. The trash isn't in user's target fs home directory + get the corresponding viewFS path for the trash root and return + it. +
    6. +
    +
  • +
  • + else, return the trash root under the root of the mount point + (/{mntpoint}/.Trash/{user}). +
  • + + + These conditions handle several different important cases: +
      +
    • File systems may need to have more local trash roots, such as + encryption zones or snapshot roots.
    • +
    • The fallback mount should use the user's home directory.
    • +
    • Cloud storage systems should not use trash in an implicity defined + home directory, per a container, unless it is the fallback fs.
    • +
    + + @param path the trash root of the path to be determined. + @return the trash root path.]]> +
    +
    + + + + + + + + + + + + +
    + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + +
  • /user -> hdfs://nnContainingUserDir/user +
  • /project/foo -> hdfs://nnProject1/projects/foo +
  • /project/bar -> hdfs://nnProject2/projects/bar +
  • /tmp -> hdfs://nnTmp/privateTmpForUserXXX + + + ViewFs is specified with the following URI: viewfs:/// +

    + To use viewfs one would typically set the default file system in the + config (i.e. fs.defaultFS < = viewfs:///) along with the + mount table config variables as described below. + +

    + ** Config variables to specify the mount table entries ** +

    + + The file system is initialized from the standard Hadoop config through + config variables. + See {@link FsConstants} for URI and Scheme constants; + See {@link Constants} for config var constants; + see {@link ConfigUtil} for convenient lib. + +

    + All the mount table config entries for view fs are prefixed by + fs.viewfs.mounttable. + For example the above example can be specified with the following + config variables: +

      +
    • fs.viewfs.mounttable.default.link./user= + hdfs://nnContainingUserDir/user +
    • fs.viewfs.mounttable.default.link./project/foo= + hdfs://nnProject1/projects/foo +
    • fs.viewfs.mounttable.default.link./project/bar= + hdfs://nnProject2/projects/bar +
    • fs.viewfs.mounttable.default.link./tmp= + hdfs://nnTmp/privateTmpForUserXXX +
    + + The default mount table (when no authority is specified) is + from config variables prefixed by fs.viewFs.mounttable.default + The authority component of a URI can be used to specify a different mount + table. For example, +
      +
    • viewfs://sanjayMountable/ +
    + is initialized from fs.viewFs.mounttable.sanjayMountable.* config variables. + +

    + **** Merge Mounts **** (NOTE: merge mounts are not implemented yet.) +

    + + One can also use "MergeMounts" to merge several directories (this is + sometimes called union-mounts or junction-mounts in the literature. + For example of the home directories are stored on say two file systems + (because they do not fit on one) then one could specify a mount + entry such as following merges two dirs: +

      +
    • /user -> hdfs://nnUser1/user,hdfs://nnUser2/user +
    + Such a mergeLink can be specified with the following config var where "," + is used as the separator for each of links to be merged: +
      +
    • fs.viewfs.mounttable.default.linkMerge./user= + hdfs://nnUser1/user,hdfs://nnUser1/user +
    + A special case of the merge mount is where mount table's root is merged + with the root (slash) of another file system: +
      +
    • fs.viewfs.mounttable.default.linkMergeSlash=hdfs://nn99/ +
    + In this cases the root of the mount table is merged with the root of + hdfs://nn99/ ]]> + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + Since these methods are often vendor- or device-specific, operators + may implement this interface in order to achieve fencing. +

    + Fencing is configured by the operator as an ordered list of methods to + attempt. Each method will be tried in turn, and the next in the list + will only be attempted if the previous one fails. See {@link NodeFencer} + for more information. +

    + If an implementation also implements {@link Configurable} then its + setConf method will be called upon instantiation.]]> + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + state (e.g ACTIVE/STANDBY) as well as + some additional information. + + @throws AccessControlException + if access is denied. + @throws IOException + if other errors happen + @see HAServiceStatus]]> + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + hadoop.http.filter.initializers. + +

      +
    • StaticUserWebFilter - An authorization plugin that makes all +users a static configured user. +
    ]]> +
    +
    + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + public class IntArrayWritable extends ArrayWritable { + public IntArrayWritable() { + super(IntWritable.class); + } + } + ]]> + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + o is a ByteWritable with the same value.]]> + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + the class of the item + @param conf the configuration to store + @param item the object to be stored + @param keyName the name of the key to use + @throws IOException : forwards Exceptions from the underlying + {@link Serialization} classes.]]> + + + + + + + + + the class of the item + @param conf the configuration to use + @param keyName the name of the key to use + @param itemClass the class of the item + @return restored object + @throws IOException : forwards Exceptions from the underlying + {@link Serialization} classes.]]> + + + + + + + + + the class of the item + @param conf the configuration to use + @param items the objects to be stored + @param keyName the name of the key to use + @throws IndexOutOfBoundsException if the items array is empty + @throws IOException : forwards Exceptions from the underlying + {@link Serialization} classes.]]> + + + + + + + + + the class of the item + @param conf the configuration to use + @param keyName the name of the key to use + @param itemClass the class of the item + @return restored object + @throws IOException : forwards Exceptions from the underlying + {@link Serialization} classes.]]> + + + + + DefaultStringifier offers convenience methods to store/load objects to/from + the configuration. + + @param the class of the objects to stringify]]> + + + + + + + + + + + + + + + + + + + + + + + + + + + o is a DoubleWritable with the same value.]]> + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + value argument is null or + its size is zero, the elementType argument must not be null. If + the argument value's size is bigger than zero, the argument + elementType is not be used. + + @param value + @param elementType]]> + + + + + value should not be null + or empty. + + @param value]]> + + + + + + + + + + + + + + value and elementType. If the value argument + is null or its size is zero, the elementType argument must not be + null. If the argument value's size is bigger than zero, the + argument elementType is not be used. + + @param value + @param elementType]]> + + + + + + + + + + + + + + + + + + + o is an EnumSetWritable with the same value, + or both are null.]]> + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + o is a FloatWritable with the same value.]]> + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + When two sequence files, which have same Key type but different Value + types, are mapped out to reduce, multiple Value types is not allowed. + In this case, this class can help you wrap instances with different types. +

    + +

    + Compared with ObjectWritable, this class is much more effective, + because ObjectWritable will append the class declaration as a String + into the output file in every Key-Value pair. +

    + +

    + Generic Writable implements {@link Configurable} interface, so that it will be + configured by the framework. The configuration is passed to the wrapped objects + implementing {@link Configurable} interface before deserialization. +

    + + how to use it:
    + 1. Write your own class, such as GenericObject, which extends GenericWritable.
    + 2. Implements the abstract method getTypes(), defines + the classes which will be wrapped in GenericObject in application. + Attention: this classes defined in getTypes() method, must + implement Writable interface. +

    + + The code looks like this: +
    + public class GenericObject extends GenericWritable {
    + 
    +   private static Class[] CLASSES = {
    +               ClassType1.class, 
    +               ClassType2.class,
    +               ClassType3.class,
    +               };
    +
    +   protected Class[] getTypes() {
    +       return CLASSES;
    +   }
    +
    + }
    + 
    + + @since Nov 8, 2006]]> +
    +
    + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + o is a IntWritable with the same value.]]> + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + closes the input and output streams + at the end. + + @param in InputStrem to read from + @param out OutputStream to write to + @param conf the Configuration object]]> + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + ignore any {@link Throwable} or + null pointers. Must only be used for cleanup in exception handlers. + + @param log the log to record problems to at debug level. Can be null. + @param closeables the objects to close + @deprecated use {@link #cleanupWithLogger(Logger, java.io.Closeable...)} + instead]]> + + + + + + + ignore any {@link Throwable} or + null pointers. Must only be used for cleanup in exception handlers. + + @param logger the log to record problems to at debug level. Can be null. + @param closeables the objects to close]]> + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + This is better than File#listDir because it does not ignore IOExceptions. + + @param dir The directory to list. + @param filter If non-null, the filter to use when listing + this directory. + @return The list of files in the directory. + + @throws IOException On I/O error]]> + + + + + + + + Borrowed from Uwe Schindler in LUCENE-5588 + @param fileToSync the file to fsync]]> + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + o is a LongWritable with the same value.]]> + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + A map is a directory containing two files, the data file, + containing all keys and values in the map, and a smaller index + file, containing a fraction of the keys. The fraction is determined by + {@link Writer#getIndexInterval()}. + +

    The index file is read entirely into memory. Thus key implementations + should try to keep themselves small. + +

    Map files are created by adding entries in-order. To maintain a large + database, perform updates by copying the previous version of a database and + merging in a sorted change list, to create a new version of the database in + a new file. Sorting large change lists can be done with {@link + SequenceFile.Sorter}.]]> + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + o is an MD5Hash whose digest contains the + same values.]]> + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + className by first finding + it in the specified conf. If the specified conf is null, + try load it directly.]]> + + + + + + + + + + + + + + + + + + + + + + + + + + + + + A {@link Comparator} that operates directly on byte representations of + objects. +

    + @param + @see DeserializerComparator]]> +
    + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + SequenceFiles are flat files consisting of binary key/value + pairs. + +

    SequenceFile provides {@link SequenceFile.Writer}, + {@link SequenceFile.Reader} and {@link Sorter} classes for writing, + reading and sorting respectively.

    + + There are three SequenceFile Writers based on the + {@link CompressionType} used to compress key/value pairs: +
      +
    1. + Writer : Uncompressed records. +
    2. +
    3. + RecordCompressWriter : Record-compressed files, only compress + values. +
    4. +
    5. + BlockCompressWriter : Block-compressed files, both keys & + values are collected in 'blocks' + separately and compressed. The size of + the 'block' is configurable. +
    + +

    The actual compression algorithm used to compress key and/or values can be + specified by using the appropriate {@link CompressionCodec}.

    + +

    The recommended way is to use the static createWriter methods + provided by the SequenceFile to chose the preferred format.

    + +

    The {@link SequenceFile.Reader} acts as the bridge and can read any of the + above SequenceFile formats.

    + +

    SequenceFile Formats

    + +

    Essentially there are 3 different formats for SequenceFiles + depending on the CompressionType specified. All of them share a + common header described below. + +

    +
      +
    • + version - 3 bytes of magic header SEQ, followed by 1 byte of actual + version number (e.g. SEQ4 or SEQ6) +
    • +
    • + keyClassName -key class +
    • +
    • + valueClassName - value class +
    • +
    • + compression - A boolean which specifies if compression is turned on for + keys/values in this file. +
    • +
    • + blockCompression - A boolean which specifies if block-compression is + turned on for keys/values in this file. +
    • +
    • + compression codec - CompressionCodec class which is used for + compression of keys and/or values (if compression is + enabled). +
    • +
    • + metadata - {@link Metadata} for this file. +
    • +
    • + sync - A sync marker to denote end of the header. +
    • +
    + +
    Uncompressed SequenceFile Format
    +
      +
    • + Header +
    • +
    • + Record +
        +
      • Record length
      • +
      • Key length
      • +
      • Key
      • +
      • Value
      • +
      +
    • +
    • + A sync-marker every few 100 bytes or so. +
    • +
    + +
    Record-Compressed SequenceFile Format
    +
      +
    • + Header +
    • +
    • + Record +
        +
      • Record length
      • +
      • Key length
      • +
      • Key
      • +
      • Compressed Value
      • +
      +
    • +
    • + A sync-marker every few 100 bytes or so. +
    • +
    + +
    Block-Compressed SequenceFile Format
    +
      +
    • + Header +
    • +
    • + Record Block +
        +
      • Uncompressed number of records in the block
      • +
      • Compressed key-lengths block-size
      • +
      • Compressed key-lengths block
      • +
      • Compressed keys block-size
      • +
      • Compressed keys block
      • +
      • Compressed value-lengths block-size
      • +
      • Compressed value-lengths block
      • +
      • Compressed values block-size
      • +
      • Compressed values block
      • +
      +
    • +
    • + A sync-marker every block. +
    • +
    + +

    The compressed blocks of key lengths and value lengths consist of the + actual lengths of individual keys/values encoded in ZeroCompressedInteger + format.

    + + @see CompressionCodec]]> +
    +
    + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + o is a ShortWritable with the same value.]]> + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + the class of the objects to stringify]]> + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + position. Note that this + method avoids using the converter or doing String instantiation + @return the Unicode scalar value at position or -1 + if the position is invalid or points to a + trailing byte]]> + + + + + + + + + + what in the backing + buffer, starting as position start. The starting + position is measured in bytes and the return value is in + terms of byte position in the buffer. The backing buffer is + not converted to a string for this operation. + @return byte position of the first occurence of the search + string in the UTF-8 buffer or -1 if not found]]> + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + Note: For performance reasons, this call does not clear the + underlying byte array that is retrievable via {@link #getBytes()}. + In order to free the byte-array memory, call {@link #set(byte[])} + with an empty byte array (For example, new byte[0]).]]> + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + o is a Text with the same contents.]]> + + + + + + + + + + + + + + + + + + + + + + + + + replace is true, then + malformed input is replaced with the + substitution character, which is U+FFFD. Otherwise the + method throws a MalformedInputException.]]> + + + + + + + + + + + + + + + replace is true, then + malformed input is replaced with the + substitution character, which is U+FFFD. Otherwise the + method throws a MalformedInputException. + @return ByteBuffer: bytes stores at ByteBuffer.array() + and length is ByteBuffer.limit()]]> + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + In + addition, it provides methods for string traversal without converting the + byte array to a string.

    Also includes utilities for + serializing/deserialing a string, coding/decoding a string, checking if a + byte array contains valid UTF8 code, calculating the length of an encoded + string.]]> + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + This is useful when a class may evolve, so that instances written by the + old version of the class may still be processed by the new version. To + handle this situation, {@link #readFields(DataInput)} + implementations should catch {@link VersionMismatchException}.]]> + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + o is a VIntWritable with the same value.]]> + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + o is a VLongWritable with the same value.]]> + + + + + + + + + + + + + + + + + + + + + + + + out. + + @param out DataOuput to serialize this object into. + @throws IOException]]> + + + + + + + in. + +

    For efficiency, implementations should attempt to re-use storage in the + existing object where possible.

    + + @param in DataInput to deseriablize this object from. + @throws IOException]]> +
    + + + Any key or value type in the Hadoop Map-Reduce + framework implements this interface.

    + +

    Implementations typically implement a static read(DataInput) + method which constructs a new instance, calls {@link #readFields(DataInput)} + and returns the instance.

    + +

    Example:

    +

    +     public class MyWritable implements Writable {
    +       // Some data     
    +       private int counter;
    +       private long timestamp;
    +       
    +       public void write(DataOutput out) throws IOException {
    +         out.writeInt(counter);
    +         out.writeLong(timestamp);
    +       }
    +       
    +       public void readFields(DataInput in) throws IOException {
    +         counter = in.readInt();
    +         timestamp = in.readLong();
    +       }
    +       
    +       public static MyWritable read(DataInput in) throws IOException {
    +         MyWritable w = new MyWritable();
    +         w.readFields(in);
    +         return w;
    +       }
    +     }
    + 

    ]]> +
    + + + + + + + + WritableComparables can be compared to each other, typically + via Comparators. Any type which is to be used as a + key in the Hadoop Map-Reduce framework should implement this + interface.

    + +

    Note that hashCode() is frequently used in Hadoop to partition + keys. It's important that your implementation of hashCode() returns the same + result across different instances of the JVM. Note also that the default + hashCode() implementation in Object does not + satisfy this property.

    + +

    Example:

    +

    +     public class MyWritableComparable implements WritableComparable {
    +       // Some data
    +       private int counter;
    +       private long timestamp;
    +       
    +       public void write(DataOutput out) throws IOException {
    +         out.writeInt(counter);
    +         out.writeLong(timestamp);
    +       }
    +       
    +       public void readFields(DataInput in) throws IOException {
    +         counter = in.readInt();
    +         timestamp = in.readLong();
    +       }
    +       
    +       public int compareTo(MyWritableComparable o) {
    +         int thisValue = this.value;
    +         int thatValue = o.value;
    +         return (thisValue < thatValue ? -1 : (thisValue==thatValue ? 0 : 1));
    +       }
    +
    +       public int hashCode() {
    +         final int prime = 31;
    +         int result = 1;
    +         result = prime * result + counter;
    +         result = prime * result + (int) (timestamp ^ (timestamp >>> 32));
    +         return result
    +       }
    +     }
    + 

    ]]> +
    +
    + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + The default implementation reads the data into two {@link + WritableComparable}s (using {@link + Writable#readFields(DataInput)}, then calls {@link + #compare(WritableComparable,WritableComparable)}.]]> + + + + + + + The default implementation uses the natural ordering, calling {@link + Comparable#compareTo(Object)}.]]> + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + This base implemenation uses the natural ordering. To define alternate + orderings, override {@link #compare(WritableComparable,WritableComparable)}. + +

    One may optimize compare-intensive operations by overriding + {@link #compare(byte[],int,int,byte[],int,int)}. Static utility methods are + provided to assist in optimized implementations of this method.]]> + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + Enum type + @param in DataInput to read from + @param enumType Class type of Enum + @return Enum represented by String read from DataInput + @throws IOException]]> + + + + + + + + + + + + + + + + len number of bytes in input streamin + @param in input stream + @param len number of bytes to skip + @throws IOException when skipped less number of bytes]]> + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + CompressionCodec for which to get the + Compressor + @param conf the Configuration object which contains confs for creating or reinit the compressor + @return Compressor for the given + CompressionCodec from the pool or a new one]]> + + + + + + + + + CompressionCodec for which to get the + Decompressor + @return Decompressor for the given + CompressionCodec the pool or a new one]]> + + + + + + Compressor to be returned to the pool]]> + + + + + + Decompressor to be returned to the + pool]]> + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + Codec aliases are case insensitive. +

    + The code alias is the short class name (without the package name). + If the short class name ends with 'Codec', then there are two aliases for + the codec, the complete short class name and the short class name without + the 'Codec' ending. For example for the 'GzipCodec' codec class name the + alias are 'gzip' and 'gzipcodec'. + + @param codecName the canonical class name of the codec + @return the codec object]]> + + + + + + + Codec aliases are case insensitive. +

    + The code alias is the short class name (without the package name). + If the short class name ends with 'Codec', then there are two aliases for + the codec, the complete short class name and the short class name without + the 'Codec' ending. For example for the 'GzipCodec' codec class name the + alias are 'gzip' and 'gzipcodec'. + + @param codecName the canonical class name of the codec + @return the codec class]]> + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + Implementations are assumed to be buffered. This permits clients to + reposition the underlying input stream then call {@link #resetState()}, + without having to also synchronize client buffers.]]> + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + true indicating that more input data is required. + + @param b Input data + @param off Start offset + @param len Length]]> + + + + + true if the input data buffer is empty and + #setInput() should be called in order to provide more input.]]> + + + + + + + + + + + + + + + + + + + + + + + + + + + + true if the end of the compressed + data output stream has been reached.]]> + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + true indicating that more input data is required. + (Both native and non-native versions of various Decompressors require + that the data passed in via b[] remain unmodified until + the caller is explicitly notified--via {@link #needsInput()}--that the + buffer may be safely modified. With this requirement, an extra + buffer-copy can be avoided.) + + @param b Input data + @param off Start offset + @param len Length]]> + + + + + true if the input data buffer is empty and + {@link #setInput(byte[], int, int)} should be called to + provide more input. + + @return true if the input data buffer is empty and + {@link #setInput(byte[], int, int)} should be called in + order to provide more input.]]> + + + + + + + + + + + + + true if a preset dictionary is needed for decompression. + @return true if a preset dictionary is needed for decompression]]> + + + + + true if the end of the decompressed + data output stream has been reached. Indicates a concatenated data stream + when finished() returns true and {@link #getRemaining()} + returns a positive value. finished() will be reset with the + {@link #reset()} method. + @return true if the end of the decompressed + data output stream has been reached.]]> + + + + + + + + + + + + + + true and getRemaining() returns a positive value. If + {@link #finished()} returns true and getRemaining() returns + a zero value, indicates that the end of data stream has been reached and + is not a concatenated data stream. + @return The number of bytes remaining in the compressed data buffer.]]> + + + + + true and {@link #getRemaining()} returns a positive value, + reset() is called before processing of the next data stream in the + concatenated data stream. {@link #finished()} will be reset and will + return false when reset() is called.]]> + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + +

  • "none" - No compression. +
  • "lzo" - LZO compression. +
  • "gz" - GZIP compression. + ]]> + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + +
  • Block Compression. +
  • Named meta data blocks. +
  • Sorted or unsorted keys. +
  • Seek by key or by file offset. + + The memory footprint of a TFile includes the following: +
      +
    • Some constant overhead of reading or writing a compressed block. +
        +
      • Each compressed block requires one compression/decompression codec for + I/O. +
      • Temporary space to buffer the key. +
      • Temporary space to buffer the value (for TFile.Writer only). Values are + chunk encoded, so that we buffer at most one chunk of user data. By default, + the chunk buffer is 1MB. Reading chunked value does not require additional + memory. +
      +
    • TFile index, which is proportional to the total number of Data Blocks. + The total amount of memory needed to hold the index can be estimated as + (56+AvgKeySize)*NumBlocks. +
    • MetaBlock index, which is proportional to the total number of Meta + Blocks.The total amount of memory needed to hold the index for Meta Blocks + can be estimated as (40+AvgMetaBlockName)*NumMetaBlock. +
    +

    + The behavior of TFile can be customized by the following variables through + Configuration: +

      +
    • tfile.io.chunk.size: Value chunk size. Integer (in bytes). Default + to 1MB. Values of the length less than the chunk size is guaranteed to have + known value length in read time (See + {@link TFile.Reader.Scanner.Entry#isValueLengthKnown()}). +
    • tfile.fs.output.buffer.size: Buffer size used for + FSDataOutputStream. Integer (in bytes). Default to 256KB. +
    • tfile.fs.input.buffer.size: Buffer size used for + FSDataInputStream. Integer (in bytes). Default to 256KB. +
    +

    + Suggestions on performance optimization. +

      +
    • Minimum block size. We recommend a setting of minimum block size between + 256KB to 1MB for general usage. Larger block size is preferred if files are + primarily for sequential access. However, it would lead to inefficient random + access (because there are more data to decompress). Smaller blocks are good + for random access, but require more memory to hold the block index, and may + be slower to create (because we must flush the compressor stream at the + conclusion of each data block, which leads to an FS I/O flush). Further, due + to the internal caching in Compression codec, the smallest possible block + size would be around 20KB-30KB. +
    • The current implementation does not offer true multi-threading for + reading. The implementation uses FSDataInputStream seek()+read(), which is + shown to be much faster than positioned-read call in single thread mode. + However, it also means that if multiple threads attempt to access the same + TFile (using multiple scanners) simultaneously, the actual I/O is carried out + sequentially even if they access different DFS blocks. +
    • Compression codec. Use "none" if the data is not very compressable (by + compressable, I mean a compression ratio at least 2:1). Generally, use "lzo" + as the starting point for experimenting. "gz" overs slightly better + compression ratio over "lzo" but requires 4x CPU to compress and 2x CPU to + decompress, comparing to "lzo". +
    • File system buffering, if the underlying FSDataInputStream and + FSDataOutputStream is already adequately buffered; or if applications + reads/writes keys and values in large buffers, we can reduce the sizes of + input/output buffering in TFile layer by setting the configuration parameters + "tfile.fs.input.buffer.size" and "tfile.fs.output.buffer.size". +
    + + Some design rationale behind TFile can be found at Hadoop-3315.]]> + + + + + + + + + + + Utils#writeVLong(out, n). + + @param out + output stream + @param n + The integer to be encoded + @throws IOException + @see Utils#writeVLong(DataOutput, long)]]> + + + + + + + + +
  • if n in [-32, 127): encode in one byte with the actual value. + Otherwise, +
  • if n in [-20*2^8, 20*2^8): encode in two bytes: byte[0] = n/256 - 52; + byte[1]=n&0xff. Otherwise, +
  • if n IN [-16*2^16, 16*2^16): encode in three bytes: byte[0]=n/2^16 - + 88; byte[1]=(n>>8)&0xff; byte[2]=n&0xff. Otherwise, +
  • if n in [-8*2^24, 8*2^24): encode in four bytes: byte[0]=n/2^24 - 112; + byte[1] = (n>>16)&0xff; byte[2] = (n>>8)&0xff; byte[3]=n&0xff. Otherwise: +
  • if n in [-2^31, 2^31): encode in five bytes: byte[0]=-125; byte[1] = + (n>>24)&0xff; byte[2]=(n>>16)&0xff; byte[3]=(n>>8)&0xff; byte[4]=n&0xff; +
  • if n in [-2^39, 2^39): encode in six bytes: byte[0]=-124; byte[1] = + (n>>32)&0xff; byte[2]=(n>>24)&0xff; byte[3]=(n>>16)&0xff; + byte[4]=(n>>8)&0xff; byte[5]=n&0xff +
  • if n in [-2^47, 2^47): encode in seven bytes: byte[0]=-123; byte[1] = + (n>>40)&0xff; byte[2]=(n>>32)&0xff; byte[3]=(n>>24)&0xff; + byte[4]=(n>>16)&0xff; byte[5]=(n>>8)&0xff; byte[6]=n&0xff; +
  • if n in [-2^55, 2^55): encode in eight bytes: byte[0]=-122; byte[1] = + (n>>48)&0xff; byte[2] = (n>>40)&0xff; byte[3]=(n>>32)&0xff; + byte[4]=(n>>24)&0xff; byte[5]=(n>>16)&0xff; byte[6]=(n>>8)&0xff; + byte[7]=n&0xff; +
  • if n in [-2^63, 2^63): encode in nine bytes: byte[0]=-121; byte[1] = + (n>>54)&0xff; byte[2] = (n>>48)&0xff; byte[3] = (n>>40)&0xff; + byte[4]=(n>>32)&0xff; byte[5]=(n>>24)&0xff; byte[6]=(n>>16)&0xff; + byte[7]=(n>>8)&0xff; byte[8]=n&0xff; + + + @param out + output stream + @param n + the integer number + @throws IOException]]> + + + + + + + (int)Utils#readVLong(in). + + @param in + input stream + @return the decoded integer + @throws IOException + + @see Utils#readVLong(DataInput)]]> + + + + + + + +
  • if (FB >= -32), return (long)FB; +
  • if (FB in [-72, -33]), return (FB+52)<<8 + NB[0]&0xff; +
  • if (FB in [-104, -73]), return (FB+88)<<16 + (NB[0]&0xff)<<8 + + NB[1]&0xff; +
  • if (FB in [-120, -105]), return (FB+112)<<24 + (NB[0]&0xff)<<16 + + (NB[1]&0xff)<<8 + NB[2]&0xff; +
  • if (FB in [-128, -121]), return interpret NB[FB+129] as a signed + big-endian integer. + + @param in + input stream + @return the decoded long integer. + @throws IOException]]> + + + + + + + + + + + + + + + + + + + + + + + + Type of the input key. + @param list + The list + @param key + The input key. + @param cmp + Comparator for the key. + @return The index to the desired element if it exists; or list.size() + otherwise.]]> + + + + + + + + + Type of the input key. + @param list + The list + @param key + The input key. + @param cmp + Comparator for the key. + @return The index to the desired element if it exists; or list.size() + otherwise.]]> + + + + + + + + Type of the input key. + @param list + The list + @param key + The input key. + @return The index to the desired element if it exists; or list.size() + otherwise.]]> + + + + + + + + Type of the input key. + @param list + The list + @param key + The input key. + @return The index to the desired element if it exists; or list.size() + otherwise.]]> + + + + + + + + + + + + + + + + + An experimental {@link Serialization} for Java {@link Serializable} classes. +

    + @see JavaSerializationComparator]]> +
    +
    + + + + + + + + + A {@link RawComparator} that uses a {@link JavaSerialization} + {@link Deserializer} to deserialize objects that are then compared via + their {@link Comparable} interfaces. +

    + @param + @see JavaSerialization]]> +
    +
    + + + + + + + + + + + + + +This package provides a mechanism for using different serialization frameworks +in Hadoop. The property "io.serializations" defines a list of +{@link org.apache.hadoop.io.serializer.Serialization}s that know how to create +{@link org.apache.hadoop.io.serializer.Serializer}s and +{@link org.apache.hadoop.io.serializer.Deserializer}s. +

    + +

    +To add a new serialization framework write an implementation of +{@link org.apache.hadoop.io.serializer.Serialization} and add its name to the +"io.serializations" property. +

    ]]> +
    +
    + + + + + + + + + + + + + + + + + + + avro.reflect.pkgs or implement + {@link AvroReflectSerializable} interface.]]> + + + + + + + + + + + + + + + + + + + + + + + + + + +This package provides Avro serialization in Hadoop. This can be used to +serialize/deserialize Avro types in Hadoop. +

    + +

    +Use {@link org.apache.hadoop.io.serializer.avro.AvroSpecificSerialization} for +serialization of classes generated by Avro's 'specific' compiler. +

    + +

    +Use {@link org.apache.hadoop.io.serializer.avro.AvroReflectSerialization} for +other classes. +{@link org.apache.hadoop.io.serializer.avro.AvroReflectSerialization} work for +any class which is either in the package list configured via +{@link org.apache.hadoop.io.serializer.avro.AvroReflectSerialization#AVRO_REFLECT_PACKAGES} +or implement {@link org.apache.hadoop.io.serializer.avro.AvroReflectSerializable} +interface. +

    ]]> +
    +
    + + + + + + + + + + + + + + + + + + + + + + + + + +The API is abstract so that it can be implemented on top of +a variety of metrics client libraries. The choice of +client library is a configuration option, and different +modules within the same application can use +different metrics implementation libraries. +

    +Sub-packages: +

    +
    org.apache.hadoop.metrics.spi
    +
    The abstract Server Provider Interface package. Those wishing to + integrate the metrics API with a particular metrics client library should + extend this package.
    + +
    org.apache.hadoop.metrics.file
    +
    An implementation package which writes the metric data to + a file, or sends it to the standard output stream.
    + +
    org.apache.hadoop.metrics.ganglia
    +
    An implementation package which sends metric data to + Ganglia.
    +
    + +

    Introduction to the Metrics API

    + +Here is a simple example of how to use this package to report a single +metric value: +
    +    private ContextFactory contextFactory = ContextFactory.getFactory();
    +    
    +    void reportMyMetric(float myMetric) {
    +        MetricsContext myContext = contextFactory.getContext("myContext");
    +        MetricsRecord myRecord = myContext.getRecord("myRecord");
    +        myRecord.setMetric("myMetric", myMetric);
    +        myRecord.update();
    +    }
    +
    + +In this example there are three names: +
    +
    myContext
    +
    The context name will typically identify either the application, or else a + module within an application or library.
    + +
    myRecord
    +
    The record name generally identifies some entity for which a set of + metrics are to be reported. For example, you could have a record named + "cacheStats" for reporting a number of statistics relating to the usage of + some cache in your application.
    + +
    myMetric
    +
    This identifies a particular metric. For example, you might have metrics + named "cache_hits" and "cache_misses". +
    +
    + +

    Tags

    + +In some cases it is useful to have multiple records with the same name. For +example, suppose that you want to report statistics about each disk on a computer. +In this case, the record name would be something like "diskStats", but you also +need to identify the disk which is done by adding a tag to the record. +The code could look something like this: +
    +    private MetricsRecord diskStats =
    +            contextFactory.getContext("myContext").getRecord("diskStats");
    +            
    +    void reportDiskMetrics(String diskName, float diskBusy, float diskUsed) {
    +        diskStats.setTag("diskName", diskName);
    +        diskStats.setMetric("diskBusy", diskBusy);
    +        diskStats.setMetric("diskUsed", diskUsed);
    +        diskStats.update();
    +    }
    +
    + +

    Buffering and Callbacks

    + +Data is not sent immediately to the metrics system when +MetricsRecord.update() is called. Instead it is stored in an +internal table, and the contents of the table are sent periodically. +This can be important for two reasons: +
      +
    1. It means that a programmer is free to put calls to this API in an + inner loop, since updates can be very frequent without slowing down + the application significantly.
    2. +
    3. Some implementations can gain efficiency by combining many metrics + into a single UDP message.
    4. +
    + +The API provides a timer-based callback via the +registerUpdater() method. The benefit of this +versus using java.util.Timer is that the callbacks will be done +immediately before sending the data, making the data as current as possible. + +

    Configuration

    + +It is possible to programmatically examine and modify configuration data +before creating a context, like this: +
    +    ContextFactory factory = ContextFactory.getFactory();
    +    ... examine and/or modify factory attributes ...
    +    MetricsContext context = factory.getContext("myContext");
    +
    +The factory attributes can be examined and modified using the following +ContextFactorymethods: +
      +
    • Object getAttribute(String attributeName)
    • +
    • String[] getAttributeNames()
    • +
    • void setAttribute(String name, Object value)
    • +
    • void removeAttribute(attributeName)
    • +
    + +

    +ContextFactory.getFactory() initializes the factory attributes by +reading the properties file hadoop-metrics.properties if it exists +on the class path. + +

    +A factory attribute named: +

    +contextName.class
    +
    +should have as its value the fully qualified name of the class to be +instantiated by a call of the CodeFactory method +getContext(contextName). If this factory attribute is not +specified, the default is to instantiate +org.apache.hadoop.metrics.file.FileContext. + +

    +Other factory attributes are specific to a particular implementation of this +API and are documented elsewhere. For example, configuration attributes for +the file and Ganglia implementations can be found in the javadoc for +their respective packages.]]> + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + +Implementation of the metrics package that sends metric data to +Ganglia. +Programmers should not normally need to use this package directly. Instead +they should use org.hadoop.metrics. + +

    +These are the implementation specific factory attributes +(See ContextFactory.getFactory()): + +

    +
    contextName.servers
    +
    Space and/or comma separated sequence of servers to which UDP + messages should be sent.
    + +
    contextName.period
    +
    The period in seconds on which the metric data is sent to the + server(s).
    + +
    contextName.multicast
    +
    Enable multicast for Ganglia
    + +
    contextName.multicast.ttl
    +
    TTL for multicast packets
    + +
    contextName.units.recordName.metricName
    +
    The units for the specified metric in the specified record.
    + +
    contextName.slope.recordName.metricName
    +
    The slope for the specified metric in the specified record.
    + +
    contextName.tmax.recordName.metricName
    +
    The tmax for the specified metric in the specified record.
    + +
    contextName.dmax.recordName.metricName
    +
    The dmax for the specified metric in the specified record.
    + +
    ]]> +
    +
    + + + + + + + + + + + + + + + + + + + + + + + + + + contextName.tableName. The returned map consists of + those attributes with the contextName and tableName stripped off.]]> + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + recordName. + Throws an exception if the metrics implementation is configured with a fixed + set of record names and recordName is not in that set. + + @param recordName the name of the record + @throws MetricsException if recordName conflicts with configuration data]]> + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + This class implements the internal table of metric data, and the timer + on which data is to be sent to the metrics system. Subclasses must + override the abstract emitRecord method in order to transmit + the data.

    + + @deprecated Use org.apache.hadoop.metrics2 package instead.]]> + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + update + and remove(). + + @deprecated Use {@link org.apache.hadoop.metrics2.impl.MetricsRecordImpl} + instead.]]> + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + hostname or hostname:port. If + the specs string is null, defaults to localhost:defaultPort. + + @return a list of InetSocketAddress objects.]]> + + + + + + + + + org.apache.hadoop.metrics.file and +org.apache.hadoop.metrics.ganglia.

    + +Plugging in an implementation involves writing a concrete subclass of +AbstractMetricsContext. The subclass should get its + configuration information using the getAttribute(attributeName) + method.]]> + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + Implementations of this interface consume the {@link MetricsRecord} generated + from {@link MetricsSource}. It registers with {@link MetricsSystem} which + periodically pushes the {@link MetricsRecord} to the sink using + {@link #putMetrics(MetricsRecord)} method. If the implementing class also + implements {@link Closeable}, then the MetricsSystem will close the sink when + it is stopped.]]> + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + the actual type of the source object + @param source object to register + @return the source object + @exception MetricsException]]> + + + + + + + + the actual type of the source object + @param source object to register + @param name of the source. Must be unique or null (then extracted from + the annotations of the source object.) + @param desc the description of the source (or null. See above.) + @return the source object + @exception MetricsException]]> + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + (aggregate). + Filter out entries that don't have at least minSamples. + + @return a map of peer DataNode Id to the average latency to that + node seen over the measurement period.]]> + + + + + + + + + + + This class maintains a group of rolling average metrics. It implements the + algorithm of rolling average, i.e. a number of sliding windows are kept to + roll over and evict old subsets of samples. Each window has a subset of + samples in a stream, where sub-sum and sub-total are collected. All sub-sums + and sub-totals in all windows will be aggregated to final-sum and final-total + used to compute final average, which is called rolling average. +

    ]]> +
    +
    + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + +
    + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + This class is a metrics sink that uses + {@link org.apache.hadoop.fs.FileSystem} to write the metrics logs. Every + roll interval a new directory will be created under the path specified by the + basepath property. All metrics will be logged to a file in the + current interval's directory in a file named <hostname>.log, where + <hostname> is the name of the host on which the metrics logging + process is running. The base path is set by the + <prefix>.sink.<instance>.basepath property. The + time zone used to create the current interval's directory name is GMT. If + the basepath property isn't specified, it will default to + "/tmp", which is the temp directory on whatever default file + system is configured for the cluster.

    + +

    The <prefix>.sink.<instance>.ignore-error + property controls whether an exception is thrown when an error is encountered + writing a log file. The default value is true. When set to + false, file errors are quietly swallowed.

    + +

    The roll-interval property sets the amount of time before + rolling the directory. The default value is 1 hour. The roll interval may + not be less than 1 minute. The property's value should be given as + number unit, where number is an integer value, and + unit is a valid unit. Valid units are minute, hour, + and day. The units are case insensitive and may be abbreviated or + plural. If no units are specified, hours are assumed. For example, + "2", "2h", "2 hour", and + "2 hours" are all valid ways to specify two hours.

    + +

    The roll-offset-interval-millis property sets the upper + bound on a random time interval (in milliseconds) that is used to delay + before the initial roll. All subsequent rolls will happen an integer + number of roll intervals after the initial roll, hence retaining the original + offset. The purpose of this property is to insert some variance in the roll + times so that large clusters using this sink on every node don't cause a + performance impact on HDFS by rolling simultaneously. The default value is + 30000 (30s). When writing to HDFS, as a rule of thumb, the roll offset in + millis should be no less than the number of sink instances times 5. + +

    The primary use of this class is for logging to HDFS. As it uses + {@link org.apache.hadoop.fs.FileSystem} to access the target file system, + however, it can be used to write to the local file system, Amazon S3, or any + other supported file system. The base path for the sink will determine the + file system used. An unqualified path will write to the default file system + set by the configuration.

    + +

    Not all file systems support the ability to append to files. In file + systems without the ability to append to files, only one writer can write to + a file at a time. To allow for concurrent writes from multiple daemons on a + single host, the source property is used to set unique headers + for the log files. The property should be set to the name of + the source daemon, e.g. namenode. The value of the + source property should typically be the same as the property's + prefix. If this property is not set, the source is taken to be + unknown.

    + +

    Instead of appending to an existing file, by default the sink + will create a new file with a suffix of ".<n>&quet;, where + n is the next lowest integer that isn't already used in a file name, + similar to the Hadoop daemon logs. NOTE: the file with the highest + sequence number is the newest file, unlike the Hadoop daemon logs.

    + +

    For file systems that allow append, the sink supports appending to the + existing file instead. If the allow-append property is set to + true, the sink will instead append to the existing file on file systems that + support appends. By default, the allow-append property is + false.

    + +

    Note that when writing to HDFS with allow-append set to true, + there is a minimum acceptable number of data nodes. If the number of data + nodes drops below that minimum, the append will succeed, but reading the + data will fail with an IOException in the DataStreamer class. The minimum + number of data nodes required for a successful append is generally 2 or + 3.

    + +

    Note also that when writing to HDFS, the file size information is not + updated until the file is closed (at the end of the interval) even though + the data is being written successfully. This is a known HDFS limitation that + exists because of the performance cost of updating the metadata. See + HDFS-5478.

    + +

    When using this sink in a secure (Kerberos) environment, two additional + properties must be set: keytab-key and + principal-key. keytab-key should contain the key by + which the keytab file can be found in the configuration, for example, + yarn.nodemanager.keytab. principal-key should + contain the key by which the principal can be found in the configuration, + for example, yarn.nodemanager.principal.]]> + + + + + + + + + + + + + + + + + + + + + + + + + CollectD StatsD plugin). +
    + To configure this plugin, you will need to add the following + entries to your hadoop-metrics2.properties file: +
    +

    + *.sink.statsd.class=org.apache.hadoop.metrics2.sink.StatsDSink
    + [prefix].sink.statsd.server.host=
    + [prefix].sink.statsd.server.port=
    + [prefix].sink.statsd.skip.hostname=true|false (optional)
    + [prefix].sink.statsd.service.name=NameNode (name you want for service)
    + 
    ]]> +
    +
    + +
    + + + + + + + + + + + + + + + ,name=" + Where the and are the supplied parameters + + @param serviceName + @param nameName + @param theMbean - the MBean to register + @return the named used to register the MBean]]> + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + hostname or hostname:port. If + the specs string is null, defaults to localhost:defaultPort. + + @param specs server specs (see description) + @param defaultPort the default port if not specified + @return a list of InetSocketAddress objects.]]> + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + This method is used when parts of Hadoop need know whether to apply + single rack vs multi-rack policies, such as during block placement. + Such algorithms behave differently if they are on multi-switch systems. +

    + + @return true if the mapping thinks that it is on a single switch]]> +
    +
    + + + + + + + + + + + + + + + + + This predicate simply assumes that all mappings not derived from + this class are multi-switch. + @param mapping the mapping to query + @return true if the base class says it is single switch, or the mapping + is not derived from this class.]]> + + + + It is not mandatory to + derive {@link DNSToSwitchMapping} implementations from it, but it is strongly + recommended, as it makes it easy for the Hadoop developers to add new methods + to this base class that are automatically picked up by all implementations. +

    + + This class does not extend the Configured + base class, and should not be changed to do so, as it causes problems + for subclasses. The constructor of the Configured calls + the {@link #setConf(Configuration)} method, which will call into the + subclasses before they have been fully constructed.]]> + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + If a name cannot be resolved to a rack, the implementation + should return {@link NetworkTopology#DEFAULT_RACK}. This + is what the bundled implementations do, though it is not a formal requirement + + @param names the list of hosts to resolve (can be empty) + @return list of resolved network paths. + If names is empty, the returned list is also empty]]> + + + + + + + + + + + + + + + + + + + + + + + + Calling {@link #setConf(Configuration)} will trigger a + re-evaluation of the configuration settings and so be used to + set up the mapping script.]]> + + + + + + + + + + + + + + + + + + + + + This will get called in the superclass constructor, so a check is needed + to ensure that the raw mapping is defined before trying to relaying a null + configuration. + @param conf]]> + + + + + + + + + + It contains a static class RawScriptBasedMapping that performs + the work: reading the configuration parameters, executing any defined + script, handling errors and such like. The outer + class extends {@link CachedDNSToSwitchMapping} to cache the delegated + queries. +

    + This DNS mapper's {@link #isSingleSwitch()} predicate returns + true if and only if a script is defined.]]> + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + Simple {@link DNSToSwitchMapping} implementation that reads a 2 column text + file. The columns are separated by whitespace. The first column is a DNS or + IP address and the second column specifies the rack where the address maps. +

    +

    + This class uses the configuration parameter {@code + net.topology.table.file.name} to locate the mapping file. +

    +

    + Calls to {@link #resolve(List)} will look up the address as defined in the + mapping file. If no entry corresponding to the address is found, the value + {@code /default-rack} is returned. +

    ]]> +
    +
    + + + + +
    + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + Avro.]]> + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + Avro.]]> + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + =} getCount(). + @param newCapacity The new capacity in bytes.]]> + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + Avro.]]> + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + Avro.]]> + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + Avro.]]> + + + + + + + + + + + + Index idx = startVector(...); + while (!idx.done()) { + .... // read element of a vector + idx.incr(); + } + + + @deprecated Replaced by Avro.]]> + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + Avro.]]> + + + + + + + + + + + + + + + + + + + + + + + + + + + Avro.]]> + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + Avro.]]> + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + Avro.]]> + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + Avro.]]> + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + Avro.]]> + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + Avro.]]> + + + + + + (DEPRECATED) Hadoop record I/O contains classes and a record description language + translator for simplifying serialization and deserialization of records in a + language-neutral manner. +

    + +

    + DEPRECATED: Replaced by Avro. +

    + +

    Introduction

    + + Software systems of any significant complexity require mechanisms for data +interchange with the outside world. These interchanges typically involve the +marshaling and unmarshaling of logical units of data to and from data streams +(files, network connections, memory buffers etc.). Applications usually have +some code for serializing and deserializing the data types that they manipulate +embedded in them. The work of serialization has several features that make +automatic code generation for it worthwhile. Given a particular output encoding +(binary, XML, etc.), serialization of primitive types and simple compositions +of primitives (structs, vectors etc.) is a very mechanical task. Manually +written serialization code can be susceptible to bugs especially when records +have a large number of fields or a record definition changes between software +versions. Lastly, it can be very useful for applications written in different +programming languages to be able to share and interchange data. This can be +made a lot easier by describing the data records manipulated by these +applications in a language agnostic manner and using the descriptions to derive +implementations of serialization in multiple target languages. + +This document describes Hadoop Record I/O, a mechanism that is aimed +at +
      +
    • enabling the specification of simple serializable data types (records) +
    • enabling the generation of code in multiple target languages for +marshaling and unmarshaling such types +
    • providing target language specific support that will enable application +programmers to incorporate generated code into their applications +
    + +The goals of Hadoop Record I/O are similar to those of mechanisms such as XDR, +ASN.1, PADS and ICE. While these systems all include a DDL that enables +the specification of most record types, they differ widely in what else they +focus on. The focus in Hadoop Record I/O is on data marshaling and +multi-lingual support. We take a translator-based approach to serialization. +Hadoop users have to describe their data in a simple data description +language. The Hadoop DDL translator rcc generates code that users +can invoke in order to read/write their data from/to simple stream +abstractions. Next we list explicitly some of the goals and non-goals of +Hadoop Record I/O. + + +

    Goals

    + +
      +
    • Support for commonly used primitive types. Hadoop should include as +primitives commonly used builtin types from programming languages we intend to +support. + +
    • Support for common data compositions (including recursive compositions). +Hadoop should support widely used composite types such as structs and +vectors. + +
    • Code generation in multiple target languages. Hadoop should be capable of +generating serialization code in multiple target languages and should be +easily extensible to new target languages. The initial target languages are +C++ and Java. + +
    • Support for generated target languages. Hadooop should include support +in the form of headers, libraries, packages for supported target languages +that enable easy inclusion and use of generated code in applications. + +
    • Support for multiple output encodings. Candidates include +packed binary, comma-separated text, XML etc. + +
    • Support for specifying record types in a backwards/forwards compatible +manner. This will probably be in the form of support for optional fields in +records. This version of the document does not include a description of the +planned mechanism, we intend to include it in the next iteration. + +
    + +

    Non-Goals

    + +
      +
    • Serializing existing arbitrary C++ classes. +
    • Serializing complex data structures such as trees, linked lists etc. +
    • Built-in indexing schemes, compression, or check-sums. +
    • Dynamic construction of objects from an XML schema. +
    + +The remainder of this document describes the features of Hadoop record I/O +in more detail. Section 2 describes the data types supported by the system. +Section 3 lays out the DDL syntax with some examples of simple records. +Section 4 describes the process of code generation with rcc. Section 5 +describes target language mappings and support for Hadoop types. We include a +fairly complete description of C++ mappings with intent to include Java and +others in upcoming iterations of this document. The last section talks about +supported output encodings. + + +

    Data Types and Streams

    + +This section describes the primitive and composite types supported by Hadoop. +We aim to support a set of types that can be used to simply and efficiently +express a wide range of record types in different programming languages. + +

    Primitive Types

    + +For the most part, the primitive types of Hadoop map directly to primitive +types in high level programming languages. Special cases are the +ustring (a Unicode string) and buffer types, which we believe +find wide use and which are usually implemented in library code and not +available as language built-ins. Hadoop also supplies these via library code +when a target language built-in is not present and there is no widely +adopted "standard" implementation. The complete list of primitive types is: + +
      +
    • byte: An 8-bit unsigned integer. +
    • boolean: A boolean value. +
    • int: A 32-bit signed integer. +
    • long: A 64-bit signed integer. +
    • float: A single precision floating point number as described by + IEEE-754. +
    • double: A double precision floating point number as described by + IEEE-754. +
    • ustring: A string consisting of Unicode characters. +
    • buffer: An arbitrary sequence of bytes. +
    + + +

    Composite Types

    +Hadoop supports a small set of composite types that enable the description +of simple aggregate types and containers. A composite type is serialized +by sequentially serializing it constituent elements. The supported +composite types are: + +
      + +
    • record: An aggregate type like a C-struct. This is a list of +typed fields that are together considered a single unit of data. A record +is serialized by sequentially serializing its constituent fields. In addition +to serialization a record has comparison operations (equality and less-than) +implemented for it, these are defined as memberwise comparisons. + +
    • vector: A sequence of entries of the same data type, primitive +or composite. + +
    • map: An associative container mapping instances of a key type to +instances of a value type. The key and value types may themselves be primitive +or composite types. + +
    + +

    Streams

    + +Hadoop generates code for serializing and deserializing record types to +abstract streams. For each target language Hadoop defines very simple input +and output stream interfaces. Application writers can usually develop +concrete implementations of these by putting a one method wrapper around +an existing stream implementation. + + +

    DDL Syntax and Examples

    + +We now describe the syntax of the Hadoop data description language. This is +followed by a few examples of DDL usage. + +

    Hadoop DDL Syntax

    + +
    
    +recfile = *include module *record
    +include = "include" path
    +path = (relative-path / absolute-path)
    +module = "module" module-name
    +module-name = name *("." name)
    +record := "class" name "{" 1*(field) "}"
    +field := type name ";"
    +name :=  ALPHA (ALPHA / DIGIT / "_" )*
    +type := (ptype / ctype)
    +ptype := ("byte" / "boolean" / "int" |
    +          "long" / "float" / "double"
    +          "ustring" / "buffer")
    +ctype := (("vector" "<" type ">") /
    +          ("map" "<" type "," type ">" ) ) / name)
    +
    + +A DDL file describes one or more record types. It begins with zero or +more include declarations, a single mandatory module declaration +followed by zero or more class declarations. The semantics of each of +these declarations are described below: + +
      + +
    • include: An include declaration specifies a DDL file to be +referenced when generating code for types in the current DDL file. Record types +in the current compilation unit may refer to types in all included files. +File inclusion is recursive. An include does not trigger code +generation for the referenced file. + +
    • module: Every Hadoop DDL file must have a single module +declaration that follows the list of includes and precedes all record +declarations. A module declaration identifies a scope within which +the names of all types in the current file are visible. Module names are +mapped to C++ namespaces, Java packages etc. in generated code. + +
    • class: Records types are specified through class +declarations. A class declaration is like a Java class declaration. +It specifies a named record type and a list of fields that constitute records +of the type. Usage is illustrated in the following examples. + +
    + +

    Examples

    + +
      +
    • A simple DDL file links.jr with just one record declaration. +
      
      +module links {
      +    class Link {
      +        ustring URL;
      +        boolean isRelative;
      +        ustring anchorText;
      +    };
      +}
      +
      + +
    • A DDL file outlinks.jr which includes another +
      
      +include "links.jr"
      +
      +module outlinks {
      +    class OutLinks {
      +        ustring baseURL;
      +        vector outLinks;
      +    };
      +}
      +
      +
    + +

    Code Generation

    + +The Hadoop translator is written in Java. Invocation is done by executing a +wrapper shell script named named rcc. It takes a list of +record description files as a mandatory argument and an +optional language argument (the default is Java) --language or +-l. Thus a typical invocation would look like: +
    
    +$ rcc -l C++  ...
    +
    + + +

    Target Language Mappings and Support

    + +For all target languages, the unit of code generation is a record type. +For each record type, Hadoop generates code for serialization and +deserialization, record comparison and access to record members. + +

    C++

    + +Support for including Hadoop generated C++ code in applications comes in the +form of a header file recordio.hh which needs to be included in source +that uses Hadoop types and a library librecordio.a which applications need +to be linked with. The header declares the Hadoop C++ namespace which defines +appropriate types for the various primitives, the basic interfaces for +records and streams and enumerates the supported serialization encodings. +Declarations of these interfaces and a description of their semantics follow: + +
    
    +namespace hadoop {
    +
    +  enum RecFormat { kBinary, kXML, kCSV };
    +
    +  class InStream {
    +  public:
    +    virtual ssize_t read(void *buf, size_t n) = 0;
    +  };
    +
    +  class OutStream {
    +  public:
    +    virtual ssize_t write(const void *buf, size_t n) = 0;
    +  };
    +
    +  class IOError : public runtime_error {
    +  public:
    +    explicit IOError(const std::string& msg);
    +  };
    +
    +  class IArchive;
    +  class OArchive;
    +
    +  class RecordReader {
    +  public:
    +    RecordReader(InStream& in, RecFormat fmt);
    +    virtual ~RecordReader(void);
    +
    +    virtual void read(Record& rec);
    +  };
    +
    +  class RecordWriter {
    +  public:
    +    RecordWriter(OutStream& out, RecFormat fmt);
    +    virtual ~RecordWriter(void);
    +
    +    virtual void write(Record& rec);
    +  };
    +
    +
    +  class Record {
    +  public:
    +    virtual std::string type(void) const = 0;
    +    virtual std::string signature(void) const = 0;
    +  protected:
    +    virtual bool validate(void) const = 0;
    +
    +    virtual void
    +    serialize(OArchive& oa, const std::string& tag) const = 0;
    +
    +    virtual void
    +    deserialize(IArchive& ia, const std::string& tag) = 0;
    +  };
    +}
    +
    + +
      + +
    • RecFormat: An enumeration of the serialization encodings supported +by this implementation of Hadoop. + +
    • InStream: A simple abstraction for an input stream. This has a +single public read method that reads n bytes from the stream into +the buffer buf. Has the same semantics as a blocking read system +call. Returns the number of bytes read or -1 if an error occurs. + +
    • OutStream: A simple abstraction for an output stream. This has a +single write method that writes n bytes to the stream from the +buffer buf. Has the same semantics as a blocking write system +call. Returns the number of bytes written or -1 if an error occurs. + +
    • RecordReader: A RecordReader reads records one at a time from +an underlying stream in a specified record format. The reader is instantiated +with a stream and a serialization format. It has a read method that +takes an instance of a record and deserializes the record from the stream. + +
    • RecordWriter: A RecordWriter writes records one at a +time to an underlying stream in a specified record format. The writer is +instantiated with a stream and a serialization format. It has a +write method that takes an instance of a record and serializes the +record to the stream. + +
    • Record: The base class for all generated record types. This has two +public methods type and signature that return the typename and the +type signature of the record. + +
    + +Two files are generated for each record file (note: not for each record). If a +record file is named "name.jr", the generated files are +"name.jr.cc" and "name.jr.hh" containing serialization +implementations and record type declarations respectively. + +For each record in the DDL file, the generated header file will contain a +class definition corresponding to the record type, method definitions for the +generated type will be present in the '.cc' file. The generated class will +inherit from the abstract class hadoop::Record. The DDL files +module declaration determines the namespace the record belongs to. +Each '.' delimited token in the module declaration results in the +creation of a namespace. For instance, the declaration module docs.links +results in the creation of a docs namespace and a nested +docs::links namespace. In the preceding examples, the Link class +is placed in the links namespace. The header file corresponding to +the links.jr file will contain: + +
    
    +namespace links {
    +  class Link : public hadoop::Record {
    +    // ....
    +  };
    +};
    +
    + +Each field within the record will cause the generation of a private member +declaration of the appropriate type in the class declaration, and one or more +acccessor methods. The generated class will implement the serialize and +deserialize methods defined in hadoop::Record+. It will also +implement the inspection methods type and signature from +hadoop::Record. A default constructor and virtual destructor will also +be generated. Serialization code will read/write records into streams that +implement the hadoop::InStream and the hadoop::OutStream interfaces. + +For each member of a record an accessor method is generated that returns +either the member or a reference to the member. For members that are returned +by value, a setter method is also generated. This is true for primitive +data members of the types byte, int, long, boolean, float and +double. For example, for a int field called MyField the folowing +code is generated. + +
    
    +...
    +private:
    +  int32_t mMyField;
    +  ...
    +public:
    +  int32_t getMyField(void) const {
    +    return mMyField;
    +  };
    +
    +  void setMyField(int32_t m) {
    +    mMyField = m;
    +  };
    +  ...
    +
    + +For a ustring or buffer or composite field. The generated code +only contains accessors that return a reference to the field. A const +and a non-const accessor are generated. For example: + +
    
    +...
    +private:
    +  std::string mMyBuf;
    +  ...
    +public:
    +
    +  std::string& getMyBuf() {
    +    return mMyBuf;
    +  };
    +
    +  const std::string& getMyBuf() const {
    +    return mMyBuf;
    +  };
    +  ...
    +
    + +

    Examples

    + +Suppose the inclrec.jr file contains: +
    
    +module inclrec {
    +    class RI {
    +        int      I32;
    +        double   D;
    +        ustring  S;
    +    };
    +}
    +
    + +and the testrec.jr file contains: + +
    
    +include "inclrec.jr"
    +module testrec {
    +    class R {
    +        vector VF;
    +        RI            Rec;
    +        buffer        Buf;
    +    };
    +}
    +
    + +Then the invocation of rcc such as: +
    
    +$ rcc -l c++ inclrec.jr testrec.jr
    +
    +will result in generation of four files: +inclrec.jr.{cc,hh} and testrec.jr.{cc,hh}. + +The inclrec.jr.hh will contain: + +
    
    +#ifndef _INCLREC_JR_HH_
    +#define _INCLREC_JR_HH_
    +
    +#include "recordio.hh"
    +
    +namespace inclrec {
    +  
    +  class RI : public hadoop::Record {
    +
    +  private:
    +
    +    int32_t      I32;
    +    double       D;
    +    std::string  S;
    +
    +  public:
    +
    +    RI(void);
    +    virtual ~RI(void);
    +
    +    virtual bool operator==(const RI& peer) const;
    +    virtual bool operator<(const RI& peer) const;
    +
    +    virtual int32_t getI32(void) const { return I32; }
    +    virtual void setI32(int32_t v) { I32 = v; }
    +
    +    virtual double getD(void) const { return D; }
    +    virtual void setD(double v) { D = v; }
    +
    +    virtual std::string& getS(void) const { return S; }
    +    virtual const std::string& getS(void) const { return S; }
    +
    +    virtual std::string type(void) const;
    +    virtual std::string signature(void) const;
    +
    +  protected:
    +
    +    virtual void serialize(hadoop::OArchive& a) const;
    +    virtual void deserialize(hadoop::IArchive& a);
    +  };
    +} // end namespace inclrec
    +
    +#endif /* _INCLREC_JR_HH_ */
    +
    +
    + +The testrec.jr.hh file will contain: + + +
    
    +
    +#ifndef _TESTREC_JR_HH_
    +#define _TESTREC_JR_HH_
    +
    +#include "inclrec.jr.hh"
    +
    +namespace testrec {
    +  class R : public hadoop::Record {
    +
    +  private:
    +
    +    std::vector VF;
    +    inclrec::RI        Rec;
    +    std::string        Buf;
    +
    +  public:
    +
    +    R(void);
    +    virtual ~R(void);
    +
    +    virtual bool operator==(const R& peer) const;
    +    virtual bool operator<(const R& peer) const;
    +
    +    virtual std::vector& getVF(void) const;
    +    virtual const std::vector& getVF(void) const;
    +
    +    virtual std::string& getBuf(void) const ;
    +    virtual const std::string& getBuf(void) const;
    +
    +    virtual inclrec::RI& getRec(void) const;
    +    virtual const inclrec::RI& getRec(void) const;
    +    
    +    virtual bool serialize(hadoop::OutArchive& a) const;
    +    virtual bool deserialize(hadoop::InArchive& a);
    +    
    +    virtual std::string type(void) const;
    +    virtual std::string signature(void) const;
    +  };
    +}; // end namespace testrec
    +#endif /* _TESTREC_JR_HH_ */
    +
    +
    + +

    Java

    + +Code generation for Java is similar to that for C++. A Java class is generated +for each record type with private members corresponding to the fields. Getters +and setters for fields are also generated. Some differences arise in the +way comparison is expressed and in the mapping of modules to packages and +classes to files. For equality testing, an equals method is generated +for each record type. As per Java requirements a hashCode method is also +generated. For comparison a compareTo method is generated for each +record type. This has the semantics as defined by the Java Comparable +interface, that is, the method returns a negative integer, zero, or a positive +integer as the invoked object is less than, equal to, or greater than the +comparison parameter. + +A .java file is generated per record type as opposed to per DDL +file as in C++. The module declaration translates to a Java +package declaration. The module name maps to an identical Java package +name. In addition to this mapping, the DDL compiler creates the appropriate +directory hierarchy for the package and places the generated .java +files in the correct directories. + +

    Mapping Summary

    + +
    
    +DDL Type        C++ Type            Java Type 
    +
    +boolean         bool                boolean
    +byte            int8_t              byte
    +int             int32_t             int
    +long            int64_t             long
    +float           float               float
    +double          double              double
    +ustring         std::string         java.lang.String
    +buffer          std::string         org.apache.hadoop.record.Buffer
    +class type      class type          class type
    +vector    std::vector   java.util.ArrayList
    +map  std::map java.util.TreeMap
    +
    + +

    Data encodings

    + +This section describes the format of the data encodings supported by Hadoop. +Currently, three data encodings are supported, namely binary, CSV and XML. + +

    Binary Serialization Format

    + +The binary data encoding format is fairly dense. Serialization of composite +types is simply defined as a concatenation of serializations of the constituent +elements (lengths are included in vectors and maps). + +Composite types are serialized as follows: +
      +
    • class: Sequence of serialized members. +
    • vector: The number of elements serialized as an int. Followed by a +sequence of serialized elements. +
    • map: The number of key value pairs serialized as an int. Followed +by a sequence of serialized (key,value) pairs. +
    + +Serialization of primitives is more interesting, with a zero compression +optimization for integral types and normalization to UTF-8 for strings. +Primitive types are serialized as follows: + +
      +
    • byte: Represented by 1 byte, as is. +
    • boolean: Represented by 1-byte (0 or 1) +
    • int/long: Integers and longs are serialized zero compressed. +Represented as 1-byte if -120 <= value < 128. Otherwise, serialized as a +sequence of 2-5 bytes for ints, 2-9 bytes for longs. The first byte represents +the number of trailing bytes, N, as the negative number (-120-N). For example, +the number 1024 (0x400) is represented by the byte sequence 'x86 x04 x00'. +This doesn't help much for 4-byte integers but does a reasonably good job with +longs without bit twiddling. +
    • float/double: Serialized in IEEE 754 single and double precision +format in network byte order. This is the format used by Java. +
    • ustring: Serialized as 4-byte zero compressed length followed by +data encoded as UTF-8. Strings are normalized to UTF-8 regardless of native +language representation. +
    • buffer: Serialized as a 4-byte zero compressed length followed by the +raw bytes in the buffer. +
    + + +

    CSV Serialization Format

    + +The CSV serialization format has a lot more structure than the "standard" +Excel CSV format, but we believe the additional structure is useful because + +
      +
    • it makes parsing a lot easier without detracting too much from legibility +
    • the delimiters around composites make it obvious when one is reading a +sequence of Hadoop records +
    + +Serialization formats for the various types are detailed in the grammar that +follows. The notable feature of the formats is the use of delimiters for +indicating the certain field types. + +
      +
    • A string field begins with a single quote ('). +
    • A buffer field begins with a sharp (#). +
    • A class, vector or map begins with 's{', 'v{' or 'm{' respectively and +ends with '}'. +
    + +The CSV format can be described by the following grammar: + +
    
    +record = primitive / struct / vector / map
    +primitive = boolean / int / long / float / double / ustring / buffer
    +
    +boolean = "T" / "F"
    +int = ["-"] 1*DIGIT
    +long = ";" ["-"] 1*DIGIT
    +float = ["-"] 1*DIGIT "." 1*DIGIT ["E" / "e" ["-"] 1*DIGIT]
    +double = ";" ["-"] 1*DIGIT "." 1*DIGIT ["E" / "e" ["-"] 1*DIGIT]
    +
    +ustring = "'" *(UTF8 char except NULL, LF, % and , / "%00" / "%0a" / "%25" / "%2c" )
    +
    +buffer = "#" *(BYTE except NULL, LF, % and , / "%00" / "%0a" / "%25" / "%2c" )
    +
    +struct = "s{" record *("," record) "}"
    +vector = "v{" [record *("," record)] "}"
    +map = "m{" [*(record "," record)] "}"
    +
    + +

    XML Serialization Format

    + +The XML serialization format is the same used by Apache XML-RPC +(http://ws.apache.org/xmlrpc/types.html). This is an extension of the original +XML-RPC format and adds some additional data types. All record I/O types are +not directly expressible in this format, and access to a DDL is required in +order to convert these to valid types. All types primitive or composite are +represented by <value> elements. The particular XML-RPC type is +indicated by a nested element in the <value> element. The encoding for +records is always UTF-8. Primitive types are serialized as follows: + +
      +
    • byte: XML tag <ex:i1>. Values: 1-byte unsigned +integers represented in US-ASCII +
    • boolean: XML tag <boolean>. Values: "0" or "1" +
    • int: XML tags <i4> or <int>. Values: 4-byte +signed integers represented in US-ASCII. +
    • long: XML tag <ex:i8>. Values: 8-byte signed integers +represented in US-ASCII. +
    • float: XML tag <ex:float>. Values: Single precision +floating point numbers represented in US-ASCII. +
    • double: XML tag <double>. Values: Double precision +floating point numbers represented in US-ASCII. +
    • ustring: XML tag <;string>. Values: String values +represented as UTF-8. XML does not permit all Unicode characters in literal +data. In particular, NULLs and control chars are not allowed. Additionally, +XML processors are required to replace carriage returns with line feeds and to +replace CRLF sequences with line feeds. Programming languages that we work +with do not impose these restrictions on string types. To work around these +restrictions, disallowed characters and CRs are percent escaped in strings. +The '%' character is also percent escaped. +
    • buffer: XML tag <string&>. Values: Arbitrary binary +data. Represented as hexBinary, each byte is replaced by its 2-byte +hexadecimal representation. +
    + +Composite types are serialized as follows: + +
      +
    • class: XML tag <struct>. A struct is a sequence of +<member> elements. Each <member> element has a <name> +element and a <value> element. The <name> is a string that must +match /[a-zA-Z][a-zA-Z0-9_]*/. The value of the member is represented +by a <value> element. + +
    • vector: XML tag <array<. An <array> contains a +single <data> element. The <data> element is a sequence of +<value> elements each of which represents an element of the vector. + +
    • map: XML tag <array>. Same as vector. + +
    + +For example: + +
    
    +class {
    +  int           MY_INT;            // value 5
    +  vector MY_VEC;            // values 0.1, -0.89, 2.45e4
    +  buffer        MY_BUF;            // value '\00\n\tabc%'
    +}
    +
    + +is serialized as + +
    
    +<value>
    +  <struct>
    +    <member>
    +      <name>MY_INT</name>
    +      <value><i4>5</i4></value>
    +    </member>
    +    <member>
    +      <name>MY_VEC</name>
    +      <value>
    +        <array>
    +          <data>
    +            <value><ex:float>0.1</ex:float></value>
    +            <value><ex:float>-0.89</ex:float></value>
    +            <value><ex:float>2.45e4</ex:float></value>
    +          </data>
    +        </array>
    +      </value>
    +    </member>
    +    <member>
    +      <name>MY_BUF</name>
    +      <value><string>%00\n\tabc%25</string></value>
    +    </member>
    +  </struct>
    +</value> 
    +
    ]]> +
    +
    + + + + + + + Avro.]]> + + + + + + + + + + + + + + + + + + + + + Avro.]]> + + + + + + + + + + + + Avro.]]> + + + + + + + + + + + + Avro.]]> + + + + + + + + + Avro.]]> + + + + + + + + + + + + Avro.]]> + + + + + + + + + + + + Avro.]]> + + + + + + + + + + + + + + + + + + + + + Avro.]]> + + + + + + + + + + + + Avro.]]> + + + + + + + + + + + + Avro.]]> + + + + + + + + + + + + Avro.]]> + + + + + + + + + + + + Avro.]]> + + + + + + + + + + + + Avro.]]> + + + + + + + + + + + + Avro.]]> + + + + + + + + + Avro.]]> + + + + + + + + + + + + Avro.]]> + + + + + + (DEPRECATED) This package contains classes needed for code generation + from the hadoop record compiler. CppGenerator and JavaGenerator + are the main entry points from the parser. There are classes + corrsponding to every primitive type and compound type + included in Hadoop record I/O syntax. +

    + +

    + DEPRECATED: Replaced by Avro. +

    ]]> +
    +
    + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + This task takes the given record definition files and compiles them into + java or c++ + files. It is then up to the user to compile the generated files. + +

    The task requires the file or the nested fileset element to be + specified. Optional attributes are language (set the output + language, default is "java"), + destdir (name of the destination directory for generated java/c++ + code, default is ".") and failonerror (specifies error handling + behavior. default is true). +

    Usage

    +
    + <recordcc
    +       destdir="${basedir}/gensrc"
    +       language="java">
    +   <fileset include="**\/*.jr" />
    + </recordcc>
    + 
    + + @deprecated Replaced by Avro.]]> +
    +
    + +
    + + + + + + ]]> + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + Avro.]]> + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + Avro.]]> + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + Avro.]]> + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + Avro.]]> + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + Avro.]]> + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + Avro.]]> + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + Avro.]]> + + + + + + (DEPRECATED) This package contains code generated by JavaCC from the + Hadoop record syntax file rcc.jj. For details about the + record file syntax please @see org.apache.hadoop.record. +

    + +

    + DEPRECATED: Replaced by Avro. +

    ]]> +
    +
    + + + + + + + + + + + + + + + + + + + + + + + + + + + + + Avro.]]> + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + Avro.]]> + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + Avro.]]> + + + + + + + + + + + + + + + + + + + Avro.]]> + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + Avro.]]> + + + + + + + + + + + + + + + + Avro.]]> + + + + + + + + + + + + + + + + + + + + + + Avro.]]> + + + + + + + + + + + + + + + + + + + + (cause==null ? null : cause.toString()) (which + typically contains the class and detail message of cause). + @param cause the cause (which is saved for later retrieval by the + {@link #getCause()} method). (A null value is + permitted, and indicates that the cause is nonexistent or + unknown.)]]> + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + mapping + and mapping]]> + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + /host@realm. + @param principalName principal name of format as described above + @return host name if the the string conforms to the above format, else null]]> + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + "jack" + + @param userName + @return userName without login method]]> + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + the return type of the run method + @param action the method to execute + @return the value from the run method]]> + + + + + + + + the return type of the run method + @param action the method to execute + @return the value from the run method + @throws IOException if the action throws an IOException + @throws Error if the action throws an Error + @throws RuntimeException if the action throws a RuntimeException + @throws InterruptedException if the action throws an InterruptedException + @throws UndeclaredThrowableException if the action throws something else]]> + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + (cause==null ? null : cause.toString()) (which + typically contains the class and detail message of cause). + @param cause the cause (which is saved for later retrieval by the + {@link #getCause()} method). (A null value is + permitted, and indicates that the cause is nonexistent or + unknown.)]]> + + + + + + + + + + + + + + does not provide the stack trace for security purposes.]]> + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + A User-Agent String is considered to be a browser if it matches + any of the regex patterns from browser-useragent-regex; the default + behavior is to consider everything a browser that matches the following: + "^Mozilla.*,^Opera.*". Subclasses can optionally override + this method to use different behavior. + + @param userAgent The User-Agent String, or null if there isn't one + @return true if the User-Agent String refers to a browser, false if not]]> + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + The type of the token identifier]]> + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + T extends TokenIdentifier]]> + + + + + + + + + + DelegationTokenAuthenticatedURL. +

    + An instance of the default {@link DelegationTokenAuthenticator} will be + used.]]> + + + + + DelegationTokenAuthenticatedURL. + + @param authenticator the {@link DelegationTokenAuthenticator} instance to + use, if null the default one will be used.]]> + + + + + DelegationTokenAuthenticatedURL using the default + {@link DelegationTokenAuthenticator} class. + + @param connConfigurator a connection configurator.]]> + + + + + DelegationTokenAuthenticatedURL. + + @param authenticator the {@link DelegationTokenAuthenticator} instance to + use, if null the default one will be used. + @param connConfigurator a connection configurator.]]> + + + + + + + + + + + + The default class is {@link KerberosDelegationTokenAuthenticator} + + @return the delegation token authenticator class to use as default.]]> + + + + + + + This method is provided to enable WebHDFS backwards compatibility. + + @param useQueryString TRUE if the token is transmitted in the + URL query string, FALSE if the delegation token is transmitted + using the {@link DelegationTokenAuthenticator#DELEGATION_TOKEN_HEADER} HTTP + header.]]> + + + + + TRUE if the token is transmitted in the URL query + string, FALSE if the delegation token is transmitted using the + {@link DelegationTokenAuthenticator#DELEGATION_TOKEN_HEADER} HTTP header.]]> + + + + + + + + + + + + + + + + + + Authenticator. + + @param url the URL to connect to. Only HTTP/S URLs are supported. + @param token the authentication token being used for the user. + @return an authenticated {@link HttpURLConnection}. + @throws IOException if an IO error occurred. + @throws AuthenticationException if an authentication exception occurred.]]> + + + + + + + + + + Authenticator. If the doAs parameter is not NULL, + the request will be done on behalf of the specified doAs user. + + @param url the URL to connect to. Only HTTP/S URLs are supported. + @param token the authentication token being used for the user. + @param doAs user to do the the request on behalf of, if NULL the request is + as self. + @return an authenticated {@link HttpURLConnection}. + @throws IOException if an IO error occurred. + @throws AuthenticationException if an authentication exception occurred.]]> + + + + + + + + + + Authenticator + for authentication. + + @param url the URL to get the delegation token from. Only HTTP/S URLs are + supported. + @param token the authentication token being used for the user where the + Delegation token will be stored. + @param renewer the renewer user. + @return a delegation token. + @throws IOException if an IO error occurred. + @throws AuthenticationException if an authentication exception occurred.]]> + + + + + + + + + + + Authenticator + for authentication. + + @param url the URL to get the delegation token from. Only HTTP/S URLs are + supported. + @param token the authentication token being used for the user where the + Delegation token will be stored. + @param renewer the renewer user. + @param doAsUser the user to do as, which will be the token owner. + @return a delegation token. + @throws IOException if an IO error occurred. + @throws AuthenticationException if an authentication exception occurred.]]> + + + + + + + + + Authenticator for authentication. + + @param url the URL to renew the delegation token from. Only HTTP/S URLs are + supported. + @param token the authentication token with the Delegation Token to renew. + @throws IOException if an IO error occurred. + @throws AuthenticationException if an authentication exception occurred.]]> + + + + + + + + + + Authenticator for authentication. + + @param url the URL to renew the delegation token from. Only HTTP/S URLs are + supported. + @param token the authentication token with the Delegation Token to renew. + @param doAsUser the user to do as, which will be the token owner. + @throws IOException if an IO error occurred. + @throws AuthenticationException if an authentication exception occurred.]]> + + + + + + + + Authenticator. + + @param url the URL to cancel the delegation token from. Only HTTP/S URLs + are supported. + @param token the authentication token with the Delegation Token to cancel. + @throws IOException if an IO error occurred.]]> + + + + + + + + + Authenticator. + + @param url the URL to cancel the delegation token from. Only HTTP/S URLs + are supported. + @param token the authentication token with the Delegation Token to cancel. + @param doAsUser the user to do as, which will be the token owner. + @throws IOException if an IO error occurred.]]> + + + + DelegationTokenAuthenticatedURL is a + {@link AuthenticatedURL} sub-class with built-in Hadoop Delegation Token + functionality. +

    + The authentication mechanisms supported by default are Hadoop Simple + authentication (also known as pseudo authentication) and Kerberos SPNEGO + authentication. +

    + Additional authentication mechanisms can be supported via {@link + DelegationTokenAuthenticator} implementations. +

    + The default {@link DelegationTokenAuthenticator} is the {@link + KerberosDelegationTokenAuthenticator} class which supports + automatic fallback from Kerberos SPNEGO to Hadoop Simple authentication via + the {@link PseudoDelegationTokenAuthenticator} class. +

    + AuthenticatedURL instances are not thread-safe.]]> + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + Authenticator + for authentication. + + @param url the URL to get the delegation token from. Only HTTP/S URLs are + supported. + @param token the authentication token being used for the user where the + Delegation token will be stored. + @param renewer the renewer user. + @throws IOException if an IO error occurred. + @throws AuthenticationException if an authentication exception occurred.]]> + + + + + + + + + + + Authenticator + for authentication. + + @param url the URL to get the delegation token from. Only HTTP/S URLs are + supported. + @param token the authentication token being used for the user where the + Delegation token will be stored. + @param renewer the renewer user. + @param doAsUser the user to do as, which will be the token owner. + @throws IOException if an IO error occurred. + @throws AuthenticationException if an authentication exception occurred.]]> + + + + + + + + + + Authenticator for authentication. + + @param url the URL to renew the delegation token from. Only HTTP/S URLs are + supported. + @param token the authentication token with the Delegation Token to renew. + @throws IOException if an IO error occurred. + @throws AuthenticationException if an authentication exception occurred.]]> + + + + + + + + + + + Authenticator for authentication. + + @param url the URL to renew the delegation token from. Only HTTP/S URLs are + supported. + @param token the authentication token with the Delegation Token to renew. + @param doAsUser the user to do as, which will be the token owner. + @throws IOException if an IO error occurred. + @throws AuthenticationException if an authentication exception occurred.]]> + + + + + + + + + Authenticator. + + @param url the URL to cancel the delegation token from. Only HTTP/S URLs + are supported. + @param token the authentication token with the Delegation Token to cancel. + @throws IOException if an IO error occurred.]]> + + + + + + + + + + Authenticator. + + @param url the URL to cancel the delegation token from. Only HTTP/S URLs + are supported. + @param token the authentication token with the Delegation Token to cancel. + @param doAsUser the user to do as, which will be the token owner. + @throws IOException if an IO error occurred.]]> + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + KerberosDelegationTokenAuthenticator provides support for + Kerberos SPNEGO authentication mechanism and support for Hadoop Delegation + Token operations. +

    + It falls back to the {@link PseudoDelegationTokenAuthenticator} if the HTTP + endpoint does not trigger a SPNEGO authentication]]> + + + + + + + + + PseudoDelegationTokenAuthenticator provides support for + Hadoop's pseudo authentication mechanism that accepts + the user name specified as a query string parameter and support for Hadoop + Delegation Token operations. +

    + This mimics the model of Hadoop Simple authentication trusting the + {@link UserGroupInformation#getCurrentUser()} value.]]> + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + live. + @return a (snapshotted) map of blocker name->description values]]> + + + + + + + + + + + + + Do nothing if the service is null or not + in a state in which it can be/needs to be stopped. +

    + The service state is checked before the operation begins. + This process is not thread safe. + @param service a service or null]]> + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + +

  • Any long-lived operation here will prevent the service state + change from completing in a timely manner.
  • +
  • If another thread is somehow invoked from the listener, and + that thread invokes the methods of the service (including + subclass-specific methods), there is a risk of a deadlock.
  • + + + + @param service the service that has changed.]]> +
    + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + +
    + + + + + + + + + + + + + + + + The base implementation logs all arguments at the debug level, + then returns the passed in config unchanged.]]> + + + + + + + The action is to signal success by returning the exit code 0.]]> + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + This method is called before {@link #init(Configuration)}; + Any non-null configuration that is returned from this operation + becomes the one that is passed on to that {@link #init(Configuration)} + operation. +

    + This permits implementations to change the configuration before + the init operation. As the ServiceLauncher only creates + an instance of the base {@link Configuration} class, it is + recommended to instantiate any subclass (such as YarnConfiguration) + that injects new resources. +

    + @param config the initial configuration build up by the + service launcher. + @param args list of arguments passed to the command line + after any launcher-specific commands have been stripped. + @return the configuration to init the service with. + Recommended: pass down the config parameter with any changes + @throws Exception any problem]]> + + + + + + + The return value becomes the exit code of the launched process. +

    + If an exception is raised, the policy is: +

      +
    1. Any subset of {@link org.apache.hadoop.util.ExitUtil.ExitException}: + the exception is passed up unmodified. +
    2. +
    3. Any exception which implements + {@link org.apache.hadoop.util.ExitCodeProvider}: + A new {@link ServiceLaunchException} is created with the exit code + and message of the thrown exception; the thrown exception becomes the + cause.
    4. +
    5. Any other exception: a new {@link ServiceLaunchException} is created + with the exit code {@link LauncherExitCodes#EXIT_EXCEPTION_THROWN} and + the message of the original exception (which becomes the cause).
    6. +
    + @return the exit code + @throws org.apache.hadoop.util.ExitUtil.ExitException an exception passed + up as the exit code and error text. + @throws Exception any exception to report. If it provides an exit code + this is used in a wrapping exception.]]> +
    +
    + + + The command line options will be passed down before the + {@link Service#init(Configuration)} operation is invoked via an + invocation of {@link LaunchableService#bindArgs(Configuration, List)} + After the service has been successfully started via {@link Service#start()} + the {@link LaunchableService#execute()} method is called to execute the + service. When this method returns, the service launcher will exit, using + the return code from the method as its exit option.]]> + +
    + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + Approximate HTTP equivalent: {@code 400 Bad Request}]]> + + + + + + approximate HTTP equivalent: Approximate HTTP equivalent: {@code 401 Unauthorized}]]> + + + + + + + + + + + Approximate HTTP equivalent: Approximate HTTP equivalent: {@code 403: Forbidden}]]> + + + + + + Approximate HTTP equivalent: {@code 404: Not Found}]]> + + + + + + Approximate HTTP equivalent: {@code 405: Not allowed}]]> + + + + + + Approximate HTTP equivalent: {@code 406: Not Acceptable}]]> + + + + + + Approximate HTTP equivalent: {@code 408: Request Timeout}]]> + + + + + + Approximate HTTP equivalent: {@code 409: Conflict}]]> + + + + + + Approximate HTTP equivalent: {@code 500 Internal Server Error}]]> + + + + + + Approximate HTTP equivalent: {@code 501: Not Implemented}]]> + + + + + + Approximate HTTP equivalent: {@code 503 Service Unavailable}]]> + + + + + + If raised, this is expected to be raised server-side and likely due + to client/server version incompatibilities. +

    + Approximate HTTP equivalent: {@code 505: Version Not Supported}]]> + + + + + + + + + + + + + + + Codes with a YARN prefix are YARN-related. +

    + Many of the exit codes are designed to resemble HTTP error codes, + squashed into a single byte. e.g 44 , "not found" is the equivalent + of 404. The various 2XX HTTP error codes aren't followed; + the Unix standard of "0" for success is used. +

    +    0-10: general command issues
    +   30-39: equivalent to the 3XX responses, where those responses are
    +          considered errors by the application.
    +   40-49: client-side/CLI/config problems
    +   50-59: service-side problems.
    +   60+  : application specific error codes
    + 
    ]]> +
    +
    + + + + + + + + + + + + + + + + + + + This uses {@link String#format(String, Object...)} + to build the formatted exception in the ENGLISH locale. +

    + If the last argument is a throwable, it becomes the cause of the exception. + It will also be used as a parameter for the format. + @param exitCode exit code + @param format format for message to use in exception + @param args list of arguments]]> + + + + + When caught by the ServiceLauncher, it will convert that + into a process exit code. + + The {@link #ServiceLaunchException(int, String, Object...)} constructor + generates formatted exceptions.]]> + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + Clients and/or applications can use the provided Progressable + to explicitly report progress to the Hadoop framework. This is especially + important for operations which take significant amount of time since, + in-lieu of the reported progress, the framework has to assume that an error + has occured and time-out the operation.

    ]]> +
    + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + Class is to be obtained + @return the correctly typed Class of the given object.]]> + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + kill -0 command or equivalent]]> + + + + + + + + + + + + + + + + + + + ".cmd" on Windows, or ".sh" otherwise. + + @param parent File parent directory + @param basename String script file basename + @return File referencing the script in the directory]]> + + + + + + ".cmd" on Windows, or ".sh" otherwise. + + @param basename String script file basename + @return String script file name]]> + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + IOException. + @return the path to {@link #WINUTILS_EXE} + @throws RuntimeException if the path is not resolvable]]> + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + Shell. + @return the thread that ran runCommand() that spawned this shell + or null if no thread is waiting for this shell to complete]]> + + + + + + + + + + + + Shell interface. + @param cmd shell command to execute. + @return the output of the executed command.]]> + + + + + + + + + Shell interface. + @param env the map of environment key=value + @param cmd shell command to execute. + @param timeout time in milliseconds after which script should be marked timeout + @return the output of the executed command. + @throws IOException on any problem.]]> + + + + + + + + Shell interface. + @param env the map of environment key=value + @param cmd shell command to execute. + @return the output of the executed command. + @throws IOException on any problem.]]> + + + + + Shell processes. + Iterates through a map of all currently running Shell + processes and destroys them one by one. This method is thread safe]]> + + + + + Shell objects.]]> + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + CreateProcess synchronization object.]]> + + + + + os.name property.]]> + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + Important: caller must check for this value being null. + The lack of such checks has led to many support issues being raised. +

    + @deprecated use one of the exception-raising getter methods, + specifically {@link #getWinUtilsPath()} or {@link #getWinUtilsFile()}]]> + + + + + + + + + + + + + + Shell can be used to run shell commands like du or + df. It also offers facilities to gate commands by + time-intervals.]]> + + + + + + + + ShutdownHookManager singleton. + + @return ShutdownHookManager singleton.]]> + + + + + + + Runnable + @param priority priority of the shutdownHook.]]> + + + + + + + + + Runnable + @param priority priority of the shutdownHook + @param timeout timeout of the shutdownHook + @param unit unit of the timeout TimeUnit]]> + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + ShutdownHookManager enables running shutdownHook + in a deterministic order, higher priority first. +

    + The JVM runs ShutdownHooks in a non-deterministic order or in parallel. + This class registers a single JVM shutdownHook and run all the + shutdownHooks registered to it (to this class) in order based on their + priority. + + Unless a hook was registered with a shutdown explicitly set through + {@link #addShutdownHook(Runnable, int, long, TimeUnit)}, + the shutdown time allocated to it is set by the configuration option + {@link CommonConfigurationKeysPublic#SERVICE_SHUTDOWN_TIMEOUT} in + {@code core-site.xml}, with a default value of + {@link CommonConfigurationKeysPublic#SERVICE_SHUTDOWN_TIMEOUT_DEFAULT} + seconds.]]> + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + Tool, is the standard for any Map-Reduce tool/application. + The tool/application should delegate the handling of + + standard command-line options to {@link ToolRunner#run(Tool, String[])} + and only handle its custom arguments.

    + +

    Here is how a typical Tool is implemented:

    +

    +     public class MyApp extends Configured implements Tool {
    +     
    +       public int run(String[] args) throws Exception {
    +         // Configuration processed by ToolRunner
    +         Configuration conf = getConf();
    +         
    +         // Create a JobConf using the processed conf
    +         JobConf job = new JobConf(conf, MyApp.class);
    +         
    +         // Process custom command-line options
    +         Path in = new Path(args[1]);
    +         Path out = new Path(args[2]);
    +         
    +         // Specify various job-specific parameters     
    +         job.setJobName("my-app");
    +         job.setInputPath(in);
    +         job.setOutputPath(out);
    +         job.setMapperClass(MyMapper.class);
    +         job.setReducerClass(MyReducer.class);
    +
    +         // Submit the job, then poll for progress until the job is complete
    +         RunningJob runningJob = JobClient.runJob(job);
    +         if (runningJob.isSuccessful()) {
    +           return 0;
    +         } else {
    +           return 1;
    +         }
    +       }
    +       
    +       public static void main(String[] args) throws Exception {
    +         // Let ToolRunner handle generic command-line options 
    +         int res = ToolRunner.run(new Configuration(), new MyApp(), args);
    +         
    +         System.exit(res);
    +       }
    +     }
    + 

    + + @see GenericOptionsParser + @see ToolRunner]]> +
    + + + + + + + + + + + + + Tool by {@link Tool#run(String[])}, after + parsing with the given generic arguments. Uses the given + Configuration, or builds one if null. + + Sets the Tool's configuration with the possibly modified + version of the conf. + + @param conf Configuration for the Tool. + @param tool Tool to run. + @param args command-line arguments to the tool. + @return exit code of the {@link Tool#run(String[])} method.]]> + + + + + + + + Tool with its Configuration. + + Equivalent to run(tool.getConf(), tool, args). + + @param tool Tool to run. + @param args command-line arguments to the tool. + @return exit code of the {@link Tool#run(String[])} method.]]> + + + + + + + + + + + + + + + + + ToolRunner can be used to run classes implementing + Tool interface. It works in conjunction with + {@link GenericOptionsParser} to parse the + + generic hadoop command line arguments and modifies the + Configuration of the Tool. The + application-specific options are passed along without being modified. +

    + + @see Tool + @see GenericOptionsParser]]> +
    +
    + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + +
    + + + + + + + + + + + this filter. + @param nbHash The number of hash function to consider. + @param hashType type of the hashing function (see + {@link org.apache.hadoop.util.hash.Hash}).]]> + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + Bloom filter, as defined by Bloom in 1970. +

    + The Bloom filter is a data structure that was introduced in 1970 and that has been adopted by + the networking research community in the past decade thanks to the bandwidth efficiencies that it + offers for the transmission of set membership information between networked hosts. A sender encodes + the information into a bit vector, the Bloom filter, that is more compact than a conventional + representation. Computation and space costs for construction are linear in the number of elements. + The receiver uses the filter to test whether various elements are members of the set. Though the + filter will occasionally return a false positive, it will never return a false negative. When creating + the filter, the sender can choose its desired point in a trade-off between the false positive rate and the size. + +

    + Originally created by + European Commission One-Lab Project 034819. + + @see Filter The general behavior of a filter + + @see Space/Time Trade-Offs in Hash Coding with Allowable Errors]]> + + + + + + + + + + + + + this filter. + @param nbHash The number of hash function to consider. + @param hashType type of the hashing function (see + {@link org.apache.hadoop.util.hash.Hash}).]]> + + + + + + + + + this counting Bloom filter. +

    + Invariant: nothing happens if the specified key does not belong to this counter Bloom filter. + @param key The key to remove.]]> + + + + + + + + + + + + key -> count map. +

    NOTE: due to the bucket size of this filter, inserting the same + key more than 15 times will cause an overflow at all filter positions + associated with this key, and it will significantly increase the error + rate for this and other keys. For this reason the filter can only be + used to store small count values 0 <= N << 15. + @param key key to be tested + @return 0 if the key is not present. Otherwise, a positive value v will + be returned such that v == count with probability equal to the + error rate of this filter, and v > count otherwise. + Additionally, if the filter experienced an underflow as a result of + {@link #delete(Key)} operation, the return value may be lower than the + count with the probability of the false negative rate of such + filter.]]> + + + + + + + + + + + + + + + + + + + + + + counting Bloom filter, as defined by Fan et al. in a ToN + 2000 paper. +

    + A counting Bloom filter is an improvement to standard a Bloom filter as it + allows dynamic additions and deletions of set membership information. This + is achieved through the use of a counting vector instead of a bit vector. +

    + Originally created by + European Commission One-Lab Project 034819. + + @see Filter The general behavior of a filter + + @see Summary cache: a scalable wide-area web cache sharing protocol]]> + + + + + + + + + + + + + + Builds an empty Dynamic Bloom filter. + @param vectorSize The number of bits in the vector. + @param nbHash The number of hash function to consider. + @param hashType type of the hashing function (see + {@link org.apache.hadoop.util.hash.Hash}). + @param nr The threshold for the maximum number of keys to record in a + dynamic Bloom filter row.]]> + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + dynamic Bloom filter, as defined in the INFOCOM 2006 paper. +

    + A dynamic Bloom filter (DBF) makes use of a s * m bit matrix but + each of the s rows is a standard Bloom filter. The creation + process of a DBF is iterative. At the start, the DBF is a 1 * m + bit matrix, i.e., it is composed of a single standard Bloom filter. + It assumes that nr elements are recorded in the + initial bit vector, where nr <= n (n is + the cardinality of the set A to record in the filter). +

    + As the size of A grows during the execution of the application, + several keys must be inserted in the DBF. When inserting a key into the DBF, + one must first get an active Bloom filter in the matrix. A Bloom filter is + active when the number of recorded keys, nr, is + strictly less than the current cardinality of A, n. + If an active Bloom filter is found, the key is inserted and + nr is incremented by one. On the other hand, if there + is no active Bloom filter, a new one is created (i.e., a new row is added to + the matrix) according to the current size of A and the element + is added in this new Bloom filter and the nr value of + this new Bloom filter is set to one. A given key is said to belong to the + DBF if the k positions are set to one in one of the matrix rows. +

    + Originally created by + European Commission One-Lab Project 034819. + + @see Filter The general behavior of a filter + @see BloomFilter A Bloom filter + + @see Theory and Network Applications of Dynamic Bloom Filters]]> + + + + + + + + + Builds a hash function that must obey to a given maximum number of returned values and a highest value. + @param maxValue The maximum highest returned value. + @param nbHash The number of resulting hashed values. + @param hashType type of the hashing function (see {@link Hash}).]]> + + + + + this hash function. A NOOP]]> + + + + + + + + + + + + + + + + + + + The idea is to randomly select a bit to reset.]]> + + + + + + The idea is to select the bit to reset that will generate the minimum + number of false negative.]]> + + + + + + The idea is to select the bit to reset that will remove the maximum number + of false positive.]]> + + + + + + The idea is to select the bit to reset that will, at the same time, remove + the maximum number of false positve while minimizing the amount of false + negative generated.]]> + + + + + Originally created by + European Commission One-Lab Project 034819.]]> + + + + + + + + + + + + + + this filter. + @param nbHash The number of hash function to consider. + @param hashType type of the hashing function (see + {@link org.apache.hadoop.util.hash.Hash}).]]> + + + + + + + + + this retouched Bloom filter. +

    + Invariant: if the false positive is null, nothing happens. + @param key The false positive key to add.]]> + + + + + + this retouched Bloom filter. + @param coll The collection of false positive.]]> + + + + + + this retouched Bloom filter. + @param keys The list of false positive.]]> + + + + + + this retouched Bloom filter. + @param keys The array of false positive.]]> + + + + + + + this retouched Bloom filter. + @param scheme The selective clearing scheme to apply.]]> + + + + + + + + + + + + retouched Bloom filter, as defined in the CoNEXT 2006 paper. +

    + It allows the removal of selected false positives at the cost of introducing + random false negatives, and with the benefit of eliminating some random false + positives at the same time. + +

    + Originally created by + European Commission One-Lab Project 034819. + + @see Filter The general behavior of a filter + @see BloomFilter A Bloom filter + @see RemoveScheme The different selective clearing algorithms + + @see Retouched Bloom Filters: Allowing Networked Applications to Trade Off Selected False Positives Against False Negatives]]> + + + + + + + + + + diff --git a/hadoop-common-project/hadoop-common/dev-support/jdiff/Apache_Hadoop_Common_3.3.3.xml b/hadoop-common-project/hadoop-common/dev-support/jdiff/Apache_Hadoop_Common_3.3.3.xml new file mode 100644 index 0000000000000..448df9ddd686b --- /dev/null +++ b/hadoop-common-project/hadoop-common/dev-support/jdiff/Apache_Hadoop_Common_3.3.3.xml @@ -0,0 +1,39037 @@ + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + UnsupportedOperationException + + If a key is deprecated in favor of multiple keys, they are all treated as + aliases of each other, and setting any one of them resets all the others + to the new value. + + If you have multiple deprecation entries to add, it is more efficient to + use #addDeprecations(DeprecationDelta[] deltas) instead. + + @param key to be deprecated + @param newKeys list of keys that take up the values of deprecated key + @param customMessage depcrication message + @deprecated use {@link #addDeprecation(String key, String newKey, + String customMessage)} instead]]> + + + + + + + + UnsupportedOperationException + + If you have multiple deprecation entries to add, it is more efficient to + use #addDeprecations(DeprecationDelta[] deltas) instead. + + @param key to be deprecated + @param newKey key that take up the values of deprecated key + @param customMessage deprecation message]]> + + + + + + + UnsupportedOperationException + + If a key is deprecated in favor of multiple keys, they are all treated as + aliases of each other, and setting any one of them resets all the others + to the new value. + + If you have multiple deprecation entries to add, it is more efficient to + use #addDeprecations(DeprecationDelta[] deltas) instead. + + @param key Key that is to be deprecated + @param newKeys list of keys that take up the values of deprecated key + @deprecated use {@link #addDeprecation(String key, String newKey)} instead]]> + + + + + + + UnsupportedOperationException + + If you have multiple deprecation entries to add, it is more efficient to + use #addDeprecations(DeprecationDelta[] deltas) instead. + + @param key Key that is to be deprecated + @param newKey key that takes up the value of deprecated key]]> + + + + + + key is deprecated. + + @param key the parameter which is to be checked for deprecation + @return true if the key is deprecated and + false otherwise.]]> + + + + + + + + + + + + + + + + + + + + + + + + + + + + final. + + @param name resource to be added, the classpath is examined for a file + with that name.]]> + + + + + + + + + + final. + + @param url url of the resource to be added, the local filesystem is + examined directly to find the resource, without referring to + the classpath.]]> + + + + + + + + + + final. + + @param file file-path of resource to be added, the local filesystem is + examined directly to find the resource, without referring to + the classpath.]]> + + + + + + + + + + final. + + WARNING: The contents of the InputStream will be cached, by this method. + So use this sparingly because it does increase the memory consumption. + + @param in InputStream to deserialize the object from. In will be read from + when a get or set is called next. After it is read the stream will be + closed.]]> + + + + + + + + + + + final. + + @param in InputStream to deserialize the object from. + @param name the name of the resource because InputStream.toString is not + very descriptive some times.]]> + + + + + + + + + + + final. + + @param conf Configuration object from which to load properties]]> + + + + + + + + + + + name property, null if + no such property exists. If the key is deprecated, it returns the value of + the first key which replaces the deprecated key and is not null. + + Values are processed for variable expansion + before being returned. + + As a side effect get loads the properties from the sources if called for + the first time as a lazy init. + + @param name the property name, will be trimmed before get value. + @return the value of the name or its replacing property, + or null if no such property exists.]]> + + + + + + + + + + + + + + + name property, but only for + names which have no valid value, usually non-existent or commented + out in XML. + + @param name the property name + @return true if the property name exists without value]]> + + + + + + name property as a trimmed String, + null if no such property exists. + If the key is deprecated, it returns the value of + the first key which replaces the deprecated key and is not null + + Values are processed for variable expansion + before being returned. + + @param name the property name. + @return the value of the name or its replacing property, + or null if no such property exists.]]> + + + + + + + name property as a trimmed String, + defaultValue if no such property exists. + See @{Configuration#getTrimmed} for more details. + + @param name the property name. + @param defaultValue the property default value. + @return the value of the name or defaultValue + if it is not set.]]> + + + + + + name property, without doing + variable expansion.If the key is + deprecated, it returns the value of the first key which replaces + the deprecated key and is not null. + + @param name the property name. + @return the value of the name property or + its replacing property and null if no such property exists.]]> + + + + + + + value of the name property. If + name is deprecated or there is a deprecated name associated to it, + it sets the value to both names. Name will be trimmed before put into + configuration. + + @param name property name. + @param value property value.]]> + + + + + + + + value of the name property. If + name is deprecated, it also sets the value to + the keys that replace the deprecated key. Name will be trimmed before put + into configuration. + + @param name property name. + @param value property value. + @param source the place that this configuration value came from + (For debugging). + @throws IllegalArgumentException when the value or name is null.]]> + + + + + + + + + + + + + + + + + + + + name. If the key is deprecated, + it returns the value of the first key which replaces the deprecated key + and is not null. + If no such property exists, + then defaultValue is returned. + + @param name property name, will be trimmed before get value. + @param defaultValue default value. + @return property value, or defaultValue if the property + doesn't exist.]]> + + + + + + + name property as an int. + + If no such property exists, the provided default value is returned, + or if the specified value is not a valid int, + then an error is thrown. + + @param name property name. + @param defaultValue default value. + @throws NumberFormatException when the value is invalid + @return property value as an int, + or defaultValue.]]> + + + + + + name property as a set of comma-delimited + int values. + + If no such property exists, an empty array is returned. + + @param name property name + @return property value interpreted as an array of comma-delimited + int values]]> + + + + + + + name property to an int. + + @param name property name. + @param value int value of the property.]]> + + + + + + + name property as a long. + If no such property exists, the provided default value is returned, + or if the specified value is not a valid long, + then an error is thrown. + + @param name property name. + @param defaultValue default value. + @throws NumberFormatException when the value is invalid + @return property value as a long, + or defaultValue.]]> + + + + + + + name property as a long or + human readable format. If no such property exists, the provided default + value is returned, or if the specified value is not a valid + long or human readable format, then an error is thrown. You + can use the following suffix (case insensitive): k(kilo), m(mega), g(giga), + t(tera), p(peta), e(exa) + + @param name property name. + @param defaultValue default value. + @throws NumberFormatException when the value is invalid + @return property value as a long, + or defaultValue.]]> + + + + + + + name property to a long. + + @param name property name. + @param value long value of the property.]]> + + + + + + + name property as a float. + If no such property exists, the provided default value is returned, + or if the specified value is not a valid float, + then an error is thrown. + + @param name property name. + @param defaultValue default value. + @throws NumberFormatException when the value is invalid + @return property value as a float, + or defaultValue.]]> + + + + + + + name property to a float. + + @param name property name. + @param value property value.]]> + + + + + + + name property as a double. + If no such property exists, the provided default value is returned, + or if the specified value is not a valid double, + then an error is thrown. + + @param name property name. + @param defaultValue default value. + @throws NumberFormatException when the value is invalid + @return property value as a double, + or defaultValue.]]> + + + + + + + name property to a double. + + @param name property name. + @param value property value.]]> + + + + + + + name property as a boolean. + If no such property is specified, or if the specified value is not a valid + boolean, then defaultValue is returned. + + @param name property name. + @param defaultValue default value. + @return property value as a boolean, + or defaultValue.]]> + + + + + + + name property to a boolean. + + @param name property name. + @param value boolean value of the property.]]> + + + + + + + + + + + + + + name property to the given type. This + is equivalent to set(<name>, value.toString()). + @param name property name + @param value new value + @param enumeration type]]> + + + + + + + enumeration type + @throws IllegalArgumentException If mapping is illegal for the type + provided + @return enumeration type]]> + + + + + + + + name to the given time duration. This + is equivalent to set(<name>, value + <time suffix>). + @param name Property name + @param value Time duration + @param unit Unit of time]]> + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + name property as a Pattern. + If no such property is specified, or if the specified value is not a valid + Pattern, then DefaultValue is returned. + Note that the returned value is NOT trimmed by this method. + + @param name property name + @param defaultValue default value + @return property value as a compiled Pattern, or defaultValue]]> + + + + + + + Pattern. + If the pattern is passed as null, sets the empty pattern which results in + further calls to getPattern(...) returning the default value. + + @param name property name + @param pattern new value]]> + + + + + + + + + + + + + + + + + + + name property as + a collection of Strings. + If no such property is specified then empty collection is returned. +

    + This is an optimized version of {@link #getStrings(String)} + + @param name property name. + @return property value as a collection of Strings.]]> + + + + + + name property as + an array of Strings. + If no such property is specified then null is returned. + + @param name property name. + @return property value as an array of Strings, + or null.]]> + + + + + + + name property as + an array of Strings. + If no such property is specified then default value is returned. + + @param name property name. + @param defaultValue The default value + @return property value as an array of Strings, + or default value.]]> + + + + + + name property as + a collection of Strings, trimmed of the leading and trailing whitespace. + If no such property is specified then empty Collection is returned. + + @param name property name. + @return property value as a collection of Strings, or empty Collection]]> + + + + + + name property as + an array of Strings, trimmed of the leading and trailing whitespace. + If no such property is specified then an empty array is returned. + + @param name property name. + @return property value as an array of trimmed Strings, + or empty array.]]> + + + + + + + name property as + an array of Strings, trimmed of the leading and trailing whitespace. + If no such property is specified then default value is returned. + + @param name property name. + @param defaultValue The default value + @return property value as an array of trimmed Strings, + or default value.]]> + + + + + + + name property as + as comma delimited values. + + @param name property name. + @param values The values]]> + + + + + + + + + + + + + + + + + + + + + + + + + + + + + hostProperty as a + InetSocketAddress. If hostProperty is + null, addressProperty will be used. This + is useful for cases where we want to differentiate between host + bind address and address clients should use to establish connection. + + @param hostProperty bind host property name. + @param addressProperty address property name. + @param defaultAddressValue the default value + @param defaultPort the default port + @return InetSocketAddress]]> + + + + + + + + name property as a + InetSocketAddress. + @param name property name. + @param defaultAddress the default value + @param defaultPort the default port + @return InetSocketAddress]]> + + + + + + + name property as + a host:port.]]> + + + + + + + + + name property as a host:port. The wildcard + address is replaced with the local host's address. If the host and address + properties are configured the host component of the address will be combined + with the port component of the addr to generate the address. This is to allow + optional control over which host name is used in multi-home bind-host + cases where a host can have multiple names + @param hostProperty the bind-host configuration name + @param addressProperty the service address configuration name + @param defaultAddressValue the service default address configuration value + @param addr InetSocketAddress of the service listener + @return InetSocketAddress for clients to connect]]> + + + + + + + name property as a host:port. The wildcard + address is replaced with the local host's address. + @param name property name. + @param addr InetSocketAddress of a listener to store in the given property + @return InetSocketAddress for clients to connect]]> + + + + + + + + + + + + + + + + + + + + name property + as an array of Class. + The value of the property specifies a list of comma separated class names. + If no such property is specified, then defaultValue is + returned. + + @param name the property name. + @param defaultValue default value. + @return property value as a Class[], + or defaultValue.]]> + + + + + + + name property as a Class. + If no such property is specified, then defaultValue is + returned. + + @param name the conf key name. + @param defaultValue default value. + @return property value as a Class, + or defaultValue.]]> + + + + + + + + name property as a Class + implementing the interface specified by xface. + + If no such property is specified, then defaultValue is + returned. + + An exception is thrown if the returned class does not implement the named + interface. + + @param name the conf key name. + @param defaultValue default value. + @param xface the interface implemented by the named class. + @return property value as a Class, + or defaultValue.]]> + + + + + + + name property as a List + of objects implementing the interface specified by xface. + + An exception is thrown if any of the classes does not exist, or if it does + not implement the named interface. + + @param name the property name. + @param xface the interface implemented by the classes named by + name. + @return a List of objects implementing xface.]]> + + + + + + + + name property to the name of a + theClass implementing the given interface xface. + + An exception is thrown if theClass does not implement the + interface xface. + + @param name property name. + @param theClass property value. + @param xface the interface implemented by the named class.]]> + + + + + + + + dirsProp with + the given path. If dirsProp contains multiple directories, + then one is chosen based on path's hash code. If the selected + directory does not exist, an attempt is made to create it. + + @param dirsProp directory in which to locate the file. + @param path file-path. + @return local file under the directory with the given path.]]> + + + + + + + + dirsProp with + the given path. If dirsProp contains multiple directories, + then one is chosen based on path's hash code. If the selected + directory does not exist, an attempt is made to create it. + + @param dirsProp directory in which to locate the file. + @param path file-path. + @return local file under the directory with the given path.]]> + + + + + + + + + + + + name. + + @param name configuration resource name. + @return an input stream attached to the resource.]]> + + + + + + name. + + @param name configuration resource name. + @return a reader attached to the resource.]]> + + + + + + + + + + + + + + + + + + + + + + String + key-value pairs in the configuration. + + @return an iterator over the entries.]]> + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + +

  • + When property name is not empty and the property exists in the + configuration, this method writes the property and its attributes + to the {@link Writer}. +
  • + +
  • + When property name is null or empty, this method writes all the + configuration properties and their attributes to the {@link Writer}. +
  • + +
  • + When property name is not empty but the property doesn't exist in + the configuration, this method throws an {@link IllegalArgumentException}. +
  • + + @param out the writer to write to.]]> +
    + + + + + + + + +
  • + When propertyName is not empty, and the property exists + in the configuration, the format of the output would be, +
    +  {
    +    "property": {
    +      "key" : "key1",
    +      "value" : "value1",
    +      "isFinal" : "key1.isFinal",
    +      "resource" : "key1.resource"
    +    }
    +  }
    +  
    +
  • + +
  • + When propertyName is null or empty, it behaves same as + {@link #dumpConfiguration(Configuration, Writer)}, the + output would be, +
    +  { "properties" :
    +      [ { key : "key1",
    +          value : "value1",
    +          isFinal : "key1.isFinal",
    +          resource : "key1.resource" },
    +        { key : "key2",
    +          value : "value2",
    +          isFinal : "ke2.isFinal",
    +          resource : "key2.resource" }
    +       ]
    +   }
    +  
    +
  • + +
  • + When propertyName is not empty, and the property is not + found in the configuration, this method will throw an + {@link IllegalArgumentException}. +
  • + +

    + @param config the configuration + @param propertyName property name + @param out the Writer to write to + @throws IOException + @throws IllegalArgumentException when property name is not + empty and the property is not found in configuration]]> + + + + + + + + + { "properties" : + [ { key : "key1", + value : "value1", + isFinal : "key1.isFinal", + resource : "key1.resource" }, + { key : "key2", + value : "value2", + isFinal : "ke2.isFinal", + resource : "key2.resource" } + ] + } + + + It does not output the properties of the configuration object which + is loaded from an input stream. +

    + + @param config the configuration + @param out the Writer to write to + @throws IOException]]> + + + + + + + + + + + + + + + + + + + true to set quiet-mode on, false + to turn it off.]]> + + + + + + + + + + + + + + + + + + + + + } with matching keys]]> + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + Resources + +

    Configurations are specified by resources. A resource contains a set of + name/value pairs as XML data. Each resource is named by either a + String or by a {@link Path}. If named by a String, + then the classpath is examined for a file with that name. If named by a + Path, then the local filesystem is examined directly, without + referring to the classpath. + +

    Unless explicitly turned off, Hadoop by default specifies two + resources, loaded in-order from the classpath:

      +
    1. + + core-default.xml: Read-only defaults for hadoop.
    2. +
    3. core-site.xml: Site-specific configuration for a given hadoop + installation.
    4. +
    + Applications may add additional resources, which are loaded + subsequent to these resources in the order they are added. + +

    Final Parameters

    + +

    Configuration parameters may be declared final. + Once a resource declares a value final, no subsequently-loaded + resource can alter that value. + For example, one might define a final parameter with: +

    
    +  <property>
    +    <name>dfs.hosts.include</name>
    +    <value>/etc/hadoop/conf/hosts.include</value>
    +    <final>true</final>
    +  </property>
    + + Administrators typically define parameters as final in + core-site.xml for values that user applications may not alter. + +

    Variable Expansion

    + +

    Value strings are first processed for variable expansion. The + available properties are:

      +
    1. Other properties defined in this Configuration; and, if a name is + undefined here,
    2. +
    3. Environment variables in {@link System#getenv()} if a name starts with + "env.", or
    4. +
    5. Properties in {@link System#getProperties()}.
    6. +
    + +

    For example, if a configuration resource contains the following property + definitions: +

    
    +  <property>
    +    <name>basedir</name>
    +    <value>/user/${user.name}</value>
    +  </property>
    +  
    +  <property>
    +    <name>tempdir</name>
    +    <value>${basedir}/tmp</value>
    +  </property>
    +
    +  <property>
    +    <name>otherdir</name>
    +    <value>${env.BASE_DIR}/other</value>
    +  </property>
    +  
    + +

    When conf.get("tempdir") is called, then ${basedir} + will be resolved to another property in this Configuration, while + ${user.name} would then ordinarily be resolved to the value + of the System property with that name. +

    When conf.get("otherdir") is called, then ${env.BASE_DIR} + will be resolved to the value of the ${BASE_DIR} environment variable. + It supports ${env.NAME:-default} and ${env.NAME-default} notations. + The former is resolved to "default" if ${NAME} environment variable is undefined + or its value is empty. + The latter behaves the same way only if ${NAME} is undefined. +

    By default, warnings will be given to any deprecated configuration + parameters and these are suppressible by configuring + log4j.logger.org.apache.hadoop.conf.Configuration.deprecation in + log4j.properties file. + +

    Tags

    + +

    Optionally we can tag related properties together by using tag + attributes. System tags are defined by hadoop.tags.system property. Users + can define there own custom tags in hadoop.tags.custom property. + +

    For example, we can tag existing property as: +

    
    +  <property>
    +    <name>dfs.replication</name>
    +    <value>3</value>
    +    <tag>HDFS,REQUIRED</tag>
    +  </property>
    +
    +  <property>
    +    <name>dfs.data.transfer.protection</name>
    +    <value>3</value>
    +    <tag>HDFS,SECURITY</tag>
    +  </property>
    + 
    +

    Properties marked with tags can be retrieved with conf + .getAllPropertiesByTag("HDFS") or conf.getAllPropertiesByTags + (Arrays.asList("YARN","SECURITY")).

    ]]> +
    +
    + + + + + + + + + + + + + + + + + + + + + + + + + + + +
    + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + This implementation generates the key material and calls the + {@link #createKey(String, byte[], Options)} method. + + @param name the base name of the key + @param options the options for the new key. + @return the version name of the first version of the key. + @throws IOException + @throws NoSuchAlgorithmException]]> + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + This implementation generates the key material and calls the + {@link #rollNewVersion(String, byte[])} method. + + @param name the basename of the key + @return the name of the new version of the key + @throws IOException]]> + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + KeyProvider implementations must be thread safe.]]> + + + + + + + + + + + + + + + + + + + + + + NULL if + a provider for the specified URI scheme could not be found. + @throws IOException thrown if the provider failed to initialize.]]> + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + uri has syntax error]]> + + + + + + + + + + + + + + + + uri is + not found]]> + + + + + + + + + + + + + + + + + + + + + + + uri + determines a configuration property name, + fs.AbstractFileSystem.scheme.impl whose value names the + AbstractFileSystem class. + + The entire URI and conf is passed to the AbstractFileSystem factory method. + + @param uri for the file system to be created. + @param conf which is passed to the file system impl. + + @return file system for the given URI. + + @throws UnsupportedFileSystemException if the file system for + uri is not supported.]]> + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + In some FileSystem implementations such as HDFS metadata + synchronization is essential to guarantee consistency of read requests + particularly in HA setting. + @throws IOException + @throws UnsupportedOperationException]]> + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + } describing modifications + @throws IOException if an ACL could not be modified]]> + + + + + + + + } describing entries to remove + @throws IOException if an ACL could not be modified]]> + + + + + + + + + + + + + + + + + + + + + + } describing modifications, must + include entries for user, group, and others for compatibility with + permission bits. + @throws IOException if an ACL could not be modified]]> + + + + + + + } which returns each AclStatus + @throws IOException if an ACL could not be read]]> + + + + + + + + + + Refer to the HDFS extended attributes user documentation for details. + + @param path Path to modify + @param name xattr name. + @param value xattr value. + @throws IOException]]> + + + + + + + + + + + Refer to the HDFS extended attributes user documentation for details. + + @param path Path to modify + @param name xattr name. + @param value xattr value. + @param flag xattr set flag + @throws IOException]]> + + + + + + + + + Refer to the HDFS extended attributes user documentation for details. + + @param path Path to get extended attribute + @param name xattr name. + @return byte[] xattr value. + @throws IOException]]> + + + + + + + + Refer to the HDFS extended attributes user documentation for details. + + @param path Path to get extended attributes + + @return {@literal Map} describing the XAttrs of the file + or directory + @throws IOException]]> + + + + + + + + + Refer to the HDFS extended attributes user documentation for details. + + @param path Path to get extended attributes + @param names XAttr names. + @return {@literal Map} describing the XAttrs of the file + or directory + @throws IOException]]> + + + + + + + + Refer to the HDFS extended attributes user documentation for details. + + @param path Path to get extended attributes + @return {@literal Map} describing the XAttrs of the file + or directory + @throws IOException]]> + + + + + + + + + Refer to the HDFS extended attributes user documentation for details. + + @param path Path to remove extended attribute + @param name xattr name + @throws IOException]]> + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + BlockLocation(offset: 0, length: BLOCK_SIZE, + hosts: {"host1:9866", "host2:9866, host3:9866"}) + + + And if the file is erasure-coded, each BlockLocation represents a logical + block groups. Value offset is the offset of a block group in the file and + value length is the total length of a block group. Hosts of a BlockLocation + are the datanodes that holding all the data blocks and parity blocks of a + block group. + Suppose we have a RS_3_2 coded file (3 data units and 2 parity units). + A BlockLocation example will be like: +
    + BlockLocation(offset: 0, length: 3 * BLOCK_SIZE, hosts: {"host1:9866",
    +   "host2:9866","host3:9866","host4:9866","host5:9866"})
    + 
    + + Please refer to + {@link FileSystem#getFileBlockLocations(FileStatus, long, long)} or + {@link FileContext#getFileBlockLocations(Path, long, long)} + for more examples.]]> +
    +
    + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + After a successful call, {@code buf.position()} will be advanced by the + number of bytes read and {@code buf.limit()} will be unchanged. +

    + In the case of an exception, the state of the buffer (the contents of the + buffer, the {@code buf.position()}, the {@code buf.limit()}, etc.) is + undefined, and callers should be prepared to recover from this + eventuality. +

    + Callers should use {@link StreamCapabilities#hasCapability(String)} with + {@link StreamCapabilities#PREADBYTEBUFFER} to check if the underlying + stream supports this interface, otherwise they might get a + {@link UnsupportedOperationException}. +

    + Implementations should treat 0-length requests as legitimate, and must not + signal an error upon their receipt. +

    + This does not change the current offset of a file, and is thread-safe. + + @param position position within file + @param buf the ByteBuffer to receive the results of the read operation. + @return the number of bytes read, possibly zero, or -1 if reached + end-of-stream + @throws IOException if there is some error performing the read]]> + + + + + + + + + This operation provides similar semantics to + {@link #read(long, ByteBuffer)}, the difference is that this method is + guaranteed to read data until the {@link ByteBuffer} is full, or until + the end of the data stream is reached. + + @param position position within file + @param buf the ByteBuffer to receive the results of the read operation. + @throws IOException if there is some error performing the read + @throws EOFException the end of the data was reached before + the read operation completed + @see #read(long, ByteBuffer)]]> + + + + + + + + + + + + + + + After a successful call, {@code buf.position()} will be advanced by the + number of bytes read and {@code buf.limit()} will be unchanged. +

    + In the case of an exception, the state of the buffer (the contents of the + buffer, the {@code buf.position()}, the {@code buf.limit()}, etc.) is + undefined, and callers should be prepared to recover from this + eventuality. +

    + Callers should use {@link StreamCapabilities#hasCapability(String)} with + {@link StreamCapabilities#READBYTEBUFFER} to check if the underlying + stream supports this interface, otherwise they might get a + {@link UnsupportedOperationException}. +

    + Implementations should treat 0-length requests as legitimate, and must not + signal an error upon their receipt. + + @param buf + the ByteBuffer to receive the results of the read operation. + @return the number of bytes read, possibly zero, or -1 if + reach end-of-stream + @throws IOException + if there is some error performing the read]]> + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + setReplication of FileSystem + @param src file name + @param replication new replication + @throws IOException + @return true if successful; + false if file does not exist or is a directory]]> + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + core-default.xml]]> + + + + + + + + + + + core-default.xml]]> + + + + + + + + + + + core-default.xml]]> + + + + + + + + + + + core-default.xml]]> + + + + + + + + + + + core-default.xml]]> + + + + + + core-default.xml]]> + + + + + + + + + + + core-default.xml]]> + + + + + + + + + + + core-default.xml]]> + + + + + + core-default.xml]]> + + + + + + core-default.xml]]> + + + + + + core-default.xml]]> + + + + + + + + core-default.xml]]> + + + + + + + + + + + + + + + + + + + + + core-default.xml]]> + + + + + + + + + +

    + A higher number here does not necessarily improve performance, especially + for object stores, where multiple threads may be attempting to create an FS + instance for the same URI. +

    + Default value: {@value}.]]> +
    + + + +

    + Default value: {@value}.]]> +
    +
    + + + + core-default.xml]]> + + + + + + core-default.xml]]> + + + + + + core-default.xml]]> + + + + + + core-default.xml]]> + + + + + + + + + + + core-default.xml]]> + + + + + + + + + + + core-default.xml]]> + + + + + + + + + + + core-default.xml]]> + + + + + + + + + + + + + + + + core-default.xml]]> + + + + + + + + + + + core-default.xml]]> + + + + + + + + + + + core-default.xml]]> + + + + + + + + + + + core-default.xml]]> + + + + + + + + + + + core-default.xml]]> + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + core-default.xml]]> + + + + + + + + + + + core-default.xml]]> + + + + + + + + + + + core-default.xml]]> + + + + + + core-default.xml]]> + + + + + + + + + + + core-default.xml]]> + + + + + + + + + + + core-default.xml]]> + + + + + + + + + + + + + + + + + + + + + + + core-default.xml]]> + + + + + + + + + + + core-default.xml]]> + + + + + + + + + + + core-default.xml]]> + + + + + + + + + + + core-default.xml]]> + + + + + + + + + + + core-default.xml]]> + + + + + + + + + + + core-default.xml]]> + + + + + + + + + + + + + + + + + + + + + core-default.xml]]> + + + + + + + + + + + core-default.xml]]> + + + + + + + + + + + core-default.xml]]> + + + + + + + + + + + core-default.xml]]> + + + + + + + + + + + core-default.xml]]> + + + + + + + + + + + core-default.xml]]> + + + + + + + + + + + + + + + + + + + + + + core-default.xml]]> + + + + + + + + core-default.xml]]> + + + + + + core-default.xml]]> + + + + + + + + + + + core-default.xml]]> + + + + + + core-default.xml]]> + + + + + + core-default.xml]]> + + + + + + core-default.xml]]> + + + + + + core-default.xml]]> + + + + + + core-default.xml]]> + + + + + + + + core-default.xml]]> + + + + + + core-default.xml]]> + + + + + + core-default.xml]]> + + + + + + core-default.xml]]> + + + + + + core-default.xml]]> + + + + + + + + + + + core-default.xml]]> + + + + + + + + + + + core-default.xml]]> + + + + + + core-default.xml]]> + + + + + + core-default.xml]]> + + + + + + core-default.xml]]> + + + + + + core-default.xml]]> + + + + + + core-default.xml]]> + + + + + + core-default.xml]]> + + + + + + core-default.xml]]> + + + + + + core-default.xml]]> + + + + + + + + + + core-default.xml]]> + + + + + + + + + + + core-default.xml]]> + + + + + + + + + + + core-default.xml]]> + + + + + + + + + + + + + + + + + core-default.xml]]> + + + + + + + + core-default.xml]]> + + + + + + core-default.xml]]> + + + + + + core-default.xml]]> + + + + + + + + + + + + + + + + core-default.xml]]> + + + + + + core-default.xml]]> + + + + + + + + + + + core-default.xml]]> + + + + + + + + + + + core-default.xml]]> + + + + + + + + + + + core-default.xml]]> + + + + + + + + + + + core-default.xml]]> + + + + + + + + + + + core-default.xml]]> + + + + + + + + + + + core-default.xml]]> + + + + + + + + + + + + + core-default.xml]]> + + + + + + + + + + + core-default.xml]]> + + + + + + + + + + + core-default.xml]]> + + + + + + + + + + + core-default.xml]]> + + + + + + core-default.xml]]> + + + + + + + + core-default.xml]]> + + + + + + + + core-default.xml]]> + + + + + + + + core-default.xml]]> + + + + + + + + + + + core-default.xml]]> + + + + + + core-default.xml]]> + + + + + + + + core-default.xml]]> + + + + + + core-default.xml]]> + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + core-default.xml]]> + + + + + + + + core-default.xml]]> + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + EnumSet.of(CreateFlag.CREATE, CreateFlag.APPEND) + +

    + + Use the CreateFlag as follows: +

      +
    1. CREATE - to create a file if it does not exist, + else throw FileAlreadyExists.
    2. +
    3. APPEND - to append to a file if it exists, + else throw FileNotFoundException.
    4. +
    5. OVERWRITE - to truncate a file if it exists, + else throw FileNotFoundException.
    6. +
    7. CREATE|APPEND - to create a file if it does not exist, + else append to an existing file.
    8. +
    9. CREATE|OVERWRITE - to create a file if it does not exist, + else overwrite an existing file.
    10. +
    11. SYNC_BLOCK - to force closed blocks to the disk device. + In addition {@link Syncable#hsync()} should be called after each write, + if true synchronous behavior is required.
    12. +
    13. LAZY_PERSIST - Create the block on transient storage (RAM) if + available.
    14. +
    15. APPEND_NEWBLOCK - Append data to a new block instead of end of the last + partial block.
    16. +
    + + Following combinations are not valid and will result in + {@link HadoopIllegalArgumentException}: +
      +
    1. APPEND|OVERWRITE
    2. +
    3. CREATE|APPEND|OVERWRITE
    4. +
    ]]> +
    +
    + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + absOrFqPath is not supported. + @throws IOException If the file system for absOrFqPath could + not be instantiated.]]> + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + defaultFsUri is not supported]]> + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + NewWdir can be one of: +
      +
    • relative path: "foo/bar";
    • +
    • absolute without scheme: "/foo/bar"
    • +
    • fully qualified with scheme: "xx://auth/foo/bar"
    • +
    +
    + Illegal WDs: +
      +
    • relative with scheme: "xx:foo/bar"
    • +
    • non existent directory
    • +
    ]]> +
    +
    + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + f does not exist + @throws AccessControlException if access denied + @throws IOException If an IO Error occurred + + Exceptions applicable to file systems accessed over RPC: + @throws RpcClientException If an exception occurred in the RPC client + @throws RpcServerException If an exception occurred in the RPC server + @throws UnexpectedServerException If server implementation throws + undeclared exception to RPC server + + RuntimeExceptions: + @throws InvalidPathException If path f is not valid]]> + + + + + + + + + + + + + + + + + + + + +
  • Progress - to report progress on the operation - default null +
  • Permission - umask is applied against permission: default is + FsPermissions:getDefault() + +
  • CreateParent - create missing parent path; default is to not + to create parents +
  • The defaults for the following are SS defaults of the file + server implementing the target path. Not all parameters make sense + for all kinds of file system - eg. localFS ignores Blocksize, + replication, checksum +
      +
    • BufferSize - buffersize used in FSDataOutputStream +
    • Blocksize - block size for file blocks +
    • ReplicationFactor - replication for blocks +
    • ChecksumParam - Checksum parameters. server default is used + if not specified. +
    + + + @return {@link FSDataOutputStream} for created file + + @throws AccessControlException If access is denied + @throws FileAlreadyExistsException If file f already exists + @throws FileNotFoundException If parent of f does not exist + and createParent is false + @throws ParentNotDirectoryException If parent of f is not a + directory. + @throws UnsupportedFileSystemException If file system for f is + not supported + @throws IOException If an I/O error occurred + + Exceptions applicable to file systems accessed over RPC: + @throws RpcClientException If an exception occurred in the RPC client + @throws RpcServerException If an exception occurred in the RPC server + @throws UnexpectedServerException If server implementation throws + undeclared exception to RPC server + + RuntimeExceptions: + @throws InvalidPathException If path f is not valid]]> + + + + + + + + + + + + + + + + + + + + + dir already + exists + @throws FileNotFoundException If parent of dir does not exist + and createParent is false + @throws ParentNotDirectoryException If parent of dir is not a + directory + @throws UnsupportedFileSystemException If file system for dir + is not supported + @throws IOException If an I/O error occurred + + Exceptions applicable to file systems accessed over RPC: + @throws RpcClientException If an exception occurred in the RPC client + @throws UnexpectedServerException If server implementation throws + undeclared exception to RPC server + + RuntimeExceptions: + @throws InvalidPathException If path dir is not valid]]> + + + + + + + + + + + f does not exist + @throws UnsupportedFileSystemException If file system for f is + not supported + @throws IOException If an I/O error occurred + + Exceptions applicable to file systems accessed over RPC: + @throws RpcClientException If an exception occurred in the RPC client + @throws RpcServerException If an exception occurred in the RPC server + @throws UnexpectedServerException If server implementation throws + undeclared exception to RPC server + + RuntimeExceptions: + @throws InvalidPathException If path f is invalid]]> + + + + + + + + + + f does not exist + @throws UnsupportedFileSystemException If file system for f + is not supported + @throws IOException If an I/O error occurred + + Exceptions applicable to file systems accessed over RPC: + @throws RpcClientException If an exception occurred in the RPC client + @throws RpcServerException If an exception occurred in the RPC server + @throws UnexpectedServerException If server implementation throws + undeclared exception to RPC server]]> + + + + + + + + + + + f does not exist + @throws UnsupportedFileSystemException If file system for f is + not supported + @throws IOException If an I/O error occurred + + Exceptions applicable to file systems accessed over RPC: + @throws RpcClientException If an exception occurred in the RPC client + @throws RpcServerException If an exception occurred in the RPC server + @throws UnexpectedServerException If server implementation throws + undeclared exception to RPC server]]> + + + + + + + + + + + +
  • Fails if path is a directory. +
  • Fails if path does not exist. +
  • Fails if path is not closed. +
  • Fails if new size is greater than current size. + + @param f The path to the file to be truncated + @param newLength The size the file is to be truncated to + + @return true if the file has been truncated to the desired + newLength and is immediately available to be reused for + write operations such as append, or + false if a background process of adjusting the length of + the last block has been started, and clients should wait for it to + complete before proceeding with further file updates. + + @throws AccessControlException If access is denied + @throws FileNotFoundException If file f does not exist + @throws UnsupportedFileSystemException If file system for f is + not supported + @throws IOException If an I/O error occurred + + Exceptions applicable to file systems accessed over RPC: + @throws RpcClientException If an exception occurred in the RPC client + @throws RpcServerException If an exception occurred in the RPC server + @throws UnexpectedServerException If server implementation throws + undeclared exception to RPC server]]> + + + + + + + + + + f does not exist + @throws IOException If an I/O error occurred + + Exceptions applicable to file systems accessed over RPC: + @throws RpcClientException If an exception occurred in the RPC client + @throws RpcServerException If an exception occurred in the RPC server + @throws UnexpectedServerException If server implementation throws + undeclared exception to RPC server]]> + + + + + + + + + + + + + + +
  • Fails if src is a file and dst is a directory. +
  • Fails if src is a directory and dst is a file. +
  • Fails if the parent of dst does not exist or is a file. + +

    + If OVERWRITE option is not passed as an argument, rename fails if the dst + already exists. +

    + If OVERWRITE option is passed as an argument, rename overwrites the dst if + it is a file or an empty directory. Rename fails if dst is a non-empty + directory. +

    + Note that atomicity of rename is dependent on the file system + implementation. Please refer to the file system documentation for details +

    + + @param src path to be renamed + @param dst new path after rename + + @throws AccessControlException If access is denied + @throws FileAlreadyExistsException If dst already exists and + options has {@link Options.Rename#OVERWRITE} + option false. + @throws FileNotFoundException If src does not exist + @throws ParentNotDirectoryException If parent of dst is not a + directory + @throws UnsupportedFileSystemException If file system for src + and dst is not supported + @throws IOException If an I/O error occurred + + Exceptions applicable to file systems accessed over RPC: + @throws RpcClientException If an exception occurred in the RPC client + @throws RpcServerException If an exception occurred in the RPC server + @throws UnexpectedServerException If server implementation throws + undeclared exception to RPC server]]> + + + + + + + + + + + f does not exist + @throws UnsupportedFileSystemException If file system for f + is not supported + @throws IOException If an I/O error occurred + + Exceptions applicable to file systems accessed over RPC: + @throws RpcClientException If an exception occurred in the RPC client + @throws RpcServerException If an exception occurred in the RPC server + @throws UnexpectedServerException If server implementation throws + undeclared exception to RPC server]]> + + + + + + + + + + + + f does not exist + @throws UnsupportedFileSystemException If file system for f is + not supported + @throws IOException If an I/O error occurred + + Exceptions applicable to file systems accessed over RPC: + @throws RpcClientException If an exception occurred in the RPC client + @throws RpcServerException If an exception occurred in the RPC server + @throws UnexpectedServerException If server implementation throws + undeclared exception to RPC server + + RuntimeExceptions: + @throws HadoopIllegalArgumentException If username or + groupname is invalid.]]> + + + + + + + + + + + + f does not exist + @throws UnsupportedFileSystemException If file system for f is + not supported + @throws IOException If an I/O error occurred + + Exceptions applicable to file systems accessed over RPC: + @throws RpcClientException If an exception occurred in the RPC client + @throws RpcServerException If an exception occurred in the RPC server + @throws UnexpectedServerException If server implementation throws + undeclared exception to RPC server]]> + + + + + + + + + f does not exist + @throws IOException If an I/O error occurred + + Exceptions applicable to file systems accessed over RPC: + @throws RpcClientException If an exception occurred in the RPC client + @throws RpcServerException If an exception occurred in the RPC server + @throws UnexpectedServerException If server implementation throws + undeclared exception to RPC server]]> + + + + + + + + + + + f does not exist + @throws UnsupportedFileSystemException If file system for f is + not supported + @throws IOException If an I/O error occurred + + Exceptions applicable to file systems accessed over RPC: + @throws RpcClientException If an exception occurred in the RPC client + @throws RpcServerException If an exception occurred in the RPC server + @throws UnexpectedServerException If server implementation throws + undeclared exception to RPC server]]> + + + + + + + + + + f does not exist + @throws UnsupportedFileSystemException If file system for f is + not supported + @throws IOException If an I/O error occurred + + Exceptions applicable to file systems accessed over RPC: + @throws RpcClientException If an exception occurred in the RPC client + @throws RpcServerException If an exception occurred in the RPC server + @throws UnexpectedServerException If server implementation throws + undeclared exception to RPC server]]> + + + + + + + + + + + + + + + + + f does not exist + @throws UnsupportedFileSystemException If file system for f is + not supported + @throws IOException If an I/O error occurred]]> + + + + + + + + + + f does not exist + @throws UnsupportedFileSystemException If file system for f is + not supported + @throws IOException If the given path does not refer to a symlink + or an I/O error occurred]]> + + + + + + + + + + f does not exist + @throws UnsupportedFileSystemException If file system for f is + not supported + @throws IOException If an I/O error occurred + + Exceptions applicable to file systems accessed over RPC: + @throws RpcClientException If an exception occurred in the RPC client + @throws RpcServerException If an exception occurred in the RPC server + @throws UnexpectedServerException If server implementation throws + undeclared exception to RPC server]]> + + + + + + + + + + + + + + + Given a path referring to a symlink of form: + + {@literal <---}X{@literal --->} + fs://host/A/B/link + {@literal <-----}Y{@literal ----->} + + In this path X is the scheme and authority that identify the file system, + and Y is the path leading up to the final path component "link". If Y is + a symlink itself then let Y' be the target of Y and X' be the scheme and + authority of Y'. Symlink targets may: + + 1. Fully qualified URIs + + fs://hostX/A/B/file Resolved according to the target file system. + + 2. Partially qualified URIs (eg scheme but no host) + + fs:///A/B/file Resolved according to the target file system. Eg resolving + a symlink to hdfs:///A results in an exception because + HDFS URIs must be fully qualified, while a symlink to + file:///A will not since Hadoop's local file systems + require partially qualified URIs. + + 3. Relative paths + + path Resolves to [Y'][path]. Eg if Y resolves to hdfs://host/A and path + is "../B/file" then [Y'][path] is hdfs://host/B/file + + 4. Absolute paths + + path Resolves to [X'][path]. Eg if Y resolves hdfs://host/A/B and path + is "/file" then [X][path] is hdfs://host/file + + + @param target the target of the symbolic link + @param link the path to be created that points to target + @param createParent if true then missing parent dirs are created if + false then parent must exist + + + @throws AccessControlException If access is denied + @throws FileAlreadyExistsException If file link already exists + @throws FileNotFoundException If target does not exist + @throws ParentNotDirectoryException If parent of link is not a + directory. + @throws UnsupportedFileSystemException If file system for + target or link is not supported + @throws IOException If an I/O error occurred]]> + + + + + + + + + + f does not exist + @throws UnsupportedFileSystemException If file system for f is + not supported + @throws IOException If an I/O error occurred + + Exceptions applicable to file systems accessed over RPC: + @throws RpcClientException If an exception occurred in the RPC client + @throws RpcServerException If an exception occurred in the RPC server + @throws UnexpectedServerException If server implementation throws + undeclared exception to RPC server]]> + + + + + + + + + + + + + + + + + f does not exist + @throws UnsupportedFileSystemException If file system for f is + not supported + @throws IOException If an I/O error occurred + + Exceptions applicable to file systems accessed over RPC: + @throws RpcClientException If an exception occurred in the RPC client + @throws RpcServerException If an exception occurred in the RPC server + @throws UnexpectedServerException If server implementation throws + undeclared exception to RPC server]]> + + + + + + + + f is + not supported + @throws IOException If an I/O error occurred + + Exceptions applicable to file systems accessed over RPC: + @throws RpcClientException If an exception occurred in the RPC client + @throws RpcServerException If an exception occurred in the RPC server + @throws UnexpectedServerException If server implementation throws + undeclared exception to RPC server]]> + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + } describing + modifications + @throws IOException if an ACL could not be modified]]> + + + + + + + + } describing entries + to remove + @throws IOException if an ACL could not be modified]]> + + + + + + + + + + + + + + + + + + + + + + } describing + modifications, must include entries for user, group, and others for + compatibility with permission bits. + @throws IOException if an ACL could not be modified]]> + + + + + + + } which returns + each AclStatus + @throws IOException if an ACL could not be read]]> + + + + + + + + + + Refer to the HDFS extended attributes user documentation for details. + + @param path Path to modify + @param name xattr name. + @param value xattr value. + @throws IOException]]> + + + + + + + + + + + Refer to the HDFS extended attributes user documentation for details. + + @param path Path to modify + @param name xattr name. + @param value xattr value. + @param flag xattr set flag + @throws IOException]]> + + + + + + + + + Refer to the HDFS extended attributes user documentation for details. + + @param path Path to get extended attribute + @param name xattr name. + @return byte[] xattr value. + @throws IOException]]> + + + + + + + + Refer to the HDFS extended attributes user documentation for details. + + @param path Path to get extended attributes + @return Map{@literal <}String, byte[]{@literal >} describing the XAttrs + of the file or directory + @throws IOException]]> + + + + + + + + + Refer to the HDFS extended attributes user documentation for details. + + @param path Path to get extended attributes + @param names XAttr names. + @return Map{@literal <}String, byte[]{@literal >} describing the XAttrs + of the file or directory + @throws IOException]]> + + + + + + + + + Refer to the HDFS extended attributes user documentation for details. + + @param path Path to remove extended attribute + @param name xattr name + @throws IOException]]> + + + + + + + + Refer to the HDFS extended attributes user documentation for details. + + @param path Path to get extended attributes + @return List{@literal <}String{@literal >} of the XAttr names of the + file or directory + @throws IOException]]> + + + + + + + Exceptions applicable to file systems accessed over RPC: + @throws RpcClientException If an exception occurred in the RPC client + @throws RpcServerException If an exception occurred in the RPC server + @throws UnexpectedServerException If server implementation throws + undeclared exception to RPC server]]> + + + + + + + + Exceptions applicable to file systems accessed over RPC: + @throws RpcClientException If an exception occurred in the RPC client + @throws RpcServerException If an exception occurred in the RPC server + @throws UnexpectedServerException If server implementation throws + undeclared exception to RPC server]]> + + + + + + + + + Exceptions applicable to file systems accessed over RPC: + @throws RpcClientException If an exception occurred in the RPC client + @throws RpcServerException If an exception occurred in the RPC server + @throws UnexpectedServerException If server implementation throws + undeclared exception to RPC server]]> + + + + + + + + Exceptions applicable to file systems accessed over RPC: + @throws RpcClientException If an exception occurred in the RPC client + @throws RpcServerException If an exception occurred in the RPC server + @throws UnexpectedServerException If server implementation throws + undeclared exception to RPC server]]> + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + Path Names + + The Hadoop file system supports a URI namespace and URI names. This enables + multiple types of file systems to be referenced using fully-qualified URIs. + Two common Hadoop file system implementations are +

      +
    • the local file system: file:///path +
    • the HDFS file system: hdfs://nnAddress:nnPort/path +
    + + The Hadoop file system also supports additional naming schemes besides URIs. + Hadoop has the concept of a default file system, which implies a + default URI scheme and authority. This enables slash-relative names + relative to the default FS, which are more convenient for users and + application writers. The default FS is typically set by the user's + environment, though it can also be manually specified. +

    + + Hadoop also supports working-directory-relative names, which are paths + relative to the current working directory (similar to Unix). The working + directory can be in a different file system than the default FS. +

    + Thus, Hadoop path names can be specified as one of the following: +

      +
    • a fully-qualified URI: scheme://authority/path (e.g. + hdfs://nnAddress:nnPort/foo/bar) +
    • a slash-relative name: path relative to the default file system (e.g. + /foo/bar) +
    • a working-directory-relative name: path relative to the working dir (e.g. + foo/bar) +
    + Relative paths with scheme (scheme:foo/bar) are illegal. + +

    Role of FileContext and Configuration Defaults

    + + The FileContext is the analogue of per-process file-related state in Unix. It + contains two properties: + +
      +
    • the default file system (for resolving slash-relative names) +
    • the umask (for file permissions) +
    + In general, these properties are obtained from the default configuration file + in the user's environment (see {@link Configuration}). + + Further file system properties are specified on the server-side. File system + operations default to using these server-side defaults unless otherwise + specified. +

    + The file system related server-side defaults are: +

      +
    • the home directory (default is "/user/userName") +
    • the initial wd (only for local fs) +
    • replication factor +
    • block size +
    • buffer size +
    • encryptDataTransfer +
    • checksum option. (checksumType and bytesPerChecksum) +
    + +

    Example Usage

    + + Example 1: use the default config read from the $HADOOP_CONFIG/core.xml. + Unspecified values come from core-defaults.xml in the release jar. +
      +
    • myFContext = FileContext.getFileContext(); // uses the default config + // which has your default FS +
    • myFContext.create(path, ...); +
    • myFContext.setWorkingDir(path); +
    • myFContext.open (path, ...); +
    • ... +
    + Example 2: Get a FileContext with a specific URI as the default FS +
      +
    • myFContext = FileContext.getFileContext(URI); +
    • myFContext.create(path, ...); +
    • ... +
    + Example 3: FileContext with local file system as the default +
      +
    • myFContext = FileContext.getLocalFSFileContext(); +
    • myFContext.create(path, ...); +
    • ... +
    + Example 4: Use a specific config, ignoring $HADOOP_CONFIG + Generally you should not need use a config unless you are doing +
      +
    • configX = someConfigSomeOnePassedToYou; +
    • myFContext = getFileContext(configX); // configX is not changed, + // is passed down +
    • myFContext.create(path, ...); +
    • ... +
    ]]> + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + This implementation throws an UnsupportedOperationException. + + @return the protocol scheme for this FileSystem. + @throws UnsupportedOperationException if the operation is unsupported + (default).]]> + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + +
  • + If the configuration has the property + {@code "fs.$SCHEME.impl.disable.cache"} set to true, + a new instance will be created, initialized with the supplied URI and + configuration, then returned without being cached. +
  • +
  • + If the there is a cached FS instance matching the same URI, it will + be returned. +
  • +
  • + Otherwise: a new FS instance will be created, initialized with the + configuration and URI, cached and returned to the caller. +
  • + + @throws IOException if the FileSystem cannot be instantiated.]]> +
    +
    + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + if f == null : + result = null + elif f.getLen() {@literal <=} start: + result = [] + else result = [ locations(FS, b) for b in blocks(FS, p, s, s+l)] + + This call is most helpful with and distributed filesystem + where the hostnames of machines that contain blocks of the given file + can be determined. + + The default implementation returns an array containing one element: +
    + BlockLocation( { "localhost:9866" },  { "localhost" }, 0, file.getLen())
    + 
    + + In HDFS, if file is three-replicated, the returned array contains + elements like: +
    + BlockLocation(offset: 0, length: BLOCK_SIZE,
    +   hosts: {"host1:9866", "host2:9866, host3:9866"})
    + BlockLocation(offset: BLOCK_SIZE, length: BLOCK_SIZE,
    +   hosts: {"host2:9866", "host3:9866, host4:9866"})
    + 
    + + And if a file is erasure-coded, the returned BlockLocation are logical + block groups. + + Suppose we have a RS_3_2 coded file (3 data units and 2 parity units). + 1. If the file size is less than one stripe size, say 2 * CELL_SIZE, then + there will be one BlockLocation returned, with 0 offset, actual file size + and 4 hosts (2 data blocks and 2 parity blocks) hosting the actual blocks. + 3. If the file size is less than one group size but greater than one + stripe size, then there will be one BlockLocation returned, with 0 offset, + actual file size with 5 hosts (3 data blocks and 2 parity blocks) hosting + the actual blocks. + 4. If the file size is greater than one group size, 3 * BLOCK_SIZE + 123 + for example, then the result will be like: +
    + BlockLocation(offset: 0, length: 3 * BLOCK_SIZE, hosts: {"host1:9866",
    +   "host2:9866","host3:9866","host4:9866","host5:9866"})
    + BlockLocation(offset: 3 * BLOCK_SIZE, length: 123, hosts: {"host1:9866",
    +   "host4:9866", "host5:9866"})
    + 
    + + @param file FilesStatus to get data from + @param start offset into the given file + @param len length for which to get locations for + @throws IOException IO failure]]> +
    +
    + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + Important: the default implementation is not atomic + @param f path to use for create + @throws IOException IO failure]]> + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + +
  • Fails if src is a file and dst is a directory.
  • +
  • Fails if src is a directory and dst is a file.
  • +
  • Fails if the parent of dst does not exist or is a file.
  • + +

    + If OVERWRITE option is not passed as an argument, rename fails + if the dst already exists. +

    + If OVERWRITE option is passed as an argument, rename overwrites + the dst if it is a file or an empty directory. Rename fails if dst is + a non-empty directory. +

    + Note that atomicity of rename is dependent on the file system + implementation. Please refer to the file system documentation for + details. This default implementation is non atomic. +

    + This method is deprecated since it is a temporary method added to + support the transition from FileSystem to FileContext for user + applications. + + @param src path to be renamed + @param dst new path after rename + @throws FileNotFoundException src path does not exist, or the parent + path of dst does not exist. + @throws FileAlreadyExistsException dest path exists and is a file + @throws ParentNotDirectoryException if the parent path of dest is not + a directory + @throws IOException on failure]]> + + + + + + + + +

  • Fails if path is a directory.
  • +
  • Fails if path does not exist.
  • +
  • Fails if path is not closed.
  • +
  • Fails if new size is greater than current size.
  • + + @param f The path to the file to be truncated + @param newLength The size the file is to be truncated to + + @return true if the file has been truncated to the desired + newLength and is immediately available to be reused for + write operations such as append, or + false if a background process of adjusting the length of + the last block has been started, and clients should wait for it to + complete before proceeding with further file updates. + @throws IOException IO failure + @throws UnsupportedOperationException if the operation is unsupported + (default).]]> +
    +
    + + + + + + + + + + + + + + + + + + + + +
  • Clean shutdown of the JVM cannot be guaranteed.
  • +
  • The time to shut down a FileSystem will depends on the number of + files to delete. For filesystems where the cost of checking + for the existence of a file/directory and the actual delete operation + (for example: object stores) is high, the time to shutdown the JVM can be + significantly extended by over-use of this feature.
  • +
  • Connectivity problems with a remote filesystem may delay shutdown + further, and may cause the files to not be deleted.
  • + + @param f the path to delete. + @return true if deleteOnExit is successful, otherwise false. + @throws IOException IO failure]]> +
    +
    + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + Does not guarantee to return the List of files/directories status in a + sorted order. +

    + Will not return null. Expect IOException upon access error. + @param f given path + @return the statuses of the files/directories in the given patch + @throws FileNotFoundException when the path does not exist + @throws IOException see specific implementation]]> + + + + + + + + + + + + + + + + + Does not guarantee to return the List of files/directories status in a + sorted order. + + @param f + a path name + @param filter + the user-supplied path filter + @return an array of FileStatus objects for the files under the given path + after applying the filter + @throws FileNotFoundException when the path does not exist + @throws IOException see specific implementation]]> + + + + + + + + + Does not guarantee to return the List of files/directories status in a + sorted order. + + @param files + a list of paths + @return a list of statuses for the files under the given paths after + applying the filter default Path filter + @throws FileNotFoundException when the path does not exist + @throws IOException see specific implementation]]> + + + + + + + + + + Does not guarantee to return the List of files/directories status in a + sorted order. + + @param files + a list of paths + @param filter + the user-supplied path filter + @return a list of statuses for the files under the given paths after + applying the filter + @throws FileNotFoundException when the path does not exist + @throws IOException see specific implementation]]> + + + + + + + Return all the files that match filePattern and are not checksum + files. Results are sorted by their names. + +

    + A filename pattern is composed of regular characters and + special pattern matching characters, which are: + +

    +
    +
    +
    ? +
    Matches any single character. + +

    +

    * +
    Matches zero or more characters. + +

    +

    [abc] +
    Matches a single character from character set + {a,b,c}. + +

    +

    [a-b] +
    Matches a single character from the character range + {a...b}. Note that character a must be + lexicographically less than or equal to character b. + +

    +

    [^a] +
    Matches a single character that is not from character set or range + {a}. Note that the ^ character must occur + immediately to the right of the opening bracket. + +

    +

    \c +
    Removes (escapes) any special meaning of character c. + +

    +

    {ab,cd} +
    Matches a string from the string set {ab, cd} + +

    +

    {ab,c{de,fh}} +
    Matches a string from the string set {ab, cde, cfh} + +
    +
    +
    + + @param pathPattern a glob specifying a path pattern + + @return an array of paths that match the path pattern + @throws IOException IO failure]]> +
    +
    + + + + + + + + + + + + + + f does not exist + @throws IOException If an I/O error occurred]]> + + + + + + + + + f does not exist + @throws IOException if any I/O error occurred]]> + + + + + + + + p does not exist + @throws IOException if any I/O error occurred]]> + + + + + + + + + + If the path is a directory, + if recursive is false, returns files in the directory; + if recursive is true, return files in the subtree rooted at the path. + If the path is a file, return the file's status and block locations. + + @param f is the path + @param recursive if the subdirectories need to be traversed recursively + + @return an iterator that traverses statuses of the files + + @throws FileNotFoundException when the path does not exist; + @throws IOException see specific implementation]]> + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + undefined. + @throws IOException IO failure]]> + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + In some FileSystem implementations such as HDFS metadata + synchronization is essential to guarantee consistency of read requests + particularly in HA setting. + @throws IOException + @throws UnsupportedOperationException]]> + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + Refer to the HDFS extended attributes user documentation for details. + + @param path Path to modify + @param name xattr name. + @param value xattr value. + @throws IOException IO failure + @throws UnsupportedOperationException if the operation is unsupported + (default outcome).]]> + + + + + + + + + + + Refer to the HDFS extended attributes user documentation for details. + + @param path Path to modify + @param name xattr name. + @param value xattr value. + @param flag xattr set flag + @throws IOException IO failure + @throws UnsupportedOperationException if the operation is unsupported + (default outcome).]]> + + + + + + + + + Refer to the HDFS extended attributes user documentation for details. + + @param path Path to get extended attribute + @param name xattr name. + @return byte[] xattr value. + @throws IOException IO failure + @throws UnsupportedOperationException if the operation is unsupported + (default outcome).]]> + + + + + + + + Refer to the HDFS extended attributes user documentation for details. + + @param path Path to get extended attributes + @return Map describing the XAttrs of the file or directory + @throws IOException IO failure + @throws UnsupportedOperationException if the operation is unsupported + (default outcome).]]> + + + + + + + + + Refer to the HDFS extended attributes user documentation for details. + + @param path Path to get extended attributes + @param names XAttr names. + @return Map describing the XAttrs of the file or directory + @throws IOException IO failure + @throws UnsupportedOperationException if the operation is unsupported + (default outcome).]]> + + + + + + + + Refer to the HDFS extended attributes user documentation for details. + + @param path Path to get extended attributes + @return List{@literal } of the XAttr names of the file or directory + @throws IOException IO failure + @throws UnsupportedOperationException if the operation is unsupported + (default outcome).]]> + + + + + + + + + Refer to the HDFS extended attributes user documentation for details. + + @param path Path to remove extended attribute + @param name xattr name + @throws IOException IO failure + @throws UnsupportedOperationException if the operation is unsupported + (default outcome).]]> + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + This is a default method which is intended to be overridden by + subclasses. The default implementation returns an empty storage statistics + object.

    + + @return The StorageStatistics for this FileSystem instance. + Will never be null.]]> +
    +
    + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + All user code that may potentially use the Hadoop Distributed + File System should be written to use a FileSystem object or its + successor, {@link FileContext}. + +

    + The local implementation is {@link LocalFileSystem} and distributed + implementation is DistributedFileSystem. There are other implementations + for object stores and (outside the Apache Hadoop codebase), + third party filesystems. +

    + Notes +

      +
    1. The behaviour of the filesystem is + + specified in the Hadoop documentation. + However, the normative specification of the behavior of this class is + actually HDFS: if HDFS does not behave the way these Javadocs or + the specification in the Hadoop documentations define, assume that + the documentation is incorrect. +
    2. +
    3. The term {@code FileSystem} refers to an instance of this class.
    4. +
    5. The acronym "FS" is used as an abbreviation of FileSystem.
    6. +
    7. The term {@code filesystem} refers to the distributed/local filesystem + itself, rather than the class used to interact with it.
    8. +
    9. The term "file" refers to a file in the remote filesystem, + rather than instances of {@code java.io.File}.
    10. +
    + + This is a carefully evolving class. + New methods may be marked as Unstable or Evolving for their initial release, + as a warning that they are new and may change based on the + experience of use in applications. +

    + Important note for developers +

    + If you are making changes here to the public API or protected methods, + you must review the following subclasses and make sure that + they are filtering/passing through new methods as appropriate. +

    + + {@link FilterFileSystem}: methods are passed through. If not, + then {@code TestFilterFileSystem.MustNotImplement} must be + updated with the unsupported interface. + Furthermore, if the new API's support is probed for via + {@link #hasPathCapability(Path, String)} then + {@link FilterFileSystem#hasPathCapability(Path, String)} + must return false, always. +

    + {@link ChecksumFileSystem}: checksums are created and + verified. +

    + {@code TestHarFileSystem} will need its {@code MustNotImplement} + interface updated. +

    + + There are some external places your changes will break things. + Do co-ordinate changes here. +

    + + HBase: HBoss +

    + Hive: HiveShim23 + {@code shims/0.23/src/main/java/org/apache/hadoop/hive/shims/Hadoop23Shims.java}]]> +
    +
    + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + } caller's + environment variables to use for expansion + @return String[] with absolute path to new jar in position 0 and + unexpanded wild card entry path in position 1 + @throws IOException if there is an I/O error while writing the jar file]]> + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + FilterFileSystem contains + some other file system, which it uses as + its basic file system, possibly transforming + the data along the way or providing additional + functionality. The class FilterFileSystem + itself simply overrides all methods of + FileSystem with versions that + pass all requests to the contained file + system. Subclasses of FilterFileSystem + may further override some of these methods + and may also provide additional methods + and fields.]]> + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + Return type on the {@link #build()} call. + @param type of builder itself.]]> + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + -1 + if there is no more data because the end of the stream has been + reached]]> + + + + + + + + + + length bytes have been read. + + @param position position in the input stream to seek + @param buffer buffer into which data is read + @param offset offset into the buffer in which data is written + @param length the number of bytes to read + @throws IOException IO problems + @throws EOFException If the end of stream is reached while reading. + If an exception is thrown an undetermined number + of bytes in the buffer may have been written.]]> + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + // Don't + if (fs instanceof FooFileSystem) { + FooFileSystem fs = (FooFileSystem) fs; + OutputStream out = dfs.createFile(path) + .optionA() + .optionB("value") + .cache() + .build() + } else if (fs instanceof BarFileSystem) { + ... + } + + // Do + OutputStream out = fs.createFile(path) + .permission(perm) + .bufferSize(bufSize) + .opt("foofs:option.a", true) + .opt("foofs:option.b", "value") + .opt("barfs:cache", true) + .must("foofs:cache", true) + .must("barfs:cache-size", 256 * 1024 * 1024) + .build(); + + + If the option is not related to the file system, the option will be ignored. + If the option is must, but not supported by the file system, a + {@link IllegalArgumentException} will be thrown.]]> + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + path is invalid]]> + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + @return file]]> + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + +

    + The interface extends {@link IOStatisticsSource} so that there is no + need to cast an instance to see if is a source of statistics. + However, implementations MAY return null for their actual statistics.]]> +
    +
    + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + ListingBatch behaves similar to a Future, in that getting the result via + {@link #get()} will throw an Exception if there was a failure.]]> + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + and the scheme is null, and the authority + is null. + + @return whether the path is absolute and the URI has no scheme nor + authority parts]]> + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + true if and only if pathname + should be included]]> + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + Warning: Not all filesystems satisfy the thread-safety requirement. + @param position position within file + @param buffer destination buffer + @param offset offset in the buffer + @param length number of bytes to read + @return actual number of bytes read; -1 means "none" + @throws IOException IO problems.]]> + + + + + + + + + + Warning: Not all filesystems satisfy the thread-safety requirement. + @param position position within file + @param buffer destination buffer + @param offset offset in the buffer + @param length number of bytes to read + @throws IOException IO problems. + @throws EOFException the end of the data was reached before + the read operation completed]]> + + + + + + + + Warning: Not all filesystems satisfy the thread-safety requirement. + @param position position within file + @param buffer destination buffer + @throws IOException IO problems. + @throws EOFException the end of the data was reached before + the read operation completed]]> + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + Note
    : Returned list is not sorted in any given order, + due to reliance on Java's {@link File#list()} API.)]]> + +
    + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + XAttr is byte[], this class is to + covert byte[] to some kind of string representation or convert back. + String representation is convenient for display and input. For example + display in screen as shell response and json response, input as http + or shell parameter.]]> + + + + + + + + + + + + + + + + + + + + + +
    + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + @return ftp]]> + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + A {@link FileSystem} backed by an FTP client provided by Apache Commons Net. +

    ]]> +
    +
    + +
    + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + This is for reporting and testing.]]> + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + These are low-cost per-instance statistics provided by any Hadoop + I/O class instance. +

    + Consult the filesystem specification document for the requirements + of an implementation of this interface.]]> + + + + + + + + + + + + + + + + + + + + + + + Exceptions are caught and downgraded to debug logging. + @param source source of statistics. + @return a string for logging.]]> + + + + + + + + + + + + + + + + + + + Whenever this object's toString() method is called, it evaluates the + statistics. +

    + This is designed to affordable to use in log statements. + @param source source of statistics -may be null. + @return an object whose toString() operation returns the current values.]]> + + + + + + + Whenever this object's toString() method is called, it evaluates the + statistics. +

    + This is for use in log statements where for the cost of creation + of this entry is low; it is affordable to use in log statements. + @param statistics statistics to stringify -may be null. + @return an object whose toString() operation returns the current values.]]> + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + It is serializable so that frameworks which can use java serialization + to propagate data (Spark, Flink...) can send the statistics + back. For this reason, TreeMaps are explicitly used as field types, + even though IDEs can recommend use of Map instead. + For security reasons, untrusted java object streams should never be + deserialized. If for some reason this is required, use + {@link #requiredSerializationClasses()} to get the list of classes + used when deserializing instances of this object. +

    +

    + It is annotated for correct serializations with jackson2. +

    ]]> +
    + + + + + + + + + This is not an atomic option. +

    + The instance can be serialized, and its + {@code toString()} method lists all the values. + @param statistics statistics + @return a snapshot of the current values.]]> + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + It can be used to accrue values so as to dynamically update + the mean. If so, know that there is no synchronization + on the methods. +

    +

    + If a statistic has 0 samples then it is considered to be empty. +

    +

    + All 'empty' statistics are equivalent, independent of the sum value. +

    +

    + For non-empty statistics, sum and sample values must match + for equality. +

    +

    + It is serializable and annotated for correct serializations with jackson2. +

    +

    + Thread safety. The operations to add/copy sample data, are thread safe. +

    +
      +
    1. {@link #add(MeanStatistic)}
    2. +
    3. {@link #addSample(long)}
    4. +
    5. {@link #clear()}
    6. +
    7. {@link #setSamplesAndSum(long, long)}
    8. +
    9. {@link #set(MeanStatistic)}
    10. +
    11. {@link #setSamples(long)} and {@link #setSum(long)}
    12. +
    +

    + So is the {@link #mean()} method. This ensures that when + used to aggregated statistics, the aggregate value and sample + count are set and evaluated consistently. +

    +

    + Other methods marked as synchronized because Findbugs overreacts + to the idea that some operations to update sum and sample count + are synchronized, but that things like equals are not. +

    ]]> +
    +
    + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + names)}: {@value}.]]> + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + When adding new common statistic name constants, please make them unique. + By convention: +

    +
      +
    • the name of the constants are uppercase, words separated by + underscores.
    • +
    • the value of the constants are lowercase of the constant names.
    • +
    ]]> +
    +
    + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + When adding new common statistic name constants, please make them unique. + By convention, they are implicitly unique: +
      +
    • + The name of the constants are uppercase, words separated by + underscores. +
    • +
    • + The value of the constants are lowercase of the constant names. +
    • +
    ]]> +
    +
    + +
    + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + Since these methods are often vendor- or device-specific, operators + may implement this interface in order to achieve fencing. +

    + Fencing is configured by the operator as an ordered list of methods to + attempt. Each method will be tried in turn, and the next in the list + will only be attempted if the previous one fails. See {@link NodeFencer} + for more information. +

    + If an implementation also implements {@link Configurable} then its + setConf method will be called upon instantiation.]]> + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + state (e.g ACTIVE/STANDBY) as well as + some additional information. + + @throws AccessControlException + if access is denied. + @throws IOException + if other errors happen + @see HAServiceStatus]]> + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + hadoop.http.filter.initializers. + +

      +
    • StaticUserWebFilter - An authorization plugin that makes all +users a static configured user. +
    ]]> +
    +
    + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + public class IntArrayWritable extends ArrayWritable { + public IntArrayWritable() { + super(IntWritable.class); + } + } + ]]> + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + o is a ByteWritable with the same value.]]> + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + the class of the item + @param conf the configuration to store + @param item the object to be stored + @param keyName the name of the key to use + @throws IOException : forwards Exceptions from the underlying + {@link Serialization} classes.]]> + + + + + + + + + the class of the item + @param conf the configuration to use + @param keyName the name of the key to use + @param itemClass the class of the item + @return restored object + @throws IOException : forwards Exceptions from the underlying + {@link Serialization} classes.]]> + + + + + + + + + the class of the item + @param conf the configuration to use + @param items the objects to be stored + @param keyName the name of the key to use + @throws IndexOutOfBoundsException if the items array is empty + @throws IOException : forwards Exceptions from the underlying + {@link Serialization} classes.]]> + + + + + + + + + the class of the item + @param conf the configuration to use + @param keyName the name of the key to use + @param itemClass the class of the item + @return restored object + @throws IOException : forwards Exceptions from the underlying + {@link Serialization} classes.]]> + + + + + DefaultStringifier offers convenience methods to store/load objects to/from + the configuration. + + @param the class of the objects to stringify]]> + + + + + + + + + + + + + + + + + + + + + + + + + + + o is a DoubleWritable with the same value.]]> + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + value argument is null or + its size is zero, the elementType argument must not be null. If + the argument value's size is bigger than zero, the argument + elementType is not be used. + + @param value + @param elementType]]> + + + + + value should not be null + or empty. + + @param value]]> + + + + + + + + + + + + + + value and elementType. If the value argument + is null or its size is zero, the elementType argument must not be + null. If the argument value's size is bigger than zero, the + argument elementType is not be used. + + @param value + @param elementType]]> + + + + + + + + + + + + + + + + + + + o is an EnumSetWritable with the same value, + or both are null.]]> + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + o is a FloatWritable with the same value.]]> + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + When two sequence files, which have same Key type but different Value + types, are mapped out to reduce, multiple Value types is not allowed. + In this case, this class can help you wrap instances with different types. +

    + +

    + Compared with ObjectWritable, this class is much more effective, + because ObjectWritable will append the class declaration as a String + into the output file in every Key-Value pair. +

    + +

    + Generic Writable implements {@link Configurable} interface, so that it will be + configured by the framework. The configuration is passed to the wrapped objects + implementing {@link Configurable} interface before deserialization. +

    + + how to use it:
    + 1. Write your own class, such as GenericObject, which extends GenericWritable.
    + 2. Implements the abstract method getTypes(), defines + the classes which will be wrapped in GenericObject in application. + Attention: this classes defined in getTypes() method, must + implement Writable interface. +

    + + The code looks like this: +
    + public class GenericObject extends GenericWritable {
    + 
    +   private static Class[] CLASSES = {
    +               ClassType1.class, 
    +               ClassType2.class,
    +               ClassType3.class,
    +               };
    +
    +   protected Class[] getTypes() {
    +       return CLASSES;
    +   }
    +
    + }
    + 
    + + @since Nov 8, 2006]]> +
    +
    + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + o is a IntWritable with the same value.]]> + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + closes the input and output streams + at the end. + + @param in InputStrem to read from + @param out OutputStream to write to + @param conf the Configuration object]]> + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + ignore any {@link Throwable} or + null pointers. Must only be used for cleanup in exception handlers. + + @param log the log to record problems to at debug level. Can be null. + @param closeables the objects to close + @deprecated use {@link #cleanupWithLogger(Logger, java.io.Closeable...)} + instead]]> + + + + + + + ignore any {@link Throwable} or + null pointers. Must only be used for cleanup in exception handlers. + + @param logger the log to record problems to at debug level. Can be null. + @param closeables the objects to close]]> + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + This is better than File#listDir because it does not ignore IOExceptions. + + @param dir The directory to list. + @param filter If non-null, the filter to use when listing + this directory. + @return The list of files in the directory. + + @throws IOException On I/O error]]> + + + + + + + + Borrowed from Uwe Schindler in LUCENE-5588 + @param fileToSync the file to fsync]]> + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + o is a LongWritable with the same value.]]> + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + A map is a directory containing two files, the data file, + containing all keys and values in the map, and a smaller index + file, containing a fraction of the keys. The fraction is determined by + {@link Writer#getIndexInterval()}. + +

    The index file is read entirely into memory. Thus key implementations + should try to keep themselves small. + +

    Map files are created by adding entries in-order. To maintain a large + database, perform updates by copying the previous version of a database and + merging in a sorted change list, to create a new version of the database in + a new file. Sorting large change lists can be done with {@link + SequenceFile.Sorter}.]]> + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + o is an MD5Hash whose digest contains the + same values.]]> + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + className by first finding + it in the specified conf. If the specified conf is null, + try load it directly.]]> + + + + + + + + + + + + + + + + + + + + + + + + + + + + + A {@link Comparator} that operates directly on byte representations of + objects. +

    + @param + @see DeserializerComparator]]> +
    + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + SequenceFiles are flat files consisting of binary key/value + pairs. + +

    SequenceFile provides {@link SequenceFile.Writer}, + {@link SequenceFile.Reader} and {@link Sorter} classes for writing, + reading and sorting respectively.

    + + There are three SequenceFile Writers based on the + {@link CompressionType} used to compress key/value pairs: +
      +
    1. + Writer : Uncompressed records. +
    2. +
    3. + RecordCompressWriter : Record-compressed files, only compress + values. +
    4. +
    5. + BlockCompressWriter : Block-compressed files, both keys & + values are collected in 'blocks' + separately and compressed. The size of + the 'block' is configurable. +
    + +

    The actual compression algorithm used to compress key and/or values can be + specified by using the appropriate {@link CompressionCodec}.

    + +

    The recommended way is to use the static createWriter methods + provided by the SequenceFile to chose the preferred format.

    + +

    The {@link SequenceFile.Reader} acts as the bridge and can read any of the + above SequenceFile formats.

    + +

    SequenceFile Formats

    + +

    Essentially there are 3 different formats for SequenceFiles + depending on the CompressionType specified. All of them share a + common header described below. + +

    +
      +
    • + version - 3 bytes of magic header SEQ, followed by 1 byte of actual + version number (e.g. SEQ4 or SEQ6) +
    • +
    • + keyClassName -key class +
    • +
    • + valueClassName - value class +
    • +
    • + compression - A boolean which specifies if compression is turned on for + keys/values in this file. +
    • +
    • + blockCompression - A boolean which specifies if block-compression is + turned on for keys/values in this file. +
    • +
    • + compression codec - CompressionCodec class which is used for + compression of keys and/or values (if compression is + enabled). +
    • +
    • + metadata - {@link Metadata} for this file. +
    • +
    • + sync - A sync marker to denote end of the header. +
    • +
    + +
    Uncompressed SequenceFile Format
    +
      +
    • + Header +
    • +
    • + Record +
        +
      • Record length
      • +
      • Key length
      • +
      • Key
      • +
      • Value
      • +
      +
    • +
    • + A sync-marker every few 100 kilobytes or so. +
    • +
    + +
    Record-Compressed SequenceFile Format
    +
      +
    • + Header +
    • +
    • + Record +
        +
      • Record length
      • +
      • Key length
      • +
      • Key
      • +
      • Compressed Value
      • +
      +
    • +
    • + A sync-marker every few 100 kilobytes or so. +
    • +
    + +
    Block-Compressed SequenceFile Format
    +
      +
    • + Header +
    • +
    • + Record Block +
        +
      • Uncompressed number of records in the block
      • +
      • Compressed key-lengths block-size
      • +
      • Compressed key-lengths block
      • +
      • Compressed keys block-size
      • +
      • Compressed keys block
      • +
      • Compressed value-lengths block-size
      • +
      • Compressed value-lengths block
      • +
      • Compressed values block-size
      • +
      • Compressed values block
      • +
      +
    • +
    • + A sync-marker every block. +
    • +
    + +

    The compressed blocks of key lengths and value lengths consist of the + actual lengths of individual keys/values encoded in ZeroCompressedInteger + format.

    + + @see CompressionCodec]]> +
    +
    + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + o is a ShortWritable with the same value.]]> + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + the class of the objects to stringify]]> + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + position. Note that this + method avoids using the converter or doing String instantiation + @return the Unicode scalar value at position or -1 + if the position is invalid or points to a + trailing byte]]> + + + + + + + + + + what in the backing + buffer, starting as position start. The starting + position is measured in bytes and the return value is in + terms of byte position in the buffer. The backing buffer is + not converted to a string for this operation. + @return byte position of the first occurrence of the search + string in the UTF-8 buffer or -1 if not found]]> + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + Note: For performance reasons, this call does not clear the + underlying byte array that is retrievable via {@link #getBytes()}. + In order to free the byte-array memory, call {@link #set(byte[])} + with an empty byte array (For example, new byte[0]).]]> + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + o is a Text with the same contents.]]> + + + + + + + + + + + + + + + + + + + + + + + + + replace is true, then + malformed input is replaced with the + substitution character, which is U+FFFD. Otherwise the + method throws a MalformedInputException.]]> + + + + + + + + + + + + + + + replace is true, then + malformed input is replaced with the + substitution character, which is U+FFFD. Otherwise the + method throws a MalformedInputException. + @return ByteBuffer: bytes stores at ByteBuffer.array() + and length is ByteBuffer.limit()]]> + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + In + addition, it provides methods for string traversal without converting the + byte array to a string.

    Also includes utilities for + serializing/deserialing a string, coding/decoding a string, checking if a + byte array contains valid UTF8 code, calculating the length of an encoded + string.]]> + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + This is useful when a class may evolve, so that instances written by the + old version of the class may still be processed by the new version. To + handle this situation, {@link #readFields(DataInput)} + implementations should catch {@link VersionMismatchException}.]]> + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + o is a VIntWritable with the same value.]]> + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + o is a VLongWritable with the same value.]]> + + + + + + + + + + + + + + + + + + + + + + + + out. + + @param out DataOuput to serialize this object into. + @throws IOException]]> + + + + + + + in. + +

    For efficiency, implementations should attempt to re-use storage in the + existing object where possible.

    + + @param in DataInput to deseriablize this object from. + @throws IOException]]> +
    + + + Any key or value type in the Hadoop Map-Reduce + framework implements this interface.

    + +

    Implementations typically implement a static read(DataInput) + method which constructs a new instance, calls {@link #readFields(DataInput)} + and returns the instance.

    + +

    Example:

    +
    +     public class MyWritable implements Writable {
    +       // Some data
    +       private int counter;
    +       private long timestamp;
    +
    +       // Default constructor to allow (de)serialization
    +       MyWritable() { }
    +
    +       public void write(DataOutput out) throws IOException {
    +         out.writeInt(counter);
    +         out.writeLong(timestamp);
    +       }
    +
    +       public void readFields(DataInput in) throws IOException {
    +         counter = in.readInt();
    +         timestamp = in.readLong();
    +       }
    +
    +       public static MyWritable read(DataInput in) throws IOException {
    +         MyWritable w = new MyWritable();
    +         w.readFields(in);
    +         return w;
    +       }
    +     }
    + 
    ]]> +
    + + + + + + + + WritableComparables can be compared to each other, typically + via Comparators. Any type which is to be used as a + key in the Hadoop Map-Reduce framework should implement this + interface.

    + +

    Note that hashCode() is frequently used in Hadoop to partition + keys. It's important that your implementation of hashCode() returns the same + result across different instances of the JVM. Note also that the default + hashCode() implementation in Object does not + satisfy this property.

    + +

    Example:

    +
    +     public class MyWritableComparable implements
    +      WritableComparable{@literal } {
    +       // Some data
    +       private int counter;
    +       private long timestamp;
    +       
    +       public void write(DataOutput out) throws IOException {
    +         out.writeInt(counter);
    +         out.writeLong(timestamp);
    +       }
    +       
    +       public void readFields(DataInput in) throws IOException {
    +         counter = in.readInt();
    +         timestamp = in.readLong();
    +       }
    +       
    +       public int compareTo(MyWritableComparable o) {
    +         int thisValue = this.value;
    +         int thatValue = o.value;
    +         return (thisValue < thatValue ? -1 : (thisValue==thatValue ? 0 : 1));
    +       }
    +
    +       public int hashCode() {
    +         final int prime = 31;
    +         int result = 1;
    +         result = prime * result + counter;
    +         result = prime * result + (int) (timestamp ^ (timestamp >>> 32));
    +         return result
    +       }
    +     }
    + 
    ]]> +
    +
    + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + The default implementation reads the data into two {@link + WritableComparable}s (using {@link + Writable#readFields(DataInput)}, then calls {@link + #compare(WritableComparable,WritableComparable)}.]]> + + + + + + + The default implementation uses the natural ordering, calling {@link + Comparable#compareTo(Object)}.]]> + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + This base implementation uses the natural ordering. To define alternate + orderings, override {@link #compare(WritableComparable,WritableComparable)}. + +

    One may optimize compare-intensive operations by overriding + {@link #compare(byte[],int,int,byte[],int,int)}. Static utility methods are + provided to assist in optimized implementations of this method.]]> + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + Enum type + @param in DataInput to read from + @param enumType Class type of Enum + @return Enum represented by String read from DataInput + @throws IOException]]> + + + + + + + + + + + + + + + + len number of bytes in input streamin + @param in input stream + @param len number of bytes to skip + @throws IOException when skipped less number of bytes]]> + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + CompressionCodec for which to get the + Compressor + @param conf the Configuration object which contains confs for creating or reinit the compressor + @return Compressor for the given + CompressionCodec from the pool or a new one]]> + + + + + + + + + CompressionCodec for which to get the + Decompressor + @return Decompressor for the given + CompressionCodec the pool or a new one]]> + + + + + + Compressor to be returned to the pool]]> + + + + + + Decompressor to be returned to the + pool]]> + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + Codec aliases are case insensitive. +

    + The code alias is the short class name (without the package name). + If the short class name ends with 'Codec', then there are two aliases for + the codec, the complete short class name and the short class name without + the 'Codec' ending. For example for the 'GzipCodec' codec class name the + alias are 'gzip' and 'gzipcodec'. + + @param codecName the canonical class name of the codec + @return the codec object]]> + + + + + + + Codec aliases are case insensitive. +

    + The code alias is the short class name (without the package name). + If the short class name ends with 'Codec', then there are two aliases for + the codec, the complete short class name and the short class name without + the 'Codec' ending. For example for the 'GzipCodec' codec class name the + alias are 'gzip' and 'gzipcodec'. + + @param codecName the canonical class name of the codec + @return the codec class]]> + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + Implementations are assumed to be buffered. This permits clients to + reposition the underlying input stream then call {@link #resetState()}, + without having to also synchronize client buffers.]]> + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + true indicating that more input data is required. + + @param b Input data + @param off Start offset + @param len Length]]> + + + + + true if the input data buffer is empty and + #setInput() should be called in order to provide more input.]]> + + + + + + + + + + + + + + + + + + + + + + + + + + + + true if the end of the compressed + data output stream has been reached.]]> + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + true indicating that more input data is required. + (Both native and non-native versions of various Decompressors require + that the data passed in via b[] remain unmodified until + the caller is explicitly notified--via {@link #needsInput()}--that the + buffer may be safely modified. With this requirement, an extra + buffer-copy can be avoided.) + + @param b Input data + @param off Start offset + @param len Length]]> + + + + + true if the input data buffer is empty and + {@link #setInput(byte[], int, int)} should be called to + provide more input. + + @return true if the input data buffer is empty and + {@link #setInput(byte[], int, int)} should be called in + order to provide more input.]]> + + + + + + + + + + + + + true if a preset dictionary is needed for decompression. + @return true if a preset dictionary is needed for decompression]]> + + + + + true if the end of the decompressed + data output stream has been reached. Indicates a concatenated data stream + when finished() returns true and {@link #getRemaining()} + returns a positive value. finished() will be reset with the + {@link #reset()} method. + @return true if the end of the decompressed + data output stream has been reached.]]> + + + + + + + + + + + + + + true and getRemaining() returns a positive value. If + {@link #finished()} returns true and getRemaining() returns + a zero value, indicates that the end of data stream has been reached and + is not a concatenated data stream. + @return The number of bytes remaining in the compressed data buffer.]]> + + + + + true and {@link #getRemaining()} returns a positive value, + reset() is called before processing of the next data stream in the + concatenated data stream. {@link #finished()} will be reset and will + return false when reset() is called.]]> + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + io.compression.codecs = org.apache.hadoop.io.compress.PassthroughCodec + io.compress.passthrough.extension = .gz + + + Note: this is not a Splittable codec: it doesn't know the + capabilities of the passed in stream. It should be possible to + extend this in a subclass: the inner classes are marked as protected + to enable this. Do not retrofit splitting to this class..]]> + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + +

  • "none" - No compression. +
  • "lzo" - LZO compression. +
  • "gz" - GZIP compression. + ]]> + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + +
  • Block Compression. +
  • Named meta data blocks. +
  • Sorted or unsorted keys. +
  • Seek by key or by file offset. + + The memory footprint of a TFile includes the following: +
      +
    • Some constant overhead of reading or writing a compressed block. +
        +
      • Each compressed block requires one compression/decompression codec for + I/O. +
      • Temporary space to buffer the key. +
      • Temporary space to buffer the value (for TFile.Writer only). Values are + chunk encoded, so that we buffer at most one chunk of user data. By default, + the chunk buffer is 1MB. Reading chunked value does not require additional + memory. +
      +
    • TFile index, which is proportional to the total number of Data Blocks. + The total amount of memory needed to hold the index can be estimated as + (56+AvgKeySize)*NumBlocks. +
    • MetaBlock index, which is proportional to the total number of Meta + Blocks.The total amount of memory needed to hold the index for Meta Blocks + can be estimated as (40+AvgMetaBlockName)*NumMetaBlock. +
    +

    + The behavior of TFile can be customized by the following variables through + Configuration: +

      +
    • tfile.io.chunk.size: Value chunk size. Integer (in bytes). Default + to 1MB. Values of the length less than the chunk size is guaranteed to have + known value length in read time (See + {@link TFile.Reader.Scanner.Entry#isValueLengthKnown()}). +
    • tfile.fs.output.buffer.size: Buffer size used for + FSDataOutputStream. Integer (in bytes). Default to 256KB. +
    • tfile.fs.input.buffer.size: Buffer size used for + FSDataInputStream. Integer (in bytes). Default to 256KB. +
    +

    + Suggestions on performance optimization. +

      +
    • Minimum block size. We recommend a setting of minimum block size between + 256KB to 1MB for general usage. Larger block size is preferred if files are + primarily for sequential access. However, it would lead to inefficient random + access (because there are more data to decompress). Smaller blocks are good + for random access, but require more memory to hold the block index, and may + be slower to create (because we must flush the compressor stream at the + conclusion of each data block, which leads to an FS I/O flush). Further, due + to the internal caching in Compression codec, the smallest possible block + size would be around 20KB-30KB. +
    • The current implementation does not offer true multi-threading for + reading. The implementation uses FSDataInputStream seek()+read(), which is + shown to be much faster than positioned-read call in single thread mode. + However, it also means that if multiple threads attempt to access the same + TFile (using multiple scanners) simultaneously, the actual I/O is carried out + sequentially even if they access different DFS blocks. +
    • Compression codec. Use "none" if the data is not very compressable (by + compressable, I mean a compression ratio at least 2:1). Generally, use "lzo" + as the starting point for experimenting. "gz" overs slightly better + compression ratio over "lzo" but requires 4x CPU to compress and 2x CPU to + decompress, comparing to "lzo". +
    • File system buffering, if the underlying FSDataInputStream and + FSDataOutputStream is already adequately buffered; or if applications + reads/writes keys and values in large buffers, we can reduce the sizes of + input/output buffering in TFile layer by setting the configuration parameters + "tfile.fs.input.buffer.size" and "tfile.fs.output.buffer.size". +
    + + Some design rationale behind TFile can be found at Hadoop-3315.]]> + + + + + + + + + + + Utils#writeVLong(out, n). + + @param out + output stream + @param n + The integer to be encoded + @throws IOException + @see Utils#writeVLong(DataOutput, long)]]> + + + + + + + + +
  • if n in [-32, 127): encode in one byte with the actual value. + Otherwise, +
  • if n in [-20*2^8, 20*2^8): encode in two bytes: byte[0] = n/256 - 52; + byte[1]=n&0xff. Otherwise, +
  • if n IN [-16*2^16, 16*2^16): encode in three bytes: byte[0]=n/2^16 - + 88; byte[1]=(n>>8)&0xff; byte[2]=n&0xff. Otherwise, +
  • if n in [-8*2^24, 8*2^24): encode in four bytes: byte[0]=n/2^24 - 112; + byte[1] = (n>>16)&0xff; byte[2] = (n>>8)&0xff; + byte[3]=n&0xff. + Otherwise: +
  • if n in [-2^31, 2^31): encode in five bytes: byte[0]=-125; byte[1] = + (n>>24)&0xff; byte[2]=(n>>16)&0xff; + byte[3]=(n>>8)&0xff; byte[4]=n&0xff; +
  • if n in [-2^39, 2^39): encode in six bytes: byte[0]=-124; byte[1] = + (n>>32)&0xff; byte[2]=(n>>24)&0xff; + byte[3]=(n>>16)&0xff; byte[4]=(n>>8)&0xff; + byte[5]=n&0xff +
  • if n in [-2^47, 2^47): encode in seven bytes: byte[0]=-123; byte[1] = + (n>>40)&0xff; byte[2]=(n>>32)&0xff; + byte[3]=(n>>24)&0xff; byte[4]=(n>>16)&0xff; + byte[5]=(n>>8)&0xff; byte[6]=n&0xff; +
  • if n in [-2^55, 2^55): encode in eight bytes: byte[0]=-122; byte[1] = + (n>>48)&0xff; byte[2] = (n>>40)&0xff; + byte[3]=(n>>32)&0xff; byte[4]=(n>>24)&0xff; byte[5]= + (n>>16)&0xff; byte[6]=(n>>8)&0xff; byte[7]=n&0xff; +
  • if n in [-2^63, 2^63): encode in nine bytes: byte[0]=-121; byte[1] = + (n>>54)&0xff; byte[2] = (n>>48)&0xff; + byte[3] = (n>>40)&0xff; byte[4]=(n>>32)&0xff; + byte[5]=(n>>24)&0xff; byte[6]=(n>>16)&0xff; byte[7]= + (n>>8)&0xff; byte[8]=n&0xff; + + + @param out + output stream + @param n + the integer number + @throws IOException]]> + + + + + + + (int)Utils#readVLong(in). + + @param in + input stream + @return the decoded integer + @throws IOException + + @see Utils#readVLong(DataInput)]]> + + + + + + + +
  • if (FB >= -32), return (long)FB; +
  • if (FB in [-72, -33]), return (FB+52)<<8 + NB[0]&0xff; +
  • if (FB in [-104, -73]), return (FB+88)<<16 + + (NB[0]&0xff)<<8 + NB[1]&0xff; +
  • if (FB in [-120, -105]), return (FB+112)<<24 + (NB[0]&0xff) + <<16 + (NB[1]&0xff)<<8 + NB[2]&0xff; +
  • if (FB in [-128, -121]), return interpret NB[FB+129] as a signed + big-endian integer. + + @param in + input stream + @return the decoded long integer. + @throws IOException]]> + + + + + + + + + + + + + + + + + + + + + + + + Type of the input key. + @param list + The list + @param key + The input key. + @param cmp + Comparator for the key. + @return The index to the desired element if it exists; or list.size() + otherwise.]]> + + + + + + + + + Type of the input key. + @param list + The list + @param key + The input key. + @param cmp + Comparator for the key. + @return The index to the desired element if it exists; or list.size() + otherwise.]]> + + + + + + + + Type of the input key. + @param list + The list + @param key + The input key. + @return The index to the desired element if it exists; or list.size() + otherwise.]]> + + + + + + + + Type of the input key. + @param list + The list + @param key + The input key. + @return The index to the desired element if it exists; or list.size() + otherwise.]]> + + + + + + + + + + + + + + + + + An experimental {@link Serialization} for Java {@link Serializable} classes. +

    + @see JavaSerializationComparator]]> +
    +
    + + + + + + + + + A {@link RawComparator} that uses a {@link JavaSerialization} + {@link Deserializer} to deserialize objects that are then compared via + their {@link Comparable} interfaces. +

    + @param + @see JavaSerialization]]> +
    +
    + + + + + + + + + + + + + +This package provides a mechanism for using different serialization frameworks +in Hadoop. The property "io.serializations" defines a list of +{@link org.apache.hadoop.io.serializer.Serialization}s that know how to create +{@link org.apache.hadoop.io.serializer.Serializer}s and +{@link org.apache.hadoop.io.serializer.Deserializer}s. +

    + +

    +To add a new serialization framework write an implementation of +{@link org.apache.hadoop.io.serializer.Serialization} and add its name to the +"io.serializations" property. +

    ]]> +
    +
    + + + + + + + + + + + + + + + + + + + avro.reflect.pkgs or implement + {@link AvroReflectSerializable} interface.]]> + + + + + + + + + + + + + + + + + + + + + + + + + + +This package provides Avro serialization in Hadoop. This can be used to +serialize/deserialize Avro types in Hadoop. +

    + +

    +Use {@link org.apache.hadoop.io.serializer.avro.AvroSpecificSerialization} for +serialization of classes generated by Avro's 'specific' compiler. +

    + +

    +Use {@link org.apache.hadoop.io.serializer.avro.AvroReflectSerialization} for +other classes. +{@link org.apache.hadoop.io.serializer.avro.AvroReflectSerialization} work for +any class which is either in the package list configured via +{@link org.apache.hadoop.io.serializer.avro.AvroReflectSerialization#AVRO_REFLECT_PACKAGES} +or implement {@link org.apache.hadoop.io.serializer.avro.AvroReflectSerializable} +interface. +

    ]]> +
    +
    + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + Implementations of this interface consume the {@link MetricsRecord} generated + from {@link MetricsSource}. It registers with {@link MetricsSystem} which + periodically pushes the {@link MetricsRecord} to the sink using + {@link #putMetrics(MetricsRecord)} method. If the implementing class also + implements {@link Closeable}, then the MetricsSystem will close the sink when + it is stopped.]]> + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + the actual type of the source object + @param source object to register + @return the source object + @exception MetricsException]]> + + + + + + + + the actual type of the source object + @param source object to register + @param name of the source. Must be unique or null (then extracted from + the annotations of the source object.) + @param desc the description of the source (or null. See above.) + @return the source object + @exception MetricsException]]> + + + + + + + + + + + + + + + + + + + + +
  • {@link MetricsSource} generate and update metrics information.
  • +
  • {@link MetricsSink} consume the metrics information
  • + + + {@link MetricsSource} and {@link MetricsSink} register with the metrics + system. Implementations of {@link MetricsSystem} polls the + {@link MetricsSource}s periodically and pass the {@link MetricsRecord}s to + {@link MetricsSink}.]]> +
    +
    + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + +
    + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + } (aggregate). + Filter out entries that don't have at least minSamples. + + @return a map of peer DataNode Id to the average latency to that + node seen over the measurement period.]]> + + + + + + + + + + + This class maintains a group of rolling average metrics. It implements the + algorithm of rolling average, i.e. a number of sliding windows are kept to + roll over and evict old subsets of samples. Each window has a subset of + samples in a stream, where sub-sum and sub-total are collected. All sub-sums + and sub-totals in all windows will be aggregated to final-sum and final-total + used to compute final average, which is called rolling average. +

    ]]> +
    +
    + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + +
    + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + This class is a metrics sink that uses + {@link org.apache.hadoop.fs.FileSystem} to write the metrics logs. Every + roll interval a new directory will be created under the path specified by the + basepath property. All metrics will be logged to a file in the + current interval's directory in a file named <hostname>.log, where + <hostname> is the name of the host on which the metrics logging + process is running. The base path is set by the + <prefix>.sink.<instance>.basepath property. The + time zone used to create the current interval's directory name is GMT. If + the basepath property isn't specified, it will default to + "/tmp", which is the temp directory on whatever default file + system is configured for the cluster.

    + +

    The <prefix>.sink.<instance>.ignore-error + property controls whether an exception is thrown when an error is encountered + writing a log file. The default value is true. When set to + false, file errors are quietly swallowed.

    + +

    The roll-interval property sets the amount of time before + rolling the directory. The default value is 1 hour. The roll interval may + not be less than 1 minute. The property's value should be given as + number unit, where number is an integer value, and + unit is a valid unit. Valid units are minute, hour, + and day. The units are case insensitive and may be abbreviated or + plural. If no units are specified, hours are assumed. For example, + "2", "2h", "2 hour", and + "2 hours" are all valid ways to specify two hours.

    + +

    The roll-offset-interval-millis property sets the upper + bound on a random time interval (in milliseconds) that is used to delay + before the initial roll. All subsequent rolls will happen an integer + number of roll intervals after the initial roll, hence retaining the original + offset. The purpose of this property is to insert some variance in the roll + times so that large clusters using this sink on every node don't cause a + performance impact on HDFS by rolling simultaneously. The default value is + 30000 (30s). When writing to HDFS, as a rule of thumb, the roll offset in + millis should be no less than the number of sink instances times 5. + +

    The primary use of this class is for logging to HDFS. As it uses + {@link org.apache.hadoop.fs.FileSystem} to access the target file system, + however, it can be used to write to the local file system, Amazon S3, or any + other supported file system. The base path for the sink will determine the + file system used. An unqualified path will write to the default file system + set by the configuration.

    + +

    Not all file systems support the ability to append to files. In file + systems without the ability to append to files, only one writer can write to + a file at a time. To allow for concurrent writes from multiple daemons on a + single host, the source property is used to set unique headers + for the log files. The property should be set to the name of + the source daemon, e.g. namenode. The value of the + source property should typically be the same as the property's + prefix. If this property is not set, the source is taken to be + unknown.

    + +

    Instead of appending to an existing file, by default the sink + will create a new file with a suffix of ".<n>", where + n is the next lowest integer that isn't already used in a file name, + similar to the Hadoop daemon logs. NOTE: the file with the highest + sequence number is the newest file, unlike the Hadoop daemon logs.

    + +

    For file systems that allow append, the sink supports appending to the + existing file instead. If the allow-append property is set to + true, the sink will instead append to the existing file on file systems that + support appends. By default, the allow-append property is + false.

    + +

    Note that when writing to HDFS with allow-append set to true, + there is a minimum acceptable number of data nodes. If the number of data + nodes drops below that minimum, the append will succeed, but reading the + data will fail with an IOException in the DataStreamer class. The minimum + number of data nodes required for a successful append is generally 2 or + 3.

    + +

    Note also that when writing to HDFS, the file size information is not + updated until the file is closed (at the end of the interval) even though + the data is being written successfully. This is a known HDFS limitation that + exists because of the performance cost of updating the metadata. See + HDFS-5478.

    + +

    When using this sink in a secure (Kerberos) environment, two additional + properties must be set: keytab-key and + principal-key. keytab-key should contain the key by + which the keytab file can be found in the configuration, for example, + yarn.nodemanager.keytab. principal-key should + contain the key by which the principal can be found in the configuration, + for example, yarn.nodemanager.principal.]]> + + + + + + + + + + + + + + + + + + + + + + + + + CollectD StatsD plugin). +
    + To configure this plugin, you will need to add the following + entries to your hadoop-metrics2.properties file: +
    +

    + *.sink.statsd.class=org.apache.hadoop.metrics2.sink.StatsDSink
    + [prefix].sink.statsd.server.host=
    + [prefix].sink.statsd.server.port=
    + [prefix].sink.statsd.skip.hostname=true|false (optional)
    + [prefix].sink.statsd.service.name=NameNode (name you want for service)
    + 
    ]]> +
    +
    + +
    + + + + + + + + + + + + + ,name=}" + Where the {@literal and } are the supplied + parameters. + + @param serviceName + @param nameName + @param theMbean - the MBean to register + @return the named used to register the MBean]]> + + + + + + + + + ,name=}" + Where the {@literal and } are the supplied + parameters. + + @param serviceName + @param nameName + @param properties - Key value pairs to define additional JMX ObjectName + properties. + @param theMbean - the MBean to register + @return the named used to register the MBean]]> + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + hostname or hostname:port. If + the specs string is null, defaults to localhost:defaultPort. + + @param specs server specs (see description) + @param defaultPort the default port if not specified + @return a list of InetSocketAddress objects.]]> + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + This method is used when parts of Hadoop need know whether to apply + single rack vs multi-rack policies, such as during block placement. + Such algorithms behave differently if they are on multi-switch systems. +

    + + @return true if the mapping thinks that it is on a single switch]]> +
    +
    + + + + + + + + + + + + + + + + + This predicate simply assumes that all mappings not derived from + this class are multi-switch. + @param mapping the mapping to query + @return true if the base class says it is single switch, or the mapping + is not derived from this class.]]> + + + + It is not mandatory to + derive {@link DNSToSwitchMapping} implementations from it, but it is strongly + recommended, as it makes it easy for the Hadoop developers to add new methods + to this base class that are automatically picked up by all implementations. +

    + + This class does not extend the Configured + base class, and should not be changed to do so, as it causes problems + for subclasses. The constructor of the Configured calls + the {@link #setConf(Configuration)} method, which will call into the + subclasses before they have been fully constructed.]]> + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + If a name cannot be resolved to a rack, the implementation + should return {@link NetworkTopology#DEFAULT_RACK}. This + is what the bundled implementations do, though it is not a formal requirement + + @param names the list of hosts to resolve (can be empty) + @return list of resolved network paths. + If names is empty, the returned list is also empty]]> + + + + + + + + + + + + + + + + + + + + + + + + Calling {@link #setConf(Configuration)} will trigger a + re-evaluation of the configuration settings and so be used to + set up the mapping script.]]> + + + + + + + + + + + + + + + + + + + + + This will get called in the superclass constructor, so a check is needed + to ensure that the raw mapping is defined before trying to relaying a null + configuration. + @param conf]]> + + + + + + + + + + It contains a static class RawScriptBasedMapping that performs + the work: reading the configuration parameters, executing any defined + script, handling errors and such like. The outer + class extends {@link CachedDNSToSwitchMapping} to cache the delegated + queries. +

    + This DNS mapper's {@link #isSingleSwitch()} predicate returns + true if and only if a script is defined.]]> + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + Simple {@link DNSToSwitchMapping} implementation that reads a 2 column text + file. The columns are separated by whitespace. The first column is a DNS or + IP address and the second column specifies the rack where the address maps. +

    +

    + This class uses the configuration parameter {@code + net.topology.table.file.name} to locate the mapping file. +

    +

    + Calls to {@link #resolve(List)} will look up the address as defined in the + mapping file. If no entry corresponding to the address is found, the value + {@code /default-rack} is returned. +

    ]]> +
    +
    + +
    + + + + + + + + + + + + + + + + + + (cause==null ? null : cause.toString()) (which + typically contains the class and detail message of cause). + @param cause the cause (which is saved for later retrieval by the + {@link #getCause()} method). (A null value is + permitted, and indicates that the cause is nonexistent or + unknown.)]]> + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + } mapping and {@literal <}groupId, groupName{@literal >} + mapping.]]> + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + }/host@realm. + @param principalName principal name of format as described above + @return host name if the the string conforms to the above format, else null]]> + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + } "jack" + + @param userName + @return userName without login method]]> + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + the return type of the run method + @param action the method to execute + @return the value from the run method]]> + + + + + + + + the return type of the run method + @param action the method to execute + @return the value from the run method + @throws IOException if the action throws an IOException + @throws Error if the action throws an Error + @throws RuntimeException if the action throws a RuntimeException + @throws InterruptedException if the action throws an InterruptedException + @throws UndeclaredThrowableException if the action throws something else]]> + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + CredentialProvider implementations must be thread safe.]]> + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + (cause==null ? null : cause.toString()) (which + typically contains the class and detail message of cause). + @param cause the cause (which is saved for later retrieval by the + {@link #getCause()} method). (A null value is + permitted, and indicates that the cause is nonexistent or + unknown.)]]> + + + + + + + + + + + + + + does not provide the stack trace for security purposes.]]> + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + A User-Agent String is considered to be a browser if it matches + any of the regex patterns from browser-useragent-regex; the default + behavior is to consider everything a browser that matches the following: + "^Mozilla.*,^Opera.*". Subclasses can optionally override + this method to use different behavior. + + @param userAgent The User-Agent String, or null if there isn't one + @return true if the User-Agent String refers to a browser, false if not]]> + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + The type of the token identifier]]> + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + T extends TokenIdentifier]]> + + + + + + + + + + DelegationTokenAuthenticatedURL. +

    + An instance of the default {@link DelegationTokenAuthenticator} will be + used.]]> + + + + + DelegationTokenAuthenticatedURL. + + @param authenticator the {@link DelegationTokenAuthenticator} instance to + use, if null the default one will be used.]]> + + + + + DelegationTokenAuthenticatedURL using the default + {@link DelegationTokenAuthenticator} class. + + @param connConfigurator a connection configurator.]]> + + + + + DelegationTokenAuthenticatedURL. + + @param authenticator the {@link DelegationTokenAuthenticator} instance to + use, if null the default one will be used. + @param connConfigurator a connection configurator.]]> + + + + + + + + + + + + The default class is {@link KerberosDelegationTokenAuthenticator} + + @return the delegation token authenticator class to use as default.]]> + + + + + + + This method is provided to enable WebHDFS backwards compatibility. + + @param useQueryString TRUE if the token is transmitted in the + URL query string, FALSE if the delegation token is transmitted + using the {@link DelegationTokenAuthenticator#DELEGATION_TOKEN_HEADER} HTTP + header.]]> + + + + + TRUE if the token is transmitted in the URL query + string, FALSE if the delegation token is transmitted using the + {@link DelegationTokenAuthenticator#DELEGATION_TOKEN_HEADER} HTTP header.]]> + + + + + + + + + + + + + + + + + + Authenticator. + + @param url the URL to connect to. Only HTTP/S URLs are supported. + @param token the authentication token being used for the user. + @return an authenticated {@link HttpURLConnection}. + @throws IOException if an IO error occurred. + @throws AuthenticationException if an authentication exception occurred.]]> + + + + + + + + + + Authenticator. If the doAs parameter is not NULL, + the request will be done on behalf of the specified doAs user. + + @param url the URL to connect to. Only HTTP/S URLs are supported. + @param token the authentication token being used for the user. + @param doAs user to do the the request on behalf of, if NULL the request is + as self. + @return an authenticated {@link HttpURLConnection}. + @throws IOException if an IO error occurred. + @throws AuthenticationException if an authentication exception occurred.]]> + + + + + + + + + + Authenticator + for authentication. + + @param url the URL to get the delegation token from. Only HTTP/S URLs are + supported. + @param token the authentication token being used for the user where the + Delegation token will be stored. + @param renewer the renewer user. + @return a delegation token. + @throws IOException if an IO error occurred. + @throws AuthenticationException if an authentication exception occurred.]]> + + + + + + + + + + + Authenticator + for authentication. + + @param url the URL to get the delegation token from. Only HTTP/S URLs are + supported. + @param token the authentication token being used for the user where the + Delegation token will be stored. + @param renewer the renewer user. + @param doAsUser the user to do as, which will be the token owner. + @return a delegation token. + @throws IOException if an IO error occurred. + @throws AuthenticationException if an authentication exception occurred.]]> + + + + + + + + + Authenticator for authentication. + + @param url the URL to renew the delegation token from. Only HTTP/S URLs are + supported. + @param token the authentication token with the Delegation Token to renew. + @throws IOException if an IO error occurred. + @throws AuthenticationException if an authentication exception occurred.]]> + + + + + + + + + + Authenticator for authentication. + + @param url the URL to renew the delegation token from. Only HTTP/S URLs are + supported. + @param token the authentication token with the Delegation Token to renew. + @param doAsUser the user to do as, which will be the token owner. + @throws IOException if an IO error occurred. + @throws AuthenticationException if an authentication exception occurred.]]> + + + + + + + + Authenticator. + + @param url the URL to cancel the delegation token from. Only HTTP/S URLs + are supported. + @param token the authentication token with the Delegation Token to cancel. + @throws IOException if an IO error occurred.]]> + + + + + + + + + Authenticator. + + @param url the URL to cancel the delegation token from. Only HTTP/S URLs + are supported. + @param token the authentication token with the Delegation Token to cancel. + @param doAsUser the user to do as, which will be the token owner. + @throws IOException if an IO error occurred.]]> + + + + DelegationTokenAuthenticatedURL is a + {@link AuthenticatedURL} sub-class with built-in Hadoop Delegation Token + functionality. +

    + The authentication mechanisms supported by default are Hadoop Simple + authentication (also known as pseudo authentication) and Kerberos SPNEGO + authentication. +

    + Additional authentication mechanisms can be supported via {@link + DelegationTokenAuthenticator} implementations. +

    + The default {@link DelegationTokenAuthenticator} is the {@link + KerberosDelegationTokenAuthenticator} class which supports + automatic fallback from Kerberos SPNEGO to Hadoop Simple authentication via + the {@link PseudoDelegationTokenAuthenticator} class. +

    + AuthenticatedURL instances are not thread-safe.]]> + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + Authenticator + for authentication. + + @param url the URL to get the delegation token from. Only HTTP/S URLs are + supported. + @param token the authentication token being used for the user where the + Delegation token will be stored. + @param renewer the renewer user. + @throws IOException if an IO error occurred. + @throws AuthenticationException if an authentication exception occurred.]]> + + + + + + + + + + + Authenticator + for authentication. + + @param url the URL to get the delegation token from. Only HTTP/S URLs are + supported. + @param token the authentication token being used for the user where the + Delegation token will be stored. + @param renewer the renewer user. + @param doAsUser the user to do as, which will be the token owner. + @throws IOException if an IO error occurred. + @throws AuthenticationException if an authentication exception occurred.]]> + + + + + + + + + + Authenticator for authentication. + + @param url the URL to renew the delegation token from. Only HTTP/S URLs are + supported. + @param token the authentication token with the Delegation Token to renew. + @throws IOException if an IO error occurred. + @throws AuthenticationException if an authentication exception occurred.]]> + + + + + + + + + + + Authenticator for authentication. + + @param url the URL to renew the delegation token from. Only HTTP/S URLs are + supported. + @param token the authentication token with the Delegation Token to renew. + @param doAsUser the user to do as, which will be the token owner. + @throws IOException if an IO error occurred. + @throws AuthenticationException if an authentication exception occurred.]]> + + + + + + + + + Authenticator. + + @param url the URL to cancel the delegation token from. Only HTTP/S URLs + are supported. + @param token the authentication token with the Delegation Token to cancel. + @throws IOException if an IO error occurred.]]> + + + + + + + + + + Authenticator. + + @param url the URL to cancel the delegation token from. Only HTTP/S URLs + are supported. + @param token the authentication token with the Delegation Token to cancel. + @param doAsUser the user to do as, which will be the token owner. + @throws IOException if an IO error occurred.]]> + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + KerberosDelegationTokenAuthenticator provides support for + Kerberos SPNEGO authentication mechanism and support for Hadoop Delegation + Token operations. +

    + It falls back to the {@link PseudoDelegationTokenAuthenticator} if the HTTP + endpoint does not trigger a SPNEGO authentication]]> + + + + + + + + + PseudoDelegationTokenAuthenticator provides support for + Hadoop's pseudo authentication mechanism that accepts + the user name specified as a query string parameter and support for Hadoop + Delegation Token operations. +

    + This mimics the model of Hadoop Simple authentication trusting the + {@link UserGroupInformation#getCurrentUser()} value.]]> + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + live. + @return a (snapshotted) map of blocker name->description values]]> + + + + + + + + + + + + + Do nothing if the service is null or not + in a state in which it can be/needs to be stopped. +

    + The service state is checked before the operation begins. + This process is not thread safe. + @param service a service or null]]> + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + +

  • Any long-lived operation here will prevent the service state + change from completing in a timely manner.
  • +
  • If another thread is somehow invoked from the listener, and + that thread invokes the methods of the service (including + subclass-specific methods), there is a risk of a deadlock.
  • + + + + @param service the service that has changed.]]> +
    + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + +
    + + + + + + + + + + + + + + + + The base implementation logs all arguments at the debug level, + then returns the passed in config unchanged.]]> + + + + + + + The action is to signal success by returning the exit code 0.]]> + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + This method is called before {@link #init(Configuration)}; + Any non-null configuration that is returned from this operation + becomes the one that is passed on to that {@link #init(Configuration)} + operation. +

    + This permits implementations to change the configuration before + the init operation. As the ServiceLauncher only creates + an instance of the base {@link Configuration} class, it is + recommended to instantiate any subclass (such as YarnConfiguration) + that injects new resources. +

    + @param config the initial configuration build up by the + service launcher. + @param args list of arguments passed to the command line + after any launcher-specific commands have been stripped. + @return the configuration to init the service with. + Recommended: pass down the config parameter with any changes + @throws Exception any problem]]> + + + + + + + The return value becomes the exit code of the launched process. +

    + If an exception is raised, the policy is: +

      +
    1. Any subset of {@link org.apache.hadoop.util.ExitUtil.ExitException}: + the exception is passed up unmodified. +
    2. +
    3. Any exception which implements + {@link org.apache.hadoop.util.ExitCodeProvider}: + A new {@link ServiceLaunchException} is created with the exit code + and message of the thrown exception; the thrown exception becomes the + cause.
    4. +
    5. Any other exception: a new {@link ServiceLaunchException} is created + with the exit code {@link LauncherExitCodes#EXIT_EXCEPTION_THROWN} and + the message of the original exception (which becomes the cause).
    6. +
    + @return the exit code + @throws org.apache.hadoop.util.ExitUtil.ExitException an exception passed + up as the exit code and error text. + @throws Exception any exception to report. If it provides an exit code + this is used in a wrapping exception.]]> +
    +
    + + + The command line options will be passed down before the + {@link Service#init(Configuration)} operation is invoked via an + invocation of {@link LaunchableService#bindArgs(Configuration, List)} + After the service has been successfully started via {@link Service#start()} + the {@link LaunchableService#execute()} method is called to execute the + service. When this method returns, the service launcher will exit, using + the return code from the method as its exit option.]]> + +
    + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + Approximate HTTP equivalent: {@code 400 Bad Request}]]> + + + + + + approximate HTTP equivalent: Approximate HTTP equivalent: {@code 401 Unauthorized}]]> + + + + + + + + + + + Approximate HTTP equivalent: Approximate HTTP equivalent: {@code 403: Forbidden}]]> + + + + + + Approximate HTTP equivalent: {@code 404: Not Found}]]> + + + + + + Approximate HTTP equivalent: {@code 405: Not allowed}]]> + + + + + + Approximate HTTP equivalent: {@code 406: Not Acceptable}]]> + + + + + + Approximate HTTP equivalent: {@code 408: Request Timeout}]]> + + + + + + Approximate HTTP equivalent: {@code 409: Conflict}]]> + + + + + + Approximate HTTP equivalent: {@code 500 Internal Server Error}]]> + + + + + + Approximate HTTP equivalent: {@code 501: Not Implemented}]]> + + + + + + Approximate HTTP equivalent: {@code 503 Service Unavailable}]]> + + + + + + If raised, this is expected to be raised server-side and likely due + to client/server version incompatibilities. +

    + Approximate HTTP equivalent: {@code 505: Version Not Supported}]]> + + + + + + + + + + + + + + + Codes with a YARN prefix are YARN-related. +

    + Many of the exit codes are designed to resemble HTTP error codes, + squashed into a single byte. e.g 44 , "not found" is the equivalent + of 404. The various 2XX HTTP error codes aren't followed; + the Unix standard of "0" for success is used. +

    +    0-10: general command issues
    +   30-39: equivalent to the 3XX responses, where those responses are
    +          considered errors by the application.
    +   40-49: client-side/CLI/config problems
    +   50-59: service-side problems.
    +   60+  : application specific error codes
    + 
    ]]> +
    +
    + + + + + + + + + + + + + + + + + + + This uses {@link String#format(String, Object...)} + to build the formatted exception in the ENGLISH locale. +

    + If the last argument is a throwable, it becomes the cause of the exception. + It will also be used as a parameter for the format. + @param exitCode exit code + @param format format for message to use in exception + @param args list of arguments]]> + + + + + + This uses {@link String#format(String, Object...)} + to build the formatted exception in the ENGLISH locale. + @param exitCode exit code + @param cause inner cause + @param format format for message to use in exception + @param args list of arguments]]> + + + + + When caught by the ServiceLauncher, it will convert that + into a process exit code. + + The {@link #ServiceLaunchException(int, String, Object...)} constructor + generates formatted exceptions.]]> + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + +

    + This will be 0 until a call + to {@link #finished()} has been made. + @return the currently recorded duration.]]> +
    + + + + + + + + + +
    + + + + + + + + + + Clients and/or applications can use the provided Progressable + to explicitly report progress to the Hadoop framework. This is especially + important for operations which take significant amount of time since, + in-lieu of the reported progress, the framework has to assume that an error + has occurred and time-out the operation.

    ]]> +
    +
    + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + Class is to be obtained + @return the correctly typed Class of the given object.]]> + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + kill -0 command or equivalent]]> + + + + + + + + + + + + + + + + + + + ".cmd" on Windows, or ".sh" otherwise. + + @param parent File parent directory + @param basename String script file basename + @return File referencing the script in the directory]]> + + + + + + ".cmd" on Windows, or ".sh" otherwise. + + @param basename String script file basename + @return String script file name]]> + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + IOException. + @return the path to {@link #WINUTILS_EXE} + @throws RuntimeException if the path is not resolvable]]> + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + Shell. + @return the thread that ran runCommand() that spawned this shell + or null if no thread is waiting for this shell to complete]]> + + + + + + + + + + + + Shell interface. + @param cmd shell command to execute. + @return the output of the executed command.]]> + + + + + + + + + Shell interface. + @param env the map of environment key=value + @param cmd shell command to execute. + @param timeout time in milliseconds after which script should be marked timeout + @return the output of the executed command. + @throws IOException on any problem.]]> + + + + + + + + Shell interface. + @param env the map of environment key=value + @param cmd shell command to execute. + @return the output of the executed command. + @throws IOException on any problem.]]> + + + + + Shell processes. + Iterates through a map of all currently running Shell + processes and destroys them one by one. This method is thread safe]]> + + + + + Shell objects.]]> + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + CreateProcess synchronization object.]]> + + + + + os.name property.]]> + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + Important: caller must check for this value being null. + The lack of such checks has led to many support issues being raised. +

    + @deprecated use one of the exception-raising getter methods, + specifically {@link #getWinUtilsPath()} or {@link #getWinUtilsFile()}]]> + + + + + + + + + + + + + + Shell can be used to run shell commands like du or + df. It also offers facilities to gate commands by + time-intervals.]]> + + + + + + + + ShutdownHookManager singleton. + + @return ShutdownHookManager singleton.]]> + + + + + + + Runnable + @param priority priority of the shutdownHook.]]> + + + + + + + + + Runnable + @param priority priority of the shutdownHook + @param timeout timeout of the shutdownHook + @param unit unit of the timeout TimeUnit]]> + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + ShutdownHookManager enables running shutdownHook + in a deterministic order, higher priority first. +

    + The JVM runs ShutdownHooks in a non-deterministic order or in parallel. + This class registers a single JVM shutdownHook and run all the + shutdownHooks registered to it (to this class) in order based on their + priority. + + Unless a hook was registered with a shutdown explicitly set through + {@link #addShutdownHook(Runnable, int, long, TimeUnit)}, + the shutdown time allocated to it is set by the configuration option + {@link CommonConfigurationKeysPublic#SERVICE_SHUTDOWN_TIMEOUT} in + {@code core-site.xml}, with a default value of + {@link CommonConfigurationKeysPublic#SERVICE_SHUTDOWN_TIMEOUT_DEFAULT} + seconds.]]> + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + Tool, is the standard for any Map-Reduce tool/application. + The tool/application should delegate the handling of + + standard command-line options to {@link ToolRunner#run(Tool, String[])} + and only handle its custom arguments.

    + +

    Here is how a typical Tool is implemented:

    +

    +     public class MyApp extends Configured implements Tool {
    +     
    +       public int run(String[] args) throws Exception {
    +         // Configuration processed by ToolRunner
    +         Configuration conf = getConf();
    +         
    +         // Create a JobConf using the processed conf
    +         JobConf job = new JobConf(conf, MyApp.class);
    +         
    +         // Process custom command-line options
    +         Path in = new Path(args[1]);
    +         Path out = new Path(args[2]);
    +         
    +         // Specify various job-specific parameters     
    +         job.setJobName("my-app");
    +         job.setInputPath(in);
    +         job.setOutputPath(out);
    +         job.setMapperClass(MyMapper.class);
    +         job.setReducerClass(MyReducer.class);
    +
    +         // Submit the job, then poll for progress until the job is complete
    +         RunningJob runningJob = JobClient.runJob(job);
    +         if (runningJob.isSuccessful()) {
    +           return 0;
    +         } else {
    +           return 1;
    +         }
    +       }
    +       
    +       public static void main(String[] args) throws Exception {
    +         // Let ToolRunner handle generic command-line options 
    +         int res = ToolRunner.run(new Configuration(), new MyApp(), args);
    +         
    +         System.exit(res);
    +       }
    +     }
    + 

    + + @see GenericOptionsParser + @see ToolRunner]]> + + + + + + + + + + + + + + Tool by {@link Tool#run(String[])}, after + parsing with the given generic arguments. Uses the given + Configuration, or builds one if null. + + Sets the Tool's configuration with the possibly modified + version of the conf. + + @param conf Configuration for the Tool. + @param tool Tool to run. + @param args command-line arguments to the tool. + @return exit code of the {@link Tool#run(String[])} method.]]> + + + + + + + + Tool with its Configuration. + + Equivalent to run(tool.getConf(), tool, args). + + @param tool Tool to run. + @param args command-line arguments to the tool. + @return exit code of the {@link Tool#run(String[])} method.]]> + + + + + + + + + + + + + + + + + ToolRunner can be used to run classes implementing + Tool interface. It works in conjunction with + {@link GenericOptionsParser} to parse the + + generic hadoop command line arguments and modifies the + Configuration of the Tool. The + application-specific options are passed along without being modified. +

    + + @see Tool + @see GenericOptionsParser]]> +
    +
    + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + +
    + + + + + + + + + + + this filter. + @param nbHash The number of hash function to consider. + @param hashType type of the hashing function (see + {@link org.apache.hadoop.util.hash.Hash}).]]> + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + Bloom filter, as defined by Bloom in 1970. +

    + The Bloom filter is a data structure that was introduced in 1970 and that has been adopted by + the networking research community in the past decade thanks to the bandwidth efficiencies that it + offers for the transmission of set membership information between networked hosts. A sender encodes + the information into a bit vector, the Bloom filter, that is more compact than a conventional + representation. Computation and space costs for construction are linear in the number of elements. + The receiver uses the filter to test whether various elements are members of the set. Though the + filter will occasionally return a false positive, it will never return a false negative. When creating + the filter, the sender can choose its desired point in a trade-off between the false positive rate and the size. + +

    + Originally created by + European Commission One-Lab Project 034819. + + @see Filter The general behavior of a filter + + @see Space/Time Trade-Offs in Hash Coding with Allowable Errors]]> + + + + + + + + + + + + + this filter. + @param nbHash The number of hash function to consider. + @param hashType type of the hashing function (see + {@link org.apache.hadoop.util.hash.Hash}).]]> + + + + + + + + + this counting Bloom filter. +

    + Invariant: nothing happens if the specified key does not belong to this counter Bloom filter. + @param key The key to remove.]]> + + + + + + + + + + + + key -> count map. +

    NOTE: due to the bucket size of this filter, inserting the same + key more than 15 times will cause an overflow at all filter positions + associated with this key, and it will significantly increase the error + rate for this and other keys. For this reason the filter can only be + used to store small count values 0 <= N << 15. + @param key key to be tested + @return 0 if the key is not present. Otherwise, a positive value v will + be returned such that v == count with probability equal to the + error rate of this filter, and v > count otherwise. + Additionally, if the filter experienced an underflow as a result of + {@link #delete(Key)} operation, the return value may be lower than the + count with the probability of the false negative rate of such + filter.]]> + + + + + + + + + + + + + + + + + + + + + + counting Bloom filter, as defined by Fan et al. in a ToN + 2000 paper. +

    + A counting Bloom filter is an improvement to standard a Bloom filter as it + allows dynamic additions and deletions of set membership information. This + is achieved through the use of a counting vector instead of a bit vector. +

    + Originally created by + European Commission One-Lab Project 034819. + + @see Filter The general behavior of a filter + + @see Summary cache: a scalable wide-area web cache sharing protocol]]> + + + + + + + + + + + + + + Builds an empty Dynamic Bloom filter. + @param vectorSize The number of bits in the vector. + @param nbHash The number of hash function to consider. + @param hashType type of the hashing function (see + {@link org.apache.hadoop.util.hash.Hash}). + @param nr The threshold for the maximum number of keys to record in a + dynamic Bloom filter row.]]> + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + dynamic Bloom filter, as defined in the INFOCOM 2006 paper. +

    + A dynamic Bloom filter (DBF) makes use of a s * m bit matrix but + each of the s rows is a standard Bloom filter. The creation + process of a DBF is iterative. At the start, the DBF is a 1 * m + bit matrix, i.e., it is composed of a single standard Bloom filter. + It assumes that nr elements are recorded in the + initial bit vector, where nr {@literal <=} n + (n is the cardinality of the set A to record in + the filter). +

    + As the size of A grows during the execution of the application, + several keys must be inserted in the DBF. When inserting a key into the DBF, + one must first get an active Bloom filter in the matrix. A Bloom filter is + active when the number of recorded keys, nr, is + strictly less than the current cardinality of A, n. + If an active Bloom filter is found, the key is inserted and + nr is incremented by one. On the other hand, if there + is no active Bloom filter, a new one is created (i.e., a new row is added to + the matrix) according to the current size of A and the element + is added in this new Bloom filter and the nr value of + this new Bloom filter is set to one. A given key is said to belong to the + DBF if the k positions are set to one in one of the matrix rows. +

    + Originally created by + European Commission One-Lab Project 034819. + + @see Filter The general behavior of a filter + @see BloomFilter A Bloom filter + + @see Theory and Network Applications of Dynamic Bloom Filters]]> + + + + + + + + + Builds a hash function that must obey to a given maximum number of returned values and a highest value. + @param maxValue The maximum highest returned value. + @param nbHash The number of resulting hashed values. + @param hashType type of the hashing function (see {@link Hash}).]]> + + + + + this hash function. A NOOP]]> + + + + + + + + + + + + + + + + + + + The idea is to randomly select a bit to reset.]]> + + + + + + The idea is to select the bit to reset that will generate the minimum + number of false negative.]]> + + + + + + The idea is to select the bit to reset that will remove the maximum number + of false positive.]]> + + + + + + The idea is to select the bit to reset that will, at the same time, remove + the maximum number of false positve while minimizing the amount of false + negative generated.]]> + + + + + Originally created by + European Commission One-Lab Project 034819.]]> + + + + + + + + + + + + + + this filter. + @param nbHash The number of hash function to consider. + @param hashType type of the hashing function (see + {@link org.apache.hadoop.util.hash.Hash}).]]> + + + + + + + + + this retouched Bloom filter. +

    + Invariant: if the false positive is null, nothing happens. + @param key The false positive key to add.]]> + + + + + + this retouched Bloom filter. + @param coll The collection of false positive.]]> + + + + + + this retouched Bloom filter. + @param keys The list of false positive.]]> + + + + + + this retouched Bloom filter. + @param keys The array of false positive.]]> + + + + + + + this retouched Bloom filter. + @param scheme The selective clearing scheme to apply.]]> + + + + + + + + + + + + retouched Bloom filter, as defined in the CoNEXT 2006 paper. +

    + It allows the removal of selected false positives at the cost of introducing + random false negatives, and with the benefit of eliminating some random false + positives at the same time. + +

    + Originally created by + European Commission One-Lab Project 034819. + + @see Filter The general behavior of a filter + @see BloomFilter A Bloom filter + @see RemoveScheme The different selective clearing algorithms + + @see Retouched Bloom Filters: Allowing Networked Applications to Trade Off Selected False Positives Against False Negatives]]> + + + + + + + + + + + + + + + + + Any exception generated in the future is + extracted and rethrown. +

    + @param future future to evaluate + @param type of the result. + @return the result, if all went well. + @throws InterruptedIOException future was interrupted + @throws IOException if something went wrong + @throws RuntimeException any nested RTE thrown]]> +
    + + + + + + + + + + + + Any exception generated in the future is + extracted and rethrown. +

    + @param future future to evaluate + @param type of the result. + @return the result, if all went well. + @throws InterruptedIOException future was interrupted + @throws IOException if something went wrong + @throws RuntimeException any nested RTE thrown + @throws TimeoutException the future timed out.]]> +
    +
    + + + + + type of return value. + @return nothing, ever. + @throws IOException either the inner IOException, or a wrapper around + any non-Runtime-Exception + @throws RuntimeException if that is the inner cause.]]> + + + + + + + type of return value. + @return nothing, ever. + @throws IOException either the inner IOException, or a wrapper around + any non-Runtime-Exception + @throws RuntimeException if that is the inner cause.]]> + + + + + + +
  • If it is an IOE: Return.
  • +
  • If it is a {@link UncheckedIOException}: return the cause
  • +
  • Completion/Execution Exceptions: extract and repeat
  • +
  • If it is an RTE or Error: throw.
  • +
  • Any other type: wrap in an IOE
  • + + + Recursively handles wrapped Execution and Completion Exceptions in + case something very complicated has happened. + @param e exception. + @return an IOException extracted or built from the cause. + @throws RuntimeException if that is the inner cause. + @throws Error if that is the inner cause.]]> +
    +
    + + + Contains methods promoted from + {@link org.apache.hadoop.fs.impl.FutureIOSupport} because they + are a key part of integrating async IO in application code. +

    +

    + One key feature is that the {@link #awaitFuture(Future)} and + {@link #awaitFuture(Future, long, TimeUnit)} calls will + extract and rethrow exceptions raised in the future's execution, + including extracting the inner IOException of any + {@code UncheckedIOException} raised in the future. + This makes it somewhat easier to execute IOException-raising + code inside futures. +

    ]]> +
    +
    + + + + + + + type + @return a remote iterator]]> + + + + + + type + @return a remote iterator]]> + + + + + + type + @return a remote iterator]]> + + + + + + type + @return a remote iterator]]> + + + + + + + source type + @param result type + @param iterator source + @param mapper transformation + @return a remote iterator]]> + + + + + + source type + @param result type + @param iterator source + @return a remote iterator]]> + + + + + + +

    + Elements are filtered in the hasNext() method; if not used + the filtering will be done on demand in the {@code next()} + call. + @param type + @param iterator source + @param filter filter + @return a remote iterator]]> +
    +
    + + + + + source type. + @return a new iterator]]> + + + + + + + type + @return a list of the values. + @throws IOException if the source RemoteIterator raises it.]]> + + + + + + + + type + @return an array of the values. + @throws IOException if the source RemoteIterator raises it.]]> + + + + + + + +

    + If the iterator is an IOStatisticsSource returning a non-null + set of statistics, and this classes log is set to DEBUG, + then the statistics of the operation are evaluated and logged at + debug. +

    + The number of entries processed is returned, as it is useful to + know this, especially during tests or when reporting values + to users. +

    + This does not close the iterator afterwards. + @param source iterator source + @param consumer consumer of the values. + @return the number of elements processed + @param type of source + @throws IOException if the source RemoteIterator or the consumer raise one.]]> +
    +
    + + + + type of source]]> + + + +

    + This aims to make it straightforward to use lambda-expressions to + transform the results of an iterator, without losing the statistics + in the process, and to chain the operations together. +

    + The closeable operation will be passed through RemoteIterators which + wrap other RemoteIterators. This is to support any iterator which + can be closed to release held connections, file handles etc. + Unless client code is written to assume that RemoteIterator instances + may be closed, this is not likely to be broadly used. It is added + to make it possible to adopt this feature in a managed way. +

    + One notable feature is that the + {@link #foreach(RemoteIterator, ConsumerRaisingIOE)} method will + LOG at debug any IOStatistics provided by the iterator, if such + statistics are provided. There's no attempt at retrieval and logging + if the LOG is not set to debug, so it is a zero cost feature unless + the logger {@code org.apache.hadoop.fs.functional.RemoteIterators} + is at DEBUG. +

    + Based on the S3A Listing code, and some some work on moving other code + to using iterative listings so as to pick up the statistics.]]> +
    +
    + +
    + + + +
    diff --git a/hadoop-common-project/hadoop-common/pom.xml b/hadoop-common-project/hadoop-common/pom.xml index 6e762f567c180..d8e2dd3542223 100644 --- a/hadoop-common-project/hadoop-common/pom.xml +++ b/hadoop-common-project/hadoop-common/pom.xml @@ -1171,6 +1171,16 @@ + + org.apache.maven.plugins + maven-javadoc-plugin + + + **/FSProtos.java + + *.proto:*.tracing:*.protobuf + + diff --git a/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/conf/ConfigRedactor.java b/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/conf/ConfigRedactor.java index 5b2d1449f9c86..881a2ce811bbe 100644 --- a/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/conf/ConfigRedactor.java +++ b/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/conf/ConfigRedactor.java @@ -57,8 +57,8 @@ public ConfigRedactor(Configuration conf) { * Given a key / value pair, decides whether or not to redact and returns * either the original value or text indicating it has been redacted. * - * @param key - * @param value + * @param key param key. + * @param value param value, will return if conditions permit. * @return Original value, or text indicating it has been redacted */ public String redact(String key, String value) { diff --git a/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/conf/Configuration.java b/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/conf/Configuration.java index 1f809b7b54706..d8ceb58aba72c 100755 --- a/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/conf/Configuration.java +++ b/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/conf/Configuration.java @@ -317,7 +317,7 @@ private static boolean getRestrictParserDefault(Object resource) { private boolean loadDefaults = true; /** - * Configuration objects + * Configuration objects. */ private static final WeakHashMap REGISTRY = new WeakHashMap(); @@ -774,7 +774,7 @@ private void updatePropertiesWithDeprecatedKeys( private void handleDeprecation() { LOG.debug("Handling deprecation for all properties in config..."); DeprecationContext deprecations = deprecationContext.get(); - Set keys = new HashSet(); + Set keys = new HashSet<>(); keys.addAll(getProps().keySet()); for (Object item: keys) { LOG.debug("Handling deprecation for " + (String)item); @@ -1908,6 +1908,7 @@ public long getTimeDuration(String name, String defaultValue, * @param name Property name * @param vStr The string value with time unit suffix to be converted. * @param unit Unit to convert the stored property, if it exists. + * @return time duration in given time unit. */ public long getTimeDurationHelper(String name, String vStr, TimeUnit unit) { return getTimeDurationHelper(name, vStr, unit, unit); @@ -1922,6 +1923,7 @@ public long getTimeDurationHelper(String name, String vStr, TimeUnit unit) { * @param vStr The string value with time unit suffix to be converted. * @param defaultUnit Unit to convert the stored property, if it exists. * @param returnUnit Unit for the returned value. + * @return time duration in given time unit. */ private long getTimeDurationHelper(String name, String vStr, TimeUnit defaultUnit, TimeUnit returnUnit) { @@ -2206,7 +2208,7 @@ private static int convertToInt(String value, int defaultValue) { } /** - * Is the given value in the set of ranges + * Is the given value in the set of ranges. * @param value the value to check * @return is the value in the ranges? */ @@ -2263,7 +2265,7 @@ public Iterator iterator() { } /** - * Parse the given attribute as a set of integer ranges + * Parse the given attribute as a set of integer ranges. * @param name the attribute name * @param defaultValue the default value if it is not set * @return a new set of ranges from the configured value @@ -2482,7 +2484,7 @@ public char[] getPasswordFromCredentialProviders(String name) /** * Fallback to clear text passwords in configuration. - * @param name + * @param name the property name. * @return clear text password or null */ protected char[] getPasswordFromConfig(String name) { @@ -2547,6 +2549,8 @@ public InetSocketAddress getSocketAddr( /** * Set the socket address for the name property as * a host:port. + * @param name property name. + * @param addr inetSocketAddress addr. */ public void setSocketAddr(String name, InetSocketAddress addr) { set(name, NetUtils.getHostPortString(addr)); @@ -2724,6 +2728,7 @@ public Class getClass(String name, Class defaultValue) { * @param name the conf key name. * @param defaultValue default value. * @param xface the interface implemented by the named class. + * @param Interface class type. * @return property value as a Class, * or defaultValue. */ @@ -2753,6 +2758,7 @@ else if (theClass != null) * @param name the property name. * @param xface the interface implemented by the classes named by * name. + * @param Interface class type. * @return a List of objects implementing xface. */ @SuppressWarnings("unchecked") @@ -2785,15 +2791,16 @@ public void setClass(String name, Class theClass, Class xface) { set(name, theClass.getName()); } - /** + /** * Get a local file under a directory named by dirsProp with * the given path. If dirsProp contains multiple directories, * then one is chosen based on path's hash code. If the selected * directory does not exist, an attempt is made to create it. - * + * * @param dirsProp directory in which to locate the file. * @param path file-path. * @return local file under the directory with the given path. + * @throws IOException raised on errors performing I/O. */ public Path getLocalPath(String dirsProp, String path) throws IOException { @@ -2817,15 +2824,16 @@ public Path getLocalPath(String dirsProp, String path) throw new IOException("No valid local directories in property: "+dirsProp); } - /** + /** * Get a local file name under a directory named in dirsProp with * the given path. If dirsProp contains multiple directories, * then one is chosen based on path's hash code. If the selected * directory does not exist, an attempt is made to create it. - * + * * @param dirsProp directory in which to locate the file. * @param path file-path. * @return local file under the directory with the given path. + * @throws IOException raised on errors performing I/O. */ public File getFile(String dirsProp, String path) throws IOException { @@ -3437,7 +3445,7 @@ void parseNext() throws IOException, XMLStreamException { /** * Add tags defined in HADOOP_TAGS_SYSTEM, HADOOP_TAGS_CUSTOM. - * @param prop + * @param prop properties. */ public void addTags(Properties prop) { // Get all system tags @@ -3538,7 +3546,7 @@ private void loadProperty(Properties properties, String name, String attr, /** * Print a warning if a property with a given name already exists with a - * different value + * different value. */ private void checkForOverride(Properties properties, String name, String attr, String value) { String propertyValue = properties.getProperty(attr); @@ -3548,11 +3556,12 @@ private void checkForOverride(Properties properties, String name, String attr, S } } - /** + /** * Write out the non-default properties in this configuration to the given * {@link OutputStream} using UTF-8 encoding. - * + * * @param out the output stream to write to. + * @throws IOException raised on errors performing I/O. */ public void writeXml(OutputStream out) throws IOException { writeXml(new OutputStreamWriter(out, "UTF-8")); @@ -3582,7 +3591,9 @@ public void writeXml(Writer out) throws IOException { * the configuration, this method throws an {@link IllegalArgumentException}. * * + * @param propertyName xml property name. * @param out the writer to write to. + * @throws IOException raised on errors performing I/O. */ public void writeXml(@Nullable String propertyName, Writer out) throws IOException, IllegalArgumentException { @@ -3736,7 +3747,7 @@ private synchronized void appendXMLProperty(Document doc, Element conf, * @param config the configuration * @param propertyName property name * @param out the Writer to write to - * @throws IOException + * @throws IOException raised on errors performing I/O. * @throws IllegalArgumentException when property name is not * empty and the property is not found in configuration **/ @@ -3783,7 +3794,7 @@ public static void dumpConfiguration(Configuration config, * * @param config the configuration * @param out the Writer to write to - * @throws IOException + * @throws IOException raised on errors performing I/O. */ public static void dumpConfiguration(Configuration config, Writer out) throws IOException { @@ -3812,7 +3823,7 @@ public static void dumpConfiguration(Configuration config, * @param jsonGen json writer * @param config configuration * @param name property name - * @throws IOException + * @throws IOException raised on errors performing I/O. */ private static void appendJSONProperty(JsonGenerator jsonGen, Configuration config, String name, ConfigRedactor redactor) @@ -3894,7 +3905,10 @@ synchronized boolean getQuietMode() { return this.quietmode; } - /** For debugging. List non-default properties to the terminal and exit. */ + /** For debugging. List non-default properties to the terminal and exit. + * @param args the argument to be parsed. + * @throws Exception exception. + */ public static void main(String[] args) throws Exception { new Configuration().writeXml(System.out); } @@ -3928,8 +3942,8 @@ public void write(DataOutput out) throws IOException { } /** - * get keys matching the the regex - * @param regex + * get keys matching the the regex. + * @param regex the regex to match against. * @return {@literal Map} with matching keys */ public Map getValByRegex(String regex) { @@ -3974,6 +3988,8 @@ public static void dumpDeprecatedKeys() { /** * Returns whether or not a deprecated name has been warned. If the name is not * deprecated then always return false + * @param name proprties. + * @return true if name is a warned deprecation. */ public static boolean hasWarnedDeprecation(String name) { DeprecationContext deprecations = deprecationContext.get(); diff --git a/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/conf/Configured.java b/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/conf/Configured.java index f06af2b98df14..77a7117d19665 100644 --- a/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/conf/Configured.java +++ b/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/conf/Configured.java @@ -33,7 +33,9 @@ public Configured() { this(null); } - /** Construct a Configured. */ + /** Construct a Configured. + * @param conf the Configuration object. + */ public Configured(Configuration conf) { setConf(conf); } diff --git a/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/conf/Reconfigurable.java b/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/conf/Reconfigurable.java index c93dc31a881a9..915faf4c237ad 100644 --- a/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/conf/Reconfigurable.java +++ b/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/conf/Reconfigurable.java @@ -33,6 +33,9 @@ public interface Reconfigurable extends Configurable { * (or null if it was not previously set). If newVal is null, set the property * to its default value; * + * @param property property name. + * @param newVal new value. + * @throws ReconfigurationException if there was an error applying newVal. * If the property cannot be changed, throw a * {@link ReconfigurationException}. */ @@ -45,11 +48,14 @@ void reconfigureProperty(String property, String newVal) * If isPropertyReconfigurable returns true for a property, * then changeConf should not throw an exception when changing * this property. + * @param property property name. + * @return true if property reconfigurable; false if not. */ boolean isPropertyReconfigurable(String property); /** * Return all the properties that can be changed at run time. + * @return reconfigurable propertys. */ Collection getReconfigurableProperties(); } diff --git a/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/conf/ReconfigurableBase.java b/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/conf/ReconfigurableBase.java index 35dfeb99f0ba6..1c451ca6d30b9 100644 --- a/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/conf/ReconfigurableBase.java +++ b/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/conf/ReconfigurableBase.java @@ -79,6 +79,7 @@ public ReconfigurableBase() { /** * Construct a ReconfigurableBase with the {@link Configuration} * conf. + * @param conf configuration. */ public ReconfigurableBase(Configuration conf) { super((conf == null) ? new Configuration() : conf); @@ -91,6 +92,7 @@ public void setReconfigurationUtil(ReconfigurationUtil ru) { /** * Create a new configuration. + * @return configuration. */ protected abstract Configuration getNewConf(); @@ -162,6 +164,7 @@ public void run() { /** * Start a reconfiguration task to reload configuration in background. + * @throws IOException raised on errors performing I/O. */ public void startReconfigurationTask() throws IOException { synchronized (reconfigLock) { diff --git a/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/conf/ReconfigurationException.java b/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/conf/ReconfigurationException.java index 0935bf025fd30..b22af76c9eb6b 100644 --- a/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/conf/ReconfigurationException.java +++ b/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/conf/ReconfigurationException.java @@ -59,6 +59,10 @@ public ReconfigurationException() { /** * Create a new instance of {@link ReconfigurationException}. + * @param property property name. + * @param newVal new value. + * @param oldVal old value. + * @param cause original exception. */ public ReconfigurationException(String property, String newVal, String oldVal, @@ -71,6 +75,9 @@ public ReconfigurationException(String property, /** * Create a new instance of {@link ReconfigurationException}. + * @param property property name. + * @param newVal new value. + * @param oldVal old value. */ public ReconfigurationException(String property, String newVal, String oldVal) { @@ -82,6 +89,7 @@ public ReconfigurationException(String property, /** * Get property that cannot be changed. + * @return property info. */ public String getProperty() { return property; @@ -89,6 +97,7 @@ public String getProperty() { /** * Get value to which property was supposed to be changed. + * @return new value. */ public String getNewValue() { return newVal; @@ -96,6 +105,7 @@ public String getNewValue() { /** * Get old value of property that cannot be changed. + * @return old value. */ public String getOldValue() { return oldVal; diff --git a/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/conf/ReconfigurationTaskStatus.java b/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/conf/ReconfigurationTaskStatus.java index 05ec90758e5fa..ca9ddb61566ef 100644 --- a/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/conf/ReconfigurationTaskStatus.java +++ b/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/conf/ReconfigurationTaskStatus.java @@ -42,7 +42,8 @@ public ReconfigurationTaskStatus(long startTime, long endTime, /** * Return true if * - A reconfiguration task has finished or - * - an active reconfiguration task is running + * - an active reconfiguration task is running. + * @return true if startTime > 0; false if not. */ public boolean hasTask() { return startTime > 0; @@ -51,6 +52,7 @@ public boolean hasTask() { /** * Return true if the latest reconfiguration task has finished and there is * no another active task running. + * @return true if endTime > 0; false if not. */ public boolean stopped() { return endTime > 0; diff --git a/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/crypto/CryptoCodec.java b/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/crypto/CryptoCodec.java index 64c754faa59d8..e6813b96a2670 100644 --- a/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/crypto/CryptoCodec.java +++ b/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/crypto/CryptoCodec.java @@ -145,14 +145,18 @@ private static List> getCodecClasses( public abstract CipherSuite getCipherSuite(); /** - * Create a {@link org.apache.hadoop.crypto.Encryptor}. - * @return Encryptor the encryptor + * Create a {@link org.apache.hadoop.crypto.Encryptor}. + * + * @return Encryptor the encryptor. + * @throws GeneralSecurityException thrown if create encryptor error. */ public abstract Encryptor createEncryptor() throws GeneralSecurityException; - + /** * Create a {@link org.apache.hadoop.crypto.Decryptor}. + * * @return Decryptor the decryptor + * @throws GeneralSecurityException thrown if create decryptor error. */ public abstract Decryptor createDecryptor() throws GeneralSecurityException; diff --git a/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/crypto/CryptoInputStream.java b/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/crypto/CryptoInputStream.java index 5ab5d341fb826..067abde9dfbb8 100644 --- a/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/crypto/CryptoInputStream.java +++ b/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/crypto/CryptoInputStream.java @@ -157,7 +157,7 @@ public InputStream getWrappedStream() { * @param off the buffer offset. * @param len the maximum number of decrypted data bytes to read. * @return int the total number of decrypted data bytes read into the buffer. - * @throws IOException + * @throws IOException raised on errors performing I/O. */ @Override public int read(byte[] b, int off, int len) throws IOException { diff --git a/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/crypto/CryptoOutputStream.java b/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/crypto/CryptoOutputStream.java index 8e7522112551e..2a1335b6e745a 100644 --- a/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/crypto/CryptoOutputStream.java +++ b/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/crypto/CryptoOutputStream.java @@ -146,7 +146,7 @@ public OutputStream getWrappedStream() { * @param b the data. * @param off the start offset in the data. * @param len the number of bytes to write. - * @throws IOException + * @throws IOException raised on errors performing I/O. */ @Override public synchronized void write(byte[] b, int off, int len) throws IOException { diff --git a/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/crypto/CryptoStreamUtils.java b/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/crypto/CryptoStreamUtils.java index 318975fd6cebd..dad4d20df2afd 100644 --- a/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/crypto/CryptoStreamUtils.java +++ b/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/crypto/CryptoStreamUtils.java @@ -39,7 +39,11 @@ public class CryptoStreamUtils { private static final Logger LOG = LoggerFactory.getLogger(CryptoStreamUtils.class); - /** Forcibly free the direct buffer. */ + /** + * Forcibly free the direct buffer. + * + * @param buffer buffer. + */ public static void freeDB(ByteBuffer buffer) { if (CleanerUtil.UNMAP_SUPPORTED) { try { @@ -52,13 +56,22 @@ public static void freeDB(ByteBuffer buffer) { } } - /** Read crypto buffer size */ + /** + * Read crypto buffer size. + * + * @param conf configuration. + * @return hadoop.security.crypto.buffer.size. + */ public static int getBufferSize(Configuration conf) { return conf.getInt(HADOOP_SECURITY_CRYPTO_BUFFER_SIZE_KEY, HADOOP_SECURITY_CRYPTO_BUFFER_SIZE_DEFAULT); } - - /** AES/CTR/NoPadding or SM4/CTR/NoPadding is required. */ + + /** + * AES/CTR/NoPadding or SM4/CTR/NoPadding is required. + * + * @param codec crypto codec. + */ public static void checkCodec(CryptoCodec codec) { if (codec.getCipherSuite() != CipherSuite.AES_CTR_NOPADDING && codec.getCipherSuite() != CipherSuite.SM4_CTR_NOPADDING) { @@ -67,17 +80,27 @@ public static void checkCodec(CryptoCodec codec) { } } - /** Check and floor buffer size */ + /** + * Check and floor buffer size. + * + * @param codec crypto codec. + * @param bufferSize the size of the buffer to be used. + * @return calc buffer size. + */ public static int checkBufferSize(CryptoCodec codec, int bufferSize) { Preconditions.checkArgument(bufferSize >= MIN_BUFFER_SIZE, "Minimum value of buffer size is " + MIN_BUFFER_SIZE + "."); return bufferSize - bufferSize % codec.getCipherSuite() .getAlgorithmBlockSize(); } - + /** * If input stream is {@link org.apache.hadoop.fs.Seekable}, return it's * current position, otherwise return 0; + * + * @param in wrapper. + * @return current position, otherwise return 0. + * @throws IOException raised on errors performing I/O. */ public static long getInputStreamOffset(InputStream in) throws IOException { if (in instanceof Seekable) { diff --git a/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/crypto/OpensslCipher.java b/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/crypto/OpensslCipher.java index 0c65b74b2913b..b166cfc8611b3 100644 --- a/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/crypto/OpensslCipher.java +++ b/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/crypto/OpensslCipher.java @@ -225,34 +225,33 @@ public int update(ByteBuffer input, ByteBuffer output) output.position(output.position() + len); return len; } - + /** * Finishes a multiple-part operation. The data is encrypted or decrypted, * depending on how this cipher was initialized. *

    - * * The result is stored in the output buffer. Upon return, the output buffer's * position will have advanced by n, where n is the value returned by this * method; the output buffer's limit will not have changed. - *

    - * + *

    * If output.remaining() bytes are insufficient to hold the result, * a ShortBufferException is thrown. *

    - * * Upon finishing, this method resets this cipher object to the state it was * in when previously initialized. That is, the object is available to encrypt * or decrypt more data. - *

    - * - * If any exception is thrown, this cipher object need to be reset before it + *

    + * If any exception is thrown, this cipher object need to be reset before it * can be used again. - * + * * @param output the output ByteBuffer * @return int number of bytes stored in output - * @throws ShortBufferException - * @throws IllegalBlockSizeException - * @throws BadPaddingException + * @throws ShortBufferException if there is insufficient space in the output buffer. + * @throws IllegalBlockSizeException This exception is thrown when the length + * of data provided to a block cipher is incorrect. + * @throws BadPaddingException This exception is thrown when a particular + * padding mechanism is expected for the input + * data but the data is not padded properly. */ public int doFinal(ByteBuffer output) throws ShortBufferException, IllegalBlockSizeException, BadPaddingException { diff --git a/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/crypto/key/KeyProvider.java b/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/crypto/key/KeyProvider.java index dafdaf7e15b25..4d1674bd7b883 100644 --- a/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/crypto/key/KeyProvider.java +++ b/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/crypto/key/KeyProvider.java @@ -242,7 +242,7 @@ protected int addVersion() { /** * Serialize the metadata to a set of bytes. * @return the serialized bytes - * @throws IOException + * @throws IOException raised on errors performing I/O. */ protected byte[] serialize() throws IOException { ByteArrayOutputStream buffer = new ByteArrayOutputStream(); @@ -281,7 +281,7 @@ protected byte[] serialize() throws IOException { /** * Deserialize a new metadata object from a set of bytes. * @param bytes the serialized metadata - * @throws IOException + * @throws IOException raised on errors performing I/O. */ protected Metadata(byte[] bytes) throws IOException { String cipher = null; @@ -450,7 +450,7 @@ public boolean isTransient() { * when decrypting data. * @param versionName the name of a specific version of the key * @return the key material - * @throws IOException + * @throws IOException raised on errors performing I/O. */ public abstract KeyVersion getKeyVersion(String versionName ) throws IOException; @@ -458,14 +458,15 @@ public abstract KeyVersion getKeyVersion(String versionName /** * Get the key names for all keys. * @return the list of key names - * @throws IOException + * @throws IOException raised on errors performing I/O. */ public abstract List getKeys() throws IOException; /** * Get key metadata in bulk. * @param names the names of the keys to get - * @throws IOException + * @throws IOException raised on errors performing I/O. + * @return Metadata Array. */ public Metadata[] getKeysMetadata(String... names) throws IOException { Metadata[] result = new Metadata[names.length]; @@ -477,8 +478,10 @@ public Metadata[] getKeysMetadata(String... names) throws IOException { /** * Get the key material for all versions of a specific key name. + * + * @param name the base name of the key. * @return the list of key material - * @throws IOException + * @throws IOException raised on errors performing I/O. */ public abstract List getKeyVersions(String name) throws IOException; @@ -488,7 +491,7 @@ public Metadata[] getKeysMetadata(String... names) throws IOException { * @param name the base name of the key * @return the version name of the current version of the key or null if the * key version doesn't exist - * @throws IOException + * @throws IOException raised on errors performing I/O. */ public KeyVersion getCurrentKey(String name) throws IOException { Metadata meta = getMetadata(name); @@ -502,7 +505,7 @@ public KeyVersion getCurrentKey(String name) throws IOException { * Get metadata about the key. * @param name the basename of the key * @return the key's metadata or null if the key doesn't exist - * @throws IOException + * @throws IOException raised on errors performing I/O. */ public abstract Metadata getMetadata(String name) throws IOException; @@ -512,7 +515,7 @@ public KeyVersion getCurrentKey(String name) throws IOException { * @param material the key material for the first version of the key. * @param options the options for the new key. * @return the version name of the first version of the key. - * @throws IOException + * @throws IOException raised on errors performing I/O. */ public abstract KeyVersion createKey(String name, byte[] material, Options options) throws IOException; @@ -537,7 +540,7 @@ private String getAlgorithm(String cipher) { * @param size length of the key. * @param algorithm algorithm to use for generating the key. * @return the generated key. - * @throws NoSuchAlgorithmException + * @throws NoSuchAlgorithmException no such algorithm exception. */ protected byte[] generateKey(int size, String algorithm) throws NoSuchAlgorithmException { @@ -558,8 +561,8 @@ protected byte[] generateKey(int size, String algorithm) * @param name the base name of the key * @param options the options for the new key. * @return the version name of the first version of the key. - * @throws IOException - * @throws NoSuchAlgorithmException + * @throws IOException raised on errors performing I/O. + * @throws NoSuchAlgorithmException no such algorithm exception. */ public KeyVersion createKey(String name, Options options) throws NoSuchAlgorithmException, IOException { @@ -570,7 +573,7 @@ public KeyVersion createKey(String name, Options options) /** * Delete the given key. * @param name the name of the key to delete - * @throws IOException + * @throws IOException raised on errors performing I/O. */ public abstract void deleteKey(String name) throws IOException; @@ -579,7 +582,7 @@ public KeyVersion createKey(String name, Options options) * @param name the basename of the key * @param material the new key material * @return the name of the new version of the key - * @throws IOException + * @throws IOException raised on errors performing I/O. */ public abstract KeyVersion rollNewVersion(String name, byte[] material @@ -601,7 +604,10 @@ public void close() throws IOException { * * @param name the basename of the key * @return the name of the new version of the key - * @throws IOException + * @throws IOException raised on errors performing I/O. + * @throws NoSuchAlgorithmException This exception is thrown when a particular + * cryptographic algorithm is requested + * but is not available in the environment. */ public KeyVersion rollNewVersion(String name) throws NoSuchAlgorithmException, IOException { @@ -620,7 +626,7 @@ public KeyVersion rollNewVersion(String name) throws NoSuchAlgorithmException, * version of the given key. * * @param name the basename of the key - * @throws IOException + * @throws IOException raised on errors performing I/O. */ public void invalidateCache(String name) throws IOException { // NOP @@ -628,7 +634,7 @@ public void invalidateCache(String name) throws IOException { /** * Ensures that any changes to the keys are written to persistent store. - * @throws IOException + * @throws IOException raised on errors performing I/O. */ public abstract void flush() throws IOException; @@ -637,7 +643,7 @@ public void invalidateCache(String name) throws IOException { * "/aaa/bbb". * @param versionName the version name to split * @return the base name of the key - * @throws IOException + * @throws IOException raised on errors performing I/O. */ public static String getBaseName(String versionName) throws IOException { int div = versionName.lastIndexOf('@'); @@ -660,9 +666,11 @@ protected static String buildVersionName(String name, int version) { /** * Find the provider with the given key. + * * @param providerList the list of providers - * @param keyName the key name we are looking for + * @param keyName the key name we are looking for. * @return the KeyProvider that has the key + * @throws IOException raised on errors performing I/O. */ public static KeyProvider findProvider(List providerList, String keyName) throws IOException { @@ -680,7 +688,7 @@ public static KeyProvider findProvider(List providerList, * means. If true, the password should be provided by the caller using * setPassword(). * @return Whether or not the provider requires a password - * @throws IOException + * @throws IOException raised on errors performing I/O. */ public boolean needsPassword() throws IOException { return false; diff --git a/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/crypto/key/KeyProviderCryptoExtension.java b/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/crypto/key/KeyProviderCryptoExtension.java index 3f3c367fc3933..f1bb314582038 100644 --- a/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/crypto/key/KeyProviderCryptoExtension.java +++ b/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/crypto/key/KeyProviderCryptoExtension.java @@ -25,10 +25,6 @@ import java.util.List; import java.util.ListIterator; -import javax.crypto.Cipher; -import javax.crypto.spec.IvParameterSpec; -import javax.crypto.spec.SecretKeySpec; - import org.apache.hadoop.util.Preconditions; import org.apache.hadoop.classification.InterfaceAudience; import org.apache.hadoop.crypto.CryptoCodec; @@ -178,6 +174,7 @@ public interface CryptoExtension extends KeyProviderExtension.Extension { * Calls to this method allows the underlying KeyProvider to warm-up any * implementation specific caches used to store the Encrypted Keys. * @param keyNames Array of Key Names + * @throws IOException thrown if the key material could not be encrypted. */ public void warmUpEncryptedKeys(String... keyNames) throws IOException; @@ -474,8 +471,9 @@ public void drain(String keyName) { /** * This constructor is to be used by sub classes that provide * delegating/proxying functionality to the {@link KeyProviderCryptoExtension} - * @param keyProvider - * @param extension + * + * @param keyProvider key provider. + * @param extension crypto extension. */ protected KeyProviderCryptoExtension(KeyProvider keyProvider, CryptoExtension extension) { @@ -486,6 +484,7 @@ protected KeyProviderCryptoExtension(KeyProvider keyProvider, * Notifies the Underlying CryptoExtension implementation to warm up any * implementation specific caches for the specified KeyVersions * @param keyNames Arrays of key Names + * @throws IOException raised on errors performing I/O. */ public void warmUpEncryptedKeys(String... keyNames) throws IOException { @@ -557,7 +556,7 @@ public EncryptedKeyVersion reencryptEncryptedKey(EncryptedKeyVersion ekv) * Calls {@link CryptoExtension#drain(String)} for the given key name on the * underlying {@link CryptoExtension}. * - * @param keyName + * @param keyName key name. */ public void drain(String keyName) { getExtension().drain(keyName); diff --git a/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/crypto/key/KeyProviderDelegationTokenExtension.java b/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/crypto/key/KeyProviderDelegationTokenExtension.java index 1fdc2fe12455b..3c1af424eb7cd 100644 --- a/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/crypto/key/KeyProviderDelegationTokenExtension.java +++ b/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/crypto/key/KeyProviderDelegationTokenExtension.java @@ -48,14 +48,14 @@ public interface DelegationTokenExtension * Renews the given token. * @param token The token to be renewed. * @return The token's lifetime after renewal, or 0 if it can't be renewed. - * @throws IOException + * @throws IOException raised on errors performing I/O. */ long renewDelegationToken(final Token token) throws IOException; /** * Cancels the given token. * @param token The token to be cancelled. - * @throws IOException + * @throws IOException raised on errors performing I/O. */ Void cancelDelegationToken(final Token token) throws IOException; diff --git a/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/crypto/key/KeyShell.java b/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/crypto/key/KeyShell.java index a75f7d3aa63bd..c18d0d41bc08a 100644 --- a/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/crypto/key/KeyShell.java +++ b/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/crypto/key/KeyShell.java @@ -75,7 +75,7 @@ public class KeyShell extends CommandShell { * * @param args Command line arguments. * @return 0 on success, 1 on failure. - * @throws IOException + * @throws IOException raised on errors performing I/O. */ @Override protected int init(String[] args) throws IOException { @@ -547,7 +547,7 @@ private String prettifyException(Exception e) { * success and 1 for failure. * * @param args Command line arguments. - * @throws Exception + * @throws Exception raised on errors performing I/O. */ public static void main(String[] args) throws Exception { int res = ToolRunner.run(new Configuration(), new KeyShell(), args); diff --git a/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/crypto/key/kms/LoadBalancingKMSClientProvider.java b/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/crypto/key/kms/LoadBalancingKMSClientProvider.java index 6f8f4585ee75f..f9cc3f4524ff5 100644 --- a/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/crypto/key/kms/LoadBalancingKMSClientProvider.java +++ b/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/crypto/key/kms/LoadBalancingKMSClientProvider.java @@ -21,6 +21,7 @@ import java.io.IOException; import java.io.InterruptedIOException; import java.net.ConnectException; +import java.net.SocketException; import java.net.URI; import java.security.GeneralSecurityException; import java.security.NoSuchAlgorithmException; @@ -29,7 +30,7 @@ import java.util.List; import java.util.concurrent.atomic.AtomicInteger; -import javax.net.ssl.SSLHandshakeException; +import javax.net.ssl.SSLException; import org.apache.hadoop.conf.Configuration; import org.apache.hadoop.crypto.key.KeyProvider; @@ -182,10 +183,10 @@ private T doOp(ProviderCallable op, int currPos, } catch (IOException ioe) { LOG.warn("KMS provider at [{}] threw an IOException: ", provider.getKMSUrl(), ioe); - // SSLHandshakeException can occur here because of lost connection + // SSLException can occur here because of lost connection // with the KMS server, creating a ConnectException from it, // so that the FailoverOnNetworkExceptionRetry policy will retry - if (ioe instanceof SSLHandshakeException) { + if (ioe instanceof SSLException || ioe instanceof SocketException) { Exception cause = ioe; ioe = new ConnectException("SSLHandshakeException: " + cause.getMessage()); diff --git a/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/crypto/key/kms/ValueQueue.java b/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/crypto/key/kms/ValueQueue.java index be2db05842c8e..65eded918d60d 100644 --- a/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/crypto/key/kms/ValueQueue.java +++ b/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/crypto/key/kms/ValueQueue.java @@ -63,7 +63,7 @@ public interface QueueRefiller { * @param keyName Key name * @param keyQueue Queue that needs to be filled * @param numValues number of Values to be added to the queue. - * @throws IOException + * @throws IOException raised on errors performing I/O. */ public void fillQueueForKey(String keyName, Queue keyQueue, int numValues) throws IOException; @@ -268,7 +268,7 @@ public ValueQueue(final int numValues, final float lowWaterMark, long expiry, * Initializes the Value Queues for the provided keys by calling the * fill Method with "numInitValues" values * @param keyNames Array of key Names - * @throws ExecutionException + * @throws ExecutionException executionException. */ public void initializeQueuesForKeys(String... keyNames) throws ExecutionException { @@ -285,8 +285,8 @@ public void initializeQueuesForKeys(String... keyNames) * function to add 1 value to Queue and then drain it. * @param keyName String key name * @return E the next value in the Queue - * @throws IOException - * @throws ExecutionException + * @throws IOException raised on errors performing I/O. + * @throws ExecutionException executionException. */ public E getNext(String keyName) throws IOException, ExecutionException { @@ -344,8 +344,8 @@ public int getSize(String keyName) { * @param keyName String key name * @param num Minimum number of values to return. * @return {@literal List} values returned - * @throws IOException - * @throws ExecutionException + * @throws IOException raised on errors performing I/O. + * @throws ExecutionException execution exception. */ public List getAtMost(String keyName, int num) throws IOException, ExecutionException { diff --git a/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/fs/AbstractFileSystem.java b/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/fs/AbstractFileSystem.java index d9818b472f0e5..a4737c548c8fa 100644 --- a/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/fs/AbstractFileSystem.java +++ b/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/fs/AbstractFileSystem.java @@ -272,7 +272,7 @@ public static AbstractFileSystem get(final URI uri, final Configuration conf) * @param supportedScheme the scheme supported by the implementor * @param authorityNeeded if true then theURI must have authority, if false * then the URI must have null authority. - * + * @param defaultPort default port to use if port is not specified in the URI. * @throws URISyntaxException uri has syntax error */ public AbstractFileSystem(final URI uri, final String supportedScheme, @@ -281,11 +281,12 @@ public AbstractFileSystem(final URI uri, final String supportedScheme, myUri = getUri(uri, supportedScheme, authorityNeeded, defaultPort); statistics = getStatistics(uri); } - + /** - * Check that the Uri's scheme matches - * @param uri - * @param supportedScheme + * Check that the Uri's scheme matches. + * + * @param uri name URI of the FS. + * @param supportedScheme supported scheme. */ public void checkScheme(URI uri, String supportedScheme) { String scheme = uri.getScheme(); @@ -362,7 +363,7 @@ public URI getUri() { * If the path is fully qualified URI, then its scheme and authority * matches that of this file system. Otherwise the path must be * slash-relative name. - * + * @param path the path. * @throws InvalidPathException if the path is invalid */ public void checkPath(Path path) { @@ -431,7 +432,7 @@ public String getUriPath(final Path p) { /** * Make the path fully qualified to this file system - * @param path + * @param path the path. * @return the qualified path */ public Path makeQualified(Path path) { @@ -496,9 +497,9 @@ public FsServerDefaults getServerDefaults(final Path f) throws IOException { * through any internal symlinks or mount point * @param p path to be resolved * @return fully qualified path - * @throws FileNotFoundException - * @throws AccessControlException - * @throws IOException + * @throws FileNotFoundException when file not find throw. + * @throws AccessControlException when accees control error throw. + * @throws IOException raised on errors performing I/O. * @throws UnresolvedLinkException if symbolic link on path cannot be * resolved internally */ @@ -513,6 +514,18 @@ public Path resolvePath(final Path p) throws FileNotFoundException, * {@link FileContext#create(Path, EnumSet, Options.CreateOpts...)} except * that the Path f must be fully qualified and the permission is absolute * (i.e. umask has been applied). + * + * @param f the path. + * @param createFlag create_flag. + * @param opts create ops. + * @throws AccessControlException access controll exception. + * @throws FileAlreadyExistsException file already exception. + * @throws FileNotFoundException file not found exception. + * @throws ParentNotDirectoryException parent not dir exception. + * @throws UnsupportedFileSystemException unsupported file system exception. + * @throws UnresolvedLinkException unresolved link exception. + * @throws IOException raised on errors performing I/O. + * @return output stream. */ public final FSDataOutputStream create(final Path f, final EnumSet createFlag, Options.CreateOpts... opts) @@ -630,6 +643,24 @@ public final FSDataOutputStream create(final Path f, * The specification of this method matches that of * {@link #create(Path, EnumSet, Options.CreateOpts...)} except that the opts * have been declared explicitly. + * + * @param f the path. + * @param flag create flag. + * @param absolutePermission absolute permission. + * @param bufferSize buffer size. + * @param replication replications. + * @param blockSize block size. + * @param progress progress. + * @param checksumOpt check sum opt. + * @param createParent create parent. + * @throws AccessControlException access control exception. + * @throws FileAlreadyExistsException file already exists exception. + * @throws FileNotFoundException file not found exception. + * @throws ParentNotDirectoryException parent not directory exception. + * @throws UnsupportedFileSystemException unsupported filesystem exception. + * @throws UnresolvedLinkException unresolved link exception. + * @throws IOException raised on errors performing I/O. + * @return output stream. */ public abstract FSDataOutputStream createInternal(Path f, EnumSet flag, FsPermission absolutePermission, @@ -644,6 +675,14 @@ public abstract FSDataOutputStream createInternal(Path f, * {@link FileContext#mkdir(Path, FsPermission, boolean)} except that the Path * f must be fully qualified and the permission is absolute (i.e. * umask has been applied). + * @param dir directory. + * @param permission permission. + * @param createParent create parent flag. + * @throws AccessControlException access control exception. + * @throws FileAlreadyExistsException file already exists exception. + * @throws FileNotFoundException file not found exception. + * @throws UnresolvedLinkException unresolved link exception. + * @throws IOException raised on errors performing I/O. */ public abstract void mkdir(final Path dir, final FsPermission permission, final boolean createParent) throws AccessControlException, @@ -654,6 +693,14 @@ public abstract void mkdir(final Path dir, final FsPermission permission, * The specification of this method matches that of * {@link FileContext#delete(Path, boolean)} except that Path f must be for * this file system. + * + * @param f the path. + * @param recursive recursive flag. + * @throws AccessControlException access control exception. + * @throws FileNotFoundException file not found exception. + * @throws UnresolvedLinkException unresolved link exception. + * @throws IOException raised on errors performing I/O. + * @return if successfully deleted success true, not false. */ public abstract boolean delete(final Path f, final boolean recursive) throws AccessControlException, FileNotFoundException, @@ -663,6 +710,13 @@ public abstract boolean delete(final Path f, final boolean recursive) * The specification of this method matches that of * {@link FileContext#open(Path)} except that Path f must be for this * file system. + * + * @param f the path. + * @throws AccessControlException access control exception. + * @throws FileNotFoundException file not found exception. + * @throws UnresolvedLinkException unresolved link exception. + * @throws IOException raised on errors performing I/O. + * @return input stream. */ public FSDataInputStream open(final Path f) throws AccessControlException, FileNotFoundException, UnresolvedLinkException, IOException { @@ -673,6 +727,14 @@ public FSDataInputStream open(final Path f) throws AccessControlException, * The specification of this method matches that of * {@link FileContext#open(Path, int)} except that Path f must be for this * file system. + * + * @param f the path. + * @param bufferSize buffer size. + * @throws AccessControlException access control exception. + * @throws FileNotFoundException file not found exception. + * @throws UnresolvedLinkException unresolved link exception. + * @throws IOException raised on errors performing I/O. + * @return if successfully open success true, not false. */ public abstract FSDataInputStream open(final Path f, int bufferSize) throws AccessControlException, FileNotFoundException, @@ -682,6 +744,14 @@ public abstract FSDataInputStream open(final Path f, int bufferSize) * The specification of this method matches that of * {@link FileContext#truncate(Path, long)} except that Path f must be for * this file system. + * + * @param f the path. + * @param newLength new length. + * @throws AccessControlException access control exception. + * @throws FileNotFoundException file not found exception. + * @throws UnresolvedLinkException unresolved link exception. + * @throws IOException raised on errors performing I/O. + * @return if successfully truncate success true, not false. */ public boolean truncate(Path f, long newLength) throws AccessControlException, FileNotFoundException, @@ -694,6 +764,14 @@ public boolean truncate(Path f, long newLength) * The specification of this method matches that of * {@link FileContext#setReplication(Path, short)} except that Path f must be * for this file system. + * + * @param f the path. + * @param replication replication. + * @return if successfully set replication success true, not false. + * @throws AccessControlException access control exception. + * @throws FileNotFoundException file not found exception. + * @throws UnresolvedLinkException unresolved link exception. + * @throws IOException raised on errors performing I/O. */ public abstract boolean setReplication(final Path f, final short replication) throws AccessControlException, @@ -703,6 +781,16 @@ public abstract boolean setReplication(final Path f, * The specification of this method matches that of * {@link FileContext#rename(Path, Path, Options.Rename...)} except that Path * f must be for this file system. + * + * @param src src. + * @param dst dst. + * @param options options. + * @throws AccessControlException access control exception. + * @throws FileAlreadyExistsException file already exists exception. + * @throws FileNotFoundException file not found exception. + * @throws ParentNotDirectoryException parent not directory exception. + * @throws UnresolvedLinkException unresolved link exception. + * @throws IOException raised on errors performing I/O. */ public final void rename(final Path src, final Path dst, final Options.Rename... options) throws AccessControlException, @@ -727,6 +815,15 @@ public final void rename(final Path src, final Path dst, * File systems that do not have a built in overwrite need implement only this * method and can take advantage of the default impl of the other * {@link #renameInternal(Path, Path, boolean)} + * + * @param src src. + * @param dst dst. + * @throws AccessControlException access control exception. + * @throws FileAlreadyExistsException file already exists exception. + * @throws FileNotFoundException file not found exception. + * @throws ParentNotDirectoryException parent not directory exception. + * @throws UnresolvedLinkException unresolved link exception. + * @throws IOException raised on errors performing I/O. */ public abstract void renameInternal(final Path src, final Path dst) throws AccessControlException, FileAlreadyExistsException, @@ -737,6 +834,16 @@ public abstract void renameInternal(final Path src, final Path dst) * The specification of this method matches that of * {@link FileContext#rename(Path, Path, Options.Rename...)} except that Path * f must be for this file system. + * + * @param src src. + * @param dst dst. + * @param overwrite overwrite flag. + * @throws AccessControlException access control exception. + * @throws FileAlreadyExistsException file already exists exception. + * @throws FileNotFoundException file not found exception. + * @throws ParentNotDirectoryException parent not directory exception. + * @throws UnresolvedLinkException unresolved link exception. + * @throws IOException raised on errors performing I/O. */ public void renameInternal(final Path src, final Path dst, boolean overwrite) throws AccessControlException, @@ -800,6 +907,12 @@ public boolean supportsSymlinks() { /** * The specification of this method matches that of * {@link FileContext#createSymlink(Path, Path, boolean)}; + * + * @param target target. + * @param link link. + * @param createParent create parent. + * @throws IOException raised on errors performing I/O. + * @throws UnresolvedLinkException unresolved link exception. */ public void createSymlink(final Path target, final Path link, final boolean createParent) throws IOException, UnresolvedLinkException { @@ -810,6 +923,8 @@ public void createSymlink(final Path target, final Path link, * Partially resolves the path. This is used during symlink resolution in * {@link FSLinkResolver}, and differs from the similarly named method * {@link FileContext#getLinkTarget(Path)}. + * @param f the path. + * @return target path. * @throws IOException subclass implementations may throw IOException */ public Path getLinkTarget(final Path f) throws IOException { @@ -822,6 +937,13 @@ public Path getLinkTarget(final Path f) throws IOException { * The specification of this method matches that of * {@link FileContext#setPermission(Path, FsPermission)} except that Path f * must be for this file system. + * + * @param f the path. + * @param permission permission. + * @throws AccessControlException access control exception. + * @throws FileNotFoundException file not found exception. + * @throws UnresolvedLinkException unresolved link exception. + * @throws IOException raised on errors performing I/O. */ public abstract void setPermission(final Path f, final FsPermission permission) throws AccessControlException, @@ -831,6 +953,14 @@ public abstract void setPermission(final Path f, * The specification of this method matches that of * {@link FileContext#setOwner(Path, String, String)} except that Path f must * be for this file system. + * + * @param f the path. + * @param username username. + * @param groupname groupname. + * @throws AccessControlException access control exception. + * @throws FileNotFoundException file not found exception. + * @throws UnresolvedLinkException unresolved link exception. + * @throws IOException raised on errors performing I/O. */ public abstract void setOwner(final Path f, final String username, final String groupname) throws AccessControlException, @@ -840,6 +970,14 @@ public abstract void setOwner(final Path f, final String username, * The specification of this method matches that of * {@link FileContext#setTimes(Path, long, long)} except that Path f must be * for this file system. + * + * @param f the path. + * @param mtime modify time. + * @param atime access time. + * @throws AccessControlException access control exception. + * @throws FileNotFoundException file not found exception. + * @throws UnresolvedLinkException unresolved link exception. + * @throws IOException raised on errors performing I/O. */ public abstract void setTimes(final Path f, final long mtime, final long atime) throws AccessControlException, FileNotFoundException, @@ -849,6 +987,13 @@ public abstract void setTimes(final Path f, final long mtime, * The specification of this method matches that of * {@link FileContext#getFileChecksum(Path)} except that Path f must be for * this file system. + * + * @param f the path. + * @throws AccessControlException access control exception. + * @throws FileNotFoundException file not found exception. + * @throws UnresolvedLinkException unresolved link exception. + * @throws IOException raised on errors performing I/O. + * @return File Check sum. */ public abstract FileChecksum getFileChecksum(final Path f) throws AccessControlException, FileNotFoundException, @@ -859,6 +1004,13 @@ public abstract FileChecksum getFileChecksum(final Path f) * {@link FileContext#getFileStatus(Path)} * except that an UnresolvedLinkException may be thrown if a symlink is * encountered in the path. + * + * @param f the path. + * @throws AccessControlException access control exception. + * @throws FileNotFoundException file not found exception. + * @throws UnresolvedLinkException unresolved link exception. + * @throws IOException raised on errors performing I/O. + * @return File Status */ public abstract FileStatus getFileStatus(final Path f) throws AccessControlException, FileNotFoundException, @@ -870,8 +1022,8 @@ public abstract FileStatus getFileStatus(final Path f) * In some FileSystem implementations such as HDFS metadata * synchronization is essential to guarantee consistency of read requests * particularly in HA setting. - * @throws IOException - * @throws UnsupportedOperationException + * @throws IOException raised on errors performing I/O. + * @throws UnsupportedOperationException Unsupported Operation Exception. */ public void msync() throws IOException, UnsupportedOperationException { throw new UnsupportedOperationException(getClass().getCanonicalName() + @@ -883,6 +1035,13 @@ public void msync() throws IOException, UnsupportedOperationException { * {@link FileContext#access(Path, FsAction)} * except that an UnresolvedLinkException may be thrown if a symlink is * encountered in the path. + * + * @param path the path. + * @param mode fsaction mode. + * @throws AccessControlException access control exception. + * @throws FileNotFoundException file not found exception. + * @throws UnresolvedLinkException unresolved link exception. + * @throws IOException raised on errors performing I/O. */ @InterfaceAudience.LimitedPrivate({"HDFS", "Hive"}) public void access(Path path, FsAction mode) throws AccessControlException, @@ -897,6 +1056,13 @@ public void access(Path path, FsAction mode) throws AccessControlException, * encountered in the path leading up to the final path component. * If the file system does not support symlinks then the behavior is * equivalent to {@link AbstractFileSystem#getFileStatus(Path)}. + * + * @param f the path. + * @throws AccessControlException access control exception. + * @throws FileNotFoundException file not found exception. + * @throws UnsupportedFileSystemException UnSupported File System Exception. + * @throws IOException raised on errors performing I/O. + * @return file status. */ public FileStatus getFileLinkStatus(final Path f) throws AccessControlException, FileNotFoundException, @@ -908,6 +1074,15 @@ public FileStatus getFileLinkStatus(final Path f) * The specification of this method matches that of * {@link FileContext#getFileBlockLocations(Path, long, long)} except that * Path f must be for this file system. + * + * @param f the path. + * @param start start. + * @param len length. + * @throws AccessControlException access control exception. + * @throws FileNotFoundException file not found exception. + * @throws UnresolvedLinkException unresolved link exception. + * @throws IOException raised on errors performing I/O. + * @return BlockLocation Array. */ public abstract BlockLocation[] getFileBlockLocations(final Path f, final long start, final long len) throws AccessControlException, @@ -917,6 +1092,13 @@ public abstract BlockLocation[] getFileBlockLocations(final Path f, * The specification of this method matches that of * {@link FileContext#getFsStatus(Path)} except that Path f must be for this * file system. + * + * @param f the path. + * @throws AccessControlException access control exception. + * @throws FileNotFoundException file not found exception. + * @throws UnresolvedLinkException unresolved link exception. + * @throws IOException raised on errors performing I/O. + * @return Fs Status. */ public FsStatus getFsStatus(final Path f) throws AccessControlException, FileNotFoundException, UnresolvedLinkException, IOException { @@ -927,6 +1109,11 @@ public FsStatus getFsStatus(final Path f) throws AccessControlException, /** * The specification of this method matches that of * {@link FileContext#getFsStatus(Path)}. + * + * @throws AccessControlException access control exception. + * @throws FileNotFoundException file not found exception. + * @throws IOException raised on errors performing I/O. + * @return Fs Status. */ public abstract FsStatus getFsStatus() throws AccessControlException, FileNotFoundException, IOException; @@ -935,6 +1122,13 @@ public abstract FsStatus getFsStatus() throws AccessControlException, * The specification of this method matches that of * {@link FileContext#listStatus(Path)} except that Path f must be for this * file system. + * + * @param f path. + * @throws AccessControlException access control exception. + * @throws FileNotFoundException file not found exception. + * @throws UnresolvedLinkException unresolved link exception. + * @throws IOException raised on errors performing I/O. + * @return FileStatus Iterator. */ public RemoteIterator listStatusIterator(final Path f) throws AccessControlException, FileNotFoundException, @@ -967,6 +1161,13 @@ public FileStatus next() { * will have different formats for replicated and erasure coded file. Please * refer to {@link FileSystem#getFileBlockLocations(FileStatus, long, long)} * for more details. + * + * @param f the path. + * @throws AccessControlException access control exception. + * @throws FileNotFoundException file not found exception. + * @throws UnresolvedLinkException unresolved link exception. + * @throws IOException raised on errors performing I/O. + * @return FileStatus Iterator. */ public RemoteIterator listLocatedStatus(final Path f) throws AccessControlException, FileNotFoundException, @@ -999,6 +1200,12 @@ public LocatedFileStatus next() throws IOException { * The specification of this method matches that of * {@link FileContext.Util#listStatus(Path)} except that Path f must be * for this file system. + * @param f the path. + * @throws AccessControlException access control exception. + * @throws FileNotFoundException file not found exception. + * @throws UnresolvedLinkException unresolved link exception. + * @throws IOException raised on errors performing I/O. + * @return FileStatus Iterator. */ public abstract FileStatus[] listStatus(final Path f) throws AccessControlException, FileNotFoundException, @@ -1007,7 +1214,8 @@ public abstract FileStatus[] listStatus(final Path f) /** * @return an iterator over the corrupt files under the given path * (may contain duplicates if a file has more than one corrupt block) - * @throws IOException + * @param path the path. + * @throws IOException raised on errors performing I/O. */ public RemoteIterator listCorruptFileBlocks(Path path) throws IOException { @@ -1020,6 +1228,10 @@ public RemoteIterator listCorruptFileBlocks(Path path) * The specification of this method matches that of * {@link FileContext#setVerifyChecksum(boolean, Path)} except that Path f * must be for this file system. + * + * @param verifyChecksum verify check sum flag. + * @throws AccessControlException access control exception. + * @throws IOException raised on errors performing I/O. */ public abstract void setVerifyChecksum(final boolean verifyChecksum) throws AccessControlException, IOException; @@ -1041,7 +1253,7 @@ public String getCanonicalServiceName() { * @param renewer the account name that is allowed to renew the token. * @return List of delegation tokens. * If delegation tokens not supported then return a list of size zero. - * @throws IOException + * @throws IOException raised on errors performing I/O. */ @InterfaceAudience.LimitedPrivate( { "HDFS", "MapReduce" }) public List> getDelegationTokens(String renewer) throws IOException { @@ -1141,7 +1353,7 @@ public AclStatus getAclStatus(Path path) throws IOException { * @param path Path to modify * @param name xattr name. * @param value xattr value. - * @throws IOException + * @throws IOException raised on errors performing I/O. */ public void setXAttr(Path path, String name, byte[] value) throws IOException { @@ -1160,7 +1372,7 @@ public void setXAttr(Path path, String name, byte[] value) * @param name xattr name. * @param value xattr value. * @param flag xattr set flag - * @throws IOException + * @throws IOException raised on errors performing I/O. */ public void setXAttr(Path path, String name, byte[] value, EnumSet flag) throws IOException { @@ -1178,7 +1390,7 @@ public void setXAttr(Path path, String name, byte[] value, * @param path Path to get extended attribute * @param name xattr name. * @return byte[] xattr value. - * @throws IOException + * @throws IOException raised on errors performing I/O. */ public byte[] getXAttr(Path path, String name) throws IOException { throw new UnsupportedOperationException(getClass().getSimpleName() @@ -1196,7 +1408,7 @@ public byte[] getXAttr(Path path, String name) throws IOException { * * @return {@literal Map} describing the XAttrs of the file * or directory - * @throws IOException + * @throws IOException raised on errors performing I/O. */ public Map getXAttrs(Path path) throws IOException { throw new UnsupportedOperationException(getClass().getSimpleName() @@ -1214,7 +1426,7 @@ public Map getXAttrs(Path path) throws IOException { * @param names XAttr names. * @return {@literal Map} describing the XAttrs of the file * or directory - * @throws IOException + * @throws IOException raised on errors performing I/O. */ public Map getXAttrs(Path path, List names) throws IOException { @@ -1232,7 +1444,7 @@ public Map getXAttrs(Path path, List names) * @param path Path to get extended attributes * @return {@literal Map} describing the XAttrs of the file * or directory - * @throws IOException + * @throws IOException raised on errors performing I/O. */ public List listXAttrs(Path path) throws IOException { @@ -1249,7 +1461,7 @@ public List listXAttrs(Path path) * * @param path Path to remove extended attribute * @param name xattr name - * @throws IOException + * @throws IOException raised on errors performing I/O. */ public void removeXAttr(Path path, String name) throws IOException { throw new UnsupportedOperationException(getClass().getSimpleName() @@ -1259,6 +1471,11 @@ public void removeXAttr(Path path, String name) throws IOException { /** * The specification of this method matches that of * {@link FileContext#createSnapshot(Path, String)}. + * + * @param path the path. + * @param snapshotName snapshot name. + * @throws IOException raised on errors performing I/O. + * @return path. */ public Path createSnapshot(final Path path, final String snapshotName) throws IOException { @@ -1269,6 +1486,11 @@ public Path createSnapshot(final Path path, final String snapshotName) /** * The specification of this method matches that of * {@link FileContext#renameSnapshot(Path, String, String)}. + * + * @param path the path. + * @param snapshotOldName snapshot old name. + * @param snapshotNewName snapshot new name. + * @throws IOException raised on errors performing I/O. */ public void renameSnapshot(final Path path, final String snapshotOldName, final String snapshotNewName) throws IOException { @@ -1279,6 +1501,10 @@ public void renameSnapshot(final Path path, final String snapshotOldName, /** * The specification of this method matches that of * {@link FileContext#deleteSnapshot(Path, String)}. + * + * @param snapshotDir snapshot dir. + * @param snapshotName snapshot name. + * @throws IOException raised on errors performing I/O. */ public void deleteSnapshot(final Path snapshotDir, final String snapshotName) throws IOException { @@ -1289,7 +1515,7 @@ public void deleteSnapshot(final Path snapshotDir, final String snapshotName) /** * Set the source path to satisfy storage policy. * @param path The source path referring to either a directory or a file. - * @throws IOException + * @throws IOException raised on errors performing I/O. */ public void satisfyStoragePolicy(final Path path) throws IOException { throw new UnsupportedOperationException( @@ -1303,6 +1529,7 @@ public void satisfyStoragePolicy(final Path path) throws IOException { * @param policyName the name of the target storage policy. The list * of supported Storage policies can be retrieved * via {@link #getAllStoragePolicies}. + * @throws IOException raised on errors performing I/O. */ public void setStoragePolicy(final Path path, final String policyName) throws IOException { @@ -1314,7 +1541,7 @@ public void setStoragePolicy(final Path path, final String policyName) /** * Unset the storage policy set for a given file or directory. * @param src file or directory path. - * @throws IOException + * @throws IOException raised on errors performing I/O. */ public void unsetStoragePolicy(final Path src) throws IOException { throw new UnsupportedOperationException(getClass().getSimpleName() @@ -1326,7 +1553,7 @@ public void unsetStoragePolicy(final Path src) throws IOException { * * @param src file or directory path. * @return storage policy for give file. - * @throws IOException + * @throws IOException raised on errors performing I/O. */ public BlockStoragePolicySpi getStoragePolicy(final Path src) throws IOException { @@ -1338,7 +1565,7 @@ public BlockStoragePolicySpi getStoragePolicy(final Path src) * Retrieve all the storage policies supported by this file system. * * @return all storage policies supported by this filesystem. - * @throws IOException + * @throws IOException raised on errors performing I/O. */ public Collection getAllStoragePolicies() throws IOException { diff --git a/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/fs/AvroFSInput.java b/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/fs/AvroFSInput.java index 213fbc24c4db0..7518dd2f7ef74 100644 --- a/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/fs/AvroFSInput.java +++ b/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/fs/AvroFSInput.java @@ -36,13 +36,22 @@ public class AvroFSInput implements Closeable, SeekableInput { private final FSDataInputStream stream; private final long len; - /** Construct given an {@link FSDataInputStream} and its length. */ + /** + * Construct given an {@link FSDataInputStream} and its length. + * + * @param in inputstream. + * @param len len. + */ public AvroFSInput(final FSDataInputStream in, final long len) { this.stream = in; this.len = len; } - /** Construct given a {@link FileContext} and a {@link Path}. */ + /** Construct given a {@link FileContext} and a {@link Path}. + * @param fc filecontext. + * @param p the path. + * @throws IOException If an I/O error occurred. + * */ public AvroFSInput(final FileContext fc, final Path p) throws IOException { FileStatus status = fc.getFileStatus(p); this.len = status.getLen(); diff --git a/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/fs/BatchedRemoteIterator.java b/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/fs/BatchedRemoteIterator.java index 607fffbcc701a..e693bcbfe89fc 100644 --- a/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/fs/BatchedRemoteIterator.java +++ b/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/fs/BatchedRemoteIterator.java @@ -68,6 +68,7 @@ public BatchedRemoteIterator(K prevKey) { * * @param prevKey The key to send. * @return A list of replies. + * @throws IOException If an I/O error occurred. */ public abstract BatchedEntries makeRequest(K prevKey) throws IOException; @@ -102,6 +103,8 @@ public boolean hasNext() throws IOException { /** * Return the next list key associated with an element. + * @param element element. + * @return K Generics Type. */ public abstract K elementToPrevKey(E element); diff --git a/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/fs/BlockLocation.java b/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/fs/BlockLocation.java index 29358dd7d1086..67687c1f0e04c 100644 --- a/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/fs/BlockLocation.java +++ b/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/fs/BlockLocation.java @@ -85,6 +85,7 @@ public BlockLocation() { /** * Copy constructor. + * @param that blocklocation. */ public BlockLocation(BlockLocation that) { this.hosts = that.hosts; @@ -100,6 +101,10 @@ public BlockLocation(BlockLocation that) { /** * Constructor with host, name, offset and length. + * @param names names array. + * @param hosts host array. + * @param offset offset. + * @param length length. */ public BlockLocation(String[] names, String[] hosts, long offset, long length) { @@ -108,6 +113,11 @@ public BlockLocation(String[] names, String[] hosts, long offset, /** * Constructor with host, name, offset, length and corrupt flag. + * @param names names. + * @param hosts hosts. + * @param offset offset. + * @param length length. + * @param corrupt corrupt. */ public BlockLocation(String[] names, String[] hosts, long offset, long length, boolean corrupt) { @@ -116,6 +126,11 @@ public BlockLocation(String[] names, String[] hosts, long offset, /** * Constructor with host, name, network topology, offset and length. + * @param names names. + * @param hosts hosts. + * @param topologyPaths topologyPaths. + * @param offset offset. + * @param length length. */ public BlockLocation(String[] names, String[] hosts, String[] topologyPaths, long offset, long length) { @@ -125,6 +140,12 @@ public BlockLocation(String[] names, String[] hosts, String[] topologyPaths, /** * Constructor with host, name, network topology, offset, length * and corrupt flag. + * @param names names. + * @param hosts hosts. + * @param topologyPaths topologyPaths. + * @param offset offset. + * @param length length. + * @param corrupt corrupt. */ public BlockLocation(String[] names, String[] hosts, String[] topologyPaths, long offset, long length, boolean corrupt) { @@ -177,6 +198,8 @@ public BlockLocation(String[] names, String[] hosts, String[] cachedHosts, /** * Get the list of hosts (hostname) hosting this block. + * @return hosts array. + * @throws IOException If an I/O error occurred. */ public String[] getHosts() throws IOException { return hosts; @@ -184,6 +207,7 @@ public String[] getHosts() throws IOException { /** * Get the list of hosts (hostname) hosting a cached replica of the block. + * @return cached hosts. */ public String[] getCachedHosts() { return cachedHosts; @@ -191,6 +215,8 @@ public String[] getCachedHosts() { /** * Get the list of names (IP:xferPort) hosting this block. + * @return names array. + * @throws IOException If an I/O error occurred. */ public String[] getNames() throws IOException { return names; @@ -199,6 +225,8 @@ public String[] getNames() throws IOException { /** * Get the list of network topology paths for each of the hosts. * The last component of the path is the "name" (IP:xferPort). + * @return topology paths. + * @throws IOException If an I/O error occurred. */ public String[] getTopologyPaths() throws IOException { return topologyPaths; @@ -206,6 +234,7 @@ public String[] getTopologyPaths() throws IOException { /** * Get the storageID of each replica of the block. + * @return storage ids. */ public String[] getStorageIds() { return storageIds; @@ -213,6 +242,7 @@ public String[] getStorageIds() { /** * Get the storage type of each replica of the block. + * @return storage type of each replica of the block. */ public StorageType[] getStorageTypes() { return storageTypes; @@ -220,6 +250,7 @@ public StorageType[] getStorageTypes() { /** * Get the start offset of file associated with this block. + * @return start offset of file associated with this block. */ public long getOffset() { return offset; @@ -227,6 +258,7 @@ public long getOffset() { /** * Get the length of the block. + * @return length of the block. */ public long getLength() { return length; @@ -234,6 +266,7 @@ public long getLength() { /** * Get the corrupt flag. + * @return corrupt flag. */ public boolean isCorrupt() { return corrupt; @@ -241,6 +274,7 @@ public boolean isCorrupt() { /** * Return true if the block is striped (erasure coded). + * @return if the block is striped true, not false. */ public boolean isStriped() { return false; @@ -248,6 +282,7 @@ public boolean isStriped() { /** * Set the start offset of file associated with this block. + * @param offset start offset. */ public void setOffset(long offset) { this.offset = offset; @@ -255,6 +290,7 @@ public void setOffset(long offset) { /** * Set the length of block. + * @param length length of block. */ public void setLength(long length) { this.length = length; @@ -262,6 +298,7 @@ public void setLength(long length) { /** * Set the corrupt flag. + * @param corrupt corrupt flag. */ public void setCorrupt(boolean corrupt) { this.corrupt = corrupt; @@ -269,6 +306,8 @@ public void setCorrupt(boolean corrupt) { /** * Set the hosts hosting this block. + * @param hosts hosts array. + * @throws IOException If an I/O error occurred. */ public void setHosts(String[] hosts) throws IOException { if (hosts == null) { @@ -280,6 +319,7 @@ public void setHosts(String[] hosts) throws IOException { /** * Set the hosts hosting a cached replica of this block. + * @param cachedHosts cached hosts. */ public void setCachedHosts(String[] cachedHosts) { if (cachedHosts == null) { @@ -291,6 +331,8 @@ public void setCachedHosts(String[] cachedHosts) { /** * Set the names (host:port) hosting this block. + * @param names names. + * @throws IOException If an I/O error occurred. */ public void setNames(String[] names) throws IOException { if (names == null) { @@ -302,6 +344,9 @@ public void setNames(String[] names) throws IOException { /** * Set the network topology paths of the hosts. + * + * @param topologyPaths topology paths. + * @throws IOException If an I/O error occurred. */ public void setTopologyPaths(String[] topologyPaths) throws IOException { if (topologyPaths == null) { diff --git a/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/fs/BufferedFSInputStream.java b/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/fs/BufferedFSInputStream.java index 59345f5d25caf..7f3171235c8f4 100644 --- a/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/fs/BufferedFSInputStream.java +++ b/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/fs/BufferedFSInputStream.java @@ -1,4 +1,4 @@ -/** +/* * 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 @@ -22,6 +22,9 @@ import java.io.FileDescriptor; import java.io.IOException; import java.util.StringJoiner; +import java.nio.ByteBuffer; +import java.util.List; +import java.util.function.IntFunction; import org.apache.hadoop.classification.InterfaceAudience; import org.apache.hadoop.classification.InterfaceStability; @@ -158,8 +161,24 @@ public IOStatistics getIOStatistics() { @Override public String toString() { return new StringJoiner(", ", - BufferedFSInputStream.class.getSimpleName() + "[", "]") - .add("in=" + in) - .toString(); + BufferedFSInputStream.class.getSimpleName() + "[", "]") + .add("in=" + in) + .toString(); + } + + @Override + public int minSeekForVectorReads() { + return ((PositionedReadable) in).minSeekForVectorReads(); + } + + @Override + public int maxReadSizeForVectorReads() { + return ((PositionedReadable) in).maxReadSizeForVectorReads(); + } + + @Override + public void readVectored(List ranges, + IntFunction allocate) throws IOException { + ((PositionedReadable) in).readVectored(ranges, allocate); } } diff --git a/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/fs/ByteBufferUtil.java b/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/fs/ByteBufferUtil.java index 6576fe5827d94..f577649dd5fce 100644 --- a/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/fs/ByteBufferUtil.java +++ b/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/fs/ByteBufferUtil.java @@ -47,6 +47,12 @@ private static boolean streamHasByteBufferRead(InputStream stream) { /** * Perform a fallback read. + * + * @param stream input stream. + * @param bufferPool bufferPool. + * @param maxLength maxLength. + * @throws IOException raised on errors performing I/O. + * @return byte buffer. */ public static ByteBuffer fallbackRead( InputStream stream, ByteBufferPool bufferPool, int maxLength) diff --git a/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/fs/CachingGetSpaceUsed.java b/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/fs/CachingGetSpaceUsed.java index 362d125b09df5..d7b61346d4e3b 100644 --- a/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/fs/CachingGetSpaceUsed.java +++ b/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/fs/CachingGetSpaceUsed.java @@ -53,6 +53,9 @@ public abstract class CachingGetSpaceUsed implements Closeable, GetSpaceUsed { /** * This is the constructor used by the builder. * All overriding classes should implement this. + * + * @param builder builder. + * @throws IOException raised on errors performing I/O. */ public CachingGetSpaceUsed(CachingGetSpaceUsed.Builder builder) throws IOException { @@ -140,6 +143,8 @@ public String getDirPath() { /** * Increment the cached value of used space. + * + * @param value dfs used value. */ public void incDfsUsed(long value) { used.addAndGet(value); @@ -154,6 +159,8 @@ boolean running() { /** * How long in between runs of the background refresh. + * + * @return refresh interval. */ @VisibleForTesting public long getRefreshInterval() { @@ -163,6 +170,8 @@ public long getRefreshInterval() { /** * Randomize the refresh interval timing by this amount, the actual interval will be chosen * uniformly between {@code interval-jitter} and {@code interval+jitter}. + * + * @return between interval-jitter and interval+jitter. */ @VisibleForTesting public long getJitter() { diff --git a/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/fs/ChecksumFileSystem.java b/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/fs/ChecksumFileSystem.java index 59ffe00bcb24d..1cca9fe2bfdb1 100644 --- a/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/fs/ChecksumFileSystem.java +++ b/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/fs/ChecksumFileSystem.java @@ -22,17 +22,24 @@ import java.io.FileNotFoundException; import java.io.IOException; import java.io.InputStream; +import java.nio.ByteBuffer; +import java.nio.IntBuffer; import java.nio.channels.ClosedChannelException; +import java.util.ArrayList; import java.util.Arrays; import java.util.EnumSet; import java.util.List; import java.util.concurrent.CompletableFuture; +import java.util.concurrent.CompletionException; +import java.util.function.IntFunction; +import java.util.zip.CRC32; import org.apache.hadoop.util.Preconditions; import org.apache.hadoop.classification.InterfaceAudience; import org.apache.hadoop.classification.InterfaceStability; import org.apache.hadoop.conf.Configuration; import org.apache.hadoop.fs.impl.AbstractFSBuilderImpl; +import org.apache.hadoop.fs.impl.CombinedFileRange; import org.apache.hadoop.fs.impl.FutureDataInputStreamBuilderImpl; import org.apache.hadoop.fs.impl.OpenFileParameters; import org.apache.hadoop.fs.permission.AclEntry; @@ -47,6 +54,7 @@ import static org.apache.hadoop.fs.Options.OpenFileOptions.FS_OPTION_OPENFILE_STANDARD_OPTIONS; import static org.apache.hadoop.fs.impl.PathCapabilitiesSupport.validatePathCapabilityArgs; import static org.apache.hadoop.fs.impl.StoreImplementationUtils.isProbeForSyncable; +import static org.apache.hadoop.fs.VectoredReadUtils.sortRanges; /**************************************************************** * Abstract Checksumed FileSystem. @@ -66,7 +74,7 @@ public abstract class ChecksumFileSystem extends FilterFileSystem { public static double getApproxChkSumLength(long size) { return ChecksumFSOutputSummer.CHKSUM_AS_FRACTION * size; } - + public ChecksumFileSystem(FileSystem fs) { super(fs); } @@ -82,7 +90,7 @@ public void setConf(Configuration conf) { bytesPerChecksum); } } - + /** * Set whether to verify checksum. */ @@ -95,32 +103,51 @@ public void setVerifyChecksum(boolean verifyChecksum) { public void setWriteChecksum(boolean writeChecksum) { this.writeChecksum = writeChecksum; } - + /** get the raw file system */ @Override public FileSystem getRawFileSystem() { return fs; } - /** Return the name of the checksum file associated with a file.*/ + /** + * Return the name of the checksum file associated with a file. + * + * @param file the file path. + * @return name of the checksum file associated with a file. + */ public Path getChecksumFile(Path file) { return new Path(file.getParent(), "." + file.getName() + ".crc"); } - /** Return true iff file is a checksum file name.*/ + /** + * Return true if file is a checksum file name. + * + * @param file the file path. + * @return if file is a checksum file true, not false. + */ public static boolean isChecksumFile(Path file) { String name = file.getName(); return name.startsWith(".") && name.endsWith(".crc"); } - /** Return the length of the checksum file given the size of the + /** + * Return the length of the checksum file given the size of the * actual file. - **/ + * + * @param file the file path. + * @param fileSize file size. + * @return checksum length. + */ public long getChecksumFileLength(Path file, long fileSize) { return getChecksumLength(fileSize, getBytesPerSum()); } - /** Return the bytes Per Checksum */ + /** + * Return the bytes Per Checksum. + * + * @return bytes per check sum. + */ public int getBytesPerSum() { return bytesPerChecksum; } @@ -139,22 +166,22 @@ private int getSumBufferSize(int bytesPerSum, int bufferSize) { * It verifies that data matches checksums. *******************************************************/ private static class ChecksumFSInputChecker extends FSInputChecker implements - IOStatisticsSource { + IOStatisticsSource, StreamCapabilities { private ChecksumFileSystem fs; private FSDataInputStream datas; private FSDataInputStream sums; - + private static final int HEADER_LENGTH = 8; - + private int bytesPerSum = 1; - + public ChecksumFSInputChecker(ChecksumFileSystem fs, Path file) throws IOException { this(fs, file, fs.getConf().getInt( - LocalFileSystemConfigKeys.LOCAL_FS_STREAM_BUFFER_SIZE_KEY, + LocalFileSystemConfigKeys.LOCAL_FS_STREAM_BUFFER_SIZE_KEY, LocalFileSystemConfigKeys.LOCAL_FS_STREAM_BUFFER_SIZE_DEFAULT)); } - + public ChecksumFSInputChecker(ChecksumFileSystem fs, Path file, int bufferSize) throws IOException { super( file, fs.getFileStatus(file).getReplication() ); @@ -170,7 +197,8 @@ public ChecksumFSInputChecker(ChecksumFileSystem fs, Path file, int bufferSize) if (!Arrays.equals(version, CHECKSUM_VERSION)) throw new IOException("Not a checksum file: "+sumFile); this.bytesPerSum = sums.readInt(); - set(fs.verifyChecksum, DataChecksum.newCrc32(), bytesPerSum, 4); + set(fs.verifyChecksum, DataChecksum.newCrc32(), bytesPerSum, + FSInputChecker.CHECKSUM_SIZE); } catch (IOException e) { // mincing the message is terrible, but java throws permission // exceptions as FNF because that's all the method signatures allow! @@ -182,21 +210,21 @@ public ChecksumFSInputChecker(ChecksumFileSystem fs, Path file, int bufferSize) set(fs.verifyChecksum, null, 1, 0); } } - + private long getChecksumFilePos( long dataPos ) { - return HEADER_LENGTH + 4*(dataPos/bytesPerSum); + return HEADER_LENGTH + FSInputChecker.CHECKSUM_SIZE*(dataPos/bytesPerSum); } - + @Override protected long getChunkPosition( long dataPos ) { return dataPos/bytesPerSum*bytesPerSum; } - + @Override public int available() throws IOException { return datas.available() + super.available(); } - + @Override public int read(long position, byte[] b, int off, int len) throws IOException { @@ -214,7 +242,7 @@ public int read(long position, byte[] b, int off, int len) } return nread; } - + @Override public void close() throws IOException { datas.close(); @@ -223,7 +251,7 @@ public void close() throws IOException { } set(fs.verifyChecksum, null, 1, 0); } - + @Override public boolean seekToNewSource(long targetPos) throws IOException { @@ -246,7 +274,7 @@ protected int readChunk(long pos, byte[] buf, int offset, int len, final int checksumsToRead = Math.min( len/bytesPerSum, // number of checksums based on len to read checksum.length / CHECKSUM_SIZE); // size of checksum buffer - long checksumPos = getChecksumFilePos(pos); + long checksumPos = getChecksumFilePos(pos); if(checksumPos != sums.getPos()) { sums.seek(checksumPos); } @@ -286,8 +314,134 @@ protected int readChunk(long pos, byte[] buf, int offset, int len, public IOStatistics getIOStatistics() { return IOStatisticsSupport.retrieveIOStatistics(datas); } + + public static long findChecksumOffset(long dataOffset, + int bytesPerSum) { + return HEADER_LENGTH + (dataOffset/bytesPerSum) * FSInputChecker.CHECKSUM_SIZE; + } + + /** + * Find the checksum ranges that correspond to the given data ranges. + * @param dataRanges the input data ranges, which are assumed to be sorted + * and non-overlapping + * @return a list of AsyncReaderUtils.CombinedFileRange that correspond to + * the checksum ranges + */ + public static List findChecksumRanges( + List dataRanges, + int bytesPerSum, + int minSeek, + int maxSize) { + List result = new ArrayList<>(); + CombinedFileRange currentCrc = null; + for(FileRange range: dataRanges) { + long crcOffset = findChecksumOffset(range.getOffset(), bytesPerSum); + long crcEnd = findChecksumOffset(range.getOffset() + range.getLength() + + bytesPerSum - 1, bytesPerSum); + if (currentCrc == null || + !currentCrc.merge(crcOffset, crcEnd, range, minSeek, maxSize)) { + currentCrc = new CombinedFileRange(crcOffset, crcEnd, range); + result.add(currentCrc); + } + } + return result; + } + + /** + * Check the data against the checksums. + * @param sumsBytes the checksum data + * @param sumsOffset where from the checksum file this buffer started + * @param data the file data + * @param dataOffset where the file data started (must be a multiple of + * bytesPerSum) + * @param bytesPerSum how many bytes per a checksum + * @param file the path of the filename + * @return the data buffer + * @throws CompletionException if the checksums don't match + */ + static ByteBuffer checkBytes(ByteBuffer sumsBytes, + long sumsOffset, + ByteBuffer data, + long dataOffset, + int bytesPerSum, + Path file) { + // determine how many bytes we need to skip at the start of the sums + int offset = + (int) (findChecksumOffset(dataOffset, bytesPerSum) - sumsOffset); + IntBuffer sums = sumsBytes.asIntBuffer(); + sums.position(offset / FSInputChecker.CHECKSUM_SIZE); + ByteBuffer current = data.duplicate(); + int numChunks = data.remaining() / bytesPerSum; + CRC32 crc = new CRC32(); + // check each chunk to ensure they match + for(int c = 0; c < numChunks; ++c) { + // set the buffer position and the limit + current.limit((c + 1) * bytesPerSum); + current.position(c * bytesPerSum); + // compute the crc + crc.reset(); + crc.update(current); + int expected = sums.get(); + int calculated = (int) crc.getValue(); + + if (calculated != expected) { + // cast of c added to silence findbugs + long errPosn = dataOffset + (long) c * bytesPerSum; + throw new CompletionException(new ChecksumException( + "Checksum error: " + file + " at " + errPosn + + " exp: " + expected + " got: " + calculated, errPosn)); + } + } + // if everything matches, we return the data + return data; + } + + @Override + public void readVectored(List ranges, + IntFunction allocate) throws IOException { + // If the stream doesn't have checksums, just delegate. + VectoredReadUtils.validateVectoredReadRanges(ranges); + if (sums == null) { + datas.readVectored(ranges, allocate); + return; + } + int minSeek = minSeekForVectorReads(); + int maxSize = maxReadSizeForVectorReads(); + List dataRanges = + VectoredReadUtils.mergeSortedRanges(Arrays.asList(sortRanges(ranges)), bytesPerSum, + minSeek, maxReadSizeForVectorReads()); + List checksumRanges = findChecksumRanges(dataRanges, + bytesPerSum, minSeek, maxSize); + sums.readVectored(checksumRanges, allocate); + datas.readVectored(dataRanges, allocate); + // Data read is correct. I have verified content of dataRanges. + // There is some bug below here as test (testVectoredReadMultipleRanges) + // is failing, should be + // somewhere while slicing the merged data into smaller user ranges. + // Spend some time figuring out but it is a complex code. + for(CombinedFileRange checksumRange: checksumRanges) { + for(FileRange dataRange: checksumRange.getUnderlying()) { + // when we have both the ranges, validate the checksum + CompletableFuture result = + checksumRange.getData().thenCombineAsync(dataRange.getData(), + (sumBuffer, dataBuffer) -> + checkBytes(sumBuffer, checksumRange.getOffset(), + dataBuffer, dataRange.getOffset(), bytesPerSum, file)); + // Now, slice the read data range to the user's ranges + for(FileRange original: ((CombinedFileRange) dataRange).getUnderlying()) { + original.setData(result.thenApply( + (b) -> VectoredReadUtils.sliceTo(b, dataRange.getOffset(), original))); + } + } + } + } + + @Override + public boolean hasCapability(String capability) { + return datas.hasCapability(capability); + } } - + private static class FSDataBoundedInputStream extends FSDataInputStream { private FileSystem fs; private Path file; @@ -298,12 +452,12 @@ private static class FSDataBoundedInputStream extends FSDataInputStream { this.fs = fs; this.file = file; } - + @Override public boolean markSupported() { return false; } - + /* Return the file length */ private long getFileLength() throws IOException { if( fileLen==-1L ) { @@ -311,7 +465,7 @@ private long getFileLength() throws IOException { } return fileLen; } - + /** * Skips over and discards n bytes of data from the * input stream. @@ -335,11 +489,11 @@ public synchronized long skip(long n) throws IOException { } return super.skip(n); } - + /** * Seek to the given position in the stream. * The next read() will be from that position. - * + * *

    This method does not allow seek past the end of the file. * This produces IOException. * @@ -362,6 +516,7 @@ public synchronized void seek(long pos) throws IOException { * Opens an FSDataInputStream at the indicated Path. * @param f the file name to open * @param bufferSize the size of the buffer to be used. + * @throws IOException if an I/O error occurs. */ @Override public FSDataInputStream open(Path f, int bufferSize) throws IOException { @@ -404,22 +559,22 @@ public void concat(final Path f, final Path[] psrcs) throws IOException { */ public static long getChecksumLength(long size, int bytesPerSum) { //the checksum length is equal to size passed divided by bytesPerSum + - //bytes written in the beginning of the checksum file. - return ((size + bytesPerSum - 1) / bytesPerSum) * 4 + - CHECKSUM_VERSION.length + 4; + //bytes written in the beginning of the checksum file. + return ((size + bytesPerSum - 1) / bytesPerSum) * FSInputChecker.CHECKSUM_SIZE + + ChecksumFSInputChecker.HEADER_LENGTH; } /** This class provides an output stream for a checksummed file. * It generates checksums for data. */ private static class ChecksumFSOutputSummer extends FSOutputSummer implements IOStatisticsSource, StreamCapabilities { - private FSDataOutputStream datas; + private FSDataOutputStream datas; private FSDataOutputStream sums; private static final float CHKSUM_AS_FRACTION = 0.01f; private boolean isClosed = false; - - public ChecksumFSOutputSummer(ChecksumFileSystem fs, - Path file, + + ChecksumFSOutputSummer(ChecksumFileSystem fs, + Path file, boolean overwrite, int bufferSize, short replication, @@ -440,7 +595,7 @@ public ChecksumFSOutputSummer(ChecksumFileSystem fs, sums.write(CHECKSUM_VERSION, 0, CHECKSUM_VERSION.length); sums.writeInt(bytesPerSum); } - + @Override public void close() throws IOException { try { @@ -451,7 +606,7 @@ public void close() throws IOException { isClosed = true; } } - + @Override protected void writeChunk(byte[] b, int offset, int len, byte[] checksum, int ckoff, int cklen) @@ -669,7 +824,7 @@ boolean apply(Path p) throws IOException { * Implement the abstract setReplication of FileSystem * @param src file name * @param replication new replication - * @throws IOException + * @throws IOException if an I/O error occurs. * @return true if successful; * false if file does not exist or is a directory */ @@ -707,7 +862,7 @@ public boolean rename(Path src, Path dst) throws IOException { value = fs.rename(srcCheckFile, dstCheckFile); } else if (fs.exists(dstCheckFile)) { // no src checksum, so remove dst checksum - value = fs.delete(dstCheckFile, true); + value = fs.delete(dstCheckFile, true); } return value; @@ -739,7 +894,7 @@ public boolean delete(Path f, boolean recursive) throws IOException{ return fs.delete(f, true); } } - + final private static PathFilter DEFAULT_FILTER = new PathFilter() { @Override public boolean accept(Path file) { @@ -750,11 +905,11 @@ public boolean accept(Path file) { /** * List the statuses of the files/directories in the given path if the path is * a directory. - * + * * @param f * given path * @return the statuses of the files/directories in the given path - * @throws IOException + * @throws IOException if an I/O error occurs. */ @Override public FileStatus[] listStatus(Path f) throws IOException { @@ -771,18 +926,18 @@ public RemoteIterator listStatusIterator(final Path p) /** * List the statuses of the files/directories in the given path if the path is * a directory. - * + * * @param f * given path * @return the statuses of the files/directories in the given patch - * @throws IOException + * @throws IOException if an I/O error occurs. */ @Override public RemoteIterator listLocatedStatus(Path f) throws IOException { return fs.listLocatedStatus(f, DEFAULT_FILTER); } - + @Override public boolean mkdirs(Path f) throws IOException { return fs.mkdirs(f); @@ -811,6 +966,10 @@ public void copyToLocalFile(boolean delSrc, Path src, Path dst) * Copy it from FS control to the local dst name. * If src and dst are directories, the copyCrc parameter * determines whether to copy CRC files. + * @param src src path. + * @param dst dst path. + * @param copyCrc copy csc flag. + * @throws IOException if an I/O error occurs. */ @SuppressWarnings("deprecation") public void copyToLocalFile(Path src, Path dst, boolean copyCrc) @@ -832,7 +991,7 @@ public void copyToLocalFile(Path src, Path dst, boolean copyCrc) } else { FileStatus[] srcs = listStatus(src); for (FileStatus srcFile : srcs) { - copyToLocalFile(srcFile.getPath(), + copyToLocalFile(srcFile.getPath(), new Path(dst, srcFile.getPath().getName()), copyCrc); } } diff --git a/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/fs/ChecksumFs.java b/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/fs/ChecksumFs.java index bc1122c56a2bd..4820c5c3045d7 100644 --- a/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/fs/ChecksumFs.java +++ b/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/fs/ChecksumFs.java @@ -70,30 +70,53 @@ public void setVerifyChecksum(boolean inVerifyChecksum) { this.verifyChecksum = inVerifyChecksum; } - /** get the raw file system. */ + /** + * get the raw file system. + * + * @return abstract file system. + */ public AbstractFileSystem getRawFs() { return getMyFs(); } - /** Return the name of the checksum file associated with a file.*/ + /** + * Return the name of the checksum file associated with a file. + * + * @param file the file path. + * @return the checksum file associated with a file. + */ public Path getChecksumFile(Path file) { return new Path(file.getParent(), "." + file.getName() + ".crc"); } - /** Return true iff file is a checksum file name.*/ + /** + * Return true iff file is a checksum file name. + * + * @param file the file path. + * @return if is checksum file true,not false. + */ public static boolean isChecksumFile(Path file) { String name = file.getName(); return name.startsWith(".") && name.endsWith(".crc"); } - /** Return the length of the checksum file given the size of the + /** + * Return the length of the checksum file given the size of the * actual file. - **/ + * + * @param file the file path. + * @param fileSize file size. + * @return check sum file length. + */ public long getChecksumFileLength(Path file, long fileSize) { return getChecksumLength(fileSize, getBytesPerSum()); } - /** Return the bytes Per Checksum. */ + /** + * Return the bytes Per Checksum. + * + * @return bytes per sum. + */ public int getBytesPerSum() { return defaultBytesPerChecksum; } @@ -433,7 +456,7 @@ private boolean isDirectory(Path f) * Implement the abstract setReplication of FileSystem * @param src file name * @param replication new replication - * @throws IOException + * @throws IOException if an I/O error occurs. * @return true if successful; * false if file does not exist or is a directory */ diff --git a/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/fs/CommonConfigurationKeysPublic.java b/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/fs/CommonConfigurationKeysPublic.java index a799e883bcf2a..7a458e8f3fccd 100644 --- a/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/fs/CommonConfigurationKeysPublic.java +++ b/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/fs/CommonConfigurationKeysPublic.java @@ -169,11 +169,11 @@ public class CommonConfigurationKeysPublic { /** * Number of filesystems instances can be created in parallel. - *

    + *

    * A higher number here does not necessarily improve performance, especially * for object stores, where multiple threads may be attempting to create an FS * instance for the same URI. - *

    + *

    * Default value: {@value}. */ public static final String FS_CREATION_PARALLEL_COUNT = @@ -181,8 +181,9 @@ public class CommonConfigurationKeysPublic { /** * Default value for {@link #FS_CREATION_PARALLEL_COUNT}. - *

    + *

    * Default value: {@value}. + *

    */ public static final int FS_CREATION_PARALLEL_COUNT_DEFAULT = 64; @@ -1053,5 +1054,13 @@ public class CommonConfigurationKeysPublic { public static final String HADOOP_HTTP_IDLE_TIMEOUT_MS_KEY = "hadoop.http.idle_timeout.ms"; public static final int HADOOP_HTTP_IDLE_TIMEOUT_MS_DEFAULT = 60000; + + /** + * To configure scheduling of server metrics update thread. This config is used to indicate + * initial delay and delay between each execution of the metric update runnable thread. + */ + public static final String IPC_SERVER_METRICS_UPDATE_RUNNER_INTERVAL = + "ipc.server.metrics.update.runner.interval"; + public static final int IPC_SERVER_METRICS_UPDATE_RUNNER_INTERVAL_DEFAULT = 5000; } diff --git a/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/fs/CompositeCrcFileChecksum.java b/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/fs/CompositeCrcFileChecksum.java index e1ed5cbcfcaa6..bdbc8f3a33f4b 100644 --- a/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/fs/CompositeCrcFileChecksum.java +++ b/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/fs/CompositeCrcFileChecksum.java @@ -37,7 +37,13 @@ public class CompositeCrcFileChecksum extends FileChecksum { private DataChecksum.Type crcType; private int bytesPerCrc; - /** Create a CompositeCrcFileChecksum. */ + /** + * Create a CompositeCrcFileChecksum. + * + * @param crc crc. + * @param crcType crcType. + * @param bytesPerCrc bytesPerCrc. + */ public CompositeCrcFileChecksum( int crc, DataChecksum.Type crcType, int bytesPerCrc) { this.crc = crc; diff --git a/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/fs/ContentSummary.java b/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/fs/ContentSummary.java index 79850e1a2f291..9f97a12fa6088 100644 --- a/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/fs/ContentSummary.java +++ b/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/fs/ContentSummary.java @@ -149,17 +149,31 @@ public ContentSummary build() { @Deprecated public ContentSummary() {} - /** Constructor, deprecated by ContentSummary.Builder + /** + * Constructor, deprecated by ContentSummary.Builder * This constructor implicitly set spaceConsumed the same as length. * spaceConsumed and length must be set explicitly with - * ContentSummary.Builder + * ContentSummary.Builder. + * + * @param length length. + * @param fileCount file count. + * @param directoryCount directory count. * */ @Deprecated public ContentSummary(long length, long fileCount, long directoryCount) { this(length, fileCount, directoryCount, -1L, length, -1L); } - /** Constructor, deprecated by ContentSummary.Builder */ + /** + * Constructor, deprecated by ContentSummary.Builder. + * + * @param length length. + * @param fileCount file count. + * @param directoryCount directory count. + * @param quota quota. + * @param spaceConsumed space consumed. + * @param spaceQuota space quota. + * */ @Deprecated public ContentSummary( long length, long fileCount, long directoryCount, long quota, @@ -172,7 +186,11 @@ public ContentSummary( setSpaceQuota(spaceQuota); } - /** Constructor for ContentSummary.Builder*/ + /** + * Constructor for ContentSummary.Builder. + * + * @param builder builder. + */ private ContentSummary(Builder builder) { super(builder); this.length = builder.length; diff --git a/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/fs/CreateFlag.java b/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/fs/CreateFlag.java index 71993713ad2eb..ca008e536931d 100644 --- a/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/fs/CreateFlag.java +++ b/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/fs/CreateFlag.java @@ -189,6 +189,8 @@ public static void validate(Object path, boolean pathExists, /** * Validate the CreateFlag for the append operation. The flag must contain * APPEND, and cannot contain OVERWRITE. + * + * @param flag enum set flag. */ public static void validateForAppend(EnumSet flag) { validate(flag); diff --git a/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/fs/DF.java b/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/fs/DF.java index da4636b2c0fbe..c5a052f3de4be 100644 --- a/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/fs/DF.java +++ b/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/fs/DF.java @@ -65,7 +65,10 @@ public String getDirPath() { return dirPath; } - /** @return a string indicating which filesystem volume we're checking. */ + /** + * @return a string indicating which filesystem volume we're checking. + * @throws IOException raised on errors performing I/O. + */ public String getFilesystem() throws IOException { if (Shell.WINDOWS) { this.filesystem = dirFile.getCanonicalPath().substring(0, 2); @@ -100,7 +103,10 @@ public int getPercentUsed() { return (int) (used * 100.0 / cap); } - /** @return the filesystem mount point for the indicated volume */ + /** + * @return the filesystem mount point for the indicated volume. + * @throws IOException raised on errors performing I/O. + */ public String getMount() throws IOException { // Abort early if specified path does not exist if (!dirFile.exists()) { diff --git a/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/fs/DelegationTokenRenewer.java b/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/fs/DelegationTokenRenewer.java index 33905dcbb77fd..6f6e30410659c 100644 --- a/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/fs/DelegationTokenRenewer.java +++ b/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/fs/DelegationTokenRenewer.java @@ -47,7 +47,11 @@ public interface Renewable { /** @return the renew token. */ public Token getRenewToken(); - /** Set delegation token. */ + /** + * Set delegation token. + * @param generic type T. + * @param token token. + */ public void setDelegationToken(Token token); } @@ -172,7 +176,11 @@ public String toString() { /** Queue to maintain the RenewActions to be processed by the {@link #run()} */ private volatile DelayQueue> queue = new DelayQueue>(); - /** For testing purposes */ + /** + * For testing purposes. + * + * @return renew queue length. + */ @VisibleForTesting protected int getRenewQueueLength() { return queue.size(); @@ -211,7 +219,13 @@ static synchronized void reset() { } } - /** Add a renew action to the queue. */ + /** + * Add a renew action to the queue. + * + * @param generic type T. + * @param fs file system. + * @return renew action. + * */ @SuppressWarnings("static-access") public RenewAction addRenewAction(final T fs) { synchronized (this) { @@ -230,8 +244,10 @@ public RenewAction addRenewAction(final T /** * Remove the associated renew action from the queue - * - * @throws IOException + * + * @param generic type T. + * @param fs file system. + * @throws IOException raised on errors performing I/O. */ public void removeRenewAction( final T fs) throws IOException { diff --git a/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/fs/FSBuilder.java b/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/fs/FSBuilder.java index a4c7254cfeb3c..56ef51f128db8 100644 --- a/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/fs/FSBuilder.java +++ b/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/fs/FSBuilder.java @@ -37,12 +37,17 @@ public interface FSBuilder> { /** * Set optional Builder parameter. + * @param key key. + * @param value value. + * @return generic type B. */ B opt(@Nonnull String key, @Nonnull String value); /** * Set optional boolean parameter for the Builder. - * + * @param key key. + * @param value value. + * @return generic type B. * @see #opt(String, String) */ B opt(@Nonnull String key, boolean value); @@ -50,6 +55,9 @@ public interface FSBuilder> { /** * Set optional int parameter for the Builder. * + * @param key key. + * @param value value. + * @return generic type B. * @see #opt(String, String) */ B opt(@Nonnull String key, int value); @@ -57,6 +65,9 @@ public interface FSBuilder> { /** * Set optional float parameter for the Builder. * + * @param key key. + * @param value value. + * @return generic type B. * @see #opt(String, String) */ B opt(@Nonnull String key, float value); @@ -64,6 +75,9 @@ public interface FSBuilder> { /** * Set optional long parameter for the Builder. * + * @param key key. + * @param value value. + * @return generic type B. * @see #opt(String, String) */ B opt(@Nonnull String key, long value); @@ -71,6 +85,9 @@ public interface FSBuilder> { /** * Set optional double parameter for the Builder. * + * @param key key. + * @param value value. + * @return generic type B. * @see #opt(String, String) */ B opt(@Nonnull String key, double value); @@ -78,6 +95,9 @@ public interface FSBuilder> { /** * Set an array of string values as optional parameter for the Builder. * + * @param key key. + * @param values values. + * @return generic type B. * @see #opt(String, String) */ B opt(@Nonnull String key, @Nonnull String... values); @@ -87,12 +107,19 @@ public interface FSBuilder> { * * If the option is not supported or unavailable, * the client should expect {@link #build()} throws IllegalArgumentException. + * + * @param key key. + * @param value value. + * @return generic type B. */ B must(@Nonnull String key, @Nonnull String value); /** * Set mandatory boolean option. * + * @param key key. + * @param value value. + * @return generic type B. * @see #must(String, String) */ B must(@Nonnull String key, boolean value); @@ -100,6 +127,9 @@ public interface FSBuilder> { /** * Set mandatory int option. * + * @param key key. + * @param value value. + * @return generic type B. * @see #must(String, String) */ B must(@Nonnull String key, int value); @@ -107,6 +137,9 @@ public interface FSBuilder> { /** * Set mandatory float option. * + * @param key key. + * @param value value. + * @return generic type B. * @see #must(String, String) */ B must(@Nonnull String key, float value); @@ -114,6 +147,9 @@ public interface FSBuilder> { /** * Set mandatory long option. * + * @param key key. + * @param value value. + * @return generic type B. * @see #must(String, String) */ B must(@Nonnull String key, long value); @@ -121,6 +157,9 @@ public interface FSBuilder> { /** * Set mandatory double option. * + * @param key key. + * @param value value. + * @return generic type B. * @see #must(String, String) */ B must(@Nonnull String key, double value); @@ -128,6 +167,9 @@ public interface FSBuilder> { /** * Set a string array as mandatory option. * + * @param key key. + * @param values values. + * @return generic type B. * @see #must(String, String) */ B must(@Nonnull String key, @Nonnull String... values); @@ -139,6 +181,7 @@ public interface FSBuilder> { * @throws UnsupportedOperationException if the filesystem does not support * the specific operation. * @throws IOException on filesystem IO errors. + * @return generic type S. */ S build() throws IllegalArgumentException, UnsupportedOperationException, IOException; diff --git a/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/fs/FSDataInputStream.java b/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/fs/FSDataInputStream.java index b143a4cb63d19..52644402ca459 100644 --- a/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/fs/FSDataInputStream.java +++ b/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/fs/FSDataInputStream.java @@ -1,4 +1,4 @@ -/** +/* * * Licensed to the Apache Software Foundation (ASF) under one * or more contributor license agreements. See the NOTICE file @@ -26,6 +26,8 @@ import java.io.InputStream; import java.nio.ByteBuffer; import java.util.EnumSet; +import java.util.List; +import java.util.function.IntFunction; import org.apache.hadoop.classification.InterfaceAudience; import org.apache.hadoop.classification.InterfaceStability; @@ -51,7 +53,7 @@ public class FSDataInputStream extends DataInputStream */ private final IdentityHashStore extendedReadBuffers - = new IdentityHashStore(0); + = new IdentityHashStore<>(0); public FSDataInputStream(InputStream in) { super(in); @@ -279,4 +281,20 @@ public void readFully(long position, ByteBuffer buf) throws IOException { public IOStatistics getIOStatistics() { return IOStatisticsSupport.retrieveIOStatistics(in); } + + @Override + public int minSeekForVectorReads() { + return ((PositionedReadable) in).minSeekForVectorReads(); + } + + @Override + public int maxReadSizeForVectorReads() { + return ((PositionedReadable) in).maxReadSizeForVectorReads(); + } + + @Override + public void readVectored(List ranges, + IntFunction allocate) throws IOException { + ((PositionedReadable) in).readVectored(ranges, allocate); + } } diff --git a/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/fs/FSDataOutputStreamBuilder.java b/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/fs/FSDataOutputStreamBuilder.java index c96d499d17ba6..16938a83a69c7 100644 --- a/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/fs/FSDataOutputStreamBuilder.java +++ b/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/fs/FSDataOutputStreamBuilder.java @@ -123,6 +123,9 @@ public abstract class FSDataOutputStreamBuilder /** * Constructor. + * + * @param fileSystem file system. + * @param p the path. */ protected FSDataOutputStreamBuilder(@Nonnull FileSystem fileSystem, @Nonnull Path p) { @@ -149,6 +152,9 @@ protected FsPermission getPermission() { /** * Set permission for the file. + * + * @param perm permission. + * @return B Generics Type. */ public B permission(@Nonnull final FsPermission perm) { checkNotNull(perm); @@ -162,6 +168,9 @@ protected int getBufferSize() { /** * Set the size of the buffer to be used. + * + * @param bufSize buffer size. + * @return Generics Type B. */ public B bufferSize(int bufSize) { bufferSize = bufSize; @@ -174,6 +183,9 @@ protected short getReplication() { /** * Set replication factor. + * + * @param replica replica. + * @return Generics Type B. */ public B replication(short replica) { replication = replica; @@ -186,6 +198,9 @@ protected long getBlockSize() { /** * Set block size. + * + * @param blkSize block size. + * @return B Generics Type. */ public B blockSize(long blkSize) { blockSize = blkSize; @@ -194,6 +209,8 @@ public B blockSize(long blkSize) { /** * Return true to create the parent directories if they do not exist. + * + * @return if create the parent directories if they do not exist true,not false. */ protected boolean isRecursive() { return recursive; @@ -201,6 +218,8 @@ protected boolean isRecursive() { /** * Create the parent directory if they do not exist. + * + * @return B Generics Type. */ public B recursive() { recursive = true; @@ -213,6 +232,9 @@ protected Progressable getProgress() { /** * Set the facility of reporting progress. + * + * @param prog progress. + * @return B Generics Type. */ public B progress(@Nonnull final Progressable prog) { checkNotNull(prog); @@ -226,6 +248,8 @@ protected EnumSet getFlags() { /** * Create an FSDataOutputStream at the specified path. + * + * @return return Generics Type B. */ public B create() { flags.add(CreateFlag.CREATE); @@ -236,6 +260,9 @@ public B create() { * Set to true to overwrite the existing file. * Set it to false, an exception will be thrown when calling {@link #build()} * if the file exists. + * + * @param overwrite overrite. + * @return Generics Type B. */ public B overwrite(boolean overwrite) { if (overwrite) { @@ -248,6 +275,8 @@ public B overwrite(boolean overwrite) { /** * Append to an existing file (optional operation). + * + * @return Generics Type B. */ public B append() { flags.add(CreateFlag.APPEND); @@ -260,6 +289,9 @@ protected ChecksumOpt getChecksumOpt() { /** * Set checksum opt. + * + * @param chksumOpt check sum opt. + * @return Generics Type B. */ public B checksumOpt(@Nonnull final ChecksumOpt chksumOpt) { checkNotNull(chksumOpt); diff --git a/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/fs/FSInputChecker.java b/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/fs/FSInputChecker.java index de66eab713ab6..ee16ca8a2cd50 100644 --- a/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/fs/FSInputChecker.java +++ b/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/fs/FSInputChecker.java @@ -82,6 +82,7 @@ protected FSInputChecker( Path file, int numOfRetries) { * @param sum the type of Checksum engine * @param chunkSize maximun chunk size * @param checksumSize the number byte of each checksum + * @param verifyChecksum verify check sum. */ protected FSInputChecker( Path file, int numOfRetries, boolean verifyChecksum, Checksum sum, int chunkSize, int checksumSize ) { @@ -118,6 +119,7 @@ protected FSInputChecker( Path file, int numOfRetries, * @param len maximum number of bytes to read * @param checksum the data buffer into which to write checksums * @return number of bytes read + * @throws IOException raised on errors performing I/O. */ abstract protected int readChunk(long pos, byte[] buf, int offset, int len, byte[] checksum) throws IOException; @@ -129,7 +131,10 @@ abstract protected int readChunk(long pos, byte[] buf, int offset, int len, */ abstract protected long getChunkPosition(long pos); - /** Return true if there is a need for checksum verification */ + /** + * Return true if there is a need for checksum verification. + * @return if there is a need for checksum verification true, not false. + */ protected synchronized boolean needChecksum() { return verifyChecksum && sum != null; } @@ -357,6 +362,9 @@ private void verifySums(final byte b[], final int off, int read) * Convert a checksum byte array to a long * This is deprecated since 0.22 since it is no longer in use * by this class. + * + * @param checksum check sum. + * @return crc. */ @Deprecated static public long checksum2long(byte[] checksum) { diff --git a/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/fs/FSLinkResolver.java b/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/fs/FSLinkResolver.java index ffe4b34ca5fdb..f85cf7a858152 100644 --- a/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/fs/FSLinkResolver.java +++ b/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/fs/FSLinkResolver.java @@ -74,7 +74,7 @@ abstract public T next(final AbstractFileSystem fs, final Path p) * @param fc FileContext used to access file systems. * @param path The path to resolve symlinks on. * @return Generic type determined by the implementation of next. - * @throws IOException + * @throws IOException raised on errors performing I/O. */ public T resolve(final FileContext fc, final Path path) throws IOException { int count = 0; diff --git a/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/fs/FSOutputSummer.java b/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/fs/FSOutputSummer.java index 6de026b9d17c0..4ef512dc257a3 100644 --- a/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/fs/FSOutputSummer.java +++ b/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/fs/FSOutputSummer.java @@ -186,6 +186,8 @@ public void flush() throws IOException { /** * Return the number of valid bytes currently in the buffer. + * + * @return buffer data size. */ protected synchronized int getBufferedDataSize() { return count; @@ -227,6 +229,10 @@ private void writeChecksumChunks(byte b[], int off, int len) /** * Converts a checksum integer value to a byte stream + * + * @param sum check sum. + * @param checksumSize check sum size. + * @return byte stream. */ static public byte[] convertToByteStream(Checksum sum, int checksumSize) { return int2byte((int)sum.getValue(), new byte[checksumSize]); @@ -245,6 +251,8 @@ static byte[] int2byte(int integer, byte[] bytes) { /** * Resets existing buffer with a new one of the specified size. + * + * @param size size. */ protected synchronized void setChecksumBufSize(int size) { this.buf = new byte[size]; diff --git a/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/fs/FileChecksum.java b/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/fs/FileChecksum.java index 6822fa485622f..62d2e3af78671 100644 --- a/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/fs/FileChecksum.java +++ b/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/fs/FileChecksum.java @@ -28,20 +28,37 @@ @InterfaceAudience.Public @InterfaceStability.Stable public abstract class FileChecksum implements Writable { - /** The checksum algorithm name */ + /** + * The checksum algorithm name. + * + * @return algorithm name. + */ public abstract String getAlgorithmName(); - /** The length of the checksum in bytes */ + /** + * The length of the checksum in bytes. + * + * @return length. + */ public abstract int getLength(); - /** The value of the checksum in bytes */ + /** + * The value of the checksum in bytes. + * + * @return byte array. + */ public abstract byte[] getBytes(); public ChecksumOpt getChecksumOpt() { return null; } - /** Return true if both the algorithms and the values are the same. */ + /** + * Return true if both the algorithms and the values are the same. + * + * @param other other. + * @return if equal true, not false. + */ @Override public boolean equals(Object other) { if (other == this) { diff --git a/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/fs/FileContext.java b/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/fs/FileContext.java index f3004ce7e03a3..22ac2ecbd7949 100644 --- a/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/fs/FileContext.java +++ b/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/fs/FileContext.java @@ -366,8 +366,8 @@ public AbstractFileSystem run() throws UnsupportedFileSystemException { * Create a FileContext with specified FS as default using the specified * config. * - * @param defFS - * @param aConf + * @param defFS default fs. + * @param aConf configutration. * @return new FileContext with specified FS as default. */ public static FileContext getFileContext(final AbstractFileSystem defFS, @@ -378,7 +378,7 @@ public static FileContext getFileContext(final AbstractFileSystem defFS, /** * Create a FileContext for specified file system using the default config. * - * @param defaultFS + * @param defaultFS default fs. * @return a FileContext with the specified AbstractFileSystem * as the default FS. */ @@ -411,6 +411,7 @@ protected static FileContext getFileContext( * * @throws UnsupportedFileSystemException If the file system from the default * configuration is not supported + * @return file context. */ public static FileContext getFileContext() throws UnsupportedFileSystemException { @@ -430,7 +431,7 @@ public static FileContext getLocalFSFileContext() /** * Create a FileContext for specified URI using the default config. * - * @param defaultFsUri + * @param defaultFsUri defaultFsUri. * @return a FileContext with the specified URI as the default FS. * * @throws UnsupportedFileSystemException If the file system for @@ -444,8 +445,8 @@ public static FileContext getFileContext(final URI defaultFsUri) /** * Create a FileContext for specified default URI using the specified config. * - * @param defaultFsUri - * @param aConf + * @param defaultFsUri defaultFsUri. + * @param aConf configrution. * @return new FileContext for specified uri * @throws UnsupportedFileSystemException If the file system with specified is * not supported @@ -476,7 +477,7 @@ public static FileContext getFileContext(final URI defaultFsUri, * {@link #getFileContext(URI, Configuration)} instead of this one. * * - * @param aConf + * @param aConf configration. * @return new FileContext * @throws UnsupportedFileSystemException If file system in the config * is not supported @@ -554,6 +555,7 @@ public void setWorkingDirectory(final Path newWDir) throws IOException { /** * Gets the working directory for wd-relative names (such a "foo/bar"). + * @return the path. */ public Path getWorkingDirectory() { return workingDir; @@ -600,13 +602,14 @@ public void setUMask(final FsPermission newUmask) { * @throws FileNotFoundException If f does not exist * @throws AccessControlException if access denied * @throws IOException If an IO Error occurred - * + * @throws UnresolvedLinkException If unresolved link occurred. + * * Exceptions applicable to file systems accessed over RPC: * @throws RpcClientException If an exception occurred in the RPC client * @throws RpcServerException If an exception occurred in the RPC server * @throws UnexpectedServerException If server implementation throws * undeclared exception to RPC server - * + * * RuntimeExceptions: * @throws InvalidPathException If path f is not valid */ @@ -620,7 +623,7 @@ public Path resolvePath(final Path f) throws FileNotFoundException, * A Fully-qualified path has scheme and authority specified and an absolute * path. * Use the default file system and working dir in this FileContext to qualify. - * @param path + * @param path the path. * @return qualified path */ public Path makeQualified(final Path path) { @@ -759,6 +762,7 @@ public FSDataOutputStream build() throws IOException { * * Client should expect {@link FSDataOutputStreamBuilder#build()} throw the * same exceptions as create(Path, EnumSet, CreateOpts...). + * @throws IOException If an I/O error occurred. */ public FSDataOutputStreamBuilder create(final Path f) throws IOException { @@ -832,6 +836,8 @@ public Void next(final AbstractFileSystem fs, final Path p) * * RuntimeExceptions: * @throws InvalidPathException If path f is invalid + * + * @return if delete success true, not false. */ public boolean delete(final Path f, final boolean recursive) throws AccessControlException, FileNotFoundException, @@ -862,6 +868,7 @@ public Boolean next(final AbstractFileSystem fs, final Path p) * @throws RpcServerException If an exception occurred in the RPC server * @throws UnexpectedServerException If server implementation throws * undeclared exception to RPC server + * @return input stream. */ public FSDataInputStream open(final Path f) throws AccessControlException, FileNotFoundException, UnsupportedFileSystemException, IOException { @@ -892,6 +899,7 @@ public FSDataInputStream next(final AbstractFileSystem fs, final Path p) * @throws RpcServerException If an exception occurred in the RPC server * @throws UnexpectedServerException If server implementation throws * undeclared exception to RPC server + * @return output stream. */ public FSDataInputStream open(final Path f, final int bufferSize) throws AccessControlException, FileNotFoundException, @@ -1001,6 +1009,7 @@ public Boolean next(final AbstractFileSystem fs, final Path p) * * @param src path to be renamed * @param dst new path after rename + * @param options rename options. * * @throws AccessControlException If access is denied * @throws FileAlreadyExistsException If dst already exists and @@ -1052,7 +1061,7 @@ public Void next(final AbstractFileSystem fs, final Path p) /** * Set permission of a path. - * @param f + * @param f the path. * @param permission - the new absolute permission (umask is not applied) * * @throws AccessControlException If access is denied @@ -1196,7 +1205,7 @@ public FileChecksum next(final AbstractFileSystem fs, final Path p) * Set the verify checksum flag for the file system denoted by the path. * This is only applicable if the * corresponding FileSystem supports checksum. By default doesn't do anything. - * @param verifyChecksum + * @param verifyChecksum verify check sum. * @param f set the verifyChecksum for the Filesystem containing this path * * @throws AccessControlException If access is denied @@ -1251,8 +1260,9 @@ public FileStatus next(final AbstractFileSystem fs, final Path p) /** * Synchronize client metadata state. * - * @throws IOException - * @throws UnsupportedOperationException + * @throws IOException If an I/O error occurred. + * @throws UnsupportedOperationException If file system for f is + * not supported. */ public void msync() throws IOException, UnsupportedOperationException { defaultFS.msync(); @@ -1613,9 +1623,12 @@ public RemoteIterator next( } /** + * List CorruptFile Blocks. + * + * @param path the path. * @return an iterator over the corrupt files under the given path * (may contain duplicates if a file has more than one corrupt block) - * @throws IOException + * @throws IOException If an I/O error occurred. */ public RemoteIterator listCorruptFileBlocks(Path path) throws IOException { @@ -1739,6 +1752,7 @@ public class Util { * @throws RpcServerException If an exception occurred in the RPC server * @throws UnexpectedServerException If server implementation throws * undeclared exception to RPC server + * @return if f exists true, not false. */ public boolean exists(final Path f) throws AccessControlException, UnsupportedFileSystemException, IOException { @@ -1799,6 +1813,12 @@ public ContentSummary getContentSummary(Path f) /** * See {@link #listStatus(Path[], PathFilter)} + * + * @param files files. + * @throws AccessControlException If access is denied. + * @throws FileNotFoundException If files does not exist. + * @throws IOException If an I/O error occurred. + * @return file status array. */ public FileStatus[] listStatus(Path[] files) throws AccessControlException, FileNotFoundException, IOException { @@ -2054,36 +2074,29 @@ public LocatedFileStatus next() throws IOException { *
    ? *
    Matches any single character. * - *

    *

    * *
    Matches zero or more characters. * - *

    *

    [abc] *
    Matches a single character from character set * {a,b,c}. * - *

    *

    [a-b] *
    Matches a single character from the character range * {a...b}. Note: character a must be * lexicographically less than or equal to character b. * - *

    *

    [^a] *
    Matches a single char that is not from character set or range * {a}. Note that the ^ character must occur * immediately to the right of the opening bracket. * - *

    *

    \c *
    Removes (escapes) any special meaning of character c. * - *

    *

    {ab,cd} *
    Matches a string from the string set {ab, cd} - * - *

    + * *

    {ab,c{de,fh}} *
    Matches a string from string set {ab, cde, cfh} * @@ -2144,6 +2157,18 @@ public FileStatus[] globStatus(final Path pathPattern, /** * Copy file from src to dest. See * {@link #copy(Path, Path, boolean, boolean)} + * + * @param src src. + * @param dst dst. + * @throws AccessControlException If access is denied. + * @throws FileAlreadyExistsException If file src already exists. + * @throws FileNotFoundException if next file does not exist any more. + * @throws ParentNotDirectoryException If parent of src is not a + * directory. + * @throws UnsupportedFileSystemException If file system for + * src/dst is not supported. + * @throws IOException If an I/O error occurred. + * @return if success copy true, not false. */ public boolean copy(final Path src, final Path dst) throws AccessControlException, FileAlreadyExistsException, @@ -2154,8 +2179,8 @@ public boolean copy(final Path src, final Path dst) /** * Copy from src to dst, optionally deleting src and overwriting dst. - * @param src - * @param dst + * @param src src. + * @param dst dst. * @param deleteSource - delete src if true * @param overwrite overwrite dst if true; throw IOException if dst exists * and overwrite is false. @@ -2276,7 +2301,7 @@ private static void checkDependencies(Path qualSrc, Path qualDst) * Are qualSrc and qualDst of the same file system? * @param qualPath1 - fully qualified path * @param qualPath2 - fully qualified path - * @return + * @return is same fs true,not false. */ private static boolean isSameFS(Path qualPath1, Path qualPath2) { URI srcUri = qualPath1.toUri(); @@ -2299,6 +2324,13 @@ public synchronized void run() { /** * Resolves all symbolic links in the specified path. * Returns the new path object. + * + * @param f the path. + * @throws FileNotFoundException If f does not exist. + * @throws UnresolvedLinkException If unresolved link occurred. + * @throws AccessControlException If access is denied. + * @throws IOException If an I/O error occurred. + * @return resolve path. */ protected Path resolve(final Path f) throws FileNotFoundException, UnresolvedLinkException, AccessControlException, IOException { @@ -2316,6 +2348,7 @@ public Path next(final AbstractFileSystem fs, final Path p) * to, but not including the final path component. * @param f path to resolve * @return the new path object. + * @throws IOException If an I/O error occurred. */ protected Path resolveIntermediate(final Path f) throws IOException { return new FSLinkResolver() { @@ -2334,13 +2367,12 @@ public FileStatus next(final AbstractFileSystem fs, final Path p) * @param f * Path which needs to be resolved * @return List of AbstractFileSystems accessed in the path - * @throws IOException + * @throws IOException If an I/O error occurred. */ Set resolveAbstractFileSystems(final Path f) throws IOException { final Path absF = fixRelativePart(f); - final HashSet result - = new HashSet(); + final HashSet result = new HashSet<>(); new FSLinkResolver() { @Override public Void next(final AbstractFileSystem fs, final Path p) @@ -2395,7 +2427,7 @@ public static Map getAllStatistics() { * @param p Path for which delegations tokens are requested. * @param renewer the account name that is allowed to renew the token. * @return List of delegation tokens. - * @throws IOException + * @throws IOException If an I/O error occurred. */ @InterfaceAudience.LimitedPrivate( { "HDFS", "MapReduce" }) public List> getDelegationTokens( @@ -2547,7 +2579,7 @@ public AclStatus next(final AbstractFileSystem fs, final Path p) * @param path Path to modify * @param name xattr name. * @param value xattr value. - * @throws IOException + * @throws IOException If an I/O error occurred. */ public void setXAttr(Path path, String name, byte[] value) throws IOException { @@ -2566,7 +2598,7 @@ public void setXAttr(Path path, String name, byte[] value) * @param name xattr name. * @param value xattr value. * @param flag xattr set flag - * @throws IOException + * @throws IOException If an I/O error occurred. */ public void setXAttr(Path path, final String name, final byte[] value, final EnumSet flag) throws IOException { @@ -2591,7 +2623,7 @@ public Void next(final AbstractFileSystem fs, final Path p) * @param path Path to get extended attribute * @param name xattr name. * @return byte[] xattr value. - * @throws IOException + * @throws IOException If an I/O error occurred. */ public byte[] getXAttr(Path path, final String name) throws IOException { final Path absF = fixRelativePart(path); @@ -2614,7 +2646,7 @@ public byte[] next(final AbstractFileSystem fs, final Path p) * @param path Path to get extended attributes * @return Map{@literal <}String, byte[]{@literal >} describing the XAttrs * of the file or directory - * @throws IOException + * @throws IOException If an I/O error occurred. */ public Map getXAttrs(Path path) throws IOException { final Path absF = fixRelativePart(path); @@ -2638,7 +2670,7 @@ public Map next(final AbstractFileSystem fs, final Path p) * @param names XAttr names. * @return Map{@literal <}String, byte[]{@literal >} describing the XAttrs * of the file or directory - * @throws IOException + * @throws IOException If an I/O error occurred. */ public Map getXAttrs(Path path, final List names) throws IOException { @@ -2661,7 +2693,7 @@ public Map next(final AbstractFileSystem fs, final Path p) * * @param path Path to remove extended attribute * @param name xattr name - * @throws IOException + * @throws IOException If an I/O error occurred. */ public void removeXAttr(Path path, final String name) throws IOException { final Path absF = fixRelativePart(path); @@ -2685,7 +2717,7 @@ public Void next(final AbstractFileSystem fs, final Path p) * @param path Path to get extended attributes * @return List{@literal <}String{@literal >} of the XAttr names of the * file or directory - * @throws IOException + * @throws IOException If an I/O error occurred. */ public List listXAttrs(Path path) throws IOException { final Path absF = fixRelativePart(path); @@ -2802,7 +2834,7 @@ public Void next(final AbstractFileSystem fs, final Path p) /** * Set the source path to satisfy storage policy. * @param path The source path referring to either a directory or a file. - * @throws IOException + * @throws IOException If an I/O error occurred. */ public void satisfyStoragePolicy(final Path path) throws IOException { @@ -2824,6 +2856,7 @@ public Void next(final AbstractFileSystem fs, final Path p) * @param policyName the name of the target storage policy. The list * of supported Storage policies can be retrieved * via {@link #getAllStoragePolicies}. + * @throws IOException If an I/O error occurred. */ public void setStoragePolicy(final Path path, final String policyName) throws IOException { @@ -2841,7 +2874,7 @@ public Void next(final AbstractFileSystem fs, final Path p) /** * Unset the storage policy set for a given file or directory. * @param src file or directory path. - * @throws IOException + * @throws IOException If an I/O error occurred. */ public void unsetStoragePolicy(final Path src) throws IOException { final Path absF = fixRelativePart(src); @@ -2860,7 +2893,7 @@ public Void next(final AbstractFileSystem fs, final Path p) * * @param path file or directory path. * @return storage policy for give file. - * @throws IOException + * @throws IOException If an I/O error occurred. */ public BlockStoragePolicySpi getStoragePolicy(Path path) throws IOException { final Path absF = fixRelativePart(path); @@ -2878,7 +2911,7 @@ public BlockStoragePolicySpi next(final AbstractFileSystem fs, * Retrieve all the storage policies supported by this file system. * * @return all storage policies supported by this filesystem. - * @throws IOException + * @throws IOException If an I/O error occurred. */ public Collection getAllStoragePolicies() throws IOException { diff --git a/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/fs/FileEncryptionInfo.java b/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/fs/FileEncryptionInfo.java index 9260b9a62c62e..f50c06cec3810 100644 --- a/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/fs/FileEncryptionInfo.java +++ b/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/fs/FileEncryptionInfo.java @@ -52,6 +52,7 @@ public class FileEncryptionInfo implements Serializable { * @param keyName name of the key used for the encryption zone * @param ezKeyVersionName name of the KeyVersion used to encrypt the * encrypted data encryption key. + * @param version version. */ public FileEncryptionInfo(final CipherSuite suite, final CryptoProtocolVersion version, final byte[] edek, @@ -134,6 +135,8 @@ public String toString() { * * NOTE: * Currently this method is used by CLI for backward compatibility. + * + * @return stable string. */ public String toStringStable() { StringBuilder builder = new StringBuilder("{") diff --git a/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/fs/FileRange.java b/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/fs/FileRange.java new file mode 100644 index 0000000000000..e55696e96507e --- /dev/null +++ b/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/fs/FileRange.java @@ -0,0 +1,67 @@ +/* + * 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.fs; + +import java.nio.ByteBuffer; +import java.util.concurrent.CompletableFuture; + +import org.apache.hadoop.fs.impl.FileRangeImpl; + +/** + * A byte range of a file. + * This is used for the asynchronous gather read API of + * {@link PositionedReadable#readVectored}. + */ +public interface FileRange { + + /** + * Get the starting offset of the range. + * @return the byte offset of the start + */ + long getOffset(); + + /** + * Get the length of the range. + * @return the number of bytes in the range. + */ + int getLength(); + + /** + * Get the future data for this range. + * @return the future for the {@link ByteBuffer} that contains the data + */ + CompletableFuture getData(); + + /** + * Set a future for this range's data. + * This method is called by {@link PositionedReadable#readVectored} to store the + * data for the user to pick up later via {@link #getData}. + * @param data the future of the ByteBuffer that will have the data + */ + void setData(CompletableFuture data); + + /** + * Factory method to create a FileRange object. + * @param offset starting offset of the range. + * @param length length of the range. + * @return a new instance of FileRangeImpl. + */ + static FileRange createFileRange(long offset, int length) { + return new FileRangeImpl(offset, length); + } +} diff --git a/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/fs/FileStatus.java b/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/fs/FileStatus.java index d7ca8f172f8e2..fcef578b072f1 100644 --- a/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/fs/FileStatus.java +++ b/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/fs/FileStatus.java @@ -116,6 +116,17 @@ public FileStatus(long length, boolean isdir, int block_replication, /** * Constructor for file systems on which symbolic links are not supported + * + * @param length length. + * @param isdir isdir. + * @param block_replication block replication. + * @param blocksize block size. + * @param modification_time modification time. + * @param access_time access_time. + * @param permission permission. + * @param owner owner. + * @param group group. + * @param path the path. */ public FileStatus(long length, boolean isdir, int block_replication, @@ -182,6 +193,7 @@ public FileStatus(long length, boolean isdir, int block_replication, * Copy constructor. * * @param other FileStatus to copy + * @throws IOException raised on errors performing I/O. */ public FileStatus(FileStatus other) throws IOException { // It's important to call the getters here instead of directly accessing the @@ -375,6 +387,8 @@ protected void setGroup(String group) { /** * @return The contents of the symbolic link. + * + * @throws IOException raised on errors performing I/O. */ public Path getSymlink() throws IOException { if (!isSymlink()) { diff --git a/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/fs/FileSystem.java b/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/fs/FileSystem.java index aa194e84a35d6..0bc419b035380 100644 --- a/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/fs/FileSystem.java +++ b/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/fs/FileSystem.java @@ -104,13 +104,13 @@ * All user code that may potentially use the Hadoop Distributed * File System should be written to use a FileSystem object or its * successor, {@link FileContext}. - * + *

    *

    * The local implementation is {@link LocalFileSystem} and distributed * implementation is DistributedFileSystem. There are other implementations * for object stores and (outside the Apache Hadoop codebase), * third party filesystems. - *

    + *

    * Notes *
      *
    1. The behaviour of the filesystem is @@ -133,13 +133,12 @@ * New methods may be marked as Unstable or Evolving for their initial release, * as a warning that they are new and may change based on the * experience of use in applications. - *

      + *

      * Important note for developers - *

      + *

      * If you are making changes here to the public API or protected methods, * you must review the following subclasses and make sure that * they are filtering/passing through new methods as appropriate. - *

      * * {@link FilterFileSystem}: methods are passed through. If not, * then {@code TestFilterFileSystem.MustNotImplement} must be @@ -148,21 +147,22 @@ * {@link #hasPathCapability(Path, String)} then * {@link FilterFileSystem#hasPathCapability(Path, String)} * must return false, always. - *

      + *

      * {@link ChecksumFileSystem}: checksums are created and * verified. - *

      + *

      * {@code TestHarFileSystem} will need its {@code MustNotImplement} * interface updated. - *

      * + *

      * There are some external places your changes will break things. * Do co-ordinate changes here. - *

      + *

      * * HBase: HBoss - *

      + *

      * Hive: HiveShim23 + *

      * {@code shims/0.23/src/main/java/org/apache/hadoop/hive/shims/Hadoop23Shims.java} * *****************************************************************/ @@ -281,6 +281,8 @@ public FileSystem run() throws IOException { /** * Returns the configured FileSystem implementation. * @param conf the configuration to use + * @return FileSystem. + * @throws IOException If an I/O error occurred. */ public static FileSystem get(Configuration conf) throws IOException { return get(getDefaultUri(conf), conf); @@ -375,6 +377,7 @@ public String getScheme() { * implement that method. * * @see #canonicalizeUri(URI) + * @return the URI of this filesystem. */ protected URI getCanonicalUri() { return canonicalizeUri(getUri()); @@ -391,6 +394,7 @@ protected URI getCanonicalUri() { * not specified and if {@link #getDefaultPort()} returns a * default port. * + * @param uri url. * @return URI * @see NetUtils#getCanonicalUri(URI, int) */ @@ -454,11 +458,21 @@ public String getCanonicalServiceName() { : null; } - /** @deprecated call {@link #getUri()} instead.*/ + /** + * @return uri to string. + * @deprecated call {@link #getUri()} instead. + */ @Deprecated public String getName() { return getUri().toString(); } - /** @deprecated call {@link #get(URI, Configuration)} instead. */ + /** + * @deprecated call {@link #get(URI, Configuration)} instead. + * + * @param name name. + * @param conf configuration. + * @return file system. + * @throws IOException If an I/O error occurred. + */ @Deprecated public static FileSystem getNamed(String name, Configuration conf) throws IOException { @@ -513,6 +527,9 @@ public static LocalFileSystem getLocal(Configuration conf) * configuration and URI, cached and returned to the caller. *
    2. *
    + * @param uri uri of the filesystem. + * @param conf configrution. + * @return filesystem instance. * @throws IOException if the FileSystem cannot be instantiated. */ public static FileSystem get(URI uri, Configuration conf) throws IOException { @@ -542,7 +559,7 @@ public static FileSystem get(URI uri, Configuration conf) throws IOException { /** * Returns the FileSystem for this URI's scheme and authority and the * given user. Internally invokes {@link #newInstance(URI, Configuration)} - * @param uri of the filesystem + * @param uri uri of the filesystem. * @param conf the configuration to use * @param user to perform the get as * @return filesystem instance @@ -860,6 +877,7 @@ protected void checkPath(Path path) { * @param start offset into the given file * @param len length for which to get locations for * @throws IOException IO failure + * @return block location array. */ public BlockLocation[] getFileBlockLocations(FileStatus file, long start, long len) throws IOException { @@ -900,6 +918,7 @@ public BlockLocation[] getFileBlockLocations(FileStatus file, * @param len length for which to get locations for * @throws FileNotFoundException when the path does not exist * @throws IOException IO failure + * @return block location array. */ public BlockLocation[] getFileBlockLocations(Path p, long start, long len) throws IOException { @@ -962,6 +981,7 @@ public Path resolvePath(final Path p) throws IOException { * @param f the file name to open * @param bufferSize the size of the buffer to be used. * @throws IOException IO failure + * @return input stream. */ public abstract FSDataInputStream open(Path f, int bufferSize) throws IOException; @@ -970,6 +990,7 @@ public abstract FSDataInputStream open(Path f, int bufferSize) * Opens an FSDataInputStream at the indicated Path. * @param f the file to open * @throws IOException IO failure + * @return input stream. */ public FSDataInputStream open(Path f) throws IOException { return open(f, getConf().getInt(IO_FILE_BUFFER_SIZE_KEY, @@ -987,6 +1008,7 @@ public FSDataInputStream open(Path f) throws IOException { * @throws IOException IO failure * @throws UnsupportedOperationException If {@link #open(PathHandle, int)} * not overridden by subclass + * @return input stream. */ public FSDataInputStream open(PathHandle fd) throws IOException { return open(fd, getConf().getInt(IO_FILE_BUFFER_SIZE_KEY, @@ -1004,6 +1026,7 @@ public FSDataInputStream open(PathHandle fd) throws IOException { * not satisfied * @throws IOException IO failure * @throws UnsupportedOperationException If not overridden by subclass + * @return input stream. */ public FSDataInputStream open(PathHandle fd, int bufferSize) throws IOException { @@ -1021,6 +1044,7 @@ public FSDataInputStream open(PathHandle fd, int bufferSize) * not overridden by subclass. * @throws UnsupportedOperationException If this FileSystem cannot enforce * the specified constraints. + * @return path handle. */ public final PathHandle getPathHandle(FileStatus stat, HandleOpt... opt) { // method is final with a default so clients calling getPathHandle(stat) @@ -1036,6 +1060,7 @@ public final PathHandle getPathHandle(FileStatus stat, HandleOpt... opt) { * @param stat Referent in the target FileSystem * @param opt Constraints that determine the validity of the * {@link PathHandle} reference. + * @return path handle. */ protected PathHandle createPathHandle(FileStatus stat, HandleOpt... opt) { throw new UnsupportedOperationException(); @@ -1046,6 +1071,7 @@ protected PathHandle createPathHandle(FileStatus stat, HandleOpt... opt) { * Files are overwritten by default. * @param f the file to create * @throws IOException IO failure + * @return output stream. */ public FSDataOutputStream create(Path f) throws IOException { return create(f, true); @@ -1057,6 +1083,7 @@ public FSDataOutputStream create(Path f) throws IOException { * @param overwrite if a file with this name already exists, then if true, * the file will be overwritten, and if false an exception will be thrown. * @throws IOException IO failure + * @return output stream. */ public FSDataOutputStream create(Path f, boolean overwrite) throws IOException { @@ -1074,6 +1101,7 @@ public FSDataOutputStream create(Path f, boolean overwrite) * @param f the file to create * @param progress to report progress * @throws IOException IO failure + * @return output stream. */ public FSDataOutputStream create(Path f, Progressable progress) throws IOException { @@ -1090,6 +1118,7 @@ public FSDataOutputStream create(Path f, Progressable progress) * @param f the file to create * @param replication the replication factor * @throws IOException IO failure + * @return output stream1 */ public FSDataOutputStream create(Path f, short replication) throws IOException { @@ -1108,6 +1137,7 @@ public FSDataOutputStream create(Path f, short replication) * @param replication the replication factor * @param progress to report progress * @throws IOException IO failure + * @return output stream. */ public FSDataOutputStream create(Path f, short replication, Progressable progress) throws IOException { @@ -1125,6 +1155,7 @@ public FSDataOutputStream create(Path f, short replication, * the file will be overwritten, and if false an error will be thrown. * @param bufferSize the size of the buffer to be used. * @throws IOException IO failure + * @return output stream. */ public FSDataOutputStream create(Path f, boolean overwrite, @@ -1144,7 +1175,9 @@ public FSDataOutputStream create(Path f, * @param overwrite if a file with this name already exists, then if true, * the file will be overwritten, and if false an error will be thrown. * @param bufferSize the size of the buffer to be used. + * @param progress to report progress. * @throws IOException IO failure + * @return output stream. */ public FSDataOutputStream create(Path f, boolean overwrite, @@ -1164,7 +1197,9 @@ public FSDataOutputStream create(Path f, * the file will be overwritten, and if false an error will be thrown. * @param bufferSize the size of the buffer to be used. * @param replication required block replication for the file. + * @param blockSize the size of the buffer to be used. * @throws IOException IO failure + * @return output stream. */ public FSDataOutputStream create(Path f, boolean overwrite, @@ -1182,7 +1217,10 @@ public FSDataOutputStream create(Path f, * the file will be overwritten, and if false an error will be thrown. * @param bufferSize the size of the buffer to be used. * @param replication required block replication for the file. + * @param blockSize the size of the buffer to be used. + * @param progress to report progress. * @throws IOException IO failure + * @return output stream. */ public FSDataOutputStream create(Path f, boolean overwrite, @@ -1209,6 +1247,7 @@ public FSDataOutputStream create(Path f, * @param progress the progress reporter * @throws IOException IO failure * @see #setPermission(Path, FsPermission) + * @return output stream. */ public abstract FSDataOutputStream create(Path f, FsPermission permission, @@ -1230,6 +1269,7 @@ public abstract FSDataOutputStream create(Path f, * @param progress the progress reporter * @throws IOException IO failure * @see #setPermission(Path, FsPermission) + * @return output stream. */ public FSDataOutputStream create(Path f, FsPermission permission, @@ -1256,6 +1296,7 @@ public FSDataOutputStream create(Path f, * found in conf will be used. * @throws IOException IO failure * @see #setPermission(Path, FsPermission) + * @return output stream. */ public FSDataOutputStream create(Path f, FsPermission permission, @@ -1277,6 +1318,16 @@ public FSDataOutputStream create(Path f, * the permission with umask before calling this method. * This a temporary method added to support the transition from FileSystem * to FileContext for user applications. + * + * @param f path. + * @param absolutePermission permission. + * @param flag create flag. + * @param bufferSize buffer size. + * @param replication replication. + * @param blockSize block size. + * @param progress progress. + * @param checksumOpt check sum opt. + * @return output stream. * @throws IOException IO failure */ @Deprecated @@ -1331,6 +1382,11 @@ protected boolean primitiveMkdir(Path f, FsPermission absolutePermission) * with umask before calling this method. * This a temporary method added to support the transition from FileSystem * to FileContext for user applications. + * + * @param f the path. + * @param absolutePermission permission. + * @param createParent create parent. + * @throws IOException IO failure. */ @Deprecated protected void primitiveMkdir(Path f, FsPermission absolutePermission, @@ -1370,6 +1426,7 @@ protected void primitiveMkdir(Path f, FsPermission absolutePermission, * @param progress the progress reporter * @throws IOException IO failure * @see #setPermission(Path, FsPermission) + * @return output stream. */ public FSDataOutputStream createNonRecursive(Path f, boolean overwrite, @@ -1393,6 +1450,7 @@ public FSDataOutputStream createNonRecursive(Path f, * @param progress the progress reporter * @throws IOException IO failure * @see #setPermission(Path, FsPermission) + * @return output stream. */ public FSDataOutputStream createNonRecursive(Path f, FsPermission permission, boolean overwrite, int bufferSize, short replication, long blockSize, @@ -1416,6 +1474,7 @@ public FSDataOutputStream createNonRecursive(Path f, FsPermission permission, * @param progress the progress reporter * @throws IOException IO failure * @see #setPermission(Path, FsPermission) + * @return output stream. */ public FSDataOutputStream createNonRecursive(Path f, FsPermission permission, EnumSet flags, int bufferSize, short replication, long blockSize, @@ -1430,6 +1489,7 @@ public FSDataOutputStream createNonRecursive(Path f, FsPermission permission, * Important: the default implementation is not atomic * @param f path to use for create * @throws IOException IO failure + * @return if create new file success true,not false. */ public boolean createNewFile(Path f) throws IOException { if (exists(f)) { @@ -1450,6 +1510,7 @@ public boolean createNewFile(Path f) throws IOException { * @throws IOException IO failure * @throws UnsupportedOperationException if the operation is unsupported * (default). + * @return output stream. */ public FSDataOutputStream append(Path f) throws IOException { return append(f, getConf().getInt(IO_FILE_BUFFER_SIZE_KEY, @@ -1464,6 +1525,7 @@ public FSDataOutputStream append(Path f) throws IOException { * @throws IOException IO failure * @throws UnsupportedOperationException if the operation is unsupported * (default). + * @return output stream. */ public FSDataOutputStream append(Path f, int bufferSize) throws IOException { return append(f, bufferSize, null); @@ -1477,6 +1539,7 @@ public FSDataOutputStream append(Path f, int bufferSize) throws IOException { * @throws IOException IO failure * @throws UnsupportedOperationException if the operation is unsupported * (default). + * @return output stream. */ public abstract FSDataOutputStream append(Path f, int bufferSize, Progressable progress) throws IOException; @@ -1515,7 +1578,7 @@ public short getReplication(Path src) throws IOException { * This is the default behavior. * @param src file name * @param replication new replication - * @throws IOException + * @throws IOException an IO failure. * @return true if successful, or the feature in unsupported; * false if replication is supported but the file does not exist, * or is a directory @@ -1544,11 +1607,12 @@ public boolean setReplication(Path src, short replication) *

    * If OVERWRITE option is not passed as an argument, rename fails * if the dst already exists. + *

    *

    * If OVERWRITE option is passed as an argument, rename overwrites * the dst if it is a file or an empty directory. Rename fails if dst is * a non-empty directory. - *

    + *

    * Note that atomicity of rename is dependent on the file system * implementation. Please refer to the file system documentation for * details. This default implementation is non atomic. @@ -1556,9 +1620,11 @@ public boolean setReplication(Path src, short replication) * This method is deprecated since it is a temporary method added to * support the transition from FileSystem to FileContext for user * applications. + *

    * * @param src path to be renamed * @param dst new path after rename + * @param options rename options. * @throws FileNotFoundException src path does not exist, or the parent * path of dst does not exist. * @throws FileAlreadyExistsException dest path exists and is a file @@ -1653,6 +1719,9 @@ public boolean truncate(Path f, long newLength) throws IOException { /** * Delete a file/directory. + * @param f the path. + * @throws IOException IO failure. + * @return if delete success true, not false. * @deprecated Use {@link #delete(Path, boolean)} instead. */ @Deprecated @@ -1769,6 +1838,7 @@ public boolean exists(Path f) throws IOException { * @param f path to check * @throws IOException IO failure * @deprecated Use {@link #getFileStatus(Path)} instead + * @return if f is directory true, not false. */ @Deprecated public boolean isDirectory(Path f) throws IOException { @@ -1786,6 +1856,7 @@ public boolean isDirectory(Path f) throws IOException { * @param f path to check * @throws IOException IO failure * @deprecated Use {@link #getFileStatus(Path)} instead + * @return if f is file true, not false. */ @Deprecated public boolean isFile(Path f) throws IOException { @@ -1798,6 +1869,7 @@ public boolean isFile(Path f) throws IOException { /** * The number of bytes in a file. + * @param f the path. * @return the number of bytes; 0 for a directory * @deprecated Use {@link #getFileStatus(Path)} instead. * @throws FileNotFoundException if the path does not resolve @@ -1812,6 +1884,7 @@ public long getLength(Path f) throws IOException { * @param f path to use * @throws FileNotFoundException if the path does not resolve * @throws IOException IO failure + * @return content summary. */ public ContentSummary getContentSummary(Path f) throws IOException { FileStatus status = getFileStatus(f); @@ -1946,9 +2019,9 @@ public boolean hasMore() { * @param f Path to list * @param token opaque iteration token returned by previous call, or null * if this is the first call. - * @return - * @throws FileNotFoundException - * @throws IOException + * @return directory entries. + * @throws FileNotFoundException when the path does not exist. + * @throws IOException If an I/O error occurred. */ @InterfaceAudience.Private protected DirectoryEntries listStatusBatch(Path f, byte[] token) throws @@ -1979,6 +2052,8 @@ private void listStatus(ArrayList results, Path f, /** * List corrupted file blocks. + * + * @param path the path. * @return an iterator over the corrupt files under the given path * (may contain duplicates if a file has more than one corrupt block) * @throws UnsupportedOperationException if the operation is unsupported @@ -2072,36 +2147,29 @@ public FileStatus[] listStatus(Path[] files, PathFilter filter) *
    ? *
    Matches any single character. * - *

    *

    * *
    Matches zero or more characters. * - *

    *

    [abc] *
    Matches a single character from character set * {a,b,c}. * - *

    *

    [a-b] *
    Matches a single character from the character range * {a...b}. Note that character a must be * lexicographically less than or equal to character b. * - *

    *

    [^a] *
    Matches a single character that is not from character set or range * {a}. Note that the ^ character must occur * immediately to the right of the opening bracket. * - *

    *

    \c *
    Removes (escapes) any special meaning of character c. * - *

    *

    {ab,cd} *
    Matches a string from the string set {ab, cd} * - *

    *

    {ab,c{de,fh}} *
    Matches a string from the string set {ab, cde, cfh} * @@ -2332,6 +2400,7 @@ public LocatedFileStatus next() throws IOException { /** Return the current user's home directory in this FileSystem. * The default implementation returns {@code "/user/$USER/"}. + * @return the path. */ public Path getHomeDirectory() { String username; @@ -2394,6 +2463,7 @@ public boolean mkdirs(Path f) throws IOException { * @param f path to create * @param permission to apply to f * @throws IOException IO failure + * @return if mkdir success true, not false. */ public abstract boolean mkdirs(Path f, FsPermission permission ) throws IOException; @@ -2441,6 +2511,7 @@ public void moveFromLocalFile(Path src, Path dst) * @param delSrc whether to delete the src * @param src path * @param dst path + * @throws IOException IO failure. */ public void copyFromLocalFile(boolean delSrc, Path src, Path dst) throws IOException { @@ -2555,6 +2626,7 @@ public void copyToLocalFile(boolean delSrc, Path src, Path dst, * @param fsOutputFile path of output file * @param tmpLocalFile path of local tmp file * @throws IOException IO failure + * @return the path. */ public Path startLocalOutput(Path fsOutputFile, Path tmpLocalFile) throws IOException { @@ -2602,6 +2674,7 @@ public void close() throws IOException { /** * Return the total size of all files in the filesystem. * @throws IOException IO failure + * @return the number of path used. */ public long getUsed() throws IOException { Path path = new Path("/"); @@ -2610,7 +2683,9 @@ public long getUsed() throws IOException { /** * Return the total size of all files from a specified path. + * @param path the path. * @throws IOException IO failure + * @return the number of path content summary. */ public long getUsed(Path path) throws IOException { return getContentSummary(path).getLength(); @@ -2633,6 +2708,7 @@ public long getBlockSize(Path f) throws IOException { * Return the number of bytes that large input files should be optimally * be split into to minimize I/O time. * @deprecated use {@link #getDefaultBlockSize(Path)} instead + * @return default block size. */ @Deprecated public long getDefaultBlockSize() { @@ -2685,8 +2761,8 @@ public short getDefaultReplication(Path path) { * In some FileSystem implementations such as HDFS metadata * synchronization is essential to guarantee consistency of read requests * particularly in HA setting. - * @throws IOException - * @throws UnsupportedOperationException + * @throws IOException If an I/O error occurred. + * @throws UnsupportedOperationException if the operation is unsupported. */ public void msync() throws IOException, UnsupportedOperationException { throw new UnsupportedOperationException(getClass().getCanonicalName() + @@ -2762,6 +2838,8 @@ static void checkAccessPermissions(FileStatus stat, FsAction mode) /** * See {@link FileContext#fixRelativePart}. + * @param p the path. + * @return relative part. */ protected Path fixRelativePart(Path p) { if (p.isUriPathAbsolute()) { @@ -2773,6 +2851,18 @@ protected Path fixRelativePart(Path p) { /** * See {@link FileContext#createSymlink(Path, Path, boolean)}. + * + * @param target target path. + * @param link link. + * @param createParent create parent. + * @throws AccessControlException if access is denied. + * @throws FileAlreadyExistsException when the path does not exist. + * @throws FileNotFoundException when the path does not exist. + * @throws ParentNotDirectoryException if the parent path of dest is not + * a directory. + * @throws UnsupportedFileSystemException if there was no known implementation + * for the scheme. + * @throws IOException raised on errors performing I/O. */ public void createSymlink(final Path target, final Path link, final boolean createParent) throws AccessControlException, @@ -2786,8 +2876,14 @@ public void createSymlink(final Path target, final Path link, /** * See {@link FileContext#getFileLinkStatus(Path)}. - * @throws FileNotFoundException when the path does not exist - * @throws IOException see specific implementation + * + * @param f the path. + * @throws AccessControlException if access is denied. + * @throws FileNotFoundException when the path does not exist. + * @throws IOException raised on errors performing I/O. + * @throws UnsupportedFileSystemException if there was no known implementation + * for the scheme. + * @return file status */ public FileStatus getFileLinkStatus(final Path f) throws AccessControlException, FileNotFoundException, @@ -2798,6 +2894,7 @@ public FileStatus getFileLinkStatus(final Path f) /** * See {@link AbstractFileSystem#supportsSymlinks()}. + * @return if support symlinkls true, not false. */ public boolean supportsSymlinks() { return false; @@ -2805,8 +2902,11 @@ public boolean supportsSymlinks() { /** * See {@link FileContext#getLinkTarget(Path)}. + * @param f the path. * @throws UnsupportedOperationException if the operation is unsupported * (default outcome). + * @throws IOException IO failure. + * @return the path. */ public Path getLinkTarget(Path f) throws IOException { // Supporting filesystems should override this method @@ -2816,8 +2916,11 @@ public Path getLinkTarget(Path f) throws IOException { /** * See {@link AbstractFileSystem#getLinkTarget(Path)}. + * @param f the path. * @throws UnsupportedOperationException if the operation is unsupported * (default outcome). + * @throws IOException IO failure. + * @return the path. */ protected Path resolveLink(Path f) throws IOException { // Supporting filesystems should override this method @@ -3221,7 +3324,7 @@ public void removeXAttr(Path path, String name) throws IOException { /** * Set the source path to satisfy storage policy. * @param path The source path referring to either a directory or a file. - * @throws IOException + * @throws IOException If an I/O error occurred. */ public void satisfyStoragePolicy(final Path path) throws IOException { throw new UnsupportedOperationException( @@ -3529,7 +3632,7 @@ FileSystem getUnique(URI uri, Configuration conf) throws IOException{ * @param conf configuration * @param key key to store/retrieve this FileSystem in the cache * @return a cached or newly instantiated FileSystem. - * @throws IOException + * @throws IOException If an I/O error occurred. */ private FileSystem getInternal(URI uri, Configuration conf, Key key) throws IOException{ @@ -4024,6 +4127,7 @@ public void run() { /** * Get or create the thread-local data associated with the current thread. + * @return statistics data. */ public StatisticsData getThreadStatistics() { StatisticsData data = threadData.get(); @@ -4382,6 +4486,7 @@ public static synchronized Map getStatistics() { /** * Return the FileSystem classes that have Statistics. * @deprecated use {@link #getGlobalStorageStatistics()} + * @return statistics lists. */ @Deprecated public static synchronized List getAllStatistics() { @@ -4390,6 +4495,7 @@ public static synchronized List getAllStatistics() { /** * Get the statistics for a particular file system. + * @param scheme scheme. * @param cls the class to lookup * @return a statistics object * @deprecated use {@link #getGlobalStorageStatistics()} @@ -4424,6 +4530,7 @@ public static synchronized void clearStatistics() { /** * Print all statistics for all file systems to {@code System.out} + * @throws IOException If an I/O error occurred. */ public static synchronized void printStatistics() throws IOException { @@ -4464,6 +4571,7 @@ public StorageStatistics getStorageStatistics() { /** * Get the global storage statistics. + * @return global storage statistics. */ public static GlobalStorageStatistics getGlobalStorageStatistics() { return GlobalStorageStatistics.INSTANCE; diff --git a/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/fs/FileSystemLinkResolver.java b/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/fs/FileSystemLinkResolver.java index 7eec0eb7cec54..593495a1daa88 100644 --- a/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/fs/FileSystemLinkResolver.java +++ b/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/fs/FileSystemLinkResolver.java @@ -38,8 +38,8 @@ public abstract class FileSystemLinkResolver { * an UnresolvedLinkException if called on an unresolved {@link Path}. * @param p Path on which to perform an operation * @return Generic type returned by operation - * @throws IOException - * @throws UnresolvedLinkException + * @throws IOException raised on errors performing I/O. + * @throws UnresolvedLinkException unresolved link exception. */ abstract public T doCall(final Path p) throws IOException, UnresolvedLinkException; @@ -54,7 +54,7 @@ abstract public T doCall(final Path p) throws IOException, * @param p * Resolved Target of path * @return Generic type determined by implementation - * @throws IOException + * @throws IOException raised on errors performing I/O. */ abstract public T next(final FileSystem fs, final Path p) throws IOException; @@ -66,7 +66,7 @@ abstract public T doCall(final Path p) throws IOException, * @param filesys FileSystem with which to try call * @param path Path with which to try call * @return Generic type determined by implementation - * @throws IOException + * @throws IOException raised on errors performing I/O. */ public T resolve(final FileSystem filesys, final Path path) throws IOException { diff --git a/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/fs/FileUtil.java b/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/fs/FileUtil.java index 7400ca36daa5c..2af0a7b9e742f 100644 --- a/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/fs/FileUtil.java +++ b/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/fs/FileUtil.java @@ -162,6 +162,8 @@ public static void fullyDeleteOnExit(final File file) { * (3) If dir is a normal file, it is deleted. * (4) If dir is a normal directory, then dir and all its contents recursively * are deleted. + * @param dir dir. + * @return fully delete status. */ public static boolean fullyDelete(final File dir) { return fullyDelete(dir, false); @@ -257,6 +259,9 @@ private static boolean deleteImpl(final File f, final boolean doLog) { * we return false, the directory may be partially-deleted. * If dir is a symlink to a directory, all the contents of the actual * directory pointed to by dir will be deleted. + * + * @param dir dir. + * @return fullyDeleteContents Status. */ public static boolean fullyDeleteContents(final File dir) { return fullyDeleteContents(dir, false); @@ -267,8 +272,11 @@ public static boolean fullyDeleteContents(final File dir) { * we return false, the directory may be partially-deleted. * If dir is a symlink to a directory, all the contents of the actual * directory pointed to by dir will be deleted. + * + * @param dir dir. * @param tryGrantPermissions if 'true', try grant +rwx permissions to this * and all the underlying directories before trying to delete their contents. + * @return fully delete contents status. */ public static boolean fullyDeleteContents(final File dir, final boolean tryGrantPermissions) { if (tryGrantPermissions) { @@ -311,7 +319,7 @@ public static boolean fullyDeleteContents(final File dir, final boolean tryGrant * * @param fs {@link FileSystem} on which the path is present * @param dir directory to recursively delete - * @throws IOException + * @throws IOException raised on errors performing I/O. * @deprecated Use {@link FileSystem#delete(Path, boolean)} */ @Deprecated @@ -343,7 +351,17 @@ private static void checkDependencies(FileSystem srcFS, } } - /** Copy files between FileSystems. */ + /** + * Copy files between FileSystems. + * @param srcFS src fs. + * @param src src. + * @param dstFS dst fs. + * @param dst dst. + * @param deleteSource delete source. + * @param conf configuration. + * @return if copy success true, not false. + * @throws IOException raised on errors performing I/O. + */ public static boolean copy(FileSystem srcFS, Path src, FileSystem dstFS, Path dst, boolean deleteSource, @@ -391,7 +409,19 @@ public static boolean copy(FileSystem srcFS, Path[] srcs, return returnVal; } - /** Copy files between FileSystems. */ + /** + * Copy files between FileSystems. + * + * @param srcFS srcFs. + * @param src src. + * @param dstFS dstFs. + * @param dst dst. + * @param deleteSource delete source. + * @param overwrite overwrite. + * @param conf configuration. + * @throws IOException raised on errors performing I/O. + * @return true if the operation succeeded. + */ public static boolean copy(FileSystem srcFS, Path src, FileSystem dstFS, Path dst, boolean deleteSource, @@ -403,20 +433,21 @@ public static boolean copy(FileSystem srcFS, Path src, /** * Copy a file/directory tree within/between filesystems. - *

    + *

    * returns true if the operation succeeded. When deleteSource is true, * this means "after the copy, delete(source) returned true" * If the destination is a directory, and mkdirs (dest) fails, * the operation will return false rather than raise any exception. - *

    + *

    * The overwrite flag is about overwriting files; it has no effect about * handing an attempt to copy a file atop a directory (expect an IOException), * or a directory over a path which contains a file (mkdir will fail, so * "false"). - *

    + *

    * The operation is recursive, and the deleteSource operation takes place * as each subdirectory is copied. Therefore, if an operation fails partway * through, the source tree may be partially deleted. + *

    * @param srcFS source filesystem * @param srcStatus status of source * @param dstFS destination filesystem @@ -471,7 +502,17 @@ public static boolean copy(FileSystem srcFS, FileStatus srcStatus, } - /** Copy local files to a FileSystem. */ + /** + * Copy local files to a FileSystem. + * + * @param src src. + * @param dstFS dstFs. + * @param dst dst. + * @param deleteSource delete source. + * @param conf configuration. + * @throws IOException raised on errors performing I/O. + * @return true if the operation succeeded. + */ public static boolean copy(File src, FileSystem dstFS, Path dst, boolean deleteSource, @@ -514,7 +555,17 @@ public static boolean copy(File src, } } - /** Copy FileSystem files to local files. */ + /** + * Copy FileSystem files to local files. + * + * @param srcFS srcFs. + * @param src src. + * @param dst dst. + * @param deleteSource delete source. + * @param conf configuration. + * @throws IOException raised on errors performing I/O. + * @return true if the operation succeeded. + */ public static boolean copy(FileSystem srcFS, Path src, File dst, boolean deleteSource, Configuration conf) throws IOException { @@ -958,7 +1009,7 @@ public static void unTar(InputStream inputStream, File untarDir, * * @param inFile The tar file as input. * @param untarDir The untar directory where to untar the tar file. - * @throws IOException + * @throws IOException an exception occurred. */ public static void unTar(File inFile, File untarDir) throws IOException { if (!untarDir.mkdirs()) { @@ -1169,6 +1220,7 @@ public static class HardLink extends org.apache.hadoop.fs.HardLink { * @param target the target for symlink * @param linkname the symlink * @return 0 on success + * @throws IOException raised on errors performing I/O. */ public static int symLink(String target, String linkname) throws IOException{ @@ -1230,8 +1282,8 @@ public static int symLink(String target, String linkname) throws IOException{ * @param filename the name of the file to change * @param perm the permission string * @return the exit code from the command - * @throws IOException - * @throws InterruptedException + * @throws IOException raised on errors performing I/O. + * @throws InterruptedException command interrupted. */ public static int chmod(String filename, String perm ) throws IOException, InterruptedException { @@ -1245,7 +1297,7 @@ public static int chmod(String filename, String perm * @param perm permission string * @param recursive true, if permissions should be changed recursively * @return the exit code from the command. - * @throws IOException + * @throws IOException raised on errors performing I/O. */ public static int chmod(String filename, String perm, boolean recursive) throws IOException { @@ -1271,7 +1323,7 @@ public static int chmod(String filename, String perm, boolean recursive) * @param file the file to change * @param username the new user owner name * @param groupname the new group owner name - * @throws IOException + * @throws IOException raised on errors performing I/O. */ public static void setOwner(File file, String username, String groupname) throws IOException { @@ -1288,7 +1340,7 @@ public static void setOwner(File file, String username, * Platform independent implementation for {@link File#setReadable(boolean)} * File#setReadable does not work as expected on Windows. * @param f input file - * @param readable + * @param readable readable. * @return true on success, false otherwise */ public static boolean setReadable(File f, boolean readable) { @@ -1309,7 +1361,7 @@ public static boolean setReadable(File f, boolean readable) { * Platform independent implementation for {@link File#setWritable(boolean)} * File#setWritable does not work as expected on Windows. * @param f input file - * @param writable + * @param writable writable. * @return true on success, false otherwise */ public static boolean setWritable(File f, boolean writable) { @@ -1333,7 +1385,7 @@ public static boolean setWritable(File f, boolean writable) { * behavior on Windows as on Unix platforms. Creating, deleting or renaming * a file within that folder will still succeed on Windows. * @param f input file - * @param executable + * @param executable executable. * @return true on success, false otherwise */ public static boolean setExecutable(File f, boolean executable) { @@ -1412,7 +1464,7 @@ public static boolean canExecute(File f) { * of forking if group == other. * @param f the file to change * @param permission the new permissions - * @throws IOException + * @throws IOException raised on errors performing I/O. */ public static void setPermission(File f, FsPermission permission ) throws IOException { @@ -1717,6 +1769,7 @@ public static List getJarsInDirectory(String path) { * wildcard path to return all jars from the directory to use in a classpath. * * @param path the path to the directory. The path may include the wildcard. + * @param useLocal use local. * @return the list of jars as URLs, or an empty list if there are no jars, or * the directory does not exist */ diff --git a/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/fs/FilterFileSystem.java b/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/fs/FilterFileSystem.java index 607aa263622f6..cdbe51e330701 100644 --- a/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/fs/FilterFileSystem.java +++ b/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/fs/FilterFileSystem.java @@ -233,7 +233,7 @@ public FSDataOutputStream createNonRecursive(Path f, FsPermission permission, * * @param src file name * @param replication new replication - * @throws IOException + * @throws IOException raised on errors performing I/O. * @return true if successful; * false if file does not exist or is a directory */ @@ -304,7 +304,7 @@ public Path getHomeDirectory() { * Set the current working directory for the given file system. All relative * paths will be resolved relative to it. * - * @param newDir + * @param newDir new dir. */ @Override public void setWorkingDirectory(Path newDir) { diff --git a/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/fs/FsShell.java b/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/fs/FsShell.java index 7275b70227f99..73258661ec191 100644 --- a/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/fs/FsShell.java +++ b/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/fs/FsShell.java @@ -130,7 +130,7 @@ public Path getCurrentTrashDir() throws IOException { * Returns the current trash location for the path specified * @param path to be deleted * @return path to the trash - * @throws IOException + * @throws IOException raised on errors performing I/O. */ public Path getCurrentTrashDir(Path path) throws IOException { return getTrash().getCurrentTrashDir(path); diff --git a/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/fs/FsStatus.java b/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/fs/FsStatus.java index d392c7d765d72..c4bc341bf4f7c 100644 --- a/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/fs/FsStatus.java +++ b/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/fs/FsStatus.java @@ -35,24 +35,39 @@ public class FsStatus implements Writable { private long used; private long remaining; - /** Construct a FsStatus object, using the specified statistics */ + /** + * Construct a FsStatus object, using the specified statistics. + * + * @param capacity capacity. + * @param used used. + * @param remaining remaining. + */ public FsStatus(long capacity, long used, long remaining) { this.capacity = capacity; this.used = used; this.remaining = remaining; } - /** Return the capacity in bytes of the file system */ + /** + * Return the capacity in bytes of the file system. + * @return capacity. + */ public long getCapacity() { return capacity; } - /** Return the number of bytes used on the file system */ + /** + * Return the number of bytes used on the file system. + * @return used. + */ public long getUsed() { return used; } - /** Return the number of remaining bytes on the file system */ + /** + * Return the number of remaining bytes on the file system. + * @return remaining. + */ public long getRemaining() { return remaining; } diff --git a/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/fs/GlobExpander.java b/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/fs/GlobExpander.java index cb430ed3f6251..c87444c6c87f7 100644 --- a/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/fs/GlobExpander.java +++ b/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/fs/GlobExpander.java @@ -56,9 +56,9 @@ public StringWithOffset(String string, int offset) { * {a,b}/{c/\d} - {a,b}/c/d * * - * @param filePattern + * @param filePattern file pattern. * @return expanded file patterns - * @throws IOException + * @throws IOException raised on errors performing I/O. */ public static List expand(String filePattern) throws IOException { List fullyExpanded = new ArrayList(); diff --git a/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/fs/GlobalStorageStatistics.java b/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/fs/GlobalStorageStatistics.java index 30ce07a422e6e..d94339034447a 100644 --- a/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/fs/GlobalStorageStatistics.java +++ b/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/fs/GlobalStorageStatistics.java @@ -104,6 +104,8 @@ public synchronized void reset() { /** * Get an iterator that we can use to iterate throw all the global storage * statistics objects. + * + * @return StorageStatistics Iterator. */ synchronized public Iterator iterator() { Entry first = map.firstEntry(); diff --git a/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/fs/HarFileSystem.java b/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/fs/HarFileSystem.java index 7e12d0a11e953..1d64b0bcbe921 100644 --- a/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/fs/HarFileSystem.java +++ b/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/fs/HarFileSystem.java @@ -463,7 +463,7 @@ static BlockLocation[] fixBlockLocations(BlockLocation[] locations, * @param start the start of the desired range in the contained file * @param len the length of the desired range * @return block locations for this segment of file - * @throws IOException + * @throws IOException raised on errors performing I/O. */ @Override public BlockLocation[] getFileBlockLocations(FileStatus file, long start, @@ -525,7 +525,7 @@ private void fileStatusesInIndex(HarStatus parent, List statuses) * Combine the status stored in the index and the underlying status. * @param h status stored in the index * @return the combined file status - * @throws IOException + * @throws IOException raised on errors performing I/O. */ private FileStatus toFileStatus(HarStatus h) throws IOException { final Path p = h.isDir ? archivePath : new Path(archivePath, h.partName); @@ -635,7 +635,7 @@ public long getModificationTime() { * while creating a hadoop archive. * @param f the path in har filesystem * @return filestatus. - * @throws IOException + * @throws IOException raised on errors performing I/O. */ @Override public FileStatus getFileStatus(Path f) throws IOException { @@ -1104,7 +1104,7 @@ public void setDropBehind(Boolean dropBehind) throws IOException { * @param start the start position in the part file * @param length the length of valid data in the part file * @param bufsize the buffer size - * @throws IOException + * @throws IOException raised on errors performing I/O. */ public HarFSDataInputStream(FileSystem fs, Path p, long start, long length, int bufsize) throws IOException { diff --git a/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/fs/HardLink.java b/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/fs/HardLink.java index 855fbb04e59b4..1624c5d395aec 100644 --- a/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/fs/HardLink.java +++ b/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/fs/HardLink.java @@ -156,6 +156,7 @@ String[] linkCount(File file) throws IOException { * Creates a hardlink. * @param file - existing source file * @param linkName - desired target link file + * @throws IOException raised on errors performing I/O. */ public static void createHardLink(File file, File linkName) throws IOException { @@ -177,6 +178,7 @@ public static void createHardLink(File file, File linkName) * @param fileBaseNames - list of path-less file names, as returned by * parentDir.list() * @param linkDir - where the hardlinks should be put. It must already exist. + * @throws IOException raised on errors performing I/O. */ public static void createHardLinkMult(File parentDir, String[] fileBaseNames, File linkDir) throws IOException { @@ -204,6 +206,10 @@ public static void createHardLinkMult(File parentDir, String[] fileBaseNames, /** * Retrieves the number of links to the specified file. + * + * @param fileName file name. + * @throws IOException raised on errors performing I/O. + * @return link count. */ public static int getLinkCount(File fileName) throws IOException { if (fileName == null) { diff --git a/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/fs/HasFileDescriptor.java b/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/fs/HasFileDescriptor.java index bcf325ceca5df..a0e89d6aeac44 100644 --- a/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/fs/HasFileDescriptor.java +++ b/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/fs/HasFileDescriptor.java @@ -33,7 +33,7 @@ public interface HasFileDescriptor { /** * @return the FileDescriptor - * @throws IOException + * @throws IOException raised on errors performing I/O. */ public FileDescriptor getFileDescriptor() throws IOException; diff --git a/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/fs/LocalDirAllocator.java b/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/fs/LocalDirAllocator.java index 5f266a7b82555..f6c9d3c7cb0dd 100644 --- a/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/fs/LocalDirAllocator.java +++ b/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/fs/LocalDirAllocator.java @@ -78,8 +78,9 @@ public class LocalDirAllocator { private final DiskValidator diskValidator; - /**Create an allocator object - * @param contextCfgItemName + /** + * Create an allocator object. + * @param contextCfgItemName contextCfgItemName. */ public LocalDirAllocator(String contextCfgItemName) { this.contextCfgItemName = contextCfgItemName; @@ -123,7 +124,7 @@ private AllocatorPerContext obtainContext(String contextCfgItemName) { * available disk) * @param conf the Configuration object * @return the complete path to the file on a local disk - * @throws IOException + * @throws IOException raised on errors performing I/O. */ public Path getLocalPathForWrite(String pathStr, Configuration conf) throws IOException { @@ -139,7 +140,7 @@ public Path getLocalPathForWrite(String pathStr, * @param size the size of the file that is going to be written * @param conf the Configuration object * @return the complete path to the file on a local disk - * @throws IOException + * @throws IOException raised on errors performing I/O. */ public Path getLocalPathForWrite(String pathStr, long size, Configuration conf) throws IOException { @@ -156,7 +157,7 @@ public Path getLocalPathForWrite(String pathStr, long size, * @param conf the Configuration object * @param checkWrite ensure that the path is writable * @return the complete path to the file on a local disk - * @throws IOException + * @throws IOException raised on errors performing I/O. */ public Path getLocalPathForWrite(String pathStr, long size, Configuration conf, @@ -171,7 +172,7 @@ public Path getLocalPathForWrite(String pathStr, long size, * @param pathStr the requested file (this will be searched) * @param conf the Configuration object * @return the complete path to the file on a local disk - * @throws IOException + * @throws IOException raised on errors performing I/O. */ public Path getLocalPathToRead(String pathStr, Configuration conf) throws IOException { @@ -184,7 +185,7 @@ public Path getLocalPathToRead(String pathStr, * @param pathStr the path underneath the roots * @param conf the configuration to look up the roots in * @return all of the paths that exist under any of the roots - * @throws IOException + * @throws IOException raised on errors performing I/O. */ public Iterable getAllLocalPathsToRead(String pathStr, Configuration conf @@ -205,7 +206,7 @@ public Iterable getAllLocalPathsToRead(String pathStr, * @param size the size of the file that is going to be written * @param conf the Configuration object * @return a unique temporary file - * @throws IOException + * @throws IOException raised on errors performing I/O. */ public File createTmpFileForWrite(String pathStr, long size, Configuration conf) throws IOException { @@ -213,8 +214,9 @@ public File createTmpFileForWrite(String pathStr, long size, return context.createTmpFileForWrite(pathStr, size, conf); } - /** Method to check whether a context is valid - * @param contextCfgItemName + /** + * Method to check whether a context is valid. + * @param contextCfgItemName contextCfgItemName. * @return true/false */ public static boolean isContextValid(String contextCfgItemName) { @@ -224,9 +226,9 @@ public static boolean isContextValid(String contextCfgItemName) { } /** - * Removes the context from the context config items + * Removes the context from the context config items. * - * @param contextCfgItemName + * @param contextCfgItemName contextCfgItemName. */ @Deprecated @InterfaceAudience.LimitedPrivate({"MapReduce"}) @@ -236,8 +238,9 @@ public static void removeContext(String contextCfgItemName) { } } - /** We search through all the configured dirs for the file's existence - * and return true when we find + /** + * We search through all the configured dirs for the file's existence + * and return true when we find. * @param pathStr the requested file (this will be searched) * @param conf the Configuration object * @return true if files exist. false otherwise diff --git a/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/fs/LocalFileSystem.java b/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/fs/LocalFileSystem.java index c41190a7b360b..590cbd9a49ece 100644 --- a/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/fs/LocalFileSystem.java +++ b/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/fs/LocalFileSystem.java @@ -71,7 +71,11 @@ public LocalFileSystem(FileSystem rawLocalFileSystem) { super(rawLocalFileSystem); } - /** Convert a path to a File. */ + /** + * Convert a path to a File. + * @param path the path. + * @return file. + */ public File pathToFile(Path path) { return ((RawLocalFileSystem)fs).pathToFile(path); } diff --git a/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/fs/MD5MD5CRC32CastagnoliFileChecksum.java b/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/fs/MD5MD5CRC32CastagnoliFileChecksum.java index 5a4a6a97cc4f7..354e4a6b4657d 100644 --- a/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/fs/MD5MD5CRC32CastagnoliFileChecksum.java +++ b/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/fs/MD5MD5CRC32CastagnoliFileChecksum.java @@ -28,7 +28,13 @@ public MD5MD5CRC32CastagnoliFileChecksum() { this(0, 0, null); } - /** Create a MD5FileChecksum */ + /** + * Create a MD5FileChecksum. + * + * @param bytesPerCRC bytesPerCRC. + * @param crcPerBlock crcPerBlock. + * @param md5 md5. + */ public MD5MD5CRC32CastagnoliFileChecksum(int bytesPerCRC, long crcPerBlock, MD5Hash md5) { super(bytesPerCRC, crcPerBlock, md5); } diff --git a/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/fs/MD5MD5CRC32FileChecksum.java b/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/fs/MD5MD5CRC32FileChecksum.java index 3fdb7e982621c..c5ac381f78238 100644 --- a/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/fs/MD5MD5CRC32FileChecksum.java +++ b/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/fs/MD5MD5CRC32FileChecksum.java @@ -44,7 +44,13 @@ public MD5MD5CRC32FileChecksum() { this(0, 0, null); } - /** Create a MD5FileChecksum */ + /** + * Create a MD5FileChecksum. + * + * @param bytesPerCRC bytesPerCRC. + * @param crcPerBlock crcPerBlock. + * @param md5 md5. + */ public MD5MD5CRC32FileChecksum(int bytesPerCRC, long crcPerBlock, MD5Hash md5) { this.bytesPerCRC = bytesPerCRC; this.crcPerBlock = crcPerBlock; @@ -76,7 +82,10 @@ public byte[] getBytes() { return WritableUtils.toByteArray(this); } - /** returns the CRC type */ + /** + * returns the CRC type. + * @return data check sum type. + */ public DataChecksum.Type getCrcType() { // default to the one that is understood by all releases. return DataChecksum.Type.CRC32; diff --git a/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/fs/MD5MD5CRC32GzipFileChecksum.java b/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/fs/MD5MD5CRC32GzipFileChecksum.java index 5164d0200d28d..f7996c8623717 100644 --- a/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/fs/MD5MD5CRC32GzipFileChecksum.java +++ b/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/fs/MD5MD5CRC32GzipFileChecksum.java @@ -28,7 +28,13 @@ public MD5MD5CRC32GzipFileChecksum() { this(0, 0, null); } - /** Create a MD5FileChecksum */ + /** + * Create a MD5FileChecksum. + * + * @param bytesPerCRC bytesPerCRC. + * @param crcPerBlock crcPerBlock. + * @param md5 md5. + */ public MD5MD5CRC32GzipFileChecksum(int bytesPerCRC, long crcPerBlock, MD5Hash md5) { super(bytesPerCRC, crcPerBlock, md5); } diff --git a/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/fs/MultipartUploader.java b/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/fs/MultipartUploader.java index dcb76b50b3429..5e4eda26c7f1d 100644 --- a/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/fs/MultipartUploader.java +++ b/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/fs/MultipartUploader.java @@ -31,10 +31,11 @@ /** * MultipartUploader is an interface for copying files multipart and across * multiple nodes. - *

    + *

    * The interface extends {@link IOStatisticsSource} so that there is no * need to cast an instance to see if is a source of statistics. * However, implementations MAY return null for their actual statistics. + *

    */ @InterfaceAudience.Public @InterfaceStability.Unstable diff --git a/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/fs/MultipartUploaderBuilder.java b/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/fs/MultipartUploaderBuilder.java index 381bfaa07f6d1..e7b0865063ee5 100644 --- a/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/fs/MultipartUploaderBuilder.java +++ b/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/fs/MultipartUploaderBuilder.java @@ -25,34 +25,43 @@ /** * Builder interface for Multipart readers. - * @param - * @param + * @param MultipartUploader Generic Type. + * @param MultipartUploaderBuilder Generic Type. */ public interface MultipartUploaderBuilder> extends FSBuilder { /** * Set permission for the file. + * @param perm permission. + * @return B Generics Type. */ B permission(@Nonnull FsPermission perm); /** * Set the size of the buffer to be used. + * @param bufSize buffer size. + * @return B Generics Type. */ B bufferSize(int bufSize); /** * Set replication factor. + * @param replica replica. + * @return B Generics Type. */ B replication(short replica); /** * Set block size. + * @param blkSize blkSize. + * @return B Generics Type. */ B blockSize(long blkSize); /** * Create an FSDataOutputStream at the specified path. + * @return B Generics Type. */ B create(); @@ -60,16 +69,21 @@ public interface MultipartUploaderBuilder ranges, + IntFunction allocate) throws IOException { + VectoredReadUtils.readVectored(this, ranges, allocate); + } } diff --git a/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/fs/QuotaUsage.java b/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/fs/QuotaUsage.java index b00a31891c867..b0103562123f4 100644 --- a/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/fs/QuotaUsage.java +++ b/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/fs/QuotaUsage.java @@ -105,7 +105,9 @@ public QuotaUsage build() { // Make it protected for the deprecated ContentSummary constructor. protected QuotaUsage() { } - /** Build the instance based on the builder. */ + /** Build the instance based on the builder. + * @param builder bulider. + */ protected QuotaUsage(Builder builder) { this.fileAndDirectoryCount = builder.fileAndDirectoryCount; this.quota = builder.quota; @@ -127,37 +129,67 @@ protected void setSpaceQuota(long spaceQuota) { this.spaceQuota = spaceQuota; } - /** Return the directory count. */ + /** + * Return the directory count. + * + * @return file and directory count. + */ public long getFileAndDirectoryCount() { return fileAndDirectoryCount; } - /** Return the directory quota. */ + /** + * Return the directory quota. + * + * @return quota. + */ public long getQuota() { return quota; } - /** Return (disk) space consumed. */ + /** + * Return (disk) space consumed. + * + * @return space consumed. + */ public long getSpaceConsumed() { return spaceConsumed; } - /** Return (disk) space quota. */ + /** + * Return (disk) space quota. + * + * @return space quota. + */ public long getSpaceQuota() { return spaceQuota; } - /** Return storage type quota. */ + /** + * Return storage type quota. + * + * @param type storage type. + * @return type quota. + */ public long getTypeQuota(StorageType type) { return (typeQuota != null) ? typeQuota[type.ordinal()] : -1L; } - /** Return storage type consumed. */ + /** + * Return storage type consumed. + * + * @param type storage type. + * @return type consumed. + */ public long getTypeConsumed(StorageType type) { return (typeConsumed != null) ? typeConsumed[type.ordinal()] : 0L; } - /** Return true if any storage type quota has been set. */ + /** + * Return true if any storage type quota has been set. + * + * @return if any storage type quota has been set true, not false. + * */ public boolean isTypeQuotaSet() { if (typeQuota != null) { for (StorageType t : StorageType.getTypesSupportingQuota()) { @@ -169,7 +201,12 @@ public boolean isTypeQuotaSet() { return false; } - /** Return true if any storage type consumption information is available. */ + /** + * Return true if any storage type consumption information is available. + * + * @return if any storage type consumption information + * is available, not false. + */ public boolean isTypeConsumedAvailable() { if (typeConsumed != null) { for (StorageType t : StorageType.getTypesSupportingQuota()) { @@ -271,12 +308,15 @@ public String toString(boolean hOption) { return toString(hOption, false, null); } - /** Return the string representation of the object in the output format. + /** + * Return the string representation of the object in the output format. * if hOption is false file sizes are returned in bytes * if hOption is true file sizes are returned in human readable * * @param hOption a flag indicating if human readable output if to be used - * @return the string representation of the object + * @param tOption type option. + * @param types storage types. + * @return the string representation of the object. */ public String toString(boolean hOption, boolean tOption, List types) { @@ -328,7 +368,7 @@ protected String getTypesQuotaUsage(boolean hOption, /** * return the header of with the StorageTypes. * - * @param storageTypes + * @param storageTypes storage types. * @return storage header string */ public static String getStorageTypeHeader(List storageTypes) { diff --git a/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/fs/RawLocalFileSystem.java b/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/fs/RawLocalFileSystem.java index edcc4a8b99e77..f525c3cba78fe 100644 --- a/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/fs/RawLocalFileSystem.java +++ b/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/fs/RawLocalFileSystem.java @@ -1,4 +1,4 @@ -/** +/* * 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 @@ -33,8 +33,11 @@ import java.io.FileDescriptor; import java.net.URI; import java.nio.ByteBuffer; +import java.nio.channels.AsynchronousFileChannel; +import java.nio.channels.CompletionHandler; import java.nio.file.Files; import java.nio.file.NoSuchFileException; +import java.nio.file.StandardOpenOption; import java.nio.file.attribute.BasicFileAttributes; import java.nio.file.attribute.BasicFileAttributeView; import java.nio.file.attribute.FileTime; @@ -44,6 +47,9 @@ import java.util.Optional; import java.util.StringTokenizer; import java.util.concurrent.atomic.AtomicLong; +import java.util.List; +import java.util.concurrent.CompletableFuture; +import java.util.function.IntFunction; import org.apache.hadoop.classification.InterfaceAudience; import org.apache.hadoop.classification.InterfaceStability; @@ -61,6 +67,7 @@ import org.apache.hadoop.util.StringUtils; import static org.apache.hadoop.fs.impl.PathCapabilitiesSupport.validatePathCapabilityArgs; +import static org.apache.hadoop.fs.VectoredReadUtils.sortRanges; import static org.apache.hadoop.fs.statistics.StreamStatisticNames.STREAM_READ_BYTES; import static org.apache.hadoop.fs.statistics.StreamStatisticNames.STREAM_READ_EXCEPTIONS; import static org.apache.hadoop.fs.statistics.StreamStatisticNames.STREAM_READ_SEEK_OPERATIONS; @@ -100,7 +107,12 @@ private Path makeAbsolute(Path f) { } } - /** Convert a path to a File. */ + /** + * Convert a path to a File. + * + * @param path the path. + * @return file. + */ public File pathToFile(Path path) { checkPath(path); if (!path.isAbsolute()) { @@ -125,7 +137,9 @@ public void initialize(URI uri, Configuration conf) throws IOException { class LocalFSFileInputStream extends FSInputStream implements HasFileDescriptor, IOStatisticsSource, StreamCapabilities { private FileInputStream fis; + private final File name; private long position; + private AsynchronousFileChannel asyncChannel = null; /** * Minimal set of counters. @@ -143,7 +157,8 @@ class LocalFSFileInputStream extends FSInputStream implements private final AtomicLong bytesRead; public LocalFSFileInputStream(Path f) throws IOException { - fis = new FileInputStream(pathToFile(f)); + name = pathToFile(f); + fis = new FileInputStream(name); bytesRead = ioStatistics.getCounterReference( STREAM_READ_BYTES); } @@ -174,10 +189,16 @@ public boolean seekToNewSource(long targetPos) throws IOException { @Override public int available() throws IOException { return fis.available(); } @Override - public void close() throws IOException { fis.close(); } - @Override public boolean markSupported() { return false; } - + + @Override + public void close() throws IOException { + fis.close(); + if (asyncChannel != null) { + asyncChannel.close(); + } + } + @Override public int read() throws IOException { try { @@ -257,6 +278,7 @@ public boolean hasCapability(String capability) { // new capabilities. switch (capability.toLowerCase(Locale.ENGLISH)) { case StreamCapabilities.IOSTATISTICS: + case StreamCapabilities.VECTOREDIO: return true; default: return false; @@ -267,8 +289,89 @@ public boolean hasCapability(String capability) { public IOStatistics getIOStatistics() { return ioStatistics; } + + AsynchronousFileChannel getAsyncChannel() throws IOException { + if (asyncChannel == null) { + synchronized (this) { + asyncChannel = AsynchronousFileChannel.open(name.toPath(), + StandardOpenOption.READ); + } + } + return asyncChannel; + } + + @Override + public void readVectored(List ranges, + IntFunction allocate) throws IOException { + + List sortedRanges = Arrays.asList(sortRanges(ranges)); + // Set up all of the futures, so that we can use them if things fail + for(FileRange range: sortedRanges) { + VectoredReadUtils.validateRangeRequest(range); + range.setData(new CompletableFuture<>()); + } + try { + AsynchronousFileChannel channel = getAsyncChannel(); + ByteBuffer[] buffers = new ByteBuffer[sortedRanges.size()]; + AsyncHandler asyncHandler = new AsyncHandler(channel, sortedRanges, buffers); + for(int i = 0; i < sortedRanges.size(); ++i) { + FileRange range = sortedRanges.get(i); + buffers[i] = allocate.apply(range.getLength()); + channel.read(buffers[i], range.getOffset(), i, asyncHandler); + } + } catch (IOException ioe) { + LOG.debug("Exception occurred during vectored read ", ioe); + for(FileRange range: sortedRanges) { + range.getData().completeExceptionally(ioe); + } + } + } } - + + /** + * A CompletionHandler that implements readFully and translates back + * into the form of CompletionHandler that our users expect. + */ + static class AsyncHandler implements CompletionHandler { + private final AsynchronousFileChannel channel; + private final List ranges; + private final ByteBuffer[] buffers; + + AsyncHandler(AsynchronousFileChannel channel, + List ranges, + ByteBuffer[] buffers) { + this.channel = channel; + this.ranges = ranges; + this.buffers = buffers; + } + + @Override + public void completed(Integer result, Integer r) { + FileRange range = ranges.get(r); + ByteBuffer buffer = buffers[r]; + if (result == -1) { + failed(new EOFException("Read past End of File"), r); + } else { + if (buffer.remaining() > 0) { + // issue a read for the rest of the buffer + // QQ: What if this fails? It has the same handler. + channel.read(buffer, range.getOffset() + buffer.position(), r, this); + } else { + // QQ: Why is this required? I think because we don't want the + // user to read data beyond limit. + buffer.flip(); + range.getData().complete(buffer); + } + } + } + + @Override + public void failed(Throwable exc, Integer r) { + LOG.debug("Failed while reading range {} ", r, exc); + ranges.get(r).getData().completeExceptionally(exc); + } + } + @Override public FSDataInputStream open(Path f, int bufferSize) throws IOException { getFileStatus(f); diff --git a/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/fs/Seekable.java b/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/fs/Seekable.java index 919c857ffa628..f7546d58e6084 100644 --- a/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/fs/Seekable.java +++ b/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/fs/Seekable.java @@ -32,17 +32,27 @@ public interface Seekable { * Seek to the given offset from the start of the file. * The next read() will be from that location. Can't * seek past the end of the file. + * + * @param pos offset from the start of the file. + * @throws IOException raised on errors performing I/O. */ void seek(long pos) throws IOException; - + /** * Return the current offset from the start of the file + * + * @return offset from the start of the file. + * @throws IOException raised on errors performing I/O. */ long getPos() throws IOException; /** - * Seeks a different copy of the data. Returns true if + * Seeks a different copy of the data. Returns true if * found a new source, false otherwise. + * + * @param targetPos target position. + * @return true if found a new source, false otherwise. + * @throws IOException raised on errors performing I/O. */ @InterfaceAudience.Private boolean seekToNewSource(long targetPos) throws IOException; diff --git a/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/fs/Stat.java b/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/fs/Stat.java index 07f0513290014..72a45309b17c9 100644 --- a/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/fs/Stat.java +++ b/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/fs/Stat.java @@ -73,8 +73,8 @@ public FileStatus getFileStatus() throws IOException { } /** - * Whether Stat is supported on the current platform - * @return + * Whether Stat is supported on the current platform. + * @return if is available true, not false. */ public static boolean isAvailable() { if (Shell.LINUX || Shell.FREEBSD || Shell.MAC) { diff --git a/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/fs/StorageStatistics.java b/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/fs/StorageStatistics.java index 2efe4566344ee..b4a86ab781280 100644 --- a/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/fs/StorageStatistics.java +++ b/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/fs/StorageStatistics.java @@ -127,6 +127,7 @@ public StorageStatistics(String name) { /** * Get the name of this StorageStatistics object. + * @return name of this StorageStatistics object */ public String getName() { return name; @@ -145,12 +146,15 @@ public String getScheme() { * * The values returned will depend on the type of FileSystem or FileContext * object. The values do not necessarily reflect a snapshot in time. + * + * @return LongStatistic Iterator. */ public abstract Iterator getLongStatistics(); /** * Get the value of a statistic. * + * @param key key. * @return null if the statistic is not being tracked or is not a * long statistic. The value of the statistic, otherwise. */ @@ -159,6 +163,7 @@ public String getScheme() { /** * Return true if a statistic is being tracked. * + * @param key key. * @return True only if the statistic is being tracked. */ public abstract boolean isTracked(String key); diff --git a/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/fs/StreamCapabilities.java b/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/fs/StreamCapabilities.java index 861178019505e..d68ef505dc3fe 100644 --- a/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/fs/StreamCapabilities.java +++ b/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/fs/StreamCapabilities.java @@ -80,6 +80,12 @@ public interface StreamCapabilities { */ String IOSTATISTICS = "iostatistics"; + /** + * Support for vectored IO api. + * See {@code PositionedReadable#readVectored(List, IntFunction)}. + */ + String VECTOREDIO = "readvectored"; + /** * Stream abort() capability implemented by {@link Abortable#abort()}. * This matches the Path Capability diff --git a/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/fs/Trash.java b/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/fs/Trash.java index e29cb9a4e0e33..a58a1a3cb8eb1 100644 --- a/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/fs/Trash.java +++ b/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/fs/Trash.java @@ -43,6 +43,7 @@ public class Trash extends Configured { /** * Construct a trash can accessor. * @param conf a Configuration + * @throws IOException raised on errors performing I/O. */ public Trash(Configuration conf) throws IOException { this(FileSystem.get(conf), conf); @@ -52,6 +53,7 @@ public Trash(Configuration conf) throws IOException { * Construct a trash can accessor for the FileSystem provided. * @param fs the FileSystem * @param conf a Configuration + * @throws IOException raised on errors performing I/O. */ public Trash(FileSystem fs, Configuration conf) throws IOException { super(conf); @@ -97,47 +99,74 @@ public static boolean moveToAppropriateTrash(FileSystem fs, Path p, } /** - * Returns whether the trash is enabled for this filesystem + * Returns whether the trash is enabled for this filesystem. + * + * @return return if isEnabled true,not false. */ public boolean isEnabled() { return trashPolicy.isEnabled(); } /** Move a file or directory to the current trash directory. + * + * @param path the path. * @return false if the item is already in the trash or trash is disabled + * @throws IOException raised on errors performing I/O. */ public boolean moveToTrash(Path path) throws IOException { return trashPolicy.moveToTrash(path); } - /** Create a trash checkpoint. */ + /** + * Create a trash checkpoint. + * @throws IOException raised on errors performing I/O. + */ public void checkpoint() throws IOException { trashPolicy.createCheckpoint(); } - /** Delete old checkpoint(s). */ + /** + * Delete old checkpoint(s). + * @throws IOException raised on errors performing I/O. + */ public void expunge() throws IOException { trashPolicy.deleteCheckpoint(); } - /** Delete all trash immediately. */ + /** + * Delete all trash immediately. + * @throws IOException raised on errors performing I/O. + */ public void expungeImmediately() throws IOException { trashPolicy.createCheckpoint(); trashPolicy.deleteCheckpointsImmediately(); } - /** get the current working directory */ + /** + * get the current working directory. + * + * @throws IOException on raised on errors performing I/O. + * @return Trash Dir. + */ Path getCurrentTrashDir() throws IOException { return trashPolicy.getCurrentTrashDir(); } - /** get the configured trash policy */ + /** + * get the configured trash policy. + * + * @return TrashPolicy. + */ TrashPolicy getTrashPolicy() { return trashPolicy; } - /** Return a {@link Runnable} that periodically empties the trash of all + /** + * Return a {@link Runnable} that periodically empties the trash of all * users, intended to be run by the superuser. + * + * @throws IOException on raised on errors performing I/O. + * @return Runnable. */ public Runnable getEmptier() throws IOException { return trashPolicy.getEmptier(); diff --git a/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/fs/TrashPolicy.java b/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/fs/TrashPolicy.java index 64fb81be99ee3..35e51f9e1cfb1 100644 --- a/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/fs/TrashPolicy.java +++ b/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/fs/TrashPolicy.java @@ -60,27 +60,34 @@ public void initialize(Configuration conf, FileSystem fs) { /** * Returns whether the Trash Policy is enabled for this filesystem. + * + * @return if isEnabled true,not false. */ public abstract boolean isEnabled(); /** * Move a file or directory to the current trash directory. + * @param path the path. * @return false if the item is already in the trash or trash is disabled + * @throws IOException raised on errors performing I/O. */ public abstract boolean moveToTrash(Path path) throws IOException; /** - * Create a trash checkpoint. + * Create a trash checkpoint. + * @throws IOException raised on errors performing I/O. */ public abstract void createCheckpoint() throws IOException; /** * Delete old trash checkpoint(s). + * @throws IOException raised on errors performing I/O. */ public abstract void deleteCheckpoint() throws IOException; /** * Delete all checkpoints immediately, ie empty trash. + * @throws IOException raised on errors performing I/O. */ public abstract void deleteCheckpointsImmediately() throws IOException; @@ -94,6 +101,8 @@ public void initialize(Configuration conf, FileSystem fs) { * TrashPolicy#getCurrentTrashDir(Path path). * It returns the trash location correctly for the path specified no matter * the path is in encryption zone or not. + * + * @return the path. */ public abstract Path getCurrentTrashDir(); @@ -102,7 +111,7 @@ public void initialize(Configuration conf, FileSystem fs) { * Policy * @param path path to be deleted * @return current trash directory for the path to be deleted - * @throws IOException + * @throws IOException raised on errors performing I/O. */ public Path getCurrentTrashDir(Path path) throws IOException { throw new UnsupportedOperationException(); @@ -111,6 +120,9 @@ public Path getCurrentTrashDir(Path path) throws IOException { /** * Return a {@link Runnable} that periodically empties the trash of all * users, intended to be run by the superuser. + * + * @throws IOException raised on errors performing I/O. + * @return Runnable. */ public abstract Runnable getEmptier() throws IOException; diff --git a/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/fs/VectoredReadUtils.java b/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/fs/VectoredReadUtils.java new file mode 100644 index 0000000000000..64107f1a18f89 --- /dev/null +++ b/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/fs/VectoredReadUtils.java @@ -0,0 +1,292 @@ +/* + * 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.fs; + +import java.io.EOFException; +import java.io.IOException; +import java.nio.ByteBuffer; +import java.util.ArrayList; +import java.util.Arrays; +import java.util.Comparator; +import java.util.List; +import java.util.concurrent.CompletableFuture; +import java.util.function.IntFunction; + +import org.apache.hadoop.fs.impl.CombinedFileRange; +import org.apache.hadoop.util.Preconditions; + +/** + * Utility class which implements helper methods used + * in vectored IO implementation. + */ +public final class VectoredReadUtils { + + /** + * Validate a single range. + * @param range file range. + * @throws EOFException any EOF Exception. + */ + public static void validateRangeRequest(FileRange range) + throws EOFException { + + Preconditions.checkArgument(range.getLength() >= 0, "length is negative"); + if (range.getOffset() < 0) { + throw new EOFException("position is negative"); + } + } + + /** + * Validate a list of vectored read ranges. + * @param ranges list of ranges. + * @throws EOFException any EOF exception. + */ + public static void validateVectoredReadRanges(List ranges) + throws EOFException { + for (FileRange range : ranges) { + validateRangeRequest(range); + } + } + + + + /** + * This is the default implementation which iterates through the ranges + * to read each synchronously, but the intent is that subclasses + * can make more efficient readers. + * The data or exceptions are pushed into {@link FileRange#getData()}. + * @param stream the stream to read the data from + * @param ranges the byte ranges to read + * @param allocate the byte buffer allocation + */ + public static void readVectored(PositionedReadable stream, + List ranges, + IntFunction allocate) { + for (FileRange range: ranges) { + range.setData(readRangeFrom(stream, range, allocate)); + } + } + + /** + * Synchronously reads a range from the stream dealing with the combinations + * of ByteBuffers buffers and PositionedReadable streams. + * @param stream the stream to read from + * @param range the range to read + * @param allocate the function to allocate ByteBuffers + * @return the CompletableFuture that contains the read data + */ + public static CompletableFuture readRangeFrom(PositionedReadable stream, + FileRange range, + IntFunction allocate) { + CompletableFuture result = new CompletableFuture<>(); + try { + ByteBuffer buffer = allocate.apply(range.getLength()); + if (stream instanceof ByteBufferPositionedReadable) { + ((ByteBufferPositionedReadable) stream).readFully(range.getOffset(), + buffer); + buffer.flip(); + } else { + readNonByteBufferPositionedReadable(stream, range, buffer); + } + result.complete(buffer); + } catch (IOException ioe) { + result.completeExceptionally(ioe); + } + return result; + } + + private static void readNonByteBufferPositionedReadable(PositionedReadable stream, + FileRange range, + ByteBuffer buffer) throws IOException { + if (buffer.isDirect()) { + buffer.put(readInDirectBuffer(stream, range)); + buffer.flip(); + } else { + stream.readFully(range.getOffset(), buffer.array(), + buffer.arrayOffset(), range.getLength()); + } + } + + private static byte[] readInDirectBuffer(PositionedReadable stream, + FileRange range) throws IOException { + // if we need to read data from a direct buffer and the stream doesn't + // support it, we allocate a byte array to use. + byte[] tmp = new byte[range.getLength()]; + stream.readFully(range.getOffset(), tmp, 0, tmp.length); + return tmp; + } + + /** + * Is the given input list. + *
      + *
    • already sorted by offset
    • + *
    • each range is more than minimumSeek apart
    • + *
    • the start and end of each range is a multiple of chunkSize
    • + *
    + * + * @param input the list of input ranges. + * @param chunkSize the size of the chunks that the offset and end must align to. + * @param minimumSeek the minimum distance between ranges. + * @return true if we can use the input list as is. + */ + public static boolean isOrderedDisjoint(List input, + int chunkSize, + int minimumSeek) { + long previous = -minimumSeek; + for (FileRange range: input) { + long offset = range.getOffset(); + long end = range.getOffset() + range.getLength(); + if (offset % chunkSize != 0 || + end % chunkSize != 0 || + (offset - previous < minimumSeek)) { + return false; + } + previous = end; + } + return true; + } + + /** + * Calculates floor value of offset based on chunk size. + * @param offset file offset. + * @param chunkSize file chunk size. + * @return floor value. + */ + public static long roundDown(long offset, int chunkSize) { + if (chunkSize > 1) { + return offset - (offset % chunkSize); + } else { + return offset; + } + } + + /** + * Calculates the ceil value of offset based on chunk size. + * @param offset file offset. + * @param chunkSize file chunk size. + * @return ceil value. + */ + public static long roundUp(long offset, int chunkSize) { + if (chunkSize > 1) { + long next = offset + chunkSize - 1; + return next - (next % chunkSize); + } else { + return offset; + } + } + + /** + * Check if the input ranges are overlapping in nature. + * We call two ranges to be overlapping when start offset + * of second is less than the end offset of first. + * End offset is calculated as start offset + length. + * @param input list if input ranges. + * @return true/false based on logic explained above. + */ + public static List validateNonOverlappingAndReturnSortedRanges( + List input) { + + if (input.size() <= 1) { + return input; + } + FileRange[] sortedRanges = sortRanges(input); + FileRange prev = sortedRanges[0]; + for (int i=1; i input) { + FileRange[] sortedRanges = input.toArray(new FileRange[0]); + Arrays.sort(sortedRanges, Comparator.comparingLong(FileRange::getOffset)); + return sortedRanges; + } + + /** + * Merge sorted ranges to optimize the access from the underlying file + * system. + * The motivations are that: + *
      + *
    • Upper layers want to pass down logical file ranges.
    • + *
    • Fewer reads have better performance.
    • + *
    • Applications want callbacks as ranges are read.
    • + *
    • Some file systems want to round ranges to be at checksum boundaries.
    • + *
    + * + * @param sortedRanges already sorted list of ranges based on offset. + * @param chunkSize round the start and end points to multiples of chunkSize + * @param minimumSeek the smallest gap that we should seek over in bytes + * @param maxSize the largest combined file range in bytes + * @return the list of sorted CombinedFileRanges that cover the input + */ + public static List mergeSortedRanges(List sortedRanges, + int chunkSize, + int minimumSeek, + int maxSize) { + + CombinedFileRange current = null; + List result = new ArrayList<>(sortedRanges.size()); + + // now merge together the ones that merge + for (FileRange range: sortedRanges) { + long start = roundDown(range.getOffset(), chunkSize); + long end = roundUp(range.getOffset() + range.getLength(), chunkSize); + if (current == null || !current.merge(start, end, range, minimumSeek, maxSize)) { + current = new CombinedFileRange(start, end, range); + result.add(current); + } + } + return result; + } + + /** + * Slice the data that was read to the user's request. + * This function assumes that the user's request is completely subsumed by the + * read data. This always creates a new buffer pointing to the same underlying + * data but with its own mark and position fields such that reading one buffer + * can't effect other's mark and position. + * @param readData the buffer with the readData + * @param readOffset the offset in the file for the readData + * @param request the user's request + * @return the readData buffer that is sliced to the user's request + */ + public static ByteBuffer sliceTo(ByteBuffer readData, long readOffset, + FileRange request) { + int offsetChange = (int) (request.getOffset() - readOffset); + int requestLength = request.getLength(); + readData = readData.slice(); + readData.position(offsetChange); + readData.limit(offsetChange + requestLength); + return readData; + } + + /** + * private constructor. + */ + private VectoredReadUtils() { + throw new UnsupportedOperationException(); + } +} diff --git a/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/fs/XAttrCodec.java b/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/fs/XAttrCodec.java index 3d65275e673d6..df878d998706c 100644 --- a/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/fs/XAttrCodec.java +++ b/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/fs/XAttrCodec.java @@ -67,7 +67,7 @@ public enum XAttrCodec { * the given string is treated as text. * @param value string representation of the value. * @return byte[] the value - * @throws IOException + * @throws IOException raised on errors performing I/O. */ public static byte[] decodeValue(String value) throws IOException { byte[] result = null; @@ -102,9 +102,9 @@ public static byte[] decodeValue(String value) throws IOException { * while strings encoded as hexadecimal and base64 are prefixed with * 0x and 0s, respectively. * @param value byte[] value - * @param encoding + * @param encoding encoding. * @return String string representation of value - * @throws IOException + * @throws IOException raised on errors performing I/O. */ public static String encodeValue(byte[] value, XAttrCodec encoding) throws IOException { diff --git a/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/fs/audit/AuditConstants.java b/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/fs/audit/AuditConstants.java index d9629e388b384..0929c2be03acf 100644 --- a/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/fs/audit/AuditConstants.java +++ b/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/fs/audit/AuditConstants.java @@ -90,6 +90,11 @@ private AuditConstants() { */ public static final String PARAM_PROCESS = "ps"; + /** + * Task Attempt ID query header: {@value}. + */ + public static final String PARAM_TASK_ATTEMPT_ID = "ta"; + /** * Thread 0: the thread which created a span {@value}. */ diff --git a/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/fs/audit/CommonAuditContext.java b/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/fs/audit/CommonAuditContext.java index e188e168e5313..2dcd4f8b3f570 100644 --- a/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/fs/audit/CommonAuditContext.java +++ b/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/fs/audit/CommonAuditContext.java @@ -124,11 +124,15 @@ private CommonAuditContext() { /** * Put a context entry. * @param key key - * @param value new value + * @param value new value., If null, triggers removal. * @return old value or null */ public Supplier put(String key, String value) { - return evaluatedEntries.put(key, () -> value); + if (value != null) { + return evaluatedEntries.put(key, () -> value); + } else { + return evaluatedEntries.remove(key); + } } /** diff --git a/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/fs/impl/AbstractFSBuilderImpl.java b/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/fs/impl/AbstractFSBuilderImpl.java index 9d3a46d633253..4256522b2a372 100644 --- a/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/fs/impl/AbstractFSBuilderImpl.java +++ b/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/fs/impl/AbstractFSBuilderImpl.java @@ -340,12 +340,14 @@ public Configuration getOptions() { /** * Get all the keys that are set as mandatory keys. + * @return mandatory keys. */ public Set getMandatoryKeys() { return Collections.unmodifiableSet(mandatoryKeys); } /** * Get all the keys that are set as optional keys. + * @return optional keys. */ public Set getOptionalKeys() { return Collections.unmodifiableSet(optionalKeys); diff --git a/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/fs/impl/AbstractMultipartUploader.java b/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/fs/impl/AbstractMultipartUploader.java index 416924e18d87c..f9ae9f55cc17f 100644 --- a/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/fs/impl/AbstractMultipartUploader.java +++ b/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/fs/impl/AbstractMultipartUploader.java @@ -127,7 +127,7 @@ protected void checkPutArguments(Path filePath, * {@inheritDoc}. * @param path path to abort uploads under. * @return a future to -1. - * @throws IOException + * @throws IOException raised on errors performing I/O. */ public CompletableFuture abortUploadsUnderPath(Path path) throws IOException { diff --git a/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/fs/impl/CombinedFileRange.java b/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/fs/impl/CombinedFileRange.java new file mode 100644 index 0000000000000..516bbb2c70c76 --- /dev/null +++ b/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/fs/impl/CombinedFileRange.java @@ -0,0 +1,70 @@ +/* + * 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.fs.impl; + +import org.apache.hadoop.fs.FileRange; + +import java.util.ArrayList; +import java.util.List; + +/** + * A file range that represents a set of underlying file ranges. + * This is used when we combine the user's FileRange objects + * together into a single read for efficiency. + */ +public class CombinedFileRange extends FileRangeImpl { + private ArrayList underlying = new ArrayList<>(); + + public CombinedFileRange(long offset, long end, FileRange original) { + super(offset, (int) (end - offset)); + this.underlying.add(original); + } + + /** + * Get the list of ranges that were merged together to form this one. + * @return the list of input ranges + */ + public List getUnderlying() { + return underlying; + } + + /** + * Merge this input range into the current one, if it is compatible. + * It is assumed that otherOffset is greater or equal the current offset, + * which typically happens by sorting the input ranges on offset. + * @param otherOffset the offset to consider merging + * @param otherEnd the end to consider merging + * @param other the underlying FileRange to add if we merge + * @param minSeek the minimum distance that we'll seek without merging the + * ranges together + * @param maxSize the maximum size that we'll merge into a single range + * @return true if we have merged the range into this one + */ + public boolean merge(long otherOffset, long otherEnd, FileRange other, + int minSeek, int maxSize) { + long end = this.getOffset() + this.getLength(); + long newEnd = Math.max(end, otherEnd); + if (otherOffset - end >= minSeek || newEnd - this.getOffset() > maxSize) { + return false; + } + this.setLength((int) (newEnd - this.getOffset())); + underlying.add(other); + return true; + } +} diff --git a/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/fs/impl/FileRangeImpl.java b/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/fs/impl/FileRangeImpl.java new file mode 100644 index 0000000000000..041e5f0a8d2d7 --- /dev/null +++ b/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/fs/impl/FileRangeImpl.java @@ -0,0 +1,74 @@ +/* + * 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.fs.impl; + +import java.nio.ByteBuffer; +import java.util.concurrent.CompletableFuture; + +import org.apache.hadoop.classification.InterfaceAudience; +import org.apache.hadoop.fs.FileRange; + +/** + * A range of bytes from a file with an optional buffer to read those bytes + * for zero copy. This shouldn't be created directly via constructor rather + * factory defined in {@code FileRange#createFileRange} should be used. + */ +@InterfaceAudience.Private +public class FileRangeImpl implements FileRange { + private long offset; + private int length; + private CompletableFuture reader; + + public FileRangeImpl(long offset, int length) { + this.offset = offset; + this.length = length; + } + + @Override + public String toString() { + return "range[" + offset + "," + (offset + length) + ")"; + } + + @Override + public long getOffset() { + return offset; + } + + @Override + public int getLength() { + return length; + } + + public void setOffset(long offset) { + this.offset = offset; + } + + public void setLength(int length) { + this.length = length; + } + + @Override + public void setData(CompletableFuture pReader) { + this.reader = pReader; + } + + @Override + public CompletableFuture getData() { + return reader; + } +} diff --git a/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/fs/impl/FutureDataInputStreamBuilderImpl.java b/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/fs/impl/FutureDataInputStreamBuilderImpl.java index 70e39de7388c3..833c21ec1a67f 100644 --- a/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/fs/impl/FutureDataInputStreamBuilderImpl.java +++ b/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/fs/impl/FutureDataInputStreamBuilderImpl.java @@ -126,6 +126,9 @@ protected int getBufferSize() { /** * Set the size of the buffer to be used. + * + * @param bufSize buffer size. + * @return FutureDataInputStreamBuilder. */ public FutureDataInputStreamBuilder bufferSize(int bufSize) { bufferSize = bufSize; @@ -137,6 +140,8 @@ public FutureDataInputStreamBuilder bufferSize(int bufSize) { * This must be used after the constructor has been invoked to create * the actual builder: it allows for subclasses to do things after * construction. + * + * @return FutureDataInputStreamBuilder. */ public FutureDataInputStreamBuilder builder() { return getThisBuilder(); diff --git a/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/fs/impl/FutureIOSupport.java b/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/fs/impl/FutureIOSupport.java index f47e5f4fbfbd6..0a080426c2b24 100644 --- a/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/fs/impl/FutureIOSupport.java +++ b/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/fs/impl/FutureIOSupport.java @@ -75,6 +75,8 @@ public static T awaitFuture(final Future future) * See {@link FutureIO#awaitFuture(Future, long, TimeUnit)}. * @param future future to evaluate * @param type of the result. + * @param timeout timeout. + * @param unit unit. * @return the result, if all went well. * @throws InterruptedIOException future was interrupted * @throws IOException if something went wrong diff --git a/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/fs/impl/MultipartUploaderBuilderImpl.java b/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/fs/impl/MultipartUploaderBuilderImpl.java index 5584e647849f5..665bcc6a95660 100644 --- a/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/fs/impl/MultipartUploaderBuilderImpl.java +++ b/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/fs/impl/MultipartUploaderBuilderImpl.java @@ -88,6 +88,9 @@ protected MultipartUploaderBuilderImpl(@Nonnull FileContext fc, /** * Constructor. + * + * @param fileSystem fileSystem. + * @param p path. */ protected MultipartUploaderBuilderImpl(@Nonnull FileSystem fileSystem, @Nonnull Path p) { diff --git a/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/fs/permission/AclStatus.java b/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/fs/permission/AclStatus.java index 25b9ba659048a..260ee7e570c9b 100644 --- a/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/fs/permission/AclStatus.java +++ b/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/fs/permission/AclStatus.java @@ -185,7 +185,8 @@ public Builder stickyBit(boolean stickyBit) { /** * Sets the permission for the file. - * @param permission + * @param permission permission. + * @return Builder. */ public Builder setPermission(FsPermission permission) { this.permission = permission; @@ -224,6 +225,7 @@ private AclStatus(String owner, String group, boolean stickyBit, /** * Get the effective permission for the AclEntry * @param entry AclEntry to get the effective action + * @return FsAction. */ public FsAction getEffectivePermission(AclEntry entry) { return getEffectivePermission(entry, permission); diff --git a/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/fs/permission/FsAction.java b/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/fs/permission/FsAction.java index 97dcf816c16ad..746e0e1e238f4 100644 --- a/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/fs/permission/FsAction.java +++ b/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/fs/permission/FsAction.java @@ -48,7 +48,8 @@ private FsAction(String s) { /** * Return true if this action implies that action. - * @param that + * @param that FsAction that. + * @return if implies true,not false. */ public boolean implies(FsAction that) { if (that != null) { @@ -57,15 +58,26 @@ public boolean implies(FsAction that) { return false; } - /** AND operation. */ + /** + * AND operation. + * @param that FsAction that. + * @return FsAction. + */ public FsAction and(FsAction that) { return vals[ordinal() & that.ordinal()]; } - /** OR operation. */ + /** + * OR operation. + * @param that FsAction that. + * @return FsAction. + */ public FsAction or(FsAction that) { return vals[ordinal() | that.ordinal()]; } - /** NOT operation. */ + /** + * NOT operation. + * @return FsAction. + */ public FsAction not() { return vals[7 - ordinal()]; } diff --git a/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/fs/permission/FsCreateModes.java b/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/fs/permission/FsCreateModes.java index 2bd6f1f3b9126..ff3b4f6d65a49 100644 --- a/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/fs/permission/FsCreateModes.java +++ b/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/fs/permission/FsCreateModes.java @@ -35,7 +35,10 @@ public final class FsCreateModes extends FsPermission { /** * Create from unmasked mode and umask. * - * If the mode is already an FsCreateModes object, return it. + * @param mode mode. + * @param umask umask. + * @return If the mode is already + * an FsCreateModes object, return it. */ public static FsPermission applyUMask(FsPermission mode, FsPermission umask) { @@ -47,6 +50,10 @@ public static FsPermission applyUMask(FsPermission mode, /** * Create from masked and unmasked modes. + * + * @param masked masked. + * @param unmasked unmasked. + * @return FsCreateModes. */ public static FsCreateModes create(FsPermission masked, FsPermission unmasked) { diff --git a/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/fs/permission/FsPermission.java b/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/fs/permission/FsPermission.java index 51c113af2702e..33fed1d303990 100644 --- a/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/fs/permission/FsPermission.java +++ b/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/fs/permission/FsPermission.java @@ -56,7 +56,11 @@ public class FsPermission implements Writable, Serializable, /** Maximum acceptable length of a permission string to parse */ public static final int MAX_PERMISSION_LENGTH = 10; - /** Create an immutable {@link FsPermission} object. */ + /** + * Create an immutable {@link FsPermission} object. + * @param permission permission. + * @return FsPermission. + */ public static FsPermission createImmutable(short permission) { return new ImmutableFsPermission(permission); } @@ -85,7 +89,7 @@ public FsPermission(FsAction u, FsAction g, FsAction o, boolean sb) { /** * Construct by the given mode. - * @param mode + * @param mode mode. * @see #toShort() */ public FsPermission(short mode) { fromShort(mode); } @@ -145,13 +149,19 @@ public FsPermission(String mode) { this(new RawParser(mode).getPermission()); } - /** Return user {@link FsAction}. */ + /** + * @return Return user {@link FsAction}. + */ public FsAction getUserAction() {return useraction;} - /** Return group {@link FsAction}. */ + /** + * @return Return group {@link FsAction}. + */ public FsAction getGroupAction() {return groupaction;} - /** Return other {@link FsAction}. */ + /** + * @return Return other {@link FsAction}. + */ public FsAction getOtherAction() {return otheraction;} private void set(FsAction u, FsAction g, FsAction o, boolean sb) { @@ -180,6 +190,7 @@ public void readFields(DataInput in) throws IOException { /** * Get masked permission if exists. + * @return masked. */ public FsPermission getMasked() { return null; @@ -187,6 +198,7 @@ public FsPermission getMasked() { /** * Get unmasked permission if exists. + * @return unmasked. */ public FsPermission getUnmasked() { return null; @@ -194,6 +206,10 @@ public FsPermission getUnmasked() { /** * Create and initialize a {@link FsPermission} from {@link DataInput}. + * + * @param in data input. + * @throws IOException raised on errors performing I/O. + * @return FsPermission. */ public static FsPermission read(DataInput in) throws IOException { FsPermission p = new FsPermission(); @@ -203,6 +219,7 @@ public static FsPermission read(DataInput in) throws IOException { /** * Encode the object to a short. + * @return object to a short. */ public short toShort() { int s = (stickyBit ? 1 << 9 : 0) | @@ -301,6 +318,9 @@ public FsPermission applyUMask(FsPermission umask) { * '-' sets bits in the mask. * * Octal umask, the specified bits are set in the file mode creation mask. + * + * @param conf configuration. + * @return FsPermission UMask. */ public static FsPermission getUMask(Configuration conf) { int umask = DEFAULT_UMASK; @@ -346,7 +366,11 @@ public boolean getAclBit() { } /** - * Returns true if the file is encrypted or directory is in an encryption zone + * Returns true if the file is encrypted or directory is in an encryption zone. + * + * @return if the file is encrypted or directory + * is in an encryption zone true, not false. + * * @deprecated Get encryption bit from the * {@link org.apache.hadoop.fs.FileStatus} object. */ @@ -357,6 +381,9 @@ public boolean getEncryptedBit() { /** * Returns true if the file or directory is erasure coded. + * + * @return if the file or directory is + * erasure coded true, not false. * @deprecated Get ec bit from the {@link org.apache.hadoop.fs.FileStatus} * object. */ @@ -365,7 +392,11 @@ public boolean getErasureCodedBit() { return false; } - /** Set the user file creation mask (umask) */ + /** + * Set the user file creation mask (umask) + * @param conf configuration. + * @param umask umask. + */ public static void setUMask(Configuration conf, FsPermission umask) { conf.set(UMASK_LABEL, String.format("%1$03o", umask.toShort())); } @@ -379,6 +410,8 @@ public static void setUMask(Configuration conf, FsPermission umask) { * {@link FsPermission#getDirDefault()} for directory, and use * {@link FsPermission#getFileDefault()} for file. * This method is kept for compatibility. + * + * @return Default FsPermission. */ public static FsPermission getDefault() { return new FsPermission((short)00777); @@ -386,6 +419,8 @@ public static FsPermission getDefault() { /** * Get the default permission for directory. + * + * @return DirDefault FsPermission. */ public static FsPermission getDirDefault() { return new FsPermission((short)00777); @@ -393,6 +428,8 @@ public static FsPermission getDirDefault() { /** * Get the default permission for file. + * + * @return FileDefault FsPermission. */ public static FsPermission getFileDefault() { return new FsPermission((short)00666); @@ -400,6 +437,8 @@ public static FsPermission getFileDefault() { /** * Get the default permission for cache pools. + * + * @return CachePoolDefault FsPermission. */ public static FsPermission getCachePoolDefault() { return new FsPermission((short)00755); @@ -408,6 +447,7 @@ public static FsPermission getCachePoolDefault() { /** * Create a FsPermission from a Unix symbolic permission string * @param unixSymbolicPermission e.g. "-rw-rw-rw-" + * @return FsPermission. */ public static FsPermission valueOf(String unixSymbolicPermission) { if (unixSymbolicPermission == null) { diff --git a/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/fs/permission/PermissionStatus.java b/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/fs/permission/PermissionStatus.java index 3c3693f613baf..be4beb506a63f 100644 --- a/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/fs/permission/PermissionStatus.java +++ b/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/fs/permission/PermissionStatus.java @@ -39,7 +39,13 @@ public class PermissionStatus implements Writable { WritableFactories.setFactory(PermissionStatus.class, FACTORY); } - /** Create an immutable {@link PermissionStatus} object. */ + /** + * Create an immutable {@link PermissionStatus} object. + * @param user user. + * @param group group. + * @param permission permission. + * @return PermissionStatus. + */ public static PermissionStatus createImmutable( String user, String group, FsPermission permission) { return new PermissionStatus(user, group, permission) { @@ -56,20 +62,35 @@ public void readFields(DataInput in) throws IOException { private PermissionStatus() {} - /** Constructor */ + /** + * Constructor. + * + * @param user user. + * @param group group. + * @param permission permission. + */ public PermissionStatus(String user, String group, FsPermission permission) { username = user; groupname = group; this.permission = permission; } - /** Return user name */ + /** + * Return user name. + * @return user name. + */ public String getUserName() {return username;} - /** Return group name */ + /** + * Return group name. + * @return group name. + */ public String getGroupName() {return groupname;} - /** Return permission */ + /** + * Return permission. + * @return FsPermission. + */ public FsPermission getPermission() {return permission;} @Override @@ -86,6 +107,9 @@ public void write(DataOutput out) throws IOException { /** * Create and initialize a {@link PermissionStatus} from {@link DataInput}. + * @param in data input. + * @throws IOException raised on errors performing I/O. + * @return PermissionStatus. */ public static PermissionStatus read(DataInput in) throws IOException { PermissionStatus p = new PermissionStatus(); @@ -95,6 +119,11 @@ public static PermissionStatus read(DataInput in) throws IOException { /** * Serialize a {@link PermissionStatus} from its base components. + * @param out out. + * @param username username. + * @param groupname groupname. + * @param permission FsPermission. + * @throws IOException raised on errors performing I/O. */ public static void write(DataOutput out, String username, diff --git a/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/fs/sftp/SFTPConnectionPool.java b/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/fs/sftp/SFTPConnectionPool.java index de86bab6d3324..eace6417dcd68 100644 --- a/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/fs/sftp/SFTPConnectionPool.java +++ b/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/fs/sftp/SFTPConnectionPool.java @@ -76,7 +76,7 @@ synchronized void returnToPool(ChannelSftp channel) { ConnectionInfo info = con2infoMap.get(channel); HashSet cons = idleConnections.get(info); if (cons == null) { - cons = new HashSet(); + cons = new HashSet<>(); idleConnections.put(info, cons); } cons.add(channel); @@ -94,7 +94,7 @@ synchronized void shutdown() { Set cons = con2infoMap.keySet(); if (cons != null && cons.size() > 0) { // make a copy since we need to modify the underlying Map - Set copy = new HashSet(cons); + Set copy = new HashSet<>(cons); // Initiate disconnect from all outstanding connections for (ChannelSftp con : copy) { try { diff --git a/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/fs/sftp/SFTPFileSystem.java b/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/fs/sftp/SFTPFileSystem.java index 46b829f14a15c..3260a0db51479 100644 --- a/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/fs/sftp/SFTPFileSystem.java +++ b/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/fs/sftp/SFTPFileSystem.java @@ -709,11 +709,11 @@ public FileStatus getFileStatus(Path f) throws IOException { @Override public void close() throws IOException { - if (closed.getAndSet(true)) { - return; - } try { super.close(); + if (closed.getAndSet(true)) { + return; + } } finally { if (connectionPool != null) { connectionPool.shutdown(); diff --git a/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/fs/shell/Command.java b/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/fs/shell/Command.java index 0bdb47730a929..7858238ee71fd 100644 --- a/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/fs/shell/Command.java +++ b/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/fs/shell/Command.java @@ -77,7 +77,11 @@ protected Command() { err = System.err; } - /** Constructor */ + /** + * Constructor. + * + * @param conf configuration. + */ protected Command(Configuration conf) { super(conf); } @@ -109,7 +113,7 @@ protected int getDepth() { * Execute the command on the input path data. Commands can override to make * use of the resolved filesystem. * @param pathData The input path with resolved filesystem - * @throws IOException + * @throws IOException raised on errors performing I/O. */ protected void run(PathData pathData) throws IOException { run(pathData.path); @@ -136,11 +140,19 @@ public int runAll() { return exitCode; } - /** sets the command factory for later use */ + /** + * sets the command factory for later use. + * @param factory factory. + */ public void setCommandFactory(CommandFactory factory) { this.commandFactory = factory; } - /** retrieves the command factory */ + + /** + * retrieves the command factory. + * + * @return command factory. + */ protected CommandFactory getCommandFactory() { return this.commandFactory; } @@ -201,7 +213,7 @@ public int run(String...argv) { * IllegalArgumentException is thrown, the FsShell object will print the * short usage of the command. * @param args the command line arguments - * @throws IOException + * @throws IOException raised on errors performing I/O. */ protected void processOptions(LinkedList args) throws IOException {} @@ -211,7 +223,7 @@ protected void processOptions(LinkedList args) throws IOException {} * {@link #expandArguments(LinkedList)} and pass the resulting list to * {@link #processArguments(LinkedList)} * @param args the list of argument strings - * @throws IOException + * @throws IOException raised on errors performing I/O. */ protected void processRawArguments(LinkedList args) throws IOException { diff --git a/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/fs/shell/CommandFormat.java b/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/fs/shell/CommandFormat.java index 4dd20d108428e..1228f76d846ab 100644 --- a/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/fs/shell/CommandFormat.java +++ b/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/fs/shell/CommandFormat.java @@ -165,7 +165,7 @@ public String getOptValue(String option) { * @return Set{@literal <}String{@literal >} of the enabled options */ public Set getOpts() { - Set optSet = new HashSet(); + Set optSet = new HashSet<>(); for (Map.Entry entry : options.entrySet()) { if (entry.getValue()) { optSet.add(entry.getKey()); diff --git a/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/fs/shell/CommandWithDestination.java b/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/fs/shell/CommandWithDestination.java index 678225f81e0e3..69a418c1925eb 100644 --- a/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/fs/shell/CommandWithDestination.java +++ b/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/fs/shell/CommandWithDestination.java @@ -119,6 +119,8 @@ protected void setDirectWrite(boolean flag) { * owner, group and permission information of the source * file will be preserved as far as target {@link FileSystem} * implementation allows. + * + * @param preserve preserve. */ protected void setPreserve(boolean preserve) { if (preserve) { @@ -175,6 +177,7 @@ protected void preserve(FileAttribute fileAttribute) { * The last arg is expected to be a local path, if only one argument is * given then the destination will be the current directory * @param args is the list of arguments + * @throws IOException raised on errors performing I/O. */ protected void getLocalDestination(LinkedList args) throws IOException { diff --git a/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/fs/shell/PathData.java b/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/fs/shell/PathData.java index 2071a16799a5c..da99ac212563e 100644 --- a/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/fs/shell/PathData.java +++ b/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/fs/shell/PathData.java @@ -610,10 +610,11 @@ public int hashCode() { /** * Open a file for sequential IO. - *

    + *

    * This uses FileSystem.openFile() to request sequential IO; * the file status is also passed in. * Filesystems may use to optimize their IO. + *

    * @return an input stream * @throws IOException failure */ diff --git a/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/fs/shell/find/BaseExpression.java b/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/fs/shell/find/BaseExpression.java index 5069d2d34e51c..cd9bbe2bc884e 100644 --- a/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/fs/shell/find/BaseExpression.java +++ b/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/fs/shell/find/BaseExpression.java @@ -38,12 +38,18 @@ public abstract class BaseExpression implements Expression, Configurable { private String[] usage = { "Not yet implemented" }; private String[] help = { "Not yet implemented" }; - /** Sets the usage text for this {@link Expression} */ + /** + * Sets the usage text for this {@link Expression} . + * @param usage usage array. + */ protected void setUsage(String[] usage) { this.usage = usage; } - /** Sets the help text for this {@link Expression} */ + /** + * Sets the help text for this {@link Expression} . + * @param help help. + */ protected void setHelp(String[] help) { this.help = help; } @@ -92,7 +98,10 @@ public void finish() throws IOException { /** Children of this expression. */ private LinkedList children = new LinkedList(); - /** Return the options to be used by this expression. */ + /** + * Return the options to be used by this expression. + * @return options. + */ protected FindOptions getOptions() { return (this.options == null) ? new FindOptions() : this.options; } @@ -265,6 +274,7 @@ protected void addArgument(String arg) { * @param depth * current depth in the process directories * @return FileStatus + * @throws IOException raised on errors performing I/O. */ protected FileStatus getFileStatus(PathData item, int depth) throws IOException { @@ -285,6 +295,8 @@ protected FileStatus getFileStatus(PathData item, int depth) * @param item * PathData * @return Path + * + * @throws IOException raised on errors performing I/O. */ protected Path getPath(PathData item) throws IOException { return item.path; @@ -295,6 +307,7 @@ protected Path getPath(PathData item) throws IOException { * * @param item PathData * @return FileSystem + * @throws IOException raised on errors performing I/O. */ protected FileSystem getFileSystem(PathData item) throws IOException { return item.fs; diff --git a/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/fs/shell/find/Expression.java b/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/fs/shell/find/Expression.java index ccad631028cc9..353fe685cc9cd 100644 --- a/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/fs/shell/find/Expression.java +++ b/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/fs/shell/find/Expression.java @@ -30,13 +30,15 @@ public interface Expression { /** * Set the options for this expression, called once before processing any * items. + * @param options options. + * @throws IOException raised on errors performing I/O. */ public void setOptions(FindOptions options) throws IOException; /** * Prepares the expression for execution, called once after setting options * and before processing any options. - * @throws IOException + * @throws IOException raised on errors performing I/O. */ public void prepare() throws IOException; @@ -46,13 +48,14 @@ public interface Expression { * @param item {@link PathData} item to be processed * @param depth distance of the item from the command line argument * @return {@link Result} of applying the expression to the item + * @throws IOException raised on errors performing I/O. */ public Result apply(PathData item, int depth) throws IOException; /** * Finishes the expression, called once after processing all items. * - * @throws IOException + * @throws IOException raised on errors performing I/O. */ public void finish() throws IOException; @@ -76,15 +79,21 @@ public interface Expression { /** * Indicates whether this expression performs an action, i.e. provides output * back to the user. + * @return if is action true, not false. */ public boolean isAction(); - /** Identifies the expression as an operator rather than a primary. */ + /** + * Identifies the expression as an operator rather than a primary. + * @return if is operator true, not false. + */ public boolean isOperator(); /** * Returns the precedence of this expression * (only applicable to operators). + * + * @return precedence. */ public int getPrecedence(); diff --git a/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/fs/shell/find/Find.java b/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/fs/shell/find/Find.java index 199038a751226..07baea89dd604 100644 --- a/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/fs/shell/find/Find.java +++ b/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/fs/shell/find/Find.java @@ -96,7 +96,7 @@ private static void addExpression(Class clazz) { private Expression rootExpression; /** Set of path items returning a {@link Result#STOP} result. */ - private HashSet stopPaths = new HashSet(); + private HashSet stopPaths = new HashSet<>(); /** Register the expressions with the expression factory. */ private static void registerExpressions(ExpressionFactory factory) { diff --git a/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/fs/shell/find/FindOptions.java b/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/fs/shell/find/FindOptions.java index b0f1be5c35c93..c605186230590 100644 --- a/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/fs/shell/find/FindOptions.java +++ b/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/fs/shell/find/FindOptions.java @@ -264,6 +264,7 @@ public void setConfiguration(Configuration configuration) { /** * Return the {@link Configuration} return configuration {@link Configuration} + * @return configuration. */ public Configuration getConfiguration() { return this.configuration; diff --git a/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/fs/shell/find/Result.java b/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/fs/shell/find/Result.java index 2ef9cb4a801d6..a242681acd030 100644 --- a/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/fs/shell/find/Result.java +++ b/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/fs/shell/find/Result.java @@ -35,23 +35,36 @@ private Result(boolean success, boolean recurse) { this.descend = recurse; } - /** Should further directories be descended. */ + /** + * Should further directories be descended. + * @return if is pass true,not false. + */ public boolean isDescend() { return this.descend; } - /** Should processing continue. */ + /** + * Should processing continue. + * @return if is pass true,not false. + */ public boolean isPass() { return this.success; } - /** Returns the combination of this and another result. */ + /** + * Returns the combination of this and another result. + * @param other other. + * @return result. + */ public Result combine(Result other) { return new Result(this.isPass() && other.isPass(), this.isDescend() && other.isDescend()); } - /** Negate this result. */ + /** + * Negate this result. + * @return Result. + */ public Result negate() { return new Result(!this.isPass(), this.isDescend()); } diff --git a/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/fs/statistics/IOStatisticsSnapshot.java b/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/fs/statistics/IOStatisticsSnapshot.java index 63d37e97c98b8..88606eb4b3055 100644 --- a/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/fs/statistics/IOStatisticsSnapshot.java +++ b/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/fs/statistics/IOStatisticsSnapshot.java @@ -53,7 +53,7 @@ * deserialized. If for some reason this is required, use * {@link #requiredSerializationClasses()} to get the list of classes * used when deserializing instances of this object. - *

    + *

    *

    * It is annotated for correct serializations with jackson2. *

    @@ -238,6 +238,8 @@ public static JsonSerialization serializer() { /** * Serialize by converting each map to a TreeMap, and saving that * to the stream. + * @param s ObjectOutputStream. + * @throws IOException raised on errors performing I/O. */ private synchronized void writeObject(ObjectOutputStream s) throws IOException { @@ -253,6 +255,10 @@ private synchronized void writeObject(ObjectOutputStream s) /** * Deserialize by loading each TreeMap, and building concurrent * hash maps from them. + * + * @param s ObjectInputStream. + * @throws IOException raised on errors performing I/O. + * @throws ClassNotFoundException class not found exception */ private void readObject(final ObjectInputStream s) throws IOException, ClassNotFoundException { diff --git a/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/fs/statistics/IOStatisticsSupport.java b/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/fs/statistics/IOStatisticsSupport.java index 75977047c0f2a..bb4d9a44587a2 100644 --- a/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/fs/statistics/IOStatisticsSupport.java +++ b/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/fs/statistics/IOStatisticsSupport.java @@ -71,6 +71,7 @@ private IOStatisticsSupport() { * Returns null if the source isn't of the write type * or the return value of * {@link IOStatisticsSource#getIOStatistics()} was null. + * @param source source. * @return an IOStatistics instance or null */ diff --git a/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/fs/statistics/MeanStatistic.java b/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/fs/statistics/MeanStatistic.java index d9ff0c25c6a21..369db49654382 100644 --- a/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/fs/statistics/MeanStatistic.java +++ b/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/fs/statistics/MeanStatistic.java @@ -207,6 +207,7 @@ public synchronized double mean() { /** * Add another MeanStatistic. * @param other other value + * @return mean statistic. */ public synchronized MeanStatistic add(final MeanStatistic other) { if (other.isEmpty()) { diff --git a/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/fs/statistics/StoreStatisticNames.java b/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/fs/statistics/StoreStatisticNames.java index c458269c3510d..c04c1bb47fcea 100644 --- a/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/fs/statistics/StoreStatisticNames.java +++ b/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/fs/statistics/StoreStatisticNames.java @@ -53,6 +53,9 @@ public final class StoreStatisticNames { /** {@value}. */ public static final String OP_CREATE = "op_create"; + /** {@value}. */ + public static final String OP_CREATE_FILE = "op_createfile"; + /** {@value}. */ public static final String OP_CREATE_NON_RECURSIVE = "op_create_non_recursive"; diff --git a/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/fs/statistics/impl/IOStatisticsBinding.java b/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/fs/statistics/impl/IOStatisticsBinding.java index c45dfc21a1b1d..6a5d01fb3b074 100644 --- a/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/fs/statistics/impl/IOStatisticsBinding.java +++ b/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/fs/statistics/impl/IOStatisticsBinding.java @@ -141,6 +141,7 @@ public static String entryToString( /** * Convert entry values to the string format used in logging. * + * @param type of values. * @param name statistic name * @param value stat value * @return formatted string @@ -178,6 +179,8 @@ private static Map copyMap( /** * A passthrough copy operation suitable for immutable * types, including numbers. + * + * @param type of values. * @param src source object * @return the source object */ @@ -437,6 +440,7 @@ public static Function trackJavaFunctionDuration( * @param input input callable. * @param return type. * @return the result of the operation. + * @throws IOException raised on errors performing I/O. */ public static B trackDuration( DurationTrackerFactory factory, diff --git a/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/fs/store/DataBlocks.java b/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/fs/store/DataBlocks.java index d9d3850ef4e2e..c70d0ee91e15e 100644 --- a/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/fs/store/DataBlocks.java +++ b/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/fs/store/DataBlocks.java @@ -107,6 +107,7 @@ private DataBlocks() { * @param len number of bytes to be written. * @throws NullPointerException for a null buffer * @throws IndexOutOfBoundsException if indices are out of range + * @throws IOException raised on errors performing I/O. */ public static void validateWriteArgs(byte[] b, int off, int len) throws IOException { @@ -287,6 +288,7 @@ protected BlockFactory(String keyToBufferDir, Configuration conf) { * @param limit limit of the block. * @param statistics stats to work with * @return a new block. + * @throws IOException raised on errors performing I/O. */ public abstract DataBlock create(long index, int limit, BlockUploadStatistics statistics) @@ -482,6 +484,8 @@ public void close() throws IOException { /** * Inner close logic for subclasses to implement. + * + * @throws IOException raised on errors performing I/O. */ protected void innerClose() throws IOException { diff --git a/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/fs/store/audit/AuditingFunctions.java b/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/fs/store/audit/AuditingFunctions.java index acc82766be190..9ad727f5a2df2 100644 --- a/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/fs/store/audit/AuditingFunctions.java +++ b/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/fs/store/audit/AuditingFunctions.java @@ -86,6 +86,8 @@ public static InvocationRaisingIOE withinAuditSpan( * activates and deactivates the span around the inner one. * @param auditSpan audit span * @param operation operation + * @param Generics Type T. + * @param Generics Type R. * @return a new invocation. */ public static FunctionRaisingIOE withinAuditSpan( diff --git a/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/fs/viewfs/ConfigUtil.java b/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/fs/viewfs/ConfigUtil.java index 09ec5d2933058..c9ee5e232d944 100644 --- a/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/fs/viewfs/ConfigUtil.java +++ b/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/fs/viewfs/ConfigUtil.java @@ -48,7 +48,7 @@ public static String getConfigViewFsPrefix() { /** * Add a link to the config for the specified mount table * @param conf - add the link to this conf - * @param mountTableName + * @param mountTableName mountTable. * @param src - the src path name * @param target - the target URI link */ @@ -71,9 +71,10 @@ public static void addLink(final Configuration conf, final String src, /** * Add a LinkMergeSlash to the config for the specified mount table. - * @param conf - * @param mountTableName - * @param target + * + * @param conf configuration. + * @param mountTableName mountTable. + * @param target target. */ public static void addLinkMergeSlash(Configuration conf, final String mountTableName, final URI target) { @@ -83,8 +84,9 @@ public static void addLinkMergeSlash(Configuration conf, /** * Add a LinkMergeSlash to the config for the default mount table. - * @param conf - * @param target + * + * @param conf configuration. + * @param target targets. */ public static void addLinkMergeSlash(Configuration conf, final URI target) { addLinkMergeSlash(conf, getDefaultMountTableName(conf), target); @@ -92,9 +94,10 @@ public static void addLinkMergeSlash(Configuration conf, final URI target) { /** * Add a LinkFallback to the config for the specified mount table. - * @param conf - * @param mountTableName - * @param target + * + * @param conf configuration. + * @param mountTableName mountTable. + * @param target targets. */ public static void addLinkFallback(Configuration conf, final String mountTableName, final URI target) { @@ -104,8 +107,9 @@ public static void addLinkFallback(Configuration conf, /** * Add a LinkFallback to the config for the default mount table. - * @param conf - * @param target + * + * @param conf configuration. + * @param target targets. */ public static void addLinkFallback(Configuration conf, final URI target) { addLinkFallback(conf, getDefaultMountTableName(conf), target); @@ -113,9 +117,10 @@ public static void addLinkFallback(Configuration conf, final URI target) { /** * Add a LinkMerge to the config for the specified mount table. - * @param conf - * @param mountTableName - * @param targets + * + * @param conf configuration. + * @param mountTableName mountTable. + * @param targets targets. */ public static void addLinkMerge(Configuration conf, final String mountTableName, final URI[] targets) { @@ -125,8 +130,9 @@ public static void addLinkMerge(Configuration conf, /** * Add a LinkMerge to the config for the default mount table. - * @param conf - * @param targets + * + * @param conf configuration. + * @param targets targets array. */ public static void addLinkMerge(Configuration conf, final URI[] targets) { addLinkMerge(conf, getDefaultMountTableName(conf), targets); @@ -134,6 +140,12 @@ public static void addLinkMerge(Configuration conf, final URI[] targets) { /** * Add nfly link to configuration for the given mount table. + * + * @param conf configuration. + * @param mountTableName mount table. + * @param src src. + * @param settings settings. + * @param targets targets. */ public static void addLinkNfly(Configuration conf, String mountTableName, String src, String settings, final String targets) { @@ -144,12 +156,13 @@ public static void addLinkNfly(Configuration conf, String mountTableName, } /** + * Add nfly link to configuration for the given mount table. * - * @param conf - * @param mountTableName - * @param src - * @param settings - * @param targets + * @param conf configuration. + * @param mountTableName mount table. + * @param src src. + * @param settings settings. + * @param targets targets. */ public static void addLinkNfly(Configuration conf, String mountTableName, String src, String settings, final URI ... targets) { @@ -202,6 +215,7 @@ public static void setHomeDirConf(final Configuration conf, * Add config variable for homedir the specified mount table * @param conf - add to this conf * @param homedir - the home dir path starting with slash + * @param mountTableName - the mount table. */ public static void setHomeDirConf(final Configuration conf, final String mountTableName, final String homedir) { @@ -247,4 +261,22 @@ public static String getDefaultMountTableName(final Configuration conf) { return conf.get(Constants.CONFIG_VIEWFS_DEFAULT_MOUNT_TABLE_NAME_KEY, Constants.CONFIG_VIEWFS_DEFAULT_MOUNT_TABLE); } + + /** + * Check the bool config whether nested mount point is supported. Default: true + * @param conf - from this conf + * @return whether nested mount point is supported + */ + public static boolean isNestedMountPointSupported(final Configuration conf) { + return conf.getBoolean(Constants.CONFIG_NESTED_MOUNT_POINT_SUPPORTED, true); + } + + /** + * Set the bool value isNestedMountPointSupported in config. + * @param conf - from this conf + * @param isNestedMountPointSupported - whether nested mount point is supported + */ + public static void setIsNestedMountPointSupported(final Configuration conf, boolean isNestedMountPointSupported) { + conf.setBoolean(Constants.CONFIG_NESTED_MOUNT_POINT_SUPPORTED, isNestedMountPointSupported); + } } diff --git a/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/fs/viewfs/Constants.java b/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/fs/viewfs/Constants.java index 21f4d99f891c2..806e69f32c6b7 100644 --- a/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/fs/viewfs/Constants.java +++ b/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/fs/viewfs/Constants.java @@ -35,7 +35,7 @@ public interface Constants { * Prefix for the config variable for the ViewFs mount-table path. */ String CONFIG_VIEWFS_MOUNTTABLE_PATH = CONFIG_VIEWFS_PREFIX + ".path"; - + /** * Prefix for the home dir for the mount table - if not specified * then the hadoop default value (/user) is used. @@ -53,12 +53,17 @@ public interface Constants { */ public static final String CONFIG_VIEWFS_DEFAULT_MOUNT_TABLE = "default"; + /** + * Config to enable nested mount point in viewfs + */ + String CONFIG_NESTED_MOUNT_POINT_SUPPORTED = CONFIG_VIEWFS_PREFIX + ".nested.mount.point.supported"; + /** * Config variable full prefix for the default mount table. */ - public static final String CONFIG_VIEWFS_PREFIX_DEFAULT_MOUNT_TABLE = + public static final String CONFIG_VIEWFS_PREFIX_DEFAULT_MOUNT_TABLE = CONFIG_VIEWFS_PREFIX + "." + CONFIG_VIEWFS_DEFAULT_MOUNT_TABLE; - + /** * Config variable for specifying a simple link */ @@ -82,7 +87,7 @@ public interface Constants { /** * Config variable for specifying a merge of the root of the mount-table - * with the root of another file system. + * with the root of another file system. */ String CONFIG_VIEWFS_LINK_MERGE_SLASH = "linkMergeSlash"; diff --git a/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/fs/viewfs/FsGetter.java b/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/fs/viewfs/FsGetter.java index c72baac25fb75..f723f238e199e 100644 --- a/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/fs/viewfs/FsGetter.java +++ b/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/fs/viewfs/FsGetter.java @@ -34,6 +34,10 @@ public class FsGetter { /** * Gets new file system instance of given uri. + * @param uri uri. + * @param conf configuration. + * @throws IOException raised on errors performing I/O. + * @return file system. */ public FileSystem getNewInstance(URI uri, Configuration conf) throws IOException { @@ -42,6 +46,11 @@ public FileSystem getNewInstance(URI uri, Configuration conf) /** * Gets file system instance of given uri. + * + * @param uri uri. + * @param conf configuration. + * @throws IOException raised on errors performing I/O. + * @return FileSystem. */ public FileSystem get(URI uri, Configuration conf) throws IOException { return FileSystem.get(uri, conf); diff --git a/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/fs/viewfs/InodeTree.java b/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/fs/viewfs/InodeTree.java index 23ad053a67d5c..5360d55e10644 100644 --- a/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/fs/viewfs/InodeTree.java +++ b/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/fs/viewfs/InodeTree.java @@ -17,6 +17,10 @@ */ package org.apache.hadoop.fs.viewfs; +import java.util.Collection; +import java.util.Comparator; +import java.util.Set; +import java.util.TreeSet; import java.util.function.Function; import org.apache.hadoop.util.Preconditions; import java.io.FileNotFoundException; @@ -55,7 +59,7 @@ * @param is AbstractFileSystem or FileSystem * * The two main methods are - * {@link #InodeTree(Configuration, String)} // constructor + * {@link #InodeTree(Configuration, String, URI, boolean)} // constructor * {@link #resolve(String, boolean)} */ @@ -81,6 +85,8 @@ enum ResultKind { private List> regexMountPointList = new ArrayList>(); + private final boolean isNestedMountPointSupported; + public static class MountPoint { String src; INodeLink target; @@ -99,7 +105,7 @@ public String getSource() { } /** - * Returns the target link. + * Returns the target INode link. * @return The target INode link */ public INodeLink getTarget() { @@ -138,6 +144,14 @@ public INode(String pathToNode, UserGroupInformation aUgi) { boolean isLink() { return !isInternalDir(); } + + /** + * Return the link if isLink. + * @return will return null, for non links. + */ + INodeLink getLink() { + return null; + } } /** @@ -212,6 +226,51 @@ void addLink(final String pathComponent, final INodeLink link) } children.put(pathComponent, link); } + + void addDirLink(final String pathComponent, final INodeDirLink dirLink) { + children.put(pathComponent, dirLink); + } + } + + /** + * Internal class to represent an INodeDir which also contains a INodeLink. This is used to support nested mount points + * where an INode is internalDir but points to a mount link. The class is a subclass of INodeDir and the semantics are + * as follows: + * isLink(): true + * isInternalDir(): true + * @param + */ + static class INodeDirLink extends INodeDir { + /** + * INodeLink wrapped in the INodeDir + */ + private final INodeLink link; + + INodeDirLink(String pathToNode, UserGroupInformation aUgi, INodeLink link) { + super(pathToNode, aUgi); + this.link = link; + } + + @Override + INodeLink getLink() { + return link; + } + + /** + * True because the INodeDirLink also contains a INodeLink + */ + @Override + boolean isLink() { + return true; + } + + /** + * True because the INodeDirLink is internal node + */ + @Override + boolean isInternalDir() { + return true; + } } /** @@ -266,8 +325,8 @@ enum LinkType { * A merge dir link is a merge (junction) of links to dirs: * example : merge of 2 dirs - * /users -> hdfs:nn1//users - * /users -> hdfs:nn2//users + * /users -> hdfs:nn1//users + * /users -> hdfs:nn2//users * * For a merge, each target is checked to be dir when created but if target * is changed later it is then ignored (a dir with null entries) @@ -305,6 +364,8 @@ public static class INodeLink extends INode { /** * Get the target of the link. If a merge link then it returned * as "," separated URI list. + * + * @return the path. */ public Path getTargetLink() { StringBuilder result = new StringBuilder(targetDirLinkList[0].toString()); @@ -320,10 +381,15 @@ boolean isInternalDir() { return false; } + @Override + INodeLink getLink() { + return this; + } + /** * Get the instance of FileSystem to use, creating one if needed. * @return An Initialized instance of T - * @throws IOException + * @throws IOException raised on errors performing I/O. */ public T getTargetFileSystem() throws IOException { if (targetFileSystem != null) { @@ -376,10 +442,17 @@ private void createLink(final String src, final String target, newDir.setInternalDirFs(getTargetFileSystem(newDir)); nextInode = newDir; } - if (nextInode.isLink()) { - // Error - expected a dir but got a link - throw new FileAlreadyExistsException("Path " + nextInode.fullPath + - " already exists as link"); + if (!nextInode.isInternalDir()) { + if (isNestedMountPointSupported) { + // nested mount detected, add a new INodeDirLink that wraps existing INodeLink to INodeTree and override existing INodelink + INodeDirLink dirLink = new INodeDirLink(nextInode.fullPath, aUgi, (INodeLink) nextInode); + curInode.addDirLink(iPath, dirLink); + curInode = dirLink; + } else { + // Error - expected a dir but got a link + throw new FileAlreadyExistsException("Path " + nextInode.fullPath + + " already exists as link"); + } } else { assert(nextInode.isInternalDir()); curInode = (INodeDir) nextInode; @@ -429,7 +502,7 @@ private void createLink(final String src, final String target, /** * The user of this class must subclass and implement the following * 3 abstract methods. - * @throws IOException + * @return Function. */ protected abstract Function initAndGetTargetFs(); @@ -445,7 +518,7 @@ private INodeDir getRootDir() { } private INodeLink getRootLink() { - Preconditions.checkState(root.isLink()); + Preconditions.checkState(!root.isInternalDir()); return (INodeLink)root; } @@ -520,14 +593,21 @@ Configuration getConfig() { } /** - * Create Inode Tree from the specified mount-table specified in Config - * @param config - the mount table keys are prefixed with - * FsConstants.CONFIG_VIEWFS_PREFIX - * @param viewName - the name of the mount table - if null use defaultMT name - * @throws UnsupportedFileSystemException - * @throws URISyntaxException - * @throws FileAlreadyExistsException - * @throws IOException + * Create Inode Tree from the specified mount-table specified in Config. + * + * @param config the mount table keys are prefixed with + * FsConstants.CONFIG_VIEWFS_PREFIX. + * @param viewName the name of the mount table + * if null use defaultMT name. + * @param theUri heUri. + * @param initingUriAsFallbackOnNoMounts initingUriAsFallbackOnNoMounts. + * @throws UnsupportedFileSystemException file system for uri is + * not found. + * @throws URISyntaxException if the URI does not have an authority + * it is badly formed. + * @throws FileAlreadyExistsException there is a file at the path specified + * or is discovered on one of its ancestors. + * @throws IOException raised on errors performing I/O. */ protected InodeTree(final Configuration config, final String viewName, final URI theUri, boolean initingUriAsFallbackOnNoMounts) @@ -538,6 +618,7 @@ protected InodeTree(final Configuration config, final String viewName, mountTableName = ConfigUtil.getDefaultMountTableName(config); } homedirPrefix = ConfigUtil.getHomeDirValue(config, mountTableName); + isNestedMountPointSupported = ConfigUtil.isNestedMountPointSupported(config); boolean isMergeSlashConfigured = false; String mergeSlashTarget = null; @@ -642,7 +723,8 @@ protected InodeTree(final Configuration config, final String viewName, getRootDir().setInternalDirFs(getTargetFileSystem(getRootDir())); getRootDir().setRoot(true); INodeLink fallbackLink = null; - for (LinkEntry le : linkEntries) { + + for (LinkEntry le : getLinkEntries(linkEntries)) { switch (le.getLinkType()) { case SINGLE_FALLBACK: if (fallbackLink != null) { @@ -682,6 +764,32 @@ protected InodeTree(final Configuration config, final String viewName, } } + /** + * Get collection of linkEntry. Sort mount point based on alphabetical order of the src paths. + * The purpose is to group nested paths(shortest path always comes first) during INodeTree creation. + * E.g. /foo is nested with /foo/bar so an INodeDirLink will be created at /foo. + * @param linkEntries input linkEntries + * @return sorted linkEntries + */ + private Collection getLinkEntries(List linkEntries) { + Set sortedLinkEntries = new TreeSet<>(new Comparator() { + @Override + public int compare(LinkEntry o1, LinkEntry o2) { + if (o1 == null) { + return -1; + } + if (o2 == null) { + return 1; + } + String src1 = o1.getSrc(); + String src2= o2.getSrc(); + return src1.compareTo(src2); + } + }); + sortedLinkEntries.addAll(linkEntries); + return sortedLinkEntries; + } + private void checkMntEntryKeyEqualsTarget( String mntEntryKey, String targetMntEntryKey) throws IOException { if (!mntEntryKey.equals(targetMntEntryKey)) { @@ -773,9 +881,9 @@ boolean isLastInternalDirLink() { /** * Resolve the pathname p relative to root InodeDir. * @param p - input path - * @param resolveLastComponent + * @param resolveLastComponent resolveLastComponent. * @return ResolveResult which allows further resolution of the remaining path - * @throws IOException + * @throws IOException raised on errors performing I/O. */ public ResolveResult resolve(final String p, final boolean resolveLastComponent) throws IOException { @@ -795,7 +903,7 @@ public ResolveResult resolve(final String p, final boolean resolveLastCompone * been linked to the root directory of a file system. * The first non-slash path component should be name of the mount table. */ - if (root.isLink()) { + if (!root.isInternalDir()) { Path remainingPath; StringBuilder remainingPathStr = new StringBuilder(); // ignore first slash @@ -818,10 +926,17 @@ public ResolveResult resolve(final String p, final boolean resolveLastCompone } int i; + INodeDirLink lastResolvedDirLink = null; + int lastResolvedDirLinkIndex = -1; // ignore first slash for (i = 1; i < path.length - (resolveLastComponent ? 0 : 1); i++) { INode nextInode = curInode.resolveInternal(path[i]); if (nextInode == null) { + // first resolve to dirlink for nested mount point + if (isNestedMountPointSupported && lastResolvedDirLink != null) { + return new ResolveResult(ResultKind.EXTERNAL_DIR, lastResolvedDirLink.getLink().getTargetFileSystem(), + lastResolvedDirLink.fullPath, getRemainingPath(path, i),true); + } if (hasFallbackLink()) { resolveResult = new ResolveResult(ResultKind.EXTERNAL_DIR, getRootFallbackLink().getTargetFileSystem(), root.fullPath, @@ -837,59 +952,67 @@ public ResolveResult resolve(final String p, final boolean resolveLastCompone } } - if (nextInode.isLink()) { + if (!nextInode.isInternalDir()) { final INodeLink link = (INodeLink) nextInode; - final Path remainingPath; - if (i >= path.length - 1) { - remainingPath = SlashPath; - } else { - StringBuilder remainingPathStr = - new StringBuilder("/" + path[i + 1]); - for (int j = i + 2; j < path.length; ++j) { - remainingPathStr.append('/').append(path[j]); - } - remainingPath = new Path(remainingPathStr.toString()); - } + final Path remainingPath = getRemainingPath(path, i + 1); resolveResult = new ResolveResult(ResultKind.EXTERNAL_DIR, link.getTargetFileSystem(), nextInode.fullPath, remainingPath, true); return resolveResult; - } else if (nextInode.isInternalDir()) { + } else { curInode = (INodeDir) nextInode; + // track last resolved nest mount point. + if (isNestedMountPointSupported && nextInode.isLink()) { + lastResolvedDirLink = (INodeDirLink) nextInode; + lastResolvedDirLinkIndex = i; + } } } - // We have resolved to an internal dir in mount table. Path remainingPath; - if (resolveLastComponent) { + if (isNestedMountPointSupported && lastResolvedDirLink != null) { + remainingPath = getRemainingPath(path, lastResolvedDirLinkIndex + 1); + resolveResult = new ResolveResult(ResultKind.EXTERNAL_DIR, lastResolvedDirLink.getLink().getTargetFileSystem(), + lastResolvedDirLink.fullPath, remainingPath,true); + } else { + remainingPath = resolveLastComponent ? SlashPath : getRemainingPath(path, i); + resolveResult = new ResolveResult(ResultKind.INTERNAL_DIR, curInode.getInternalDirFs(), + curInode.fullPath, remainingPath, false); + } + return resolveResult; + } + + /** + * Return remaining path from specified index to the end of the path array. + * @param path An array of path components split by slash + * @param startIndex the specified start index of the path array + * @return remaining path. + */ + private Path getRemainingPath(String[] path, int startIndex) { + Path remainingPath; + if (startIndex >= path.length) { remainingPath = SlashPath; } else { - // note we have taken care of when path is "/" above - // for internal dirs rem-path does not start with / since the lookup - // that follows will do a children.get(remaningPath) and will have to - // strip-out the initial / - StringBuilder remainingPathStr = new StringBuilder("/" + path[i]); - for (int j = i + 1; j < path.length; ++j) { - remainingPathStr.append('/').append(path[j]); + StringBuilder remainingPathStr = new StringBuilder(); + for (int j = startIndex; j < path.length; j++) { + remainingPathStr.append("/").append(path[j]); } remainingPath = new Path(remainingPathStr.toString()); } - resolveResult = new ResolveResult(ResultKind.INTERNAL_DIR, - curInode.getInternalDirFs(), curInode.fullPath, remainingPath, false); - return resolveResult; + return remainingPath; } /** * Walk through all regex mount points to see * whether the path match any regex expressions. - * E.g. link: ^/user/(?\\w+) => s3://$user.apache.com/_${user} + * E.g. link: ^/user/(?<username>\\w+) => s3://$user.apache.com/_${user} * srcPath: is /user/hadoop/dir1 * resolveLastComponent: true * then return value is s3://hadoop.apache.com/_hadoop * - * @param srcPath - * @param resolveLastComponent - * @return + * @param srcPath srcPath. + * @param resolveLastComponent resolveLastComponent. + * @return ResolveResult. */ protected ResolveResult tryResolveInRegexMountpoint(final String srcPath, final boolean resolveLastComponent) { @@ -907,7 +1030,7 @@ protected ResolveResult tryResolveInRegexMountpoint(final String srcPath, * Build resolve result. * Here's an example * Mountpoint: fs.viewfs.mounttable.mt - * .linkRegex.replaceresolveddstpath:_:-#.^/user/(?\w+) + * .linkRegex.replaceresolveddstpath:_:-#.^/user/(??<username>\w+) * Value: /targetTestRoot/$username * Dir path to test: * viewfs://mt/user/hadoop_user1/hadoop_dir1 @@ -916,6 +1039,10 @@ protected ResolveResult tryResolveInRegexMountpoint(final String srcPath, * targetOfResolvedPathStr: /targetTestRoot/hadoop-user1 * remainingPath: /hadoop_dir1 * + * @param resultKind resultKind. + * @param resolvedPathStr resolvedPathStr. + * @param targetOfResolvedPathStr targetOfResolvedPathStr. + * @param remainingPath remainingPath. * @return targetFileSystem or null on exceptions. */ protected ResolveResult buildResolveResultForRegexMountPoint( diff --git a/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/fs/viewfs/MountTableConfigLoader.java b/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/fs/viewfs/MountTableConfigLoader.java index bc2c3ea93c58c..5fcd77cd29155 100644 --- a/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/fs/viewfs/MountTableConfigLoader.java +++ b/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/fs/viewfs/MountTableConfigLoader.java @@ -38,6 +38,7 @@ public interface MountTableConfigLoader { * a directory in the case of multiple versions of mount-table * files(Recommended option). * @param conf - Configuration object to add mount table. + * @throws IOException raised on errors performing I/O. */ void load(String mountTableConfigPath, Configuration conf) throws IOException; diff --git a/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/fs/viewfs/ViewFileSystem.java b/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/fs/viewfs/ViewFileSystem.java index b30e086cf06b0..e31a701a6eaa7 100644 --- a/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/fs/viewfs/ViewFileSystem.java +++ b/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/fs/viewfs/ViewFileSystem.java @@ -107,6 +107,8 @@ static AccessControlException readOnlyMountTable(final String operation, /** * Gets file system creator instance. + * + * @return fs getter. */ protected FsGetter fsGetter() { return new FsGetter(); @@ -253,9 +255,9 @@ public String[] getTargetFileSystemPaths() { private RenameStrategy renameStrategy = RenameStrategy.SAME_MOUNTPOINT; /** * Make the path Absolute and get the path-part of a pathname. - * Checks that URI matches this file system + * Checks that URI matches this file system * and that the path-part is a valid name. - * + * * @param p path * @return path-part of the Path p */ @@ -263,17 +265,17 @@ String getUriPath(final Path p) { checkPath(p); return makeAbsolute(p).toUri().getPath(); } - + private Path makeAbsolute(final Path f) { return f.isAbsolute() ? f : new Path(workingDir, f); } - + /** * This is the constructor with the signature needed by * {@link FileSystem#createFileSystem(URI, Configuration)} - * + * * After this constructor is called initialize() is called. - * @throws IOException + * @throws IOException raised on errors performing I/O. */ public ViewFileSystem() throws IOException { ugi = UserGroupInformation.getCurrentUser(); @@ -382,31 +384,31 @@ protected FileSystem getTargetFileSystem(final String settings, } /** - * Convenience Constructor for apps to call directly + * Convenience Constructor for apps to call directly. * @param theUri which must be that of ViewFileSystem - * @param conf - * @throws IOException + * @param conf conf configuration. + * @throws IOException raised on errors performing I/O. */ ViewFileSystem(final URI theUri, final Configuration conf) throws IOException { this(); initialize(theUri, conf); } - + /** - * Convenience Constructor for apps to call directly - * @param conf - * @throws IOException + * Convenience Constructor for apps to call directly. + * @param conf configuration. + * @throws IOException raised on errors performing I/O. */ public ViewFileSystem(final Configuration conf) throws IOException { this(FsConstants.VIEWFS_URI, conf); } - + @Override public URI getUri() { return myUri; } - + @Override public Path resolvePath(final Path f) throws IOException { final InodeTree.ResolveResult res; @@ -416,7 +418,7 @@ public Path resolvePath(final Path f) throws IOException { } return res.targetFileSystem.resolvePath(res.remainingPath); } - + @Override public Path getHomeDirectory() { if (homeDir == null) { @@ -424,13 +426,13 @@ public Path getHomeDirectory() { if (base == null) { base = "/user"; } - homeDir = (base.equals("/") ? + homeDir = (base.equals("/") ? this.makeQualified(new Path(base + ugi.getShortUserName())): this.makeQualified(new Path(base + "/" + ugi.getShortUserName()))); } return homeDir; } - + @Override public Path getWorkingDirectory() { return workingDir; @@ -441,11 +443,11 @@ public void setWorkingDirectory(final Path new_dir) { getUriPath(new_dir); // this validates the path workingDir = makeAbsolute(new_dir); } - + @Override public FSDataOutputStream append(final Path f, final int bufferSize, final Progressable progress) throws IOException { - InodeTree.ResolveResult res = + InodeTree.ResolveResult res = fsState.resolve(getUriPath(f), true); return res.targetFileSystem.append(res.remainingPath, bufferSize, progress); } @@ -464,7 +466,7 @@ public FSDataOutputStream createNonRecursive(Path f, FsPermission permission, return res.targetFileSystem.createNonRecursive(res.remainingPath, permission, flags, bufferSize, replication, blockSize, progress); } - + @Override public FSDataOutputStream create(final Path f, final FsPermission permission, final boolean overwrite, final int bufferSize, final short replication, @@ -480,11 +482,11 @@ public FSDataOutputStream create(final Path f, final FsPermission permission, overwrite, bufferSize, replication, blockSize, progress); } - + @Override public boolean delete(final Path f, final boolean recursive) throws AccessControlException, FileNotFoundException, IOException { - InodeTree.ResolveResult res = + InodeTree.ResolveResult res = fsState.resolve(getUriPath(f), true); // If internal dir or target is a mount link (ie remainingPath is Slash) if (res.isInternalDir() || res.remainingPath == InodeTree.SlashPath) { @@ -492,18 +494,18 @@ public boolean delete(final Path f, final boolean recursive) } return res.targetFileSystem.delete(res.remainingPath, recursive); } - + @Override @SuppressWarnings("deprecation") public boolean delete(final Path f) throws AccessControlException, FileNotFoundException, IOException { return delete(f, true); } - + @Override - public BlockLocation[] getFileBlockLocations(FileStatus fs, + public BlockLocation[] getFileBlockLocations(FileStatus fs, long start, long len) throws IOException { - final InodeTree.ResolveResult res = + final InodeTree.ResolveResult res = fsState.resolve(getUriPath(fs.getPath()), true); return res.targetFileSystem.getFileBlockLocations( new ViewFsFileStatus(fs, res.remainingPath), start, len); @@ -513,7 +515,7 @@ public BlockLocation[] getFileBlockLocations(FileStatus fs, public FileChecksum getFileChecksum(final Path f) throws AccessControlException, FileNotFoundException, IOException { - InodeTree.ResolveResult res = + InodeTree.ResolveResult res = fsState.resolve(getUriPath(f), true); return res.targetFileSystem.getFileChecksum(res.remainingPath); } @@ -570,7 +572,7 @@ public FileStatus getFileStatus(final Path f) throws AccessControlException, FileStatus status = res.targetFileSystem.getFileStatus(res.remainingPath); return fixFileStatus(status, this.makeQualified(f)); } - + @Override public void access(Path path, FsAction mode) throws AccessControlException, FileNotFoundException, IOException { @@ -611,7 +613,7 @@ public FileStatus[] listStatus(final Path f) throws AccessControlException, FileNotFoundException, IOException { InodeTree.ResolveResult res = fsState.resolve(getUriPath(f), true); - + FileStatus[] statusLst = res.targetFileSystem.listStatus(res.remainingPath); if (!res.isInternalDir()) { // We need to change the name in the FileStatus as described in @@ -675,7 +677,7 @@ public boolean mkdirs(Path dir) throws IOException { @Override public boolean mkdirs(final Path dir, final FsPermission permission) throws IOException { - InodeTree.ResolveResult res = + InodeTree.ResolveResult res = fsState.resolve(getUriPath(dir), false); return res.targetFileSystem.mkdirs(res.remainingPath, permission); } @@ -683,15 +685,15 @@ public boolean mkdirs(final Path dir, final FsPermission permission) @Override public FSDataInputStream open(final Path f, final int bufferSize) throws AccessControlException, FileNotFoundException, IOException { - InodeTree.ResolveResult res = + InodeTree.ResolveResult res = fsState.resolve(getUriPath(f), true); return res.targetFileSystem.open(res.remainingPath, bufferSize); } - + @Override public boolean rename(final Path src, final Path dst) throws IOException { - // passing resolveLastComponet as false to catch renaming a mount point to + // passing resolveLastComponet as false to catch renaming a mount point to // itself. We need to catch this as an internal operation and fail if no // fallback. InodeTree.ResolveResult resSrc = @@ -802,28 +804,28 @@ public boolean truncate(final Path f, final long newLength) fsState.resolve(getUriPath(f), true); return res.targetFileSystem.truncate(res.remainingPath, newLength); } - + @Override public void setOwner(final Path f, final String username, final String groupname) throws AccessControlException, FileNotFoundException, IOException { - InodeTree.ResolveResult res = + InodeTree.ResolveResult res = fsState.resolve(getUriPath(f), true); - res.targetFileSystem.setOwner(res.remainingPath, username, groupname); + res.targetFileSystem.setOwner(res.remainingPath, username, groupname); } @Override public void setPermission(final Path f, final FsPermission permission) throws AccessControlException, FileNotFoundException, IOException { - InodeTree.ResolveResult res = + InodeTree.ResolveResult res = fsState.resolve(getUriPath(f), true); - res.targetFileSystem.setPermission(res.remainingPath, permission); + res.targetFileSystem.setPermission(res.remainingPath, permission); } @Override public boolean setReplication(final Path f, final short replication) throws AccessControlException, FileNotFoundException, IOException { - InodeTree.ResolveResult res = + InodeTree.ResolveResult res = fsState.resolve(getUriPath(f), true); return res.targetFileSystem.setReplication(res.remainingPath, replication); } @@ -831,9 +833,9 @@ public boolean setReplication(final Path f, final short replication) @Override public void setTimes(final Path f, final long mtime, final long atime) throws AccessControlException, FileNotFoundException, IOException { - InodeTree.ResolveResult res = + InodeTree.ResolveResult res = fsState.resolve(getUriPath(f), true); - res.targetFileSystem.setTimes(res.remainingPath, mtime, atime); + res.targetFileSystem.setTimes(res.remainingPath, mtime, atime); } @Override @@ -955,7 +957,7 @@ private Map initializeMountedFileSystems( } return fsMap; } - + @Override public long getDefaultBlockSize() { throw new NotInMountpointException("getDefaultBlockSize"); @@ -978,7 +980,7 @@ public long getDefaultBlockSize(Path f) { fsState.resolve(getUriPath(f), true); return res.targetFileSystem.getDefaultBlockSize(res.remainingPath); } catch (FileNotFoundException e) { - throw new NotInMountpointException(f, "getDefaultBlockSize"); + throw new NotInMountpointException(f, "getDefaultBlockSize"); } catch (IOException e) { throw new RuntimeException("Not able to initialize fs in " + " getDefaultBlockSize for path " + f + " with exception", e); @@ -992,7 +994,7 @@ public short getDefaultReplication(Path f) { fsState.resolve(getUriPath(f), true); return res.targetFileSystem.getDefaultReplication(res.remainingPath); } catch (FileNotFoundException e) { - throw new NotInMountpointException(f, "getDefaultReplication"); + throw new NotInMountpointException(f, "getDefaultReplication"); } catch (IOException e) { throw new RuntimeException("Not able to initialize fs in " + " getDefaultReplication for path " + f + " with exception", e); @@ -1035,7 +1037,7 @@ public FileSystem[] getChildFileSystems() { List> mountPoints = fsState.getMountPoints(); Map fsMap = initializeMountedFileSystems(mountPoints); - Set children = new HashSet(); + Set children = new HashSet<>(); for (InodeTree.MountPoint mountPoint : mountPoints) { FileSystem targetFs = fsMap.get(mountPoint.src); children.addAll(Arrays.asList(targetFs.getChildFileSystems())); @@ -1054,11 +1056,11 @@ public FileSystem[] getChildFileSystems() { } return children.toArray(new FileSystem[]{}); } - + public MountPoint[] getMountPoints() { - List> mountPoints = + List> mountPoints = fsState.getMountPoints(); - + MountPoint[] result = new MountPoint[mountPoints.size()]; for ( int i = 0; i < mountPoints.size(); ++i ) { result[i] = new MountPoint(new Path(mountPoints.get(i).src), @@ -1314,7 +1316,7 @@ public FsStatus getStatus(Path p) throws IOException { * Constants#CONFIG_VIEWFS_LINK_MERGE_SLASH} is supported and is a valid * mount point. Else, throw NotInMountpointException. * - * @throws IOException + * @throws IOException raised on errors performing I/O. */ @Override public long getUsed() throws IOException { @@ -1375,9 +1377,9 @@ public boolean hasPathCapability(Path path, String capability) * are not allowed. * If called on create or mkdir then this target is the parent of the * directory in which one is trying to create or mkdir; hence - * in this case the path name passed in is the last component. + * in this case the path name passed in is the last component. * Otherwise this target is the end point of the path and hence - * the path name passed in is null. + * the path name passed in is null. */ static class InternalDirOfViewFs extends FileSystem { final InodeTree.INodeDir theInternalDir; @@ -1386,7 +1388,7 @@ static class InternalDirOfViewFs extends FileSystem { final URI myUri; private final boolean showMountLinksAsSymlinks; private InodeTree fsState; - + public InternalDirOfViewFs(final InodeTree.INodeDir dir, final long cTime, final UserGroupInformation ugi, URI uri, Configuration config, InodeTree fsState) throws URISyntaxException { @@ -1411,7 +1413,7 @@ static private void checkPathIsSlash(final Path f) throws IOException { "Internal implementation error: expected file name to be /"); } } - + @Override public URI getUri() { return myUri; @@ -1481,7 +1483,7 @@ public boolean delete(final Path f, final boolean recursive) checkPathIsSlash(f); throw readOnlyMountTable("delete", f); } - + @Override @SuppressWarnings("deprecation") public boolean delete(final Path f) @@ -1529,7 +1531,7 @@ public FileStatus getFileStatus(Path f) throws IOException { new Path(theInternalDir.fullPath).makeQualified( myUri, ROOT_PATH)); } - + @Override public FileStatus[] listStatus(Path f) throws AccessControlException, @@ -1544,7 +1546,7 @@ public FileStatus[] listStatus(Path f) throws AccessControlException, INode inode = iEntry.getValue(); Path path = new Path(inode.fullPath).makeQualified(myUri, null); if (inode.isLink()) { - INodeLink link = (INodeLink) inode; + INodeLink link = inode.getLink(); if (showMountLinksAsSymlinks) { // To maintain backward compatibility, with default option(showing @@ -1721,7 +1723,7 @@ public boolean rename(Path src, Path dst) throws AccessControlException, IOException { checkPathIsSlash(src); checkPathIsSlash(dst); - throw readOnlyMountTable("rename", src); + throw readOnlyMountTable("rename", src); } @Override @@ -1740,7 +1742,7 @@ public void setOwner(Path f, String username, String groupname) public void setPermission(Path f, FsPermission permission) throws AccessControlException, IOException { checkPathIsSlash(f); - throw readOnlyMountTable("setPermission", f); + throw readOnlyMountTable("setPermission", f); } @Override @@ -1754,7 +1756,7 @@ public boolean setReplication(Path f, short replication) public void setTimes(Path f, long mtime, long atime) throws AccessControlException, IOException { checkPathIsSlash(f); - throw readOnlyMountTable("setTimes", f); + throw readOnlyMountTable("setTimes", f); } @Override @@ -1766,7 +1768,7 @@ public void setVerifyChecksum(boolean verifyChecksum) { public FsServerDefaults getServerDefaults(Path f) throws IOException { throw new NotInMountpointException(f, "getServerDefaults"); } - + @Override public long getDefaultBlockSize(Path f) { throw new NotInMountpointException(f, "getDefaultBlockSize"); diff --git a/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/fs/viewfs/ViewFileSystemOverloadScheme.java b/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/fs/viewfs/ViewFileSystemOverloadScheme.java index e91b66512d5bf..1c25a9536e121 100644 --- a/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/fs/viewfs/ViewFileSystemOverloadScheme.java +++ b/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/fs/viewfs/ViewFileSystemOverloadScheme.java @@ -139,6 +139,8 @@ public boolean supportAutoAddingFallbackOnNoMounts() { /** * Sets whether to add fallback automatically when no mount points found. + * + * @param addAutoFallbackOnNoMounts addAutoFallbackOnNoMounts. */ public void setSupportAutoAddingFallbackOnNoMounts( boolean addAutoFallbackOnNoMounts) { @@ -320,7 +322,8 @@ private T newInstance(Class theClass, URI uri, Configuration conf) { * * @param path - fs uri path * @param conf - configuration - * @throws IOException + * @throws IOException raised on errors performing I/O. + * @return file system. */ public FileSystem getRawFileSystem(Path path, Configuration conf) throws IOException { @@ -339,6 +342,11 @@ public FileSystem getRawFileSystem(Path path, Configuration conf) /** * Gets the mount path info, which contains the target file system and * remaining path to pass to the target file system. + * + * @param path the path. + * @param conf configuration. + * @return mount path info. + * @throws IOException raised on errors performing I/O. */ public MountPathInfo getMountPathInfo(Path path, Configuration conf) throws IOException { diff --git a/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/fs/viewfs/ViewFileSystemUtil.java b/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/fs/viewfs/ViewFileSystemUtil.java index f486a10b4c8f9..c9c6767097b87 100644 --- a/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/fs/viewfs/ViewFileSystemUtil.java +++ b/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/fs/viewfs/ViewFileSystemUtil.java @@ -44,7 +44,7 @@ private ViewFileSystemUtil() { /** * Check if the FileSystem is a ViewFileSystem. * - * @param fileSystem + * @param fileSystem file system. * @return true if the fileSystem is ViewFileSystem */ public static boolean isViewFileSystem(final FileSystem fileSystem) { @@ -54,7 +54,7 @@ public static boolean isViewFileSystem(final FileSystem fileSystem) { /** * Check if the FileSystem is a ViewFileSystemOverloadScheme. * - * @param fileSystem + * @param fileSystem file system. * @return true if the fileSystem is ViewFileSystemOverloadScheme */ public static boolean isViewFileSystemOverloadScheme( @@ -101,6 +101,7 @@ public static boolean isViewFileSystemOverloadScheme( * @param fileSystem - ViewFileSystem on which mount point exists * @param path - URI for which FsStatus is requested * @return Map of ViewFsMountPoint and FsStatus + * @throws IOException raised on errors performing I/O. */ public static Map getStatus( FileSystem fileSystem, Path path) throws IOException { diff --git a/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/fs/viewfs/ViewFs.java b/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/fs/viewfs/ViewFs.java index 6e35ddf19053b..5f54c9cdd06aa 100644 --- a/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/fs/viewfs/ViewFs.java +++ b/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/fs/viewfs/ViewFs.java @@ -80,8 +80,8 @@ /** * ViewFs (extends the AbstractFileSystem interface) implements a client-side * mount table. The viewFs file system is implemented completely in memory on - * the client side. The client-side mount table allows a client to provide a - * customized view of a file system namespace that is composed from + * the client side. The client-side mount table allows a client to provide a + * customized view of a file system namespace that is composed from * one or more individual file systems (a localFs or Hdfs, S3fs, etc). * For example one could have a mount table that provides links such as *
      @@ -89,26 +89,26 @@ *
    • /project/foo {@literal ->} hdfs://nnProject1/projects/foo *
    • /project/bar {@literal ->} hdfs://nnProject2/projects/bar *
    • /tmp {@literal ->} hdfs://nnTmp/privateTmpForUserXXX - *
    - * - * ViewFs is specified with the following URI: viewfs:/// + * + * + * ViewFs is specified with the following URI: viewfs:/// *

    * To use viewfs one would typically set the default file system in the * config (i.e. fs.defaultFS {@literal <} = viewfs:///) along with the - * mount table config variables as described below. - * + * mount table config variables as described below. + * *

    * ** Config variables to specify the mount table entries ** *

    - * + * * The file system is initialized from the standard Hadoop config through * config variables. - * See {@link FsConstants} for URI and Scheme constants; - * See {@link Constants} for config var constants; + * See {@link FsConstants} for URI and Scheme constants; + * See {@link Constants} for config var constants; * see {@link ConfigUtil} for convenient lib. - * + * *

    - * All the mount table config entries for view fs are prefixed by + * All the mount table config entries for view fs are prefixed by * fs.viewfs.mounttable. * For example the above example can be specified with the following * config variables: @@ -122,8 +122,8 @@ *

  • fs.viewfs.mounttable.default.link./tmp= * hdfs://nnTmp/privateTmpForUserXXX * - * - * The default mount table (when no authority is specified) is + * + * The default mount table (when no authority is specified) is * from config variables prefixed by fs.viewFs.mounttable.default * The authority component of a URI can be used to specify a different mount * table. For example, @@ -131,11 +131,11 @@ *
  • viewfs://sanjayMountable/ * * is initialized from fs.viewFs.mounttable.sanjayMountable.* config variables. - * - *

    + * + *

    * **** Merge Mounts **** (NOTE: merge mounts are not implemented yet.) *

    - * + * * One can also use "MergeMounts" to merge several directories (this is * sometimes called union-mounts or junction-mounts in the literature. * For example of the home directories are stored on say two file systems @@ -156,7 +156,7 @@ *

  • fs.viewfs.mounttable.default.linkMergeSlash=hdfs://nn99/ * * In this cases the root of the mount table is merged with the root of - * hdfs://nn99/ + * hdfs://nn99/ */ @InterfaceAudience.Public @@ -182,8 +182,8 @@ static AccessControlException readOnlyMountTable(final String operation, final Path p) { return readOnlyMountTable(operation, p.toString()); } - - + + static public class MountPoint { // the src of the mount private Path src; @@ -214,15 +214,15 @@ public ViewFs(final Configuration conf) throws IOException, URISyntaxException { this(FsConstants.VIEWFS_URI, conf); } - + /** * This constructor has the signature needed by * {@link AbstractFileSystem#createFileSystem(URI, Configuration)}. - * + * * @param theUri which must be that of ViewFs * @param conf * @throws IOException - * @throws URISyntaxException + * @throws URISyntaxException */ ViewFs(final URI theUri, final Configuration conf) throws IOException, URISyntaxException { @@ -292,7 +292,7 @@ protected AbstractFileSystem getTargetFileSystem(final String settings, @Override @Deprecated public FsServerDefaults getServerDefaults() throws IOException { - return LocalConfigKeys.getServerDefaults(); + return LocalConfigKeys.getServerDefaults(); } @Override @@ -310,7 +310,7 @@ public FsServerDefaults getServerDefaults(final Path f) throws IOException { public int getUriDefaultPort() { return -1; } - + @Override public Path getHomeDirectory() { if (homeDir == null) { @@ -318,13 +318,13 @@ public Path getHomeDirectory() { if (base == null) { base = "/user"; } - homeDir = (base.equals("/") ? + homeDir = (base.equals("/") ? this.makeQualified(new Path(base + ugi.getShortUserName())): this.makeQualified(new Path(base + "/" + ugi.getShortUserName()))); } return homeDir; } - + @Override public Path resolvePath(final Path f) throws FileNotFoundException, AccessControlException, UnresolvedLinkException, IOException { @@ -336,7 +336,7 @@ public Path resolvePath(final Path f) throws FileNotFoundException, return res.targetFileSystem.resolvePath(res.remainingPath); } - + @Override public FSDataOutputStream createInternal(final Path f, final EnumSet flag, final FsPermission absolutePermission, @@ -367,7 +367,7 @@ public FSDataOutputStream createInternal(final Path f, public boolean delete(final Path f, final boolean recursive) throws AccessControlException, FileNotFoundException, UnresolvedLinkException, IOException { - InodeTree.ResolveResult res = + InodeTree.ResolveResult res = fsState.resolve(getUriPath(f), true); // If internal dir or target is a mount link (ie remainingPath is Slash) if (res.isInternalDir() || res.remainingPath == InodeTree.SlashPath) { @@ -381,7 +381,7 @@ public boolean delete(final Path f, final boolean recursive) public BlockLocation[] getFileBlockLocations(final Path f, final long start, final long len) throws AccessControlException, FileNotFoundException, UnresolvedLinkException, IOException { - InodeTree.ResolveResult res = + InodeTree.ResolveResult res = fsState.resolve(getUriPath(f), true); return res.targetFileSystem.getFileBlockLocations(res.remainingPath, start, len); @@ -391,7 +391,7 @@ public BlockLocation[] getFileBlockLocations(final Path f, final long start, public FileChecksum getFileChecksum(final Path f) throws AccessControlException, FileNotFoundException, UnresolvedLinkException, IOException { - InodeTree.ResolveResult res = + InodeTree.ResolveResult res = fsState.resolve(getUriPath(f), true); return res.targetFileSystem.getFileChecksum(res.remainingPath); } @@ -407,20 +407,20 @@ public FileChecksum getFileChecksum(final Path f) @Override public FileStatus getFileStatus(final Path f) throws AccessControlException, FileNotFoundException, UnresolvedLinkException, IOException { - InodeTree.ResolveResult res = + InodeTree.ResolveResult res = fsState.resolve(getUriPath(f), true); - // FileStatus#getPath is a fully qualified path relative to the root of + // FileStatus#getPath is a fully qualified path relative to the root of // target file system. // We need to change it to viewfs URI - relative to root of mount table. - + // The implementors of RawLocalFileSystem were trying to be very smart. // They implement FileStatus#getOwener lazily -- the object // returned is really a RawLocalFileSystem that expect the // FileStatus#getPath to be unchanged so that it can get owner when needed. // Hence we need to interpose a new ViewFsFileStatus that works around. - - + + FileStatus status = res.targetFileSystem.getFileStatus(res.remainingPath); return new ViewFsFileStatus(status, this.makeQualified(f)); } @@ -437,11 +437,11 @@ public void access(Path path, FsAction mode) throws AccessControlException, public FileStatus getFileLinkStatus(final Path f) throws AccessControlException, FileNotFoundException, UnsupportedFileSystemException, IOException { - InodeTree.ResolveResult res = + InodeTree.ResolveResult res = fsState.resolve(getUriPath(f), false); // do not follow mount link return res.targetFileSystem.getFileLinkStatus(res.remainingPath); } - + @Override public FsStatus getFsStatus() throws AccessControlException, FileNotFoundException, IOException { @@ -488,7 +488,7 @@ public LocatedFileStatus getViewFsFileStatus(LocatedFileStatus stat, } }; } - + /** * {@inheritDoc} * @@ -520,7 +520,7 @@ public FileStatus[] listStatus(final Path f) throws AccessControlException, FileNotFoundException, UnresolvedLinkException, IOException { InodeTree.ResolveResult res = fsState.resolve(getUriPath(f), true); - + FileStatus[] statusLst = res.targetFileSystem.listStatus(res.remainingPath); if (!res.isInternalDir()) { // We need to change the name in the FileStatus as described in @@ -542,7 +542,7 @@ public void mkdir(final Path dir, final FsPermission permission, final boolean createParent) throws AccessControlException, FileAlreadyExistsException, FileNotFoundException, UnresolvedLinkException, IOException { - InodeTree.ResolveResult res = + InodeTree.ResolveResult res = fsState.resolve(getUriPath(dir), false); res.targetFileSystem.mkdir(res.remainingPath, permission, createParent); } @@ -551,7 +551,7 @@ public void mkdir(final Path dir, final FsPermission permission, public FSDataInputStream open(final Path f, final int bufferSize) throws AccessControlException, FileNotFoundException, UnresolvedLinkException, IOException { - InodeTree.ResolveResult res = + InodeTree.ResolveResult res = fsState.resolve(getUriPath(f), true); return res.targetFileSystem.open(res.remainingPath, bufferSize); } @@ -568,7 +568,7 @@ public boolean truncate(final Path f, final long newLength) @Override public void renameInternal(final Path src, final Path dst, final boolean overwrite) throws IOException, UnresolvedLinkException { - // passing resolveLastComponet as false to catch renaming a mount point + // passing resolveLastComponet as false to catch renaming a mount point // itself we need to catch this as an internal operation and fail if no // fallback. InodeTree.ResolveResult resSrc = @@ -642,12 +642,12 @@ public void renameInternal(final Path src, final Path dst) UnresolvedLinkException, IOException { renameInternal(src, dst, false); } - + @Override public boolean supportsSymlinks() { return true; } - + @Override public void createSymlink(final Path target, final Path link, final boolean createParent) throws IOException, UnresolvedLinkException { @@ -663,12 +663,12 @@ public void createSymlink(final Path target, final Path link, } assert(res.remainingPath != null); res.targetFileSystem.createSymlink(target, res.remainingPath, - createParent); + createParent); } @Override public Path getLinkTarget(final Path f) throws IOException { - InodeTree.ResolveResult res = + InodeTree.ResolveResult res = fsState.resolve(getUriPath(f), false); // do not follow mount link return res.targetFileSystem.getLinkTarget(res.remainingPath); } @@ -677,26 +677,26 @@ public Path getLinkTarget(final Path f) throws IOException { public void setOwner(final Path f, final String username, final String groupname) throws AccessControlException, FileNotFoundException, UnresolvedLinkException, IOException { - InodeTree.ResolveResult res = + InodeTree.ResolveResult res = fsState.resolve(getUriPath(f), true); - res.targetFileSystem.setOwner(res.remainingPath, username, groupname); + res.targetFileSystem.setOwner(res.remainingPath, username, groupname); } @Override public void setPermission(final Path f, final FsPermission permission) throws AccessControlException, FileNotFoundException, UnresolvedLinkException, IOException { - InodeTree.ResolveResult res = + InodeTree.ResolveResult res = fsState.resolve(getUriPath(f), true); - res.targetFileSystem.setPermission(res.remainingPath, permission); - + res.targetFileSystem.setPermission(res.remainingPath, permission); + } @Override public boolean setReplication(final Path f, final short replication) throws AccessControlException, FileNotFoundException, UnresolvedLinkException, IOException { - InodeTree.ResolveResult res = + InodeTree.ResolveResult res = fsState.resolve(getUriPath(f), true); return res.targetFileSystem.setReplication(res.remainingPath, replication); } @@ -705,41 +705,41 @@ public boolean setReplication(final Path f, final short replication) public void setTimes(final Path f, final long mtime, final long atime) throws AccessControlException, FileNotFoundException, UnresolvedLinkException, IOException { - InodeTree.ResolveResult res = + InodeTree.ResolveResult res = fsState.resolve(getUriPath(f), true); - res.targetFileSystem.setTimes(res.remainingPath, mtime, atime); + res.targetFileSystem.setTimes(res.remainingPath, mtime, atime); } @Override public void setVerifyChecksum(final boolean verifyChecksum) throws AccessControlException, IOException { - // This is a file system level operations, however ViewFs - // points to many file systems. Noop for ViewFs. + // This is a file system level operations, however ViewFs + // points to many file systems. Noop for ViewFs. } - + public MountPoint[] getMountPoints() { - List> mountPoints = + List> mountPoints = fsState.getMountPoints(); - + MountPoint[] result = new MountPoint[mountPoints.size()]; for ( int i = 0; i < mountPoints.size(); ++i ) { - result[i] = new MountPoint(new Path(mountPoints.get(i).src), + result[i] = new MountPoint(new Path(mountPoints.get(i).src), mountPoints.get(i).target.targetDirLinkList); } return result; } - + @Override public List> getDelegationTokens(String renewer) throws IOException { - List> mountPoints = + List> mountPoints = fsState.getMountPoints(); int initialListSize = 0; for (InodeTree.MountPoint im : mountPoints) { - initialListSize += im.target.targetDirLinkList.length; + initialListSize += im.target.targetDirLinkList.length; } List> result = new ArrayList>(initialListSize); for ( int i = 0; i < mountPoints.size(); ++i ) { - List> tokens = + List> tokens = mountPoints.get(i).target.getTargetFileSystem() .getDelegationTokens(renewer); if (tokens != null) { @@ -909,7 +909,7 @@ public void unsetStoragePolicy(final Path src) * * @param src file or directory path. * @return storage policy for give file. - * @throws IOException + * @throws IOException raised on errors performing I/O. */ public BlockStoragePolicySpi getStoragePolicy(final Path src) throws IOException { @@ -955,18 +955,18 @@ public T next() throws IOException { } /* - * An instance of this class represents an internal dir of the viewFs + * An instance of this class represents an internal dir of the viewFs * ie internal dir of the mount table. * It is a ready only mount tbale and create, mkdir or delete operations * are not allowed. * If called on create or mkdir then this target is the parent of the * directory in which one is trying to create or mkdir; hence - * in this case the path name passed in is the last component. + * in this case the path name passed in is the last component. * Otherwise this target is the end point of the path and hence - * the path name passed in is null. + * the path name passed in is null. */ static class InternalDirOfViewFs extends AbstractFileSystem { - + final InodeTree.INodeDir theInternalDir; final long creationTime; // of the the mount table final UserGroupInformation ugi; // the user/group of user who created mtable @@ -1085,7 +1085,7 @@ public FileStatus getFileStatus(final Path f) throws IOException { new Path(theInternalDir.fullPath).makeQualified( myUri, null)); } - + @Override public FileStatus getFileLinkStatus(final Path f) throws IOException { @@ -1098,8 +1098,7 @@ public FileStatus getFileLinkStatus(final Path f) } FileStatus result; if (inode.isLink()) { - INodeLink inodelink = - (INodeLink) inode; + INodeLink inodelink = inode.getLink(); try { String linkedPath = inodelink.getTargetFileSystem() .getUri().getPath(); @@ -1127,7 +1126,7 @@ public FileStatus getFileLinkStatus(final Path f) } return result; } - + @Override public FsStatus getFsStatus() { return new FsStatus(0, 0, 0); @@ -1169,8 +1168,7 @@ public FileStatus[] listStatus(final Path f) throws IOException { INode inode = iEntry.getValue(); Path path = new Path(inode.fullPath).makeQualified(myUri, null); if (inode.isLink()) { - INodeLink link = - (INodeLink) inode; + INodeLink link = inode.getLink(); if (showMountLinksAsSymlinks) { // To maintain backward compatibility, with default option(showing @@ -1319,18 +1317,18 @@ public void renameInternal(final Path src, final Path dst) throws AccessControlException, IOException { checkPathIsSlash(src); checkPathIsSlash(dst); - throw readOnlyMountTable("rename", src); + throw readOnlyMountTable("rename", src); } @Override public boolean supportsSymlinks() { return true; } - + @Override public void createSymlink(final Path target, final Path link, final boolean createParent) throws AccessControlException { - throw readOnlyMountTable("createSymlink", link); + throw readOnlyMountTable("createSymlink", link); } @Override @@ -1350,7 +1348,7 @@ public void setOwner(final Path f, final String username, public void setPermission(final Path f, final FsPermission permission) throws AccessControlException, IOException { checkPathIsSlash(f); - throw readOnlyMountTable("setPermission", f); + throw readOnlyMountTable("setPermission", f); } @Override @@ -1364,13 +1362,13 @@ public boolean setReplication(final Path f, final short replication) public void setTimes(final Path f, final long mtime, final long atime) throws AccessControlException, IOException { checkPathIsSlash(f); - throw readOnlyMountTable("setTimes", f); + throw readOnlyMountTable("setTimes", f); } @Override public void setVerifyChecksum(final boolean verifyChecksum) throws AccessControlException { - throw readOnlyMountTable("setVerifyChecksum", ""); + throw readOnlyMountTable("setVerifyChecksum", ""); } @Override diff --git a/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/ha/ActiveStandbyElector.java b/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/ha/ActiveStandbyElector.java index 041f8cab49c4d..edd15af534a76 100644 --- a/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/ha/ActiveStandbyElector.java +++ b/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/ha/ActiveStandbyElector.java @@ -91,6 +91,8 @@ public interface ActiveStandbyElectorCallback { * * Callback implementations are expected to manage their own * timeouts (e.g. when making an RPC to a remote node). + * + * @throws ServiceFailedException Service Failed Exception. */ void becomeActive() throws ServiceFailedException; @@ -119,6 +121,8 @@ public interface ActiveStandbyElectorCallback { * If there is any fatal error (e.g. wrong ACL's, unexpected Zookeeper * errors or Zookeeper persistent unavailability) then notifyFatalError is * called to notify the app about it. + * + * @param errorMessage error message. */ void notifyFatalError(String errorMessage); @@ -204,8 +208,12 @@ enum State { * ZK connection * @param app * reference to callback interface object - * @throws IOException + * @param maxRetryNum maxRetryNum. + * @throws IOException raised on errors performing I/O. * @throws HadoopIllegalArgumentException + * if valid data is not supplied. + * @throws KeeperException + * other zookeeper operation errors. */ public ActiveStandbyElector(String zookeeperHostPorts, int zookeeperSessionTimeout, String parentZnodeName, List acl, @@ -245,8 +253,13 @@ public ActiveStandbyElector(String zookeeperHostPorts, * reference to callback interface object * @param failFast * whether need to add the retry when establishing ZK connection. + * @param maxRetryNum max Retry Num * @throws IOException + * raised on errors performing I/O. * @throws HadoopIllegalArgumentException + * if valid data is not supplied. + * @throws KeeperException + * other zookeeper operation errors. */ public ActiveStandbyElector(String zookeeperHostPorts, int zookeeperSessionTimeout, String parentZnodeName, List acl, @@ -312,6 +325,8 @@ public synchronized void joinElection(byte[] data) /** * @return true if the configured parent znode exists + * @throws IOException raised on errors performing I/O. + * @throws InterruptedException interrupted exception. */ public synchronized boolean parentZNodeExists() throws IOException, InterruptedException { @@ -327,6 +342,10 @@ public synchronized boolean parentZNodeExists() /** * Utility function to ensure that the configured base znode exists. * This recursively creates the znode as well as all of its parents. + * + * @throws IOException raised on errors performing I/O. + * @throws InterruptedException interrupted exception. + * @throws KeeperException other zookeeper operation errors. */ public synchronized void ensureParentZNode() throws IOException, InterruptedException, KeeperException { @@ -371,6 +390,9 @@ public synchronized void ensureParentZNode() * This recursively deletes everything within the znode as well as the * parent znode itself. It should only be used when it's certain that * no electors are currently participating in the election. + * + * @throws IOException raised on errors performing I/O. + * @throws InterruptedException interrupted exception. */ public synchronized void clearParentZNode() throws IOException, InterruptedException { @@ -435,6 +457,7 @@ public static class ActiveNotFoundException extends Exception { * @throws KeeperException * other zookeeper operation errors * @throws InterruptedException + * interrupted exception. * @throws IOException * when ZooKeeper connection could not be established */ @@ -684,7 +707,7 @@ synchronized void processWatchEvent(ZooKeeper zk, WatchedEvent event) { * inherit and mock out the zookeeper instance * * @return new zookeeper client instance - * @throws IOException + * @throws IOException raised on errors performing I/O. * @throws KeeperException zookeeper connectionloss exception */ protected synchronized ZooKeeper connectToZooKeeper() throws IOException, @@ -714,7 +737,7 @@ protected synchronized ZooKeeper connectToZooKeeper() throws IOException, * inherit and pass in a mock object for zookeeper * * @return new zookeeper client instance - * @throws IOException + * @throws IOException raised on errors performing I/O. */ protected ZooKeeper createZooKeeper() throws IOException { return new ZooKeeper(zkHostPort, zkSessionTimeout, watcher); @@ -781,6 +804,8 @@ private void reJoinElection(int sleepTime) { * Sleep for the given number of milliseconds. * This is non-static, and separated out, so that unit tests * can override the behavior not to sleep. + * + * @param sleepMs sleep ms. */ @VisibleForTesting protected void sleepFor(int sleepMs) { diff --git a/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/ha/HAAdmin.java b/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/ha/HAAdmin.java index c6949e561e2a2..9eeaacd76bca5 100644 --- a/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/ha/HAAdmin.java +++ b/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/ha/HAAdmin.java @@ -326,6 +326,9 @@ private int getServiceState(final CommandLine cmd) /** * Return the serviceId as is, we are assuming it was * given as a service address of form {@literal <}host:ipcport{@literal >}. + * + * @param serviceId serviceId. + * @return service addr. */ protected String getServiceAddr(String serviceId) { return serviceId; diff --git a/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/ha/HAServiceProtocol.java b/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/ha/HAServiceProtocol.java index 74a3d121a1abe..56c848617ffbc 100644 --- a/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/ha/HAServiceProtocol.java +++ b/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/ha/HAServiceProtocol.java @@ -118,7 +118,8 @@ public void monitorHealth() throws HealthCheckFailedException, /** * Request service to transition to active state. No operation, if the * service is already in active state. - * + * + * @param reqInfo reqInfo. * @throws ServiceFailedException * if transition from standby to active fails. * @throws AccessControlException @@ -135,7 +136,8 @@ public void transitionToActive(StateChangeRequestInfo reqInfo) /** * Request service to transition to standby state. No operation, if the * service is already in standby state. - * + * + * @param reqInfo reqInfo. * @throws ServiceFailedException * if transition from active to standby fails. * @throws AccessControlException @@ -153,6 +155,7 @@ public void transitionToStandby(StateChangeRequestInfo reqInfo) * Request service to transition to observer state. No operation, if the * service is already in observer state. * + * @param reqInfo reqInfo. * @throws ServiceFailedException * if transition from standby to observer fails. * @throws AccessControlException @@ -176,6 +179,7 @@ void transitionToObserver(StateChangeRequestInfo reqInfo) * @throws IOException * if other errors happen * @see HAServiceStatus + * @return HAServiceStatus. */ @Idempotent public HAServiceStatus getServiceStatus() throws AccessControlException, diff --git a/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/ha/HAServiceTarget.java b/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/ha/HAServiceTarget.java index 2e6b1fe113479..288a9dcbe0e53 100644 --- a/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/ha/HAServiceTarget.java +++ b/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/ha/HAServiceTarget.java @@ -93,6 +93,9 @@ public abstract void checkFencingConfigured() /** * @return a proxy to connect to the target HA Service. + * @param timeoutMs timeout in milliseconds. + * @param conf Configuration. + * @throws IOException raised on errors performing I/O. */ public HAServiceProtocol getProxy(Configuration conf, int timeoutMs) throws IOException { @@ -115,7 +118,7 @@ public HAServiceProtocol.HAServiceState getTransitionTargetHAStatus() { * returned proxy defaults to using {@link #getAddress()}, which means this * method's behavior is identical to {@link #getProxy(Configuration, int)}. * - * @param conf Configuration + * @param conf configuration. * @param timeoutMs timeout in milliseconds * @return a proxy to connect to the target HA service for health monitoring * @throws IOException if there is an error @@ -154,6 +157,9 @@ private HAServiceProtocol getProxyForAddress(Configuration conf, /** * @return a proxy to the ZKFC which is associated with this HA service. + * @param conf configuration. + * @param timeoutMs timeout in milliseconds. + * @throws IOException raised on errors performing I/O. */ public ZKFCProtocol getZKFCProxy(Configuration conf, int timeoutMs) throws IOException { diff --git a/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/ha/HealthMonitor.java b/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/ha/HealthMonitor.java index 7e90fb77a0702..d222d52e37349 100644 --- a/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/ha/HealthMonitor.java +++ b/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/ha/HealthMonitor.java @@ -184,6 +184,9 @@ private void tryConnect() { /** * Connect to the service to be monitored. Stubbed out for easier testing. + * + * @throws IOException raised on errors performing I/O. + * @return HAServiceProtocol. */ protected HAServiceProtocol createProxy() throws IOException { return targetToMonitor.getHealthMonitorProxy(conf, rpcTimeout, rpcConnectRetries); diff --git a/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/ha/ZKFailoverController.java b/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/ha/ZKFailoverController.java index 87a80b868cdb1..d24d5630c5917 100644 --- a/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/ha/ZKFailoverController.java +++ b/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/ha/ZKFailoverController.java @@ -153,6 +153,8 @@ protected abstract void checkRpcAdminAccess() * the ZKFC will do all of its work. This is so that multiple federated * nameservices can run on the same ZK quorum without having to manually * configure them to separate subdirectories. + * + * @return ScopeInsideParentNode. */ protected abstract String getScopeInsideParentNode(); diff --git a/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/http/HtmlQuoting.java b/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/http/HtmlQuoting.java index 51db21c185f20..5f47ddb339212 100644 --- a/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/http/HtmlQuoting.java +++ b/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/http/HtmlQuoting.java @@ -80,6 +80,7 @@ public static boolean needsQuoting(String str) { * @param buffer the byte array to take the characters from * @param off the index of the first byte to quote * @param len the number of bytes to quote + * @throws IOException raised on errors performing I/O. */ public static void quoteHtmlChars(OutputStream output, byte[] buffer, int off, int len) throws IOException { diff --git a/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/http/HttpServer2.java b/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/http/HttpServer2.java index 49807ac4b4597..2928f88598207 100644 --- a/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/http/HttpServer2.java +++ b/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/http/HttpServer2.java @@ -270,6 +270,7 @@ public Builder setName(String name){ * specifies the binding address, and the port specifies the * listening port. Unspecified or zero port means that the server * can listen to any port. + * @return Builder. */ public Builder addEndpoint(URI endpoint) { endpoints.add(endpoint); @@ -280,6 +281,9 @@ public Builder addEndpoint(URI endpoint) { * Set the hostname of the http server. The host name is used to resolve the * _HOST field in Kerberos principals. The hostname of the first listener * will be used if the name is unspecified. + * + * @param hostName hostName. + * @return Builder. */ public Builder hostName(String hostName) { this.hostName = hostName; @@ -308,6 +312,9 @@ public Builder keyPassword(String password) { /** * Specify whether the server should authorize the client in SSL * connections. + * + * @param value value. + * @return Builder. */ public Builder needsClientAuth(boolean value) { this.needsClientAuth = value; @@ -332,6 +339,9 @@ public Builder setConf(Configuration conf) { /** * Specify the SSL configuration to load. This API provides an alternative * to keyStore/keyPassword/trustStore. + * + * @param sslCnf sslCnf. + * @return Builder. */ public Builder setSSLConf(Configuration sslCnf) { this.sslConf = sslCnf; @@ -898,8 +908,11 @@ private static FilterInitializer[] getFilterInitializers(Configuration conf) { /** * Add default apps. + * + * @param parent contexthandlercollection. * @param appDir The application directory - * @throws IOException + * @param conf configuration. + * @throws IOException raised on errors performing I/O. */ protected void addDefaultApps(ContextHandlerCollection parent, final String appDir, Configuration conf) throws IOException { @@ -1180,6 +1193,12 @@ public void addGlobalFilter(String name, String classname, /** * Define a filter for a context and set up default url mappings. + * + * @param ctx ctx. + * @param name name. + * @param classname classname. + * @param parameters parameters. + * @param urls urls. */ public static void defineFilter(ServletContextHandler ctx, String name, String classname, Map parameters, String[] urls) { @@ -1290,6 +1309,7 @@ public int getPort() { /** * Get the address that corresponds to a particular connector. * + * @param index index. * @return the corresponding address for the connector, or null if there's no * such connector or the connector is not bounded or was closed. */ @@ -1309,6 +1329,9 @@ public InetSocketAddress getConnectorAddress(int index) { /** * Set the min, max number of worker threads (simultaneous connections). + * + * @param min min. + * @param max max. */ public void setThreads(int min, int max) { QueuedThreadPool pool = (QueuedThreadPool) webServer.getThreadPool(); @@ -1335,6 +1358,8 @@ private void initSpnego(Configuration conf, String hostName, /** * Start the server. Does not wait for the server to start. + * + * @throws IOException raised on errors performing I/O. */ public void start() throws IOException { try { @@ -1509,7 +1534,9 @@ void openListeners() throws Exception { } /** - * stop the server + * stop the server. + * + * @throws Exception exception. */ public void stop() throws Exception { MultiException exception = null; @@ -1610,6 +1637,7 @@ public String toString() { * @param request the servlet request. * @param response the servlet response. * @return TRUE/FALSE based on the logic decribed above. + * @throws IOException raised on errors performing I/O. */ public static boolean isInstrumentationAccessAllowed( ServletContext servletContext, HttpServletRequest request, @@ -1631,9 +1659,11 @@ public static boolean isInstrumentationAccessAllowed( * Does the user sending the HttpServletRequest has the administrator ACLs? If * it isn't the case, response will be modified to send an error to the user. * + * @param servletContext servletContext. + * @param request request. * @param response used to send the error response if user does not have admin access. * @return true if admin-authorized, false otherwise - * @throws IOException + * @throws IOException raised on errors performing I/O. */ public static boolean hasAdministratorAccess( ServletContext servletContext, HttpServletRequest request, diff --git a/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/io/AbstractMapWritable.java b/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/io/AbstractMapWritable.java index eef74628e16b1..8cf82f425090d 100644 --- a/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/io/AbstractMapWritable.java +++ b/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/io/AbstractMapWritable.java @@ -84,7 +84,10 @@ private synchronized void addToMap(Class clazz, byte id) { idToClassMap.put(id, clazz); } - /** Add a Class to the maps if it is not already present. */ + /** + * Add a Class to the maps if it is not already present. + * @param clazz clazz. + */ protected synchronized void addToMap(Class clazz) { if (classToIdMap.containsKey(clazz)) { return; @@ -97,17 +100,28 @@ protected synchronized void addToMap(Class clazz) { addToMap(clazz, id); } - /** @return the Class class for the specified id */ + /** + * the Class class for the specified id. + * @param id id. + * @return the Class class for the specified id. + */ protected Class getClass(byte id) { return idToClassMap.get(id); } - /** @return the id for the specified Class */ + /** + * get id. + * @return the id for the specified Class. + * @param clazz clazz. + */ protected byte getId(Class clazz) { return classToIdMap.containsKey(clazz) ? classToIdMap.get(clazz) : -1; } - /** Used by child copy constructors. */ + /** + * Used by child copy constructors. + * @param other other. + */ protected synchronized void copy(Writable other) { if (other != null) { try { diff --git a/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/io/ArrayFile.java b/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/io/ArrayFile.java index bee5fd2cb430c..313caa6360827 100644 --- a/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/io/ArrayFile.java +++ b/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/io/ArrayFile.java @@ -38,7 +38,15 @@ protected ArrayFile() {} // no public ctor public static class Writer extends MapFile.Writer { private LongWritable count = new LongWritable(0); - /** Create the named file for values of the named class. */ + /** + * Create the named file for values of the named class. + * + * @param conf configuration. + * @param fs file system. + * @param file file. + * @param valClass valClass. + * @throws IOException raised on errors performing I/O. + */ public Writer(Configuration conf, FileSystem fs, String file, Class valClass) throws IOException { @@ -46,7 +54,17 @@ public Writer(Configuration conf, FileSystem fs, valueClass(valClass)); } - /** Create the named file for values of the named class. */ + /** + * Create the named file for values of the named class. + * + * @param conf configuration. + * @param fs file system. + * @param file file. + * @param valClass valClass. + * @param compress compress. + * @param progress progress. + * @throws IOException raised on errors performing I/O. + */ public Writer(Configuration conf, FileSystem fs, String file, Class valClass, CompressionType compress, Progressable progress) @@ -58,7 +76,11 @@ public Writer(Configuration conf, FileSystem fs, progressable(progress)); } - /** Append a value to the file. */ + /** + * Append a value to the file. + * @param value value. + * @throws IOException raised on errors performing I/O. + */ public synchronized void append(Writable value) throws IOException { super.append(count, value); // add to map count.set(count.get()+1); // increment count @@ -69,31 +91,59 @@ public synchronized void append(Writable value) throws IOException { public static class Reader extends MapFile.Reader { private LongWritable key = new LongWritable(); - /** Construct an array reader for the named file.*/ + /** + * Construct an array reader for the named file. + * @param fs FileSystem. + * @param file file. + * @param conf configuration. + * @throws IOException raised on errors performing I/O. + */ public Reader(FileSystem fs, String file, Configuration conf) throws IOException { super(new Path(file), conf); } - /** Positions the reader before its nth value. */ + /** + * Positions the reader before its nth value. + * + * @param n n key. + * @throws IOException raised on errors performing I/O. + */ public synchronized void seek(long n) throws IOException { key.set(n); seek(key); } - /** Read and return the next value in the file. */ + /** + * Read and return the next value in the file. + * + * @param value value. + * @throws IOException raised on errors performing I/O. + * @return Writable. + */ public synchronized Writable next(Writable value) throws IOException { return next(key, value) ? value : null; } - /** Returns the key associated with the most recent call to {@link + /** + * Returns the key associated with the most recent call to {@link * #seek(long)}, {@link #next(Writable)}, or {@link - * #get(long,Writable)}. */ + * #get(long,Writable)}. + * + * @return key key. + * @throws IOException raised on errors performing I/O. + */ public synchronized long key() throws IOException { return key.get(); } - /** Return the nth value in the file. */ + /** + * Return the nth value in the file. + * @param n n key. + * @param value value. + * @throws IOException raised on errors performing I/O. + * @return writable. + */ public synchronized Writable get(long n, Writable value) throws IOException { key.set(n); diff --git a/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/io/ArrayPrimitiveWritable.java b/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/io/ArrayPrimitiveWritable.java index 2b6f3166bc282..ce7813e7483a6 100644 --- a/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/io/ArrayPrimitiveWritable.java +++ b/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/io/ArrayPrimitiveWritable.java @@ -106,7 +106,9 @@ public ArrayPrimitiveWritable() { /** * Construct an instance of known type but no value yet - * for use with type-specific wrapper classes + * for use with type-specific wrapper classes. + * + * @param componentType componentType. */ public ArrayPrimitiveWritable(Class componentType) { checkPrimitive(componentType); diff --git a/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/io/BinaryComparable.java b/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/io/BinaryComparable.java index a32c44c8e5058..a78ff8b6c583e 100644 --- a/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/io/BinaryComparable.java +++ b/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/io/BinaryComparable.java @@ -31,11 +31,15 @@ public abstract class BinaryComparable implements Comparable { /** * Return n st bytes 0..n-1 from {#getBytes()} are valid. + * + * @return length. */ public abstract int getLength(); /** * Return representative byte array for this instance. + * + * @return getBytes. */ public abstract byte[] getBytes(); @@ -53,6 +57,11 @@ public int compareTo(BinaryComparable other) { /** * Compare bytes from {#getBytes()} to those provided. + * + * @param other other. + * @param off off. + * @param len len. + * @return compareBytes. */ public int compareTo(byte[] other, int off, int len) { return WritableComparator.compareBytes(getBytes(), 0, getLength(), diff --git a/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/io/BloomMapFile.java b/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/io/BloomMapFile.java index 519fcd74cbb71..91ea07d5de412 100644 --- a/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/io/BloomMapFile.java +++ b/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/io/BloomMapFile.java @@ -259,7 +259,7 @@ private void initBloomFilter(Path dirName, * probability of false positives. * @param key key to check * @return false iff key doesn't exist, true if key probably exists. - * @throws IOException + * @throws IOException raised on errors performing I/O. */ public boolean probablyHasKey(WritableComparable key) throws IOException { if (bloomFilter == null) { diff --git a/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/io/BooleanWritable.java b/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/io/BooleanWritable.java index 0079079a7921d..789b866255b01 100644 --- a/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/io/BooleanWritable.java +++ b/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/io/BooleanWritable.java @@ -35,21 +35,24 @@ public class BooleanWritable implements WritableComparable { */ public BooleanWritable() {}; - /** + /** + * @param value value. */ public BooleanWritable(boolean value) { set(value); } /** - * Set the value of the BooleanWritable + * Set the value of the BooleanWritable. + * @param value value. */ public void set(boolean value) { this.value = value; } /** - * Returns the value of the BooleanWritable + * Returns the value of the BooleanWritable. + * @return the value of the BooleanWritable. */ public boolean get() { return value; diff --git a/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/io/BoundedByteArrayOutputStream.java b/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/io/BoundedByteArrayOutputStream.java index c27449d36189c..542721f318d0a 100644 --- a/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/io/BoundedByteArrayOutputStream.java +++ b/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/io/BoundedByteArrayOutputStream.java @@ -114,20 +114,28 @@ public void reset() { this.currentPointer = startOffset; } - /** Return the current limit */ + /** + * Return the current limit. + * @return limit. + */ public int getLimit() { return limit; } - /** Returns the underlying buffer. + /** + * Returns the underlying buffer. * Data is only valid to {@link #size()}. + * @return the underlying buffer. */ public byte[] getBuffer() { return buffer; } - /** Returns the length of the valid data + /** + * Returns the length of the valid data * currently in the buffer. + * + * @return the length of the valid data. */ public int size() { return currentPointer - startOffset; diff --git a/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/io/ByteBufferPool.java b/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/io/ByteBufferPool.java index aa5f8731c54a7..b30e7cfb9c5f0 100644 --- a/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/io/ByteBufferPool.java +++ b/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/io/ByteBufferPool.java @@ -45,4 +45,9 @@ public interface ByteBufferPool { * @param buffer a direct bytebuffer */ void putBuffer(ByteBuffer buffer); + + /** + * Clear the buffer pool thus releasing all the buffers. + */ + default void release() { } } diff --git a/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/io/ByteWritable.java b/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/io/ByteWritable.java index ffcdea2c9a3ab..c4b88f4b5c98b 100644 --- a/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/io/ByteWritable.java +++ b/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/io/ByteWritable.java @@ -33,10 +33,16 @@ public ByteWritable() {} public ByteWritable(byte value) { set(value); } - /** Set the value of this ByteWritable. */ + /** + * Set the value of this ByteWritable. + * @param value value. + */ public void set(byte value) { this.value = value; } - /** Return the value of this ByteWritable. */ + /** + * Return the value of this ByteWritable. + * @return value bytes. + */ public byte get() { return value; } @Override diff --git a/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/io/BytesWritable.java b/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/io/BytesWritable.java index c5538c9e56e85..80a23f86ce80b 100644 --- a/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/io/BytesWritable.java +++ b/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/io/BytesWritable.java @@ -77,6 +77,8 @@ public BytesWritable(byte[] bytes, int length) { /** * Get a copy of the bytes that is exactly the length of the data. * See {@link #getBytes()} for faster access to the underlying array. + * + * @return copyBytes. */ public byte[] copyBytes() { return Arrays.copyOf(bytes, size); @@ -95,6 +97,7 @@ public byte[] getBytes() { /** * Get the data from the BytesWritable. * @deprecated Use {@link #getBytes()} instead. + * @return data from the BytesWritable. */ @Deprecated public byte[] get() { @@ -112,6 +115,7 @@ public int getLength() { /** * Get the current size of the buffer. * @deprecated Use {@link #getLength()} instead. + * @return current size of the buffer. */ @Deprecated public int getSize() { diff --git a/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/io/CompressedWritable.java b/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/io/CompressedWritable.java index 6550e1f2fde04..c0315ab828c3b 100644 --- a/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/io/CompressedWritable.java +++ b/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/io/CompressedWritable.java @@ -67,7 +67,11 @@ protected void ensureInflated() { } } - /** Subclasses implement this instead of {@link #readFields(DataInput)}. */ + /** + * Subclasses implement this instead of {@link #readFields(DataInput)}. + * @param in data input. + * @throws IOException raised on errors performing I/O. + */ protected abstract void readFieldsCompressed(DataInput in) throws IOException; @@ -87,7 +91,12 @@ public final void write(DataOutput out) throws IOException { out.write(compressed); } - /** Subclasses implement this instead of {@link #write(DataOutput)}. */ + /** + * Subclasses implement this instead of {@link #write(DataOutput)}. + * + * @param out data output. + * @throws IOException raised on errors performing I/O. + */ protected abstract void writeCompressed(DataOutput out) throws IOException; } diff --git a/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/io/DataInputBuffer.java b/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/io/DataInputBuffer.java index 63c41c2e75008..85e905d870096 100644 --- a/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/io/DataInputBuffer.java +++ b/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/io/DataInputBuffer.java @@ -140,12 +140,23 @@ private DataInputBuffer(Buffer buffer) { this.buffer = buffer; } - /** Resets the data that the buffer reads. */ + /** + * Resets the data that the buffer reads. + * + * @param input input. + * @param length length. + */ public void reset(byte[] input, int length) { buffer.reset(input, 0, length); } - /** Resets the data that the buffer reads. */ + /** + * Resets the data that the buffer reads. + * + * @param input input. + * @param start start. + * @param length length. + */ public void reset(byte[] input, int start, int length) { buffer.reset(input, start, length); } @@ -154,12 +165,18 @@ public byte[] getData() { return buffer.getData(); } - /** Returns the current position in the input. */ + /** + * Returns the current position in the input. + * + * @return position. + */ public int getPosition() { return buffer.getPosition(); } /** * Returns the index one greater than the last valid character in the input * stream buffer. + * + * @return length. */ public int getLength() { return buffer.getLength(); } diff --git a/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/io/DataOutputBuffer.java b/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/io/DataOutputBuffer.java index 1d86b89701c03..4c1fa41e149c4 100644 --- a/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/io/DataOutputBuffer.java +++ b/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/io/DataOutputBuffer.java @@ -99,27 +99,45 @@ private DataOutputBuffer(Buffer buffer) { this.buffer = buffer; } - /** Returns the current contents of the buffer. + /** + * Returns the current contents of the buffer. * Data is only valid to {@link #getLength()}. + * + * @return data byte. */ public byte[] getData() { return buffer.getData(); } - /** Returns the length of the valid data currently in the buffer. */ + /** + * Returns the length of the valid data currently in the buffer. + * @return length. + */ public int getLength() { return buffer.getLength(); } - /** Resets the buffer to empty. */ + /** + * Resets the buffer to empty. + * @return DataOutputBuffer. + */ public DataOutputBuffer reset() { this.written = 0; buffer.reset(); return this; } - /** Writes bytes from a DataInput directly into the buffer. */ + /** + * Writes bytes from a DataInput directly into the buffer. + * @param in data input. + * @param length length. + * @throws IOException raised on errors performing I/O. + */ public void write(DataInput in, int length) throws IOException { buffer.write(in, length); } - /** Write to a file stream */ + /** + * Write to a file stream. + * @param out OutputStream. + * @throws IOException raised on errors performing I/O. + */ public void writeTo(OutputStream out) throws IOException { buffer.writeTo(out); } @@ -128,6 +146,10 @@ public void writeTo(OutputStream out) throws IOException { * Overwrite an integer into the internal buffer. Note that this call can only * be used to overwrite existing data in the buffer, i.e., buffer#count cannot * be increased, and DataOutputStream#written cannot be increased. + * + * @param v v. + * @param offset offset. + * @throws IOException raised on errors performing I/O. */ public void writeInt(int v, int offset) throws IOException { Preconditions.checkState(offset + 4 <= buffer.getLength()); diff --git a/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/io/ElasticByteBufferPool.java b/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/io/ElasticByteBufferPool.java index 6a162c3ff2087..c4c2940622729 100644 --- a/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/io/ElasticByteBufferPool.java +++ b/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/io/ElasticByteBufferPool.java @@ -36,8 +36,8 @@ */ @InterfaceAudience.Public @InterfaceStability.Stable -public final class ElasticByteBufferPool implements ByteBufferPool { - private static final class Key implements Comparable { +public class ElasticByteBufferPool implements ByteBufferPool { + protected static final class Key implements Comparable { private final int capacity; private final long insertionTime; diff --git a/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/io/EnumSetWritable.java b/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/io/EnumSetWritable.java index be86159519b87..4b1dc7513d054 100644 --- a/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/io/EnumSetWritable.java +++ b/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/io/EnumSetWritable.java @@ -64,8 +64,8 @@ public boolean add(E e) { * the argument value's size is bigger than zero, the argument * elementType is not be used. * - * @param value - * @param elementType + * @param value enumSet value. + * @param elementType elementType. */ public EnumSetWritable(EnumSet value, Class elementType) { set(value, elementType); @@ -75,7 +75,7 @@ public EnumSetWritable(EnumSet value, Class elementType) { * Construct a new EnumSetWritable. Argument value should not be null * or empty. * - * @param value + * @param value enumSet value. */ public EnumSetWritable(EnumSet value) { this(value, null); @@ -88,8 +88,8 @@ public EnumSetWritable(EnumSet value) { * null. If the argument value's size is bigger than zero, the * argument elementType is not be used. * - * @param value - * @param elementType + * @param value enumSet Value. + * @param elementType elementType. */ public void set(EnumSet value, Class elementType) { if ((value == null || value.size() == 0) @@ -106,7 +106,10 @@ public void set(EnumSet value, Class elementType) { } } - /** Return the value of this EnumSetWritable. */ + /** + * Return the value of this EnumSetWritable. + * @return EnumSet. + */ public EnumSet get() { return value; } diff --git a/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/io/FloatWritable.java b/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/io/FloatWritable.java index 367fc946da135..864bb8752f5c4 100644 --- a/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/io/FloatWritable.java +++ b/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/io/FloatWritable.java @@ -33,10 +33,16 @@ public FloatWritable() {} public FloatWritable(float value) { set(value); } - /** Set the value of this FloatWritable. */ + /** + * Set the value of this FloatWritable. + * @param value value. + */ public void set(float value) { this.value = value; } - /** Return the value of this FloatWritable. */ + /** + * Return the value of this FloatWritable. + * @return value. + */ public float get() { return value; } @Override diff --git a/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/io/GenericWritable.java b/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/io/GenericWritable.java index 7cfeed7f931d7..6de927467e478 100644 --- a/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/io/GenericWritable.java +++ b/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/io/GenericWritable.java @@ -90,7 +90,7 @@ public abstract class GenericWritable implements Writable, Configurable { /** * Set the instance that is wrapped. * - * @param obj + * @param obj input obj. */ public void set(Writable obj) { instance = obj; @@ -109,6 +109,7 @@ public void set(Writable obj) { /** * Return the wrapped instance. + * @return the wrapped instance. */ public Writable get() { return instance; @@ -145,6 +146,7 @@ public void write(DataOutput out) throws IOException { /** * Return all classes that may be wrapped. Subclasses should implement this * to return a constant array of classes. + * @return all classes that may be wrapped. */ abstract protected Class[] getTypes(); diff --git a/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/io/IOUtils.java b/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/io/IOUtils.java index 121af64b01182..f0a9b0b6952f2 100644 --- a/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/io/IOUtils.java +++ b/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/io/IOUtils.java @@ -59,7 +59,8 @@ public class IOUtils { * @param out OutputStream to write to * @param buffSize the size of the buffer * @param close whether or not close the InputStream and - * OutputStream at the end. The streams are closed in the finally clause. + * OutputStream at the end. The streams are closed in the finally clause. + * @throws IOException raised on errors performing I/O. */ public static void copyBytes(InputStream in, OutputStream out, int buffSize, boolean close) @@ -85,7 +86,8 @@ public static void copyBytes(InputStream in, OutputStream out, * * @param in InputStrem to read from * @param out OutputStream to write to - * @param buffSize the size of the buffer + * @param buffSize the size of the buffer. + * @throws IOException raised on errors performing I/O. */ public static void copyBytes(InputStream in, OutputStream out, int buffSize) throws IOException { @@ -107,7 +109,8 @@ public static void copyBytes(InputStream in, OutputStream out, int buffSize) * * @param in InputStrem to read from * @param out OutputStream to write to - * @param conf the Configuration object + * @param conf the Configuration object. + * @throws IOException raised on errors performing I/O. */ public static void copyBytes(InputStream in, OutputStream out, Configuration conf) throws IOException { @@ -123,6 +126,7 @@ public static void copyBytes(InputStream in, OutputStream out, Configuration con * @param conf the Configuration object * @param close whether or not close the InputStream and * OutputStream at the end. The streams are closed in the finally clause. + * @throws IOException raised on errors performing I/O. */ public static void copyBytes(InputStream in, OutputStream out, Configuration conf, boolean close) throws IOException { @@ -181,6 +185,7 @@ public static void copyBytes(InputStream in, OutputStream out, long count, * @param off - offset within buf * @param len - amount of data to be read * @return number of bytes read + * @throws IOException raised on errors performing I/O. */ public static int wrappedReadForCompressedData(InputStream is, byte[] buf, int off, int len) throws IOException { @@ -407,6 +412,7 @@ public static List listDirectory(File dir, FilenameFilter filter) * once the sync is done.
    * Borrowed from Uwe Schindler in LUCENE-5588 * @param fileToSync the file to fsync + * @throws IOException raised on errors performing I/O. */ public static void fsync(File fileToSync) throws IOException { if (!fileToSync.exists()) { @@ -440,7 +446,7 @@ public static void fsync(File fileToSync) throws IOException { * @param isDir if true, the given file is a directory (Channel should be * opened for read and ignore IOExceptions, because not all file * systems and operating systems allow to fsync on a directory) - * @throws IOException + * @throws IOException raised on errors performing I/O. */ public static void fsync(FileChannel channel, boolean isDir) throws IOException { diff --git a/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/io/InputBuffer.java b/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/io/InputBuffer.java index 0d084b8396f16..686b359f57d32 100644 --- a/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/io/InputBuffer.java +++ b/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/io/InputBuffer.java @@ -75,20 +75,35 @@ private InputBuffer(Buffer buffer) { this.buffer = buffer; } - /** Resets the data that the buffer reads. */ + /** + * Resets the data that the buffer reads. + * @param input input. + * @param length length. + */ public void reset(byte[] input, int length) { buffer.reset(input, 0, length); } - /** Resets the data that the buffer reads. */ + /** + * Resets the data that the buffer reads. + * @param input input. + * @param start start. + * @param length length. + */ public void reset(byte[] input, int start, int length) { buffer.reset(input, start, length); } - /** Returns the current position in the input. */ + /** + * Returns the current position in the input. + * @return the current position in the input. + */ public int getPosition() { return buffer.getPosition(); } - /** Returns the length of the input. */ + /** + * Returns the length of the input. + * @return length of the input. + */ public int getLength() { return buffer.getLength(); } } diff --git a/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/io/IntWritable.java b/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/io/IntWritable.java index f656d028cb054..ffcf93946d06a 100644 --- a/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/io/IntWritable.java +++ b/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/io/IntWritable.java @@ -36,10 +36,16 @@ public IntWritable() {} public IntWritable(int value) { set(value); } - /** Set the value of this IntWritable. */ + /** + * Set the value of this IntWritable. + * @param value input value. + */ public void set(int value) { this.value = value; } - /** Return the value of this IntWritable. */ + /** + * Return the value of this IntWritable. + * @return value of this IntWritable. + */ public int get() { return value; } @Override diff --git a/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/io/LongWritable.java b/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/io/LongWritable.java index b77ca6781a639..9262af87bc2e1 100644 --- a/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/io/LongWritable.java +++ b/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/io/LongWritable.java @@ -36,10 +36,16 @@ public LongWritable() {} public LongWritable(long value) { set(value); } - /** Set the value of this LongWritable. */ + /** + * Set the value of this LongWritable. + * @param value value. + */ public void set(long value) { this.value = value; } - /** Return the value of this LongWritable. */ + /** + * Return the value of this LongWritable. + * @return value of this LongWritable. + */ public long get() { return value; } @Override diff --git a/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/io/MD5Hash.java b/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/io/MD5Hash.java index 99c17acdd43d2..edfcf6e1e7754 100644 --- a/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/io/MD5Hash.java +++ b/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/io/MD5Hash.java @@ -54,12 +54,18 @@ public MD5Hash() { this.digest = new byte[MD5_LEN]; } - /** Constructs an MD5Hash from a hex string. */ + /** + * Constructs an MD5Hash from a hex string. + * @param hex input hex. + */ public MD5Hash(String hex) { setDigest(hex); } - /** Constructs an MD5Hash with a specified value. */ + /** + * Constructs an MD5Hash with a specified value. + * @param digest digest. + */ public MD5Hash(byte[] digest) { if (digest.length != MD5_LEN) throw new IllegalArgumentException("Wrong length: " + digest.length); @@ -72,7 +78,12 @@ public void readFields(DataInput in) throws IOException { in.readFully(digest); } - /** Constructs, reads and returns an instance. */ + /** + * Constructs, reads and returns an instance. + * @param in in. + * @throws IOException raised on errors performing I/O. + * @return MD5Hash. + */ public static MD5Hash read(DataInput in) throws IOException { MD5Hash result = new MD5Hash(); result.readFields(in); @@ -85,21 +96,32 @@ public void write(DataOutput out) throws IOException { out.write(digest); } - /** Copy the contents of another instance into this instance. */ + /** + * Copy the contents of another instance into this instance. + * @param that that. + */ public void set(MD5Hash that) { System.arraycopy(that.digest, 0, this.digest, 0, MD5_LEN); } - /** Returns the digest bytes. */ + /** + * Returns the digest bytes. + * @return digest. + */ public byte[] getDigest() { return digest; } - /** Construct a hash value for a byte array. */ + /** + * Construct a hash value for a byte array. + * @param data data. + * @return MD5Hash. + */ public static MD5Hash digest(byte[] data) { return digest(data, 0, data.length); } /** - * Create a thread local MD5 digester + * Create a thread local MD5 digester. + * @return MessageDigest. */ public static MessageDigest getDigester() { MessageDigest digester = DIGESTER_FACTORY.get(); @@ -107,7 +129,12 @@ public static MessageDigest getDigester() { return digester; } - /** Construct a hash value for the content from the InputStream. */ + /** + * Construct a hash value for the content from the InputStream. + * @param in input stream. + * @return MD5Hash. + * @throws IOException raised on errors performing I/O. + */ public static MD5Hash digest(InputStream in) throws IOException { final byte[] buffer = new byte[4*1024]; @@ -119,7 +146,13 @@ public static MD5Hash digest(InputStream in) throws IOException { return new MD5Hash(digester.digest()); } - /** Construct a hash value for a byte array. */ + /** + * Construct a hash value for a byte array. + * @param data data. + * @param start start. + * @param len len. + * @return MD5Hash. + */ public static MD5Hash digest(byte[] data, int start, int len) { byte[] digest; MessageDigest digester = getDigester(); @@ -128,7 +161,13 @@ public static MD5Hash digest(byte[] data, int start, int len) { return new MD5Hash(digest); } - /** Construct a hash value for an array of byte array. */ + /** + * Construct a hash value for an array of byte array. + * @param dataArr dataArr. + * @param start start. + * @param len len. + * @return MD5Hash. + */ public static MD5Hash digest(byte[][] dataArr, int start, int len) { byte[] digest; MessageDigest digester = getDigester(); @@ -139,17 +178,28 @@ public static MD5Hash digest(byte[][] dataArr, int start, int len) { return new MD5Hash(digest); } - /** Construct a hash value for a String. */ + /** + * Construct a hash value for a String. + * @param string string. + * @return MD5Hash. + */ public static MD5Hash digest(String string) { return digest(UTF8.getBytes(string)); } - /** Construct a hash value for a String. */ + /** + * Construct a hash value for a String. + * @param utf8 utf8. + * @return MD5Hash. + */ public static MD5Hash digest(UTF8 utf8) { return digest(utf8.getBytes(), 0, utf8.getLength()); } - /** Construct a half-sized version of this MD5. Fits in a long **/ + /** + * Construct a half-sized version of this MD5. Fits in a long. + * @return halfDigest. + */ public long halfDigest() { long value = 0; for (int i = 0; i < 8; i++) @@ -226,7 +276,10 @@ public String toString() { return buf.toString(); } - /** Sets the digest value from a hex string. */ + /** + * Sets the digest value from a hex string. + * @param hex hex. + */ public void setDigest(String hex) { if (hex.length() != MD5_LEN*2) throw new IllegalArgumentException("Wrong length: " + hex.length()); diff --git a/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/io/MapFile.java b/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/io/MapFile.java index 51db0b3f0afef..7b3cd78e3ccf3 100644 --- a/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/io/MapFile.java +++ b/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/io/MapFile.java @@ -98,8 +98,16 @@ public static class Writer implements java.io.Closeable { private long lastIndexKeyCount = Long.MIN_VALUE; - /** Create the named map for keys of the named class. + /** + * Create the named map for keys of the named class. * @deprecated Use Writer(Configuration, Path, Option...) instead. + * + * @param conf configuration. + * @param fs filesystem. + * @param dirName dirName. + * @param keyClass keyClass. + * @param valClass valClass. + * @throws IOException raised on errors performing I/O. */ @Deprecated public Writer(Configuration conf, FileSystem fs, String dirName, @@ -108,8 +116,18 @@ public Writer(Configuration conf, FileSystem fs, String dirName, this(conf, new Path(dirName), keyClass(keyClass), valueClass(valClass)); } - /** Create the named map for keys of the named class. + /** + * Create the named map for keys of the named class. * @deprecated Use Writer(Configuration, Path, Option...) instead. + * + * @param conf configuration. + * @param fs fs. + * @param dirName dirName. + * @param keyClass keyClass. + * @param valClass valClass. + * @param compress compress. + * @param progress progress. + * @throws IOException raised on errors performing I/O. */ @Deprecated public Writer(Configuration conf, FileSystem fs, String dirName, @@ -120,8 +138,19 @@ public Writer(Configuration conf, FileSystem fs, String dirName, compression(compress), progressable(progress)); } - /** Create the named map for keys of the named class. + /** + * Create the named map for keys of the named class. * @deprecated Use Writer(Configuration, Path, Option...) instead. + * + * @param conf configuration. + * @param fs FileSystem. + * @param dirName dirName. + * @param keyClass keyClass. + * @param valClass valClass. + * @param compress compress. + * @param codec codec. + * @param progress progress. + * @throws IOException raised on errors performing I/O. */ @Deprecated public Writer(Configuration conf, FileSystem fs, String dirName, @@ -132,8 +161,16 @@ public Writer(Configuration conf, FileSystem fs, String dirName, compression(compress, codec), progressable(progress)); } - /** Create the named map for keys of the named class. + /** + * Create the named map for keys of the named class. * @deprecated Use Writer(Configuration, Path, Option...) instead. + * @param conf configuration. + * @param fs fs. + * @param dirName dirName. + * @param keyClass keyClass. + * @param valClass valClass. + * @param compress compress. + * @throws IOException raised on errors performing I/O. */ @Deprecated public Writer(Configuration conf, FileSystem fs, String dirName, @@ -145,6 +182,12 @@ public Writer(Configuration conf, FileSystem fs, String dirName, /** Create the named map using the named key comparator. * @deprecated Use Writer(Configuration, Path, Option...) instead. + * @param conf configuration. + * @param fs fs. + * @param dirName dirName. + * @param comparator comparator. + * @param valClass valClass. + * @throws IOException raised on errors performing I/O. */ @Deprecated public Writer(Configuration conf, FileSystem fs, String dirName, @@ -154,7 +197,14 @@ public Writer(Configuration conf, FileSystem fs, String dirName, valueClass(valClass)); } - /** Create the named map using the named key comparator. + /** Create the named map using the named key comparator. + * @param conf configuration. + * @param fs filesystem. + * @param dirName dirName. + * @param comparator comparator. + * @param valClass valClass. + * @param compress compress. + * @throws IOException raised on errors performing I/O. * @deprecated Use Writer(Configuration, Path, Option...) instead. */ @Deprecated @@ -165,8 +215,18 @@ public Writer(Configuration conf, FileSystem fs, String dirName, valueClass(valClass), compression(compress)); } - /** Create the named map using the named key comparator. + /** + * Create the named map using the named key comparator. * @deprecated Use Writer(Configuration, Path, Option...)} instead. + * + * @param conf configuration. + * @param fs filesystem. + * @param dirName dirName. + * @param comparator comparator. + * @param valClass valClass. + * @param compress CompressionType. + * @param progress progress. + * @throws IOException raised on errors performing I/O. */ @Deprecated public Writer(Configuration conf, FileSystem fs, String dirName, @@ -178,8 +238,19 @@ public Writer(Configuration conf, FileSystem fs, String dirName, progressable(progress)); } - /** Create the named map using the named key comparator. + /** + * Create the named map using the named key comparator. * @deprecated Use Writer(Configuration, Path, Option...) instead. + * + * @param conf configuration. + * @param fs FileSystem. + * @param dirName dirName. + * @param comparator comparator. + * @param valClass valClass. + * @param compress CompressionType. + * @param codec codec. + * @param progress progress. + * @throws IOException raised on errors performing I/O. */ @Deprecated public Writer(Configuration conf, FileSystem fs, String dirName, @@ -285,16 +356,26 @@ public Writer(Configuration conf, this.index = SequenceFile.createWriter(conf, indexOptions); } - /** The number of entries that are added before an index entry is added.*/ + /** + * The number of entries that are added before an index entry is added. + * @return indexInterval + */ public int getIndexInterval() { return indexInterval; } - /** Sets the index interval. + /** + * Sets the index interval. * @see #getIndexInterval() + * + * @param interval interval. */ public void setIndexInterval(int interval) { indexInterval = interval; } - /** Sets the index interval and stores it in conf + /** + * Sets the index interval and stores it in conf. * @see #getIndexInterval() + * + * @param conf configuration. + * @param interval interval. */ public static void setIndexInterval(Configuration conf, int interval) { conf.setInt(INDEX_INTERVAL, interval); @@ -307,8 +388,14 @@ public synchronized void close() throws IOException { index.close(); } - /** Append a key/value pair to the map. The key must be greater or equal - * to the previous key added to the map. */ + /** + * Append a key/value pair to the map. The key must be greater or equal + * to the previous key added to the map. + * + * @param key key. + * @param val value. + * @throws IOException raised on errors performing I/O. + */ public synchronized void append(WritableComparable key, Writable val) throws IOException { @@ -370,10 +457,18 @@ public static class Reader implements java.io.Closeable { private WritableComparable[] keys; private long[] positions; - /** Returns the class of keys in this file. */ + /** + * Returns the class of keys in this file. + * + * @return keyClass. + */ public Class getKeyClass() { return data.getKeyClass(); } - /** Returns the class of values in this file. */ + /** + * Returns the class of values in this file. + * + * @return Value Class. + */ public Class getValueClass() { return data.getValueClass(); } public static interface Option extends SequenceFile.Reader.Option {} @@ -403,8 +498,14 @@ public Reader(Path dir, Configuration conf, open(dir, comparator, conf, opts); } - /** Construct a map reader for the named map. + /** + * Construct a map reader for the named map. * @deprecated + * + * @param fs FileSystem. + * @param dirName dirName. + * @param conf configuration. + * @throws IOException raised on errors performing I/O. */ @Deprecated public Reader(FileSystem fs, String dirName, @@ -412,8 +513,15 @@ public Reader(FileSystem fs, String dirName, this(new Path(dirName), conf); } - /** Construct a map reader for the named map using the named comparator. + /** + * Construct a map reader for the named map using the named comparator. * @deprecated + * + * @param fs FileSystem. + * @param dirName dirName. + * @param comparator WritableComparator. + * @param conf Configuration. + * @throws IOException raised on errors performing I/O. */ @Deprecated public Reader(FileSystem fs, String dirName, WritableComparator comparator, @@ -450,6 +558,12 @@ protected synchronized void open(Path dir, /** * Override this method to specialize the type of * {@link SequenceFile.Reader} returned. + * + * @param dataFile data file. + * @param conf configuration. + * @param options options. + * @throws IOException raised on errors performing I/O. + * @return SequenceFile.Reader. */ protected SequenceFile.Reader createDataFileReader(Path dataFile, Configuration conf, @@ -516,13 +630,21 @@ private void readIndex() throws IOException { } } - /** Re-positions the reader before its first key. */ + /** + * Re-positions the reader before its first key. + * + * @throws IOException raised on errors performing I/O. + */ public synchronized void reset() throws IOException { data.seek(firstPosition); } - /** Get the key at approximately the middle of the file. Or null if the - * file is empty. + /** + * Get the key at approximately the middle of the file. Or null if the + * file is empty. + * + * @throws IOException raised on errors performing I/O. + * @return WritableComparable. */ public synchronized WritableComparable midKey() throws IOException { @@ -534,9 +656,11 @@ public synchronized WritableComparable midKey() throws IOException { return keys[(count - 1) / 2]; } - /** Reads the final key from the file. + /** + * Reads the final key from the file. * * @param key key to read into + * @throws IOException raised on errors performing I/O. */ public synchronized void finalKey(WritableComparable key) throws IOException { @@ -556,9 +680,14 @@ public synchronized void finalKey(WritableComparable key) } } - /** Positions the reader at the named key, or if none such exists, at the + /** + * Positions the reader at the named key, or if none such exists, at the * first entry after the named key. Returns true iff the named key exists * in this map. + * + * @param key key. + * @throws IOException raised on errors performing I/O. + * @return if the named key exists in this map true, not false. */ public synchronized boolean seek(WritableComparable key) throws IOException { return seekInternal(key) == 0; @@ -669,15 +798,28 @@ else if (cmp > 0) return -(low + 1); // key not found. } - /** Read the next key/value pair in the map into key and + /** + * Read the next key/value pair in the map into key and * val. Returns true if such a pair exists and false when at - * the end of the map */ + * the end of the map. + * + * @param key WritableComparable. + * @param val Writable. + * @return if such a pair exists true,not false. + * @throws IOException raised on errors performing I/O. + */ public synchronized boolean next(WritableComparable key, Writable val) throws IOException { return data.next(key, val); } - /** Return the value for the named key, or null if none exists. */ + /** + * Return the value for the named key, or null if none exists. + * @param key key. + * @param val val. + * @return Writable if such a pair exists true,not false. + * @throws IOException raised on errors performing I/O. + */ public synchronized Writable get(WritableComparable key, Writable val) throws IOException { if (seek(key)) { @@ -692,9 +834,10 @@ public synchronized Writable get(WritableComparable key, Writable val) * Returns key or if it does not exist, at the first entry * after the named key. * -- * @param key - key that we're trying to find -- * @param val - data value if key is found -- * @return - the key that was the closest match or null if eof. + * @param key key that we're trying to find. + * @param val data value if key is found. + * @return the key that was the closest match or null if eof. + * @throws IOException raised on errors performing I/O. */ public synchronized WritableComparable getClosest(WritableComparable key, Writable val) @@ -711,6 +854,7 @@ public synchronized WritableComparable getClosest(WritableComparable key, * the first entry that falls just before the key. Otherwise, * return the record that sorts just after. * @return - the key that was the closest match or null if eof. + * @throws IOException raised on errors performing I/O. */ public synchronized WritableComparable getClosest(WritableComparable key, Writable val, final boolean before) @@ -730,7 +874,10 @@ public synchronized WritableComparable getClosest(WritableComparable key, return nextKey; } - /** Close the map. */ + /** + * Close the map. + * @throws IOException raised on errors performing I/O. + */ @Override public synchronized void close() throws IOException { if (!indexClosed) { @@ -741,7 +888,13 @@ public synchronized void close() throws IOException { } - /** Renames an existing map directory. */ + /** + * Renames an existing map directory. + * @param fs fs. + * @param oldName oldName. + * @param newName newName. + * @throws IOException raised on errors performing I/O. + */ public static void rename(FileSystem fs, String oldName, String newName) throws IOException { Path oldDir = new Path(oldName); @@ -751,7 +904,12 @@ public static void rename(FileSystem fs, String oldName, String newName) } } - /** Deletes the named map file. */ + /** + * Deletes the named map file. + * @param fs input fs. + * @param name input name. + * @throws IOException raised on errors performing I/O. + */ public static void delete(FileSystem fs, String name) throws IOException { Path dir = new Path(name); Path data = new Path(dir, DATA_FILE_NAME); @@ -769,8 +927,9 @@ public static void delete(FileSystem fs, String name) throws IOException { * @param keyClass key class (has to be a subclass of Writable) * @param valueClass value class (has to be a subclass of Writable) * @param dryrun do not perform any changes, just report what needs to be done + * @param conf configuration. * @return number of valid entries in this MapFile, or -1 if no fixing was needed - * @throws Exception + * @throws Exception Exception. */ public static long fix(FileSystem fs, Path dir, Class keyClass, @@ -870,11 +1029,12 @@ public Merger(Configuration conf) throws IOException { } /** - * Merge multiple MapFiles to one Mapfile + * Merge multiple MapFiles to one Mapfile. * - * @param inMapFiles - * @param outMapFile - * @throws IOException + * @param inMapFiles input inMapFiles. + * @param deleteInputs deleteInputs. + * @param outMapFile input outMapFile. + * @throws IOException raised on errors performing I/O. */ public void merge(Path[] inMapFiles, boolean deleteInputs, Path outMapFile) throws IOException { diff --git a/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/io/MultipleIOException.java b/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/io/MultipleIOException.java index c9d7ade43064f..452965b7c8220 100644 --- a/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/io/MultipleIOException.java +++ b/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/io/MultipleIOException.java @@ -42,7 +42,11 @@ private MultipleIOException(List exceptions) { /** @return the underlying exceptions */ public List getExceptions() {return exceptions;} - /** A convenient method to create an {@link IOException}. */ + /** + * A convenient method to create an {@link IOException}. + * @param exceptions IOException List. + * @return IOException. + */ public static IOException createIOException(List exceptions) { if (exceptions == null || exceptions.isEmpty()) { return null; @@ -60,7 +64,10 @@ public static IOException createIOException(List exceptions) { public static class Builder { private List exceptions; - /** Add the given {@link Throwable} to the exception list. */ + /** + * Add the given {@link Throwable} to the exception list. + * @param t Throwable. + */ public void add(Throwable t) { if (exceptions == null) { exceptions = new ArrayList<>(); diff --git a/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/io/NullWritable.java b/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/io/NullWritable.java index 77c590fdb6344..d6e4846264f98 100644 --- a/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/io/NullWritable.java +++ b/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/io/NullWritable.java @@ -32,7 +32,10 @@ public class NullWritable implements WritableComparable { private NullWritable() {} // no public ctor - /** Returns the single instance of this class. */ + /** + * Returns the single instance of this class. + * @return the single instance of this class. + */ public static NullWritable get() { return THIS; } @Override diff --git a/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/io/ObjectWritable.java b/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/io/ObjectWritable.java index b35a32f288b4b..29c06a01ad6e3 100644 --- a/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/io/ObjectWritable.java +++ b/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/io/ObjectWritable.java @@ -54,13 +54,22 @@ public ObjectWritable(Class declaredClass, Object instance) { this.instance = instance; } - /** Return the instance, or null if none. */ + /** + * Return the instance, or null if none. + * @return the instance, or null if none. + */ public Object get() { return instance; } - /** Return the class this is meant to be. */ + /** + * Return the class this is meant to be. + * @return the class this is meant to be. + */ public Class getDeclaredClass() { return declaredClass; } - /** Reset the instance. */ + /** + * Reset the instance. + * @param instance instance. + */ public void set(Object instance) { this.declaredClass = instance.getClass(); this.instance = instance; @@ -120,8 +129,16 @@ public void write(DataOutput out) throws IOException { } } - /** Write a {@link Writable}, {@link String}, primitive type, or an array of - * the preceding. */ + /** + * Write a {@link Writable}, {@link String}, primitive type, or an array of + * the preceding. + * + * @param out DataOutput. + * @param instance instance. + * @param conf Configuration. + * @param declaredClass declaredClass. + * @throws IOException raised on errors performing I/O. + */ public static void writeObject(DataOutput out, Object instance, Class declaredClass, Configuration conf) throws IOException { @@ -137,6 +154,13 @@ public static void writeObject(DataOutput out, Object instance, * usages, to preserve the ability to interchange files with other clusters * that may not be running the same version of software. Sometime in ~2013 * we can consider removing this parameter and always using the compact format. + * + * @param conf configuration. + * @param out dataoutput. + * @param declaredClass declaredClass. + * @param instance instance. + * @throws IOException raised on errors performing I/O. + * */ public static void writeObject(DataOutput out, Object instance, Class declaredClass, Configuration conf, boolean allowCompactArrays) @@ -210,15 +234,30 @@ public static void writeObject(DataOutput out, Object instance, } - /** Read a {@link Writable}, {@link String}, primitive type, or an array of - * the preceding. */ + /** + * Read a {@link Writable}, {@link String}, primitive type, or an array of + * the preceding. + * + * @param conf configuration. + * @param in DataInput. + * @return Object. + * @throws IOException raised on errors performing I/O. + */ public static Object readObject(DataInput in, Configuration conf) throws IOException { return readObject(in, null, conf); } - /** Read a {@link Writable}, {@link String}, primitive type, or an array of - * the preceding. */ + /** + * Read a {@link Writable}, {@link String}, primitive type, or an array of + * the preceding. + * + * @param in DataInput. + * @param objectWritable objectWritable. + * @param conf configuration. + * @return Object. + * @throws IOException raised on errors performing I/O. + */ @SuppressWarnings("unchecked") public static Object readObject(DataInput in, ObjectWritable objectWritable, Configuration conf) throws IOException { @@ -365,6 +404,10 @@ static Method getStaticProtobufMethod(Class declaredClass, String method, * Find and load the class with given name className by first finding * it in the specified conf. If the specified conf is null, * try load it directly. + * + * @param conf configuration. + * @param className classname. + * @return Class. */ public static Class loadClass(Configuration conf, String className) { Class declaredClass = null; diff --git a/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/io/OutputBuffer.java b/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/io/OutputBuffer.java index 15a396dc2bf55..f80c0a71883d6 100644 --- a/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/io/OutputBuffer.java +++ b/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/io/OutputBuffer.java @@ -77,21 +77,33 @@ private OutputBuffer(Buffer buffer) { this.buffer = buffer; } - /** Returns the current contents of the buffer. + /** + * Returns the current contents of the buffer. * Data is only valid to {@link #getLength()}. + * + * @return the current contents of the buffer. */ public byte[] getData() { return buffer.getData(); } - /** Returns the length of the valid data currently in the buffer. */ + /** + * Returns the length of the valid data currently in the buffer. + * @return the length of the valid data + * currently in the buffer. + */ public int getLength() { return buffer.getLength(); } - /** Resets the buffer to empty. */ + /** @return Resets the buffer to empty. */ public OutputBuffer reset() { buffer.reset(); return this; } - /** Writes bytes from a InputStream directly into the buffer. */ + /** + * Writes bytes from a InputStream directly into the buffer. + * @param in input in. + * @param length input length. + * @throws IOException raised on errors performing I/O. + */ public void write(InputStream in, int length) throws IOException { buffer.write(in, length); } diff --git a/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/io/RawComparator.java b/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/io/RawComparator.java index a52190db5f4d1..354dda964e92b 100644 --- a/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/io/RawComparator.java +++ b/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/io/RawComparator.java @@ -29,7 +29,7 @@ * A {@link Comparator} that operates directly on byte representations of * objects. *

    - * @param + * @param generic type. * @see DeserializerComparator */ @InterfaceAudience.Public diff --git a/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/io/ReadaheadPool.java b/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/io/ReadaheadPool.java index 65e751eca417e..2a6fafce545f3 100644 --- a/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/io/ReadaheadPool.java +++ b/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/io/ReadaheadPool.java @@ -50,7 +50,7 @@ public class ReadaheadPool { private static ReadaheadPool instance; /** - * Return the singleton instance for the current process. + * @return Return the singleton instance for the current process. */ public static ReadaheadPool getInstance() { synchronized (ReadaheadPool.class) { diff --git a/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/io/SecureIOUtils.java b/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/io/SecureIOUtils.java index 016daf9f352c1..cddddcc6c9a45 100644 --- a/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/io/SecureIOUtils.java +++ b/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/io/SecureIOUtils.java @@ -90,7 +90,7 @@ public class SecureIOUtils { private final static FileSystem rawFilesystem; /** - * Open the given File for random read access, verifying the expected user/ + * @return Open the given File for random read access, verifying the expected user/ * group constraints if security is enabled. * * Note that this function provides no additional security checks if hadoop @@ -114,8 +114,14 @@ public static RandomAccessFile openForRandomRead(File f, } /** - * Same as openForRandomRead except that it will run even if security is off. + * @return Same as openForRandomRead except that it will run even if security is off. * This is used by unit tests. + * + * @param f input f. + * @param mode input mode. + * @param expectedOwner input expectedOwner. + * @param expectedGroup input expectedGroup. + * @throws IOException raised on errors performing I/O. */ @VisibleForTesting protected static RandomAccessFile forceSecureOpenForRandomRead(File f, @@ -145,6 +151,7 @@ protected static RandomAccessFile forceSecureOpenForRandomRead(File f, * @param expectedGroup the expected group owner for the file * @throws IOException if an IO Error occurred or the user/group does not * match if security is enabled + * @return FSDataInputStream. */ public static FSDataInputStream openFSDataInputStream(File file, String expectedOwner, String expectedGroup) throws IOException { @@ -157,6 +164,12 @@ public static FSDataInputStream openFSDataInputStream(File file, /** * Same as openFSDataInputStream except that it will run even if security is * off. This is used by unit tests. + * + * @param file input file. + * @param expectedOwner input expectedOwner. + * @param expectedGroup input expectedGroup. + * @throws IOException raised on errors performing I/O. + * @return FSDataInputStream. */ @VisibleForTesting protected static FSDataInputStream forceSecureOpenFSDataInputStream( @@ -182,7 +195,7 @@ protected static FSDataInputStream forceSecureOpenFSDataInputStream( * Open the given File for read access, verifying the expected user/group * constraints if security is enabled. * - * Note that this function provides no additional checks if Hadoop + * @return Note that this function provides no additional checks if Hadoop * security is disabled, since doing the checks would be too expensive * when native libraries are not available. * @@ -201,8 +214,12 @@ public static FileInputStream openForRead(File f, String expectedOwner, } /** - * Same as openForRead() except that it will run even if security is off. + * @return Same as openForRead() except that it will run even if security is off. * This is used by unit tests. + * @param f input f. + * @param expectedOwner input expectedOwner. + * @param expectedGroup input expectedGroup. + * @throws IOException raised on errors performing I/O. */ @VisibleForTesting protected static FileInputStream forceSecureOpenForRead(File f, String expectedOwner, @@ -251,6 +268,7 @@ private static FileOutputStream insecureCreateForWrite(File f, * * @throws AlreadyExistsException if the file already exists * @throws IOException if any other error occurred + * @return createForWrite FileOutputStream. */ public static FileOutputStream createForWrite(File f, int permissions) throws IOException { diff --git a/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/io/SequenceFile.java b/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/io/SequenceFile.java index 890e7916ab076..a0b45814f1c77 100644 --- a/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/io/SequenceFile.java +++ b/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/io/SequenceFile.java @@ -269,7 +269,7 @@ static public void setDefaultCompressionType(Configuration job, * @param conf the configuration to use * @param opts the options to create the file with * @return a new Writer - * @throws IOException + * @throws IOException raised on errors performing I/O. */ public static Writer createWriter(Configuration conf, Writer.Option... opts ) throws IOException { @@ -301,7 +301,7 @@ public static Writer createWriter(Configuration conf, Writer.Option... opts * @param keyClass The 'key' type. * @param valClass The 'value' type. * @return Returns the handle to the constructed SequenceFile Writer. - * @throws IOException + * @throws IOException raised on errors performing I/O. * @deprecated Use {@link #createWriter(Configuration, Writer.Option...)} * instead. */ @@ -323,7 +323,7 @@ public static Writer createWriter(Configuration conf, Writer.Option... opts * @param valClass The 'value' type. * @param compressionType The compression type. * @return Returns the handle to the constructed SequenceFile Writer. - * @throws IOException + * @throws IOException raised on errors performing I/O. * @deprecated Use {@link #createWriter(Configuration, Writer.Option...)} * instead. */ @@ -348,7 +348,7 @@ public static Writer createWriter(Configuration conf, Writer.Option... opts * @param compressionType The compression type. * @param progress The Progressable object to track progress. * @return Returns the handle to the constructed SequenceFile Writer. - * @throws IOException + * @throws IOException raised on errors performing I/O. * @deprecated Use {@link #createWriter(Configuration, Writer.Option...)} * instead. */ @@ -375,7 +375,7 @@ public static Writer createWriter(Configuration conf, Writer.Option... opts * @param compressionType The compression type. * @param codec The compression codec. * @return Returns the handle to the constructed SequenceFile Writer. - * @throws IOException + * @throws IOException raised on errors performing I/O. * @deprecated Use {@link #createWriter(Configuration, Writer.Option...)} * instead. */ @@ -403,7 +403,7 @@ public static Writer createWriter(Configuration conf, Writer.Option... opts * @param progress The Progressable object to track progress. * @param metadata The metadata of the file. * @return Returns the handle to the constructed SequenceFile Writer. - * @throws IOException + * @throws IOException raised on errors performing I/O. * @deprecated Use {@link #createWriter(Configuration, Writer.Option...)} * instead. */ @@ -437,7 +437,7 @@ public static Writer createWriter(Configuration conf, Writer.Option... opts * @param progress The Progressable object to track progress. * @param metadata The metadata of the file. * @return Returns the handle to the constructed SequenceFile Writer. - * @throws IOException + * @throws IOException raised on errors performing I/O. * @deprecated Use {@link #createWriter(Configuration, Writer.Option...)} * instead. */ @@ -475,7 +475,7 @@ public static Writer createWriter(Configuration conf, Writer.Option... opts * @param codec The compression codec. * @param metadata The metadata of the file. * @return Returns the handle to the constructed SequenceFile Writer. - * @throws IOException + * @throws IOException raised on errors performing I/O. */ @Deprecated public static Writer @@ -508,7 +508,7 @@ public static Writer createWriter(Configuration conf, Writer.Option... opts * @param createFlag gives the semantics of create: overwrite, append etc. * @param opts file creation options; see {@link CreateOpts}. * @return Returns the handle to the constructed SequenceFile Writer. - * @throws IOException + * @throws IOException raised on errors performing I/O. */ public static Writer createWriter(FileContext fc, Configuration conf, Path name, @@ -532,7 +532,7 @@ public static Writer createWriter(Configuration conf, Writer.Option... opts * @param codec The compression codec. * @param progress The Progressable object to track progress. * @return Returns the handle to the constructed SequenceFile Writer. - * @throws IOException + * @throws IOException raised on errors performing I/O. * @deprecated Use {@link #createWriter(Configuration, Writer.Option...)} * instead. */ @@ -560,7 +560,7 @@ public static Writer createWriter(Configuration conf, Writer.Option... opts * @param codec The compression codec. * @param metadata The metadata of the file. * @return Returns the handle to the constructed SequenceFile Writer. - * @throws IOException + * @throws IOException raised on errors performing I/O. * @deprecated Use {@link #createWriter(Configuration, Writer.Option...)} * instead. */ @@ -585,7 +585,7 @@ public static Writer createWriter(Configuration conf, Writer.Option... opts * @param compressionType The compression type. * @param codec The compression codec. * @return Returns the handle to the constructed SequenceFile Writer. - * @throws IOException + * @throws IOException raised on errors performing I/O. * @deprecated Use {@link #createWriter(Configuration, Writer.Option...)} * instead. */ @@ -603,22 +603,26 @@ public static Writer createWriter(Configuration conf, Writer.Option... opts /** The interface to 'raw' values of SequenceFiles. */ public static interface ValueBytes { - /** Writes the uncompressed bytes to the outStream. + /** + * Writes the uncompressed bytes to the outStream. * @param outStream : Stream to write uncompressed bytes into. - * @throws IOException + * @throws IOException raised on errors performing I/O. */ public void writeUncompressedBytes(DataOutputStream outStream) throws IOException; - /** Write compressed bytes to outStream. + /** + * Write compressed bytes to outStream. * Note: that it will NOT compress the bytes if they are not compressed. * @param outStream : Stream to write compressed bytes into. + * @throws IllegalArgumentException an illegal or inappropriate argument. + * @throws IOException raised on errors performing I/O. */ public void writeCompressedBytes(DataOutputStream outStream) throws IllegalArgumentException, IOException; /** - * Size of stored data. + * @return Size of stored data. */ public int getSize(); } @@ -1190,10 +1194,17 @@ public static Option syncInterval(int value) { codec, metadata, syncInterval); } - /** Create the named file. + /** + * Create the named file. * @deprecated Use * {@link SequenceFile#createWriter(Configuration, Writer.Option...)} * instead. + * @param fs input filesystem. + * @param conf input configuration. + * @param name input name. + * @param keyClass input keyClass. + * @param valClass input valClass. + * @throws IOException raised on errors performing I/O. */ @Deprecated public Writer(FileSystem fs, Configuration conf, Path name, @@ -1203,10 +1214,19 @@ public Writer(FileSystem fs, Configuration conf, Path name, new Metadata(), SYNC_INTERVAL); } - /** Create the named file with write-progress reporter. + /** + * Create the named file with write-progress reporter. * @deprecated Use * {@link SequenceFile#createWriter(Configuration, Writer.Option...)} * instead. + * @param fs input filesystem. + * @param conf input configuration. + * @param name input name. + * @param keyClass input keyClass. + * @param valClass input valClass. + * @param progress input progress. + * @param metadata input metadata. + * @throws IOException raised on errors performing I/O. */ @Deprecated public Writer(FileSystem fs, Configuration conf, Path name, @@ -1217,10 +1237,22 @@ public Writer(FileSystem fs, Configuration conf, Path name, null, metadata, SYNC_INTERVAL); } - /** Create the named file with write-progress reporter. + /** + * Create the named file with write-progress reporter. * @deprecated Use * {@link SequenceFile#createWriter(Configuration, Writer.Option...)} * instead. + * @param fs input filesystem. + * @param conf input configuration. + * @param name input name. + * @param keyClass input keyClass. + * @param valClass input valClass. + * @param bufferSize input bufferSize. + * @param replication input replication. + * @param blockSize input blockSize. + * @param progress input progress. + * @param metadata input metadata. + * @throws IOException raised on errors performing I/O. */ @Deprecated public Writer(FileSystem fs, Configuration conf, Path name, @@ -1321,16 +1353,19 @@ void init(Configuration config, FSDataOutputStream outStream, } } - /** Returns the class of keys in this file. */ + /** @return Returns the class of keys in this file. */ public Class getKeyClass() { return keyClass; } - /** Returns the class of values in this file. */ + /** @return Returns the class of values in this file. */ public Class getValueClass() { return valClass; } - /** Returns the compression codec of data in this file. */ + /** @return Returns the compression codec of data in this file. */ public CompressionCodec getCompressionCodec() { return codec; } - /** create a sync point */ + /** + * create a sync point. + * @throws IOException raised on errors performing I/O. + */ public void sync() throws IOException { if (sync != null && lastSyncPos != out.getPos()) { out.writeInt(SYNC_ESCAPE); // mark the start of the sync @@ -1340,8 +1375,9 @@ public void sync() throws IOException { } /** - * flush all currently written data to the file system + * flush all currently written data to the file system. * @deprecated Use {@link #hsync()} or {@link #hflush()} instead + * @throws IOException raised on errors performing I/O. */ @Deprecated public void syncFs() throws IOException { @@ -1413,13 +1449,23 @@ synchronized void checkAndWriteSync() throws IOException { } } - /** Append a key/value pair. */ + /** + * Append a key/value pair. + * @param key input Writable key. + * @param val input Writable val. + * @throws IOException raised on errors performing I/O. + */ public void append(Writable key, Writable val) throws IOException { append((Object) key, (Object) val); } - /** Append a key/value pair. */ + /** + * Append a key/value pair. + * @param key input Object key. + * @param val input Object val. + * @throws IOException raised on errors performing I/O. + */ @SuppressWarnings("unchecked") public synchronized void append(Object key, Object val) throws IOException { @@ -1470,14 +1516,16 @@ public synchronized void appendRaw(byte[] keyData, int keyOffset, val.writeUncompressedBytes(out); // value } - /** Returns the current length of the output file. + /** @return Returns the current length of the output file. * *

    This always returns a synchronized position. In other words, * immediately after calling {@link SequenceFile.Reader#seek(long)} with a position * returned by this method, {@link SequenceFile.Reader#next(Writable)} may be called. However * the key may be earlier in the file than key last written when this * method was called (e.g., with block-compression, it may be the first key - * in the block that was being written when this method was called). + * in the block that was being written when this method was called).

    + * + * @throws IOException raised on errors performing I/O. */ public synchronized long getLength() throws IOException { return out.getPos(); @@ -1888,7 +1936,7 @@ public Reader(Configuration conf, Option... opts) throws IOException { * @param fs The file system used to open the file. * @param file The file being read. * @param conf Configuration - * @throws IOException + * @throws IOException raised on errors performing I/O. * @deprecated Use Reader(Configuration, Option...) instead. */ @Deprecated @@ -1904,7 +1952,7 @@ public Reader(FileSystem fs, Path file, * @param start The starting position. * @param length The length being read. * @param conf Configuration - * @throws IOException + * @throws IOException raised on errors performing I/O. * @deprecated Use Reader(Configuration, Reader.Option...) instead. */ @Deprecated @@ -1949,7 +1997,7 @@ private void initialize(Path filename, FSDataInputStream in, * @param length The length being read if it is {@literal >=} 0. * Otherwise, the length is not available. * @return The opened stream. - * @throws IOException + * @throws IOException raised on errors performing I/O. */ protected FSDataInputStream openFile(FileSystem fs, Path file, int bufferSize, long length) throws IOException { @@ -2139,12 +2187,12 @@ public synchronized void close() throws IOException { in.close(); } - /** Returns the name of the key class. */ + /** @return Returns the name of the key class. */ public String getKeyClassName() { return keyClassName; } - /** Returns the class of keys in this file. */ + /** @return Returns the class of keys in this file. */ public synchronized Class getKeyClass() { if (null == keyClass) { try { @@ -2156,12 +2204,12 @@ public synchronized Class getKeyClass() { return keyClass; } - /** Returns the name of the value class. */ + /** @return Returns the name of the value class. */ public String getValueClassName() { return valClassName; } - /** Returns the class of values in this file. */ + /** @return Returns the class of values in this file. */ public synchronized Class getValueClass() { if (null == valClass) { try { @@ -2173,13 +2221,22 @@ public synchronized Class getValueClass() { return valClass; } - /** Returns true if values are compressed. */ + /** + * Returns true if values are compressed. + * @return if values are compressed true, not false. + */ public boolean isCompressed() { return decompress; } - /** Returns true if records are block-compressed. */ + /** + * Returns true if records are block-compressed. + * @return if records are block-compressed true, not false. + */ public boolean isBlockCompressed() { return blockCompressed; } - /** Returns the compression codec of data in this file. */ + /** + * Returns the compression codec of data in this file. + * @return CompressionCodec. + */ public CompressionCodec getCompressionCodec() { return codec; } private byte[] getSync() { @@ -2202,7 +2259,10 @@ public CompressionType getCompressionType() { } } - /** Returns the metadata object of the file */ + /** + * Returns the metadata object of the file. + * @return metadata. + */ public Metadata getMetadata() { return this.metadata; } @@ -2311,7 +2371,7 @@ private synchronized void seekToCurrentValue() throws IOException { /** * Get the 'value' corresponding to the last read 'key'. * @param val : The 'value' to be read. - * @throws IOException + * @throws IOException raised on errors performing I/O. */ public synchronized void getCurrentValue(Writable val) throws IOException { @@ -2348,9 +2408,9 @@ public synchronized void getCurrentValue(Writable val) } /** - * Get the 'value' corresponding to the last read 'key'. + * @return Get the 'value' corresponding to the last read 'key'. * @param val : The 'value' to be read. - * @throws IOException + * @throws IOException raised on errors performing I/O. */ public synchronized Object getCurrentValue(Object val) throws IOException { @@ -2392,8 +2452,13 @@ private Object deserializeValue(Object val) throws IOException { return valDeserializer.deserialize(val); } - /** Read the next key in the file into key, skipping its - * value. True if another entry exists, and false at end of file. */ + /** + * @return Read the next key in the file into key, skipping its + * value.True if another entry exists, and false at end of file. + * + * @param key key. + * @throws IOException raised on errors performing I/O. + */ public synchronized boolean next(Writable key) throws IOException { if (key.getClass() != getKeyClass()) throw new IOException("wrong key class: "+key.getClass().getName() @@ -2440,9 +2505,16 @@ public synchronized boolean next(Writable key) throws IOException { return true; } - /** Read the next key/value pair in the file into key and - * val. Returns true if such a pair exists and false when at - * end of file */ + /** + * Read the next key/value pair in the file into key and + * val. + * @return Returns true if such a pair exists and false when at + * end of file. + * + * @param key input key. + * @param val input val. + * @throws IOException raised on errors performing I/O. + */ public synchronized boolean next(Writable key, Writable val) throws IOException { if (val.getClass() != getValueClass()) @@ -2526,7 +2598,7 @@ public ValueBytes createValueBytes() { * @param key - The buffer into which the key is read * @param val - The 'raw' value * @return Returns the total record length or -1 for end of file - * @throws IOException + * @throws IOException raised on errors performing I/O. */ public synchronized int nextRaw(DataOutputBuffer key, ValueBytes val) throws IOException { @@ -2585,7 +2657,7 @@ public synchronized int nextRaw(DataOutputBuffer key, ValueBytes val) * Read 'raw' keys. * @param key - The buffer into which the key is read * @return Returns the key length or -1 for end of file - * @throws IOException + * @throws IOException raised on errors performing I/O. */ public synchronized int nextRawKey(DataOutputBuffer key) throws IOException { @@ -2624,8 +2696,14 @@ public synchronized int nextRawKey(DataOutputBuffer key) } - /** Read the next key in the file, skipping its - * value. Return null at end of file. */ + /** + * Read the next key in the file, skipping its + * value. + * + * @param key input Object key. + * @throws IOException raised on errors performing I/O. + * @return Return null at end of file. + */ public synchronized Object next(Object key) throws IOException { if (key != null && key.getClass() != getKeyClass()) { throw new IOException("wrong key class: "+key.getClass().getName() @@ -2682,7 +2760,7 @@ private Object deserializeKey(Object key) throws IOException { * Read 'raw' values. * @param val - The 'raw' value * @return Returns the value length - * @throws IOException + * @throws IOException raised on errors performing I/O. */ public synchronized int nextRawValue(ValueBytes val) throws IOException { @@ -2722,16 +2800,20 @@ private void handleChecksumException(ChecksumException e) } } - /** disables sync. often invoked for tmp files */ + /** disables sync. often invoked for tmp files. */ synchronized void ignoreSync() { sync = null; } - /** Set the current byte position in the input file. + /** + * Set the current byte position in the input file. * *

    The position passed must be a position returned by {@link * SequenceFile.Writer#getLength()} when writing this file. To seek to an arbitrary - * position, use {@link SequenceFile.Reader#sync(long)}. + * position, use {@link SequenceFile.Reader#sync(long)}.

    + * + * @param position input position. + * @throws IOException raised on errors performing I/O. */ public synchronized void seek(long position) throws IOException { in.seek(position); @@ -2741,7 +2823,11 @@ public synchronized void seek(long position) throws IOException { } } - /** Seek to the next sync mark past a given position.*/ + /** + * Seek to the next sync mark past a given position. + * @param position position. + * @throws IOException raised on errors performing I/O. + */ public synchronized void sync(long position) throws IOException { if (position+SYNC_SIZE >= end) { seek(end); @@ -2777,10 +2863,13 @@ public synchronized void sync(long position) throws IOException { } } - /** Returns true iff the previous call to next passed a sync mark.*/ + /** @return Returns true iff the previous call to next passed a sync mark.*/ public synchronized boolean syncSeen() { return syncSeen; } - /** Return the current byte position in the input file. */ + /** + * @return Return the current byte position in the input file. + * @throws IOException raised on errors performing I/O. + */ public synchronized long getPosition() throws IOException { return in.getPos(); } @@ -2822,19 +2911,40 @@ public static class Sorter { private Progressable progressable = null; - /** Sort and merge files containing the named classes. */ + /** + * Sort and merge files containing the named classes. + * @param fs input FileSystem. + * @param keyClass input keyClass. + * @param valClass input valClass. + * @param conf input Configuration. + */ public Sorter(FileSystem fs, Class keyClass, Class valClass, Configuration conf) { this(fs, WritableComparator.get(keyClass, conf), keyClass, valClass, conf); } - /** Sort and merge using an arbitrary {@link RawComparator}. */ + /** + * Sort and merge using an arbitrary {@link RawComparator}. + * @param fs input FileSystem. + * @param comparator input RawComparator. + * @param keyClass input keyClass. + * @param valClass input valClass. + * @param conf input Configuration. + */ public Sorter(FileSystem fs, RawComparator comparator, Class keyClass, Class valClass, Configuration conf) { this(fs, comparator, keyClass, valClass, conf, new Metadata()); } - /** Sort and merge using an arbitrary {@link RawComparator}. */ + /** + * Sort and merge using an arbitrary {@link RawComparator}. + * @param fs input FileSystem. + * @param comparator input RawComparator. + * @param keyClass input keyClass. + * @param valClass input valClass. + * @param conf input Configuration. + * @param metadata input metadata. + */ @SuppressWarnings("deprecation") public Sorter(FileSystem fs, RawComparator comparator, Class keyClass, Class valClass, Configuration conf, Metadata metadata) { @@ -2863,19 +2973,28 @@ public Sorter(FileSystem fs, RawComparator comparator, Class keyClass, this.metadata = metadata; } - /** Set the number of streams to merge at once.*/ + /** + * Set the number of streams to merge at once. + * @param factor factor. + */ public void setFactor(int factor) { this.factor = factor; } - /** Get the number of streams to merge at once.*/ + /** @return Get the number of streams to merge at once.*/ public int getFactor() { return factor; } - /** Set the total amount of buffer memory, in bytes.*/ + /** + * Set the total amount of buffer memory, in bytes. + * @param memory buffer memory. + */ public void setMemory(int memory) { this.memory = memory; } - /** Get the total amount of buffer memory, in bytes.*/ + /** @return Get the total amount of buffer memory, in bytes.*/ public int getMemory() { return memory; } - /** Set the progressable object in order to report progress. */ + /** + * Set the progressable object in order to report progress. + * @param progressable input Progressable. + */ public void setProgressable(Progressable progressable) { this.progressable = progressable; } @@ -2885,6 +3004,7 @@ public void setProgressable(Progressable progressable) { * @param inFiles the files to be sorted * @param outFile the sorted output file * @param deleteInput should the input files be deleted as they are read? + * @throws IOException raised on errors performing I/O. */ public void sort(Path[] inFiles, Path outFile, boolean deleteInput) throws IOException { @@ -2907,6 +3027,7 @@ public void sort(Path[] inFiles, Path outFile, * @param tempDir the directory where temp files are created during sort * @param deleteInput should the input files be deleted as they are read? * @return iterator the RawKeyValueIterator + * @throws IOException raised on errors performing I/O. */ public RawKeyValueIterator sortAndIterate(Path[] inFiles, Path tempDir, boolean deleteInput) throws IOException { @@ -2932,8 +3053,9 @@ else if (segments == 1) /** * The backwards compatible interface to sort. - * @param inFile the input file to sort - * @param outFile the sorted output file + * @param inFile the input file to sort. + * @param outFile the sorted output file. + * @throws IOException raised on errors performing I/O. */ public void sort(Path inFile, Path outFile) throws IOException { sort(new Path[]{inFile}, outFile, false); @@ -3151,27 +3273,32 @@ public void setProgressable(Progressable progressable) /** The interface to iterate over raw keys/values of SequenceFiles. */ public static interface RawKeyValueIterator { - /** Gets the current raw key + /** + * Gets the current raw key. * @return DataOutputBuffer - * @throws IOException + * @throws IOException raised on errors performing I/O. */ DataOutputBuffer getKey() throws IOException; - /** Gets the current raw value + /** + * Gets the current raw value. * @return ValueBytes - * @throws IOException + * @throws IOException raised on errors performing I/O. */ ValueBytes getValue() throws IOException; - /** Sets up the current key and value (for getKey and getValue) + /** + * Sets up the current key and value (for getKey and getValue). * @return true if there exists a key/value, false otherwise - * @throws IOException + * @throws IOException raised on errors performing I/O. */ boolean next() throws IOException; - /** closes the iterator so that the underlying streams can be closed - * @throws IOException + /** + * closes the iterator so that the underlying streams can be closed. + * @throws IOException raised on errors performing I/O. */ void close() throws IOException; - /** Gets the Progress object; this has a float (0.0 - 1.0) - * indicating the bytes processed by the iterator so far + /** + * @return Gets the Progress object; this has a float (0.0 - 1.0) + * indicating the bytes processed by the iterator so far. */ Progress getProgress(); } @@ -3181,7 +3308,7 @@ public static interface RawKeyValueIterator { * @param segments the list of SegmentDescriptors * @param tmpDir the directory to write temporary files into * @return RawKeyValueIterator - * @throws IOException + * @throws IOException raised on errors performing I/O. */ public RawKeyValueIterator merge(List segments, Path tmpDir) @@ -3199,7 +3326,7 @@ public RawKeyValueIterator merge(List segments, * unnecessary * @param tmpDir the directory to write temporary files into * @return RawKeyValueIteratorMergeQueue - * @throws IOException + * @throws IOException raised on errors performing I/O. */ public RawKeyValueIterator merge(Path [] inNames, boolean deleteInputs, Path tmpDir) @@ -3217,7 +3344,7 @@ public RawKeyValueIterator merge(Path [] inNames, boolean deleteInputs, * @param factor the factor that will be used as the maximum merge fan-in * @param tmpDir the directory to write temporary files into * @return RawKeyValueIteratorMergeQueue - * @throws IOException + * @throws IOException raised on errors performing I/O. */ public RawKeyValueIterator merge(Path [] inNames, boolean deleteInputs, int factor, Path tmpDir) @@ -3243,7 +3370,7 @@ public RawKeyValueIterator merge(Path [] inNames, boolean deleteInputs, * @param deleteInputs true if the input files should be deleted when * unnecessary * @return RawKeyValueIteratorMergeQueue - * @throws IOException + * @throws IOException raised on errors performing I/O. */ public RawKeyValueIterator merge(Path [] inNames, Path tempDir, boolean deleteInputs) @@ -3274,7 +3401,7 @@ public RawKeyValueIterator merge(Path [] inNames, Path tempDir, * @param outputFile the path of the output file * @param prog the Progressable to report status during the file write * @return Writer - * @throws IOException + * @throws IOException raised on errors performing I/O. */ public Writer cloneFileAttributes(Path inputFile, Path outputFile, Progressable prog) throws IOException { @@ -3296,10 +3423,10 @@ public Writer cloneFileAttributes(Path inputFile, Path outputFile, /** * Writes records from RawKeyValueIterator into a file represented by the - * passed writer + * passed writer. * @param records the RawKeyValueIterator * @param writer the Writer created earlier - * @throws IOException + * @throws IOException raised on errors performing I/O. */ public void writeFile(RawKeyValueIterator records, Writer writer) throws IOException { @@ -3313,7 +3440,7 @@ public void writeFile(RawKeyValueIterator records, Writer writer) /** Merge the provided files. * @param inFiles the array of input path names * @param outFile the final output file - * @throws IOException + * @throws IOException raised on errors performing I/O. */ public void merge(Path[] inFiles, Path outFile) throws IOException { if (fs.exists(outFile)) { @@ -3649,10 +3776,13 @@ public SegmentDescriptor (long segmentOffset, long segmentLength, this.segmentPathName = segmentPathName; } - /** Do the sync checks */ + /** Do the sync checks. */ public void doSync() {ignoreSync = false;} - /** Whether to delete the files when no longer needed */ + /** + * Whether to delete the files when no longer needed. + * @param preserve input boolean preserve. + */ public void preserveInput(boolean preserve) { preserveInput = preserve; } @@ -3694,9 +3824,10 @@ public int hashCode() { return 37 * 17 + (int) (segmentOffset^(segmentOffset>>>32)); } - /** Fills up the rawKey object with the key returned by the Reader + /** + * Fills up the rawKey object with the key returned by the Reader. * @return true if there is a key returned; false, otherwise - * @throws IOException + * @throws IOException raised on errors performing I/O. */ public boolean nextRawKey() throws IOException { if (in == null) { @@ -3725,18 +3856,19 @@ public boolean nextRawKey() throws IOException { return (keyLength >= 0); } - /** Fills up the passed rawValue with the value corresponding to the key - * read earlier - * @param rawValue + /** + * Fills up the passed rawValue with the value corresponding to the key + * read earlier. + * @param rawValue input ValueBytes rawValue. * @return the length of the value - * @throws IOException + * @throws IOException raised on errors performing I/O. */ public int nextRawValue(ValueBytes rawValue) throws IOException { int valLength = in.nextRawValue(rawValue); return valLength; } - /** Returns the stored rawKey */ + /** @return Returns the stored rawKey */ public DataOutputBuffer getKey() { return rawKey; } @@ -3747,8 +3879,10 @@ private void close() throws IOException { this.in = null; } - /** The default cleanup. Subclasses can override this with a custom - * cleanup + /** + * The default cleanup. Subclasses can override this with a custom + * cleanup. + * @throws IOException raised on errors performing I/O. */ public void cleanup() throws IOException { close(); diff --git a/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/io/SetFile.java b/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/io/SetFile.java index 118cce75136ed..de75810df0f70 100644 --- a/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/io/SetFile.java +++ b/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/io/SetFile.java @@ -39,15 +39,29 @@ protected SetFile() {} // no public ctor */ public static class Writer extends MapFile.Writer { - /** Create the named set for keys of the named class. - * @deprecated pass a Configuration too + /** + * Create the named set for keys of the named class. + * @deprecated pass a Configuration too + * @param fs input FileSystem. + * @param dirName input dirName. + * @param keyClass input keyClass. + * @throws IOException raised on errors performing I/O. */ public Writer(FileSystem fs, String dirName, Class keyClass) throws IOException { super(new Configuration(), fs, dirName, keyClass, NullWritable.class); } - /** Create a set naming the element class and compression type. */ + /** + * Create a set naming the element class and compression type. + * + * @param conf input Configuration. + * @param fs input FileSystem. + * @param dirName input dirName. + * @param keyClass input keyClass. + * @param compress input compress. + * @throws IOException raised on errors performing I/O. + */ public Writer(Configuration conf, FileSystem fs, String dirName, Class keyClass, SequenceFile.CompressionType compress) @@ -55,7 +69,16 @@ public Writer(Configuration conf, FileSystem fs, String dirName, this(conf, fs, dirName, WritableComparator.get(keyClass, conf), compress); } - /** Create a set naming the element comparator and compression type. */ + /** + * Create a set naming the element comparator and compression type. + * + * @param conf input Configuration. + * @param fs input FileSystem. + * @param dirName input dirName. + * @param comparator input comparator. + * @param compress input compress. + * @throws IOException raised on errors performing I/O. + */ public Writer(Configuration conf, FileSystem fs, String dirName, WritableComparator comparator, SequenceFile.CompressionType compress) throws IOException { @@ -65,8 +88,12 @@ public Writer(Configuration conf, FileSystem fs, String dirName, compression(compress)); } - /** Append a key to a set. The key must be strictly greater than the - * previous key added to the set. */ + /** + * Append a key to a set. The key must be strictly greater than the + * previous key added to the set. + * @param key input key. + * @throws IOException raised on errors performing I/O. + */ public void append(WritableComparable key) throws IOException{ append(key, NullWritable.get()); } @@ -75,12 +102,25 @@ public void append(WritableComparable key) throws IOException{ /** Provide access to an existing set file. */ public static class Reader extends MapFile.Reader { - /** Construct a set reader for the named set.*/ + /** + * Construct a set reader for the named set. + * @param fs input FileSystem. + * @param dirName input dirName. + * @param conf input Configuration. + * @throws IOException raised on errors performing I/O. + */ public Reader(FileSystem fs, String dirName, Configuration conf) throws IOException { super(fs, dirName, conf); } - /** Construct a set reader for the named set using the named comparator.*/ + /** + * Construct a set reader for the named set using the named comparator. + * @param fs input FileSystem. + * @param dirName input dirName. + * @param comparator input comparator. + * @param conf input Configuration. + * @throws IOException raised on errors performing I/O. + */ public Reader(FileSystem fs, String dirName, WritableComparator comparator, Configuration conf) throws IOException { super(new Path(dirName), conf, comparator(comparator)); @@ -93,15 +133,26 @@ public boolean seek(WritableComparable key) return super.seek(key); } - /** Read the next key in a set into key. Returns - * true if such a key exists and false when at the end of the set. */ + /** + * Read the next key in a set into key. + * + * @param key input key. + * @return Returns true if such a key exists + * and false when at the end of the set. + * @throws IOException raised on errors performing I/O. + */ public boolean next(WritableComparable key) throws IOException { return next(key, NullWritable.get()); } - /** Read the matching key from a set into key. - * Returns key, or null if no match exists. */ + /** + * Read the matching key from a set into key. + * + * @param key input key. + * @return Returns key, or null if no match exists. + * @throws IOException raised on errors performing I/O. + */ public WritableComparable get(WritableComparable key) throws IOException { if (seek(key)) { diff --git a/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/io/ShortWritable.java b/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/io/ShortWritable.java index be09df18017b7..96e6cacae8773 100644 --- a/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/io/ShortWritable.java +++ b/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/io/ShortWritable.java @@ -38,12 +38,15 @@ public ShortWritable(short value) { set(value); } - /** Set the value of this ShortWritable. */ + /** + * Set the value of this ShortWritable. + * @param value input value. + */ public void set(short value) { this.value = value; } - /** Return the value of this ShortWritable. */ + /** @return Return the value of this ShortWritable. */ public short get() { return value; } diff --git a/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/io/Text.java b/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/io/Text.java index 5ca7f3c84cab0..86fb1ff9a54a2 100644 --- a/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/io/Text.java +++ b/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/io/Text.java @@ -90,6 +90,7 @@ public Text() { /** * Construct from a string. + * @param string input string. */ public Text(String string) { set(string); @@ -97,6 +98,7 @@ public Text(String string) { /** * Construct from another text. + * @param utf8 input utf8. */ public Text(Text utf8) { set(utf8); @@ -104,13 +106,15 @@ public Text(Text utf8) { /** * Construct from a byte array. + * + * @param utf8 input utf8. */ public Text(byte[] utf8) { set(utf8); } /** - * Get a copy of the bytes that is exactly the length of the data. + * @return Get a copy of the bytes that is exactly the length of the data. * See {@link #getBytes()} for faster access to the underlying array. */ public byte[] copyBytes() { @@ -136,7 +140,7 @@ public int getLength() { } /** - * Returns the length of this text. The length is equal to the number of + * @return Returns the length of this text. The length is equal to the number of * Unicode code units in the text. */ public int getTextLength() { @@ -149,7 +153,9 @@ public int getTextLength() { /** * Returns the Unicode Scalar Value (32-bit integer value) * for the character at position. Note that this - * method avoids using the converter or doing String instantiation + * method avoids using the converter or doing String instantiation. + * + * @param position input position. * @return the Unicode scalar value at position or -1 * if the position is invalid or points to a * trailing byte @@ -172,6 +178,9 @@ public int find(String what) { * position is measured in bytes and the return value is in * terms of byte position in the buffer. The backing buffer is * not converted to a string for this operation. + * + * @param what input what. + * @param start input start. * @return byte position of the first occurrence of the search * string in the UTF-8 buffer or -1 if not found */ @@ -213,6 +222,8 @@ public int find(String what, int start) { /** * Set to contain the contents of a string. + * + * @param string input string. */ public void set(String string) { try { @@ -229,6 +240,8 @@ public void set(String string) { * Set to a utf8 byte array. If the length of utf8 is * zero, actually clear {@link #bytes} and any existing * data is lost. + * + * @param utf8 input utf8. */ public void set(byte[] utf8) { if (utf8.length == 0) { @@ -242,6 +255,7 @@ public void set(byte[] utf8) { /** * Copy a text. + * @param other other. */ public void set(Text other) { set(other.getBytes(), 0, other.getLength()); @@ -349,6 +363,8 @@ public void readFields(DataInput in, int maxLength) throws IOException { /** * Skips over one Text in the input. + * @param in input in. + * @throws IOException raised on errors performing I/O. */ public static void skip(DataInput in) throws IOException { int length = WritableUtils.readVInt(in); @@ -359,6 +375,10 @@ public static void skip(DataInput in) throws IOException { * Read a Text object whose length is already known. * This allows creating Text from a stream which uses a different serialization * format. + * + * @param in input in. + * @param len input len. + * @throws IOException raised on errors performing I/O. */ public void readWithKnownLength(DataInput in, int len) throws IOException { ensureCapacity(len); @@ -426,9 +446,13 @@ public int compare(byte[] b1, int s1, int l1, /// STATIC UTILITIES FROM HERE DOWN /** - * Converts the provided byte array to a String using the + * @return Converts the provided byte array to a String using the * UTF-8 encoding. If the input is malformed, * replace by a default value. + * + * @param utf8 input utf8. + * @throws CharacterCodingException when a character + * encoding or decoding error occurs. */ public static String decode(byte[] utf8) throws CharacterCodingException { return decode(ByteBuffer.wrap(utf8), true); @@ -440,11 +464,18 @@ public static String decode(byte[] utf8, int start, int length) } /** - * Converts the provided byte array to a String using the + * @return Converts the provided byte array to a String using the * UTF-8 encoding. If replace is true, then * malformed input is replaced with the * substitution character, which is U+FFFD. Otherwise the * method throws a MalformedInputException. + * + * @param utf8 input utf8. + * @param start input start. + * @param length input length. + * @param replace input replace. + * @throws CharacterCodingException when a character + * encoding or decoding error occurs. */ public static String decode(byte[] utf8, int start, int length, boolean replace) throws CharacterCodingException { @@ -472,8 +503,12 @@ private static String decode(ByteBuffer utf8, boolean replace) * Converts the provided String to bytes using the * UTF-8 encoding. If the input is malformed, * invalid chars are replaced by a default value. + * + * @param string input string. * @return ByteBuffer: bytes stores at ByteBuffer.array() * and length is ByteBuffer.limit() + * @throws CharacterCodingException when a character + * encoding or decoding error occurs. */ public static ByteBuffer encode(String string) @@ -487,8 +522,13 @@ public static ByteBuffer encode(String string) * malformed input is replaced with the * substitution character, which is U+FFFD. Otherwise the * method throws a MalformedInputException. + * + * @param string input string. + * @param replace input replace. * @return ByteBuffer: bytes stores at ByteBuffer.array() * and length is ByteBuffer.limit() + * @throws CharacterCodingException when a character + * encoding or decoding error occurs. */ public static ByteBuffer encode(String string, boolean replace) throws CharacterCodingException { @@ -508,13 +548,20 @@ public static ByteBuffer encode(String string, boolean replace) static final public int DEFAULT_MAX_LEN = 1024 * 1024; - /** Read a UTF8 encoded string from in + /** + * @return Read a UTF8 encoded string from in. + * @param in input in. + * @throws IOException raised on errors performing I/O. */ public static String readString(DataInput in) throws IOException { return readString(in, Integer.MAX_VALUE); } - /** Read a UTF8 encoded string with a maximum size + /** + * @return Read a UTF8 encoded string with a maximum size. + * @param in input datainput. + * @param maxLength input maxLength. + * @throws IOException raised on errors performing I/O. */ public static String readString(DataInput in, int maxLength) throws IOException { @@ -526,6 +573,11 @@ public static String readString(DataInput in, int maxLength) /** * Write a UTF8 encoded string to out. + * + * @param out input out. + * @param s input s. + * @throws IOException raised on errors performing I/O. + * @return a UTF8 encoded string to out. */ public static int writeString(DataOutput out, String s) throws IOException { ByteBuffer bytes = encode(s); @@ -536,7 +588,12 @@ public static int writeString(DataOutput out, String s) throws IOException { } /** - * Write a UTF8 encoded string with a maximum size to out. + * @return Write a UTF8 encoded string with a maximum size to out. + * + * @param out input out. + * @param s input s. + * @param maxLength input maxLength. + * @throws IOException raised on errors performing I/O. */ public static int writeString(DataOutput out, String s, int maxLength) throws IOException { @@ -670,9 +727,11 @@ public static void validateUTF8(byte[] utf8, int start, int len) 3, 3, 3, 3, 3, 3, 3, 4, 4, 4, 4, 5, 5, 5, 5 }; /** - * Returns the next code point at the current position in + * @return Returns the next code point at the current position in * the buffer. The buffer's position will be incremented. * Any mark set on this buffer will be changed by this method! + * + * @param bytes input bytes. */ public static int bytesToCodePoint(ByteBuffer bytes) { bytes.mark(); diff --git a/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/io/UTF8.java b/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/io/UTF8.java index f5d33a13005d7..fdee830e6fea8 100644 --- a/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/io/UTF8.java +++ b/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/io/UTF8.java @@ -63,27 +63,36 @@ public UTF8() { //set(""); } - /** Construct from a given string. */ + /** + * Construct from a given string. + * @param string input string. + */ public UTF8(String string) { set(string); } - /** Construct from a given string. */ + /** + * Construct from a given string. + * @param utf8 input utf8. + */ public UTF8(UTF8 utf8) { set(utf8); } - /** The raw bytes. */ + /** @return The raw bytes. */ public byte[] getBytes() { return bytes; } - /** The number of bytes in the encoded string. */ + /** @return The number of bytes in the encoded string. */ public int getLength() { return length; } - /** Set to contain the contents of a string. */ + /** + * Set to contain the contents of a string. + * @param string input string. + */ public void set(String string) { if (string.length() > 0xffff/3) { // maybe too long LOG.warn("truncating long string: " + string.length() @@ -108,7 +117,10 @@ public void set(String string) { } } - /** Set to contain the contents of a string. */ + /** + * Set to contain the contents of a string. + * @param other input other. + */ public void set(UTF8 other) { length = other.length; if (bytes == null || length > bytes.length) // grow buffer @@ -124,7 +136,11 @@ public void readFields(DataInput in) throws IOException { in.readFully(bytes, 0, length); } - /** Skips over one UTF8 in the input. */ + /** + * Skips over one UTF8 in the input. + * @param in datainput. + * @throws IOException raised on errors performing I/O. + */ public static void skip(DataInput in) throws IOException { int length = in.readUnsignedShort(); WritableUtils.skipFully(in, length); @@ -214,8 +230,10 @@ public int compare(byte[] b1, int s1, int l1, /// These are probably not used much anymore, and might be removed... - /** Convert a string to a UTF-8 encoded byte array. + /** + * @return Convert a string to a UTF-8 encoded byte array. * @see String#getBytes(String) + * @param string input string. */ public static byte[] getBytes(String string) { byte[] result = new byte[utf8Length(string)]; @@ -231,8 +249,9 @@ public static byte[] getBytes(String string) { } /** - * Convert a UTF-8 encoded byte array back into a string. + * @return Convert a UTF-8 encoded byte array back into a string. * + * @param bytes input bytes. * @throws IOException if the byte array is invalid UTF8 */ public static String fromBytes(byte[] bytes) throws IOException { @@ -243,9 +262,12 @@ public static String fromBytes(byte[] bytes) throws IOException { return buf.toString(); } - /** Read a UTF-8 encoded string. + /** + * @return Read a UTF-8 encoded string. * * @see DataInput#readUTF() + * @param in DataInput. + * @throws IOException raised on errors performing I/O. */ public static String readString(DataInput in) throws IOException { int bytes = in.readUnsignedShort(); @@ -318,9 +340,13 @@ private static char lowSurrogate(int codePoint) { return (char) ((codePoint & 0x3ff) + Character.MIN_LOW_SURROGATE); } - /** Write a UTF-8 encoded string. + /** + * @return Write a UTF-8 encoded string. * * @see DataOutput#writeUTF(String) + * @param out input out. + * @param s input s. + * @throws IOException raised on errors performing I/O. */ public static int writeString(DataOutput out, String s) throws IOException { if (s.length() > 0xffff/3) { // maybe too long diff --git a/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/io/VIntWritable.java b/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/io/VIntWritable.java index f537524c4b40a..7d3f680858ec3 100644 --- a/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/io/VIntWritable.java +++ b/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/io/VIntWritable.java @@ -37,10 +37,13 @@ public VIntWritable() {} public VIntWritable(int value) { set(value); } - /** Set the value of this VIntWritable. */ + /** + * Set the value of this VIntWritable. + * @param value input value. + */ public void set(int value) { this.value = value; } - /** Return the value of this VIntWritable. */ + /** @return Return the value of this VIntWritable. */ public int get() { return value; } @Override diff --git a/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/io/VLongWritable.java b/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/io/VLongWritable.java index a9fac30605be6..a72a7fc2fd39a 100644 --- a/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/io/VLongWritable.java +++ b/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/io/VLongWritable.java @@ -37,10 +37,13 @@ public VLongWritable() {} public VLongWritable(long value) { set(value); } - /** Set the value of this LongWritable. */ + /** + * Set the value of this LongWritable. + * @param value input value. + */ public void set(long value) { this.value = value; } - /** Return the value of this LongWritable. */ + /** @return Return the value of this LongWritable. */ public long get() { return value; } @Override diff --git a/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/io/VersionedWritable.java b/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/io/VersionedWritable.java index c2db55520c918..421b8daeeae8e 100644 --- a/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/io/VersionedWritable.java +++ b/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/io/VersionedWritable.java @@ -36,7 +36,7 @@ @InterfaceStability.Stable public abstract class VersionedWritable implements Writable { - /** Return the version number of the current implementation. */ + /** @return Return the version number of the current implementation. */ public abstract byte getVersion(); // javadoc from Writable diff --git a/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/io/WeakReferencedElasticByteBufferPool.java b/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/io/WeakReferencedElasticByteBufferPool.java new file mode 100644 index 0000000000000..c71c44e798a65 --- /dev/null +++ b/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/io/WeakReferencedElasticByteBufferPool.java @@ -0,0 +1,155 @@ +/** + * 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.io; + +import java.lang.ref.WeakReference; +import java.nio.ByteBuffer; +import java.util.Map; +import java.util.TreeMap; + +import org.apache.hadoop.classification.InterfaceAudience; +import org.apache.hadoop.classification.InterfaceStability; +import org.apache.hadoop.classification.VisibleForTesting; + +/** + * Buffer pool implementation which uses weak references to store + * buffers in the pool, such that they are garbage collected when + * there are no references to the buffer during a gc run. This is + * important as direct buffers don't get garbage collected automatically + * during a gc run as they are not stored on heap memory. + * Also the buffers are stored in a tree map which helps in returning + * smallest buffer whose size is just greater than requested length. + * This is a thread safe implementation. + */ +@InterfaceAudience.Private +@InterfaceStability.Unstable +public final class WeakReferencedElasticByteBufferPool extends ElasticByteBufferPool { + + /** + * Map to store direct byte buffers of different sizes in the pool. + * Used tree map such that we can return next greater than capacity + * buffer if buffer with exact capacity is unavailable. + * This must be accessed in synchronized blocks. + */ + private final TreeMap> directBuffers = + new TreeMap<>(); + + /** + * Map to store heap based byte buffers of different sizes in the pool. + * Used tree map such that we can return next greater than capacity + * buffer if buffer with exact capacity is unavailable. + * This must be accessed in synchronized blocks. + */ + private final TreeMap> heapBuffers = + new TreeMap<>(); + + /** + * Method to get desired buffer tree. + * @param isDirect whether the buffer is heap based or direct. + * @return corresponding buffer tree. + */ + private TreeMap> getBufferTree(boolean isDirect) { + return isDirect + ? directBuffers + : heapBuffers; + } + + /** + * {@inheritDoc} + * + * @param direct whether we want a direct byte buffer or a heap one. + * @param length length of requested buffer. + * @return returns equal or next greater than capacity buffer from + * pool if already available and not garbage collected else creates + * a new buffer and return it. + */ + @Override + public synchronized ByteBuffer getBuffer(boolean direct, int length) { + TreeMap> buffersTree = getBufferTree(direct); + + // Scan the entire tree and remove all weak null references. + buffersTree.entrySet().removeIf(next -> next.getValue().get() == null); + + Map.Entry> entry = + buffersTree.ceilingEntry(new Key(length, 0)); + // If there is no buffer present in the pool with desired size. + if (entry == null) { + return direct ? ByteBuffer.allocateDirect(length) : + ByteBuffer.allocate(length); + } + // buffer is available in the pool and not garbage collected. + WeakReference bufferInPool = entry.getValue(); + buffersTree.remove(entry.getKey()); + ByteBuffer buffer = bufferInPool.get(); + if (buffer != null) { + return buffer; + } + // buffer was in pool but already got garbage collected. + return direct + ? ByteBuffer.allocateDirect(length) + : ByteBuffer.allocate(length); + } + + /** + * Return buffer to the pool. + * @param buffer buffer to be returned. + */ + @Override + public synchronized void putBuffer(ByteBuffer buffer) { + buffer.clear(); + TreeMap> buffersTree = getBufferTree(buffer.isDirect()); + // Buffers are indexed by (capacity, time). + // If our key is not unique on the first try, we try again, since the + // time will be different. Since we use nanoseconds, it's pretty + // unlikely that we'll loop even once, unless the system clock has a + // poor granularity or multi-socket systems have clocks slightly out + // of sync. + while (true) { + Key keyToInsert = new Key(buffer.capacity(), System.nanoTime()); + if (!buffersTree.containsKey(keyToInsert)) { + buffersTree.put(keyToInsert, new WeakReference<>(buffer)); + return; + } + } + } + + /** + * Clear the buffer pool thus releasing all the buffers. + * The caller must remove all references of + * existing buffers before calling this method to avoid + * memory leaks. + */ + @Override + public synchronized void release() { + heapBuffers.clear(); + directBuffers.clear(); + } + + /** + * Get current buffers count in the pool. + * @param isDirect whether we want to count the heap or direct buffers. + * @return count of buffers. + */ + @VisibleForTesting + public synchronized int getCurrentBuffersCount(boolean isDirect) { + return isDirect + ? directBuffers.size() + : heapBuffers.size(); + } +} diff --git a/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/io/Writable.java b/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/io/Writable.java index b94de6c3c72bd..56b46d554fee6 100644 --- a/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/io/Writable.java +++ b/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/io/Writable.java @@ -71,7 +71,7 @@ public interface Writable { * Serialize the fields of this object to out. * * @param out DataOuput to serialize this object into. - * @throws IOException + * @throws IOException any other problem for write. */ void write(DataOutput out) throws IOException; @@ -82,7 +82,7 @@ public interface Writable { * existing object where possible.

    * * @param in DataInput to deseriablize this object from. - * @throws IOException + * @throws IOException any other problem for readFields. */ void readFields(DataInput in) throws IOException; } diff --git a/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/io/WritableComparator.java b/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/io/WritableComparator.java index 1754b8d06f6fa..05d4e3c5c533f 100644 --- a/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/io/WritableComparator.java +++ b/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/io/WritableComparator.java @@ -46,12 +46,22 @@ public class WritableComparator implements RawComparator, Configurable { private Configuration conf; - /** For backwards compatibility. **/ + /** + * For backwards compatibility. + * + * @param c WritableComparable Type. + * @return WritableComparator. + */ public static WritableComparator get(Class c) { return get(c, null); } - /** Get a comparator for a {@link WritableComparable} implementation. */ + /** + * Get a comparator for a {@link WritableComparable} implementation. + * @param c class. + * @param conf configuration. + * @return WritableComparator. + */ public static WritableComparator get( Class c, Configuration conf) { WritableComparator comparator = comparators.get(c); @@ -95,9 +105,13 @@ private static void forceInit(Class cls) { } } - /** Register an optimized comparator for a {@link WritableComparable} + /** + * Register an optimized comparator for a {@link WritableComparable} * implementation. Comparators registered with this method must be - * thread-safe. */ + * thread-safe. + * @param c class. + * @param comparator WritableComparator. + */ public static void define(Class c, WritableComparator comparator) { comparators.put(c, comparator); } @@ -111,7 +125,10 @@ protected WritableComparator() { this(null); } - /** Construct for a {@link WritableComparable} implementation. */ + /** + * Construct for a {@link WritableComparable} implementation. + * @param keyClass WritableComparable Class. + */ protected WritableComparator(Class keyClass) { this(keyClass, null, false); } @@ -136,10 +153,16 @@ protected WritableComparator(Class keyClass, } } - /** Returns the WritableComparable implementation class. */ + /** + * Returns the WritableComparable implementation class. + * @return WritableComparable. + */ public Class getKeyClass() { return keyClass; } - /** Construct a new {@link WritableComparable} instance. */ + /** + * Construct a new {@link WritableComparable} instance. + * @return WritableComparable. + */ public WritableComparable newKey() { return ReflectionUtils.newInstance(keyClass, conf); } @@ -168,27 +191,54 @@ public int compare(byte[] b1, int s1, int l1, byte[] b2, int s2, int l2) { return compare(key1, key2); // compare them } - /** Compare two WritableComparables. + /** + * Compare two WritableComparables. * - *

    The default implementation uses the natural ordering, calling {@link - * Comparable#compareTo(Object)}. */ + * The default implementation uses the natural ordering, calling {@link + * Comparable#compareTo(Object)}. + * @param a the first object to be compared. + * @param b the second object to be compared. + * @return compare result. + */ @SuppressWarnings("unchecked") public int compare(WritableComparable a, WritableComparable b) { return a.compareTo(b); } + /** + * Compare two Object. + * + * @param a the first object to be compared. + * @param b the second object to be compared. + * @return compare result. + */ @Override public int compare(Object a, Object b) { return compare((WritableComparable)a, (WritableComparable)b); } - /** Lexicographic order of binary data. */ + /** + * Lexicographic order of binary data. + * @param b1 b1. + * @param s1 s1. + * @param l1 l1. + * @param b2 b2. + * @param s2 s2. + * @param l2 l2. + * @return compare bytes. + */ public static int compareBytes(byte[] b1, int s1, int l1, byte[] b2, int s2, int l2) { return FastByteComparisons.compareTo(b1, s1, l1, b2, s2, l2); } - /** Compute hash for binary data. */ + /** + * Compute hash for binary data. + * @param bytes bytes. + * @param offset offset. + * @param length length. + * @return hash for binary data. + */ public static int hashBytes(byte[] bytes, int offset, int length) { int hash = 1; for (int i = offset; i < offset + length; i++) @@ -196,18 +246,33 @@ public static int hashBytes(byte[] bytes, int offset, int length) { return hash; } - /** Compute hash for binary data. */ + /** + * Compute hash for binary data. + * @param bytes bytes. + * @param length length. + * @return hash for binary data. + */ public static int hashBytes(byte[] bytes, int length) { return hashBytes(bytes, 0, length); } - /** Parse an unsigned short from a byte array. */ + /** + * Parse an unsigned short from a byte array. + * @param bytes bytes. + * @param start start. + * @return unsigned short from a byte array + */ public static int readUnsignedShort(byte[] bytes, int start) { return (((bytes[start] & 0xff) << 8) + ((bytes[start+1] & 0xff))); } - /** Parse an integer from a byte array. */ + /** + * Parse an integer from a byte array. + * @param bytes bytes. + * @param start start. + * @return integer from a byte array + */ public static int readInt(byte[] bytes, int start) { return (((bytes[start ] & 0xff) << 24) + ((bytes[start+1] & 0xff) << 16) + @@ -216,18 +281,33 @@ public static int readInt(byte[] bytes, int start) { } - /** Parse a float from a byte array. */ + /** + * Parse a float from a byte array. + * @param bytes bytes. + * @param start start. + * @return float from a byte array + */ public static float readFloat(byte[] bytes, int start) { return Float.intBitsToFloat(readInt(bytes, start)); } - /** Parse a long from a byte array. */ + /** + * Parse a long from a byte array. + * @param bytes bytes. + * @param start start. + * @return long from a byte array + */ public static long readLong(byte[] bytes, int start) { return ((long)(readInt(bytes, start)) << 32) + (readInt(bytes, start+4) & 0xFFFFFFFFL); } - /** Parse a double from a byte array. */ + /** + * Parse a double from a byte array. + * @param bytes bytes. + * @param start start. + * @return double from a byte array. + */ public static double readDouble(byte[] bytes, int start) { return Double.longBitsToDouble(readLong(bytes, start)); } @@ -236,7 +316,7 @@ public static double readDouble(byte[] bytes, int start) { * Reads a zero-compressed encoded long from a byte array and returns it. * @param bytes byte array with decode long * @param start starting index - * @throws java.io.IOException + * @throws IOException raised on errors performing I/O. * @return deserialized long */ public static long readVLong(byte[] bytes, int start) throws IOException { @@ -261,7 +341,7 @@ public static long readVLong(byte[] bytes, int start) throws IOException { * Reads a zero-compressed encoded integer from a byte array and returns it. * @param bytes byte array with the encoded integer * @param start start index - * @throws java.io.IOException + * @throws IOException raised on errors performing I/O. * @return deserialized integer */ public static int readVInt(byte[] bytes, int start) throws IOException { diff --git a/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/io/WritableFactories.java b/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/io/WritableFactories.java index a8fdbfe98dfdc..9dd231e488780 100644 --- a/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/io/WritableFactories.java +++ b/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/io/WritableFactories.java @@ -35,17 +35,31 @@ public class WritableFactories { private WritableFactories() {} // singleton - /** Define a factory for a class. */ + /** + * Define a factory for a class. + * @param c input c. + * @param factory input factory. + */ public static void setFactory(Class c, WritableFactory factory) { CLASS_TO_FACTORY.put(c, factory); } - /** Define a factory for a class. */ + /** + * Define a factory for a class. + * @param c input c. + * @return a factory for a class. + */ public static WritableFactory getFactory(Class c) { return CLASS_TO_FACTORY.get(c); } - /** Create a new instance of a class with a defined factory. */ + /** + * Create a new instance of a class with a defined factory. + * + * @param c input c. + * @param conf input configuration. + * @return a new instance of a class with a defined factory. + */ public static Writable newInstance(Class c, Configuration conf) { WritableFactory factory = WritableFactories.getFactory(c); if (factory != null) { @@ -59,7 +73,11 @@ public static Writable newInstance(Class c, Configuration co } } - /** Create a new instance of a class with a defined factory. */ + /** + * Create a new instance of a class with a defined factory. + * @param c input c. + * @return a new instance of a class with a defined factory. + */ public static Writable newInstance(Class c) { return newInstance(c, null); } diff --git a/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/io/WritableFactory.java b/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/io/WritableFactory.java index bb8af974f051e..d9e9b543c7d96 100644 --- a/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/io/WritableFactory.java +++ b/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/io/WritableFactory.java @@ -27,7 +27,7 @@ @InterfaceAudience.Public @InterfaceStability.Stable public interface WritableFactory { - /** Return a new instance. */ + /** @return Return a new instance. */ Writable newInstance(); } diff --git a/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/io/WritableName.java b/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/io/WritableName.java index 43d396edad7ab..e5e74875225cc 100644 --- a/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/io/WritableName.java +++ b/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/io/WritableName.java @@ -45,19 +45,33 @@ public class WritableName { private WritableName() {} // no public ctor - /** Set the name that a class should be known as to something other than the - * class name. */ + /** + * Set the name that a class should be known as to something other than the + * class name. + * + * @param writableClass input writableClass. + * @param name input name. + */ public static synchronized void setName(Class writableClass, String name) { CLASS_TO_NAME.put(writableClass, name); NAME_TO_CLASS.put(name, writableClass); } - /** Add an alternate name for a class. */ + /** + * Add an alternate name for a class. + * @param writableClass input writableClass. + * @param name input name. + */ public static synchronized void addName(Class writableClass, String name) { NAME_TO_CLASS.put(name, writableClass); } - /** Return the name for a class. Default is {@link Class#getName()}. */ + /** + * Return the name for a class. + * Default is {@link Class#getName()}. + * @param writableClass input writableClass. + * @return name for a class. + */ public static synchronized String getName(Class writableClass) { String name = CLASS_TO_NAME.get(writableClass); if (name != null) @@ -65,7 +79,15 @@ public static synchronized String getName(Class writableClass) { return writableClass.getName(); } - /** Return the class for a name. Default is {@link Class#forName(String)}.*/ + /** + * Return the class for a name. + * Default is {@link Class#forName(String)}. + * + * @param name input name. + * @param conf input configuration. + * @return class for a name. + * @throws IOException raised on errors performing I/O. + */ public static synchronized Class getClass(String name, Configuration conf ) throws IOException { Class writableClass = NAME_TO_CLASS.get(name); diff --git a/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/io/WritableUtils.java b/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/io/WritableUtils.java index 2062fb6fe3705..187398de0ec86 100644 --- a/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/io/WritableUtils.java +++ b/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/io/WritableUtils.java @@ -208,7 +208,10 @@ public static void displayByteArray(byte[] record){ /** * Make a copy of a writable object using serialization to a buffer. + * + * @param Generics Type T. * @param orig The object to copy + * @param conf input Configuration. * @return The copied object */ public static T clone(T orig, Configuration conf) { @@ -223,10 +226,10 @@ public static T clone(T orig, Configuration conf) { } /** - * Make a copy of the writable object using serialization to a buffer + * Make a copy of the writable object using serialization to a buffer. * @param dst the object to copy from * @param src the object to copy into, which is destroyed - * @throws IOException + * @throws IOException raised on errors performing I/O. * @deprecated use ReflectionUtils.cloneInto instead. */ @Deprecated @@ -248,7 +251,7 @@ public static void cloneInto(Writable dst, Writable src) throws IOException { * * @param stream Binary output stream * @param i Integer to be serialized - * @throws java.io.IOException + * @throws IOException raised on errors performing I/O. */ public static void writeVInt(DataOutput stream, int i) throws IOException { writeVLong(stream, i); @@ -268,7 +271,7 @@ public static void writeVInt(DataOutput stream, int i) throws IOException { * * @param stream Binary output stream * @param i Long to be serialized - * @throws java.io.IOException + * @throws IOException raised on errors performing I/O. */ public static void writeVLong(DataOutput stream, long i) throws IOException { if (i >= -112 && i <= 127) { @@ -303,7 +306,7 @@ public static void writeVLong(DataOutput stream, long i) throws IOException { /** * Reads a zero-compressed encoded long from input stream and returns it. * @param stream Binary input stream - * @throws java.io.IOException + * @throws IOException raised on errors performing I/O. * @return deserialized long from stream. */ public static long readVLong(DataInput stream) throws IOException { @@ -324,7 +327,7 @@ public static long readVLong(DataInput stream) throws IOException { /** * Reads a zero-compressed encoded integer from input stream and returns it. * @param stream Binary input stream - * @throws java.io.IOException + * @throws IOException raised on errors performing I/O. * @return deserialized integer from stream. */ public static int readVInt(DataInput stream) throws IOException { @@ -342,8 +345,10 @@ public static int readVInt(DataInput stream) throws IOException { * inclusive. * * @param stream Binary input stream - * @throws java.io.IOException - * @return deserialized integer from stream + * @param lower input lower. + * @param upper input upper. + * @throws IOException raised on errors performing I/O. + * @return deserialized integer from stream. */ public static int readVIntInRange(DataInput stream, int lower, int upper) throws IOException { @@ -387,7 +392,8 @@ public static int decodeVIntSize(byte value) { } /** - * Get the encoded length if an integer is stored in a variable-length format + * Get the encoded length if an integer is stored in a variable-length format. + * @param i input i. * @return the encoded length */ public static int getVIntSize(long i) { @@ -410,7 +416,7 @@ public static int getVIntSize(long i) { * @param in DataInput to read from * @param enumType Class type of Enum * @return Enum represented by String read from DataInput - * @throws IOException + * @throws IOException raised on errors performing I/O. */ public static > T readEnum(DataInput in, Class enumType) throws IOException{ @@ -420,7 +426,7 @@ public static > T readEnum(DataInput in, Class enumType) * writes String value of enum to DataOutput. * @param out Dataoutput stream * @param enumVal enum value - * @throws IOException + * @throws IOException raised on errors performing I/O. */ public static void writeEnum(DataOutput out, Enum enumVal) throws IOException{ @@ -446,7 +452,11 @@ public static void skipFully(DataInput in, int len) throws IOException { } } - /** Convert writables to a byte array */ + /** + * Convert writables to a byte array. + * @param writables input writables. + * @return ByteArray. + */ public static byte[] toByteArray(Writable... writables) { final DataOutputBuffer out = new DataOutputBuffer(); try { diff --git a/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/io/compress/BZip2Codec.java b/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/io/compress/BZip2Codec.java index 7fd5633daa698..7640f7ed7a6f7 100644 --- a/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/io/compress/BZip2Codec.java +++ b/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/io/compress/BZip2Codec.java @@ -99,7 +99,7 @@ public BZip2Codec() { } * @param out the location for the final output stream * @return a stream the user can write uncompressed data to, to have it * compressed - * @throws IOException + * @throws IOException raised on errors performing I/O. */ @Override public CompressionOutputStream createOutputStream(OutputStream out) @@ -116,7 +116,7 @@ public CompressionOutputStream createOutputStream(OutputStream out) * @param compressor compressor to use * @return a stream the user can write uncompressed data to, to have it * compressed - * @throws IOException + * @throws IOException raised on errors performing I/O. */ @Override public CompressionOutputStream createOutputStream(OutputStream out, @@ -154,7 +154,7 @@ public Compressor createCompressor() { * * @param in the stream to read compressed bytes from * @return a stream to read uncompressed bytes from - * @throws IOException + * @throws IOException raised on errors performing I/O. */ @Override public CompressionInputStream createInputStream(InputStream in) @@ -171,7 +171,7 @@ public CompressionInputStream createInputStream(InputStream in) * @param in the stream to read compressed bytes from * @param decompressor decompressor to use * @return a stream to read uncompressed bytes from - * @throws IOException + * @throws IOException raised on errors performing I/O. */ @Override public CompressionInputStream createInputStream(InputStream in, diff --git a/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/io/compress/BlockDecompressorStream.java b/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/io/compress/BlockDecompressorStream.java index de457d192400d..ff10332ea8d5a 100644 --- a/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/io/compress/BlockDecompressorStream.java +++ b/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/io/compress/BlockDecompressorStream.java @@ -43,7 +43,7 @@ public class BlockDecompressorStream extends DecompressorStream { * @param in input stream * @param decompressor decompressor to use * @param bufferSize size of buffer - * @throws IOException + * @throws IOException raised on errors performing I/O. */ public BlockDecompressorStream(InputStream in, Decompressor decompressor, int bufferSize) throws IOException { @@ -55,7 +55,7 @@ public BlockDecompressorStream(InputStream in, Decompressor decompressor, * * @param in input stream * @param decompressor decompressor to use - * @throws IOException + * @throws IOException raised on errors performing I/O. */ public BlockDecompressorStream(InputStream in, Decompressor decompressor) throws IOException { super(in, decompressor); diff --git a/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/io/compress/CodecPool.java b/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/io/compress/CodecPool.java index 2ac2ca65173f0..1f095c6c6736e 100644 --- a/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/io/compress/CodecPool.java +++ b/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/io/compress/CodecPool.java @@ -109,7 +109,7 @@ private static boolean payback(Map, Set> pool, T codec) { synchronized (pool) { codecSet = pool.get(codecClass); if (codecSet == null) { - codecSet = new HashSet(); + codecSet = new HashSet<>(); pool.put(codecClass, codecSet); } } @@ -235,7 +235,10 @@ public static void returnDecompressor(Decompressor decompressor) { /** * Return the number of leased {@link Compressor}s for this - * {@link CompressionCodec} + * {@link CompressionCodec}. + * + * @param codec codec. + * @return the number of leased. */ public static int getLeasedCompressorsCount(CompressionCodec codec) { return (codec == null) ? 0 : getLeaseCount(compressorCounts, @@ -244,7 +247,10 @@ public static int getLeasedCompressorsCount(CompressionCodec codec) { /** * Return the number of leased {@link Decompressor}s for this - * {@link CompressionCodec} + * {@link CompressionCodec}. + * + * @param codec codec. + * @return the number of leased */ public static int getLeasedDecompressorsCount(CompressionCodec codec) { return (codec == null) ? 0 : getLeaseCount(decompressorCounts, diff --git a/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/io/compress/CompressionCodec.java b/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/io/compress/CompressionCodec.java index f37aadfcb57f3..d064e1b914707 100644 --- a/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/io/compress/CompressionCodec.java +++ b/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/io/compress/CompressionCodec.java @@ -39,7 +39,7 @@ public interface CompressionCodec { * * @param out the location for the final output stream * @return a stream the user can write uncompressed data to have it compressed - * @throws IOException + * @throws IOException raised on errors performing I/O. */ CompressionOutputStream createOutputStream(OutputStream out) throws IOException; @@ -51,7 +51,7 @@ CompressionOutputStream createOutputStream(OutputStream out) * @param out the location for the final output stream * @param compressor compressor to use * @return a stream the user can write uncompressed data to have it compressed - * @throws IOException + * @throws IOException raised on errors performing I/O. */ CompressionOutputStream createOutputStream(OutputStream out, Compressor compressor) @@ -77,7 +77,7 @@ CompressionOutputStream createOutputStream(OutputStream out, * * @param in the stream to read compressed bytes from * @return a stream to read uncompressed bytes from - * @throws IOException + * @throws IOException raised on errors performing I/O. */ CompressionInputStream createInputStream(InputStream in) throws IOException; @@ -88,7 +88,7 @@ CompressionOutputStream createOutputStream(OutputStream out, * @param in the stream to read compressed bytes from * @param decompressor decompressor to use * @return a stream to read uncompressed bytes from - * @throws IOException + * @throws IOException raised on errors performing I/O. */ CompressionInputStream createInputStream(InputStream in, Decompressor decompressor) diff --git a/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/io/compress/CompressionCodecFactory.java b/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/io/compress/CompressionCodecFactory.java index a195ed4e77fd4..8e920a2e64c27 100644 --- a/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/io/compress/CompressionCodecFactory.java +++ b/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/io/compress/CompressionCodecFactory.java @@ -171,6 +171,8 @@ public static void setCodecClasses(Configuration conf, /** * Find the codecs specified in the config value io.compression.codecs * and register them. Defaults to gzip and deflate. + * + * @param conf configuration. */ public CompressionCodecFactory(Configuration conf) { codecs = new TreeMap(); @@ -293,7 +295,8 @@ public static String removeSuffix(String filename, String suffix) { /** * A little test program. - * @param args + * @param args arguments. + * @throws Exception exception. */ public static void main(String[] args) throws Exception { Configuration conf = new Configuration(); diff --git a/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/io/compress/CompressionInputStream.java b/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/io/compress/CompressionInputStream.java index 55bb132e9c87c..5bfec01ec945d 100644 --- a/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/io/compress/CompressionInputStream.java +++ b/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/io/compress/CompressionInputStream.java @@ -53,7 +53,7 @@ public abstract class CompressionInputStream extends InputStream * the decompressed bytes from the given stream. * * @param in The input stream to be compressed. - * @throws IOException + * @throws IOException raised on errors performing I/O. */ protected CompressionInputStream(InputStream in) throws IOException { if (!(in instanceof Seekable) || !(in instanceof PositionedReadable)) { @@ -93,6 +93,8 @@ public IOStatistics getIOStatistics() { /** * Reset the decompressor to its initial state and discard any buffered data, * as the underlying stream may have been repositioned. + * + * @throws IOException raised on errors performing I/O. */ public abstract void resetState() throws IOException; @@ -118,7 +120,7 @@ public long getPos() throws IOException { /** * This method is current not supported. * - * @throws UnsupportedOperationException + * @throws UnsupportedOperationException Unsupported Operation Exception. */ @Override @@ -129,7 +131,7 @@ public void seek(long pos) throws UnsupportedOperationException { /** * This method is current not supported. * - * @throws UnsupportedOperationException + * @throws UnsupportedOperationException Unsupported Operation Exception. */ @Override public boolean seekToNewSource(long targetPos) throws UnsupportedOperationException { diff --git a/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/io/compress/CompressionOutputStream.java b/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/io/compress/CompressionOutputStream.java index 2a11ace81702c..2e412dcd58fce 100644 --- a/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/io/compress/CompressionOutputStream.java +++ b/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/io/compress/CompressionOutputStream.java @@ -48,7 +48,7 @@ public abstract class CompressionOutputStream extends OutputStream /** * Create a compression output stream that writes * the compressed bytes to the given stream. - * @param out + * @param out out. */ protected CompressionOutputStream(OutputStream out) { this.out = out; @@ -89,12 +89,14 @@ public void flush() throws IOException { /** * Finishes writing compressed data to the output stream * without closing the underlying stream. + * @throws IOException raised on errors performing I/O. */ public abstract void finish() throws IOException; /** * Reset the compression to the initial state. * Does not reset the underlying stream. + * @throws IOException raised on errors performing I/O. */ public abstract void resetState() throws IOException; diff --git a/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/io/compress/Compressor.java b/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/io/compress/Compressor.java index 537837faa0a51..7e2a6e679f43c 100644 --- a/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/io/compress/Compressor.java +++ b/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/io/compress/Compressor.java @@ -65,11 +65,13 @@ public interface Compressor { /** * Return number of uncompressed bytes input so far. + * @return bytes read. */ public long getBytesRead(); /** * Return number of compressed bytes output so far. + * @return bytes written. */ public long getBytesWritten(); @@ -97,6 +99,7 @@ public interface Compressor { * @param off Start offset of the data * @param len Size of the buffer * @return The actual number of bytes of compressed data. + * @throws IOException raised on errors performing I/O. */ public int compress(byte[] b, int off, int len) throws IOException; diff --git a/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/io/compress/Decompressor.java b/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/io/compress/Decompressor.java index e9558fab87325..30d4e29892eb7 100644 --- a/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/io/compress/Decompressor.java +++ b/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/io/compress/Decompressor.java @@ -96,7 +96,7 @@ public interface Decompressor { * @param off Start offset of the data * @param len Size of the buffer * @return The actual number of bytes of uncompressed data. - * @throws IOException + * @throws IOException raised on errors performing I/O. */ public int decompress(byte[] b, int off, int len) throws IOException; diff --git a/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/io/compress/DecompressorStream.java b/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/io/compress/DecompressorStream.java index 570d15c7f16aa..745105ce873af 100644 --- a/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/io/compress/DecompressorStream.java +++ b/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/io/compress/DecompressorStream.java @@ -80,7 +80,7 @@ public DecompressorStream(InputStream in, Decompressor decompressor) * Allow derived classes to directly set the underlying stream. * * @param in Underlying input stream. - * @throws IOException + * @throws IOException raised on errors performing I/O. */ protected DecompressorStream(InputStream in) throws IOException { super(in); diff --git a/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/io/compress/DefaultCodec.java b/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/io/compress/DefaultCodec.java index d2ffb22eaafb3..b407ddb11046c 100644 --- a/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/io/compress/DefaultCodec.java +++ b/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/io/compress/DefaultCodec.java @@ -26,7 +26,6 @@ import org.apache.hadoop.classification.InterfaceStability; import org.apache.hadoop.conf.Configurable; import org.apache.hadoop.conf.Configuration; -import org.apache.hadoop.io.compress.zlib.ZlibDecompressor; import org.apache.hadoop.io.compress.zlib.ZlibFactory; import org.slf4j.Logger; import org.slf4j.LoggerFactory; diff --git a/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/io/compress/Lz4Codec.java b/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/io/compress/Lz4Codec.java index 8bfb7fe95c4e2..a5afb706c99c1 100644 --- a/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/io/compress/Lz4Codec.java +++ b/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/io/compress/Lz4Codec.java @@ -61,7 +61,7 @@ public Configuration getConf() { * * @param out the location for the final output stream * @return a stream the user can write uncompressed data to have it compressed - * @throws IOException + * @throws IOException raised on errors performing I/O. */ @Override public CompressionOutputStream createOutputStream(OutputStream out) @@ -77,7 +77,7 @@ public CompressionOutputStream createOutputStream(OutputStream out) * @param out the location for the final output stream * @param compressor compressor to use * @return a stream the user can write uncompressed data to have it compressed - * @throws IOException + * @throws IOException raised on errors performing I/O. */ @Override public CompressionOutputStream createOutputStream(OutputStream out, @@ -125,7 +125,7 @@ public Compressor createCompressor() { * * @param in the stream to read compressed bytes from * @return a stream to read uncompressed bytes from - * @throws IOException + * @throws IOException raised on errors performing I/O. */ @Override public CompressionInputStream createInputStream(InputStream in) @@ -141,7 +141,7 @@ public CompressionInputStream createInputStream(InputStream in) * @param in the stream to read compressed bytes from * @param decompressor decompressor to use * @return a stream to read uncompressed bytes from - * @throws IOException + * @throws IOException raised on errors performing I/O. */ @Override public CompressionInputStream createInputStream(InputStream in, diff --git a/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/io/compress/SnappyCodec.java b/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/io/compress/SnappyCodec.java index 77cf36a339b34..d64c6e512f87c 100644 --- a/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/io/compress/SnappyCodec.java +++ b/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/io/compress/SnappyCodec.java @@ -61,7 +61,7 @@ public Configuration getConf() { * * @param out the location for the final output stream * @return a stream the user can write uncompressed data to have it compressed - * @throws IOException + * @throws IOException raised on errors performing I/O. */ @Override public CompressionOutputStream createOutputStream(OutputStream out) @@ -77,7 +77,7 @@ public CompressionOutputStream createOutputStream(OutputStream out) * @param out the location for the final output stream * @param compressor compressor to use * @return a stream the user can write uncompressed data to have it compressed - * @throws IOException + * @throws IOException raised on errors performing I/O. */ @Override public CompressionOutputStream createOutputStream(OutputStream out, @@ -122,7 +122,7 @@ public Compressor createCompressor() { * * @param in the stream to read compressed bytes from * @return a stream to read uncompressed bytes from - * @throws IOException + * @throws IOException raised on errors performing I/O. */ @Override public CompressionInputStream createInputStream(InputStream in) @@ -138,7 +138,7 @@ public CompressionInputStream createInputStream(InputStream in) * @param in the stream to read compressed bytes from * @param decompressor decompressor to use * @return a stream to read uncompressed bytes from - * @throws IOException + * @throws IOException raised on errors performing I/O. */ @Override public CompressionInputStream createInputStream(InputStream in, diff --git a/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/io/compress/SplittableCompressionCodec.java b/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/io/compress/SplittableCompressionCodec.java index a756f47260c33..f2e28774a46db 100644 --- a/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/io/compress/SplittableCompressionCodec.java +++ b/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/io/compress/SplittableCompressionCodec.java @@ -61,6 +61,7 @@ public enum READ_MODE {CONTINUOUS, BYBLOCK}; * Create a stream as dictated by the readMode. This method is used when * the codecs wants the ability to work with the underlying stream positions. * + * @param decompressor decompressor. * @param seekableIn The seekable input stream (seeks in compressed data) * @param start The start offset into the compressed stream. May be changed * by the underlying codec. @@ -69,6 +70,7 @@ public enum READ_MODE {CONTINUOUS, BYBLOCK}; * @param readMode Controls whether stream position is reported continuously * from the compressed stream only only at block boundaries. * @return a stream to read uncompressed bytes from + * @throws IOException raised on errors performing I/O. */ SplitCompressionInputStream createInputStream(InputStream seekableIn, Decompressor decompressor, long start, long end, READ_MODE readMode) diff --git a/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/io/compress/ZStandardCodec.java b/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/io/compress/ZStandardCodec.java index a7afebc0c49ae..139e81eb73cc2 100644 --- a/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/io/compress/ZStandardCodec.java +++ b/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/io/compress/ZStandardCodec.java @@ -116,7 +116,7 @@ private static int getBufferSize(Configuration conf) { * * @param out the location for the final output stream * @return a stream the user can write uncompressed data to have compressed - * @throws IOException + * @throws IOException raised on errors performing I/O. */ @Override public CompressionOutputStream createOutputStream(OutputStream out) @@ -132,7 +132,7 @@ public CompressionOutputStream createOutputStream(OutputStream out) * @param out the location for the final output stream * @param compressor compressor to use * @return a stream the user can write uncompressed data to have compressed - * @throws IOException + * @throws IOException raised on errors performing I/O. */ @Override public CompressionOutputStream createOutputStream(OutputStream out, @@ -173,7 +173,7 @@ public Compressor createCompressor() { * * @param in the stream to read compressed bytes from * @return a stream to read uncompressed bytes from - * @throws IOException + * @throws IOException raised on errors performing I/O. */ @Override public CompressionInputStream createInputStream(InputStream in) @@ -189,7 +189,7 @@ public CompressionInputStream createInputStream(InputStream in) * @param in the stream to read compressed bytes from * @param decompressor decompressor to use * @return a stream to read uncompressed bytes from - * @throws IOException + * @throws IOException raised on errors performing I/O. */ @Override public CompressionInputStream createInputStream(InputStream in, diff --git a/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/io/compress/bzip2/Bzip2Compressor.java b/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/io/compress/bzip2/Bzip2Compressor.java index 5713c56df6aef..9d1d85332489a 100644 --- a/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/io/compress/bzip2/Bzip2Compressor.java +++ b/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/io/compress/bzip2/Bzip2Compressor.java @@ -67,6 +67,7 @@ public Bzip2Compressor() { /** * Creates a new compressor, taking settings from the configuration. + * @param conf configuration. */ public Bzip2Compressor(Configuration conf) { this(Bzip2Factory.getBlockSize(conf), diff --git a/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/io/compress/bzip2/Bzip2Decompressor.java b/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/io/compress/bzip2/Bzip2Decompressor.java index 72ba97630e206..acd806b9b300a 100644 --- a/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/io/compress/bzip2/Bzip2Decompressor.java +++ b/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/io/compress/bzip2/Bzip2Decompressor.java @@ -50,6 +50,8 @@ public class Bzip2Decompressor implements Decompressor { /** * Creates a new decompressor. + * @param conserveMemory conserveMemory. + * @param directBufferSize directBufferSize. */ public Bzip2Decompressor(boolean conserveMemory, int directBufferSize) { this.conserveMemory = conserveMemory; diff --git a/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/io/compress/bzip2/CBZip2InputStream.java b/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/io/compress/bzip2/CBZip2InputStream.java index 8426d25c2950e..187fe481588c8 100644 --- a/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/io/compress/bzip2/CBZip2InputStream.java +++ b/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/io/compress/bzip2/CBZip2InputStream.java @@ -152,6 +152,7 @@ public enum STATE { * This method reports the processed bytes so far. Please note that this * statistic is only updated on block boundaries and only when the stream is * initiated in BYBLOCK mode. + * @return ProcessedByteCount. */ public long getProcessedByteCount() { return reportedBytesReadFromCompressedStream; @@ -209,7 +210,7 @@ private int readAByte(InputStream inStream) throws IOException { * @param marker The bit pattern to be found in the stream * @param markerBitLength No of bits in the marker * @return true if the marker was found otherwise false - * @throws IOException + * @throws IOException raised on errors performing I/O. * @throws IllegalArgumentException if marketBitLength is greater than 63 */ public boolean skipToNextMarker(long marker, int markerBitLength) @@ -282,7 +283,8 @@ private void makeMaps() { * the magic. Thus callers have to skip the first two bytes. Otherwise this * constructor will throw an exception. *

    - * + * @param in in. + * @param readMode READ_MODE. * @throws IOException * if the stream content is malformed or an I/O error occurs. * @throws NullPointerException @@ -326,7 +328,7 @@ private CBZip2InputStream(final InputStream in, READ_MODE readMode, boolean skip * * @return long Number of bytes between current stream position and the * next BZip2 block start marker. - * @throws IOException + * @throws IOException raised on errors performing I/O. * */ public static long numberOfBytesTillNextMarker(final InputStream in) throws IOException{ diff --git a/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/io/compress/bzip2/CBZip2OutputStream.java b/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/io/compress/bzip2/CBZip2OutputStream.java index 850fec77c5109..39c3638b0f497 100644 --- a/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/io/compress/bzip2/CBZip2OutputStream.java +++ b/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/io/compress/bzip2/CBZip2OutputStream.java @@ -64,13 +64,10 @@ * * * - * + * * * * - * - * - * * @@ -213,6 +210,10 @@ public class CBZip2OutputStream extends OutputStream implements BZip2Constants { /** * This method is accessible by subclasses for historical purposes. If you * don't know what it does then you don't need it. + * @param len len. + * @param freq freq. + * @param alphaSize alphaSize. + * @param maxLen maxLen. */ protected static void hbMakeCodeLengths(char[] len, int[] freq, int alphaSize, int maxLen) { @@ -849,6 +850,7 @@ private void endCompression() throws IOException { /** * Returns the blocksize parameter specified at construction time. + * @return blocksize. */ public final int getBlockSize() { return this.blockSize100k; diff --git a/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/io/compress/lz4/Lz4Decompressor.java b/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/io/compress/lz4/Lz4Decompressor.java index 2b62ef78b2859..719d216abaed0 100644 --- a/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/io/compress/lz4/Lz4Decompressor.java +++ b/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/io/compress/lz4/Lz4Decompressor.java @@ -199,7 +199,7 @@ public synchronized boolean finished() { * @param off Start offset of the data * @param len Size of the buffer * @return The actual number of bytes of uncompressed data. - * @throws IOException + * @throws IOException raised on errors performing I/O. */ @Override public synchronized int decompress(byte[] b, int off, int len) diff --git a/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/io/compress/snappy/SnappyDecompressor.java b/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/io/compress/snappy/SnappyDecompressor.java index d3775e286e895..58987c4dda3de 100644 --- a/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/io/compress/snappy/SnappyDecompressor.java +++ b/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/io/compress/snappy/SnappyDecompressor.java @@ -187,7 +187,7 @@ public boolean finished() { * @param off Start offset of the data * @param len Size of the buffer * @return The actual number of bytes of compressed data. - * @throws IOException + * @throws IOException raised on errors performing I/O. */ @Override public int decompress(byte[] b, int off, int len) diff --git a/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/io/compress/zlib/ZlibCompressor.java b/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/io/compress/zlib/ZlibCompressor.java index da8a90bb3170e..89e05fc6d07be 100644 --- a/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/io/compress/zlib/ZlibCompressor.java +++ b/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/io/compress/zlib/ZlibCompressor.java @@ -240,6 +240,7 @@ public ZlibCompressor() { /** * Creates a new compressor, taking settings from the configuration. + * @param conf configuration. */ public ZlibCompressor(Configuration conf) { this(ZlibFactory.getCompressionLevel(conf), diff --git a/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/io/compress/zlib/ZlibDecompressor.java b/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/io/compress/zlib/ZlibDecompressor.java index f642d7713035d..c2615548d23ee 100644 --- a/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/io/compress/zlib/ZlibDecompressor.java +++ b/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/io/compress/zlib/ZlibDecompressor.java @@ -101,6 +101,8 @@ static boolean isNativeZlibLoaded() { /** * Creates a new decompressor. + * @param header header. + * @param directBufferSize directBufferSize. */ public ZlibDecompressor(CompressionHeader header, int directBufferSize) { this.header = header; diff --git a/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/io/compress/zlib/ZlibFactory.java b/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/io/compress/zlib/ZlibFactory.java index 883f1717eea93..c2de494457ccd 100644 --- a/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/io/compress/zlib/ZlibFactory.java +++ b/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/io/compress/zlib/ZlibFactory.java @@ -66,7 +66,7 @@ public static void loadNativeZLib() { /** * Set the flag whether to use native library. Used for testing non-native * libraries - * + * @param isLoaded isLoaded. */ @VisibleForTesting public static void setNativeZlibLoaded(final boolean isLoaded) { diff --git a/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/io/compress/zstd/ZStandardCompressor.java b/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/io/compress/zstd/ZStandardCompressor.java index bc51f3d98a505..a77b59640cdaf 100644 --- a/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/io/compress/zstd/ZStandardCompressor.java +++ b/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/io/compress/zstd/ZStandardCompressor.java @@ -84,6 +84,8 @@ public static int getRecommendedBufferSize() { /** * Creates a new compressor with the default compression level. * Compressed data will be generated in ZStandard format. + * @param level level. + * @param bufferSize bufferSize. */ public ZStandardCompressor(int level, int bufferSize) { this(level, bufferSize, bufferSize); diff --git a/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/io/compress/zstd/ZStandardDecompressor.java b/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/io/compress/zstd/ZStandardDecompressor.java index adf2fe629f8f7..792547a62faea 100644 --- a/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/io/compress/zstd/ZStandardDecompressor.java +++ b/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/io/compress/zstd/ZStandardDecompressor.java @@ -73,6 +73,7 @@ public ZStandardDecompressor() { /** * Creates a new decompressor. + * @param bufferSize bufferSize. */ public ZStandardDecompressor(int bufferSize) { this.directBufferSize = bufferSize; diff --git a/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/io/erasurecode/CodecUtil.java b/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/io/erasurecode/CodecUtil.java index 2632f4b82f070..f89a0d9812d7c 100644 --- a/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/io/erasurecode/CodecUtil.java +++ b/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/io/erasurecode/CodecUtil.java @@ -83,6 +83,7 @@ private CodecUtil() { } /** * Create encoder corresponding to given codec. * @param options Erasure codec options + * @param conf configuration. * @return erasure encoder */ public static ErasureEncoder createEncoder(Configuration conf, @@ -100,6 +101,7 @@ public static ErasureEncoder createEncoder(Configuration conf, /** * Create decoder corresponding to given codec. * @param options Erasure codec options + * @param conf configuration. * @return erasure decoder */ public static ErasureDecoder createDecoder(Configuration conf, diff --git a/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/io/erasurecode/ErasureCodeNative.java b/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/io/erasurecode/ErasureCodeNative.java index ec317eee4dc3e..83a3151282096 100644 --- a/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/io/erasurecode/ErasureCodeNative.java +++ b/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/io/erasurecode/ErasureCodeNative.java @@ -61,6 +61,7 @@ private ErasureCodeNative() {} /** * Are native libraries loaded? + * @return if is native code loaded true,not false. */ public static boolean isNativeCodeLoaded() { return LOADING_FAILURE_REASON == null; @@ -82,6 +83,7 @@ public static void checkNativeCodeLoaded() { /** * Get the native library name that's available or supported. + * @return library name. */ public static native String getLibraryName(); diff --git a/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/io/erasurecode/codec/ErasureCodec.java b/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/io/erasurecode/codec/ErasureCodec.java index c75eaead83d01..22ab632a49512 100644 --- a/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/io/erasurecode/codec/ErasureCodec.java +++ b/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/io/erasurecode/codec/ErasureCodec.java @@ -19,7 +19,6 @@ import org.apache.hadoop.classification.InterfaceAudience; import org.apache.hadoop.conf.Configuration; -import org.apache.hadoop.io.erasurecode.CodecUtil; import org.apache.hadoop.io.erasurecode.ECSchema; import org.apache.hadoop.io.erasurecode.ErasureCodecOptions; import org.apache.hadoop.io.erasurecode.ErasureCoderOptions; diff --git a/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/io/erasurecode/coder/ErasureCoder.java b/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/io/erasurecode/coder/ErasureCoder.java index b5ae1f1e399a8..ab1775538bd5f 100644 --- a/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/io/erasurecode/coder/ErasureCoder.java +++ b/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/io/erasurecode/coder/ErasureCoder.java @@ -68,6 +68,7 @@ public interface ErasureCoder extends Configurable { * * @param blockGroup the erasure coding block group containing all necessary * information for codec calculation + * @return ErasureCodingStep. */ ErasureCodingStep calculateCoding(ECBlockGroup blockGroup); diff --git a/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/io/erasurecode/coder/ErasureCodingStep.java b/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/io/erasurecode/coder/ErasureCodingStep.java index fb89d99a0540c..333647c982b9f 100644 --- a/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/io/erasurecode/coder/ErasureCodingStep.java +++ b/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/io/erasurecode/coder/ErasureCodingStep.java @@ -46,8 +46,9 @@ public interface ErasureCodingStep { /** * Perform encoding or decoding given the input chunks, and generated results * will be written to the output chunks. - * @param inputChunks - * @param outputChunks + * @param inputChunks inputChunks. + * @param outputChunks outputChunks. + * @throws IOException raised on errors performing I/O. */ void performCoding(ECChunk[] inputChunks, ECChunk[] outputChunks) throws IOException; diff --git a/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/io/erasurecode/coder/ErasureDecoder.java b/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/io/erasurecode/coder/ErasureDecoder.java index 004fd38df1141..30020b9959f0b 100644 --- a/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/io/erasurecode/coder/ErasureDecoder.java +++ b/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/io/erasurecode/coder/ErasureDecoder.java @@ -65,7 +65,7 @@ public ErasureCoderOptions getOptions() { /** * We have all the data blocks and parity blocks as input blocks for * recovering by default. It's codec specific - * @param blockGroup + * @param blockGroup blockGroup. * @return input blocks */ protected ECBlock[] getInputBlocks(ECBlockGroup blockGroup) { @@ -83,7 +83,7 @@ protected ECBlock[] getInputBlocks(ECBlockGroup blockGroup) { /** * Which blocks were erased ? - * @param blockGroup + * @param blockGroup blockGroup. * @return output blocks to recover */ protected ECBlock[] getOutputBlocks(ECBlockGroup blockGroup) { @@ -118,7 +118,7 @@ public void release() { /** * Perform decoding against a block blockGroup. - * @param blockGroup + * @param blockGroup blockGroup. * @return decoding step for caller to do the real work */ protected abstract ErasureCodingStep prepareDecodingStep( @@ -126,7 +126,7 @@ protected abstract ErasureCodingStep prepareDecodingStep( /** * Get the number of erased blocks in the block group. - * @param blockGroup + * @param blockGroup blockGroup. * @return number of erased blocks */ protected int getNumErasedBlocks(ECBlockGroup blockGroup) { @@ -153,7 +153,7 @@ protected static int getNumErasedBlocks(ECBlock[] inputBlocks) { /** * Get indexes of erased blocks from inputBlocks - * @param inputBlocks + * @param inputBlocks inputBlocks. * @return indexes of erased blocks from inputBlocks */ protected int[] getErasedIndexes(ECBlock[] inputBlocks) { diff --git a/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/io/erasurecode/coder/ErasureDecodingStep.java b/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/io/erasurecode/coder/ErasureDecodingStep.java index 24f55470e1727..20a396d313678 100644 --- a/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/io/erasurecode/coder/ErasureDecodingStep.java +++ b/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/io/erasurecode/coder/ErasureDecodingStep.java @@ -37,10 +37,10 @@ public class ErasureDecodingStep implements ErasureCodingStep { /** * The constructor with all the necessary info. - * @param inputBlocks + * @param inputBlocks inputBlocks. * @param erasedIndexes the indexes of erased blocks in inputBlocks array - * @param outputBlocks - * @param rawDecoder + * @param outputBlocks outputBlocks. + * @param rawDecoder rawDecoder. */ public ErasureDecodingStep(ECBlock[] inputBlocks, int[] erasedIndexes, ECBlock[] outputBlocks, diff --git a/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/io/erasurecode/coder/ErasureEncoder.java b/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/io/erasurecode/coder/ErasureEncoder.java index 81666e9b76b2e..cca272f69a28d 100644 --- a/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/io/erasurecode/coder/ErasureEncoder.java +++ b/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/io/erasurecode/coder/ErasureEncoder.java @@ -83,7 +83,7 @@ public void release() { /** * Perform encoding against a block group. - * @param blockGroup + * @param blockGroup blockGroup. * @return encoding step for caller to do the real work */ protected abstract ErasureCodingStep prepareEncodingStep( diff --git a/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/io/erasurecode/coder/ErasureEncodingStep.java b/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/io/erasurecode/coder/ErasureEncodingStep.java index 5fc5c7a09928f..9e696d2c58477 100644 --- a/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/io/erasurecode/coder/ErasureEncodingStep.java +++ b/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/io/erasurecode/coder/ErasureEncodingStep.java @@ -36,9 +36,9 @@ public class ErasureEncodingStep implements ErasureCodingStep { /** * The constructor with all the necessary info. - * @param inputBlocks - * @param outputBlocks - * @param rawEncoder + * @param inputBlocks inputBlocks. + * @param outputBlocks outputBlocks. + * @param rawEncoder rawEncoder. */ public ErasureEncodingStep(ECBlock[] inputBlocks, ECBlock[] outputBlocks, RawErasureEncoder rawEncoder) { diff --git a/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/io/erasurecode/coder/HHErasureCodingStep.java b/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/io/erasurecode/coder/HHErasureCodingStep.java index a0f5b72710679..46f0a76da17df 100644 --- a/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/io/erasurecode/coder/HHErasureCodingStep.java +++ b/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/io/erasurecode/coder/HHErasureCodingStep.java @@ -38,8 +38,8 @@ public abstract class HHErasureCodingStep /** * Constructor given input blocks and output blocks. * - * @param inputBlocks - * @param outputBlocks + * @param inputBlocks inputBlocks. + * @param outputBlocks outputBlocks. */ public HHErasureCodingStep(ECBlock[] inputBlocks, ECBlock[] outputBlocks) { diff --git a/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/io/erasurecode/coder/HHXORErasureDecodingStep.java b/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/io/erasurecode/coder/HHXORErasureDecodingStep.java index 16a3c0fa61c4b..4d594f476dfd9 100644 --- a/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/io/erasurecode/coder/HHXORErasureDecodingStep.java +++ b/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/io/erasurecode/coder/HHXORErasureDecodingStep.java @@ -43,9 +43,9 @@ public class HHXORErasureDecodingStep extends HHErasureCodingStep { /** * The constructor with all the necessary info. - * @param inputBlocks + * @param inputBlocks inputBlocks. * @param erasedIndexes the indexes of erased blocks in inputBlocks array - * @param outputBlocks + * @param outputBlocks outputBlocks. * @param rawDecoder underlying RS decoder for hitchhiker decoding * @param rawEncoder underlying XOR encoder for hitchhiker decoding */ diff --git a/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/io/erasurecode/coder/HHXORErasureEncodingStep.java b/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/io/erasurecode/coder/HHXORErasureEncodingStep.java index 6a5644270117b..f571e932b6a85 100644 --- a/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/io/erasurecode/coder/HHXORErasureEncodingStep.java +++ b/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/io/erasurecode/coder/HHXORErasureEncodingStep.java @@ -40,8 +40,8 @@ public class HHXORErasureEncodingStep extends HHErasureCodingStep { /** * The constructor with all the necessary info. * - * @param inputBlocks - * @param outputBlocks + * @param inputBlocks inputBlocks. + * @param outputBlocks outputBlocks. * @param rsRawEncoder underlying RS encoder for hitchhiker encoding * @param xorRawEncoder underlying XOR encoder for hitchhiker encoding */ diff --git a/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/io/erasurecode/coder/XORErasureDecoder.java b/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/io/erasurecode/coder/XORErasureDecoder.java index 1a0e5c030e070..9aae5e43c86f7 100644 --- a/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/io/erasurecode/coder/XORErasureDecoder.java +++ b/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/io/erasurecode/coder/XORErasureDecoder.java @@ -53,7 +53,7 @@ protected ErasureCodingStep prepareDecodingStep( /** * Which blocks were erased ? For XOR it's simple we only allow and return one * erased block, either data or parity. - * @param blockGroup + * @param blockGroup blockGroup. * @return output blocks to recover */ @Override diff --git a/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/io/erasurecode/coder/util/HHUtil.java b/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/io/erasurecode/coder/util/HHUtil.java index 91d02415bfd93..7f771c9677da8 100644 --- a/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/io/erasurecode/coder/util/HHUtil.java +++ b/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/io/erasurecode/coder/util/HHUtil.java @@ -202,6 +202,8 @@ public static ByteBuffer getPiggyBackForDecode(ByteBuffer[][] inputs, /** * Find the valid input from all the inputs. + * + * @param Generics Type T. * @param inputs input buffers to look for valid input * @return the first valid input */ diff --git a/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/io/erasurecode/grouper/BlockGrouper.java b/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/io/erasurecode/grouper/BlockGrouper.java index 3f1b0c22941bd..0407d16120819 100644 --- a/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/io/erasurecode/grouper/BlockGrouper.java +++ b/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/io/erasurecode/grouper/BlockGrouper.java @@ -33,7 +33,7 @@ public class BlockGrouper { /** * Set EC schema. - * @param schema + * @param schema schema. */ public void setSchema(ECSchema schema) { this.schema = schema; @@ -41,7 +41,7 @@ public void setSchema(ECSchema schema) { /** * Get EC schema. - * @return + * @return ECSchema. */ protected ECSchema getSchema() { return schema; @@ -67,7 +67,7 @@ public int getRequiredNumParityBlocks() { * Calculating and organizing BlockGroup, to be called by ECManager * @param dataBlocks Data blocks to compute parity blocks against * @param parityBlocks To be computed parity blocks - * @return + * @return ECBlockGroup. */ public ECBlockGroup makeBlockGroup(ECBlock[] dataBlocks, ECBlock[] parityBlocks) { diff --git a/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/io/erasurecode/rawcoder/DecodingValidator.java b/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/io/erasurecode/rawcoder/DecodingValidator.java index 396aac08cc517..a9bc297739810 100644 --- a/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/io/erasurecode/rawcoder/DecodingValidator.java +++ b/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/io/erasurecode/rawcoder/DecodingValidator.java @@ -68,7 +68,7 @@ public DecodingValidator(RawErasureDecoder decoder) { * @param erasedIndexes indexes of erased units used for decoding * @param outputs decoded output buffers, which are ready to be read after * the call - * @throws IOException + * @throws IOException raised on errors performing I/O. */ public void validate(ByteBuffer[] inputs, int[] erasedIndexes, ByteBuffer[] outputs) throws IOException { @@ -133,7 +133,7 @@ public void validate(ByteBuffer[] inputs, int[] erasedIndexes, * @param inputs input buffers used for decoding * @param erasedIndexes indexes of erased units used for decoding * @param outputs decoded output buffers - * @throws IOException + * @throws IOException raised on errors performing I/O. */ public void validate(ECChunk[] inputs, int[] erasedIndexes, ECChunk[] outputs) throws IOException { diff --git a/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/io/erasurecode/rawcoder/RawErasureDecoder.java b/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/io/erasurecode/rawcoder/RawErasureDecoder.java index 2ebe94b0385ab..329bf7c3aaf7f 100644 --- a/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/io/erasurecode/rawcoder/RawErasureDecoder.java +++ b/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/io/erasurecode/rawcoder/RawErasureDecoder.java @@ -80,6 +80,7 @@ public RawErasureDecoder(ErasureCoderOptions coderOptions) { * @param erasedIndexes indexes of erased units in the inputs array * @param outputs output buffers to put decoded data into according to * erasedIndexes, ready for read after the call + * @throws IOException raised on errors performing I/O. */ public synchronized void decode(ByteBuffer[] inputs, int[] erasedIndexes, ByteBuffer[] outputs) throws IOException { @@ -117,6 +118,7 @@ public synchronized void decode(ByteBuffer[] inputs, int[] erasedIndexes, /** * Perform the real decoding using Direct ByteBuffer. * @param decodingState the decoding state + * @throws IOException raised on errors performing I/O. */ protected abstract void doDecode(ByteBufferDecodingState decodingState) throws IOException; diff --git a/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/io/erasurecode/rawcoder/RawErasureEncoder.java b/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/io/erasurecode/rawcoder/RawErasureEncoder.java index 6d2ecd20525f4..d5ccb12c9d6a2 100644 --- a/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/io/erasurecode/rawcoder/RawErasureEncoder.java +++ b/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/io/erasurecode/rawcoder/RawErasureEncoder.java @@ -100,7 +100,8 @@ public void encode(ByteBuffer[] inputs, ByteBuffer[] outputs) /** * Perform the real encoding work using direct ByteBuffer. - * @param encodingState the encoding state + * @param encodingState the encoding state. + * @throws IOException raised on errors performing I/O. */ protected abstract void doEncode(ByteBufferEncodingState encodingState) throws IOException; @@ -111,6 +112,7 @@ protected abstract void doEncode(ByteBufferEncodingState encodingState) * @param inputs input buffers to read data from * @param outputs output buffers to put the encoded data into, read to read * after the call + * @throws IOException raised on errors performing I/O. */ public void encode(byte[][] inputs, byte[][] outputs) throws IOException { ByteArrayEncodingState baeState = new ByteArrayEncodingState( @@ -128,6 +130,7 @@ public void encode(byte[][] inputs, byte[][] outputs) throws IOException { * Perform the real encoding work using bytes array, supporting offsets * and lengths. * @param encodingState the encoding state + * @throws IOException raised on errors performing I/O. */ protected abstract void doEncode(ByteArrayEncodingState encodingState) throws IOException; diff --git a/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/io/erasurecode/rawcoder/util/DumpUtil.java b/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/io/erasurecode/rawcoder/util/DumpUtil.java index 6de0716174319..90e57201c545b 100644 --- a/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/io/erasurecode/rawcoder/util/DumpUtil.java +++ b/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/io/erasurecode/rawcoder/util/DumpUtil.java @@ -36,6 +36,10 @@ private DumpUtil() { /** * Convert bytes into format like 0x02 02 00 80. * If limit is negative or too large, then all bytes will be converted. + * + * @param bytes bytes. + * @param limit limit. + * @return bytesToHex. */ public static String bytesToHex(byte[] bytes, int limit) { if (limit <= 0 || limit > bytes.length) { @@ -70,8 +74,8 @@ public static void dumpMatrix(byte[] matrix, /** * Print data in hex format in an array of chunks. - * @param header - * @param chunks + * @param header header. + * @param chunks chunks. */ public static void dumpChunks(String header, ECChunk[] chunks) { System.out.println(); @@ -84,7 +88,7 @@ public static void dumpChunks(String header, ECChunk[] chunks) { /** * Print data in hex format in a chunk. - * @param chunk + * @param chunk chunk. */ public static void dumpChunk(ECChunk chunk) { String str; diff --git a/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/io/erasurecode/rawcoder/util/GF256.java b/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/io/erasurecode/rawcoder/util/GF256.java index 35534f307a7a0..b48a23f8b7085 100644 --- a/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/io/erasurecode/rawcoder/util/GF256.java +++ b/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/io/erasurecode/rawcoder/util/GF256.java @@ -195,6 +195,10 @@ public static byte gfInv(byte a) { * Invert a matrix assuming it's invertible. * * Ported from Intel ISA-L library. + * + * @param inMatrix inMatrix. + * @param outMatrix outMatrix. + * @param n n */ public static void gfInvertMatrix(byte[] inMatrix, byte[] outMatrix, int n) { byte temp; @@ -262,7 +266,11 @@ public static void gfInvertMatrix(byte[] inMatrix, byte[] outMatrix, int n) { * * Calculates const table gftbl in GF(2^8) from single input A * gftbl(A) = {A{00}, A{01}, A{02}, ... , A{0f} }, {A{00}, A{10}, A{20}, - * ... , A{f0} } -- from ISA-L implementation + * ... , A{f0} } -- from ISA-L implementation. + * + * @param c c. + * @param tbl tbl. + * @param offset offset. */ public static void gfVectMulInit(byte c, byte[] tbl, int offset) { byte c2 = (byte) ((c << 1) ^ ((c & 0x80) != 0 ? 0x1d : 0)); diff --git a/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/io/erasurecode/rawcoder/util/GaloisField.java b/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/io/erasurecode/rawcoder/util/GaloisField.java index f80fceca94c34..6d22ff0f62eb3 100644 --- a/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/io/erasurecode/rawcoder/util/GaloisField.java +++ b/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/io/erasurecode/rawcoder/util/GaloisField.java @@ -93,10 +93,11 @@ private GaloisField(int fieldSize, int primitivePolynomial) { } /** - * Get the object performs Galois field arithmetics + * Get the object performs Galois field arithmetics. * * @param fieldSize size of the field * @param primitivePolynomial a primitive polynomial corresponds to the size + * @return GaloisField. */ public static GaloisField getInstance(int fieldSize, int primitivePolynomial) { @@ -114,7 +115,8 @@ public static GaloisField getInstance(int fieldSize, } /** - * Get the object performs Galois field arithmetic with default setting + * Get the object performs Galois field arithmetic with default setting. + * @return GaloisField. */ public static GaloisField getInstance() { return getInstance(DEFAULT_FIELD_SIZE, DEFAULT_PRIMITIVE_POLYNOMIAL); @@ -236,7 +238,13 @@ public void solveVandermondeSystem(int[] x, int[] y, int len) { } /** - * A "bulk" version to the solving of Vandermonde System + * A "bulk" version to the solving of Vandermonde System. + * + * @param x input x. + * @param y input y. + * @param outputOffsets input outputOffsets. + * @param len input len. + * @param dataLen input dataLen. */ public void solveVandermondeSystem(int[] x, byte[][] y, int[] outputOffsets, int len, int dataLen) { @@ -269,6 +277,10 @@ public void solveVandermondeSystem(int[] x, byte[][] y, int[] outputOffsets, /** * A "bulk" version of the solveVandermondeSystem, using ByteBuffer. + * + * @param x input x. + * @param y input y. + * @param len input len. */ public void solveVandermondeSystem(int[] x, ByteBuffer[] y, int len) { ByteBuffer p; @@ -413,10 +425,10 @@ public void substitute(byte[][] p, byte[] q, int x) { * Tends to be 2X faster than the "int" substitute in a loop. * * @param p input polynomial - * @param offsets - * @param len + * @param offsets input offset. + * @param len input len. * @param q store the return result - * @param offset + * @param offset input offset. * @param x input field */ public void substitute(byte[][] p, int[] offsets, @@ -440,6 +452,7 @@ public void substitute(byte[][] p, int[] offsets, * @param p input polynomial * @param q store the return result * @param x input field + * @param len input len. */ public void substitute(ByteBuffer[] p, int len, ByteBuffer q, int x) { int y = 1, iIdx, oIdx; @@ -459,6 +472,9 @@ public void substitute(ByteBuffer[] p, int len, ByteBuffer q, int x) { /** * The "bulk" version of the remainder. * Warning: This function will modify the "dividend" inputs. + * + * @param divisor divisor. + * @param dividend dividend. */ public void remainder(byte[][] dividend, int[] divisor) { for (int i = dividend.length - divisor.length; i >= 0; i--) { @@ -476,6 +492,11 @@ public void remainder(byte[][] dividend, int[] divisor) { /** * The "bulk" version of the remainder. * Warning: This function will modify the "dividend" inputs. + * + * @param dividend dividend. + * @param offsets offsets. + * @param len len. + * @param divisor divisor. */ public void remainder(byte[][] dividend, int[] offsets, int len, int[] divisor) { @@ -497,6 +518,9 @@ public void remainder(byte[][] dividend, int[] offsets, /** * The "bulk" version of the remainder, using ByteBuffer. * Warning: This function will modify the "dividend" inputs. + * + * @param dividend dividend. + * @param divisor divisor. */ public void remainder(ByteBuffer[] dividend, int[] divisor) { int idx1, idx2; @@ -519,6 +543,8 @@ public void remainder(ByteBuffer[] dividend, int[] divisor) { /** * Perform Gaussian elimination on the given matrix. This matrix has to be a * fat matrix (number of rows > number of columns). + * + * @param matrix matrix. */ public void gaussianElimination(int[][] matrix) { assert(matrix != null && matrix.length > 0 && matrix[0].length > 0 diff --git a/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/io/erasurecode/rawcoder/util/RSUtil.java b/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/io/erasurecode/rawcoder/util/RSUtil.java index 43823d0f8c300..b1fdc82a11628 100644 --- a/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/io/erasurecode/rawcoder/util/RSUtil.java +++ b/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/io/erasurecode/rawcoder/util/RSUtil.java @@ -59,6 +59,10 @@ public static void initTables(int k, int rows, byte[] codingMatrix, /** * Ported from Intel ISA-L library. + * + * @param k k. + * @param a a. + * @param m m. */ public static void genCauchyMatrix(byte[] a, int m, int k) { // Identity matrix in high position @@ -82,6 +86,13 @@ public static void genCauchyMatrix(byte[] a, int m, int k) { * * The algorithm is ported from Intel ISA-L library for compatible. It * leverages Java auto-vectorization support for performance. + * + * @param gfTables gfTables. + * @param dataLen dataLen. + * @param inputs inputs. + * @param inputOffsets inputOffsets. + * @param outputs outputs. + * @param outputOffsets outputOffsets. */ public static void encodeData(byte[] gfTables, int dataLen, byte[][] inputs, int[] inputOffsets, byte[][] outputs, @@ -133,6 +144,10 @@ public static void encodeData(byte[] gfTables, int dataLen, byte[][] inputs, /** * See above. Try to use the byte[] version when possible. + * + * @param gfTables gfTables. + * @param inputs inputs. + * @param outputs outputs. */ public static void encodeData(byte[] gfTables, ByteBuffer[] inputs, ByteBuffer[] outputs) { diff --git a/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/io/file/tfile/ByteArray.java b/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/io/file/tfile/ByteArray.java index c6c8b3fe3e1fe..964fb04c1b976 100644 --- a/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/io/file/tfile/ByteArray.java +++ b/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/io/file/tfile/ByteArray.java @@ -35,7 +35,7 @@ public final class ByteArray implements RawComparable { /** * Constructing a ByteArray from a {@link BytesWritable}. * - * @param other + * @param other other. */ public ByteArray(BytesWritable other) { this(other.getBytes(), 0, other.getLength()); diff --git a/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/io/file/tfile/TFile.java b/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/io/file/tfile/TFile.java index 09cd2825e3cf2..aeacc16a78f9b 100644 --- a/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/io/file/tfile/TFile.java +++ b/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/io/file/tfile/TFile.java @@ -276,7 +276,7 @@ private enum State { * * @param conf * The configuration object. - * @throws IOException + * @throws IOException raised on errors performing I/O. */ public Writer(FSDataOutputStream fsdos, int minBlockSize, String compressName, String comparator, Configuration conf) @@ -350,7 +350,7 @@ public void close() throws IOException { * Buffer for key. * @param value * Buffer for value. - * @throws IOException + * @throws IOException raised on errors performing I/O. */ public void append(byte[] key, byte[] value) throws IOException { append(key, 0, key.length, value, 0, value.length); @@ -521,7 +521,7 @@ public void close() throws IOException { * exactly as many bytes as specified here before calling close on * the returned output stream. * @return The key appending output stream. - * @throws IOException + * @throws IOException raised on errors performing I/O. * */ public DataOutputStream prepareAppendKey(int length) throws IOException { @@ -548,8 +548,8 @@ public DataOutputStream prepareAppendKey(int length) throws IOException { * the returned output stream. Advertising the value size up-front * guarantees that the value is encoded in one chunk, and avoids * intermediate chunk buffering. - * @throws IOException - * + * @throws IOException raised on errors performing I/O. + * @return DataOutputStream. */ public DataOutputStream prepareAppendValue(int length) throws IOException { if (state != State.END_KEY) { @@ -588,7 +588,7 @@ public DataOutputStream prepareAppendValue(int length) throws IOException { * {@link TFile#getSupportedCompressionAlgorithms()}. * @return A DataOutputStream that can be used to write Meta Block data. * Closing the stream would signal the ending of the block. - * @throws IOException + * @throws IOException raised on errors performing I/O. * @throws MetaBlockAlreadyExists * the Meta Block with the same name already exists. */ @@ -616,7 +616,7 @@ public DataOutputStream prepareMetaBlock(String name, String compressName) * Name of the meta block. * @return A DataOutputStream that can be used to write Meta Block data. * Closing the stream would signal the ending of the block. - * @throws IOException + * @throws IOException raised on errors performing I/O. * @throws MetaBlockAlreadyExists * the Meta Block with the same name already exists. */ @@ -796,8 +796,8 @@ public boolean equals(Object obj) { * The length of TFile. This is required because we have no easy * way of knowing the actual size of the input file through the * File input stream. - * @param conf - * @throws IOException + * @param conf configuration. + * @throws IOException raised on errors performing I/O. */ public Reader(FSDataInputStream fsdis, long fileLength, Configuration conf) throws IOException { @@ -896,7 +896,7 @@ synchronized void checkTFileDataIndex() throws IOException { * Get the first key in the TFile. * * @return The first key in the TFile. - * @throws IOException + * @throws IOException raised on errors performing I/O. */ public RawComparable getFirstKey() throws IOException { checkTFileDataIndex(); @@ -907,7 +907,7 @@ public RawComparable getFirstKey() throws IOException { * Get the last key in the TFile. * * @return The last key in the TFile. - * @throws IOException + * @throws IOException raised on errors performing I/O. */ public RawComparable getLastKey() throws IOException { checkTFileDataIndex(); @@ -1043,7 +1043,7 @@ Location getLocationNear(long offset) { * the user supplied offset. * @return the RecordNum to the corresponding entry. If no such entry * exists, it returns the total entry count. - * @throws IOException + * @throws IOException raised on errors performing I/O. */ public long getRecordNumNear(long offset) throws IOException { return getRecordNumByLocation(getLocationNear(offset)); @@ -1058,7 +1058,7 @@ public long getRecordNumNear(long offset) throws IOException { * @return the key that fits the requirement; or null if no such key exists * (which could happen if the offset is close to the end of the * TFile). - * @throws IOException + * @throws IOException raised on errors performing I/O. */ public RawComparable getKeyNear(long offset) throws IOException { int blockIndex = readerBCF.getBlockIndexNear(offset); @@ -1072,7 +1072,7 @@ public RawComparable getKeyNear(long offset) throws IOException { * * @return The scanner object. A valid Scanner is always returned even if * the TFile is empty. - * @throws IOException + * @throws IOException raised on errors performing I/O. */ public Scanner createScanner() throws IOException { return new Scanner(this, begin, end); @@ -1089,7 +1089,7 @@ public Scanner createScanner() throws IOException { * specified byte-region but always round up to the compression * block boundaries. It is possible that the returned scanner * contains zero key-value pairs even if length is positive. - * @throws IOException + * @throws IOException raised on errors performing I/O. */ public Scanner createScannerByByteRange(long offset, long length) throws IOException { return new Scanner(this, offset, offset + length); @@ -1106,7 +1106,7 @@ public Scanner createScannerByByteRange(long offset, long length) throws IOExcep * key-value entry of the TFile. * @return The actual coverage of the returned scanner will cover all keys * greater than or equal to the beginKey and less than the endKey. - * @throws IOException + * @throws IOException raised on errors performing I/O. * * @deprecated Use {@link #createScannerByKey(byte[], byte[])} instead. */ @@ -1127,7 +1127,7 @@ public Scanner createScanner(byte[] beginKey, byte[] endKey) * key-value entry of the TFile. * @return The actual coverage of the returned scanner will cover all keys * greater than or equal to the beginKey and less than the endKey. - * @throws IOException + * @throws IOException raised on errors performing I/O. */ public Scanner createScannerByKey(byte[] beginKey, byte[] endKey) throws IOException { @@ -1147,7 +1147,7 @@ public Scanner createScannerByKey(byte[] beginKey, byte[] endKey) * key-value entry of the TFile. * @return The actual coverage of the returned scanner will cover all keys * greater than or equal to the beginKey and less than the endKey. - * @throws IOException + * @throws IOException raised on errors performing I/O. * * @deprecated Use {@link #createScannerByKey(RawComparable, RawComparable)} * instead. @@ -1169,7 +1169,7 @@ public Scanner createScanner(RawComparable beginKey, RawComparable endKey) * key-value entry of the TFile. * @return The actual coverage of the returned scanner will cover all keys * greater than or equal to the beginKey and less than the endKey. - * @throws IOException + * @throws IOException raised on errors performing I/O. */ public Scanner createScannerByKey(RawComparable beginKey, RawComparable endKey) throws IOException { @@ -1189,7 +1189,7 @@ public Scanner createScannerByKey(RawComparable beginKey, RawComparable endKey) * The RecordNum for the last record (exclusive). To scan the whole * file, either specify endRecNum==-1 or endRecNum==getEntryCount(). * @return The TFile scanner that covers the specified range of records. - * @throws IOException + * @throws IOException raised on errors performing I/O. */ public Scanner createScannerByRecordNum(long beginRecNum, long endRecNum) throws IOException { @@ -1313,7 +1313,7 @@ protected Scanner(Reader reader, long offBegin, long offEnd) * @param endKey * End key of the scan. If null, scan up to the last <K, V> * entry of the TFile. - * @throws IOException + * @throws IOException raised on errors performing I/O. */ protected Scanner(Reader reader, RawComparable beginKey, RawComparable endKey) throws IOException { @@ -1338,7 +1338,7 @@ protected Scanner(Reader reader, RawComparable beginKey, * @param key * The input key * @return true if we find an equal key. - * @throws IOException + * @throws IOException raised on errors performing I/O. */ public boolean seekTo(byte[] key) throws IOException { return seekTo(key, 0, key.length); @@ -1356,7 +1356,7 @@ public boolean seekTo(byte[] key) throws IOException { * @param keyLen * key buffer length. * @return true if we find an equal key; false otherwise. - * @throws IOException + * @throws IOException raised on errors performing I/O. */ public boolean seekTo(byte[] key, int keyOffset, int keyLen) throws IOException { @@ -1432,7 +1432,7 @@ private void seekTo(Location l) throws IOException { * Rewind to the first entry in the scanner. The entry returned by the * previous entry() call will be invalid. * - * @throws IOException + * @throws IOException raised on errors performing I/O. */ public void rewind() throws IOException { seekTo(beginLocation); @@ -1442,7 +1442,7 @@ public void rewind() throws IOException { * Seek to the end of the scanner. The entry returned by the previous * entry() call will be invalid. * - * @throws IOException + * @throws IOException raised on errors performing I/O. */ public void seekToEnd() throws IOException { parkCursorAtEnd(); @@ -1455,7 +1455,7 @@ public void seekToEnd() throws IOException { * * @param key * The input key - * @throws IOException + * @throws IOException raised on errors performing I/O. */ public void lowerBound(byte[] key) throws IOException { lowerBound(key, 0, key.length); @@ -1472,7 +1472,7 @@ public void lowerBound(byte[] key) throws IOException { * offset in the key buffer. * @param keyLen * key buffer length. - * @throws IOException + * @throws IOException raised on errors performing I/O. */ public void lowerBound(byte[] key, int keyOffset, int keyLen) throws IOException { @@ -1486,7 +1486,7 @@ public void lowerBound(byte[] key, int keyOffset, int keyLen) * * @param key * The input key - * @throws IOException + * @throws IOException raised on errors performing I/O. */ public void upperBound(byte[] key) throws IOException { upperBound(key, 0, key.length); @@ -1503,7 +1503,7 @@ public void upperBound(byte[] key) throws IOException { * offset in the key buffer. * @param keyLen * key buffer length. - * @throws IOException + * @throws IOException raised on errors performing I/O. */ public void upperBound(byte[] key, int keyOffset, int keyLen) throws IOException { @@ -1516,7 +1516,7 @@ public void upperBound(byte[] key, int keyOffset, int keyLen) * * @return true if the cursor successfully moves. False when cursor is * already at the end location and cannot be advanced. - * @throws IOException + * @throws IOException raised on errors performing I/O. */ public boolean advance() throws IOException { if (atEnd()) { @@ -1614,7 +1614,7 @@ void checkKey() throws IOException { * Get an entry to access the key and value. * * @return The Entry object to access the key and value. - * @throws IOException + * @throws IOException raised on errors performing I/O. */ public Entry entry() throws IOException { checkKey(); @@ -1624,7 +1624,7 @@ public Entry entry() throws IOException { /** * Get the RecordNum corresponding to the entry pointed by the cursor. * @return The RecordNum corresponding to the entry pointed by the cursor. - * @throws IOException + * @throws IOException raised on errors performing I/O. */ public long getRecordNum() throws IOException { return reader.getRecordNumByLocation(currentLocation); @@ -1670,7 +1670,7 @@ byte[] getKeyBuffer() { * BytesWritable to hold key. * @param value * BytesWritable to hold value - * @throws IOException + * @throws IOException raised on errors performing I/O. */ public void get(BytesWritable key, BytesWritable value) throws IOException { @@ -1684,7 +1684,8 @@ public void get(BytesWritable key, BytesWritable value) * * @param key * BytesWritable to hold the key. - * @throws IOException + * @throws IOException raised on errors performing I/O. + * @return the key into BytesWritable. */ public int getKey(BytesWritable key) throws IOException { key.setSize(getKeyLength()); @@ -1698,8 +1699,9 @@ public int getKey(BytesWritable key) throws IOException { * directly uses the buffer inside BytesWritable for storing the value. * The call does not require the value length to be known. * - * @param value - * @throws IOException + * @param value value. + * @throws IOException raised on errors performing I/O. + * @return long value. */ public long getValue(BytesWritable value) throws IOException { DataInputStream dis = getValueStream(); @@ -1725,7 +1727,7 @@ public long getValue(BytesWritable value) throws IOException { * @param out * The output stream * @return the length of the key. - * @throws IOException + * @throws IOException raised on errors performing I/O. */ public int writeKey(OutputStream out) throws IOException { out.write(keyBuffer, 0, klen); @@ -1740,7 +1742,7 @@ public int writeKey(OutputStream out) throws IOException { * @param out * The output stream * @return the length of the value - * @throws IOException + * @throws IOException raised on errors performing I/O. */ public long writeValue(OutputStream out) throws IOException { DataInputStream dis = getValueStream(); @@ -1768,7 +1770,7 @@ public long writeValue(OutputStream out) throws IOException { * not be shorter than the key length. * @return The length of the key. * - * @throws IOException + * @throws IOException raised on errors performing I/O. */ public int getKey(byte[] buf) throws IOException { return getKey(buf, 0); @@ -1784,7 +1786,7 @@ public int getKey(byte[] buf) throws IOException { * the key into. Requiring the key-length + offset no greater * than the buffer length. * @return The length of the key. - * @throws IOException + * @throws IOException raised on errors performing I/O. */ public int getKey(byte[] buf, int offset) throws IOException { if ((offset | (buf.length - offset - klen)) < 0) { @@ -1828,10 +1830,11 @@ public int getValueLength() { * without moving the cursor will result in exception: * {@link #getValue(byte[])}, {@link #getValue(byte[], int)}, * {@link #getValueStream}. - * + * + * @param buf buf. * @return the length of the value. Does not require * isValueLengthKnown() to be true. - * @throws IOException + * @throws IOException raised on errors performing I/O. * */ public int getValue(byte[] buf) throws IOException { @@ -1846,10 +1849,12 @@ public int getValue(byte[] buf) throws IOException { * functions more than once without moving the cursor will result in * exception: {@link #getValue(byte[])}, {@link #getValue(byte[], int)}, * {@link #getValueStream}. - * + * + * @param buf buf. + * @param offset offset. * @return the length of the value. Does not require * isValueLengthKnown() to be true. - * @throws IOException + * @throws IOException raised on errors performing I/O. */ public int getValue(byte[] buf, int offset) throws IOException { DataInputStream dis = getValueStream(); @@ -1892,7 +1897,7 @@ public int getValue(byte[] buf, int offset) throws IOException { * {@link #getValue(byte[], int)}, {@link #getValueStream}. * * @return The input stream for reading the value. - * @throws IOException + * @throws IOException raised on errors performing I/O. */ public DataInputStream getValueStream() throws IOException { if (valueChecked == true) { diff --git a/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/io/file/tfile/Utils.java b/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/io/file/tfile/Utils.java index 17a27f16b9a4a..714dc5a12acd2 100644 --- a/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/io/file/tfile/Utils.java +++ b/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/io/file/tfile/Utils.java @@ -49,7 +49,7 @@ private Utils() { * output stream * @param n * The integer to be encoded - * @throws IOException + * @throws IOException raised on errors performing I/O. * @see Utils#writeVLong(DataOutput, long) */ public static void writeVInt(DataOutput out, int n) throws IOException { @@ -95,7 +95,7 @@ public static void writeVInt(DataOutput out, int n) throws IOException { * output stream * @param n * the integer number - * @throws IOException + * @throws IOException raised on errors performing I/O. */ @SuppressWarnings("fallthrough") public static void writeVLong(DataOutput out, long n) throws IOException { @@ -170,7 +170,7 @@ public static void writeVLong(DataOutput out, long n) throws IOException { * @param in * input stream * @return the decoded integer - * @throws IOException + * @throws IOException raised on errors performing I/O. * * @see Utils#readVLong(DataInput) */ @@ -199,7 +199,7 @@ public static int readVInt(DataInput in) throws IOException { * @param in * input stream * @return the decoded long integer. - * @throws IOException + * @throws IOException raised on errors performing I/O. */ public static long readVLong(DataInput in) throws IOException { @@ -249,9 +249,9 @@ public static long readVLong(DataInput in) throws IOException { /** * Write a String as a VInt n, followed by n Bytes as in Text format. * - * @param out - * @param s - * @throws IOException + * @param out out. + * @param s s. + * @throws IOException raised on errors performing I/O. */ public static void writeString(DataOutput out, String s) throws IOException { if (s != null) { @@ -271,7 +271,7 @@ public static void writeString(DataOutput out, String s) throws IOException { * @param in * The input stream. * @return The string - * @throws IOException + * @throws IOException raised on errors performing I/O. */ public static String readString(DataInput in) throws IOException { int length = readVInt(in); @@ -299,7 +299,7 @@ public static final class Version implements Comparable { * * @param in * input stream - * @throws IOException + * @throws IOException raised on errors performing I/O. */ public Version(DataInput in) throws IOException { major = in.readShort(); @@ -326,7 +326,7 @@ public Version(short major, short minor) { * * @param out * The DataOutput object. - * @throws IOException + * @throws IOException raised on errors performing I/O. */ public void write(DataOutput out) throws IOException { out.writeShort(major); diff --git a/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/io/nativeio/NativeIO.java b/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/io/nativeio/NativeIO.java index ebe7f213ceeb1..5cf820c50ca7c 100644 --- a/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/io/nativeio/NativeIO.java +++ b/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/io/nativeio/NativeIO.java @@ -355,7 +355,7 @@ public boolean verifyCanMlock() { } /** - * Return true if the JNI-based native IO extensions are available. + * @return Return true if the JNI-based native IO extensions are available. */ public static boolean isAvailable() { return NativeCodeLoader.isNativeCodeLoaded() && nativeLoaded; @@ -367,7 +367,14 @@ private static void assertCodeLoaded() throws IOException { } } - /** Wrapper around open(2) */ + /** + * Wrapper around open(2) . + * @param path input path. + * @param flags input flags. + * @param mode input mode. + * @return FileDescriptor. + * @throws IOException raised on errors performing I/O. + */ public static native FileDescriptor open(String path, int flags, int mode) throws IOException; /** Wrapper around fstat(2) */ private static native Stat fstat(FileDescriptor fd) throws IOException; @@ -428,6 +435,10 @@ static void posixFadviseIfPossible(String identifier, * for this syscall for more information. On systems where this * call is not available, does nothing. * + * @param fd input fd. + * @param offset input offset. + * @param nbytes input nbytes. + * @param flags input flag. * @throws NativeIOException if there is an error with the syscall */ public static void syncFileRangeIfPossible( @@ -712,7 +723,14 @@ public static void createDirectoryWithMode(File path, int mode) private static native void createDirectoryWithMode0(String path, int mode) throws NativeIOException; - /** Wrapper around CreateFile() on Windows */ + /** + * @return Wrapper around CreateFile() on Windows. + * @param path input path. + * @param desiredAccess input desiredAccess. + * @param shareMode input shareMode. + * @param creationDisposition input creationDisposition. + * @throws IOException raised on errors performing I/O. + */ public static native FileDescriptor createFile(String path, long desiredAccess, long shareMode, long creationDisposition) throws IOException; @@ -749,7 +767,13 @@ private static native FileDescriptor createFileWithMode0(String path, long desiredAccess, long shareMode, long creationDisposition, int mode) throws NativeIOException; - /** Wrapper around SetFilePointer() on Windows */ + /** + * @return Wrapper around SetFilePointer() on Windows. + * @param fd input fd. + * @param distanceToMove input distanceToMove. + * @param moveMethod input moveMethod. + * @throws IOException raised on errors performing I/O. + */ public static native long setFilePointer(FileDescriptor fd, long distanceToMove, long moveMethod) throws IOException; @@ -840,7 +864,7 @@ public static boolean access(String path, AccessRight desiredAccess) } /** - * Return true if the JNI-based native IO extensions are available. + * @return Return true if the JNI-based native IO extensions are available. */ public static boolean isAvailable() { return NativeCodeLoader.isNativeCodeLoaded() && nativeLoaded; @@ -898,6 +922,7 @@ public CachedUid(String username, long timestamp) { * * @param name the full principal name containing the domain * @return name with domain removed + * @throws IOException raised on errors performing I/O. */ private static String stripDomain(String name) { int i = name.indexOf('\\'); @@ -933,6 +958,11 @@ public static String getOwner(FileDescriptor fd) throws IOException { * file opened at a given offset, i.e. other process can delete * the file the FileDescriptor is reading. Only Windows implementation * uses the native interface. + * + * @param f input f. + * @param seekOffset input seekOffset. + * @return FileDescriptor. + * @throws IOException raised on errors performing I/O. */ public static FileDescriptor getShareDeleteFileDescriptor( File f, long seekOffset) throws IOException { @@ -961,7 +991,7 @@ public static FileDescriptor getShareDeleteFileDescriptor( } /** - * Create the specified File for write access, ensuring that it does not exist. + * @return Create the specified File for write access, ensuring that it does not exist. * @param f the file that we want to create * @param permissions we want to have on the file (if security is enabled) * @@ -1045,7 +1075,7 @@ public static void renameTo(File src, File dst) * * @param src source file * @param dst hardlink location - * @throws IOException + * @throws IOException raised on errors performing I/O. */ @Deprecated public static void link(File src, File dst) throws IOException { @@ -1103,7 +1133,7 @@ private static native void link0(String src, String dst) * * @param src The source path * @param dst The destination path - * @throws IOException + * @throws IOException raised on errors performing I/O. */ public static void copyFileUnbuffered(File src, File dst) throws IOException { if (nativeLoaded && Shell.WINDOWS) { diff --git a/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/io/retry/AsyncCallHandler.java b/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/io/retry/AsyncCallHandler.java index 06dd0d45b3c2d..60210ccd920c2 100644 --- a/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/io/retry/AsyncCallHandler.java +++ b/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/io/retry/AsyncCallHandler.java @@ -28,7 +28,6 @@ import org.slf4j.Logger; import org.slf4j.LoggerFactory; -import java.io.InterruptedIOException; import java.lang.reflect.Method; import java.util.Iterator; import java.util.Queue; @@ -49,7 +48,11 @@ public class AsyncCallHandler { private static final ThreadLocal> ASYNC_RETURN = new ThreadLocal<>(); - /** @return the async return value from {@link AsyncCallHandler}. */ + /** + * @return the async return value from {@link AsyncCallHandler}. + * @param T. + * @param R. + */ @InterfaceStability.Unstable @SuppressWarnings("unchecked") public static AsyncGet getAsyncReturn() { @@ -62,7 +65,10 @@ public static AsyncGet getAsyncReturn() { } } - /** For the lower rpc layers to set the async return value. */ + /** + * For the lower rpc layers to set the async return value. + * @param asyncReturn asyncReturn. + */ @InterfaceStability.Unstable public static void setLowerLayerAsyncReturn( AsyncGet asyncReturn) { diff --git a/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/io/retry/RetryPolicies.java b/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/io/retry/RetryPolicies.java index 842811edb399a..d7693f868eb30 100644 --- a/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/io/retry/RetryPolicies.java +++ b/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/io/retry/RetryPolicies.java @@ -41,7 +41,6 @@ import org.apache.hadoop.net.ConnectTimeoutException; import org.apache.hadoop.security.AccessControlException; import org.apache.hadoop.security.token.SecretManager.InvalidToken; -import org.ietf.jgss.GSSException; import org.apache.hadoop.classification.VisibleForTesting; import org.slf4j.Logger; @@ -75,6 +74,10 @@ public class RetryPolicies { *

    * Keep trying forever with a fixed time between attempts. *

    + * + * @param sleepTime sleepTime. + * @param timeUnit timeUnit. + * @return RetryPolicy. */ public static final RetryPolicy retryForeverWithFixedSleep(long sleepTime, TimeUnit timeUnit) { @@ -87,6 +90,11 @@ public static final RetryPolicy retryForeverWithFixedSleep(long sleepTime, * Keep trying a limited number of times, waiting a fixed time between attempts, * and then fail by re-throwing the exception. *

    + * + * @param maxRetries maxRetries. + * @param sleepTime sleepTime. + * @param timeUnit timeUnit. + * @return RetryPolicy. */ public static final RetryPolicy retryUpToMaximumCountWithFixedSleep(int maxRetries, long sleepTime, TimeUnit timeUnit) { return new RetryUpToMaximumCountWithFixedSleep(maxRetries, sleepTime, timeUnit); @@ -97,6 +105,11 @@ public static final RetryPolicy retryUpToMaximumCountWithFixedSleep(int maxRetri * Keep trying for a maximum time, waiting a fixed time between attempts, * and then fail by re-throwing the exception. *

    + * + * @param timeUnit timeUnit. + * @param sleepTime sleepTime. + * @param maxTime maxTime. + * @return RetryPolicy. */ public static final RetryPolicy retryUpToMaximumTimeWithFixedSleep(long maxTime, long sleepTime, TimeUnit timeUnit) { return new RetryUpToMaximumTimeWithFixedSleep(maxTime, sleepTime, timeUnit); @@ -108,6 +121,11 @@ public static final RetryPolicy retryUpToMaximumTimeWithFixedSleep(long maxTime, * and then fail by re-throwing the exception. * The time between attempts is sleepTime mutliplied by the number of tries so far. *

    + * + * @param sleepTime sleepTime. + * @param maxRetries maxRetries. + * @param timeUnit timeUnit. + * @return RetryPolicy. */ public static final RetryPolicy retryUpToMaximumCountWithProportionalSleep(int maxRetries, long sleepTime, TimeUnit timeUnit) { return new RetryUpToMaximumCountWithProportionalSleep(maxRetries, sleepTime, timeUnit); @@ -120,6 +138,12 @@ public static final RetryPolicy retryUpToMaximumCountWithProportionalSleep(int m * The time between attempts is sleepTime mutliplied by a random * number in the range of [0, 2 to the number of retries) *

    + * + * + * @param timeUnit timeUnit. + * @param maxRetries maxRetries. + * @param sleepTime sleepTime. + * @return RetryPolicy. */ public static final RetryPolicy exponentialBackoffRetry( int maxRetries, long sleepTime, TimeUnit timeUnit) { @@ -130,6 +154,10 @@ public static final RetryPolicy exponentialBackoffRetry( *

    * Set a default policy with some explicit handlers for specific exceptions. *

    + * + * @param exceptionToPolicyMap exceptionToPolicyMap. + * @param defaultPolicy defaultPolicy. + * @return RetryPolicy. */ public static final RetryPolicy retryByException(RetryPolicy defaultPolicy, Map, RetryPolicy> exceptionToPolicyMap) { @@ -141,6 +169,10 @@ public static final RetryPolicy retryByException(RetryPolicy defaultPolicy, * A retry policy for RemoteException * Set a default policy with some explicit handlers for specific exceptions. *

    + * + * @param defaultPolicy defaultPolicy. + * @param exceptionToPolicyMap exceptionToPolicyMap. + * @return RetryPolicy. */ public static final RetryPolicy retryByRemoteException( RetryPolicy defaultPolicy, @@ -150,6 +182,9 @@ public static final RetryPolicy retryByRemoteException( /** * A retry policy for exceptions other than RemoteException. + * @param defaultPolicy defaultPolicy. + * @param exceptionToPolicyMap exceptionToPolicyMap. + * @return RetryPolicy. */ public static final RetryPolicy retryOtherThanRemoteException( RetryPolicy defaultPolicy, @@ -437,6 +472,7 @@ public String toString() { * where t_i and n_i are the i-th pair of sleep time and number of retries. * Note that the white spaces in the string are ignored. * + * @param s input string. * @return the parsed object, or null if the parsing fails. */ public static MultipleLinearRandomRetry parseCommaSeparatedString(String s) { diff --git a/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/io/retry/RetryProxy.java b/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/io/retry/RetryProxy.java index 7fcd5fd4b0080..eaff5bbd528d8 100644 --- a/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/io/retry/RetryProxy.java +++ b/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/io/retry/RetryProxy.java @@ -34,6 +34,7 @@ public class RetryProxy { * @param iface the interface that the retry will implement * @param implementation the instance whose methods should be retried * @param retryPolicy the policy for retrying method call failures + * @param T. * @return the retry proxy */ public static Object create(Class iface, T implementation, @@ -51,6 +52,7 @@ public static Object create(Class iface, T implementation, * @param iface the interface that the retry will implement * @param proxyProvider provides implementation instances whose methods should be retried * @param retryPolicy the policy for retrying or failing over method call failures + * @param T. * @return the retry proxy */ public static Object create(Class iface, @@ -69,6 +71,7 @@ public static Object create(Class iface, * {@link RetryPolicies#TRY_ONCE_THEN_FAIL} is used. * * @param iface the interface that the retry will implement + * @param T. * @param implementation the instance whose methods should be retried * @param methodNameToPolicyMap a map of method names to retry policies * @return the retry proxy @@ -90,6 +93,8 @@ public static Object create(Class iface, T implementation, * @param iface the interface that the retry will implement * @param proxyProvider provides implementation instances whose methods should be retried * @param methodNameToPolicyMap map of method names to retry policies + * @param defaultPolicy defaultPolicy. + * @param T. * @return the retry proxy */ public static Object create(Class iface, diff --git a/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/io/retry/RetryUtils.java b/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/io/retry/RetryUtils.java index c035a42d4a751..d2fb070ee2c25 100644 --- a/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/io/retry/RetryUtils.java +++ b/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/io/retry/RetryUtils.java @@ -48,7 +48,7 @@ public class RetryUtils { * - non-IOException. * * - * @param conf + * @param conf configuration. * @param retryPolicyEnabledKey conf property key for enabling retry * @param defaultRetryPolicyEnabled default retryPolicyEnabledKey conf value * @param retryPolicySpecKey conf property key for retry policy spec @@ -168,7 +168,7 @@ public String toString() { * Retry policy spec: * N pairs of sleep-time and number-of-retries "s1,n1,s2,n2,..." * - * @param conf + * @param conf configuration. * @param retryPolicyEnabledKey conf property key for enabling retry * @param defaultRetryPolicyEnabled default retryPolicyEnabledKey conf value * @param retryPolicySpecKey conf property key for retry policy spec diff --git a/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/io/serializer/Deserializer.java b/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/io/serializer/Deserializer.java index 3c8dfccafa8bb..4bdd60d90c382 100644 --- a/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/io/serializer/Deserializer.java +++ b/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/io/serializer/Deserializer.java @@ -35,13 +35,15 @@ * other producers may read from the input between calls to * {@link #deserialize(Object)}. *

    - * @param + * @param generic type. */ @InterfaceAudience.LimitedPrivate({"HDFS", "MapReduce"}) @InterfaceStability.Evolving public interface Deserializer { /** *

    Prepare the deserializer for reading.

    + * @param in input stream. + * @throws IOException raised on errors performing I/O. */ void open(InputStream in) throws IOException; @@ -53,12 +55,15 @@ public interface Deserializer { * stream. Otherwise, if the object t is null a new * deserialized object will be created. *

    + * @param t t. * @return the deserialized object + * @throws IOException raised on errors performing I/O. */ T deserialize(T t) throws IOException; /** *

    Close the underlying input stream and clear up any resources.

    + * @throws IOException raised on errors performing I/O. */ void close() throws IOException; } diff --git a/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/io/serializer/DeserializerComparator.java b/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/io/serializer/DeserializerComparator.java index 05205c5523cc6..29c04f66d4370 100644 --- a/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/io/serializer/DeserializerComparator.java +++ b/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/io/serializer/DeserializerComparator.java @@ -37,7 +37,7 @@ * implementation of {@link RawComparator} that operates directly * on byte representations. *

    - * @param + * @param generic type. */ @InterfaceAudience.LimitedPrivate({"HDFS", "MapReduce"}) @InterfaceStability.Evolving diff --git a/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/io/serializer/JavaSerializationComparator.java b/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/io/serializer/JavaSerializationComparator.java index f9bf692f1fcc8..d53f7ab75c503 100644 --- a/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/io/serializer/JavaSerializationComparator.java +++ b/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/io/serializer/JavaSerializationComparator.java @@ -31,7 +31,7 @@ * {@link Deserializer} to deserialize objects that are then compared via * their {@link Comparable} interfaces. *

    - * @param + * @param generic type. * @see JavaSerialization */ @InterfaceAudience.Public diff --git a/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/io/serializer/Serialization.java b/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/io/serializer/Serialization.java index 6f2097f7bf9da..0793dc1ca0184 100644 --- a/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/io/serializer/Serialization.java +++ b/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/io/serializer/Serialization.java @@ -25,7 +25,7 @@ *

    * Encapsulates a {@link Serializer}/{@link Deserializer} pair. *

    - * @param + * @param generic type. */ @InterfaceAudience.LimitedPrivate({"HDFS", "MapReduce"}) @InterfaceStability.Evolving @@ -34,16 +34,21 @@ public interface Serialization { /** * Allows clients to test whether this {@link Serialization} * supports the given class. + * + * @param c class. + * @return if accept true,not false. */ boolean accept(Class c); /** * @return a {@link Serializer} for the given class. + * @param c class. */ Serializer getSerializer(Class c); /** * @return a {@link Deserializer} for the given class. + * @param c class. */ Deserializer getDeserializer(Class c); } diff --git a/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/io/serializer/SerializationFactory.java b/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/io/serializer/SerializationFactory.java index ce0c3fe398eed..b531ae85233e8 100644 --- a/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/io/serializer/SerializationFactory.java +++ b/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/io/serializer/SerializationFactory.java @@ -52,6 +52,8 @@ public class SerializationFactory extends Configured { * property from conf, which is a comma-delimited list of * classnames. *

    + * + * @param conf configuration. */ public SerializationFactory(Configuration conf) { super(conf); diff --git a/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/io/serializer/Serializer.java b/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/io/serializer/Serializer.java index 5ada541370ee0..c44b3678fc3be 100644 --- a/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/io/serializer/Serializer.java +++ b/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/io/serializer/Serializer.java @@ -35,23 +35,28 @@ * other producers may write to the output between calls to * {@link #serialize(Object)}. *

    - * @param + * @param generic type. */ @InterfaceAudience.LimitedPrivate({"HDFS", "MapReduce"}) @InterfaceStability.Evolving public interface Serializer { /** *

    Prepare the serializer for writing.

    + * @param out output stream. + * @throws IOException raised on errors performing I/O. */ void open(OutputStream out) throws IOException; /** *

    Serialize t to the underlying output stream.

    + * @param t t. + * @throws IOException raised on errors performing I/O. */ void serialize(T t) throws IOException; /** *

    Close the underlying output stream and clear up any resources.

    + * @throws IOException raised on errors performing I/O. */ void close() throws IOException; } diff --git a/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/io/serializer/avro/AvroReflectSerialization.java b/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/io/serializer/avro/AvroReflectSerialization.java index cfbc60d10452b..544958e682a50 100644 --- a/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/io/serializer/avro/AvroReflectSerialization.java +++ b/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/io/serializer/avro/AvroReflectSerialization.java @@ -64,7 +64,7 @@ public synchronized boolean accept(Class c) { private void getPackages() { String[] pkgList = getConf().getStrings(AVRO_REFLECT_PACKAGES); - packages = new HashSet(); + packages = new HashSet<>(); if (pkgList != null) { for (String pkg : pkgList) { packages.add(pkg.trim()); diff --git a/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/io/serializer/avro/AvroSerialization.java b/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/io/serializer/avro/AvroSerialization.java index f340cb3a98a44..2327fd2d55a2e 100644 --- a/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/io/serializer/avro/AvroSerialization.java +++ b/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/io/serializer/avro/AvroSerialization.java @@ -61,18 +61,24 @@ public Serializer getSerializer(Class c) { /** * Return an Avro Schema instance for the given class. + * @param t Generics Type T. + * @return schema. */ @InterfaceAudience.Private public abstract Schema getSchema(T t); /** * Create and return Avro DatumWriter for the given class. + * @param clazz clazz. + * @return DatumWriter. */ @InterfaceAudience.Private public abstract DatumWriter getWriter(Class clazz); /** * Create and return Avro DatumReader for the given class. + * @param clazz clazz. + * @return DatumReader. */ @InterfaceAudience.Private public abstract DatumReader getReader(Class clazz); diff --git a/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/ipc/AlignmentContext.java b/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/ipc/AlignmentContext.java index fbf825bcb91b1..3d309235fe891 100644 --- a/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/ipc/AlignmentContext.java +++ b/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/ipc/AlignmentContext.java @@ -71,7 +71,7 @@ public interface AlignmentContext { * misaligned with the client state. * See implementation for more details. * @return state id required for the server to execute the call. - * @throws IOException + * @throws IOException raised on errors performing I/O. */ long receiveRequestState(RpcRequestHeaderProto header, long threshold) throws IOException; diff --git a/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/ipc/CallQueueManager.java b/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/ipc/CallQueueManager.java index 6cc2540c174d3..fa6f34adaf3bd 100644 --- a/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/ipc/CallQueueManager.java +++ b/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/ipc/CallQueueManager.java @@ -405,6 +405,12 @@ public static int[] getDefaultQueueCapacityWeights(int priorityLevels) { /** * Replaces active queue with the newly requested one and transfers * all calls to the newQ before returning. + * + * @param schedulerClass input schedulerClass. + * @param queueClassToUse input queueClassToUse. + * @param maxSize input maxSize. + * @param ns input ns. + * @param conf input configuration. */ public synchronized void swapQueue( Class schedulerClass, diff --git a/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/ipc/Client.java b/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/ipc/Client.java index 49432aff11789..2fe8aca85ed9a 100644 --- a/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/ipc/Client.java +++ b/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/ipc/Client.java @@ -112,7 +112,12 @@ protected Boolean initialValue() { return (AsyncGet) ASYNC_RPC_RESPONSE.get(); } - /** Set call id and retry count for the next call. */ + /** + * Set call id and retry count for the next call. + * @param cid input cid. + * @param rc input rc. + * @param externalHandler input externalHandler. + */ public static void setCallIdAndRetryCount(int cid, int rc, Object externalHandler) { Preconditions.checkArgument(cid != RpcConstants.INVALID_CALL_ID); @@ -1349,8 +1354,14 @@ private void cleanupCalls() { } } - /** Construct an IPC client whose values are of the given {@link Writable} - * class. */ + /** + * Construct an IPC client whose values are of the given {@link Writable} + * class. + * + * @param valueClass input valueClass. + * @param conf input configuration. + * @param factory input factory. + */ public Client(Class valueClass, Configuration conf, SocketFactory factory) { this.valueClass = valueClass; @@ -1372,9 +1383,9 @@ public Client(Class valueClass, Configuration conf, } /** - * Construct an IPC client with the default SocketFactory - * @param valueClass - * @param conf + * Construct an IPC client with the default SocketFactory. + * @param valueClass input valueClass. + * @param conf input Configuration. */ public Client(Class valueClass, Configuration conf) { this(valueClass, conf, NetUtils.getDefaultSocketFactory(conf)); @@ -1432,7 +1443,7 @@ public void stop() { * Make a call, passing rpcRequest, to the IPC server defined by * remoteId, returning the rpc respond. * - * @param rpcKind + * @param rpcKind - input rpcKind. * @param rpcRequest - contains serialized method and method parameters * @param remoteId - the target rpc server * @param fallbackToSimpleAuth - set to true or false during this method to @@ -1440,6 +1451,7 @@ public void stop() { * @return the rpc response * Throws exceptions if there are network problems or if the remote code * threw an exception. + * @throws IOException raised on errors performing I/O. */ public Writable call(RPC.RpcKind rpcKind, Writable rpcRequest, ConnectionId remoteId, AtomicBoolean fallbackToSimpleAuth) @@ -1760,7 +1772,7 @@ public int getMaxRetriesOnSasl() { return maxRetriesOnSasl; } - /** max connection retries on socket time outs */ + /** @return max connection retries on socket time outs */ public int getMaxRetriesOnSocketTimeouts() { return maxRetriesOnSocketTimeouts; } diff --git a/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/ipc/ClientCache.java b/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/ipc/ClientCache.java index b7257c8b2a69c..c5d0183dba25d 100644 --- a/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/ipc/ClientCache.java +++ b/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/ipc/ClientCache.java @@ -93,6 +93,8 @@ public synchronized Client getClient(Configuration conf, SocketFactory factory) /** * Stop a RPC client connection * A RPC client is closed only when its reference count becomes zero. + * + * @param client input client. */ public void stopClient(Client client) { if (Client.LOG.isDebugEnabled()) { diff --git a/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/ipc/ClientId.java b/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/ipc/ClientId.java index 152e062392fcc..bab1de753f265 100644 --- a/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/ipc/ClientId.java +++ b/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/ipc/ClientId.java @@ -35,7 +35,7 @@ public class ClientId { private static final int shiftWidth = 8; /** - * Return clientId as byte[] + * @return Return clientId as byte[]. */ public static byte[] getClientId() { UUID uuid = UUID.randomUUID(); @@ -45,7 +45,10 @@ public static byte[] getClientId() { return buf.array(); } - /** Convert a clientId byte[] to string */ + /** + * @return Convert a clientId byte[] to string. + * @param clientId input clientId. + */ public static String toString(byte[] clientId) { // clientId can be null or an empty array if (clientId == null || clientId.length == 0) { @@ -74,7 +77,10 @@ public static long getLsb(byte[] clientId) { return lsb; } - /** Convert from clientId string byte[] representation of clientId */ + /** + * @return Convert from clientId string byte[] representation of clientId. + * @param id input id. + */ public static byte[] toBytes(String id) { if (id == null || "".equals(id)) { return new byte[0]; diff --git a/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/ipc/GenericRefreshProtocol.java b/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/ipc/GenericRefreshProtocol.java index bfa055bcb0997..10e661a3095cd 100644 --- a/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/ipc/GenericRefreshProtocol.java +++ b/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/ipc/GenericRefreshProtocol.java @@ -41,7 +41,11 @@ public interface GenericRefreshProtocol { /** * Refresh the resource based on identity passed in. - * @throws IOException + * + * @param identifier input identifier. + * @param args input args. + * @throws IOException raised on errors performing I/O. + * @return Collection RefreshResponse. */ @Idempotent Collection refresh(String identifier, String[] args) diff --git a/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/ipc/ProtobufHelper.java b/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/ipc/ProtobufHelper.java index 1e110b9011313..9ed0640c8dcfa 100644 --- a/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/ipc/ProtobufHelper.java +++ b/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/ipc/ProtobufHelper.java @@ -85,7 +85,7 @@ public static IOException getRemoteException( /** * Get the ByteString for frequently used fixed and small set strings. * @param key string - * @return + * @return the ByteString for frequently used fixed and small set strings. */ public static ByteString getFixedByteString(Text key) { ByteString value = FIXED_BYTESTRING_CACHE.get(key); @@ -99,7 +99,7 @@ public static ByteString getFixedByteString(Text key) { /** * Get the ByteString for frequently used fixed and small set strings. * @param key string - * @return + * @return ByteString for frequently used fixed and small set strings. */ public static ByteString getFixedByteString(String key) { ByteString value = FIXED_BYTESTRING_CACHE.get(key); diff --git a/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/ipc/ProtobufRpcEngine.java b/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/ipc/ProtobufRpcEngine.java index c4457a653e35f..e53f57b1fc9dd 100644 --- a/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/ipc/ProtobufRpcEngine.java +++ b/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/ipc/ProtobufRpcEngine.java @@ -144,6 +144,10 @@ protected Invoker(Class protocol, InetSocketAddress addr, /** * This constructor takes a connectionId, instead of creating a new one. + * @param protocol input protocol. + * @param connId input connId. + * @param conf input Configuration. + * @param factory input factory. */ protected Invoker(Class protocol, Client.ConnectionId connId, Configuration conf, SocketFactory factory) { @@ -423,6 +427,10 @@ public static ProtobufRpcEngineCallback registerForDeferredResponse() { * @param portRangeConfig A config parameter that can be used to restrict * the range of ports used when port is 0 (an ephemeral port) * @param alignmentContext provides server state info on client responses + * @param secretManager input secretManager. + * @param queueSizePerHandler input queueSizePerHandler. + * @param numReaders input numReaders. + * @throws IOException raised on errors performing I/O. */ public Server(Class protocolClass, Object protocolImpl, Configuration conf, String bindAddress, int port, int numHandlers, diff --git a/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/ipc/ProtobufRpcEngine2.java b/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/ipc/ProtobufRpcEngine2.java index e1ee374282897..3a8c6275820c4 100644 --- a/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/ipc/ProtobufRpcEngine2.java +++ b/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/ipc/ProtobufRpcEngine2.java @@ -151,6 +151,11 @@ protected Invoker(Class protocol, InetSocketAddress addr, /** * This constructor takes a connectionId, instead of creating a new one. + * + * @param protocol input protocol. + * @param connId input connId. + * @param conf input Configuration. + * @param factory input factory. */ protected Invoker(Class protocol, Client.ConnectionId connId, Configuration conf, SocketFactory factory) { @@ -458,6 +463,7 @@ public static ProtobufRpcEngineCallback2 registerForDeferredResponse2() { * @param portRangeConfig A config parameter that can be used to restrict * the range of ports used when port is 0 (an ephemeral port) * @param alignmentContext provides server state info on client responses + * @throws IOException raised on errors performing I/O. */ public Server(Class protocolClass, Object protocolImpl, Configuration conf, String bindAddress, int port, int numHandlers, diff --git a/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/ipc/ProtocolMetaInterface.java b/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/ipc/ProtocolMetaInterface.java index 29c07ac29cb87..f23c05936a356 100644 --- a/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/ipc/ProtocolMetaInterface.java +++ b/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/ipc/ProtocolMetaInterface.java @@ -36,7 +36,7 @@ public interface ProtocolMetaInterface { * It is assumed that all method names are unique for a protocol. * @param methodName The name of the method * @return true if method is supported, otherwise false. - * @throws IOException + * @throws IOException raised on errors performing I/O. */ public boolean isMethodSupported(String methodName) throws IOException; } diff --git a/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/ipc/ProtocolProxy.java b/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/ipc/ProtocolProxy.java index cc66958d14e08..f5f212b29276d 100644 --- a/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/ipc/ProtocolProxy.java +++ b/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/ipc/ProtocolProxy.java @@ -69,7 +69,7 @@ private void fetchServerMethods(Method method) throws IOException { } int[] serverMethodsCodes = serverInfo.getMethods(); if (serverMethodsCodes != null) { - serverMethods = new HashSet(serverMethodsCodes.length); + serverMethods = new HashSet<>(serverMethodsCodes.length); for (int m : serverMethodsCodes) { this.serverMethods.add(Integer.valueOf(m)); } @@ -85,11 +85,12 @@ public T getProxy() { } /** - * Check if a method is supported by the server or not + * Check if a method is supported by the server or not. * * @param methodName a method's name in String format * @param parameterTypes a method's parameter types * @return true if the method is supported by the server + * @throws IOException raised on errors performing I/O. */ public synchronized boolean isMethodSupported(String methodName, Class... parameterTypes) diff --git a/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/ipc/RPC.java b/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/ipc/RPC.java index 3bbd82d153a09..818305b316984 100644 --- a/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/ipc/RPC.java +++ b/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/ipc/RPC.java @@ -150,6 +150,9 @@ static Class[] getProtocolInterfaces(Class protocol) { * Get the protocol name. * If the protocol class has a ProtocolAnnotation, then get the protocol * name from the annotation; otherwise the class name is the protocol name. + * + * @param protocol input protocol. + * @return protocol name. */ static public String getProtocolName(Class protocol) { if (protocol == null) { @@ -164,6 +167,9 @@ static public String getProtocolName(Class protocol) { * If the protocol class has a ProtocolAnnotation, * then get the protocol version from the annotation; * otherwise get it from the versionID field of the protocol class. + * + * @param protocol input protocol. + * @return ProtocolVersion. */ static public long getProtocolVersion(Class protocol) { if (protocol == null) { @@ -258,14 +264,14 @@ public String getInterfaceName() { } /** - * Get the client's preferred version + * @return Get the client's preferred version. */ public long getClientVersion() { return clientVersion; } /** - * Get the server's agreed to version. + * @return Get the server's agreed to version. */ public long getServerVersion() { return serverVersion; @@ -286,8 +292,9 @@ public RpcErrorCodeProto getRpcErrorCodeProto() { } /** - * Get a proxy connection to a remote server - * + * Get a proxy connection to a remote server. + * + * @param Generics Type T. * @param protocol protocol class * @param clientVersion client version * @param addr remote address @@ -306,8 +313,9 @@ public static T waitForProxy( /** * Get a protocol proxy that contains a proxy connection to a remote server - * and a set of methods that are supported by the server - * + * and a set of methods that are supported by the server. + * + * @param Generics Type T. * @param protocol protocol class * @param clientVersion client version * @param addr remote address @@ -324,8 +332,9 @@ public static ProtocolProxy waitForProtocolProxy(Class protocol, } /** - * Get a proxy connection to a remote server - * + * Get a proxy connection to a remote server. + * + * @param Generics Type T. * @param protocol protocol class * @param clientVersion client version * @param addr remote address @@ -344,7 +353,8 @@ public static T waitForProxy(Class protocol, long clientVersion, /** * Get a protocol proxy that contains a proxy connection to a remote server * and a set of methods that are supported by the server - * + * + * @param Generics Type T. * @param protocol protocol class * @param clientVersion client version * @param addr remote address @@ -362,8 +372,9 @@ public static ProtocolProxy waitForProtocolProxy(Class protocol, } /** - * Get a proxy connection to a remote server - * + * Get a proxy connection to a remote server. + * + * @param Generics Type T. * @param protocol protocol class * @param clientVersion client version * @param addr remote address @@ -384,16 +395,18 @@ public static T waitForProxy(Class protocol, /** * Get a protocol proxy that contains a proxy connection to a remote server - * and a set of methods that are supported by the server - * + * and a set of methods that are supported by the server. + * + * @param Generics Type. * @param protocol protocol class * @param clientVersion client version * @param addr remote address * @param conf configuration to use * @param rpcTimeout timeout for each RPC + * @param connectionRetryPolicy input connectionRetryPolicy. * @param timeout time in milliseconds before giving up * @return the proxy - * @throws IOException if the far end through a RemoteException + * @throws IOException if the far end through a RemoteException. */ public static ProtocolProxy waitForProtocolProxy(Class protocol, long clientVersion, @@ -439,9 +452,18 @@ public static ProtocolProxy waitForProtocolProxy(Class protocol, } } - /** Construct a client-side proxy object that implements the named protocol, + /** + * Construct a client-side proxy object that implements the named protocol, * talking to a server at the named address. - * @param */ + * @param Generics Type T. + * @param protocol input protocol. + * @param clientVersion input clientVersion. + * @param addr input addr. + * @param conf input Configuration. + * @param factory input factory. + * @throws IOException raised on errors performing I/O. + * @return proxy. + */ public static T getProxy(Class protocol, long clientVersion, InetSocketAddress addr, Configuration conf, @@ -452,8 +474,9 @@ public static T getProxy(Class protocol, /** * Get a protocol proxy that contains a proxy connection to a remote server - * and a set of methods that are supported by the server - * + * and a set of methods that are supported by the server. + * + * @param Generics Type T. * @param protocol protocol class * @param clientVersion client version * @param addr remote address @@ -470,9 +493,21 @@ public static ProtocolProxy getProtocolProxy(Class protocol, return getProtocolProxy(protocol, clientVersion, addr, ugi, conf, factory); } - /** Construct a client-side proxy object that implements the named protocol, + /** + * Construct a client-side proxy object that implements the named protocol, * talking to a server at the named address. - * @param */ + * + * @param Generics Type T. + * @param protocol input protocol. + * @param clientVersion input clientVersion. + * @param addr input addr. + * @param ticket input tocket. + * @param conf input conf. + * @param factory input factory. + * @return the protocol proxy. + * @throws IOException raised on errors performing I/O. + * + */ public static T getProxy(Class protocol, long clientVersion, InetSocketAddress addr, @@ -486,7 +521,8 @@ public static T getProxy(Class protocol, /** * Get a protocol proxy that contains a proxy connection to a remote server * and a set of methods that are supported by the server - * + * + * @param Generics Type T. * @param protocol protocol class * @param clientVersion client version * @param addr remote address @@ -509,8 +545,8 @@ public static ProtocolProxy getProtocolProxy(Class protocol, /** * Construct a client-side proxy that implements the named protocol, * talking to a server at the named address. - * @param - * + * + * @param Generics Type T. * @param protocol protocol * @param clientVersion client's version * @param addr server address @@ -534,8 +570,9 @@ public static T getProxy(Class protocol, /** * Get a protocol proxy that contains a proxy connection to a remote server - * and a set of methods that are supported by the server - * + * and a set of methods that are supported by the server. + * + * @param Generics Type T. * @param protocol protocol * @param clientVersion client's version * @param addr server address @@ -561,8 +598,9 @@ public static ProtocolProxy getProtocolProxy(Class protocol, /** * Get a protocol proxy that contains a proxy connection to a remote server - * and a set of methods that are supported by the server + * and a set of methods that are supported by the server. * + * @param Generics Type T. * @param protocol protocol * @param clientVersion client's version * @param addr server address @@ -609,6 +647,7 @@ public static ProtocolProxy getProtocolProxy(Class protocol, * @param fallbackToSimpleAuth set to true or false during calls to indicate * if a secure client falls back to simple auth * @param alignmentContext state alignment context + * @param Generics Type T. * @return the proxy * @throws IOException if any error occurs */ @@ -632,15 +671,15 @@ public static ProtocolProxy getProtocolProxy(Class protocol, } /** - * Construct a client-side proxy object with the default SocketFactory - * @param - * - * @param protocol - * @param clientVersion - * @param addr - * @param conf + * Construct a client-side proxy object with the default SocketFactory. + * + * @param Generics Type T. + * @param protocol input protocol. + * @param clientVersion input clientVersion. + * @param addr input addr. + * @param conf input Configuration. * @return a proxy instance - * @throws IOException + * @throws IOException if the thread is interrupted. */ public static T getProxy(Class protocol, long clientVersion, @@ -651,7 +690,8 @@ public static T getProxy(Class protocol, } /** - * Returns the server address for a given proxy. + * @return Returns the server address for a given proxy. + * @param proxy input proxy. */ public static InetSocketAddress getServerAddress(Object proxy) { return getConnectionIdForProxy(proxy).getAddress(); @@ -678,12 +718,13 @@ public static ConnectionId getConnectionIdForProxy(Object proxy) { * Get a protocol proxy that contains a proxy connection to a remote server * and a set of methods that are supported by the server * - * @param protocol - * @param clientVersion - * @param addr - * @param conf + * @param protocol input protocol. + * @param clientVersion input clientVersion. + * @param addr input addr. + * @param conf input configuration. + * @param Generics Type T. * @return a protocol proxy - * @throws IOException + * @throws IOException if the thread is interrupted. */ public static ProtocolProxy getProtocolProxy(Class protocol, long clientVersion, @@ -767,75 +808,109 @@ public Builder(Configuration conf) { this.conf = conf; } - /** Mandatory field */ + /** + * @return Mandatory field. + * @param protocol input protocol. + */ public Builder setProtocol(Class protocol) { this.protocol = protocol; return this; } - /** Mandatory field */ + /** + * @return Mandatory field. + * @param instance input instance. + */ public Builder setInstance(Object instance) { this.instance = instance; return this; } - /** Default: 0.0.0.0 */ + /** + * @return Default: 0.0.0.0. + * @param bindAddress input bindAddress. + */ public Builder setBindAddress(String bindAddress) { this.bindAddress = bindAddress; return this; } - /** Default: 0 */ + /** + * @return Default: 0. + * @param port input port. + */ public Builder setPort(int port) { this.port = port; return this; } - /** Default: 1 */ + /** + * @return Default: 1. + * @param numHandlers input numHandlers. + */ public Builder setNumHandlers(int numHandlers) { this.numHandlers = numHandlers; return this; } - /** Default: -1 */ + /** + * @return Default: -1. + * @param numReaders input numReaders. + */ public Builder setnumReaders(int numReaders) { this.numReaders = numReaders; return this; } - /** Default: -1 */ + /** + * @return Default: -1. + * @param queueSizePerHandler + * input queueSizePerHandler. + */ public Builder setQueueSizePerHandler(int queueSizePerHandler) { this.queueSizePerHandler = queueSizePerHandler; return this; } - /** Default: false */ + /** + * @return Default: false. + * @param verbose input verbose. + */ public Builder setVerbose(boolean verbose) { this.verbose = verbose; return this; } - /** Default: null */ + /** + * @return Default: null. + * @param secretManager input secretManager. + */ public Builder setSecretManager( SecretManager secretManager) { this.secretManager = secretManager; return this; } - /** Default: null */ + /** + * @return Default: null. + * @param portRangeConfig input portRangeConfig. + */ public Builder setPortRangeConfig(String portRangeConfig) { this.portRangeConfig = portRangeConfig; return this; } - /** Default: null */ + /** + * @return Default: null. + * @param alignmentContext input alignmentContext. + */ public Builder setAlignmentContext(AlignmentContext alignmentContext) { this.alignmentContext = alignmentContext; return this; } /** - * Build the RPC Server. + * @return Build the RPC Server. * @throws IOException on error * @throws HadoopIllegalArgumentException when mandatory fields are not set */ @@ -1077,6 +1152,7 @@ private void initProtocolMetaInfo(Configuration conf) { /** * Add a protocol to the existing server. + * @param rpcKind - input rpcKind * @param protocolClass - the protocol class * @param protocolImpl - the impl of the protocol that will be called * @return the server (for convenience) diff --git a/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/ipc/RefreshCallQueueProtocol.java b/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/ipc/RefreshCallQueueProtocol.java index 553f9a00d4cca..b1aa0197040a2 100644 --- a/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/ipc/RefreshCallQueueProtocol.java +++ b/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/ipc/RefreshCallQueueProtocol.java @@ -41,7 +41,7 @@ public interface RefreshCallQueueProtocol { /** * Refresh the callqueue. - * @throws IOException + * @throws IOException raised on errors performing I/O. */ @Idempotent void refreshCallQueue() throws IOException; diff --git a/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/ipc/RefreshRegistry.java b/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/ipc/RefreshRegistry.java index 95f1323410fe0..0cc0b8ba3d8b8 100644 --- a/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/ipc/RefreshRegistry.java +++ b/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/ipc/RefreshRegistry.java @@ -72,6 +72,7 @@ public synchronized void register(String identifier, RefreshHandler handler) { /** * Remove the registered object for a given identity. * @param identifier the resource to unregister + * @param handler input handler. * @return the true if removed */ public synchronized boolean unregister(String identifier, RefreshHandler handler) { diff --git a/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/ipc/RemoteException.java b/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/ipc/RemoteException.java index f1142d35e72c2..da08c3d152e61 100644 --- a/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/ipc/RemoteException.java +++ b/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/ipc/RemoteException.java @@ -124,8 +124,9 @@ private IOException instantiateException(Class cls) } /** - * Create RemoteException from attributes - * @param attrs may not be null + * Create RemoteException from attributes. + * @param attrs may not be null. + * @return RemoteException. */ public static RemoteException valueOf(Attributes attrs) { return new RemoteException(attrs.getValue("class"), diff --git a/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/ipc/RetryCache.java b/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/ipc/RetryCache.java index c9e04ab82b615..3d64a84bfb46f 100644 --- a/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/ipc/RetryCache.java +++ b/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/ipc/RetryCache.java @@ -49,11 +49,11 @@ public class RetryCache { private static final int MAX_CAPACITY = 16; /** - * CacheEntry is tracked using unique client ID and callId of the RPC request + * CacheEntry is tracked using unique client ID and callId of the RPC request. */ public static class CacheEntry implements LightWeightCache.Entry { /** - * Processing state of the requests + * Processing state of the requests. */ private static byte INPROGRESS = 0; private static byte SUCCESS = 1; @@ -233,7 +233,7 @@ public RetryCacheMetrics getMetricsForTests() { } /** - * This method returns cache name for metrics. + * @return This method returns cache name for metrics. */ public String getCacheName() { return cacheName; @@ -302,6 +302,9 @@ private CacheEntry waitForCompletion(CacheEntry newEntry) { /** * Add a new cache entry into the retry cache. The cache entry consists of * clientId and callId extracted from editlog. + * + * @param clientId input clientId. + * @param callId input callId. */ public void addCacheEntry(byte[] clientId, int callId) { CacheEntry newEntry = new CacheEntry(clientId, callId, System.nanoTime() @@ -340,7 +343,11 @@ private static CacheEntryWithPayload newEntry(Object payload, payload, System.nanoTime() + expirationTime); } - /** Static method that provides null check for retryCache */ + /** + * Static method that provides null check for retryCache. + * @param cache input Cache. + * @return CacheEntry. + */ public static CacheEntry waitForCompletion(RetryCache cache) { if (skipRetryCache()) { return null; @@ -349,7 +356,12 @@ public static CacheEntry waitForCompletion(RetryCache cache) { .waitForCompletion(newEntry(cache.expirationTime)) : null; } - /** Static method that provides null check for retryCache */ + /** + * Static method that provides null check for retryCache. + * @param cache input cache. + * @param payload input payload. + * @return CacheEntryWithPayload. + */ public static CacheEntryWithPayload waitForCompletion(RetryCache cache, Object payload) { if (skipRetryCache()) { diff --git a/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/ipc/RpcClientUtil.java b/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/ipc/RpcClientUtil.java index 0ce78e54a43a0..4af35ad9270f1 100644 --- a/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/ipc/RpcClientUtil.java +++ b/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/ipc/RpcClientUtil.java @@ -103,7 +103,7 @@ private static Map getVersionSignatureMap( * @param version The version at the client. * @param methodName Name of the method. * @return true if the method is supported, false otherwise. - * @throws IOException + * @throws IOException raised on errors performing I/O. */ public static boolean isMethodSupported(Object rpcProxy, Class protocol, RPC.RpcKind rpcKind, long version, String methodName) throws IOException { @@ -200,6 +200,8 @@ private static ProtocolMetaInfoPB getProtocolMetaInfoProxy(Object proxy, * * the format we want is: * ClientNamenodeProtocol#getServerDefaults + * @param method input method. + * @return methodToTraceString. */ public static String methodToTraceString(Method method) { Class clazz = method.getDeclaringClass(); @@ -221,6 +223,8 @@ public static String methodToTraceString(Method method) { * * the format we want is: * ClientProtocol#getBlockLocations + * @param fullName input fullName. + * @return toTraceName. */ public static String toTraceName(String fullName) { int lastPeriod = fullName.lastIndexOf('.'); diff --git a/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/ipc/RpcEngine.java b/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/ipc/RpcEngine.java index 0f5769e705028..afc9d035b097c 100644 --- a/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/ipc/RpcEngine.java +++ b/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/ipc/RpcEngine.java @@ -36,15 +36,44 @@ @InterfaceStability.Evolving public interface RpcEngine { - /** Construct a client-side proxy object. - * @param */ + /** + * Construct a client-side proxy object. + * + * @param Generics Type T. + * @param protocol input protocol. + * @param clientVersion input clientVersion. + * @param addr input addr. + * @param ticket input ticket. + * @param conf input Configuration. + * @param factory input factory. + * @param rpcTimeout input rpcTimeout. + * @param connectionRetryPolicy input connectionRetryPolicy. + * @throws IOException raised on errors performing I/O. + * @return ProtocolProxy. + */ ProtocolProxy getProxy(Class protocol, long clientVersion, InetSocketAddress addr, UserGroupInformation ticket, Configuration conf, SocketFactory factory, int rpcTimeout, RetryPolicy connectionRetryPolicy) throws IOException; - /** Construct a client-side proxy object. */ + /** + * Construct a client-side proxy object. + * + * @param Generics Type T. + * @param protocol input protocol. + * @param clientVersion input clientVersion. + * @param addr input addr. + * @param ticket input tocket. + * @param conf input Configuration. + * @param factory input factory. + * @param rpcTimeout input rpcTimeout. + * @param connectionRetryPolicy input connectionRetryPolicy. + * @param fallbackToSimpleAuth input fallbackToSimpleAuth. + * @param alignmentContext input alignmentContext. + * @throws IOException raised on errors performing I/O. + * @return ProtocolProxy. + */ ProtocolProxy getProxy(Class protocol, long clientVersion, InetSocketAddress addr, UserGroupInformation ticket, Configuration conf, @@ -87,7 +116,7 @@ RPC.Server getServer(Class protocol, Object instance, String bindAddress, * @param conf, Configuration. * @param factory, Socket factory. * @return Proxy object. - * @throws IOException + * @throws IOException raised on errors performing I/O. */ ProtocolProxy getProtocolMetaInfoProxy( ConnectionId connId, Configuration conf, SocketFactory factory) diff --git a/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/ipc/RpcScheduler.java b/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/ipc/RpcScheduler.java index 8c423b8e5e1bd..bffe5f2d257fc 100644 --- a/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/ipc/RpcScheduler.java +++ b/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/ipc/RpcScheduler.java @@ -26,7 +26,8 @@ */ public interface RpcScheduler { /** - * Returns priority level greater than zero as a hint for scheduling. + * @return Returns priority level greater than zero as a hint for scheduling. + * @param obj input obj. */ int getPriorityLevel(Schedulable obj); @@ -37,6 +38,12 @@ public interface RpcScheduler { * implementations. It will not be called by any Hadoop code, and should not * be implemented by new implementations. * + * @param name input name. + * @param priorityLevel input priorityLevel. + * @param queueTime input queueTime. + * @param processingTime input processingTime. + * @throws UnsupportedOperationException + * the requested operation is not supported. * @deprecated Use * {@link #addResponseTime(String, Schedulable, ProcessingDetails)} instead. */ diff --git a/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/ipc/RpcServerException.java b/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/ipc/RpcServerException.java index 992997ead25de..ce4aac54b6cd2 100644 --- a/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/ipc/RpcServerException.java +++ b/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/ipc/RpcServerException.java @@ -47,14 +47,14 @@ public RpcServerException(final String message, final Throwable cause) { } /** - * get the rpc status corresponding to this exception + * @return get the rpc status corresponding to this exception. */ public RpcStatusProto getRpcStatusProto() { return RpcStatusProto.ERROR; } /** - * get the detailed rpc status corresponding to this exception + * @return get the detailed rpc status corresponding to this exception. */ public RpcErrorCodeProto getRpcErrorCodeProto() { return RpcErrorCodeProto.ERROR_RPC_SERVER; diff --git a/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/ipc/Server.java b/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/ipc/Server.java index c5732c68b1517..e79612f7a5a0f 100644 --- a/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/ipc/Server.java +++ b/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/ipc/Server.java @@ -65,9 +65,12 @@ import java.util.concurrent.BlockingQueue; import java.util.concurrent.ConcurrentHashMap; import java.util.concurrent.LinkedBlockingQueue; +import java.util.concurrent.ScheduledExecutorService; +import java.util.concurrent.ScheduledThreadPoolExecutor; import java.util.concurrent.TimeUnit; import java.util.concurrent.atomic.AtomicInteger; import java.util.concurrent.atomic.AtomicLong; +import java.util.concurrent.atomic.LongAdder; import java.util.stream.Collectors; import javax.security.sasl.Sasl; @@ -127,6 +130,8 @@ import org.apache.hadoop.tracing.TraceUtils; import com.fasterxml.jackson.databind.ObjectMapper; import org.apache.hadoop.classification.VisibleForTesting; + +import org.apache.hadoop.thirdparty.com.google.common.util.concurrent.ThreadFactoryBuilder; import org.apache.hadoop.thirdparty.protobuf.ByteString; import org.apache.hadoop.thirdparty.protobuf.CodedOutputStream; import org.apache.hadoop.thirdparty.protobuf.Message; @@ -266,10 +271,10 @@ static class RpcKindMapValue { * Register a RPC kind and the class to deserialize the rpc request. * * Called by static initializers of rpcKind Engines - * @param rpcKind + * @param rpcKind - input rpcKind. * @param rpcRequestWrapperClass - this class is used to deserialze the * the rpc request. - * @param rpcInvoker - use to process the calls on SS. + * @param rpcInvoker - use to process the calls on SS. */ public static void registerProtocolEngine(RPC.RpcKind rpcKind, @@ -328,7 +333,7 @@ static Class getProtocolClass(String protocolName, Configuration conf) return protocol; } - /** Returns the server instance called under or null. May be called under + /** @return Returns the server instance called under or null. May be called under * {@link #call(Writable, long)} implementations, and under {@link Writable} * methods of paramters and return values. Permits applications to access * the server context.*/ @@ -341,7 +346,7 @@ public static Server get() { */ private static final ThreadLocal CurCall = new ThreadLocal(); - /** Get the current call */ + /** @return Get the current call. */ @VisibleForTesting public static ThreadLocal getCurCall() { return CurCall; @@ -368,7 +373,8 @@ public static int getCallRetryCount() { return call != null ? call.retryCount : RpcConstants.INVALID_RETRY_COUNT; } - /** Returns the remote side ip address when invoked inside an RPC + /** + * @return Returns the remote side ip address when invoked inside an RPC * Returns null in case of an error. */ public static InetAddress getRemoteIp() { @@ -377,7 +383,7 @@ public static InetAddress getRemoteIp() { } /** - * Returns the remote side port when invoked inside an RPC + * @return Returns the remote side port when invoked inside an RPC * Returns 0 in case of an error. */ public static int getRemotePort() { @@ -412,14 +418,14 @@ public static String getAuxiliaryPortEstablishedQOP() { } /** - * Returns the clientId from the current RPC request + * @return Returns the clientId from the current RPC request. */ public static byte[] getClientId() { Call call = CurCall.get(); return call != null ? call.clientId : RpcConstants.DUMMY_CLIENT_ID; } - /** Returns remote address as a string when invoked inside an RPC. + /** @return Returns remote address as a string when invoked inside an RPC. * Returns null in case of an error. */ public static String getRemoteAddress() { @@ -441,14 +447,14 @@ public static String getProtocol() { return (call != null) ? call.getProtocol() : null; } - /** Return true if the invocation was through an RPC. + /** @return Return true if the invocation was through an RPC. */ public static boolean isRpcInvocation() { return CurCall.get() != null; } /** - * Return the priority level assigned by call queue to an RPC + * @return Return the priority level assigned by call queue to an RPC * Returns 0 in case no priority is assigned. */ public static int getPriorityLevel() { @@ -499,6 +505,11 @@ protected ResponseBuffer initialValue() { private Responder responder = null; private Handler[] handlers = null; private final AtomicInteger numInProcessHandler = new AtomicInteger(); + private final LongAdder totalRequests = new LongAdder(); + private long lastSeenTotalRequests = 0; + private long totalRequestsPerSecond = 0; + private final long metricsUpdaterInterval; + private final ScheduledExecutorService scheduledExecutorService; private boolean logSlowRPC = false; @@ -514,9 +525,17 @@ public int getNumInProcessHandler() { return numInProcessHandler.get(); } + public long getTotalRequests() { + return totalRequests.sum(); + } + + public long getTotalRequestsPerSecond() { + return totalRequestsPerSecond; + } + /** * Sets slow RPC flag. - * @param logSlowRPCFlag + * @param logSlowRPCFlag input logSlowRPCFlag. */ @VisibleForTesting protected void setLogSlowRPC(boolean logSlowRPCFlag) { @@ -577,6 +596,7 @@ void logSlowRpcCalls(String methodName, Call call, } void updateMetrics(Call call, long startTime, boolean connDropped) { + totalRequests.increment(); // delta = handler + processing + response long deltaNanos = Time.monotonicNowNanos() - startTime; long timestampNanos = call.timestampNanos; @@ -707,6 +727,9 @@ Connection[] getConnections() { /** * Refresh the service authorization ACL for the service handled by this server. + * + * @param conf input Configuration. + * @param provider input PolicyProvider. */ public void refreshServiceAcl(Configuration conf, PolicyProvider provider) { serviceAuthorizationManager.refresh(conf, provider); @@ -715,6 +738,9 @@ public void refreshServiceAcl(Configuration conf, PolicyProvider provider) { /** * Refresh the service authorization ACL for the service handled by this server * using the specified Configuration. + * + * @param conf input Configuration. + * @param provider input provider. */ @Private public void refreshServiceAclWithLoadedConfiguration(Configuration conf, @@ -2380,7 +2406,7 @@ private void checkDataLength(int dataLength) throws IOException { * @return -1 in case of error, else num bytes read so far * @throws IOException - internal error that should not be returned to * client, typically failure to respond to client - * @throws InterruptedException + * @throws InterruptedException - if the thread is interrupted. */ public int readAndProcess() throws IOException, InterruptedException { while (!shouldClose()) { // stop if a fatal response has been sent. @@ -3198,6 +3224,18 @@ protected Server(String bindAddress, int port, * Class, RPC.RpcInvoker)} * This parameter has been retained for compatibility with existing tests * and usage. + * + * @param bindAddress input bindAddress. + * @param port input port. + * @param rpcRequestClass input rpcRequestClass. + * @param handlerCount input handlerCount. + * @param numReaders input numReaders. + * @param queueSizePerHandler input queueSizePerHandler. + * @param conf input Configuration. + * @param serverName input serverName. + * @param secretManager input secretManager. + * @param portRangeConfig input portRangeConfig. + * @throws IOException raised on errors performing I/O. */ @SuppressWarnings("unchecked") protected Server(String bindAddress, int port, @@ -3285,6 +3323,14 @@ protected Server(String bindAddress, int port, this.exceptionsHandler.addTerseLoggingExceptions(StandbyException.class); this.exceptionsHandler.addTerseLoggingExceptions( HealthCheckFailedException.class); + this.metricsUpdaterInterval = + conf.getLong(CommonConfigurationKeysPublic.IPC_SERVER_METRICS_UPDATE_RUNNER_INTERVAL, + CommonConfigurationKeysPublic.IPC_SERVER_METRICS_UPDATE_RUNNER_INTERVAL_DEFAULT); + this.scheduledExecutorService = new ScheduledThreadPoolExecutor(1, + new ThreadFactoryBuilder().setDaemon(true).setNameFormat("Hadoop-Metrics-Updater-%d") + .build()); + this.scheduledExecutorService.scheduleWithFixedDelay(new MetricsUpdateRunner(), + metricsUpdaterInterval, metricsUpdaterInterval, TimeUnit.MILLISECONDS); } public synchronized void addAuxiliaryListener(int auxiliaryPort) @@ -3530,7 +3576,10 @@ Configuration getConf() { return conf; } - /** Sets the socket buffer size used for responding to RPCs */ + /** + * Sets the socket buffer size used for responding to RPCs. + * @param size input size. + */ public void setSocketSendBufSize(int size) { this.socketSendBufferSize = size; } public void setTracer(Tracer t) { @@ -3576,13 +3625,30 @@ public synchronized void stop() { } responder.interrupt(); notifyAll(); + shutdownMetricsUpdaterExecutor(); this.rpcMetrics.shutdown(); this.rpcDetailedMetrics.shutdown(); } - /** Wait for the server to be stopped. + private void shutdownMetricsUpdaterExecutor() { + this.scheduledExecutorService.shutdown(); + try { + boolean isExecutorShutdown = + this.scheduledExecutorService.awaitTermination(3, TimeUnit.SECONDS); + if (!isExecutorShutdown) { + LOG.info("Hadoop Metrics Updater executor could not be shutdown."); + } + } catch (InterruptedException e) { + Thread.currentThread().interrupt(); + LOG.info("Hadoop Metrics Updater executor shutdown interrupted.", e); + } + } + + /** + * Wait for the server to be stopped. * Does not wait for all subthreads to finish. * See {@link #stop()}. + * @throws InterruptedException if the thread is interrupted. */ public synchronized void join() throws InterruptedException { while (running) { @@ -3619,13 +3685,25 @@ public synchronized Set getAuxiliaryListenerAddresses() { * Called for each call. * @deprecated Use {@link #call(RPC.RpcKind, String, * Writable, long)} instead + * @param param input param. + * @param receiveTime input receiveTime. + * @throws Exception if any error occurs. + * @return Call */ @Deprecated public Writable call(Writable param, long receiveTime) throws Exception { return call(RPC.RpcKind.RPC_BUILTIN, null, param, receiveTime); } - /** Called for each call. */ + /** + * Called for each call. + * @param rpcKind input rpcKind. + * @param protocol input protocol. + * @param param input param. + * @param receiveTime input receiveTime. + * @return Call. + * @throws Exception raised on errors performing I/O. + */ public abstract Writable call(RPC.RpcKind rpcKind, String protocol, Writable param, long receiveTime) throws Exception; @@ -3673,7 +3751,7 @@ public int getNumOpenConnections() { } /** - * Get the NumOpenConnections/User. + * @return Get the NumOpenConnections/User. */ public String getNumOpenConnectionsPerUser() { ObjectMapper mapper = new ObjectMapper(); @@ -4025,4 +4103,32 @@ protected int getMaxIdleTime() { public String getServerName() { return serverName; } + + /** + * Server metrics updater thread, used to update some metrics on a regular basis. + * For instance, requests per second. + */ + private class MetricsUpdateRunner implements Runnable { + + private long lastExecuted = 0; + + @Override + public synchronized void run() { + long currentTime = Time.monotonicNow(); + if (lastExecuted == 0) { + lastExecuted = currentTime - metricsUpdaterInterval; + } + long currentTotalRequests = totalRequests.sum(); + long totalRequestsDiff = currentTotalRequests - lastSeenTotalRequests; + lastSeenTotalRequests = currentTotalRequests; + if ((currentTime - lastExecuted) > 0) { + double totalRequestsPerSecInDouble = + (double) totalRequestsDiff / TimeUnit.MILLISECONDS.toSeconds( + currentTime - lastExecuted); + totalRequestsPerSecond = ((long) totalRequestsPerSecInDouble); + } + lastExecuted = currentTime; + } + } + } diff --git a/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/ipc/VersionedProtocol.java b/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/ipc/VersionedProtocol.java index 4d02027a0e688..98daa84187464 100644 --- a/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/ipc/VersionedProtocol.java +++ b/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/ipc/VersionedProtocol.java @@ -46,6 +46,7 @@ public long getProtocolVersion(String protocol, * a list of its supported methods * @see ProtocolSignature#getProtocolSignature(VersionedProtocol, String, * long, int) for a default implementation + * @throws IOException raised on errors performing I/O. */ public ProtocolSignature getProtocolSignature(String protocol, long clientVersion, diff --git a/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/ipc/WritableRpcEngine.java b/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/ipc/WritableRpcEngine.java index d790e49f5dcf2..21181f860d98a 100644 --- a/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/ipc/WritableRpcEngine.java +++ b/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/ipc/WritableRpcEngine.java @@ -282,9 +282,20 @@ static Client getClient(Configuration conf) { return CLIENTS.getClient(conf); } - /** Construct a client-side proxy object that implements the named protocol, + /** + * Construct a client-side proxy object that implements the named protocol, * talking to a server at the named address. - * @param */ + * @param Generics Type T + * @param protocol input protocol. + * @param clientVersion input clientVersion. + * @param addr input addr. + * @param ticket input ticket. + * @param conf input configuration. + * @param factory input factory. + * @param rpcTimeout input rpcTimeout. + * @param connectionRetryPolicy input connectionRetryPolicy. + * @throws IOException raised on errors performing I/O. + */ @Override public ProtocolProxy getProxy(Class protocol, long clientVersion, InetSocketAddress addr, UserGroupInformation ticket, @@ -295,9 +306,22 @@ public ProtocolProxy getProxy(Class protocol, long clientVersion, rpcTimeout, connectionRetryPolicy, null, null); } - /** Construct a client-side proxy object that implements the named protocol, + /** + * Construct a client-side proxy object that implements the named protocol, * talking to a server at the named address. - * @param */ + * @param Generics Type. + * @param protocol input protocol. + * @param clientVersion input clientVersion. + * @param addr input addr. + * @param ticket input ticket. + * @param conf input configuration. + * @param factory input factory. + * @param rpcTimeout input rpcTimeout. + * @param connectionRetryPolicy input connectionRetryPolicy. + * @param fallbackToSimpleAuth input fallbackToSimpleAuth. + * @param alignmentContext input alignmentContext. + * @return ProtocolProxy. + */ @Override @SuppressWarnings("unchecked") public ProtocolProxy getProxy(Class protocol, long clientVersion, @@ -345,7 +369,8 @@ public static class Server extends RPC.Server { * @param bindAddress the address to bind on to listen for connection * @param port the port to listen for connections on * - * @deprecated Use #Server(Class, Object, Configuration, String, int) + * @deprecated Use #Server(Class, Object, Configuration, String, int) + * @throws IOException raised on errors performing I/O. */ @Deprecated public Server(Object instance, Configuration conf, String bindAddress, @@ -360,6 +385,7 @@ public Server(Object instance, Configuration conf, String bindAddress, * @param conf the configuration to use * @param bindAddress the address to bind on to listen for connection * @param port the port to listen for connections on + * @throws IOException raised on errors performing I/O. */ public Server(Class protocolClass, Object protocolImpl, Configuration conf, String bindAddress, int port) @@ -376,9 +402,13 @@ public Server(Class protocolClass, Object protocolImpl, * @param port the port to listen for connections on * @param numHandlers the number of method handler threads to run * @param verbose whether each call should be logged + * @param numReaders input numberReaders. + * @param queueSizePerHandler input queueSizePerHandler. + * @param secretManager input secretManager. * * @deprecated use Server#Server(Class, Object, * Configuration, String, int, int, int, int, boolean, SecretManager) + * @throws IOException raised on errors performing I/O. */ @Deprecated public Server(Object protocolImpl, Configuration conf, String bindAddress, @@ -401,9 +431,14 @@ public Server(Object protocolImpl, Configuration conf, String bindAddress, * @param port the port to listen for connections on * @param numHandlers the number of method handler threads to run * @param verbose whether each call should be logged + * @param secretManager input secretManager. + * @param queueSizePerHandler input queueSizePerHandler. + * @param portRangeConfig input portRangeConfig. + * @param numReaders input numReaders. * * @deprecated use Server#Server(Class, Object, * Configuration, String, int, int, int, int, boolean, SecretManager) + * @throws IOException raised on errors performing I/O. */ @Deprecated public Server(Class protocolClass, Object protocolImpl, @@ -428,6 +463,11 @@ public Server(Class protocolClass, Object protocolImpl, * @param numHandlers the number of method handler threads to run * @param verbose whether each call should be logged * @param alignmentContext provides server state info on client responses + * @param numReaders input numReaders. + * @param portRangeConfig input portRangeConfig. + * @param queueSizePerHandler input queueSizePerHandler. + * @param secretManager input secretManager. + * @throws IOException raised on errors performing I/O. */ public Server(Class protocolClass, Object protocolImpl, Configuration conf, String bindAddress, int port, diff --git a/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/ipc/metrics/DecayRpcSchedulerDetailedMetrics.java b/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/ipc/metrics/DecayRpcSchedulerDetailedMetrics.java index b86381706d67b..0bfe5c7d88000 100644 --- a/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/ipc/metrics/DecayRpcSchedulerDetailedMetrics.java +++ b/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/ipc/metrics/DecayRpcSchedulerDetailedMetrics.java @@ -65,6 +65,7 @@ public static DecayRpcSchedulerDetailedMetrics create(String ns) { /** * Initialize the metrics for JMX with priority levels. + * @param numLevels input numLevels. */ public void init(int numLevels) { LOG.info("Initializing RPC stats for {} priority levels", numLevels); @@ -106,14 +107,16 @@ public void shutdown() { } /** - * Returns the rate name inside the metric. + * @return Returns the rate name inside the metric. + * @param priority input priority. */ public String getQueueName(int priority) { return "DecayRPCSchedulerPriority."+priority+".RpcQueueTime"; } /** - * Returns the rate name inside the metric. + * @return Returns the rate name inside the metric. + * @param priority input priority. */ public String getProcessingName(int priority) { return "DecayRPCSchedulerPriority."+priority+".RpcProcessingTime"; diff --git a/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/ipc/metrics/RpcMetrics.java b/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/ipc/metrics/RpcMetrics.java index a67530b3c97b2..bf21e3865fa8a 100644 --- a/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/ipc/metrics/RpcMetrics.java +++ b/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/ipc/metrics/RpcMetrics.java @@ -151,6 +151,16 @@ public String numOpenConnectionsPerUser() { return server.getNumDroppedConnections(); } + @Metric("Number of total requests") + public long getTotalRequests() { + return server.getTotalRequests(); + } + + @Metric("Number of total requests per second") + public long getTotalRequestsPerSecond() { + return server.getTotalRequestsPerSecond(); + } + public TimeUnit getMetricsTimeUnit() { return metricsTimeUnit; } diff --git a/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/jmx/JMXJsonServlet.java b/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/jmx/JMXJsonServlet.java index f20933b5c8668..85f2d2828562d 100644 --- a/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/jmx/JMXJsonServlet.java +++ b/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/jmx/JMXJsonServlet.java @@ -64,27 +64,31 @@ * functionality is provided through the * {@link MBeanServer#queryNames(ObjectName, javax.management.QueryExp)} * method. + *

    *

    * For example http://.../jmx?qry=Hadoop:* will return * all hadoop metrics exposed through JMX. + *

    *

    * The optional get parameter is used to query an specific * attribute of a JMX bean. The format of the URL is * http://.../jmx?get=MXBeanName::AttributeName + *

    *

    * For example * * http://../jmx?get=Hadoop:service=NameNode,name=NameNodeInfo::ClusterId * will return the cluster id of the namenode mxbean. + *

    *

    * If the qry or the get parameter is not formatted - * correctly then a 400 BAD REQUEST http response code will be returned. + * correctly then a 400 BAD REQUEST http response code will be returned. + *

    *

    * If a resouce such as a mbean or attribute can not be found, * a 404 SC_NOT_FOUND http response code will be returned. - *

    + *

    * The return format is JSON and in the form - *

    *

    
      *  {
      *    "beans" : [
    @@ -95,7 +99,6 @@
      *    ]
      *  }
      *  
    - *

    * The servlet attempts to convert the the JMXBeans into JSON. Each * bean's attributes will be converted to a JSON object member. * diff --git a/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/log/LogLevel.java b/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/log/LogLevel.java index c8a88236aeb39..e2ad16fce2c57 100644 --- a/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/log/LogLevel.java +++ b/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/log/LogLevel.java @@ -66,6 +66,8 @@ public class LogLevel { public static final String PROTOCOL_HTTPS = "https"; /** * A command line implementation + * @param args input args. + * @throws Exception exception. */ public static void main(String[] args) throws Exception { CLI cli = new CLI(new Configuration()); diff --git a/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/log/LogThrottlingHelper.java b/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/log/LogThrottlingHelper.java index 622ee5405c892..af5f852143389 100644 --- a/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/log/LogThrottlingHelper.java +++ b/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/log/LogThrottlingHelper.java @@ -88,21 +88,22 @@ public class LogThrottlingHelper { public interface LogAction { /** - * Return the number of records encapsulated in this action; that is, the + * @return Return the number of records encapsulated in this action; that is, the * number of times {@code record} was called to produce this action, * including the current one. */ int getCount(); /** - * Return summary information for the value that was recorded at index + * @return Return summary information for the value that was recorded at index * {@code idx}. Corresponds to the ordering of values passed to * {@link #record(double...)}. + * @param idx input idx. */ SummaryStatistics getStats(int idx); /** - * If this is true, the caller should write to its log. Otherwise, the + * @return If this is true, the caller should write to its log. Otherwise, the * caller should take no action, and it is an error to call other methods * on this object. */ @@ -139,6 +140,7 @@ public interface LogAction { * Create a log helper without any primary recorder. * * @see #LogThrottlingHelper(long, String) + * @param minLogPeriodMs input minLogPeriodMs. */ public LogThrottlingHelper(long minLogPeriodMs) { this(minLogPeriodMs, null); diff --git a/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/metrics2/MetricsSystem.java b/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/metrics2/MetricsSystem.java index a277abd6e1384..fef8c4b7e4ba9 100644 --- a/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/metrics2/MetricsSystem.java +++ b/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/metrics2/MetricsSystem.java @@ -50,7 +50,7 @@ public abstract class MetricsSystem implements MetricsSystemMXBean { * the annotations of the source object.) * @param desc the description of the source (or null. See above.) * @return the source object - * @exception MetricsException + * @exception MetricsException Metrics Exception. */ public abstract T register(String name, String desc, T source); @@ -65,7 +65,7 @@ public abstract class MetricsSystem implements MetricsSystemMXBean { * @param the actual type of the source object * @param source object to register * @return the source object - * @exception MetricsException + * @exception MetricsException Metrics Exception. */ public T register(T source) { return register(null, null, source); @@ -85,7 +85,7 @@ public T register(T source) { * @param name of the sink. Must be unique. * @param desc the description of the sink * @return the sink - * @exception MetricsException + * @exception MetricsException Metrics Exception. */ public abstract T register(String name, String desc, T sink); diff --git a/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/metrics2/MetricsSystemMXBean.java b/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/metrics2/MetricsSystemMXBean.java index e471ab7498ce4..8656da6f316c0 100644 --- a/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/metrics2/MetricsSystemMXBean.java +++ b/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/metrics2/MetricsSystemMXBean.java @@ -29,19 +29,19 @@ public interface MetricsSystemMXBean { /** * Start the metrics system - * @throws MetricsException + * @throws MetricsException Metrics Exception. */ public void start(); /** * Stop the metrics system - * @throws MetricsException + * @throws MetricsException Metrics Exception. */ public void stop(); /** * Start metrics MBeans - * @throws MetricsException + * @throws MetricsException Metrics Exception. */ public void startMetricsMBeans(); @@ -49,7 +49,7 @@ public interface MetricsSystemMXBean { * Stop metrics MBeans. * Note, it doesn't stop the metrics system control MBean, * i.e this interface. - * @throws MetricsException + * @throws MetricsException Metrics Exception. */ public void stopMetricsMBeans(); @@ -57,7 +57,7 @@ public interface MetricsSystemMXBean { * @return the current config * Avoided getConfig, as it'll turn into a "Config" attribute, * which doesn't support multiple line values in jconsole. - * @throws MetricsException + * @throws MetricsException Metrics Exception. */ public String currentConfig(); } diff --git a/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/metrics2/lib/MutableMetricsFactory.java b/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/metrics2/lib/MutableMetricsFactory.java index c7adaa5d9917f..e4886cb603e4e 100644 --- a/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/metrics2/lib/MutableMetricsFactory.java +++ b/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/metrics2/lib/MutableMetricsFactory.java @@ -146,8 +146,10 @@ protected MetricsInfo getInfo(Class cls, Metrics annotation) { } /** - * Remove the prefix "get", if any, from the method name. Return the + * @return Remove the prefix "get", if any, from the method name. Return the * capacitalized method name." + * + * @param method input method. */ protected String getName(Method method) { String methodName = method.getName(); diff --git a/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/metrics2/lib/MutableRates.java b/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/metrics2/lib/MutableRates.java index 19696bd839400..90b5da01c062f 100644 --- a/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/metrics2/lib/MutableRates.java +++ b/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/metrics2/lib/MutableRates.java @@ -19,11 +19,10 @@ package org.apache.hadoop.metrics2.lib; import java.lang.reflect.Method; +import java.util.HashSet; import java.util.Set; import static org.apache.hadoop.util.Preconditions.*; -import org.apache.hadoop.util.Sets; - import org.apache.hadoop.classification.InterfaceAudience; import org.apache.hadoop.classification.InterfaceStability; import org.apache.hadoop.metrics2.MetricsRecordBuilder; @@ -44,7 +43,7 @@ public class MutableRates extends MutableMetric { static final Logger LOG = LoggerFactory.getLogger(MutableRates.class); private final MetricsRegistry registry; - private final Set> protocolCache = Sets.newHashSet(); + private final Set> protocolCache = new HashSet<>(); MutableRates(MetricsRegistry registry) { this.registry = checkNotNull(registry, "metrics registry"); diff --git a/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/metrics2/lib/MutableRatesWithAggregation.java b/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/metrics2/lib/MutableRatesWithAggregation.java index dc37f96f4f449..4c5f0a844aaab 100644 --- a/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/metrics2/lib/MutableRatesWithAggregation.java +++ b/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/metrics2/lib/MutableRatesWithAggregation.java @@ -18,9 +18,9 @@ package org.apache.hadoop.metrics2.lib; -import org.apache.hadoop.util.Sets; import java.lang.ref.WeakReference; import java.lang.reflect.Method; +import java.util.HashSet; import java.util.Iterator; import java.util.Map; import java.util.Set; @@ -52,7 +52,7 @@ public class MutableRatesWithAggregation extends MutableMetric { LoggerFactory.getLogger(MutableRatesWithAggregation.class); private final Map globalMetrics = new ConcurrentHashMap<>(); - private final Set> protocolCache = Sets.newHashSet(); + private final Set> protocolCache = new HashSet<>(); private final ConcurrentLinkedDeque>> weakReferenceQueue = new ConcurrentLinkedDeque<>(); diff --git a/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/metrics2/lib/MutableRollingAverages.java b/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/metrics2/lib/MutableRollingAverages.java index aa4d4b9ca0c64..016ecdd4d1e99 100644 --- a/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/metrics2/lib/MutableRollingAverages.java +++ b/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/metrics2/lib/MutableRollingAverages.java @@ -139,7 +139,7 @@ public long getSnapshotTimeStamp() { /** * Constructor for {@link MutableRollingAverages}. - * @param metricValueName + * @param metricValueName input metricValueName. */ public MutableRollingAverages(String metricValueName) { if (metricValueName == null) { @@ -285,6 +285,7 @@ public void close() throws IOException { * Retrieve a map of metric name {@literal ->} (aggregate). * Filter out entries that don't have at least minSamples. * + * @param minSamples input minSamples. * @return a map of peer DataNode Id to the average latency to that * node seen over the measurement period. */ @@ -314,6 +315,7 @@ public synchronized Map getStats(long minSamples) { /** * Use for test only. + * @param value input value. */ @VisibleForTesting public synchronized void setRecordValidityMs(long value) { diff --git a/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/metrics2/lib/MutableStat.java b/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/metrics2/lib/MutableStat.java index e04b4b58ece0b..f2e072545ad28 100644 --- a/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/metrics2/lib/MutableStat.java +++ b/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/metrics2/lib/MutableStat.java @@ -179,7 +179,7 @@ public void resetMinMax() { } /** - * Return the SampleStat snapshot timestamp + * @return Return the SampleStat snapshot timestamp. */ public long getSnapshotTimeStamp() { return snapshotTimeStamp; diff --git a/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/metrics2/package-info.java b/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/metrics2/package-info.java index 8fd3b33b3a253..196469be9dce2 100644 --- a/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/metrics2/package-info.java +++ b/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/metrics2/package-info.java @@ -87,7 +87,7 @@ usually does not need to reference any class here.

    Getting started

    Implementing metrics sources

    Memory usage by blocksize
    Memory usage by blocksize
    Blocksize Compression
    * memory usage
    Decompression
    * memory usage
    - + @@ -290,10 +290,10 @@ metrics system decouples the concept for context (for grouping) with the backend that can handle multiple contexts (file, gangalia etc.):

    Implementing metrics sources
    Using annotationsUsing MetricsSource interface
    - + - +
    Migration from previous system
    BeforeAfterBeforeAfter
    @@ -312,10 +312,10 @@ backend that can handle multiple contexts (file, gangalia etc.):
         using the context option in the sink options like the following:
       

    - + - + ",e.document[0]).appendTo(n)):"tr"===s?e._createTrPlaceholder(e.currentItem,n):"img"===s&&n.attr("src",e.currentItem.attr("src")),i||n.css("visibility","hidden"),n},update:function(t,n){(!i||s.forcePlaceholderSize)&&(n.height()||n.height(e.currentItem.innerHeight()-parseInt(e.currentItem.css("paddingTop")||0,10)-parseInt(e.currentItem.css("paddingBottom")||0,10)),n.width()||n.width(e.currentItem.innerWidth()-parseInt(e.currentItem.css("paddingLeft")||0,10)-parseInt(e.currentItem.css("paddingRight")||0,10)))}}),e.placeholder=t(s.placeholder.element.call(e.element,e.currentItem)),e.currentItem.after(e.placeholder),s.placeholder.update(e,e.placeholder)},_createTrPlaceholder:function(e,i){var s=this;e.children().each(function(){t("",s.document[0]).attr("colspan",t(this).attr("colspan")||1).appendTo(i)})},_contactContainers:function(e){var i,s,n,o,a,r,h,l,c,u,d=null,p=null;for(i=this.containers.length-1;i>=0;i--)if(!t.contains(this.currentItem[0],this.containers[i].element[0]))if(this._intersectsWith(this.containers[i].containerCache)){if(d&&t.contains(this.containers[i].element[0],d.element[0]))continue;d=this.containers[i],p=i}else this.containers[i].containerCache.over&&(this.containers[i]._trigger("out",e,this._uiHash(this)),this.containers[i].containerCache.over=0);if(d)if(1===this.containers.length)this.containers[p].containerCache.over||(this.containers[p]._trigger("over",e,this._uiHash(this)),this.containers[p].containerCache.over=1);else{for(n=1e4,o=null,c=d.floating||this._isFloating(this.currentItem),a=c?"left":"top",r=c?"width":"height",u=c?"pageX":"pageY",s=this.items.length-1;s>=0;s--)t.contains(this.containers[p].element[0],this.items[s].item[0])&&this.items[s].item[0]!==this.currentItem[0]&&(h=this.items[s].item.offset()[a],l=!1,e[u]-h>this.items[s][r]/2&&(l=!0),n>Math.abs(e[u]-h)&&(n=Math.abs(e[u]-h),o=this.items[s],this.direction=l?"up":"down"));if(!o&&!this.options.dropOnEmpty)return;if(this.currentContainer===this.containers[p])return this.currentContainer.containerCache.over||(this.containers[p]._trigger("over",e,this._uiHash()),this.currentContainer.containerCache.over=1),void 0;o?this._rearrange(e,o,null,!0):this._rearrange(e,null,this.containers[p].element,!0),this._trigger("change",e,this._uiHash()),this.containers[p]._trigger("change",e,this._uiHash(this)),this.currentContainer=this.containers[p],this.options.placeholder.update(this.currentContainer,this.placeholder),this.containers[p]._trigger("over",e,this._uiHash(this)),this.containers[p].containerCache.over=1}},_createHelper:function(e){var i=this.options,s=t.isFunction(i.helper)?t(i.helper.apply(this.element[0],[e,this.currentItem])):"clone"===i.helper?this.currentItem.clone():this.currentItem;return s.parents("body").length||t("parent"!==i.appendTo?i.appendTo:this.currentItem[0].parentNode)[0].appendChild(s[0]),s[0]===this.currentItem[0]&&(this._storedCSS={width:this.currentItem[0].style.width,height:this.currentItem[0].style.height,position:this.currentItem.css("position"),top:this.currentItem.css("top"),left:this.currentItem.css("left")}),(!s[0].style.width||i.forceHelperSize)&&s.width(this.currentItem.width()),(!s[0].style.height||i.forceHelperSize)&&s.height(this.currentItem.height()),s},_adjustOffsetFromHelper:function(e){"string"==typeof e&&(e=e.split(" ")),t.isArray(e)&&(e={left:+e[0],top:+e[1]||0}),"left"in e&&(this.offset.click.left=e.left+this.margins.left),"right"in e&&(this.offset.click.left=this.helperProportions.width-e.right+this.margins.left),"top"in e&&(this.offset.click.top=e.top+this.margins.top),"bottom"in e&&(this.offset.click.top=this.helperProportions.height-e.bottom+this.margins.top)},_getParentOffset:function(){this.offsetParent=this.helper.offsetParent();var e=this.offsetParent.offset();return"absolute"===this.cssPosition&&this.scrollParent[0]!==this.document[0]&&t.contains(this.scrollParent[0],this.offsetParent[0])&&(e.left+=this.scrollParent.scrollLeft(),e.top+=this.scrollParent.scrollTop()),(this.offsetParent[0]===this.document[0].body||this.offsetParent[0].tagName&&"html"===this.offsetParent[0].tagName.toLowerCase()&&t.ui.ie)&&(e={top:0,left:0}),{top:e.top+(parseInt(this.offsetParent.css("borderTopWidth"),10)||0),left:e.left+(parseInt(this.offsetParent.css("borderLeftWidth"),10)||0)}},_getRelativeOffset:function(){if("relative"===this.cssPosition){var t=this.currentItem.position();return{top:t.top-(parseInt(this.helper.css("top"),10)||0)+this.scrollParent.scrollTop(),left:t.left-(parseInt(this.helper.css("left"),10)||0)+this.scrollParent.scrollLeft()}}return{top:0,left:0}},_cacheMargins:function(){this.margins={left:parseInt(this.currentItem.css("marginLeft"),10)||0,top:parseInt(this.currentItem.css("marginTop"),10)||0}},_cacheHelperProportions:function(){this.helperProportions={width:this.helper.outerWidth(),height:this.helper.outerHeight()}},_setContainment:function(){var e,i,s,n=this.options;"parent"===n.containment&&(n.containment=this.helper[0].parentNode),("document"===n.containment||"window"===n.containment)&&(this.containment=[0-this.offset.relative.left-this.offset.parent.left,0-this.offset.relative.top-this.offset.parent.top,"document"===n.containment?this.document.width():this.window.width()-this.helperProportions.width-this.margins.left,("document"===n.containment?this.document.height()||document.body.parentNode.scrollHeight:this.window.height()||this.document[0].body.parentNode.scrollHeight)-this.helperProportions.height-this.margins.top]),/^(document|window|parent)$/.test(n.containment)||(e=t(n.containment)[0],i=t(n.containment).offset(),s="hidden"!==t(e).css("overflow"),this.containment=[i.left+(parseInt(t(e).css("borderLeftWidth"),10)||0)+(parseInt(t(e).css("paddingLeft"),10)||0)-this.margins.left,i.top+(parseInt(t(e).css("borderTopWidth"),10)||0)+(parseInt(t(e).css("paddingTop"),10)||0)-this.margins.top,i.left+(s?Math.max(e.scrollWidth,e.offsetWidth):e.offsetWidth)-(parseInt(t(e).css("borderLeftWidth"),10)||0)-(parseInt(t(e).css("paddingRight"),10)||0)-this.helperProportions.width-this.margins.left,i.top+(s?Math.max(e.scrollHeight,e.offsetHeight):e.offsetHeight)-(parseInt(t(e).css("borderTopWidth"),10)||0)-(parseInt(t(e).css("paddingBottom"),10)||0)-this.helperProportions.height-this.margins.top])},_convertPositionTo:function(e,i){i||(i=this.position);var s="absolute"===e?1:-1,n="absolute"!==this.cssPosition||this.scrollParent[0]!==this.document[0]&&t.contains(this.scrollParent[0],this.offsetParent[0])?this.scrollParent:this.offsetParent,o=/(html|body)/i.test(n[0].tagName);return{top:i.top+this.offset.relative.top*s+this.offset.parent.top*s-("fixed"===this.cssPosition?-this.scrollParent.scrollTop():o?0:n.scrollTop())*s,left:i.left+this.offset.relative.left*s+this.offset.parent.left*s-("fixed"===this.cssPosition?-this.scrollParent.scrollLeft():o?0:n.scrollLeft())*s}},_generatePosition:function(e){var i,s,n=this.options,o=e.pageX,a=e.pageY,r="absolute"!==this.cssPosition||this.scrollParent[0]!==this.document[0]&&t.contains(this.scrollParent[0],this.offsetParent[0])?this.scrollParent:this.offsetParent,h=/(html|body)/i.test(r[0].tagName);return"relative"!==this.cssPosition||this.scrollParent[0]!==this.document[0]&&this.scrollParent[0]!==this.offsetParent[0]||(this.offset.relative=this._getRelativeOffset()),this.originalPosition&&(this.containment&&(e.pageX-this.offset.click.leftthis.containment[2]&&(o=this.containment[2]+this.offset.click.left),e.pageY-this.offset.click.top>this.containment[3]&&(a=this.containment[3]+this.offset.click.top)),n.grid&&(i=this.originalPageY+Math.round((a-this.originalPageY)/n.grid[1])*n.grid[1],a=this.containment?i-this.offset.click.top>=this.containment[1]&&i-this.offset.click.top<=this.containment[3]?i:i-this.offset.click.top>=this.containment[1]?i-n.grid[1]:i+n.grid[1]:i,s=this.originalPageX+Math.round((o-this.originalPageX)/n.grid[0])*n.grid[0],o=this.containment?s-this.offset.click.left>=this.containment[0]&&s-this.offset.click.left<=this.containment[2]?s:s-this.offset.click.left>=this.containment[0]?s-n.grid[0]:s+n.grid[0]:s)),{top:a-this.offset.click.top-this.offset.relative.top-this.offset.parent.top+("fixed"===this.cssPosition?-this.scrollParent.scrollTop():h?0:r.scrollTop()),left:o-this.offset.click.left-this.offset.relative.left-this.offset.parent.left+("fixed"===this.cssPosition?-this.scrollParent.scrollLeft():h?0:r.scrollLeft())}},_rearrange:function(t,e,i,s){i?i[0].appendChild(this.placeholder[0]):e.item[0].parentNode.insertBefore(this.placeholder[0],"down"===this.direction?e.item[0]:e.item[0].nextSibling),this.counter=this.counter?++this.counter:1;var n=this.counter; -this._delay(function(){n===this.counter&&this.refreshPositions(!s)})},_clear:function(t,e){function i(t,e,i){return function(s){i._trigger(t,s,e._uiHash(e))}}this.reverting=!1;var s,n=[];if(!this._noFinalSort&&this.currentItem.parent().length&&this.placeholder.before(this.currentItem),this._noFinalSort=null,this.helper[0]===this.currentItem[0]){for(s in this._storedCSS)("auto"===this._storedCSS[s]||"static"===this._storedCSS[s])&&(this._storedCSS[s]="");this.currentItem.css(this._storedCSS),this._removeClass(this.currentItem,"ui-sortable-helper")}else this.currentItem.show();for(this.fromOutside&&!e&&n.push(function(t){this._trigger("receive",t,this._uiHash(this.fromOutside))}),!this.fromOutside&&this.domPosition.prev===this.currentItem.prev().not(".ui-sortable-helper")[0]&&this.domPosition.parent===this.currentItem.parent()[0]||e||n.push(function(t){this._trigger("update",t,this._uiHash())}),this!==this.currentContainer&&(e||(n.push(function(t){this._trigger("remove",t,this._uiHash())}),n.push(function(t){return function(e){t._trigger("receive",e,this._uiHash(this))}}.call(this,this.currentContainer)),n.push(function(t){return function(e){t._trigger("update",e,this._uiHash(this))}}.call(this,this.currentContainer)))),s=this.containers.length-1;s>=0;s--)e||n.push(i("deactivate",this,this.containers[s])),this.containers[s].containerCache.over&&(n.push(i("out",this,this.containers[s])),this.containers[s].containerCache.over=0);if(this.storedCursor&&(this.document.find("body").css("cursor",this.storedCursor),this.storedStylesheet.remove()),this._storedOpacity&&this.helper.css("opacity",this._storedOpacity),this._storedZIndex&&this.helper.css("zIndex","auto"===this._storedZIndex?"":this._storedZIndex),this.dragging=!1,e||this._trigger("beforeStop",t,this._uiHash()),this.placeholder[0].parentNode.removeChild(this.placeholder[0]),this.cancelHelperRemoval||(this.helper[0]!==this.currentItem[0]&&this.helper.remove(),this.helper=null),!e){for(s=0;n.length>s;s++)n[s].call(this,t);this._trigger("stop",t,this._uiHash())}return this.fromOutside=!1,!this.cancelHelperRemoval},_trigger:function(){t.Widget.prototype._trigger.apply(this,arguments)===!1&&this.cancel()},_uiHash:function(e){var i=e||this;return{helper:i.helper,placeholder:i.placeholder||t([]),position:i.position,originalPosition:i.originalPosition,offset:i.positionAbs,item:i.currentItem,sender:e?e.element:null}}}),t.widget("ui.spinner",{version:"1.12.1",defaultElement:"",widgetEventPrefix:"spin",options:{classes:{"ui-spinner":"ui-corner-all","ui-spinner-down":"ui-corner-br","ui-spinner-up":"ui-corner-tr"},culture:null,icons:{down:"ui-icon-triangle-1-s",up:"ui-icon-triangle-1-n"},incremental:!0,max:null,min:null,numberFormat:null,page:10,step:1,change:null,spin:null,start:null,stop:null},_create:function(){this._setOption("max",this.options.max),this._setOption("min",this.options.min),this._setOption("step",this.options.step),""!==this.value()&&this._value(this.element.val(),!0),this._draw(),this._on(this._events),this._refresh(),this._on(this.window,{beforeunload:function(){this.element.removeAttr("autocomplete")}})},_getCreateOptions:function(){var e=this._super(),i=this.element;return t.each(["min","max","step"],function(t,s){var n=i.attr(s);null!=n&&n.length&&(e[s]=n)}),e},_events:{keydown:function(t){this._start(t)&&this._keydown(t)&&t.preventDefault()},keyup:"_stop",focus:function(){this.previous=this.element.val()},blur:function(t){return this.cancelBlur?(delete this.cancelBlur,void 0):(this._stop(),this._refresh(),this.previous!==this.element.val()&&this._trigger("change",t),void 0)},mousewheel:function(t,e){if(e){if(!this.spinning&&!this._start(t))return!1;this._spin((e>0?1:-1)*this.options.step,t),clearTimeout(this.mousewheelTimer),this.mousewheelTimer=this._delay(function(){this.spinning&&this._stop(t)},100),t.preventDefault()}},"mousedown .ui-spinner-button":function(e){function i(){var e=this.element[0]===t.ui.safeActiveElement(this.document[0]);e||(this.element.trigger("focus"),this.previous=s,this._delay(function(){this.previous=s}))}var s;s=this.element[0]===t.ui.safeActiveElement(this.document[0])?this.previous:this.element.val(),e.preventDefault(),i.call(this),this.cancelBlur=!0,this._delay(function(){delete this.cancelBlur,i.call(this)}),this._start(e)!==!1&&this._repeat(null,t(e.currentTarget).hasClass("ui-spinner-up")?1:-1,e)},"mouseup .ui-spinner-button":"_stop","mouseenter .ui-spinner-button":function(e){return t(e.currentTarget).hasClass("ui-state-active")?this._start(e)===!1?!1:(this._repeat(null,t(e.currentTarget).hasClass("ui-spinner-up")?1:-1,e),void 0):void 0},"mouseleave .ui-spinner-button":"_stop"},_enhance:function(){this.uiSpinner=this.element.attr("autocomplete","off").wrap("").parent().append("")},_draw:function(){this._enhance(),this._addClass(this.uiSpinner,"ui-spinner","ui-widget ui-widget-content"),this._addClass("ui-spinner-input"),this.element.attr("role","spinbutton"),this.buttons=this.uiSpinner.children("a").attr("tabIndex",-1).attr("aria-hidden",!0).button({classes:{"ui-button":""}}),this._removeClass(this.buttons,"ui-corner-all"),this._addClass(this.buttons.first(),"ui-spinner-button ui-spinner-up"),this._addClass(this.buttons.last(),"ui-spinner-button ui-spinner-down"),this.buttons.first().button({icon:this.options.icons.up,showLabel:!1}),this.buttons.last().button({icon:this.options.icons.down,showLabel:!1}),this.buttons.height()>Math.ceil(.5*this.uiSpinner.height())&&this.uiSpinner.height()>0&&this.uiSpinner.height(this.uiSpinner.height())},_keydown:function(e){var i=this.options,s=t.ui.keyCode;switch(e.keyCode){case s.UP:return this._repeat(null,1,e),!0;case s.DOWN:return this._repeat(null,-1,e),!0;case s.PAGE_UP:return this._repeat(null,i.page,e),!0;case s.PAGE_DOWN:return this._repeat(null,-i.page,e),!0}return!1},_start:function(t){return this.spinning||this._trigger("start",t)!==!1?(this.counter||(this.counter=1),this.spinning=!0,!0):!1},_repeat:function(t,e,i){t=t||500,clearTimeout(this.timer),this.timer=this._delay(function(){this._repeat(40,e,i)},t),this._spin(e*this.options.step,i)},_spin:function(t,e){var i=this.value()||0;this.counter||(this.counter=1),i=this._adjustValue(i+t*this._increment(this.counter)),this.spinning&&this._trigger("spin",e,{value:i})===!1||(this._value(i),this.counter++)},_increment:function(e){var i=this.options.incremental;return i?t.isFunction(i)?i(e):Math.floor(e*e*e/5e4-e*e/500+17*e/200+1):1},_precision:function(){var t=this._precisionOf(this.options.step);return null!==this.options.min&&(t=Math.max(t,this._precisionOf(this.options.min))),t},_precisionOf:function(t){var e=""+t,i=e.indexOf(".");return-1===i?0:e.length-i-1},_adjustValue:function(t){var e,i,s=this.options;return e=null!==s.min?s.min:0,i=t-e,i=Math.round(i/s.step)*s.step,t=e+i,t=parseFloat(t.toFixed(this._precision())),null!==s.max&&t>s.max?s.max:null!==s.min&&s.min>t?s.min:t},_stop:function(t){this.spinning&&(clearTimeout(this.timer),clearTimeout(this.mousewheelTimer),this.counter=0,this.spinning=!1,this._trigger("stop",t))},_setOption:function(t,e){var i,s,n;return"culture"===t||"numberFormat"===t?(i=this._parse(this.element.val()),this.options[t]=e,this.element.val(this._format(i)),void 0):(("max"===t||"min"===t||"step"===t)&&"string"==typeof e&&(e=this._parse(e)),"icons"===t&&(s=this.buttons.first().find(".ui-icon"),this._removeClass(s,null,this.options.icons.up),this._addClass(s,null,e.up),n=this.buttons.last().find(".ui-icon"),this._removeClass(n,null,this.options.icons.down),this._addClass(n,null,e.down)),this._super(t,e),void 0)},_setOptionDisabled:function(t){this._super(t),this._toggleClass(this.uiSpinner,null,"ui-state-disabled",!!t),this.element.prop("disabled",!!t),this.buttons.button(t?"disable":"enable")},_setOptions:r(function(t){this._super(t)}),_parse:function(t){return"string"==typeof t&&""!==t&&(t=window.Globalize&&this.options.numberFormat?Globalize.parseFloat(t,10,this.options.culture):+t),""===t||isNaN(t)?null:t},_format:function(t){return""===t?"":window.Globalize&&this.options.numberFormat?Globalize.format(t,this.options.numberFormat,this.options.culture):t},_refresh:function(){this.element.attr({"aria-valuemin":this.options.min,"aria-valuemax":this.options.max,"aria-valuenow":this._parse(this.element.val())})},isValid:function(){var t=this.value();return null===t?!1:t===this._adjustValue(t)},_value:function(t,e){var i;""!==t&&(i=this._parse(t),null!==i&&(e||(i=this._adjustValue(i)),t=this._format(i))),this.element.val(t),this._refresh()},_destroy:function(){this.element.prop("disabled",!1).removeAttr("autocomplete role aria-valuemin aria-valuemax aria-valuenow"),this.uiSpinner.replaceWith(this.element)},stepUp:r(function(t){this._stepUp(t)}),_stepUp:function(t){this._start()&&(this._spin((t||1)*this.options.step),this._stop())},stepDown:r(function(t){this._stepDown(t)}),_stepDown:function(t){this._start()&&(this._spin((t||1)*-this.options.step),this._stop())},pageUp:r(function(t){this._stepUp((t||1)*this.options.page)}),pageDown:r(function(t){this._stepDown((t||1)*this.options.page)}),value:function(t){return arguments.length?(r(this._value).call(this,t),void 0):this._parse(this.element.val())},widget:function(){return this.uiSpinner}}),t.uiBackCompat!==!1&&t.widget("ui.spinner",t.ui.spinner,{_enhance:function(){this.uiSpinner=this.element.attr("autocomplete","off").wrap(this._uiSpinnerHtml()).parent().append(this._buttonHtml())},_uiSpinnerHtml:function(){return""},_buttonHtml:function(){return""}}),t.ui.spinner,t.widget("ui.tabs",{version:"1.12.1",delay:300,options:{active:null,classes:{"ui-tabs":"ui-corner-all","ui-tabs-nav":"ui-corner-all","ui-tabs-panel":"ui-corner-bottom","ui-tabs-tab":"ui-corner-top"},collapsible:!1,event:"click",heightStyle:"content",hide:null,show:null,activate:null,beforeActivate:null,beforeLoad:null,load:null},_isLocal:function(){var t=/#.*$/;return function(e){var i,s;i=e.href.replace(t,""),s=location.href.replace(t,"");try{i=decodeURIComponent(i)}catch(n){}try{s=decodeURIComponent(s)}catch(n){}return e.hash.length>1&&i===s}}(),_create:function(){var e=this,i=this.options;this.running=!1,this._addClass("ui-tabs","ui-widget ui-widget-content"),this._toggleClass("ui-tabs-collapsible",null,i.collapsible),this._processTabs(),i.active=this._initialActive(),t.isArray(i.disabled)&&(i.disabled=t.unique(i.disabled.concat(t.map(this.tabs.filter(".ui-state-disabled"),function(t){return e.tabs.index(t)}))).sort()),this.active=this.options.active!==!1&&this.anchors.length?this._findActive(i.active):t(),this._refresh(),this.active.length&&this.load(i.active)},_initialActive:function(){var e=this.options.active,i=this.options.collapsible,s=location.hash.substring(1);return null===e&&(s&&this.tabs.each(function(i,n){return t(n).attr("aria-controls")===s?(e=i,!1):void 0}),null===e&&(e=this.tabs.index(this.tabs.filter(".ui-tabs-active"))),(null===e||-1===e)&&(e=this.tabs.length?0:!1)),e!==!1&&(e=this.tabs.index(this.tabs.eq(e)),-1===e&&(e=i?!1:0)),!i&&e===!1&&this.anchors.length&&(e=0),e},_getCreateEventData:function(){return{tab:this.active,panel:this.active.length?this._getPanelForTab(this.active):t()}},_tabKeydown:function(e){var i=t(t.ui.safeActiveElement(this.document[0])).closest("li"),s=this.tabs.index(i),n=!0;if(!this._handlePageNav(e)){switch(e.keyCode){case t.ui.keyCode.RIGHT:case t.ui.keyCode.DOWN:s++;break;case t.ui.keyCode.UP:case t.ui.keyCode.LEFT:n=!1,s--;break;case t.ui.keyCode.END:s=this.anchors.length-1;break;case t.ui.keyCode.HOME:s=0;break;case t.ui.keyCode.SPACE:return e.preventDefault(),clearTimeout(this.activating),this._activate(s),void 0;case t.ui.keyCode.ENTER:return e.preventDefault(),clearTimeout(this.activating),this._activate(s===this.options.active?!1:s),void 0;default:return}e.preventDefault(),clearTimeout(this.activating),s=this._focusNextTab(s,n),e.ctrlKey||e.metaKey||(i.attr("aria-selected","false"),this.tabs.eq(s).attr("aria-selected","true"),this.activating=this._delay(function(){this.option("active",s)},this.delay))}},_panelKeydown:function(e){this._handlePageNav(e)||e.ctrlKey&&e.keyCode===t.ui.keyCode.UP&&(e.preventDefault(),this.active.trigger("focus"))},_handlePageNav:function(e){return e.altKey&&e.keyCode===t.ui.keyCode.PAGE_UP?(this._activate(this._focusNextTab(this.options.active-1,!1)),!0):e.altKey&&e.keyCode===t.ui.keyCode.PAGE_DOWN?(this._activate(this._focusNextTab(this.options.active+1,!0)),!0):void 0},_findNextTab:function(e,i){function s(){return e>n&&(e=0),0>e&&(e=n),e}for(var n=this.tabs.length-1;-1!==t.inArray(s(),this.options.disabled);)e=i?e+1:e-1;return e},_focusNextTab:function(t,e){return t=this._findNextTab(t,e),this.tabs.eq(t).trigger("focus"),t},_setOption:function(t,e){return"active"===t?(this._activate(e),void 0):(this._super(t,e),"collapsible"===t&&(this._toggleClass("ui-tabs-collapsible",null,e),e||this.options.active!==!1||this._activate(0)),"event"===t&&this._setupEvents(e),"heightStyle"===t&&this._setupHeightStyle(e),void 0)},_sanitizeSelector:function(t){return t?t.replace(/[!"$%&'()*+,.\/:;<=>?@\[\]\^`{|}~]/g,"\\$&"):""},refresh:function(){var e=this.options,i=this.tablist.children(":has(a[href])");e.disabled=t.map(i.filter(".ui-state-disabled"),function(t){return i.index(t)}),this._processTabs(),e.active!==!1&&this.anchors.length?this.active.length&&!t.contains(this.tablist[0],this.active[0])?this.tabs.length===e.disabled.length?(e.active=!1,this.active=t()):this._activate(this._findNextTab(Math.max(0,e.active-1),!1)):e.active=this.tabs.index(this.active):(e.active=!1,this.active=t()),this._refresh()},_refresh:function(){this._setOptionDisabled(this.options.disabled),this._setupEvents(this.options.event),this._setupHeightStyle(this.options.heightStyle),this.tabs.not(this.active).attr({"aria-selected":"false","aria-expanded":"false",tabIndex:-1}),this.panels.not(this._getPanelForTab(this.active)).hide().attr({"aria-hidden":"true"}),this.active.length?(this.active.attr({"aria-selected":"true","aria-expanded":"true",tabIndex:0}),this._addClass(this.active,"ui-tabs-active","ui-state-active"),this._getPanelForTab(this.active).show().attr({"aria-hidden":"false"})):this.tabs.eq(0).attr("tabIndex",0)},_processTabs:function(){var e=this,i=this.tabs,s=this.anchors,n=this.panels;this.tablist=this._getList().attr("role","tablist"),this._addClass(this.tablist,"ui-tabs-nav","ui-helper-reset ui-helper-clearfix ui-widget-header"),this.tablist.on("mousedown"+this.eventNamespace,"> li",function(e){t(this).is(".ui-state-disabled")&&e.preventDefault()}).on("focus"+this.eventNamespace,".ui-tabs-anchor",function(){t(this).closest("li").is(".ui-state-disabled")&&this.blur()}),this.tabs=this.tablist.find("> li:has(a[href])").attr({role:"tab",tabIndex:-1}),this._addClass(this.tabs,"ui-tabs-tab","ui-state-default"),this.anchors=this.tabs.map(function(){return t("a",this)[0]}).attr({role:"presentation",tabIndex:-1}),this._addClass(this.anchors,"ui-tabs-anchor"),this.panels=t(),this.anchors.each(function(i,s){var n,o,a,r=t(s).uniqueId().attr("id"),h=t(s).closest("li"),l=h.attr("aria-controls");e._isLocal(s)?(n=s.hash,a=n.substring(1),o=e.element.find(e._sanitizeSelector(n))):(a=h.attr("aria-controls")||t({}).uniqueId()[0].id,n="#"+a,o=e.element.find(n),o.length||(o=e._createPanel(a),o.insertAfter(e.panels[i-1]||e.tablist)),o.attr("aria-live","polite")),o.length&&(e.panels=e.panels.add(o)),l&&h.data("ui-tabs-aria-controls",l),h.attr({"aria-controls":a,"aria-labelledby":r}),o.attr("aria-labelledby",r)}),this.panels.attr("role","tabpanel"),this._addClass(this.panels,"ui-tabs-panel","ui-widget-content"),i&&(this._off(i.not(this.tabs)),this._off(s.not(this.anchors)),this._off(n.not(this.panels)))},_getList:function(){return this.tablist||this.element.find("ol, ul").eq(0)},_createPanel:function(e){return t("
    ").attr("id",e).data("ui-tabs-destroy",!0)},_setOptionDisabled:function(e){var i,s,n;for(t.isArray(e)&&(e.length?e.length===this.anchors.length&&(e=!0):e=!1),n=0;s=this.tabs[n];n++)i=t(s),e===!0||-1!==t.inArray(n,e)?(i.attr("aria-disabled","true"),this._addClass(i,null,"ui-state-disabled")):(i.removeAttr("aria-disabled"),this._removeClass(i,null,"ui-state-disabled"));this.options.disabled=e,this._toggleClass(this.widget(),this.widgetFullName+"-disabled",null,e===!0)},_setupEvents:function(e){var i={};e&&t.each(e.split(" "),function(t,e){i[e]="_eventHandler"}),this._off(this.anchors.add(this.tabs).add(this.panels)),this._on(!0,this.anchors,{click:function(t){t.preventDefault()}}),this._on(this.anchors,i),this._on(this.tabs,{keydown:"_tabKeydown"}),this._on(this.panels,{keydown:"_panelKeydown"}),this._focusable(this.tabs),this._hoverable(this.tabs)},_setupHeightStyle:function(e){var i,s=this.element.parent();"fill"===e?(i=s.height(),i-=this.element.outerHeight()-this.element.height(),this.element.siblings(":visible").each(function(){var e=t(this),s=e.css("position");"absolute"!==s&&"fixed"!==s&&(i-=e.outerHeight(!0))}),this.element.children().not(this.panels).each(function(){i-=t(this).outerHeight(!0)}),this.panels.each(function(){t(this).height(Math.max(0,i-t(this).innerHeight()+t(this).height()))}).css("overflow","auto")):"auto"===e&&(i=0,this.panels.each(function(){i=Math.max(i,t(this).height("").height())}).height(i))},_eventHandler:function(e){var i=this.options,s=this.active,n=t(e.currentTarget),o=n.closest("li"),a=o[0]===s[0],r=a&&i.collapsible,h=r?t():this._getPanelForTab(o),l=s.length?this._getPanelForTab(s):t(),c={oldTab:s,oldPanel:l,newTab:r?t():o,newPanel:h};e.preventDefault(),o.hasClass("ui-state-disabled")||o.hasClass("ui-tabs-loading")||this.running||a&&!i.collapsible||this._trigger("beforeActivate",e,c)===!1||(i.active=r?!1:this.tabs.index(o),this.active=a?t():o,this.xhr&&this.xhr.abort(),l.length||h.length||t.error("jQuery UI Tabs: Mismatching fragment identifier."),h.length&&this.load(this.tabs.index(o),e),this._toggle(e,c))},_toggle:function(e,i){function s(){o.running=!1,o._trigger("activate",e,i)}function n(){o._addClass(i.newTab.closest("li"),"ui-tabs-active","ui-state-active"),a.length&&o.options.show?o._show(a,o.options.show,s):(a.show(),s())}var o=this,a=i.newPanel,r=i.oldPanel;this.running=!0,r.length&&this.options.hide?this._hide(r,this.options.hide,function(){o._removeClass(i.oldTab.closest("li"),"ui-tabs-active","ui-state-active"),n()}):(this._removeClass(i.oldTab.closest("li"),"ui-tabs-active","ui-state-active"),r.hide(),n()),r.attr("aria-hidden","true"),i.oldTab.attr({"aria-selected":"false","aria-expanded":"false"}),a.length&&r.length?i.oldTab.attr("tabIndex",-1):a.length&&this.tabs.filter(function(){return 0===t(this).attr("tabIndex")}).attr("tabIndex",-1),a.attr("aria-hidden","false"),i.newTab.attr({"aria-selected":"true","aria-expanded":"true",tabIndex:0})},_activate:function(e){var i,s=this._findActive(e);s[0]!==this.active[0]&&(s.length||(s=this.active),i=s.find(".ui-tabs-anchor")[0],this._eventHandler({target:i,currentTarget:i,preventDefault:t.noop}))},_findActive:function(e){return e===!1?t():this.tabs.eq(e)},_getIndex:function(e){return"string"==typeof e&&(e=this.anchors.index(this.anchors.filter("[href$='"+t.ui.escapeSelector(e)+"']"))),e},_destroy:function(){this.xhr&&this.xhr.abort(),this.tablist.removeAttr("role").off(this.eventNamespace),this.anchors.removeAttr("role tabIndex").removeUniqueId(),this.tabs.add(this.panels).each(function(){t.data(this,"ui-tabs-destroy")?t(this).remove():t(this).removeAttr("role tabIndex aria-live aria-busy aria-selected aria-labelledby aria-hidden aria-expanded")}),this.tabs.each(function(){var e=t(this),i=e.data("ui-tabs-aria-controls");i?e.attr("aria-controls",i).removeData("ui-tabs-aria-controls"):e.removeAttr("aria-controls")}),this.panels.show(),"content"!==this.options.heightStyle&&this.panels.css("height","")},enable:function(e){var i=this.options.disabled;i!==!1&&(void 0===e?i=!1:(e=this._getIndex(e),i=t.isArray(i)?t.map(i,function(t){return t!==e?t:null}):t.map(this.tabs,function(t,i){return i!==e?i:null})),this._setOptionDisabled(i))},disable:function(e){var i=this.options.disabled;if(i!==!0){if(void 0===e)i=!0;else{if(e=this._getIndex(e),-1!==t.inArray(e,i))return;i=t.isArray(i)?t.merge([e],i).sort():[e]}this._setOptionDisabled(i)}},load:function(e,i){e=this._getIndex(e);var s=this,n=this.tabs.eq(e),o=n.find(".ui-tabs-anchor"),a=this._getPanelForTab(n),r={tab:n,panel:a},h=function(t,e){"abort"===e&&s.panels.stop(!1,!0),s._removeClass(n,"ui-tabs-loading"),a.removeAttr("aria-busy"),t===s.xhr&&delete s.xhr};this._isLocal(o[0])||(this.xhr=t.ajax(this._ajaxSettings(o,i,r)),this.xhr&&"canceled"!==this.xhr.statusText&&(this._addClass(n,"ui-tabs-loading"),a.attr("aria-busy","true"),this.xhr.done(function(t,e,n){setTimeout(function(){a.html(t),s._trigger("load",i,r),h(n,e)},1)}).fail(function(t,e){setTimeout(function(){h(t,e)},1)})))},_ajaxSettings:function(e,i,s){var n=this;return{url:e.attr("href").replace(/#.*$/,""),beforeSend:function(e,o){return n._trigger("beforeLoad",i,t.extend({jqXHR:e,ajaxSettings:o},s))}}},_getPanelForTab:function(e){var i=t(e).attr("aria-controls");return this.element.find(this._sanitizeSelector("#"+i))}}),t.uiBackCompat!==!1&&t.widget("ui.tabs",t.ui.tabs,{_processTabs:function(){this._superApply(arguments),this._addClass(this.tabs,"ui-tab")}}),t.ui.tabs,t.widget("ui.tooltip",{version:"1.12.1",options:{classes:{"ui-tooltip":"ui-corner-all ui-widget-shadow"},content:function(){var e=t(this).attr("title")||"";return t("").text(e).html()},hide:!0,items:"[title]:not([disabled])",position:{my:"left top+15",at:"left bottom",collision:"flipfit flip"},show:!0,track:!1,close:null,open:null},_addDescribedBy:function(e,i){var s=(e.attr("aria-describedby")||"").split(/\s+/);s.push(i),e.data("ui-tooltip-id",i).attr("aria-describedby",t.trim(s.join(" ")))},_removeDescribedBy:function(e){var i=e.data("ui-tooltip-id"),s=(e.attr("aria-describedby")||"").split(/\s+/),n=t.inArray(i,s);-1!==n&&s.splice(n,1),e.removeData("ui-tooltip-id"),s=t.trim(s.join(" ")),s?e.attr("aria-describedby",s):e.removeAttr("aria-describedby")},_create:function(){this._on({mouseover:"open",focusin:"open"}),this.tooltips={},this.parents={},this.liveRegion=t("
    ").attr({role:"log","aria-live":"assertive","aria-relevant":"additions"}).appendTo(this.document[0].body),this._addClass(this.liveRegion,null,"ui-helper-hidden-accessible"),this.disabledTitles=t([])},_setOption:function(e,i){var s=this;this._super(e,i),"content"===e&&t.each(this.tooltips,function(t,e){s._updateContent(e.element)})},_setOptionDisabled:function(t){this[t?"_disable":"_enable"]()},_disable:function(){var e=this;t.each(this.tooltips,function(i,s){var n=t.Event("blur");n.target=n.currentTarget=s.element[0],e.close(n,!0)}),this.disabledTitles=this.disabledTitles.add(this.element.find(this.options.items).addBack().filter(function(){var e=t(this);return e.is("[title]")?e.data("ui-tooltip-title",e.attr("title")).removeAttr("title"):void 0}))},_enable:function(){this.disabledTitles.each(function(){var e=t(this);e.data("ui-tooltip-title")&&e.attr("title",e.data("ui-tooltip-title"))}),this.disabledTitles=t([])},open:function(e){var i=this,s=t(e?e.target:this.element).closest(this.options.items);s.length&&!s.data("ui-tooltip-id")&&(s.attr("title")&&s.data("ui-tooltip-title",s.attr("title")),s.data("ui-tooltip-open",!0),e&&"mouseover"===e.type&&s.parents().each(function(){var e,s=t(this);s.data("ui-tooltip-open")&&(e=t.Event("blur"),e.target=e.currentTarget=this,i.close(e,!0)),s.attr("title")&&(s.uniqueId(),i.parents[this.id]={element:this,title:s.attr("title")},s.attr("title",""))}),this._registerCloseHandlers(e,s),this._updateContent(s,e))},_updateContent:function(t,e){var i,s=this.options.content,n=this,o=e?e.type:null;return"string"==typeof s||s.nodeType||s.jquery?this._open(e,t,s):(i=s.call(t[0],function(i){n._delay(function(){t.data("ui-tooltip-open")&&(e&&(e.type=o),this._open(e,t,i))})}),i&&this._open(e,t,i),void 0)},_open:function(e,i,s){function n(t){l.of=t,a.is(":hidden")||a.position(l)}var o,a,r,h,l=t.extend({},this.options.position);if(s){if(o=this._find(i))return o.tooltip.find(".ui-tooltip-content").html(s),void 0;i.is("[title]")&&(e&&"mouseover"===e.type?i.attr("title",""):i.removeAttr("title")),o=this._tooltip(i),a=o.tooltip,this._addDescribedBy(i,a.attr("id")),a.find(".ui-tooltip-content").html(s),this.liveRegion.children().hide(),h=t("
    ").html(a.find(".ui-tooltip-content").html()),h.removeAttr("name").find("[name]").removeAttr("name"),h.removeAttr("id").find("[id]").removeAttr("id"),h.appendTo(this.liveRegion),this.options.track&&e&&/^mouse/.test(e.type)?(this._on(this.document,{mousemove:n}),n(e)):a.position(t.extend({of:i},this.options.position)),a.hide(),this._show(a,this.options.show),this.options.track&&this.options.show&&this.options.show.delay&&(r=this.delayedShow=setInterval(function(){a.is(":visible")&&(n(l.of),clearInterval(r))},t.fx.interval)),this._trigger("open",e,{tooltip:a})}},_registerCloseHandlers:function(e,i){var s={keyup:function(e){if(e.keyCode===t.ui.keyCode.ESCAPE){var s=t.Event(e);s.currentTarget=i[0],this.close(s,!0)}}};i[0]!==this.element[0]&&(s.remove=function(){this._removeTooltip(this._find(i).tooltip)}),e&&"mouseover"!==e.type||(s.mouseleave="close"),e&&"focusin"!==e.type||(s.focusout="close"),this._on(!0,i,s)},close:function(e){var i,s=this,n=t(e?e.currentTarget:this.element),o=this._find(n);return o?(i=o.tooltip,o.closing||(clearInterval(this.delayedShow),n.data("ui-tooltip-title")&&!n.attr("title")&&n.attr("title",n.data("ui-tooltip-title")),this._removeDescribedBy(n),o.hiding=!0,i.stop(!0),this._hide(i,this.options.hide,function(){s._removeTooltip(t(this))}),n.removeData("ui-tooltip-open"),this._off(n,"mouseleave focusout keyup"),n[0]!==this.element[0]&&this._off(n,"remove"),this._off(this.document,"mousemove"),e&&"mouseleave"===e.type&&t.each(this.parents,function(e,i){t(i.element).attr("title",i.title),delete s.parents[e]}),o.closing=!0,this._trigger("close",e,{tooltip:i}),o.hiding||(o.closing=!1)),void 0):(n.removeData("ui-tooltip-open"),void 0)},_tooltip:function(e){var i=t("
    ").attr("role","tooltip"),s=t("
    ").appendTo(i),n=i.uniqueId().attr("id");return this._addClass(s,"ui-tooltip-content"),this._addClass(i,"ui-tooltip","ui-widget ui-widget-content"),i.appendTo(this._appendTo(e)),this.tooltips[n]={element:e,tooltip:i}},_find:function(t){var e=t.data("ui-tooltip-id");return e?this.tooltips[e]:null},_removeTooltip:function(t){t.remove(),delete this.tooltips[t.attr("id")]},_appendTo:function(t){var e=t.closest(".ui-front, dialog");return e.length||(e=this.document[0].body),e},_destroy:function(){var e=this;t.each(this.tooltips,function(i,s){var n=t.Event("blur"),o=s.element;n.target=n.currentTarget=o[0],e.close(n,!0),t("#"+i).remove(),o.data("ui-tooltip-title")&&(o.attr("title")||o.attr("title",o.data("ui-tooltip-title")),o.removeData("ui-tooltip-title"))}),this.liveRegion.remove()}}),t.uiBackCompat!==!1&&t.widget("ui.tooltip",t.ui.tooltip,{options:{tooltipClass:null},_tooltip:function(){var t=this._superApply(arguments);return this.options.tooltipClass&&t.tooltip.addClass(this.options.tooltipClass),t}}),t.ui.tooltip}); \ No newline at end of file diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/main/resources/webapps/static/jquery/jquery-ui-1.13.1.custom.min.js b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/main/resources/webapps/static/jquery/jquery-ui-1.13.1.custom.min.js new file mode 100644 index 0000000000000..de4b5fc8d1a9b --- /dev/null +++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/main/resources/webapps/static/jquery/jquery-ui-1.13.1.custom.min.js @@ -0,0 +1,6 @@ +/*! jQuery UI - v1.13.1 - 2022-04-24 +* http://jqueryui.com +* Includes: widget.js, position.js, data.js, disable-selection.js, focusable.js, form-reset-mixin.js, jquery-patch.js, keycode.js, labels.js, scroll-parent.js, tabbable.js, unique-id.js, widgets/draggable.js, widgets/droppable.js, widgets/resizable.js, widgets/selectable.js, widgets/sortable.js, widgets/accordion.js, widgets/autocomplete.js, widgets/button.js, widgets/checkboxradio.js, widgets/controlgroup.js, widgets/datepicker.js, widgets/dialog.js, widgets/menu.js, widgets/mouse.js, widgets/progressbar.js, widgets/selectmenu.js, widgets/slider.js, widgets/spinner.js, widgets/tabs.js, widgets/tooltip.js, effect.js, effects/effect-blind.js, effects/effect-bounce.js, effects/effect-clip.js, effects/effect-drop.js, effects/effect-explode.js, effects/effect-fade.js, effects/effect-fold.js, effects/effect-highlight.js, effects/effect-puff.js, effects/effect-pulsate.js, effects/effect-scale.js, effects/effect-shake.js, effects/effect-size.js, effects/effect-slide.js, effects/effect-transfer.js +* Copyright jQuery Foundation and other contributors; Licensed MIT */ + +!function(t){"use strict";"function"==typeof define&&define.amd?define(["jquery"],t):t(jQuery)}(function(V){"use strict";V.ui=V.ui||{};V.ui.version="1.13.1";var n,i=0,a=Array.prototype.hasOwnProperty,r=Array.prototype.slice;V.cleanData=(n=V.cleanData,function(t){for(var e,i,s=0;null!=(i=t[s]);s++)(e=V._data(i,"events"))&&e.remove&&V(i).triggerHandler("remove");n(t)}),V.widget=function(t,i,e){var s,n,o,a={},r=t.split(".")[0],l=r+"-"+(t=t.split(".")[1]);return e||(e=i,i=V.Widget),Array.isArray(e)&&(e=V.extend.apply(null,[{}].concat(e))),V.expr.pseudos[l.toLowerCase()]=function(t){return!!V.data(t,l)},V[r]=V[r]||{},s=V[r][t],n=V[r][t]=function(t,e){if(!this||!this._createWidget)return new n(t,e);arguments.length&&this._createWidget(t,e)},V.extend(n,s,{version:e.version,_proto:V.extend({},e),_childConstructors:[]}),(o=new i).options=V.widget.extend({},o.options),V.each(e,function(e,s){function n(){return i.prototype[e].apply(this,arguments)}function o(t){return i.prototype[e].apply(this,t)}a[e]="function"==typeof s?function(){var t,e=this._super,i=this._superApply;return this._super=n,this._superApply=o,t=s.apply(this,arguments),this._super=e,this._superApply=i,t}:s}),n.prototype=V.widget.extend(o,{widgetEventPrefix:s&&o.widgetEventPrefix||t},a,{constructor:n,namespace:r,widgetName:t,widgetFullName:l}),s?(V.each(s._childConstructors,function(t,e){var i=e.prototype;V.widget(i.namespace+"."+i.widgetName,n,e._proto)}),delete s._childConstructors):i._childConstructors.push(n),V.widget.bridge(t,n),n},V.widget.extend=function(t){for(var e,i,s=r.call(arguments,1),n=0,o=s.length;n",options:{classes:{},disabled:!1,create:null},_createWidget:function(t,e){e=V(e||this.defaultElement||this)[0],this.element=V(e),this.uuid=i++,this.eventNamespace="."+this.widgetName+this.uuid,this.bindings=V(),this.hoverable=V(),this.focusable=V(),this.classesElementLookup={},e!==this&&(V.data(e,this.widgetFullName,this),this._on(!0,this.element,{remove:function(t){t.target===e&&this.destroy()}}),this.document=V(e.style?e.ownerDocument:e.document||e),this.window=V(this.document[0].defaultView||this.document[0].parentWindow)),this.options=V.widget.extend({},this.options,this._getCreateOptions(),t),this._create(),this.options.disabled&&this._setOptionDisabled(this.options.disabled),this._trigger("create",null,this._getCreateEventData()),this._init()},_getCreateOptions:function(){return{}},_getCreateEventData:V.noop,_create:V.noop,_init:V.noop,destroy:function(){var i=this;this._destroy(),V.each(this.classesElementLookup,function(t,e){i._removeClass(e,t)}),this.element.off(this.eventNamespace).removeData(this.widgetFullName),this.widget().off(this.eventNamespace).removeAttr("aria-disabled"),this.bindings.off(this.eventNamespace)},_destroy:V.noop,widget:function(){return this.element},option:function(t,e){var i,s,n,o=t;if(0===arguments.length)return V.widget.extend({},this.options);if("string"==typeof t)if(o={},t=(i=t.split(".")).shift(),i.length){for(s=o[t]=V.widget.extend({},this.options[t]),n=0;n
    "),i=e.children()[0];return V("body").append(e),t=i.offsetWidth,e.css("overflow","scroll"),t===(i=i.offsetWidth)&&(i=e[0].clientWidth),e.remove(),s=t-i},getScrollInfo:function(t){var e=t.isWindow||t.isDocument?"":t.element.css("overflow-x"),i=t.isWindow||t.isDocument?"":t.element.css("overflow-y"),e="scroll"===e||"auto"===e&&t.widthx(k(s),k(n))?o.important="horizontal":o.important="vertical",u.using.call(this,t,o)}),a.offset(V.extend(h,{using:t}))})},V.ui.position={fit:{left:function(t,e){var i=e.within,s=i.isWindow?i.scrollLeft:i.offset.left,n=i.width,o=t.left-e.collisionPosition.marginLeft,a=s-o,r=o+e.collisionWidth-n-s;e.collisionWidth>n?0n?0=this.options.distance},_mouseDelayMet:function(){return this.mouseDelayMet},_mouseStart:function(){},_mouseDrag:function(){},_mouseStop:function(){},_mouseCapture:function(){return!0}}),V.ui.plugin={add:function(t,e,i){var s,n=V.ui[t].prototype;for(s in i)n.plugins[s]=n.plugins[s]||[],n.plugins[s].push([e,i[s]])},call:function(t,e,i,s){var n,o=t.plugins[e];if(o&&(s||t.element[0].parentNode&&11!==t.element[0].parentNode.nodeType))for(n=0;n").css("position","absolute").appendTo(t.parent()).outerWidth(t.outerWidth()).outerHeight(t.outerHeight()).offset(t.offset())[0]})},_unblockFrames:function(){this.iframeBlocks&&(this.iframeBlocks.remove(),delete this.iframeBlocks)},_blurActiveElement:function(t){var e=V.ui.safeActiveElement(this.document[0]);V(t.target).closest(e).length||V.ui.safeBlur(e)},_mouseStart:function(t){var e=this.options;return this.helper=this._createHelper(t),this._addClass(this.helper,"ui-draggable-dragging"),this._cacheHelperProportions(),V.ui.ddmanager&&(V.ui.ddmanager.current=this),this._cacheMargins(),this.cssPosition=this.helper.css("position"),this.scrollParent=this.helper.scrollParent(!0),this.offsetParent=this.helper.offsetParent(),this.hasFixedAncestor=0i[2]&&(o=i[2]+this.offset.click.left),t.pageY-this.offset.click.top>i[3]&&(a=i[3]+this.offset.click.top)),s.grid&&(t=s.grid[1]?this.originalPageY+Math.round((a-this.originalPageY)/s.grid[1])*s.grid[1]:this.originalPageY,a=!i||t-this.offset.click.top>=i[1]||t-this.offset.click.top>i[3]?t:t-this.offset.click.top>=i[1]?t-s.grid[1]:t+s.grid[1],t=s.grid[0]?this.originalPageX+Math.round((o-this.originalPageX)/s.grid[0])*s.grid[0]:this.originalPageX,o=!i||t-this.offset.click.left>=i[0]||t-this.offset.click.left>i[2]?t:t-this.offset.click.left>=i[0]?t-s.grid[0]:t+s.grid[0]),"y"===s.axis&&(o=this.originalPageX),"x"===s.axis&&(a=this.originalPageY)),{top:a-this.offset.click.top-this.offset.relative.top-this.offset.parent.top+("fixed"===this.cssPosition?-this.offset.scroll.top:n?0:this.offset.scroll.top),left:o-this.offset.click.left-this.offset.relative.left-this.offset.parent.left+("fixed"===this.cssPosition?-this.offset.scroll.left:n?0:this.offset.scroll.left)}},_clear:function(){this._removeClass(this.helper,"ui-draggable-dragging"),this.helper[0]===this.element[0]||this.cancelHelperRemoval||this.helper.remove(),this.helper=null,this.cancelHelperRemoval=!1,this.destroyOnClear&&this.destroy()},_trigger:function(t,e,i){return i=i||this._uiHash(),V.ui.plugin.call(this,t,[e,i,this],!0),/^(drag|start|stop)/.test(t)&&(this.positionAbs=this._convertPositionTo("absolute"),i.offset=this.positionAbs),V.Widget.prototype._trigger.call(this,t,e,i)},plugins:{},_uiHash:function(){return{helper:this.helper,position:this.position,originalPosition:this.originalPosition,offset:this.positionAbs}}}),V.ui.plugin.add("draggable","connectToSortable",{start:function(e,t,i){var s=V.extend({},t,{item:i.element});i.sortables=[],V(i.options.connectToSortable).each(function(){var t=V(this).sortable("instance");t&&!t.options.disabled&&(i.sortables.push(t),t.refreshPositions(),t._trigger("activate",e,s))})},stop:function(e,t,i){var s=V.extend({},t,{item:i.element});i.cancelHelperRemoval=!1,V.each(i.sortables,function(){var t=this;t.isOver?(t.isOver=0,i.cancelHelperRemoval=!0,t.cancelHelperRemoval=!1,t._storedCSS={position:t.placeholder.css("position"),top:t.placeholder.css("top"),left:t.placeholder.css("left")},t._mouseStop(e),t.options.helper=t.options._helper):(t.cancelHelperRemoval=!0,t._trigger("deactivate",e,s))})},drag:function(i,s,n){V.each(n.sortables,function(){var t=!1,e=this;e.positionAbs=n.positionAbs,e.helperProportions=n.helperProportions,e.offset.click=n.offset.click,e._intersectsWith(e.containerCache)&&(t=!0,V.each(n.sortables,function(){return this.positionAbs=n.positionAbs,this.helperProportions=n.helperProportions,this.offset.click=n.offset.click,t=this!==e&&this._intersectsWith(this.containerCache)&&V.contains(e.element[0],this.element[0])?!1:t})),t?(e.isOver||(e.isOver=1,n._parent=s.helper.parent(),e.currentItem=s.helper.appendTo(e.element).data("ui-sortable-item",!0),e.options._helper=e.options.helper,e.options.helper=function(){return s.helper[0]},i.target=e.currentItem[0],e._mouseCapture(i,!0),e._mouseStart(i,!0,!0),e.offset.click.top=n.offset.click.top,e.offset.click.left=n.offset.click.left,e.offset.parent.left-=n.offset.parent.left-e.offset.parent.left,e.offset.parent.top-=n.offset.parent.top-e.offset.parent.top,n._trigger("toSortable",i),n.dropped=e.element,V.each(n.sortables,function(){this.refreshPositions()}),n.currentItem=n.element,e.fromOutside=n),e.currentItem&&(e._mouseDrag(i),s.position=e.position)):e.isOver&&(e.isOver=0,e.cancelHelperRemoval=!0,e.options._revert=e.options.revert,e.options.revert=!1,e._trigger("out",i,e._uiHash(e)),e._mouseStop(i,!0),e.options.revert=e.options._revert,e.options.helper=e.options._helper,e.placeholder&&e.placeholder.remove(),s.helper.appendTo(n._parent),n._refreshOffsets(i),s.position=n._generatePosition(i,!0),n._trigger("fromSortable",i),n.dropped=!1,V.each(n.sortables,function(){this.refreshPositions()}))})}}),V.ui.plugin.add("draggable","cursor",{start:function(t,e,i){var s=V("body"),i=i.options;s.css("cursor")&&(i._cursor=s.css("cursor")),s.css("cursor",i.cursor)},stop:function(t,e,i){i=i.options;i._cursor&&V("body").css("cursor",i._cursor)}}),V.ui.plugin.add("draggable","opacity",{start:function(t,e,i){e=V(e.helper),i=i.options;e.css("opacity")&&(i._opacity=e.css("opacity")),e.css("opacity",i.opacity)},stop:function(t,e,i){i=i.options;i._opacity&&V(e.helper).css("opacity",i._opacity)}}),V.ui.plugin.add("draggable","scroll",{start:function(t,e,i){i.scrollParentNotHidden||(i.scrollParentNotHidden=i.helper.scrollParent(!1)),i.scrollParentNotHidden[0]!==i.document[0]&&"HTML"!==i.scrollParentNotHidden[0].tagName&&(i.overflowOffset=i.scrollParentNotHidden.offset())},drag:function(t,e,i){var s=i.options,n=!1,o=i.scrollParentNotHidden[0],a=i.document[0];o!==a&&"HTML"!==o.tagName?(s.axis&&"x"===s.axis||(i.overflowOffset.top+o.offsetHeight-t.pageY
    ").css({overflow:"hidden",position:this.element.css("position"),width:this.element.outerWidth(),height:this.element.outerHeight(),top:this.element.css("top"),left:this.element.css("left")})),this.element=this.element.parent().data("ui-resizable",this.element.resizable("instance")),this.elementIsWrapper=!0,t={marginTop:this.originalElement.css("marginTop"),marginRight:this.originalElement.css("marginRight"),marginBottom:this.originalElement.css("marginBottom"),marginLeft:this.originalElement.css("marginLeft")},this.element.css(t),this.originalElement.css("margin",0),this.originalResizeStyle=this.originalElement.css("resize"),this.originalElement.css("resize","none"),this._proportionallyResizeElements.push(this.originalElement.css({position:"static",zoom:1,display:"block"})),this.originalElement.css(t),this._proportionallyResize()),this._setupHandles(),e.autoHide&&V(this.element).on("mouseenter",function(){e.disabled||(i._removeClass("ui-resizable-autohide"),i._handles.show())}).on("mouseleave",function(){e.disabled||i.resizing||(i._addClass("ui-resizable-autohide"),i._handles.hide())}),this._mouseInit()},_destroy:function(){this._mouseDestroy(),this._addedHandles.remove();function t(t){V(t).removeData("resizable").removeData("ui-resizable").off(".resizable")}var e;return this.elementIsWrapper&&(t(this.element),e=this.element,this.originalElement.css({position:e.css("position"),width:e.outerWidth(),height:e.outerHeight(),top:e.css("top"),left:e.css("left")}).insertAfter(e),e.remove()),this.originalElement.css("resize",this.originalResizeStyle),t(this.originalElement),this},_setOption:function(t,e){switch(this._super(t,e),t){case"handles":this._removeHandles(),this._setupHandles();break;case"aspectRatio":this._aspectRatio=!!e}},_setupHandles:function(){var t,e,i,s,n,o=this.options,a=this;if(this.handles=o.handles||(V(".ui-resizable-handle",this.element).length?{n:".ui-resizable-n",e:".ui-resizable-e",s:".ui-resizable-s",w:".ui-resizable-w",se:".ui-resizable-se",sw:".ui-resizable-sw",ne:".ui-resizable-ne",nw:".ui-resizable-nw"}:"e,s,se"),this._handles=V(),this._addedHandles=V(),this.handles.constructor===String)for("all"===this.handles&&(this.handles="n,e,s,w,se,sw,ne,nw"),i=this.handles.split(","),this.handles={},e=0;e"),this._addClass(n,"ui-resizable-handle "+s),n.css({zIndex:o.zIndex}),this.handles[t]=".ui-resizable-"+t,this.element.children(this.handles[t]).length||(this.element.append(n),this._addedHandles=this._addedHandles.add(n));this._renderAxis=function(t){var e,i,s;for(e in t=t||this.element,this.handles)this.handles[e].constructor===String?this.handles[e]=this.element.children(this.handles[e]).first().show():(this.handles[e].jquery||this.handles[e].nodeType)&&(this.handles[e]=V(this.handles[e]),this._on(this.handles[e],{mousedown:a._mouseDown})),this.elementIsWrapper&&this.originalElement[0].nodeName.match(/^(textarea|input|select|button)$/i)&&(i=V(this.handles[e],this.element),s=/sw|ne|nw|se|n|s/.test(e)?i.outerHeight():i.outerWidth(),i=["padding",/ne|nw|n/.test(e)?"Top":/se|sw|s/.test(e)?"Bottom":/^e$/.test(e)?"Right":"Left"].join(""),t.css(i,s),this._proportionallyResize()),this._handles=this._handles.add(this.handles[e])},this._renderAxis(this.element),this._handles=this._handles.add(this.element.find(".ui-resizable-handle")),this._handles.disableSelection(),this._handles.on("mouseover",function(){a.resizing||(this.className&&(n=this.className.match(/ui-resizable-(se|sw|ne|nw|n|e|s|w)/i)),a.axis=n&&n[1]?n[1]:"se")}),o.autoHide&&(this._handles.hide(),this._addClass("ui-resizable-autohide"))},_removeHandles:function(){this._addedHandles.remove()},_mouseCapture:function(t){var e,i,s=!1;for(e in this.handles)(i=V(this.handles[e])[0])!==t.target&&!V.contains(i,t.target)||(s=!0);return!this.options.disabled&&s},_mouseStart:function(t){var e,i,s=this.options,n=this.element;return this.resizing=!0,this._renderProxy(),e=this._num(this.helper.css("left")),i=this._num(this.helper.css("top")),s.containment&&(e+=V(s.containment).scrollLeft()||0,i+=V(s.containment).scrollTop()||0),this.offset=this.helper.offset(),this.position={left:e,top:i},this.size=this._helper?{width:this.helper.width(),height:this.helper.height()}:{width:n.width(),height:n.height()},this.originalSize=this._helper?{width:n.outerWidth(),height:n.outerHeight()}:{width:n.width(),height:n.height()},this.sizeDiff={width:n.outerWidth()-n.width(),height:n.outerHeight()-n.height()},this.originalPosition={left:e,top:i},this.originalMousePosition={left:t.pageX,top:t.pageY},this.aspectRatio="number"==typeof s.aspectRatio?s.aspectRatio:this.originalSize.width/this.originalSize.height||1,s=V(".ui-resizable-"+this.axis).css("cursor"),V("body").css("cursor","auto"===s?this.axis+"-resize":s),this._addClass("ui-resizable-resizing"),this._propagate("start",t),!0},_mouseDrag:function(t){var e=this.originalMousePosition,i=this.axis,s=t.pageX-e.left||0,e=t.pageY-e.top||0,i=this._change[i];return this._updatePrevProperties(),i&&(e=i.apply(this,[t,s,e]),this._updateVirtualBoundaries(t.shiftKey),(this._aspectRatio||t.shiftKey)&&(e=this._updateRatio(e,t)),e=this._respectSize(e,t),this._updateCache(e),this._propagate("resize",t),e=this._applyChanges(),!this._helper&&this._proportionallyResizeElements.length&&this._proportionallyResize(),V.isEmptyObject(e)||(this._updatePrevProperties(),this._trigger("resize",t,this.ui()),this._applyChanges())),!1},_mouseStop:function(t){this.resizing=!1;var e,i,s,n=this.options,o=this;return this._helper&&(s=(e=(i=this._proportionallyResizeElements).length&&/textarea/i.test(i[0].nodeName))&&this._hasScroll(i[0],"left")?0:o.sizeDiff.height,i=e?0:o.sizeDiff.width,e={width:o.helper.width()-i,height:o.helper.height()-s},i=parseFloat(o.element.css("left"))+(o.position.left-o.originalPosition.left)||null,s=parseFloat(o.element.css("top"))+(o.position.top-o.originalPosition.top)||null,n.animate||this.element.css(V.extend(e,{top:s,left:i})),o.helper.height(o.size.height),o.helper.width(o.size.width),this._helper&&!n.animate&&this._proportionallyResize()),V("body").css("cursor","auto"),this._removeClass("ui-resizable-resizing"),this._propagate("stop",t),this._helper&&this.helper.remove(),!1},_updatePrevProperties:function(){this.prevPosition={top:this.position.top,left:this.position.left},this.prevSize={width:this.size.width,height:this.size.height}},_applyChanges:function(){var t={};return this.position.top!==this.prevPosition.top&&(t.top=this.position.top+"px"),this.position.left!==this.prevPosition.left&&(t.left=this.position.left+"px"),this.size.width!==this.prevSize.width&&(t.width=this.size.width+"px"),this.size.height!==this.prevSize.height&&(t.height=this.size.height+"px"),this.helper.css(t),t},_updateVirtualBoundaries:function(t){var e,i,s=this.options,n={minWidth:this._isNumber(s.minWidth)?s.minWidth:0,maxWidth:this._isNumber(s.maxWidth)?s.maxWidth:1/0,minHeight:this._isNumber(s.minHeight)?s.minHeight:0,maxHeight:this._isNumber(s.maxHeight)?s.maxHeight:1/0};(this._aspectRatio||t)&&(e=n.minHeight*this.aspectRatio,i=n.minWidth/this.aspectRatio,s=n.maxHeight*this.aspectRatio,t=n.maxWidth/this.aspectRatio,e>n.minWidth&&(n.minWidth=e),i>n.minHeight&&(n.minHeight=i),st.width,a=this._isNumber(t.height)&&e.minHeight&&e.minHeight>t.height,r=this.originalPosition.left+this.originalSize.width,l=this.originalPosition.top+this.originalSize.height,h=/sw|nw|w/.test(i),i=/nw|ne|n/.test(i);return o&&(t.width=e.minWidth),a&&(t.height=e.minHeight),s&&(t.width=e.maxWidth),n&&(t.height=e.maxHeight),o&&h&&(t.left=r-e.minWidth),s&&h&&(t.left=r-e.maxWidth),a&&i&&(t.top=l-e.minHeight),n&&i&&(t.top=l-e.maxHeight),t.width||t.height||t.left||!t.top?t.width||t.height||t.top||!t.left||(t.left=null):t.top=null,t},_getPaddingPlusBorderDimensions:function(t){for(var e=0,i=[],s=[t.css("borderTopWidth"),t.css("borderRightWidth"),t.css("borderBottomWidth"),t.css("borderLeftWidth")],n=[t.css("paddingTop"),t.css("paddingRight"),t.css("paddingBottom"),t.css("paddingLeft")];e<4;e++)i[e]=parseFloat(s[e])||0,i[e]+=parseFloat(n[e])||0;return{height:i[0]+i[2],width:i[1]+i[3]}},_proportionallyResize:function(){if(this._proportionallyResizeElements.length)for(var t,e=0,i=this.helper||this.element;e
    ").css({overflow:"hidden"}),this._addClass(this.helper,this._helper),this.helper.css({width:this.element.outerWidth(),height:this.element.outerHeight(),position:"absolute",left:this.elementOffset.left+"px",top:this.elementOffset.top+"px",zIndex:++e.zIndex}),this.helper.appendTo("body").disableSelection()):this.helper=this.element},_change:{e:function(t,e){return{width:this.originalSize.width+e}},w:function(t,e){var i=this.originalSize;return{left:this.originalPosition.left+e,width:i.width-e}},n:function(t,e,i){var s=this.originalSize;return{top:this.originalPosition.top+i,height:s.height-i}},s:function(t,e,i){return{height:this.originalSize.height+i}},se:function(t,e,i){return V.extend(this._change.s.apply(this,arguments),this._change.e.apply(this,[t,e,i]))},sw:function(t,e,i){return V.extend(this._change.s.apply(this,arguments),this._change.w.apply(this,[t,e,i]))},ne:function(t,e,i){return V.extend(this._change.n.apply(this,arguments),this._change.e.apply(this,[t,e,i]))},nw:function(t,e,i){return V.extend(this._change.n.apply(this,arguments),this._change.w.apply(this,[t,e,i]))}},_propagate:function(t,e){V.ui.plugin.call(this,t,[e,this.ui()]),"resize"!==t&&this._trigger(t,e,this.ui())},plugins:{},ui:function(){return{originalElement:this.originalElement,element:this.element,helper:this.helper,position:this.position,size:this.size,originalSize:this.originalSize,originalPosition:this.originalPosition}}}),V.ui.plugin.add("resizable","animate",{stop:function(e){var i=V(this).resizable("instance"),t=i.options,s=i._proportionallyResizeElements,n=s.length&&/textarea/i.test(s[0].nodeName),o=n&&i._hasScroll(s[0],"left")?0:i.sizeDiff.height,a=n?0:i.sizeDiff.width,n={width:i.size.width-a,height:i.size.height-o},a=parseFloat(i.element.css("left"))+(i.position.left-i.originalPosition.left)||null,o=parseFloat(i.element.css("top"))+(i.position.top-i.originalPosition.top)||null;i.element.animate(V.extend(n,o&&a?{top:o,left:a}:{}),{duration:t.animateDuration,easing:t.animateEasing,step:function(){var t={width:parseFloat(i.element.css("width")),height:parseFloat(i.element.css("height")),top:parseFloat(i.element.css("top")),left:parseFloat(i.element.css("left"))};s&&s.length&&V(s[0]).css({width:t.width,height:t.height}),i._updateCache(t),i._propagate("resize",e)}})}}),V.ui.plugin.add("resizable","containment",{start:function(){var i,s,n=V(this).resizable("instance"),t=n.options,e=n.element,o=t.containment,a=o instanceof V?o.get(0):/parent/.test(o)?e.parent().get(0):o;a&&(n.containerElement=V(a),/document/.test(o)||o===document?(n.containerOffset={left:0,top:0},n.containerPosition={left:0,top:0},n.parentData={element:V(document),left:0,top:0,width:V(document).width(),height:V(document).height()||document.body.parentNode.scrollHeight}):(i=V(a),s=[],V(["Top","Right","Left","Bottom"]).each(function(t,e){s[t]=n._num(i.css("padding"+e))}),n.containerOffset=i.offset(),n.containerPosition=i.position(),n.containerSize={height:i.innerHeight()-s[3],width:i.innerWidth()-s[1]},t=n.containerOffset,e=n.containerSize.height,o=n.containerSize.width,o=n._hasScroll(a,"left")?a.scrollWidth:o,e=n._hasScroll(a)?a.scrollHeight:e,n.parentData={element:a,left:t.left,top:t.top,width:o,height:e}))},resize:function(t){var e=V(this).resizable("instance"),i=e.options,s=e.containerOffset,n=e.position,o=e._aspectRatio||t.shiftKey,a={top:0,left:0},r=e.containerElement,t=!0;r[0]!==document&&/static/.test(r.css("position"))&&(a=s),n.left<(e._helper?s.left:0)&&(e.size.width=e.size.width+(e._helper?e.position.left-s.left:e.position.left-a.left),o&&(e.size.height=e.size.width/e.aspectRatio,t=!1),e.position.left=i.helper?s.left:0),n.top<(e._helper?s.top:0)&&(e.size.height=e.size.height+(e._helper?e.position.top-s.top:e.position.top),o&&(e.size.width=e.size.height*e.aspectRatio,t=!1),e.position.top=e._helper?s.top:0),i=e.containerElement.get(0)===e.element.parent().get(0),n=/relative|absolute/.test(e.containerElement.css("position")),i&&n?(e.offset.left=e.parentData.left+e.position.left,e.offset.top=e.parentData.top+e.position.top):(e.offset.left=e.element.offset().left,e.offset.top=e.element.offset().top),n=Math.abs(e.sizeDiff.width+(e._helper?e.offset.left-a.left:e.offset.left-s.left)),s=Math.abs(e.sizeDiff.height+(e._helper?e.offset.top-a.top:e.offset.top-s.top)),n+e.size.width>=e.parentData.width&&(e.size.width=e.parentData.width-n,o&&(e.size.height=e.size.width/e.aspectRatio,t=!1)),s+e.size.height>=e.parentData.height&&(e.size.height=e.parentData.height-s,o&&(e.size.width=e.size.height*e.aspectRatio,t=!1)),t||(e.position.left=e.prevPosition.left,e.position.top=e.prevPosition.top,e.size.width=e.prevSize.width,e.size.height=e.prevSize.height)},stop:function(){var t=V(this).resizable("instance"),e=t.options,i=t.containerOffset,s=t.containerPosition,n=t.containerElement,o=V(t.helper),a=o.offset(),r=o.outerWidth()-t.sizeDiff.width,o=o.outerHeight()-t.sizeDiff.height;t._helper&&!e.animate&&/relative/.test(n.css("position"))&&V(this).css({left:a.left-s.left-i.left,width:r,height:o}),t._helper&&!e.animate&&/static/.test(n.css("position"))&&V(this).css({left:a.left-s.left-i.left,width:r,height:o})}}),V.ui.plugin.add("resizable","alsoResize",{start:function(){var t=V(this).resizable("instance").options;V(t.alsoResize).each(function(){var t=V(this);t.data("ui-resizable-alsoresize",{width:parseFloat(t.width()),height:parseFloat(t.height()),left:parseFloat(t.css("left")),top:parseFloat(t.css("top"))})})},resize:function(t,i){var e=V(this).resizable("instance"),s=e.options,n=e.originalSize,o=e.originalPosition,a={height:e.size.height-n.height||0,width:e.size.width-n.width||0,top:e.position.top-o.top||0,left:e.position.left-o.left||0};V(s.alsoResize).each(function(){var t=V(this),s=V(this).data("ui-resizable-alsoresize"),n={},e=t.parents(i.originalElement[0]).length?["width","height"]:["width","height","top","left"];V.each(e,function(t,e){var i=(s[e]||0)+(a[e]||0);i&&0<=i&&(n[e]=i||null)}),t.css(n)})},stop:function(){V(this).removeData("ui-resizable-alsoresize")}}),V.ui.plugin.add("resizable","ghost",{start:function(){var t=V(this).resizable("instance"),e=t.size;t.ghost=t.originalElement.clone(),t.ghost.css({opacity:.25,display:"block",position:"relative",height:e.height,width:e.width,margin:0,left:0,top:0}),t._addClass(t.ghost,"ui-resizable-ghost"),!1!==V.uiBackCompat&&"string"==typeof t.options.ghost&&t.ghost.addClass(this.options.ghost),t.ghost.appendTo(t.helper)},resize:function(){var t=V(this).resizable("instance");t.ghost&&t.ghost.css({position:"relative",height:t.size.height,width:t.size.width})},stop:function(){var t=V(this).resizable("instance");t.ghost&&t.helper&&t.helper.get(0).removeChild(t.ghost.get(0))}}),V.ui.plugin.add("resizable","grid",{resize:function(){var t,e=V(this).resizable("instance"),i=e.options,s=e.size,n=e.originalSize,o=e.originalPosition,a=e.axis,r="number"==typeof i.grid?[i.grid,i.grid]:i.grid,l=r[0]||1,h=r[1]||1,c=Math.round((s.width-n.width)/l)*l,u=Math.round((s.height-n.height)/h)*h,d=n.width+c,p=n.height+u,f=i.maxWidth&&i.maxWidthd,s=i.minHeight&&i.minHeight>p;i.grid=r,m&&(d+=l),s&&(p+=h),f&&(d-=l),g&&(p-=h),/^(se|s|e)$/.test(a)?(e.size.width=d,e.size.height=p):/^(ne)$/.test(a)?(e.size.width=d,e.size.height=p,e.position.top=o.top-u):/^(sw)$/.test(a)?(e.size.width=d,e.size.height=p,e.position.left=o.left-c):((p-h<=0||d-l<=0)&&(t=e._getPaddingPlusBorderDimensions(this)),0"),this._addClass(this.helper,"ui-selectable-helper")},_destroy:function(){this.selectees.removeData("selectable-item"),this._mouseDestroy()},_mouseStart:function(i){var s=this,t=this.options;this.opos=[i.pageX,i.pageY],this.elementPos=V(this.element[0]).offset(),this.options.disabled||(this.selectees=V(t.filter,this.element[0]),this._trigger("start",i),V(t.appendTo).append(this.helper),this.helper.css({left:i.pageX,top:i.pageY,width:0,height:0}),t.autoRefresh&&this.refresh(),this.selectees.filter(".ui-selected").each(function(){var t=V.data(this,"selectable-item");t.startselected=!0,i.metaKey||i.ctrlKey||(s._removeClass(t.$element,"ui-selected"),t.selected=!1,s._addClass(t.$element,"ui-unselecting"),t.unselecting=!0,s._trigger("unselecting",i,{unselecting:t.element}))}),V(i.target).parents().addBack().each(function(){var t,e=V.data(this,"selectable-item");if(e)return t=!i.metaKey&&!i.ctrlKey||!e.$element.hasClass("ui-selected"),s._removeClass(e.$element,t?"ui-unselecting":"ui-selected")._addClass(e.$element,t?"ui-selecting":"ui-unselecting"),e.unselecting=!t,e.selecting=t,(e.selected=t)?s._trigger("selecting",i,{selecting:e.element}):s._trigger("unselecting",i,{unselecting:e.element}),!1}))},_mouseDrag:function(s){if(this.dragged=!0,!this.options.disabled){var t,n=this,o=this.options,a=this.opos[0],r=this.opos[1],l=s.pageX,h=s.pageY;return ll||i.righth||i.bottoma&&i.rightr&&i.bottom *",opacity:!1,placeholder:!1,revert:!1,scroll:!0,scrollSensitivity:20,scrollSpeed:20,scope:"default",tolerance:"intersect",zIndex:1e3,activate:null,beforeStop:null,change:null,deactivate:null,out:null,over:null,receive:null,remove:null,sort:null,start:null,stop:null,update:null},_isOverAxis:function(t,e,i){return e<=t&&t*{ cursor: "+o.cursor+" !important; }").appendTo(n)),o.zIndex&&(this.helper.css("zIndex")&&(this._storedZIndex=this.helper.css("zIndex")),this.helper.css("zIndex",o.zIndex)),o.opacity&&(this.helper.css("opacity")&&(this._storedOpacity=this.helper.css("opacity")),this.helper.css("opacity",o.opacity)),this.scrollParent[0]!==this.document[0]&&"HTML"!==this.scrollParent[0].tagName&&(this.overflowOffset=this.scrollParent.offset()),this._trigger("start",t,this._uiHash()),this._preserveHelperProportions||this._cacheHelperProportions(),!i)for(s=this.containers.length-1;0<=s;s--)this.containers[s]._trigger("activate",t,this._uiHash(this));return V.ui.ddmanager&&(V.ui.ddmanager.current=this),V.ui.ddmanager&&!o.dropBehaviour&&V.ui.ddmanager.prepareOffsets(this,t),this.dragging=!0,this._addClass(this.helper,"ui-sortable-helper"),this.helper.parent().is(this.appendTo)||(this.helper.detach().appendTo(this.appendTo),this.offset.parent=this._getParentOffset()),this.position=this.originalPosition=this._generatePosition(t),this.originalPageX=t.pageX,this.originalPageY=t.pageY,this.lastPositionAbs=this.positionAbs=this._convertPositionTo("absolute"),this._mouseDrag(t),!0},_scroll:function(t){var e=this.options,i=!1;return this.scrollParent[0]!==this.document[0]&&"HTML"!==this.scrollParent[0].tagName?(this.overflowOffset.top+this.scrollParent[0].offsetHeight-t.pageYt[this.floating?"width":"height"]?h&&c:o",i.document[0]);return i._addClass(t,"ui-sortable-placeholder",s||i.currentItem[0].className)._removeClass(t,"ui-sortable-helper"),"tbody"===n?i._createTrPlaceholder(i.currentItem.find("tr").eq(0),V("
    ",i.document[0]).appendTo(t)):"tr"===n?i._createTrPlaceholder(i.currentItem,t):"img"===n&&t.attr("src",i.currentItem.attr("src")),s||t.css("visibility","hidden"),t},update:function(t,e){s&&!o.forcePlaceholderSize||(e.height()&&(!o.forcePlaceholderSize||"tbody"!==n&&"tr"!==n)||e.height(i.currentItem.innerHeight()-parseInt(i.currentItem.css("paddingTop")||0,10)-parseInt(i.currentItem.css("paddingBottom")||0,10)),e.width()||e.width(i.currentItem.innerWidth()-parseInt(i.currentItem.css("paddingLeft")||0,10)-parseInt(i.currentItem.css("paddingRight")||0,10)))}}),i.placeholder=V(o.placeholder.element.call(i.element,i.currentItem)),i.currentItem.after(i.placeholder),o.placeholder.update(i,i.placeholder)},_createTrPlaceholder:function(t,e){var i=this;t.children().each(function(){V("",i.document[0]).attr("colspan",V(this).attr("colspan")||1).appendTo(e)})},_contactContainers:function(t){for(var e,i,s,n,o,a,r,l,h,c=null,u=null,d=this.containers.length-1;0<=d;d--)V.contains(this.currentItem[0],this.containers[d].element[0])||(this._intersectsWith(this.containers[d].containerCache)?c&&V.contains(this.containers[d].element[0],c.element[0])||(c=this.containers[d],u=d):this.containers[d].containerCache.over&&(this.containers[d]._trigger("out",t,this._uiHash(this)),this.containers[d].containerCache.over=0));if(c)if(1===this.containers.length)this.containers[u].containerCache.over||(this.containers[u]._trigger("over",t,this._uiHash(this)),this.containers[u].containerCache.over=1);else{for(i=1e4,s=null,n=(l=c.floating||this._isFloating(this.currentItem))?"left":"top",o=l?"width":"height",h=l?"pageX":"pageY",e=this.items.length-1;0<=e;e--)V.contains(this.containers[u].element[0],this.items[e].item[0])&&this.items[e].item[0]!==this.currentItem[0]&&(a=this.items[e].item.offset()[n],r=!1,t[h]-a>this.items[e][o]/2&&(r=!0),Math.abs(t[h]-a)this.containment[2]&&(i=this.containment[2]+this.offset.click.left),t.pageY-this.offset.click.top>this.containment[3]&&(s=this.containment[3]+this.offset.click.top)),e.grid&&(t=this.originalPageY+Math.round((s-this.originalPageY)/e.grid[1])*e.grid[1],s=!this.containment||t-this.offset.click.top>=this.containment[1]&&t-this.offset.click.top<=this.containment[3]?t:t-this.offset.click.top>=this.containment[1]?t-e.grid[1]:t+e.grid[1],t=this.originalPageX+Math.round((i-this.originalPageX)/e.grid[0])*e.grid[0],i=!this.containment||t-this.offset.click.left>=this.containment[0]&&t-this.offset.click.left<=this.containment[2]?t:t-this.offset.click.left>=this.containment[0]?t-e.grid[0]:t+e.grid[0])),{top:s-this.offset.click.top-this.offset.relative.top-this.offset.parent.top+("fixed"===this.cssPosition?-this.scrollParent.scrollTop():o?0:n.scrollTop()),left:i-this.offset.click.left-this.offset.relative.left-this.offset.parent.left+("fixed"===this.cssPosition?-this.scrollParent.scrollLeft():o?0:n.scrollLeft())}},_rearrange:function(t,e,i,s){i?i[0].appendChild(this.placeholder[0]):e.item[0].parentNode.insertBefore(this.placeholder[0],"down"===this.direction?e.item[0]:e.item[0].nextSibling),this.counter=this.counter?++this.counter:1;var n=this.counter;this._delay(function(){n===this.counter&&this.refreshPositions(!s)})},_clear:function(t,e){this.reverting=!1;var i,s=[];if(!this._noFinalSort&&this.currentItem.parent().length&&this.placeholder.before(this.currentItem),this._noFinalSort=null,this.helper[0]===this.currentItem[0]){for(i in this._storedCSS)"auto"!==this._storedCSS[i]&&"static"!==this._storedCSS[i]||(this._storedCSS[i]="");this.currentItem.css(this._storedCSS),this._removeClass(this.currentItem,"ui-sortable-helper")}else this.currentItem.show();function n(e,i,s){return function(t){s._trigger(e,t,i._uiHash(i))}}for(this.fromOutside&&!e&&s.push(function(t){this._trigger("receive",t,this._uiHash(this.fromOutside))}),!this.fromOutside&&this.domPosition.prev===this.currentItem.prev().not(".ui-sortable-helper")[0]&&this.domPosition.parent===this.currentItem.parent()[0]||e||s.push(function(t){this._trigger("update",t,this._uiHash())}),this!==this.currentContainer&&(e||(s.push(function(t){this._trigger("remove",t,this._uiHash())}),s.push(function(e){return function(t){e._trigger("receive",t,this._uiHash(this))}}.call(this,this.currentContainer)),s.push(function(e){return function(t){e._trigger("update",t,this._uiHash(this))}}.call(this,this.currentContainer)))),i=this.containers.length-1;0<=i;i--)e||s.push(n("deactivate",this,this.containers[i])),this.containers[i].containerCache.over&&(s.push(n("out",this,this.containers[i])),this.containers[i].containerCache.over=0);if(this.storedCursor&&(this.document.find("body").css("cursor",this.storedCursor),this.storedStylesheet.remove()),this._storedOpacity&&this.helper.css("opacity",this._storedOpacity),this._storedZIndex&&this.helper.css("zIndex","auto"===this._storedZIndex?"":this._storedZIndex),this.dragging=!1,e||this._trigger("beforeStop",t,this._uiHash()),this.placeholder[0].parentNode.removeChild(this.placeholder[0]),this.cancelHelperRemoval||(this.helper[0]!==this.currentItem[0]&&this.helper.remove(),this.helper=null),!e){for(i=0;i li > :first-child").add(t.find("> :not(li)").even())},heightStyle:"auto",icons:{activeHeader:"ui-icon-triangle-1-s",header:"ui-icon-triangle-1-e"},activate:null,beforeActivate:null},hideProps:{borderTopWidth:"hide",borderBottomWidth:"hide",paddingTop:"hide",paddingBottom:"hide",height:"hide"},showProps:{borderTopWidth:"show",borderBottomWidth:"show",paddingTop:"show",paddingBottom:"show",height:"show"},_create:function(){var t=this.options;this.prevShow=this.prevHide=V(),this._addClass("ui-accordion","ui-widget ui-helper-reset"),this.element.attr("role","tablist"),t.collapsible||!1!==t.active&&null!=t.active||(t.active=0),this._processPanels(),t.active<0&&(t.active+=this.headers.length),this._refresh()},_getCreateEventData:function(){return{header:this.active,panel:this.active.length?this.active.next():V()}},_createIcons:function(){var t,e=this.options.icons;e&&(t=V(""),this._addClass(t,"ui-accordion-header-icon","ui-icon "+e.header),t.prependTo(this.headers),t=this.active.children(".ui-accordion-header-icon"),this._removeClass(t,e.header)._addClass(t,null,e.activeHeader)._addClass(this.headers,"ui-accordion-icons"))},_destroyIcons:function(){this._removeClass(this.headers,"ui-accordion-icons"),this.headers.children(".ui-accordion-header-icon").remove()},_destroy:function(){var t;this.element.removeAttr("role"),this.headers.removeAttr("role aria-expanded aria-selected aria-controls tabIndex").removeUniqueId(),this._destroyIcons(),t=this.headers.next().css("display","").removeAttr("role aria-hidden aria-labelledby").removeUniqueId(),"content"!==this.options.heightStyle&&t.css("height","")},_setOption:function(t,e){"active"!==t?("event"===t&&(this.options.event&&this._off(this.headers,this.options.event),this._setupEvents(e)),this._super(t,e),"collapsible"!==t||e||!1!==this.options.active||this._activate(0),"icons"===t&&(this._destroyIcons(),e&&this._createIcons())):this._activate(e)},_setOptionDisabled:function(t){this._super(t),this.element.attr("aria-disabled",t),this._toggleClass(null,"ui-state-disabled",!!t),this._toggleClass(this.headers.add(this.headers.next()),null,"ui-state-disabled",!!t)},_keydown:function(t){if(!t.altKey&&!t.ctrlKey){var e=V.ui.keyCode,i=this.headers.length,s=this.headers.index(t.target),n=!1;switch(t.keyCode){case e.RIGHT:case e.DOWN:n=this.headers[(s+1)%i];break;case e.LEFT:case e.UP:n=this.headers[(s-1+i)%i];break;case e.SPACE:case e.ENTER:this._eventHandler(t);break;case e.HOME:n=this.headers[0];break;case e.END:n=this.headers[i-1]}n&&(V(t.target).attr("tabIndex",-1),V(n).attr("tabIndex",0),V(n).trigger("focus"),t.preventDefault())}},_panelKeyDown:function(t){t.keyCode===V.ui.keyCode.UP&&t.ctrlKey&&V(t.currentTarget).prev().trigger("focus")},refresh:function(){var t=this.options;this._processPanels(),!1===t.active&&!0===t.collapsible||!this.headers.length?(t.active=!1,this.active=V()):!1===t.active?this._activate(0):this.active.length&&!V.contains(this.element[0],this.active[0])?this.headers.length===this.headers.find(".ui-state-disabled").length?(t.active=!1,this.active=V()):this._activate(Math.max(0,t.active-1)):t.active=this.headers.index(this.active),this._destroyIcons(),this._refresh()},_processPanels:function(){var t=this.headers,e=this.panels;"function"==typeof this.options.header?this.headers=this.options.header(this.element):this.headers=this.element.find(this.options.header),this._addClass(this.headers,"ui-accordion-header ui-accordion-header-collapsed","ui-state-default"),this.panels=this.headers.next().filter(":not(.ui-accordion-content-active)").hide(),this._addClass(this.panels,"ui-accordion-content","ui-helper-reset ui-widget-content"),e&&(this._off(t.not(this.headers)),this._off(e.not(this.panels)))},_refresh:function(){var i,t=this.options,e=t.heightStyle,s=this.element.parent();this.active=this._findActive(t.active),this._addClass(this.active,"ui-accordion-header-active","ui-state-active")._removeClass(this.active,"ui-accordion-header-collapsed"),this._addClass(this.active.next(),"ui-accordion-content-active"),this.active.next().show(),this.headers.attr("role","tab").each(function(){var t=V(this),e=t.uniqueId().attr("id"),i=t.next(),s=i.uniqueId().attr("id");t.attr("aria-controls",s),i.attr("aria-labelledby",e)}).next().attr("role","tabpanel"),this.headers.not(this.active).attr({"aria-selected":"false","aria-expanded":"false",tabIndex:-1}).next().attr({"aria-hidden":"true"}).hide(),this.active.length?this.active.attr({"aria-selected":"true","aria-expanded":"true",tabIndex:0}).next().attr({"aria-hidden":"false"}):this.headers.eq(0).attr("tabIndex",0),this._createIcons(),this._setupEvents(t.event),"fill"===e?(i=s.height(),this.element.siblings(":visible").each(function(){var t=V(this),e=t.css("position");"absolute"!==e&&"fixed"!==e&&(i-=t.outerHeight(!0))}),this.headers.each(function(){i-=V(this).outerHeight(!0)}),this.headers.next().each(function(){V(this).height(Math.max(0,i-V(this).innerHeight()+V(this).height()))}).css("overflow","auto")):"auto"===e&&(i=0,this.headers.next().each(function(){var t=V(this).is(":visible");t||V(this).show(),i=Math.max(i,V(this).css("height","").height()),t||V(this).hide()}).height(i))},_activate:function(t){t=this._findActive(t)[0];t!==this.active[0]&&(t=t||this.active[0],this._eventHandler({target:t,currentTarget:t,preventDefault:V.noop}))},_findActive:function(t){return"number"==typeof t?this.headers.eq(t):V()},_setupEvents:function(t){var i={keydown:"_keydown"};t&&V.each(t.split(" "),function(t,e){i[e]="_eventHandler"}),this._off(this.headers.add(this.headers.next())),this._on(this.headers,i),this._on(this.headers.next(),{keydown:"_panelKeyDown"}),this._hoverable(this.headers),this._focusable(this.headers)},_eventHandler:function(t){var e=this.options,i=this.active,s=V(t.currentTarget),n=s[0]===i[0],o=n&&e.collapsible,a=o?V():s.next(),r=i.next(),a={oldHeader:i,oldPanel:r,newHeader:o?V():s,newPanel:a};t.preventDefault(),n&&!e.collapsible||!1===this._trigger("beforeActivate",t,a)||(e.active=!o&&this.headers.index(s),this.active=n?V():s,this._toggle(a),this._removeClass(i,"ui-accordion-header-active","ui-state-active"),e.icons&&(i=i.children(".ui-accordion-header-icon"),this._removeClass(i,null,e.icons.activeHeader)._addClass(i,null,e.icons.header)),n||(this._removeClass(s,"ui-accordion-header-collapsed")._addClass(s,"ui-accordion-header-active","ui-state-active"),e.icons&&(n=s.children(".ui-accordion-header-icon"),this._removeClass(n,null,e.icons.header)._addClass(n,null,e.icons.activeHeader)),this._addClass(s.next(),"ui-accordion-content-active")))},_toggle:function(t){var e=t.newPanel,i=this.prevShow.length?this.prevShow:t.oldPanel;this.prevShow.add(this.prevHide).stop(!0,!0),this.prevShow=e,this.prevHide=i,this.options.animate?this._animate(e,i,t):(i.hide(),e.show(),this._toggleComplete(t)),i.attr({"aria-hidden":"true"}),i.prev().attr({"aria-selected":"false","aria-expanded":"false"}),e.length&&i.length?i.prev().attr({tabIndex:-1,"aria-expanded":"false"}):e.length&&this.headers.filter(function(){return 0===parseInt(V(this).attr("tabIndex"),10)}).attr("tabIndex",-1),e.attr("aria-hidden","false").prev().attr({"aria-selected":"true","aria-expanded":"true",tabIndex:0})},_animate:function(t,i,e){var s,n,o,a=this,r=0,l=t.css("box-sizing"),h=t.length&&(!i.length||t.index()",delay:300,options:{icons:{submenu:"ui-icon-caret-1-e"},items:"> *",menus:"ul",position:{my:"left top",at:"right top"},role:"menu",blur:null,focus:null,select:null},_create:function(){this.activeMenu=this.element,this.mouseHandled=!1,this.lastMousePosition={x:null,y:null},this.element.uniqueId().attr({role:this.options.role,tabIndex:0}),this._addClass("ui-menu","ui-widget ui-widget-content"),this._on({"mousedown .ui-menu-item":function(t){t.preventDefault(),this._activateItem(t)},"click .ui-menu-item":function(t){var e=V(t.target),i=V(V.ui.safeActiveElement(this.document[0]));!this.mouseHandled&&e.not(".ui-state-disabled").length&&(this.select(t),t.isPropagationStopped()||(this.mouseHandled=!0),e.has(".ui-menu").length?this.expand(t):!this.element.is(":focus")&&i.closest(".ui-menu").length&&(this.element.trigger("focus",[!0]),this.active&&1===this.active.parents(".ui-menu").length&&clearTimeout(this.timer)))},"mouseenter .ui-menu-item":"_activateItem","mousemove .ui-menu-item":"_activateItem",mouseleave:"collapseAll","mouseleave .ui-menu":"collapseAll",focus:function(t,e){var i=this.active||this._menuItems().first();e||this.focus(t,i)},blur:function(t){this._delay(function(){V.contains(this.element[0],V.ui.safeActiveElement(this.document[0]))||this.collapseAll(t)})},keydown:"_keydown"}),this.refresh(),this._on(this.document,{click:function(t){this._closeOnDocumentClick(t)&&this.collapseAll(t,!0),this.mouseHandled=!1}})},_activateItem:function(t){var e,i;this.previousFilter||t.clientX===this.lastMousePosition.x&&t.clientY===this.lastMousePosition.y||(this.lastMousePosition={x:t.clientX,y:t.clientY},e=V(t.target).closest(".ui-menu-item"),i=V(t.currentTarget),e[0]===i[0]&&(i.is(".ui-state-active")||(this._removeClass(i.siblings().children(".ui-state-active"),null,"ui-state-active"),this.focus(t,i))))},_destroy:function(){var t=this.element.find(".ui-menu-item").removeAttr("role aria-disabled").children(".ui-menu-item-wrapper").removeUniqueId().removeAttr("tabIndex role aria-haspopup");this.element.removeAttr("aria-activedescendant").find(".ui-menu").addBack().removeAttr("role aria-labelledby aria-expanded aria-hidden aria-disabled tabIndex").removeUniqueId().show(),t.children().each(function(){var t=V(this);t.data("ui-menu-submenu-caret")&&t.remove()})},_keydown:function(t){var e,i,s,n=!0;switch(t.keyCode){case V.ui.keyCode.PAGE_UP:this.previousPage(t);break;case V.ui.keyCode.PAGE_DOWN:this.nextPage(t);break;case V.ui.keyCode.HOME:this._move("first","first",t);break;case V.ui.keyCode.END:this._move("last","last",t);break;case V.ui.keyCode.UP:this.previous(t);break;case V.ui.keyCode.DOWN:this.next(t);break;case V.ui.keyCode.LEFT:this.collapse(t);break;case V.ui.keyCode.RIGHT:this.active&&!this.active.is(".ui-state-disabled")&&this.expand(t);break;case V.ui.keyCode.ENTER:case V.ui.keyCode.SPACE:this._activate(t);break;case V.ui.keyCode.ESCAPE:this.collapse(t);break;default:e=this.previousFilter||"",s=n=!1,i=96<=t.keyCode&&t.keyCode<=105?(t.keyCode-96).toString():String.fromCharCode(t.keyCode),clearTimeout(this.filterTimer),i===e?s=!0:i=e+i,e=this._filterMenuItems(i),(e=s&&-1!==e.index(this.active.next())?this.active.nextAll(".ui-menu-item"):e).length||(i=String.fromCharCode(t.keyCode),e=this._filterMenuItems(i)),e.length?(this.focus(t,e),this.previousFilter=i,this.filterTimer=this._delay(function(){delete this.previousFilter},1e3)):delete this.previousFilter}n&&t.preventDefault()},_activate:function(t){this.active&&!this.active.is(".ui-state-disabled")&&(this.active.children("[aria-haspopup='true']").length?this.expand(t):this.select(t))},refresh:function(){var t,e,s=this,n=this.options.icons.submenu,i=this.element.find(this.options.menus);this._toggleClass("ui-menu-icons",null,!!this.element.find(".ui-icon").length),e=i.filter(":not(.ui-menu)").hide().attr({role:this.options.role,"aria-hidden":"true","aria-expanded":"false"}).each(function(){var t=V(this),e=t.prev(),i=V("").data("ui-menu-submenu-caret",!0);s._addClass(i,"ui-menu-icon","ui-icon "+n),e.attr("aria-haspopup","true").prepend(i),t.attr("aria-labelledby",e.attr("id"))}),this._addClass(e,"ui-menu","ui-widget ui-widget-content ui-front"),(t=i.add(this.element).find(this.options.items)).not(".ui-menu-item").each(function(){var t=V(this);s._isDivider(t)&&s._addClass(t,"ui-menu-divider","ui-widget-content")}),i=(e=t.not(".ui-menu-item, .ui-menu-divider")).children().not(".ui-menu").uniqueId().attr({tabIndex:-1,role:this._itemRole()}),this._addClass(e,"ui-menu-item")._addClass(i,"ui-menu-item-wrapper"),t.filter(".ui-state-disabled").attr("aria-disabled","true"),this.active&&!V.contains(this.element[0],this.active[0])&&this.blur()},_itemRole:function(){return{menu:"menuitem",listbox:"option"}[this.options.role]},_setOption:function(t,e){var i;"icons"===t&&(i=this.element.find(".ui-menu-icon"),this._removeClass(i,null,this.options.icons.submenu)._addClass(i,null,e.submenu)),this._super(t,e)},_setOptionDisabled:function(t){this._super(t),this.element.attr("aria-disabled",String(t)),this._toggleClass(null,"ui-state-disabled",!!t)},focus:function(t,e){var i;this.blur(t,t&&"focus"===t.type),this._scrollIntoView(e),this.active=e.first(),i=this.active.children(".ui-menu-item-wrapper"),this._addClass(i,null,"ui-state-active"),this.options.role&&this.element.attr("aria-activedescendant",i.attr("id")),i=this.active.parent().closest(".ui-menu-item").children(".ui-menu-item-wrapper"),this._addClass(i,null,"ui-state-active"),t&&"keydown"===t.type?this._close():this.timer=this._delay(function(){this._close()},this.delay),(i=e.children(".ui-menu")).length&&t&&/^mouse/.test(t.type)&&this._startOpening(i),this.activeMenu=e.parent(),this._trigger("focus",t,{item:e})},_scrollIntoView:function(t){var e,i,s;this._hasScroll()&&(i=parseFloat(V.css(this.activeMenu[0],"borderTopWidth"))||0,s=parseFloat(V.css(this.activeMenu[0],"paddingTop"))||0,e=t.offset().top-this.activeMenu.offset().top-i-s,i=this.activeMenu.scrollTop(),s=this.activeMenu.height(),t=t.outerHeight(),e<0?this.activeMenu.scrollTop(i+e):s",options:{appendTo:null,autoFocus:!1,delay:300,minLength:1,position:{my:"left top",at:"left bottom",collision:"none"},source:null,change:null,close:null,focus:null,open:null,response:null,search:null,select:null},requestIndex:0,pending:0,liveRegionTimer:null,_create:function(){var i,s,n,t=this.element[0].nodeName.toLowerCase(),e="textarea"===t,t="input"===t;this.isMultiLine=e||!t&&this._isContentEditable(this.element),this.valueMethod=this.element[e||t?"val":"text"],this.isNewMenu=!0,this._addClass("ui-autocomplete-input"),this.element.attr("autocomplete","off"),this._on(this.element,{keydown:function(t){if(this.element.prop("readOnly"))s=n=i=!0;else{s=n=i=!1;var e=V.ui.keyCode;switch(t.keyCode){case e.PAGE_UP:i=!0,this._move("previousPage",t);break;case e.PAGE_DOWN:i=!0,this._move("nextPage",t);break;case e.UP:i=!0,this._keyEvent("previous",t);break;case e.DOWN:i=!0,this._keyEvent("next",t);break;case e.ENTER:this.menu.active&&(i=!0,t.preventDefault(),this.menu.select(t));break;case e.TAB:this.menu.active&&this.menu.select(t);break;case e.ESCAPE:this.menu.element.is(":visible")&&(this.isMultiLine||this._value(this.term),this.close(t),t.preventDefault());break;default:s=!0,this._searchTimeout(t)}}},keypress:function(t){if(i)return i=!1,void(this.isMultiLine&&!this.menu.element.is(":visible")||t.preventDefault());if(!s){var e=V.ui.keyCode;switch(t.keyCode){case e.PAGE_UP:this._move("previousPage",t);break;case e.PAGE_DOWN:this._move("nextPage",t);break;case e.UP:this._keyEvent("previous",t);break;case e.DOWN:this._keyEvent("next",t)}}},input:function(t){if(n)return n=!1,void t.preventDefault();this._searchTimeout(t)},focus:function(){this.selectedItem=null,this.previous=this._value()},blur:function(t){clearTimeout(this.searching),this.close(t),this._change(t)}}),this._initSource(),this.menu=V("
    Metrics2
    BeforeAfterBeforeAfter
    diff --git a/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/metrics2/sink/PrometheusMetricsSink.java b/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/metrics2/sink/PrometheusMetricsSink.java
    index a59ad5f227be5..9024203700ee1 100644
    --- a/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/metrics2/sink/PrometheusMetricsSink.java
    +++ b/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/metrics2/sink/PrometheusMetricsSink.java
    @@ -84,6 +84,10 @@ public void putMetrics(MetricsRecord metricsRecord) {
       /**
        * Convert CamelCase based names to lower-case names where the separator
        * is the underscore, to follow prometheus naming conventions.
    +   *
    +   * @param metricName metricName.
    +   * @param recordName recordName.
    +   * @return prometheusName.
        */
       public String prometheusName(String recordName,
                                    String metricName) {
    diff --git a/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/metrics2/sink/ganglia/AbstractGangliaSink.java b/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/metrics2/sink/ganglia/AbstractGangliaSink.java
    index 804e90330fba3..d3d794fa74a91 100644
    --- a/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/metrics2/sink/ganglia/AbstractGangliaSink.java
    +++ b/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/metrics2/sink/ganglia/AbstractGangliaSink.java
    @@ -212,7 +212,7 @@ private void loadGangliaConf(GangliaConfType gtype) {
       /**
        * Lookup GangliaConf from cache. If not found, return default values
        *
    -   * @param metricName
    +   * @param metricName metricName.
        * @return looked up GangliaConf
        */
       protected GangliaConf getGangliaConfForMetric(String metricName) {
    @@ -253,6 +253,7 @@ private void pad() {
     
       /**
        * Puts an integer into the buffer as 4 bytes, big-endian.
    +   * @param i i.
        */
       protected void xdr_int(int i) {
         buffer[offset++] = (byte) ((i >> 24) & 0xff);
    @@ -263,7 +264,7 @@ protected void xdr_int(int i) {
     
       /**
        * Sends Ganglia Metrics to the configured hosts
    -   * @throws IOException
    +   * @throws IOException raised on errors performing I/O.
        */
       protected void emitToGangliaHosts() throws IOException {
         try {
    diff --git a/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/metrics2/sink/ganglia/GangliaSink30.java b/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/metrics2/sink/ganglia/GangliaSink30.java
    index 3e8314ee884d8..196824f433c81 100644
    --- a/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/metrics2/sink/ganglia/GangliaSink30.java
    +++ b/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/metrics2/sink/ganglia/GangliaSink30.java
    @@ -216,7 +216,7 @@ private GangliaSlope calculateSlope(GangliaConf gConf,
        * @param value The value of the metric
        * @param gConf The GangliaConf for this metric
        * @param gSlope The slope for this metric
    -   * @throws IOException
    +   * @throws IOException raised on errors performing I/O.
        */
       protected void emitMetric(String groupName, String name, String type,
           String value, GangliaConf gConf, GangliaSlope gSlope) throws IOException {
    diff --git a/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/metrics2/sink/ganglia/GangliaSink31.java b/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/metrics2/sink/ganglia/GangliaSink31.java
    index 5aebff8c031a9..fae0d4e85e1ec 100644
    --- a/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/metrics2/sink/ganglia/GangliaSink31.java
    +++ b/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/metrics2/sink/ganglia/GangliaSink31.java
    @@ -42,7 +42,7 @@ public class GangliaSink31 extends GangliaSink30 {
        * @param value The value of the metric
        * @param gConf The GangliaConf for this metric
        * @param gSlope The slope for this metric
    -   * @throws IOException
    +   * @throws IOException raised on errors performing I/O.
        */
       @Override
       protected void emitMetric(String groupName, String name, String type,
    diff --git a/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/metrics2/util/MBeans.java b/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/metrics2/util/MBeans.java
    index 20b1cd6051961..58081449ee74f 100644
    --- a/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/metrics2/util/MBeans.java
    +++ b/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/metrics2/util/MBeans.java
    @@ -63,8 +63,8 @@ private MBeans() {
        * Where the {@literal  and } are the supplied
        * parameters.
        *
    -   * @param serviceName
    -   * @param nameName
    +   * @param serviceName serviceName.
    +   * @param nameName nameName.
        * @param theMbean - the MBean to register
        * @return the named used to register the MBean
        */
    @@ -79,8 +79,8 @@ static public ObjectName register(String serviceName, String nameName,
        * Where the {@literal  and } are the supplied
        * parameters.
        *
    -   * @param serviceName
    -   * @param nameName
    +   * @param serviceName serviceName.
    +   * @param nameName nameName.
        * @param properties - Key value pairs to define additional JMX ObjectName
        *                     properties.
        * @param theMbean    - the MBean to register
    diff --git a/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/metrics2/util/SampleQuantiles.java b/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/metrics2/util/SampleQuantiles.java
    index 737ccc0d788dd..46a9d35f9d242 100644
    --- a/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/metrics2/util/SampleQuantiles.java
    +++ b/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/metrics2/util/SampleQuantiles.java
    @@ -108,7 +108,7 @@ private double allowableError(int rank) {
       /**
        * Add a new value from the stream.
        * 
    -   * @param v
    +   * @param v v.
        */
       synchronized public void insert(long v) {
         buffer[bufferCount] = v;
    diff --git a/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/net/AbstractDNSToSwitchMapping.java b/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/net/AbstractDNSToSwitchMapping.java
    index f050219398721..5a13b00098a44 100644
    --- a/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/net/AbstractDNSToSwitchMapping.java
    +++ b/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/net/AbstractDNSToSwitchMapping.java
    @@ -115,7 +115,7 @@ public String dumpTopology() {
         builder.append("Mapping: ").append(toString()).append("\n");
         if (rack != null) {
           builder.append("Map:\n");
    -      Set switches = new HashSet();
    +      Set switches = new HashSet<>();
           for (Map.Entry entry : rack.entrySet()) {
             builder.append("  ")
                 .append(entry.getKey())
    diff --git a/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/net/DNS.java b/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/net/DNS.java
    index 83be2f1579f53..53bb44fb2e9ef 100644
    --- a/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/net/DNS.java
    +++ b/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/net/DNS.java
    @@ -142,8 +142,12 @@ private static LinkedHashSet getSubinterfaceInetAddrs(
       }
     
       /**
    -   * Like {@link DNS#getIPs(String, boolean)}, but returns all
    +   * @return Like {@link DNS#getIPs(String, boolean)}, but returns all
        * IPs associated with the given interface and its subinterfaces.
    +   *
    +   * @param strInterface input strInterface.
    +   * @throws UnknownHostException
    +   * If no IP address for the local host could be found.
        */
       public static String[] getIPs(String strInterface)
           throws UnknownHostException {
    @@ -346,6 +350,8 @@ public static String[] getHosts(String strInterface)
        *            The name of the network interface to query (e.g. eth0)
        * @param nameserver
        *            The DNS host name
    +   * @param tryfallbackResolution
    +   *            Input tryfallbackResolution.
        * @return The default host names associated with IPs bound to the network
        *         interface
        * @throws UnknownHostException
    @@ -385,7 +391,7 @@ public static String getDefaultHost(@Nullable String strInterface)
       }
     
       /**
    -   * Returns the default (first) host name associated by the provided
    +   * @return Returns the default (first) host name associated by the provided
        * nameserver with the address bound to the specified network interface.
        *
        * @param strInterface
    diff --git a/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/net/DNSToSwitchMapping.java b/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/net/DNSToSwitchMapping.java
    index 1e6f5f500849f..d29c6e3077df5 100644
    --- a/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/net/DNSToSwitchMapping.java
    +++ b/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/net/DNSToSwitchMapping.java
    @@ -65,6 +65,8 @@ public interface DNSToSwitchMapping {
        *
        * If there is a cache on these nodes, this method will clear it, so that 
        * future accesses will see updated data.
    +   *
    +   * @param names input names.
        */
       public void reloadCachedMappings(List names);
     }
    diff --git a/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/net/DomainNameResolver.java b/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/net/DomainNameResolver.java
    index 4c44e9da4c063..debfe2feaa8b7 100644
    --- a/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/net/DomainNameResolver.java
    +++ b/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/net/DomainNameResolver.java
    @@ -30,9 +30,10 @@ public interface DomainNameResolver {
        * Takes one domain name and returns its IP addresses based on the actual
        * service discovery methods.
        *
    -   * @param domainName
    +   * @param domainName input domainName.
        * @return all IP addresses
    -   * @throws UnknownHostException
    +   * @throws UnknownHostException indicate that the IP address of a
    +   * host could not be determined.
        */
       InetAddress[] getAllByDomainName(String domainName)
           throws UnknownHostException;
    @@ -40,7 +41,7 @@ InetAddress[] getAllByDomainName(String domainName)
       /**
        * Reverse lookup an IP address and get the fully qualified domain name(fqdn).
        *
    -   * @param address
    +   * @param address input address.
        * @return fully qualified domain name
        */
       String getHostnameByIP(InetAddress address);
    @@ -52,10 +53,12 @@ InetAddress[] getAllByDomainName(String domainName)
        * This function is necessary in secure environment since Kerberos uses fqdn
        * in the service principal instead of IP.
        *
    -   * @param domainName
    +   * @param domainName input domainName.
    +   * @param useFQDN input useFQDN.
        * @return all fully qualified domain names belonging to the IPs resolved from
        * the input domainName
    -   * @throws UnknownHostException
    +   * @throws UnknownHostException indicate that the IP address of a
    +   * host could not be determined.
        */
        String[] getAllResolvedHostnameByDomainName(
            String domainName, boolean useFQDN) throws UnknownHostException;
    diff --git a/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/net/InnerNode.java b/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/net/InnerNode.java
    index efd1cc07d44bd..df4a01af27bc9 100644
    --- a/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/net/InnerNode.java
    +++ b/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/net/InnerNode.java
    @@ -27,7 +27,10 @@
     @InterfaceStability.Unstable
     public interface InnerNode extends Node {
       interface Factory {
    -    /** Construct an InnerNode from a path-like string */
    +    /**
    +     * @return Construct an InnerNode from a path-like string.
    +     * @param path input path.
    +     */
         N newInnerNode(String path);
       }
     
    diff --git a/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/net/InnerNodeImpl.java b/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/net/InnerNodeImpl.java
    index 923515b6efe7e..1dd3105080778 100644
    --- a/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/net/InnerNodeImpl.java
    +++ b/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/net/InnerNodeImpl.java
    @@ -41,13 +41,22 @@ public InnerNodeImpl newInnerNode(String path) {
       protected final Map childrenMap = new HashMap<>();
       protected int numOfLeaves;
     
    -  /** Construct an InnerNode from a path-like string. */
    +  /**
    +   * Construct an InnerNode from a path-like string.
    +   * @param path input path.
    +   */
       protected InnerNodeImpl(String path) {
         super(path);
       }
     
    -  /** Construct an InnerNode
    -   * from its name, its network location, its parent, and its level. */
    +  /**
    +   * Construct an InnerNode
    +   * from its name, its network location, its parent, and its level.
    +   * @param name input name.
    +   * @param location input location.
    +   * @param parent input parent.
    +   * @param level input level.
    +   */
       protected InnerNodeImpl(String name, String location,
           InnerNode parent, int level) {
         super(name, location, parent, level);
    diff --git a/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/net/NetUtils.java b/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/net/NetUtils.java
    index fead87d7907d7..c49706d66f27d 100644
    --- a/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/net/NetUtils.java
    +++ b/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/net/NetUtils.java
    @@ -133,7 +133,8 @@ public static SocketFactory getDefaultSocketFactory(Configuration conf) {
        * Get the socket factory corresponding to the given proxy URI. If the
        * given proxy URI corresponds to an absence of configuration parameter,
        * returns null. If the URI is malformed raises an exception.
    -   * 
    +   *
    +   * @param conf configuration.
        * @param propValue the property which is the class name of the
        *        SocketFactory to instantiate; assumed non null and non empty.
        * @return a socket factory as defined in the property value.
    @@ -151,19 +152,26 @@ public static SocketFactory getSocketFactoryFromProperty(
       }
     
       /**
    -   * Util method to build socket addr from either:
    +   * Util method to build socket addr from either.
        *   {@literal :}
        *   {@literal ://:/}
    +   *
    +   * @param target target.
    +   * @return socket addr.
        */
       public static InetSocketAddress createSocketAddr(String target) {
         return createSocketAddr(target, -1);
       }
     
       /**
    -   * Util method to build socket addr from either:
    +   * Util method to build socket addr from either.
        *   {@literal }
        *   {@literal :}
        *   {@literal ://:/}
    +   *
    +   * @param target target.
    +   * @param defaultPort default port.
    +   * @return socket addr.
        */
       public static InetSocketAddress createSocketAddr(String target,
                                                        int defaultPort) {
    @@ -183,6 +191,7 @@ public static InetSocketAddress createSocketAddr(String target,
        * @param configName the name of the configuration from which
        *                   target was loaded. This is used in the
        *                   exception message in the case that parsing fails.
    +   * @return socket addr.
        */
       public static InetSocketAddress createSocketAddr(String target,
                                                        int defaultPort,
    @@ -204,6 +213,7 @@ public static InetSocketAddress createSocketAddr(String target,
        *                   target was loaded. This is used in the
        *                   exception message in the case that parsing fails.
        * @param useCacheIfPresent Whether use cache when create URI
    +   * @return  socket addr
        */
       public static InetSocketAddress createSocketAddr(String target,
                                                        int defaultPort,
    @@ -361,8 +371,8 @@ private static String canonicalizeHost(String host) {
        * daemons, one can set up mappings from those hostnames to "localhost".
        * {@link NetUtils#getStaticResolution(String)} can be used to query for
        * the actual hostname. 
    -   * @param host
    -   * @param resolvedName
    +   * @param host the hostname or IP use to instantiate the object.
    +   * @param resolvedName resolved name.
        */
       public static void addStaticResolution(String host, String resolvedName) {
         synchronized (hostToResolved) {
    @@ -374,7 +384,7 @@ public static void addStaticResolution(String host, String resolvedName) {
        * Retrieves the resolved name for the passed host. The resolved name must
        * have been set earlier using 
        * {@link NetUtils#addStaticResolution(String, String)}
    -   * @param host
    +   * @param host the hostname or IP use to instantiate the object.
        * @return the resolution
        */
       public static String getStaticResolution(String host) {
    @@ -410,7 +420,7 @@ public static List  getAllStaticResolutions() {
        * the server binds to "0.0.0.0". This returns "hostname:port" of the server,
        * or "127.0.0.1:port" when the getListenerAddress() returns "0.0.0.0:port".
        * 
    -   * @param server
    +   * @param server server.
        * @return socket address that a client can use to connect to the server.
        */
       public static InetSocketAddress getConnectAddress(Server server) {
    @@ -438,8 +448,10 @@ public static InetSocketAddress getConnectAddress(InetSocketAddress addr) {
       
       /**
        * Same as getInputStream(socket, socket.getSoTimeout()).
    -   * 

    - * + * + * @param socket socket. + * @throws IOException raised on errors performing I/O. + * @return SocketInputWrapper for reading from the socket. * @see #getInputStream(Socket, long) */ public static SocketInputWrapper getInputStream(Socket socket) @@ -462,11 +474,11 @@ public static SocketInputWrapper getInputStream(Socket socket) * * @see Socket#getChannel() * - * @param socket + * @param socket socket. * @param timeout timeout in milliseconds. zero for waiting as * long as necessary. * @return SocketInputWrapper for reading from the socket. - * @throws IOException + * @throws IOException raised on errors performing I/O. */ public static SocketInputWrapper getInputStream(Socket socket, long timeout) throws IOException { @@ -494,9 +506,9 @@ public static SocketInputWrapper getInputStream(Socket socket, long timeout) * * @see #getOutputStream(Socket, long) * - * @param socket + * @param socket socket. * @return OutputStream for writing to the socket. - * @throws IOException + * @throws IOException raised on errors performing I/O. */ public static OutputStream getOutputStream(Socket socket) throws IOException { @@ -516,11 +528,11 @@ public static OutputStream getOutputStream(Socket socket) * * @see Socket#getChannel() * - * @param socket + * @param socket socket. * @param timeout timeout in milliseconds. This may not always apply. zero * for waiting as long as necessary. * @return OutputStream for writing to the socket. - * @throws IOException + * @throws IOException raised on errors performing I/O. */ public static OutputStream getOutputStream(Socket socket, long timeout) throws IOException { @@ -541,9 +553,10 @@ public static OutputStream getOutputStream(Socket socket, long timeout) * * @see java.net.Socket#connect(java.net.SocketAddress, int) * - * @param socket + * @param socket socket. * @param address the remote address * @param timeout timeout in milliseconds + * @throws IOException raised on errors performing I/O. */ public static void connect(Socket socket, SocketAddress address, @@ -555,10 +568,11 @@ public static void connect(Socket socket, * Like {@link NetUtils#connect(Socket, SocketAddress, int)} but * also takes a local address and port to bind the socket to. * - * @param socket + * @param socket socket. * @param endpoint the remote address * @param localAddr the local address to bind the socket to * @param timeout timeout in milliseconds + * @throws IOException raised on errors performing I/O. */ public static void connect(Socket socket, SocketAddress endpoint, @@ -644,7 +658,7 @@ public static List normalizeHostNames(Collection names) { * Performs a sanity check on the list of hostnames/IPs to verify they at least * appear to be valid. * @param names - List of hostnames/IPs - * @throws UnknownHostException + * @throws UnknownHostException Unknown Host Exception. */ public static void verifyHostnames(String[] names) throws UnknownHostException { for (String name: names) { @@ -735,6 +749,9 @@ public static String getHostname() { /** * Compose a "host:port" string from the address. + * + * @param addr address. + * @return hort port string. */ public static String getHostPortString(InetSocketAddress addr) { return addr.getHostName() + ":" + addr.getPort(); @@ -969,6 +986,8 @@ private static String quoteHost(final String hostname) { } /** + * isValidSubnet. + * @param subnet subnet. * @return true if the given string is a subnet specified * using CIDR notation, false otherwise */ @@ -1004,6 +1023,7 @@ private static void addMatchingAddrs(NetworkInterface nif, * @param returnSubinterfaces * whether to return IPs associated with subinterfaces * @throws IllegalArgumentException if subnet is invalid + * @return ips. */ public static List getIPs(String subnet, boolean returnSubinterfaces) { @@ -1083,8 +1103,8 @@ public static Set getFreeSocketPorts(int numOfPorts) { * Return an @{@link InetAddress} to bind to. If bindWildCardAddress is true * than returns null. * - * @param localAddr - * @param bindWildCardAddress + * @param localAddr local addr. + * @param bindWildCardAddress bind wildcard address. * @return InetAddress */ public static InetAddress bindToLocalAddress(InetAddress localAddr, boolean diff --git a/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/net/NetworkTopology.java b/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/net/NetworkTopology.java index 137c940001c0c..ebb354e7db3cb 100644 --- a/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/net/NetworkTopology.java +++ b/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/net/NetworkTopology.java @@ -415,14 +415,16 @@ public boolean isOnSameRack(Node node1, Node node2) { } /** - * Check if network topology is aware of NodeGroup + * @return Check if network topology is aware of NodeGroup. */ public boolean isNodeGroupAware() { return false; } /** - * Return false directly as not aware of NodeGroup, to be override in sub-class + * @return Return false directly as not aware of NodeGroup, to be override in sub-class. + * @param node1 input node1. + * @param node2 input node2. */ public boolean isOnSameNodeGroup(Node node1, Node node2) { return false; @@ -729,11 +731,10 @@ public String toString() { } /** - * Divide networklocation string into two parts by last separator, and get + * @return Divide networklocation string into two parts by last separator, and get * the first part here. * - * @param networkLocation - * @return + * @param networkLocation input networkLocation. */ public static String getFirstHalf(String networkLocation) { int index = networkLocation.lastIndexOf(NodeBase.PATH_SEPARATOR_STR); @@ -741,11 +742,10 @@ public static String getFirstHalf(String networkLocation) { } /** - * Divide networklocation string into two parts by last separator, and get + * @return Divide networklocation string into two parts by last separator, and get * the second part here. * - * @param networkLocation - * @return + * @param networkLocation input networkLocation. */ public static String getLastHalf(String networkLocation) { int index = networkLocation.lastIndexOf(NodeBase.PATH_SEPARATOR_STR); @@ -897,7 +897,7 @@ public void sortByDistance(Node reader, Node[] nodes, int activeLen) { * or on a different rack from the reader. Sorting the nodes based on network * distance from the reader reduces network traffic and improves * performance. - *

    + *

    * As an additional twist, we also randomize the nodes at each network * distance. This helps with load balancing when there is data skew. * @@ -906,6 +906,7 @@ public void sortByDistance(Node reader, Node[] nodes, int activeLen) { * @param activeLen Number of active nodes at the front of the array * @param secondarySort a secondary sorting strategy which can inject into * that point from outside to help sort the same distance. + * @param Generics Type T */ public void sortByDistance(Node reader, T[] nodes, int activeLen, Consumer> secondarySort){ @@ -918,7 +919,7 @@ public void sortByDistance(Node reader, T[] nodes, * is not a datanode. Sorting the nodes based on network distance * from the reader reduces network traffic and improves * performance. - *

    + *

    * * @param reader Node where data will be read * @param nodes Available replicas with the requested data @@ -939,13 +940,14 @@ public void sortByDistanceUsingNetworkLocation(Node reader, Node[] nodes, * is not a datanode. Sorting the nodes based on network distance * from the reader reduces network traffic and improves * performance. - *

    + *

    * * @param reader Node where data will be read * @param nodes Available replicas with the requested data * @param activeLen Number of active nodes at the front of the array * @param secondarySort a secondary sorting strategy which can inject into * that point from outside to help sort the same distance. + * @param Generics Type T. */ public void sortByDistanceUsingNetworkLocation(Node reader, T[] nodes, int activeLen, Consumer> secondarySort) { @@ -1084,7 +1086,7 @@ private void interAddNodeWithEmptyRack(Node node) { String rackname = node.getNetworkLocation(); Set nodes = rackMap.get(rackname); if (nodes == null) { - nodes = new HashSet(); + nodes = new HashSet<>(); } if (!decommissionNodes.contains(node.getName())) { nodes.add(node.getName()); diff --git a/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/net/ScriptBasedMapping.java b/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/net/ScriptBasedMapping.java index 4db8155ffed3b..60ae442b4f602 100644 --- a/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/net/ScriptBasedMapping.java +++ b/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/net/ScriptBasedMapping.java @@ -124,12 +124,13 @@ public String toString() { } /** - * {@inheritDoc} + * {@inheritDoc}. *

    * This will get called in the superclass constructor, so a check is needed * to ensure that the raw mapping is defined before trying to relaying a null * configuration. - * @param conf + *

    + * @param conf input Configuration. */ @Override public void setConf(Configuration conf) { @@ -212,8 +213,9 @@ public List resolve(List names) { /** * Build and execute the resolution command. The command is * executed in the directory specified by the system property - * "user.dir" if set; otherwise the current working directory is used + * "user.dir" if set; otherwise the current working directory is used. * @param args a list of arguments + * @param commandScriptName input commandScriptName. * @return null if the number of arguments is out of range, * or the output of the command. */ diff --git a/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/net/ScriptBasedMappingWithDependency.java b/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/net/ScriptBasedMappingWithDependency.java index e05fae6496a15..4c1a547baabe0 100644 --- a/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/net/ScriptBasedMappingWithDependency.java +++ b/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/net/ScriptBasedMappingWithDependency.java @@ -74,12 +74,13 @@ public String toString() { } /** - * {@inheritDoc} + * {@inheritDoc}. *

    * This will get called in the superclass constructor, so a check is needed * to ensure that the raw mapping is defined before trying to relaying a null * configuration. - * @param conf + *

    + * @param conf input Configuration. */ @Override public void setConf(Configuration conf) { diff --git a/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/net/SocketInputStream.java b/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/net/SocketInputStream.java index cfa7b01e8136a..99e646a975b22 100644 --- a/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/net/SocketInputStream.java +++ b/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/net/SocketInputStream.java @@ -67,7 +67,7 @@ int performIO(ByteBuffer buf) throws IOException { * Channel for reading, should also be a {@link SelectableChannel}. * The channel will be configured to be non-blocking. * @param timeout timeout in milliseconds. must not be negative. - * @throws IOException + * @throws IOException raised on errors performing I/O. */ public SocketInputStream(ReadableByteChannel channel, long timeout) throws IOException { @@ -86,7 +86,7 @@ public SocketInputStream(ReadableByteChannel channel, long timeout) * * @param socket should have a channel associated with it. * @param timeout timeout timeout in milliseconds. must not be negative. - * @throws IOException + * @throws IOException raised on errors performing I/O. */ public SocketInputStream(Socket socket, long timeout) throws IOException { @@ -103,7 +103,7 @@ public SocketInputStream(Socket socket, long timeout) * @see SocketInputStream#SocketInputStream(ReadableByteChannel, long) * * @param socket should have a channel associated with it. - * @throws IOException + * @throws IOException raised on errors performing I/O. */ public SocketInputStream(Socket socket) throws IOException { this(socket.getChannel(), socket.getSoTimeout()); @@ -141,7 +141,7 @@ public synchronized void close() throws IOException { } /** - * Returns underlying channel used by inputstream. + * @return Returns underlying channel used by inputstream. * This is useful in certain cases like channel for * {@link FileChannel#transferFrom(ReadableByteChannel, long, long)}. */ diff --git a/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/net/SocketOutputStream.java b/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/net/SocketOutputStream.java index 93f4f56d78d63..3f6ea098a7200 100644 --- a/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/net/SocketOutputStream.java +++ b/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/net/SocketOutputStream.java @@ -72,7 +72,7 @@ int performIO(ByteBuffer buf) throws IOException { * Channel for writing, should also be a {@link SelectableChannel}. * The channel will be configured to be non-blocking. * @param timeout timeout in milliseconds. must not be negative. - * @throws IOException + * @throws IOException raised on errors performing I/O. */ public SocketOutputStream(WritableByteChannel channel, long timeout) throws IOException { @@ -91,7 +91,7 @@ public SocketOutputStream(WritableByteChannel channel, long timeout) * * @param socket should have a channel associated with it. * @param timeout timeout timeout in milliseconds. must not be negative. - * @throws IOException + * @throws IOException raised on errors performing I/O. */ public SocketOutputStream(Socket socket, long timeout) throws IOException { @@ -138,7 +138,7 @@ public synchronized void close() throws IOException { } /** - * Returns underlying channel used by this stream. + * @return Returns underlying channel used by this stream. * This is useful in certain cases like channel for * {@link FileChannel#transferTo(long, long, WritableByteChannel)} */ @@ -254,7 +254,12 @@ public void transferToFully(FileChannel fileCh, long position, int count, * Call * {@link #transferToFully(FileChannel, long, int, LongWritable, LongWritable) * } - * with null waitForWritableTime and transferToTime + * with null waitForWritableTime and transferToTime. + * + * @param fileCh input fileCh. + * @param position input position. + * @param count input count. + * @throws IOException raised on errors performing I/O. */ public void transferToFully(FileChannel fileCh, long position, int count) throws IOException { diff --git a/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/net/unix/DomainSocket.java b/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/net/unix/DomainSocket.java index 82c087737cbad..73fff0313a58c 100644 --- a/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/net/unix/DomainSocket.java +++ b/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/net/unix/DomainSocket.java @@ -106,6 +106,8 @@ native static void validateSocketPathSecurity0(String path, /** * Return true only if UNIX domain sockets are available. + * + * @return loadingFailureReason. */ public static String getLoadingFailureReason() { return loadingFailureReason; @@ -184,6 +186,7 @@ private void unreference(boolean checkClosed) throws ClosedChannelException { * * @param path The path to bind and listen on. * @return The new DomainSocket. + * @throws IOException raised on errors performing I/O. */ public static DomainSocket bindAndListen(String path) throws IOException { if (loadingFailureReason != null) { @@ -387,7 +390,7 @@ public void close() throws IOException { /** * Call shutdown(SHUT_RDWR) on the UNIX domain socket. * - * @throws IOException + * @throws IOException raised on errors performing I/O. */ public void shutdown() throws IOException { refCount.reference(); @@ -413,6 +416,7 @@ private native static void sendFileDescriptors0(int fd, * one byte. * @param offset The offset in the jbuf array to start at. * @param length Length of the jbuf array to use. + * @throws IOException raised on errors performing I/O. */ public void sendFileDescriptors(FileDescriptor descriptors[], byte jbuf[], int offset, int length) throws IOException { @@ -433,6 +437,13 @@ private static native int receiveFileDescriptors0(int fd, /** * Receive some FileDescriptor objects from the process on the other side of * this socket, and wrap them in FileInputStream objects. + * + * @param streams input stream. + * @param buf input buf. + * @param offset input offset. + * @param length input length. + * @return wrap them in FileInputStream objects. + * @throws IOException raised on errors performing I/O. */ public int recvFileInputStreams(FileInputStream[] streams, byte buf[], int offset, int length) throws IOException { diff --git a/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/security/CompositeGroupsMapping.java b/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/security/CompositeGroupsMapping.java index 6f799c1542095..deca6f1152ba4 100644 --- a/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/security/CompositeGroupsMapping.java +++ b/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/security/CompositeGroupsMapping.java @@ -109,7 +109,7 @@ public void cacheGroupsAdd(List groups) throws IOException { @Override public synchronized Set getGroupsSet(String user) throws IOException { - Set groupSet = new HashSet(); + Set groupSet = new HashSet<>(); Set groups = null; for (GroupMappingServiceProvider provider : providersList) { diff --git a/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/security/Credentials.java b/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/security/Credentials.java index e71bf6d40dd21..ef309cb2247fd 100644 --- a/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/security/Credentials.java +++ b/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/security/Credentials.java @@ -138,6 +138,8 @@ public void addToken(Text alias, Token t) { /** * Return all the tokens in the in-memory map. + * + * @return all the tokens in the in-memory map. */ public Collection> getAllTokens() { return tokenMap.values(); @@ -145,6 +147,8 @@ public Collection> getAllTokens() { /** * Returns an unmodifiable version of the full map of aliases to Tokens. + * + * @return TokenMap. */ public Map> getTokenMap() { return Collections.unmodifiableMap(tokenMap); @@ -192,6 +196,8 @@ public void removeSecretKey(Text alias) { /** * Return all the secret key entries in the in-memory map. + * + * @return Text List. */ public List getAllSecretKeys() { List list = new java.util.ArrayList(); @@ -202,6 +208,8 @@ public List getAllSecretKeys() { /** * Returns an unmodifiable version of the full map of aliases to secret keys. + * + * @return SecretKeyMap. */ public Map getSecretKeyMap() { return Collections.unmodifiableMap(secretKeysMap); @@ -209,9 +217,10 @@ public Map getSecretKeyMap() { /** * Convenience method for reading a token storage file and loading its Tokens. - * @param filename - * @param conf - * @throws IOException + * @param filename filename. + * @param conf configuration. + * @throws IOException raised on errors performing I/O. + * @return Credentials. */ public static Credentials readTokenStorageFile(Path filename, Configuration conf) @@ -233,9 +242,10 @@ public static Credentials readTokenStorageFile(Path filename, /** * Convenience method for reading a token storage file and loading its Tokens. - * @param filename - * @param conf - * @throws IOException + * @param filename filename. + * @param conf configuration. + * @throws IOException raised on errors performing I/O. + * @return Token. */ public static Credentials readTokenStorageFile(File filename, Configuration conf) @@ -256,6 +266,9 @@ public static Credentials readTokenStorageFile(File filename, /** * Convenience method for reading a token from a DataInputStream. + * + * @param in DataInputStream. + * @throws IOException raised on errors performing I/O. */ public void readTokenStorageStream(DataInputStream in) throws IOException { byte[] magic = new byte[TOKEN_STORAGE_MAGIC.length]; @@ -335,8 +348,8 @@ public void writeTokenStorageFile(Path filename, Configuration conf, /** * Stores all the keys to DataOutput. - * @param out - * @throws IOException + * @param out DataOutput. + * @throws IOException raised on errors performing I/O. */ @Override public void write(DataOutput out) throws IOException { @@ -401,8 +414,8 @@ void readProto(DataInput in) throws IOException { /** * Loads all the keys. - * @param in - * @throws IOException + * @param in DataInput. + * @throws IOException raised on errors performing I/O. */ @Override public void readFields(DataInput in) throws IOException { diff --git a/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/security/GroupMappingServiceProvider.java b/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/security/GroupMappingServiceProvider.java index 3a9073bbffaba..f37089fb55a79 100644 --- a/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/security/GroupMappingServiceProvider.java +++ b/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/security/GroupMappingServiceProvider.java @@ -40,18 +40,18 @@ public interface GroupMappingServiceProvider { * Returns EMPTY list in case of non-existing user * @param user User's name * @return group memberships of user - * @throws IOException + * @throws IOException raised on errors performing I/O. */ public List getGroups(String user) throws IOException; /** * Refresh the cache of groups and user mapping - * @throws IOException + * @throws IOException raised on errors performing I/O. */ public void cacheGroupsRefresh() throws IOException; /** * Caches the group user information * @param groups list of groups to add to cache - * @throws IOException + * @throws IOException raised on errors performing I/O. */ public void cacheGroupsAdd(List groups) throws IOException; @@ -60,7 +60,7 @@ public interface GroupMappingServiceProvider { * Returns EMPTY set in case of non-existing user * @param user User's name * @return set of group memberships of user - * @throws IOException + * @throws IOException raised on errors performing I/O. */ default Set getGroupsSet(String user) throws IOException { //Override to form the set directly to avoid another conversion diff --git a/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/security/Groups.java b/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/security/Groups.java index 70c633cdf8a23..1b3adc14283ee 100644 --- a/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/security/Groups.java +++ b/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/security/Groups.java @@ -465,7 +465,7 @@ public static Groups getUserToGroupsMappingService() { /** * Get the groups being used to map user-to-groups. - * @param conf + * @param conf configuration. * @return the groups being used to map user-to-groups. */ public static synchronized Groups getUserToGroupsMappingService( @@ -482,7 +482,7 @@ public static synchronized Groups getUserToGroupsMappingService( /** * Create new groups used to map user-to-groups with loaded configuration. - * @param conf + * @param conf configuration. * @return the groups being used to map user-to-groups. */ @Private diff --git a/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/security/HadoopKerberosName.java b/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/security/HadoopKerberosName.java index df96c500cd08b..b66f8444528a4 100644 --- a/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/security/HadoopKerberosName.java +++ b/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/security/HadoopKerberosName.java @@ -45,7 +45,7 @@ public class HadoopKerberosName extends KerberosName { /** * Create a name from the full Kerberos principal name. - * @param name + * @param name name. */ public HadoopKerberosName(String name) { super(name); @@ -58,7 +58,7 @@ public HadoopKerberosName(String name) { * method should be invoked directly. * * @param conf the new configuration - * @throws IOException + * @throws IOException raised on errors performing I/O. */ public static void setConfiguration(Configuration conf) throws IOException { final String defaultRule; diff --git a/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/security/IdMappingServiceProvider.java b/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/security/IdMappingServiceProvider.java index 86edab7de7097..08cacdc248fa4 100644 --- a/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/security/IdMappingServiceProvider.java +++ b/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/security/IdMappingServiceProvider.java @@ -18,11 +18,9 @@ package org.apache.hadoop.security; import java.io.IOException; -import java.util.List; import org.apache.hadoop.classification.InterfaceAudience; import org.apache.hadoop.classification.InterfaceStability; -import org.apache.hadoop.fs.CommonConfigurationKeysPublic; /** * An interface for the implementation of {@literal <}userId, diff --git a/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/security/KDiag.java b/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/security/KDiag.java index b2797871339e3..ee6a127f0e24f 100644 --- a/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/security/KDiag.java +++ b/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/security/KDiag.java @@ -435,7 +435,8 @@ protected boolean isSimpleAuthentication(Configuration conf) { * This is a recurrent problem * (that is: it keeps creeping back with JVM updates); * a fast failure is the best tactic. - * @throws NoSuchAlgorithmException + * @throws NoSuchAlgorithmException when a particular cryptographic algorithm is + * requested but is not available in the environment. */ protected void validateKeyLength() throws NoSuchAlgorithmException { @@ -1046,7 +1047,7 @@ private void failif(boolean condition, * @param conf configuration * @param argv argument list * @return an exception - * @throws Exception + * @throws Exception Exception. */ public static int exec(Configuration conf, String... argv) throws Exception { try(KDiag kdiag = new KDiag()) { diff --git a/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/security/KerberosInfo.java b/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/security/KerberosInfo.java index 062dcff61e1d9..e79492adf94e9 100644 --- a/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/security/KerberosInfo.java +++ b/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/security/KerberosInfo.java @@ -31,7 +31,10 @@ @InterfaceAudience.LimitedPrivate({"HDFS", "MapReduce"}) @InterfaceStability.Evolving public @interface KerberosInfo { - /** Key for getting server's Kerberos principal name from Configuration */ + /** + * Key for getting server's Kerberos principal name from Configuration. + * @return serverPrincipal. + */ String serverPrincipal(); String clientPrincipal() default ""; } diff --git a/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/security/NetgroupCache.java b/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/security/NetgroupCache.java index aa06c59a64814..5e466033fb713 100644 --- a/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/security/NetgroupCache.java +++ b/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/security/NetgroupCache.java @@ -65,7 +65,7 @@ public static List getNetgroupNames() { } private static Set getGroups() { - Set allGroups = new HashSet (); + Set allGroups = new HashSet<>(); for (Set userGroups : userToNetgroupsMap.values()) { allGroups.addAll(userGroups); } diff --git a/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/security/NullGroupsMapping.java b/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/security/NullGroupsMapping.java index 9592ecc32c012..aebb50a0af4a4 100644 --- a/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/security/NullGroupsMapping.java +++ b/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/security/NullGroupsMapping.java @@ -39,7 +39,7 @@ public void cacheGroupsAdd(List groups) { * * @param user User's name * @return set of group memberships of user - * @throws IOException + * @throws IOException raised on errors performing I/O. */ @Override public Set getGroupsSet(String user) throws IOException { diff --git a/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/security/ProviderUtils.java b/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/security/ProviderUtils.java index 9c4fb64d149c3..9cd85499f5803 100644 --- a/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/security/ProviderUtils.java +++ b/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/security/ProviderUtils.java @@ -136,6 +136,7 @@ public static URI nestURIForLocalJavaKeyStoreProvider(final URI localFile) * @param config the existing configuration with provider path * @param fileSystemClass the class which providers must be compatible * @return Configuration clone with new provider path + * @throws IOException raised on errors performing I/O. */ public static Configuration excludeIncompatibleCredentialProviders( Configuration config, Class fileSystemClass) diff --git a/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/security/RefreshUserMappingsProtocol.java b/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/security/RefreshUserMappingsProtocol.java index 005b2948ea2a6..c4f636e374519 100644 --- a/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/security/RefreshUserMappingsProtocol.java +++ b/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/security/RefreshUserMappingsProtocol.java @@ -42,14 +42,14 @@ public interface RefreshUserMappingsProtocol { /** * Refresh user to group mappings. - * @throws IOException + * @throws IOException raised on errors performing I/O. */ @Idempotent public void refreshUserToGroupsMappings() throws IOException; /** * Refresh superuser proxy group list - * @throws IOException + * @throws IOException raised on errors performing I/O. */ @Idempotent public void refreshSuperUserGroupsConfiguration() throws IOException; diff --git a/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/security/SaslInputStream.java b/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/security/SaslInputStream.java index a91a90ac7c901..2a8c3bf30c75f 100644 --- a/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/security/SaslInputStream.java +++ b/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/security/SaslInputStream.java @@ -281,7 +281,7 @@ public int read(byte[] b, int off, int len) throws IOException { *

    * Fewer bytes than requested might be skipped. The actual number of bytes * skipped is equal to n or the result of a call to - * {@link #available() available}, whichever is smaller. If + * {@link #available()}, whichever is smaller. If * n is less than zero, no bytes are skipped. * *

    diff --git a/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/security/SaslPropertiesResolver.java b/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/security/SaslPropertiesResolver.java index dd6c42e1491a8..25cc4a8144f05 100644 --- a/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/security/SaslPropertiesResolver.java +++ b/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/security/SaslPropertiesResolver.java @@ -46,7 +46,7 @@ public class SaslPropertiesResolver implements Configurable{ * Looks up the configuration to see if there is custom class specified. * Constructs the instance by passing the configuration directly to the * constructor to achieve thread safety using final fields. - * @param conf + * @param conf configuration. * @return SaslPropertiesResolver */ public static SaslPropertiesResolver getInstance(Configuration conf) { diff --git a/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/security/SaslRpcClient.java b/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/security/SaslRpcClient.java index 938eeeba96786..e5d62389abab7 100644 --- a/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/security/SaslRpcClient.java +++ b/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/security/SaslRpcClient.java @@ -351,9 +351,9 @@ String getServerPrincipal(SaslAuth authType) throws IOException { /** * Do client side SASL authentication with server via the given IpcStreams. * - * @param ipcStreams + * @param ipcStreams ipcStreams. * @return AuthMethod used to negotiate the connection - * @throws IOException + * @throws IOException raised on errors performing I/O. */ public AuthMethod saslConnect(IpcStreams ipcStreams) throws IOException { // redefined if/when a SASL negotiation starts, can be queried if the @@ -521,7 +521,7 @@ private boolean useWrap() { * * @param in - InputStream used to make the connection * @return InputStream that may be using SASL unwrap - * @throws IOException + * @throws IOException raised on errors performing I/O. */ public InputStream getInputStream(InputStream in) throws IOException { if (useWrap()) { @@ -537,7 +537,7 @@ public InputStream getInputStream(InputStream in) throws IOException { * * @param out - OutputStream used to make the connection * @return OutputStream that may be using wrapping - * @throws IOException + * @throws IOException raised on errors performing I/O. */ public OutputStream getOutputStream(OutputStream out) throws IOException { if (useWrap()) { @@ -638,7 +638,11 @@ public void write(byte[] buf, int off, int len) throws IOException { } } - /** Release resources used by wrapped saslClient */ + /** + * Release resources used by wrapped saslClient. + * @throws SaslException if authentication or generating response fails, + * or SASL protocol mixup + */ public void dispose() throws SaslException { if (saslClient != null) { saslClient.dispose(); diff --git a/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/security/SaslRpcServer.java b/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/security/SaslRpcServer.java index 7c3f14da21cf5..b61b6cc18414d 100644 --- a/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/security/SaslRpcServer.java +++ b/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/security/SaslRpcServer.java @@ -208,7 +208,11 @@ static char[] encodePassword(byte[] password) { StandardCharsets.UTF_8).toCharArray(); } - /** Splitting fully qualified Kerberos name into parts */ + /** + * Splitting fully qualified Kerberos name into parts. + * @param fullName fullName. + * @return splitKerberosName. + */ public static String[] splitKerberosName(String fullName) { return fullName.split("[/@]"); } @@ -240,17 +244,30 @@ private static AuthMethod valueOf(byte code) { return i < 0 || i >= values().length ? null : values()[i]; } - /** Return the SASL mechanism name */ + /** + * Return the SASL mechanism name. + * @return mechanismName. + */ public String getMechanismName() { return mechanismName; } - /** Read from in */ + /** + * Read from in. + * + * @param in DataInput. + * @throws IOException raised on errors performing I/O. + * @return AuthMethod. + */ public static AuthMethod read(DataInput in) throws IOException { return valueOf(in.readByte()); } - /** Write to out */ + /** + * Write to out. + * @param out DataOutput. + * @throws IOException raised on errors performing I/O. + */ public void write(DataOutput out) throws IOException { out.write(code); } diff --git a/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/security/SecurityUtil.java b/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/security/SecurityUtil.java index c9423490635cb..2b9822a3d4817 100644 --- a/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/security/SecurityUtil.java +++ b/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/security/SecurityUtil.java @@ -115,7 +115,9 @@ private static void setConfigurationInternal(Configuration conf) { } /** - * For use only by tests and initialization + * For use only by tests and initialization. + * + * @param flag flag. */ @InterfaceAudience.Private @VisibleForTesting @@ -487,6 +489,10 @@ public static Text buildTokenService(URI uri) { * Perform the given action as the daemon's login user. If the login * user cannot be determined, this will log a FATAL error and exit * the whole JVM. + * + * @param action action. + * @param generic type T. + * @return generic type T. */ public static T doAsLoginUserOrFatal(PrivilegedAction action) { if (UserGroupInformation.isSecurityEnabled()) { @@ -509,6 +515,7 @@ public static T doAsLoginUserOrFatal(PrivilegedAction action) { * InterruptedException is thrown, it is converted to an IOException. * * @param action the action to perform + * @param Generics Type T. * @return the result of the action * @throws IOException in the event of error */ @@ -522,6 +529,7 @@ public static T doAsLoginUser(PrivilegedExceptionAction action) * InterruptedException is thrown, it is converted to an IOException. * * @param action the action to perform + * @param generic type T. * @return the result of the action * @throws IOException in the event of error */ @@ -745,9 +753,13 @@ public static boolean isPrivilegedPort(final int port) { /** * Utility method to fetch ZK auth info from the configuration. + * + * @param conf configuration. + * @param configKey config key. * @throws java.io.IOException if the Zookeeper ACLs configuration file * cannot be read * @throws ZKUtil.BadAuthFormatException if the auth format is invalid + * @return ZKAuthInfo List. */ public static List getZKAuthInfos(Configuration conf, String configKey) throws IOException { diff --git a/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/security/ShellBasedIdMapping.java b/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/security/ShellBasedIdMapping.java index d1eab8f4e1bbd..c28471a3bdad9 100644 --- a/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/security/ShellBasedIdMapping.java +++ b/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/security/ShellBasedIdMapping.java @@ -210,7 +210,14 @@ private static Integer parseId(final String idStr) { /** * Get the list of users or groups returned by the specified command, * and save them in the corresponding map. - * @throws IOException + * + * @param map map. + * @param mapName mapName. + * @param command command. + * @param staticMapping staticMapping. + * @param regex regex. + * @throws IOException raised on errors performing I/O. + * @return updateMapInternal. */ @VisibleForTesting public static boolean updateMapInternal(BiMap map, diff --git a/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/security/ShellBasedUnixGroupsMapping.java b/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/security/ShellBasedUnixGroupsMapping.java index f4db520ac24c0..d0c4e11cbefc3 100644 --- a/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/security/ShellBasedUnixGroupsMapping.java +++ b/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/security/ShellBasedUnixGroupsMapping.java @@ -215,7 +215,7 @@ private Set getUnixGroups(String user) throws IOException { groups = resolvePartialGroupNames(user, e.getMessage(), executor.getOutput()); } catch (PartialGroupNameException pge) { - LOG.warn("unable to return groups for user {}", user, pge); + LOG.debug("unable to return groups for user {}", user, pge); return EMPTY_GROUPS_SET; } } diff --git a/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/security/ShellBasedUnixGroupsNetgroupMapping.java b/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/security/ShellBasedUnixGroupsNetgroupMapping.java index eff6985471b4c..01d6f299d17d5 100644 --- a/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/security/ShellBasedUnixGroupsNetgroupMapping.java +++ b/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/security/ShellBasedUnixGroupsNetgroupMapping.java @@ -92,6 +92,7 @@ public void cacheGroupsAdd(List groups) throws IOException { * * @param netgroup return users for this netgroup * @return list of users for a given netgroup + * @throws IOException raised on errors performing I/O. */ protected List getUsersForNetgroup(String netgroup) throws IOException { @@ -128,6 +129,7 @@ protected List getUsersForNetgroup(String netgroup) * * @param netgroup get users for this netgroup * @return string of users for a given netgroup in getent netgroups format + * @throws IOException raised on errors performing I/O. */ protected String execShellGetUserForNetgroup(final String netgroup) throws IOException { diff --git a/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/security/UserGroupInformation.java b/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/security/UserGroupInformation.java index b2efe502144cb..9671d8da38fd3 100644 --- a/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/security/UserGroupInformation.java +++ b/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/security/UserGroupInformation.java @@ -589,6 +589,7 @@ public static UserGroupInformation getCurrentUser() throws IOException { * @param user The user name, or NULL if none is specified. * * @return The most appropriate UserGroupInformation + * @throws IOException raised on errors performing I/O. */ public static UserGroupInformation getBestUGI( String ticketCachePath, String user) throws IOException { @@ -609,6 +610,7 @@ public static UserGroupInformation getBestUGI( * @param ticketCache the path to the ticket cache file * * @throws IOException if the kerberos login fails + * @return UserGroupInformation. */ @InterfaceAudience.Public @InterfaceStability.Evolving @@ -630,8 +632,9 @@ public static UserGroupInformation getUGIFromTicketCache( * The creator of subject is responsible for * renewing credentials. * - * @throws IOException + * @throws IOException raised on errors performing I/O. * @throws KerberosAuthException if the kerberos login fails + * @return UserGroupInformation */ public static UserGroupInformation getUGIFromSubject(Subject subject) throws IOException { @@ -686,7 +689,7 @@ public static UserGroupInformation getLoginUser() throws IOException { * remove the login method that is followed by a space from the username * e.g. "jack (auth:SIMPLE)" {@literal ->} "jack" * - * @param userName + * @param userName userName. * @return userName without login method */ public static String trimLoginMethod(String userName) { @@ -1106,7 +1109,7 @@ static long getNextTgtRenewalTime(final long tgtEndTime, final long now, * file and logs them in. They become the currently logged-in user. * @param user the principal name to load from the keytab * @param path the path to the keytab file - * @throws IOException + * @throws IOException raised on errors performing I/O. * @throws KerberosAuthException if it's a kerberos login exception. */ @InterfaceAudience.Public @@ -1136,7 +1139,7 @@ static void loginUserFromKeytab(String user, * This method assumes that the user logged in by calling * {@link #loginUserFromKeytab(String, String)}. * - * @throws IOException + * @throws IOException raised on errors performing I/O. * @throws KerberosAuthException if a failure occurred in logout, * or if the user did not log in by invoking loginUserFromKeyTab() before. */ @@ -1176,7 +1179,7 @@ public void logoutUserFromKeytab() throws IOException { /** * Re-login a user from keytab if TGT is expired or is close to expiry. * - * @throws IOException + * @throws IOException raised on errors performing I/O. * @throws KerberosAuthException if it's a kerberos login exception. */ public void checkTGTAndReloginFromKeytab() throws IOException { @@ -1224,7 +1227,7 @@ void fixKerberosTicketOrder() { * happened already. * The Subject field of this UserGroupInformation object is updated to have * the new credentials. - * @throws IOException + * @throws IOException raised on errors performing I/O. * @throws KerberosAuthException on a failure */ @InterfaceAudience.Public @@ -1241,7 +1244,7 @@ public void reloginFromKeytab() throws IOException { * Subject field of this UserGroupInformation object is updated to have the * new credentials. * - * @throws IOException + * @throws IOException raised on errors performing I/O. * @throws KerberosAuthException on a failure */ @InterfaceAudience.Public @@ -1278,7 +1281,7 @@ private void reloginFromKeytab(boolean checkTGT, boolean ignoreLastLoginTime) * method assumes that login had happened already. * The Subject field of this UserGroupInformation object is updated to have * the new credentials. - * @throws IOException + * @throws IOException raised on errors performing I/O. * @throws KerberosAuthException on a failure */ @InterfaceAudience.Public @@ -1346,6 +1349,7 @@ private void unprotectedRelogin(HadoopLoginContext login, * @param user the principal name to load from the keytab * @param path the path to the keytab file * @throws IOException if the keytab file can't be read + * @return UserGroupInformation. */ public static UserGroupInformation loginUserFromKeytabAndReturnUGI(String user, @@ -1372,8 +1376,9 @@ private boolean hasSufficientTimeElapsed(long now) { } /** - * Did the login happen via keytab + * Did the login happen via keytab. * @return true or false + * @throws IOException raised on errors performing I/O. */ @InterfaceAudience.Public @InterfaceStability.Evolving @@ -1382,8 +1387,9 @@ public static boolean isLoginKeytabBased() throws IOException { } /** - * Did the login happen via ticket cache + * Did the login happen via ticket cache. * @return true or false + * @throws IOException raised on errors performing I/O. */ public static boolean isLoginTicketBased() throws IOException { return getLoginUser().isFromTicket(); @@ -1405,6 +1411,7 @@ public static UserGroupInformation createRemoteUser(String user) { * Create a user from a login name. It is intended to be used for remote * users in RPC, since it won't have any credentials. * @param user the full user principal name, must not be empty or null + * @param authMethod authMethod. * @return the UserGroupInformation for the remote user. */ @InterfaceAudience.Public @@ -1474,8 +1481,8 @@ public static AuthenticationMethod valueOf(AuthMethod authMethod) { /** * Create a proxy user using username of the effective user and the ugi of the * real user. - * @param user - * @param realUser + * @param user user. + * @param realUser realUser. * @return proxyUser ugi */ @InterfaceAudience.Public @@ -1788,7 +1795,7 @@ public String toString() { /** * Sets the authentication method in the subject * - * @param authMethod + * @param authMethod authMethod. */ public synchronized void setAuthenticationMethod(AuthenticationMethod authMethod) { @@ -1798,7 +1805,7 @@ void setAuthenticationMethod(AuthenticationMethod authMethod) { /** * Sets the authentication method in the subject * - * @param authMethod + * @param authMethod authMethod. */ public void setAuthenticationMethod(AuthMethod authMethod) { user.setAuthenticationMethod(AuthenticationMethod.valueOf(authMethod)); @@ -1831,7 +1838,7 @@ public synchronized AuthenticationMethod getRealAuthenticationMethod() { * Returns the authentication method of a ugi. If the authentication method is * PROXY, returns the authentication method of the real user. * - * @param ugi + * @param ugi ugi. * @return AuthenticationMethod */ public static AuthenticationMethod getRealAuthenticationMethod( @@ -1933,6 +1940,8 @@ public T doAs(PrivilegedExceptionAction action /** * Log current UGI and token information into specified log. * @param ugi - UGI + * @param log log. + * @param caption caption. */ @InterfaceAudience.LimitedPrivate({"HDFS", "KMS"}) @InterfaceStability.Unstable @@ -1950,7 +1959,8 @@ public static void logUserInfo(Logger log, String caption, /** * Log all (current, real, login) UGI and token info into specified log. * @param ugi - UGI - * @throws IOException + * @param log - log. + * @throws IOException raised on errors performing I/O. */ @InterfaceAudience.LimitedPrivate({"HDFS", "KMS"}) @InterfaceStability.Unstable @@ -1968,7 +1978,7 @@ public static void logAllUserInfo(Logger log, UserGroupInformation ugi) throws /** * Log all (current, real, login) UGI and token info into UGI debug log. * @param ugi - UGI - * @throws IOException + * @throws IOException raised on errors performing I/O. */ public static void logAllUserInfo(UserGroupInformation ugi) throws IOException { @@ -2246,7 +2256,7 @@ private static String prependFileAuthority(String keytabPath) { * A test method to print out the current user's UGI. * @param args if there are two arguments, read the user from the keytab * and print it out. - * @throws Exception + * @throws Exception Exception. */ public static void main(String [] args) throws Exception { System.out.println("Getting UGI for current user"); diff --git a/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/security/alias/CredentialProvider.java b/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/security/alias/CredentialProvider.java index 113dcaeb5e644..2779194d85e00 100644 --- a/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/security/alias/CredentialProvider.java +++ b/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/security/alias/CredentialProvider.java @@ -91,7 +91,7 @@ public boolean isTransient() { /** * Ensures that any changes to the credentials are written to persistent * store. - * @throws IOException + * @throws IOException raised on errors performing I/O. */ public abstract void flush() throws IOException; @@ -99,7 +99,7 @@ public boolean isTransient() { * Get the credential entry for a specific alias. * @param alias the name of a specific credential * @return the credentialEntry - * @throws IOException + * @throws IOException raised on errors performing I/O. */ public abstract CredentialEntry getCredentialEntry(String alias) throws IOException; @@ -107,7 +107,7 @@ public abstract CredentialEntry getCredentialEntry(String alias) /** * Get the aliases for all credentials. * @return the list of alias names - * @throws IOException + * @throws IOException raised on errors performing I/O. */ public abstract List getAliases() throws IOException; @@ -115,7 +115,8 @@ public abstract CredentialEntry getCredentialEntry(String alias) * Create a new credential. The given alias must not already exist. * @param name the alias of the credential * @param credential the credential value for the alias. - * @throws IOException + * @throws IOException raised on errors performing I/O. + * @return CredentialEntry. */ public abstract CredentialEntry createCredentialEntry(String name, char[] credential) throws IOException; @@ -123,7 +124,7 @@ public abstract CredentialEntry createCredentialEntry(String name, /** * Delete the given credential. * @param name the alias of the credential to delete - * @throws IOException + * @throws IOException raised on errors performing I/O. */ public abstract void deleteCredentialEntry(String name) throws IOException; @@ -133,7 +134,7 @@ public abstract CredentialEntry createCredentialEntry(String name, * means. If true, the password should be provided by the caller using * setPassword(). * @return Whether or not the provider requires a password - * @throws IOException + * @throws IOException raised on errors performing I/O. */ public boolean needsPassword() throws IOException { return false; diff --git a/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/security/alias/CredentialShell.java b/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/security/alias/CredentialShell.java index 06d42207ecba5..66df17a181e54 100644 --- a/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/security/alias/CredentialShell.java +++ b/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/security/alias/CredentialShell.java @@ -70,9 +70,9 @@ public class CredentialShell extends CommandShell { * % hadoop credential check alias [-provider providerPath] * % hadoop credential delete alias [-provider providerPath] [-f] *

    - * @param args + * @param args args. * @return 0 if the argument(s) were recognized, 1 otherwise - * @throws IOException + * @throws IOException raised on errors performing I/O. */ @Override protected int init(String[] args) throws IOException { @@ -523,7 +523,7 @@ public void format(String message) { * * @param args * Command line arguments - * @throws Exception + * @throws Exception exception. */ public static void main(String[] args) throws Exception { int res = ToolRunner.run(new Configuration(), new CredentialShell(), args); diff --git a/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/security/authorize/AccessControlList.java b/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/security/authorize/AccessControlList.java index aa5b01fbed113..6fabbfb47b9f8 100644 --- a/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/security/authorize/AccessControlList.java +++ b/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/security/authorize/AccessControlList.java @@ -105,8 +105,8 @@ public AccessControlList(String users, String groups) { * @param userGroupStrings build ACL from array of Strings */ private void buildACL(String[] userGroupStrings) { - users = new HashSet(); - groups = new HashSet(); + users = new HashSet<>(); + groups = new HashSet<>(); for (String aclPart : userGroupStrings) { if (aclPart != null && isWildCardACLValue(aclPart)) { allAllowed = true; @@ -296,6 +296,7 @@ else if (!users.isEmpty()) { /** * Returns the access control list as a String that can be used for building a * new instance by sending it to the constructor of {@link AccessControlList}. + * @return acl string. */ public String getAclString() { StringBuilder sb = new StringBuilder(INITIAL_CAPACITY); diff --git a/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/security/authorize/ImpersonationProvider.java b/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/security/authorize/ImpersonationProvider.java index eff77d8942cf7..129e1e4dad26a 100644 --- a/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/security/authorize/ImpersonationProvider.java +++ b/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/security/authorize/ImpersonationProvider.java @@ -46,7 +46,7 @@ public interface ImpersonationProvider extends Configurable { * be preferred to avoid possibly re-resolving the ip address. * @param user ugi of the effective or proxy user which contains a real user. * @param remoteAddress the ip address of client. - * @throws AuthorizationException + * @throws AuthorizationException Authorization Exception. */ default void authorize(UserGroupInformation user, String remoteAddress) throws AuthorizationException { @@ -62,7 +62,7 @@ default void authorize(UserGroupInformation user, String remoteAddress) * * @param user ugi of the effective or proxy user which contains a real user * @param remoteAddress the ip address of client - * @throws AuthorizationException + * @throws AuthorizationException Authorization Exception. */ void authorize(UserGroupInformation user, InetAddress remoteAddress) throws AuthorizationException; diff --git a/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/security/authorize/ProxyServers.java b/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/security/authorize/ProxyServers.java index 410e25f583966..6f5283074dca6 100644 --- a/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/security/authorize/ProxyServers.java +++ b/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/security/authorize/ProxyServers.java @@ -33,7 +33,7 @@ public static void refresh() { } public static void refresh(Configuration conf){ - Collection tempServers = new HashSet(); + Collection tempServers = new HashSet<>(); // trusted proxy servers such as http proxies for (String host : conf.getTrimmedStrings(CONF_HADOOP_PROXYSERVERS)) { InetSocketAddress addr = new InetSocketAddress(host, 0); diff --git a/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/security/authorize/ProxyUsers.java b/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/security/authorize/ProxyUsers.java index be05e110b59cf..cc80708f1854e 100644 --- a/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/security/authorize/ProxyUsers.java +++ b/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/security/authorize/ProxyUsers.java @@ -94,7 +94,7 @@ public static void refreshSuperUserGroupsConfiguration(Configuration conf) { * * @param user ugi of the effective or proxy user which contains a real user * @param remoteAddress the ip address of client - * @throws AuthorizationException + * @throws AuthorizationException Authorization Exception. */ public static void authorize(UserGroupInformation user, String remoteAddress) throws AuthorizationException { @@ -106,7 +106,7 @@ public static void authorize(UserGroupInformation user, * * @param user ugi of the effective or proxy user which contains a real user * @param remoteAddress the inet address of client - * @throws AuthorizationException + * @throws AuthorizationException Authorization Exception. */ public static void authorize(UserGroupInformation user, InetAddress remoteAddress) throws AuthorizationException { @@ -125,10 +125,10 @@ private static ImpersonationProvider getSip() { /** * This function is kept to provide backward compatibility. - * @param user - * @param remoteAddress - * @param conf - * @throws AuthorizationException + * @param user user. + * @param remoteAddress remote address. + * @param conf configuration. + * @throws AuthorizationException Authorization Exception. * @deprecated use {@link #authorize(UserGroupInformation, String)} instead. */ @Deprecated diff --git a/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/security/authorize/RefreshAuthorizationPolicyProtocol.java b/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/security/authorize/RefreshAuthorizationPolicyProtocol.java index 0f0b25d8344e2..51a900fa71cb0 100644 --- a/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/security/authorize/RefreshAuthorizationPolicyProtocol.java +++ b/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/security/authorize/RefreshAuthorizationPolicyProtocol.java @@ -41,7 +41,7 @@ public interface RefreshAuthorizationPolicyProtocol { /** * Refresh the service-level authorization policy in-effect. - * @throws IOException + * @throws IOException raised on errors performing I/O. */ @Idempotent void refreshServiceAcl() throws IOException; diff --git a/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/security/http/CrossOriginFilter.java b/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/security/http/CrossOriginFilter.java index 059cdc4b653de..ef342f257a937 100644 --- a/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/security/http/CrossOriginFilter.java +++ b/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/security/http/CrossOriginFilter.java @@ -22,7 +22,6 @@ import java.util.ArrayList; import java.util.Arrays; import java.util.List; -import java.util.regex.Matcher; import java.util.regex.Pattern; import javax.servlet.Filter; diff --git a/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/security/http/RestCsrfPreventionFilter.java b/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/security/http/RestCsrfPreventionFilter.java index b81ed8e90155e..7363ca0ba6450 100644 --- a/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/security/http/RestCsrfPreventionFilter.java +++ b/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/security/http/RestCsrfPreventionFilter.java @@ -94,7 +94,7 @@ public void init(FilterConfig filterConfig) throws ServletException { void parseBrowserUserAgents(String userAgents) { String[] agentsArray = userAgents.split(","); - browserUserAgents = new HashSet(); + browserUserAgents = new HashSet<>(); for (String patternString : agentsArray) { browserUserAgents.add(Pattern.compile(patternString)); } @@ -102,7 +102,7 @@ void parseBrowserUserAgents(String userAgents) { void parseMethodsToIgnore(String mti) { String[] methods = mti.split(","); - methodsToIgnore = new HashSet(); + methodsToIgnore = new HashSet<>(); for (int i = 0; i < methods.length; i++) { methodsToIgnore.add(methods[i]); } diff --git a/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/security/ssl/ReloadingX509KeystoreManager.java b/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/security/ssl/ReloadingX509KeystoreManager.java index 216d949de1048..429304ef64c1c 100644 --- a/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/security/ssl/ReloadingX509KeystoreManager.java +++ b/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/security/ssl/ReloadingX509KeystoreManager.java @@ -63,8 +63,8 @@ public class ReloadingX509KeystoreManager extends X509ExtendedKeyManager { * @param location local path to the keystore file. * @param storePassword password of the keystore file. * @param keyPassword The password of the key. - * @throws IOException - * @throws GeneralSecurityException + * @throws IOException raised on errors performing I/O. + * @throws GeneralSecurityException thrown if create encryptor error. */ public ReloadingX509KeystoreManager(String type, String location, String storePassword, String keyPassword) diff --git a/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/security/ssl/ReloadingX509TrustManager.java b/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/security/ssl/ReloadingX509TrustManager.java index c6049a91b5a51..3dc5017ba6377 100644 --- a/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/security/ssl/ReloadingX509TrustManager.java +++ b/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/security/ssl/ReloadingX509TrustManager.java @@ -27,7 +27,6 @@ import javax.net.ssl.TrustManager; import javax.net.ssl.TrustManagerFactory; import javax.net.ssl.X509TrustManager; -import java.io.File; import java.io.IOException; import java.io.InputStream; import java.nio.file.Files; diff --git a/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/security/token/DelegationTokenIssuer.java b/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/security/token/DelegationTokenIssuer.java index 7b0a78bcd3c0d..77e74a271fc0e 100644 --- a/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/security/token/DelegationTokenIssuer.java +++ b/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/security/token/DelegationTokenIssuer.java @@ -39,17 +39,24 @@ public interface DelegationTokenIssuer { * The service name used as the alias for the token in the credential * token map. addDelegationTokens will use this to determine if * a token exists, and if not, add a new token with this alias. + * @return the token. */ String getCanonicalServiceName(); /** * Unconditionally get a new token with the optional renewer. Returning * null indicates the service does not issue tokens. + * @param renewer renewer. + * @return the token. + * @throws IOException raised on errors performing I/O. */ Token getDelegationToken(String renewer) throws IOException; /** * Issuers may need tokens from additional services. + * + * @return delegation token issuer. + * @throws IOException raised on errors performing I/O. */ default DelegationTokenIssuer[] getAdditionalTokenIssuers() throws IOException { @@ -81,6 +88,12 @@ default Token[] addDelegationTokens( /** * NEVER call this method directly. + * + * @param issuer issuer. + * @param renewer renewer. + * @param credentials cache in which to add new delegation tokens. + * @param tokens list of new delegation tokens. + * @throws IOException raised on errors performing I/O. */ @InterfaceAudience.Private static void collectDelegationTokens( diff --git a/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/security/token/DtFetcher.java b/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/security/token/DtFetcher.java index d74e7bdb10272..4b22df2043e8c 100644 --- a/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/security/token/DtFetcher.java +++ b/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/security/token/DtFetcher.java @@ -28,14 +28,29 @@ * compilation units. Resolution of fetcher impl will be done at runtime. */ public interface DtFetcher { - /** Return a key used to identify the object/service implementation. */ + /** + * Return a key used to identify the object/service implementation. + * @return ServiceName. + */ Text getServiceName(); - /** Used to allow the service API to indicate whether a token is required. */ + /** + * Used to allow the service API to indicate whether a token is required. + * @return isTokenRequired. + */ boolean isTokenRequired(); - /** Add any number of delegation tokens to Credentials object and return - * a token instance that is appropriate for aliasing, or null if none. */ + /** + * Add any number of delegation tokens to Credentials object and return + * a token instance that is appropriate for aliasing, or null if none. + * + * @param conf configuration. + * @param creds credentials. + * @param renewer renewer. + * @param url url. + * @throws Exception Exception. + * @return DelegationTokens. + */ Token addDelegationTokens(Configuration conf, Credentials creds, String renewer, String url) throws Exception; } diff --git a/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/security/token/DtFileOperations.java b/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/security/token/DtFileOperations.java index 2160d8b6a82a1..5d80a45f79f22 100644 --- a/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/security/token/DtFileOperations.java +++ b/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/security/token/DtFileOperations.java @@ -99,7 +99,7 @@ private static Path fileToPath(File f) { * @param format a string equal to FORMAT_PB or FORMAT_JAVA. * @param creds the Credentials object to be written out. * @param conf a Configuration object passed along. - * @throws IOException + * @throws IOException raised on errors performing I/O. */ public static void doFormattedWrite( File f, String format, Credentials creds, Configuration conf) @@ -118,7 +118,7 @@ public static void doFormattedWrite( * @param alias print only tokens matching alias (null matches all). * @param conf Configuration object passed along. * @param out print to this stream. - * @throws IOException + * @throws IOException raised on errors performing I/O. */ public static void printTokenFile( File tokenFile, Text alias, Configuration conf, PrintStream out) @@ -170,7 +170,7 @@ public static void printCredentials( * @param url pass this URL to fetcher after stripping any http/s prefix. * @param renewer pass this renewer to the fetcher. * @param conf Configuration object passed along. - * @throws IOException + * @throws IOException raised on errors performing I/O. */ public static void getTokenFile(File tokenFile, String fileFormat, Text alias, Text service, String url, String renewer, Configuration conf) @@ -225,7 +225,7 @@ public static void getTokenFile(File tokenFile, String fileFormat, * @param alias overwrite service field of fetched token with this text. * @param service only apply alias to tokens matching this service text. * @param conf Configuration object passed along. - * @throws IOException + * @throws IOException raised on errors performing I/O. */ public static void aliasTokenFile(File tokenFile, String fileFormat, Text alias, Text service, Configuration conf) throws Exception { @@ -246,7 +246,7 @@ public static void aliasTokenFile(File tokenFile, String fileFormat, * @param tokenFiles list of local File objects. Last file holds the output. * @param fileFormat a string equal to FORMAT_PB or FORMAT_JAVA, for output * @param conf Configuration object passed along. - * @throws IOException + * @throws IOException raised on errors performing I/O. */ public static void appendTokenFiles( ArrayList tokenFiles, String fileFormat, Configuration conf) @@ -269,8 +269,8 @@ public static void appendTokenFiles( * @param fileFormat a string equal to FORMAT_PB or FORMAT_JAVA, for output * @param alias remove only tokens matching alias; null matches all. * @param conf Configuration object passed along. - * @throws IOException - * @throws InterruptedException + * @throws IOException raised on errors performing I/O. + * @throws InterruptedException if the thread is interrupted. */ public static void removeTokenFromFile(boolean cancel, File tokenFile, String fileFormat, Text alias, Configuration conf) @@ -295,8 +295,8 @@ public static void removeTokenFromFile(boolean cancel, * @param fileFormat a string equal to FORMAT_PB or FORMAT_JAVA, for output * @param alias renew only tokens matching alias; null matches all. * @param conf Configuration object passed along. - * @throws IOException - * @throws InterruptedException + * @throws IOException raised on errors performing I/O. + * @throws InterruptedException if the thread is interrupted. */ public static void renewTokenFile( File tokenFile, String fileFormat, Text alias, Configuration conf) diff --git a/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/security/token/DtUtilShell.java b/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/security/token/DtUtilShell.java index bc2d1b6e11a7e..9e34ebf4a2a58 100644 --- a/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/security/token/DtUtilShell.java +++ b/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/security/token/DtUtilShell.java @@ -109,9 +109,9 @@ private String[] maybeDoLoginFromKeytabAndPrincipal(String[] args) * Parse the command line arguments and initialize subcommand. * Also will attempt to perform Kerberos login if both -principal and -keytab * flags are passed in args array. - * @param args + * @param args args. * @return 0 if the argument(s) were recognized, 1 otherwise - * @throws Exception + * @throws Exception Exception. */ @Override protected int init(String[] args) throws Exception { diff --git a/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/security/token/Token.java b/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/security/token/Token.java index 0141af8237b1b..33314060a5563 100644 --- a/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/security/token/Token.java +++ b/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/security/token/Token.java @@ -193,7 +193,7 @@ public synchronized Text getKind() { /** * Set the token kind. This is only intended to be used by services that * wrap another service's token. - * @param newKind + * @param newKind newKind. */ @InterfaceAudience.Private public synchronized void setKind(Text newKind) { @@ -367,7 +367,7 @@ private static void decodeWritable(Writable obj, /** * Encode this token as a url safe string. * @return the encoded string - * @throws IOException + * @throws IOException raised on errors performing I/O. */ public String encodeToUrlString() throws IOException { return encodeWritable(this); @@ -376,7 +376,7 @@ public String encodeToUrlString() throws IOException { /** * Decode the given url safe string into this token. * @param newValue the encoded string - * @throws IOException + * @throws IOException raised on errors performing I/O. */ public void decodeFromUrlString(String newValue) throws IOException { decodeWritable(this, newValue); @@ -481,6 +481,7 @@ private synchronized TokenRenewer getRenewer() throws IOException { /** * Is this token managed so that it can be renewed or cancelled? * @return true, if it can be renewed and cancelled. + * @throws IOException raised on errors performing I/O. */ public boolean isManaged() throws IOException { return getRenewer().isManaged(this); @@ -488,9 +489,10 @@ public boolean isManaged() throws IOException { /** * Renew this delegation token. + * @param conf configuration. * @return the new expiration time - * @throws IOException - * @throws InterruptedException + * @throws IOException raised on errors performing I/O. + * @throws InterruptedException if the thread is interrupted. */ public long renew(Configuration conf ) throws IOException, InterruptedException { @@ -499,8 +501,10 @@ public long renew(Configuration conf /** * Cancel this delegation token. - * @throws IOException - * @throws InterruptedException + * + * @param conf configuration. + * @throws IOException raised on errors performing I/O. + * @throws InterruptedException if the thread is interrupted. */ public void cancel(Configuration conf ) throws IOException, InterruptedException { diff --git a/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/security/token/TokenInfo.java b/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/security/token/TokenInfo.java index cc76824eb0e13..9234b23202eca 100644 --- a/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/security/token/TokenInfo.java +++ b/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/security/token/TokenInfo.java @@ -31,6 +31,10 @@ @InterfaceAudience.Public @InterfaceStability.Evolving public @interface TokenInfo { - /** The type of TokenSelector to be used */ + /** + * The type of TokenSelector to be used. + * + * @return TokenSelector + */ Class> value(); } \ No newline at end of file diff --git a/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/security/token/TokenRenewer.java b/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/security/token/TokenRenewer.java index 11e275f3213d2..eba4bf6daa42f 100644 --- a/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/security/token/TokenRenewer.java +++ b/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/security/token/TokenRenewer.java @@ -44,25 +44,37 @@ public abstract class TokenRenewer { * cancelled. * @param token the token being checked * @return true if the token may be renewed or cancelled - * @throws IOException + * @throws IOException raised on errors performing I/O. */ public abstract boolean isManaged(Token token) throws IOException; - - /** - * Renew the given token. - * @return the new expiration time - * @throws IOException - * @throws InterruptedException - */ + + /** + * Renew the given token. + * + * @param token the token being checked. + * @param conf configuration. + * + * @return the new expiration time. + * @throws IOException raised on errors performing I/O. + * @throws InterruptedException thrown when a thread is waiting, sleeping, + * or otherwise occupied, and the thread is interrupted, + * either before or during the activity. + */ public abstract long renew(Token token, Configuration conf ) throws IOException, InterruptedException; - - /** - * Cancel the given token - * @throws IOException - * @throws InterruptedException - */ + + /** + * Cancel the given token. + * + * @param token the token being checked. + * @param conf configuration. + * + * @throws IOException raised on errors performing I/O. + * @throws InterruptedException thrown when a thread is waiting, sleeping, + * or otherwise occupied, and the thread is interrupted, + * either before or during the activity. + */ public abstract void cancel(Token token, Configuration conf ) throws IOException, InterruptedException; diff --git a/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/security/token/delegation/AbstractDelegationTokenSecretManager.java b/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/security/token/delegation/AbstractDelegationTokenSecretManager.java index 6d2b1136da8c9..d0c0fac6e88df 100644 --- a/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/security/token/delegation/AbstractDelegationTokenSecretManager.java +++ b/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/security/token/delegation/AbstractDelegationTokenSecretManager.java @@ -153,7 +153,10 @@ public AbstractDelegationTokenSecretManager(long delegationKeyUpdateInterval, this.storeTokenTrackingId = false; } - /** should be called before this object is used */ + /** + * should be called before this object is used. + * @throws IOException raised on errors performing I/O. + */ public void startThreads() throws IOException { Preconditions.checkState(!running); updateCurrentKey(); @@ -176,6 +179,8 @@ public synchronized void reset() { /** * Total count of active delegation tokens. + * + * @return currentTokens.size. */ public long getCurrentTokensSize() { return currentTokens.size(); @@ -183,8 +188,11 @@ public long getCurrentTokensSize() { /** * Add a previously used master key to cache (when NN restarts), - * should be called before activate(). - * */ + * should be called before activate(). + * + * @param key delegation key. + * @throws IOException raised on errors performing I/O. + */ public synchronized void addKey(DelegationKey key) throws IOException { if (running) // a safety check throw new IOException("Can't add delegation key to a running SecretManager."); @@ -234,7 +242,9 @@ protected void updateStoredToken(TokenIdent ident, long renewDate) throws IOExce /** * For subclasses externalizing the storage, for example Zookeeper - * based implementations + * based implementations. + * + * @return currentId. */ protected synchronized int getCurrentKeyId() { return currentId; @@ -242,7 +252,9 @@ protected synchronized int getCurrentKeyId() { /** * For subclasses externalizing the storage, for example Zookeeper - * based implementations + * based implementations. + * + * @return currentId. */ protected synchronized int incrementCurrentKeyId() { return ++currentId; @@ -250,7 +262,9 @@ protected synchronized int incrementCurrentKeyId() { /** * For subclasses externalizing the storage, for example Zookeeper - * based implementations + * based implementations. + * + * @param keyId keyId. */ protected synchronized void setCurrentKeyId(int keyId) { currentId = keyId; @@ -258,7 +272,9 @@ protected synchronized void setCurrentKeyId(int keyId) { /** * For subclasses externalizing the storage, for example Zookeeper - * based implementations + * based implementations. + * + * @return delegationTokenSequenceNumber. */ protected synchronized int getDelegationTokenSeqNum() { return delegationTokenSequenceNumber; @@ -266,7 +282,9 @@ protected synchronized int getDelegationTokenSeqNum() { /** * For subclasses externalizing the storage, for example Zookeeper - * based implementations + * based implementations. + * + * @return delegationTokenSequenceNumber. */ protected synchronized int incrementDelegationTokenSeqNum() { return ++delegationTokenSequenceNumber; @@ -274,7 +292,9 @@ protected synchronized int incrementDelegationTokenSeqNum() { /** * For subclasses externalizing the storage, for example Zookeeper - * based implementations + * based implementations. + * + * @param seqNum seqNum. */ protected synchronized void setDelegationTokenSeqNum(int seqNum) { delegationTokenSequenceNumber = seqNum; @@ -282,7 +302,10 @@ protected synchronized void setDelegationTokenSeqNum(int seqNum) { /** * For subclasses externalizing the storage, for example Zookeeper - * based implementations + * based implementations. + * + * @param keyId keyId. + * @return DelegationKey. */ protected DelegationKey getDelegationKey(int keyId) { return allKeys.get(keyId); @@ -290,7 +313,10 @@ protected DelegationKey getDelegationKey(int keyId) { /** * For subclasses externalizing the storage, for example Zookeeper - * based implementations + * based implementations. + * + * @param key DelegationKey. + * @throws IOException raised on errors performing I/O. */ protected void storeDelegationKey(DelegationKey key) throws IOException { allKeys.put(key.getKeyId(), key); @@ -299,7 +325,10 @@ protected void storeDelegationKey(DelegationKey key) throws IOException { /** * For subclasses externalizing the storage, for example Zookeeper - * based implementations + * based implementations. + * + * @param key DelegationKey. + * @throws IOException raised on errors performing I/O. */ protected void updateDelegationKey(DelegationKey key) throws IOException { allKeys.put(key.getKeyId(), key); @@ -308,6 +337,9 @@ protected void updateDelegationKey(DelegationKey key) throws IOException { /** * For subclasses externalizing the storage, for example Zookeeper * based implementations + * + * @param ident ident. + * @return DelegationTokenInformation. */ protected DelegationTokenInformation getTokenInfo(TokenIdent ident) { return currentTokens.get(ident); @@ -315,7 +347,11 @@ protected DelegationTokenInformation getTokenInfo(TokenIdent ident) { /** * For subclasses externalizing the storage, for example Zookeeper - * based implementations + * based implementations. + * + * @param ident ident. + * @param tokenInfo tokenInfo. + * @throws IOException raised on errors performing I/O. */ protected void storeToken(TokenIdent ident, DelegationTokenInformation tokenInfo) throws IOException { @@ -326,7 +362,11 @@ protected void storeToken(TokenIdent ident, /** * For subclasses externalizing the storage, for example Zookeeper - * based implementations + * based implementations. + * + * @param ident ident. + * @param tokenInfo tokenInfo. + * @throws IOException raised on errors performing I/O. */ protected void updateToken(TokenIdent ident, DelegationTokenInformation tokenInfo) throws IOException { @@ -342,7 +382,7 @@ protected void updateToken(TokenIdent ident, * startThreads() is called) * @param identifier identifier read from persistent storage * @param renewDate token renew time - * @throws IOException + * @throws IOException raised on errors performing I/O. */ public synchronized void addPersistedDelegationToken( TokenIdent identifier, long renewDate) throws IOException { @@ -461,20 +501,28 @@ protected synchronized byte[] createPassword(TokenIdent identifier) { * Find the DelegationTokenInformation for the given token id, and verify that * if the token is expired. Note that this method should be called with * acquiring the secret manager's monitor. + * + * @param identifier identifier. + * @throws InvalidToken invalid token exception. + * @return DelegationTokenInformation. */ protected DelegationTokenInformation checkToken(TokenIdent identifier) throws InvalidToken { assert Thread.holdsLock(this); DelegationTokenInformation info = getTokenInfo(identifier); + String err; if (info == null) { - throw new InvalidToken("token " + formatTokenId(identifier) - + " can't be found in cache"); + err = "Token for real user: " + identifier.getRealUser() + ", can't be found in cache"; + LOG.warn("{}, Token={}", err, formatTokenId(identifier)); + throw new InvalidToken(err); } long now = Time.now(); if (info.getRenewDate() < now) { - throw new InvalidToken("token " + formatTokenId(identifier) + " is " + - "expired, current time: " + Time.formatTime(now) + - " expected renewal time: " + Time.formatTime(info.getRenewDate())); + err = "Token " + identifier.getRealUser() + " has expired, current time: " + + Time.formatTime(now) + " expected renewal time: " + Time + .formatTime(info.getRenewDate()); + LOG.info("{}, Token={}", err, formatTokenId(identifier)); + throw new InvalidToken(err); } return info; } @@ -504,7 +552,7 @@ public synchronized String getTokenTrackingId(TokenIdent identifier) { * Verifies that the given identifier and password are valid and match. * @param identifier Token identifier. * @param password Password in the token. - * @throws InvalidToken + * @throws InvalidToken InvalidToken. */ public synchronized void verifyToken(TokenIdent identifier, byte[] password) throws InvalidToken { @@ -578,6 +626,9 @@ public synchronized long renewToken(Token token, /** * Cancel a token by removing it from cache. + * + * @param token token. + * @param canceller canceller. * @return Identifier of the canceled token * @throws InvalidToken for invalid token * @throws AccessControlException if the user isn't allowed to cancel @@ -641,15 +692,22 @@ public DelegationTokenInformation(long renewDate, byte[] password, this.password = password; this.trackingId = trackingId; } - /** returns renew date */ + /** + * @return returns renew date. + */ public long getRenewDate() { return renewDate; } - /** returns password */ + /** + * @return returns password. + */ byte[] getPassword() { return password; } - /** returns tracking id */ + + /** + * @return returns tracking id. + */ public String getTrackingId() { return trackingId; } @@ -658,7 +716,7 @@ public String getTrackingId() { /** Remove expired delegation tokens from cache */ private void removeExpiredToken() throws IOException { long now = Time.now(); - Set expiredTokens = new HashSet(); + Set expiredTokens = new HashSet<>(); synchronized (this) { Iterator> i = currentTokens.entrySet().iterator(); @@ -754,7 +812,7 @@ public void run() { * * @param token the token where to extract the identifier * @return the delegation token identifier - * @throws IOException + * @throws IOException raised on errors performing I/O. */ public TokenIdent decodeTokenIdentifier(Token token) throws IOException { return token.decodeIdentifier(); diff --git a/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/security/token/delegation/web/DelegationTokenAuthenticatedURL.java b/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/security/token/delegation/web/DelegationTokenAuthenticatedURL.java index 0988826605fbb..31eef61c8f57c 100644 --- a/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/security/token/delegation/web/DelegationTokenAuthenticatedURL.java +++ b/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/security/token/delegation/web/DelegationTokenAuthenticatedURL.java @@ -336,6 +336,10 @@ public HttpURLConnection openConnection(URL url, Token token, String doAs) /** * Select a delegation token from all tokens in credentials, based on url. + * + * @param url url. + * @param creds credentials. + * @return token. */ @InterfaceAudience.Private public org.apache.hadoop.security.token.Token @@ -407,6 +411,7 @@ public HttpURLConnection openConnection(URL url, Token token, String doAs) * @param token the authentication token with the Delegation Token to renew. * @throws IOException if an IO error occurred. * @throws AuthenticationException if an authentication exception occurred. + * @return delegation token long value. */ public long renewDelegationToken(URL url, Token token) throws IOException, AuthenticationException { @@ -423,6 +428,7 @@ public long renewDelegationToken(URL url, Token token) * @param doAsUser the user to do as, which will be the token owner. * @throws IOException if an IO error occurred. * @throws AuthenticationException if an authentication exception occurred. + * @return delegation token long value. */ public long renewDelegationToken(URL url, Token token, String doAsUser) throws IOException, AuthenticationException { diff --git a/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/security/token/delegation/web/DelegationTokenAuthenticationFilter.java b/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/security/token/delegation/web/DelegationTokenAuthenticationFilter.java index be061bb63f3ee..571e54c5f907c 100644 --- a/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/security/token/delegation/web/DelegationTokenAuthenticationFilter.java +++ b/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/security/token/delegation/web/DelegationTokenAuthenticationFilter.java @@ -125,6 +125,7 @@ protected Properties getConfiguration(String configPrefix, * Set AUTH_TYPE property to the name of the corresponding authentication * handler class based on the input properties. * @param props input properties. + * @throws ServletException servlet exception. */ protected void setAuthHandlerClass(Properties props) throws ServletException { diff --git a/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/security/token/delegation/web/DelegationTokenAuthenticationHandler.java b/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/security/token/delegation/web/DelegationTokenAuthenticationHandler.java index b55214451ec25..f4ede6f35edb0 100644 --- a/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/security/token/delegation/web/DelegationTokenAuthenticationHandler.java +++ b/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/security/token/delegation/web/DelegationTokenAuthenticationHandler.java @@ -89,7 +89,7 @@ public abstract class DelegationTokenAuthenticationHandler public static final String TOKEN_KIND = PREFIX + "token-kind"; - private static final Set DELEGATION_TOKEN_OPS = new HashSet(); + private static final Set DELEGATION_TOKEN_OPS = new HashSet<>(); public static final String DELEGATION_TOKEN_UGI_ATTRIBUTE = "hadoop.security.delegation-token.ugi"; diff --git a/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/security/token/delegation/web/DelegationTokenAuthenticator.java b/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/security/token/delegation/web/DelegationTokenAuthenticator.java index 19427dcfafeb4..1b2b6ca1ef693 100644 --- a/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/security/token/delegation/web/DelegationTokenAuthenticator.java +++ b/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/security/token/delegation/web/DelegationTokenAuthenticator.java @@ -163,6 +163,7 @@ public void authenticate(URL url, AuthenticatedURL.Token token) * @param renewer the renewer user. * @throws IOException if an IO error occurred. * @throws AuthenticationException if an authentication exception occurred. + * @return abstract delegation token identifier. */ public Token getDelegationToken(URL url, AuthenticatedURL.Token token, String renewer) @@ -182,6 +183,7 @@ public Token getDelegationToken(URL url, * @param doAsUser the user to do as, which will be the token owner. * @throws IOException if an IO error occurred. * @throws AuthenticationException if an authentication exception occurred. + * @return abstract delegation token identifier. */ public Token getDelegationToken(URL url, AuthenticatedURL.Token token, String renewer, String doAsUser) @@ -207,8 +209,10 @@ public Token getDelegationToken(URL url, * @param url the URL to renew the delegation token from. Only HTTP/S URLs are * supported. * @param token the authentication token with the Delegation Token to renew. + * @param dToken abstract delegation token identifier. * @throws IOException if an IO error occurred. * @throws AuthenticationException if an authentication exception occurred. + * @return delegation token long value. */ public long renewDelegationToken(URL url, AuthenticatedURL.Token token, @@ -225,8 +229,10 @@ public long renewDelegationToken(URL url, * supported. * @param token the authentication token with the Delegation Token to renew. * @param doAsUser the user to do as, which will be the token owner. + * @param dToken abstract delegation token identifier. * @throws IOException if an IO error occurred. * @throws AuthenticationException if an authentication exception occurred. + * @return delegation token long value. */ public long renewDelegationToken(URL url, AuthenticatedURL.Token token, @@ -245,6 +251,7 @@ public long renewDelegationToken(URL url, * @param url the URL to cancel the delegation token from. Only HTTP/S URLs * are supported. * @param token the authentication token with the Delegation Token to cancel. + * @param dToken abstract delegation token identifier. * @throws IOException if an IO error occurred. */ public void cancelDelegationToken(URL url, @@ -261,6 +268,7 @@ public void cancelDelegationToken(URL url, * @param url the URL to cancel the delegation token from. Only HTTP/S URLs * are supported. * @param token the authentication token with the Delegation Token to cancel. + * @param dToken abstract delegation token identifier. * @param doAsUser the user to do as, which will be the token owner. * @throws IOException if an IO error occurred. */ diff --git a/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/security/token/delegation/web/HttpUserGroupInformation.java b/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/security/token/delegation/web/HttpUserGroupInformation.java index 614c0d3b36bcf..1f18b1c2f6ae0 100644 --- a/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/security/token/delegation/web/HttpUserGroupInformation.java +++ b/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/security/token/delegation/web/HttpUserGroupInformation.java @@ -20,8 +20,6 @@ import org.apache.hadoop.classification.InterfaceAudience; import org.apache.hadoop.security.UserGroupInformation; -import javax.servlet.http.HttpServletRequest; - /** * Util class that returns the remote {@link UserGroupInformation} in scope * for the HTTP request. diff --git a/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/service/AbstractService.java b/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/service/AbstractService.java index 9b50e7c524270..bd85a740186b7 100644 --- a/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/service/AbstractService.java +++ b/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/service/AbstractService.java @@ -241,7 +241,7 @@ public void stop() { /** * Relay to {@link #stop()} - * @throws IOException + * @throws IOException raised on errors performing I/O. */ @Override public final void close() throws IOException { diff --git a/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/service/CompositeService.java b/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/service/CompositeService.java index ee66e90f7c4ee..b08f9952d863e 100644 --- a/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/service/CompositeService.java +++ b/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/service/CompositeService.java @@ -82,7 +82,7 @@ protected void addService(Service service) { /** * If the passed object is an instance of {@link Service}, * add it to the list of services managed by this {@link CompositeService} - * @param object + * @param object object. * @return true if a service is added, false otherwise. */ protected boolean addIfService(Object object) { diff --git a/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/service/ServiceStateModel.java b/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/service/ServiceStateModel.java index 9f282b9f93483..c075cbb89a8a6 100644 --- a/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/service/ServiceStateModel.java +++ b/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/service/ServiceStateModel.java @@ -54,6 +54,8 @@ public class ServiceStateModel { /** * Create the service state model in the {@link Service.STATE#NOTINITED} * state. + * + * @param name input name. */ public ServiceStateModel(String name) { this(name, Service.STATE.NOTINITED); @@ -62,6 +64,7 @@ public ServiceStateModel(String name) { /** * Create a service state model instance in the chosen state * @param state the starting state + * @param name input name. */ public ServiceStateModel(String name, Service.STATE state) { this.state = state; diff --git a/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/service/launcher/AbstractLaunchableService.java b/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/service/launcher/AbstractLaunchableService.java index be28c5be2d017..66f8ee430f6e1 100644 --- a/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/service/launcher/AbstractLaunchableService.java +++ b/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/service/launcher/AbstractLaunchableService.java @@ -42,6 +42,8 @@ public abstract class AbstractLaunchableService extends AbstractService /** * Construct an instance with the given name. + * + * @param name input name. */ protected AbstractLaunchableService(String name) { super(name); diff --git a/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/service/launcher/ServiceLauncher.java b/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/service/launcher/ServiceLauncher.java index 7fd4657f3fd39..cef69ebd4b789 100644 --- a/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/service/launcher/ServiceLauncher.java +++ b/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/service/launcher/ServiceLauncher.java @@ -413,7 +413,7 @@ protected List getConfigurationsToCreate() { } /** - * This creates all the configurations defined by + * @return This creates all the configurations defined by * {@link #getConfigurationsToCreate()} , ensuring that * the resources have been pushed in. * If one cannot be loaded it is logged and the operation continues @@ -567,6 +567,7 @@ public ExitUtil.ExitException launchService(Configuration conf, * @throws Exception any other failure -if it implements * {@link ExitCodeProvider} then it defines the exit code for any * containing exception + * @return status code. */ protected int coreServiceLaunch(Configuration conf, @@ -648,7 +649,7 @@ protected int coreServiceLaunch(Configuration conf, } /** - * Instantiate the service defined in {@code serviceClassName}. + * @return Instantiate the service defined in {@code serviceClassName}. * * Sets the {@code configuration} field * to the the value of {@code conf}, @@ -852,6 +853,7 @@ protected void error(String message, Throwable thrown) { * The service launcher code assumes that after this method is invoked, * no other code in the same method is called. * @param exitCode code to exit + * @param message input message. */ protected void exit(int exitCode, String message) { ExitUtil.terminate(exitCode, message); @@ -1003,7 +1005,7 @@ protected void verifyConfigurationFilesExist(String[] filenames) { } /** - * Build a log message for starting up and shutting down. + * @return Build a log message for starting up and shutting down. * @param classname the class of the server * @param args arguments */ diff --git a/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/service/launcher/package-info.java b/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/service/launcher/package-info.java index f582fa2d97875..ae7acfc1b17f6 100644 --- a/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/service/launcher/package-info.java +++ b/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/service/launcher/package-info.java @@ -192,9 +192,8 @@ At this point, rather than block waiting for the service to terminate (as during the {@code execute()} method takes priority over any exit codes returned by the method. This allows services to signal failures simply by raising exceptions with exit codes. -

    +

    -

    To view the workflow in sequence, it is:

    1. (prepare configuration files —covered later)
    2. @@ -221,7 +220,7 @@ At this point, rather than block waiting for the service to terminate (as

      For a basic service, the return code is 0 unless an exception was raised. -

      +

      For a {@link org.apache.hadoop.service.launcher.LaunchableService}, the return code is the number returned from the {@link org.apache.hadoop.service.launcher.LaunchableService#execute()} @@ -235,7 +234,7 @@ At this point, rather than block waiting for the service to terminate (as of returning error codes to signal failures and for normal Services to return any error code at all. -

      +

      Any exception which implements the {@link org.apache.hadoop.util.ExitCodeProvider} interface is considered be a provider of the exit code: the method @@ -269,7 +268,7 @@ interface listing common exception codes. These are exception codes Note that {@link org.apache.hadoop.util.ExitUtil.ExitException} itself implements {@link org.apache.hadoop.util.ExitCodeProvider#getExitCode()} -

      +

      If an exception does not implement {@link org.apache.hadoop.util.ExitCodeProvider#getExitCode()}, it will be wrapped in an {@link org.apache.hadoop.util.ExitUtil.ExitException} @@ -324,7 +323,7 @@ interface listing common exception codes. These are exception codes when received, attempts to stop the service in a limited period of time. It then triggers a JVM shutdown by way of {@link org.apache.hadoop.util.ExitUtil#terminate(int, String)} -

      + If a second signal is received, the {@link org.apache.hadoop.service.launcher.InterruptEscalator} reacts by triggering an immediate JVM halt, invoking @@ -342,7 +341,7 @@ interface listing common exception codes. These are exception codes stop the service if a shutdown request is received, so ensuring that if the JVM is exited by any thread, an attempt to shut down the service will be made. - +

      Configuration class creation

      @@ -355,12 +354,12 @@ interface listing common exception codes. These are exception codes What the launcher does do is use reflection to try and create instances of these classes simply to force in the common resources. If the classes are not on the classpath this fact will be logged. -

      +

      Applications may consider it essential to either force load in the relevant configuration, or pass it down to the service being created. In which case further measures may be needed. -

      1: Creation in an extended {@code ServiceLauncher} + 1: Creation in an extended {@code ServiceLauncher}

      Subclass the Service launcher and override its @@ -371,9 +370,9 @@ interface listing common exception codes. These are exception codes HDFS or YARN. It does imply a dedicated script to invoke the custom {@code main()} method. -

      2: Creation in {@code bindArgs()} + 2: Creation in {@code bindArgs()} -

      +

      In {@link org.apache.hadoop.service.launcher.LaunchableService#bindArgs(Configuration, List)}, a new configuration is created: @@ -390,7 +389,7 @@ interface listing common exception codes. These are exception codes instances created via the service launcher. It does imply that this is expected to be only way that services will be launched. -

      3: Creation in {@code serviceInit()} +

      3: Creation in {@code serviceInit()}

        protected void serviceInit(Configuration conf) throws Exception {
      @@ -406,7 +405,7 @@ protected void serviceInit(Configuration conf) throws Exception {
        propagating information between peer services in a
        {@link org.apache.hadoop.service.CompositeService}.
        While a dangerous practice, it does happen.
      -
      + 

      Summary: the ServiceLauncher makes a best-effort attempt to load the standard Configuration subclasses, but does not fail if they are not present. @@ -429,7 +428,7 @@ class (the one created by the If this argument is repeated multiple times, all configuration files are merged with the latest file on the command line being the last one to be applied. -

      +

      All the {@code --conf <file>} argument pairs are stripped off the argument list provided to the instantiated service; they get the merged configuration, but not the commands used to create it. diff --git a/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/tools/CommandShell.java b/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/tools/CommandShell.java index a53e2259e0e25..25f79a63a233c 100644 --- a/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/tools/CommandShell.java +++ b/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/tools/CommandShell.java @@ -36,6 +36,7 @@ public abstract class CommandShell extends Configured implements Tool { /** * Return usage string for the command including any summary of subcommands. + * @return command usage. */ public abstract String getCommandUsage(); @@ -84,8 +85,9 @@ public int run(String[] args) throws Exception { /** * Parse the command line arguments and initialize subcommand instance. - * @param args + * @param args arguments. * @return 0 if the argument(s) were recognized, 1 otherwise + * @throws Exception init exception. */ protected abstract int init(String[] args) throws Exception; diff --git a/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/tools/GetGroupsBase.java b/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/tools/GetGroupsBase.java index 92cdb5835e77d..548409ea58e09 100644 --- a/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/tools/GetGroupsBase.java +++ b/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/tools/GetGroupsBase.java @@ -38,7 +38,7 @@ public abstract class GetGroupsBase extends Configured implements Tool { /** * Create an instance of this tool using the given configuration. - * @param conf + * @param conf configuration. */ protected GetGroupsBase(Configuration conf) { this(conf, System.out); @@ -84,7 +84,7 @@ public int run(String[] args) throws Exception { * * @param conf The configuration to use. * @return The address where the service is listening. - * @throws IOException + * @throws IOException raised on errors performing I/O. */ protected abstract InetSocketAddress getProtocolAddress(Configuration conf) throws IOException; @@ -92,7 +92,7 @@ protected abstract InetSocketAddress getProtocolAddress(Configuration conf) /** * Get a client of the {@link GetUserMappingsProtocol}. * @return A {@link GetUserMappingsProtocol} client proxy. - * @throws IOException + * @throws IOException raised on errors performing I/O. */ protected GetUserMappingsProtocol getUgmProtocol() throws IOException { GetUserMappingsProtocol userGroupMappingProtocol = diff --git a/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/tools/GetUserMappingsProtocol.java b/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/tools/GetUserMappingsProtocol.java index 3e80ac030d3ef..8b1922372ba3c 100644 --- a/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/tools/GetUserMappingsProtocol.java +++ b/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/tools/GetUserMappingsProtocol.java @@ -40,7 +40,7 @@ public interface GetUserMappingsProtocol { * Get the groups which are mapped to the given user. * @param user The user to get the groups for. * @return The set of groups the user belongs to. - * @throws IOException + * @throws IOException raised on errors performing I/O. */ @Idempotent public String[] getGroupsForUser(String user) throws IOException; diff --git a/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/tools/TableListing.java b/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/tools/TableListing.java index 348f86fe1368b..d1d933c737d2b 100644 --- a/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/tools/TableListing.java +++ b/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/tools/TableListing.java @@ -155,7 +155,9 @@ public Builder addField(String title, Justification justification, } /** - * Whether to hide column headers in table output + * Whether to hide column headers in table output. + * + * @return Builder. */ public Builder hideHeaders() { this.showHeader = false; @@ -164,6 +166,8 @@ public Builder hideHeaders() { /** * Whether to show column headers in table output. This is the default. + * + * @return Builder. */ public Builder showHeaders() { this.showHeader = true; @@ -173,6 +177,9 @@ public Builder showHeaders() { /** * Set the maximum width of a row in the TableListing. Must have one or * more wrappable fields for this to take effect. + * + * @param width width. + * @return Builder. */ public Builder wrapWidth(int width) { this.wrapWidth = width; @@ -181,6 +188,8 @@ public Builder wrapWidth(int width) { /** * Create a new TableListing. + * + * @return TableListing. */ public TableListing build() { return new TableListing(columns.toArray(new Column[0]), showHeader, diff --git a/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/util/AsyncDiskService.java b/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/util/AsyncDiskService.java index 8e48cb955a3a7..a3bf4faf0a980 100644 --- a/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/util/AsyncDiskService.java +++ b/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/util/AsyncDiskService.java @@ -94,6 +94,9 @@ public Thread newThread(Runnable r) { /** * Execute the task sometime in the future, using ThreadPools. + * + * @param root root. + * @param task task. */ public synchronized void execute(String root, Runnable task) { ThreadPoolExecutor executor = executors.get(root); @@ -123,7 +126,7 @@ public synchronized void shutdown() { * * @param milliseconds The number of milliseconds to wait * @return true if all thread pools are terminated without time limit - * @throws InterruptedException + * @throws InterruptedException if the thread is interrupted. */ public synchronized boolean awaitTermination(long milliseconds) throws InterruptedException { @@ -145,6 +148,8 @@ public synchronized boolean awaitTermination(long milliseconds) /** * Shut down all ThreadPools immediately. + * + * @return Runnable List. */ public synchronized List shutdownNow() { diff --git a/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/util/BlockingThreadPoolExecutorService.java b/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/util/BlockingThreadPoolExecutorService.java index d08e84f99de29..5c90e4bd2d601 100644 --- a/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/util/BlockingThreadPoolExecutorService.java +++ b/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/util/BlockingThreadPoolExecutorService.java @@ -117,6 +117,7 @@ private BlockingThreadPoolExecutorService(int permitCount, * @param keepAliveTime time until threads are cleaned up in {@code unit} * @param unit time unit * @param prefixName prefix of name for threads + * @return BlockingThreadPoolExecutorService. */ public static BlockingThreadPoolExecutorService newInstance( int activeTasks, diff --git a/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/util/CrcComposer.java b/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/util/CrcComposer.java index 4023995941f5c..5bf773cef3836 100644 --- a/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/util/CrcComposer.java +++ b/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/util/CrcComposer.java @@ -51,6 +51,11 @@ public class CrcComposer { /** * Returns a CrcComposer which will collapse all ingested CRCs into a single * value. + * + * @param type type. + * @param bytesPerCrcHint bytesPerCrcHint. + * @throws IOException raised on errors performing I/O. + * @return a CrcComposer which will collapse all ingested CRCs into a single value. */ public static CrcComposer newCrcComposer( DataChecksum.Type type, long bytesPerCrcHint) @@ -67,6 +72,13 @@ public static CrcComposer newCrcComposer( * final digest, each corresponding to 10 underlying data bytes. Using * a stripeLength greater than the total underlying data size is equivalent * to using a non-striped CrcComposer. + * + * @param type type. + * @param bytesPerCrcHint bytesPerCrcHint. + * @param stripeLength stripeLength. + * @return a CrcComposer which will collapse CRCs for every combined. + * underlying data size which aligns with the specified stripe boundary. + * @throws IOException raised on errors performing I/O. */ public static CrcComposer newStripedCrcComposer( DataChecksum.Type type, long bytesPerCrcHint, long stripeLength) @@ -102,7 +114,11 @@ public static CrcComposer newStripedCrcComposer( * each CRC expected to correspond to exactly {@code bytesPerCrc} underlying * data bytes. * + * @param crcBuffer crcBuffer. + * @param offset offset. * @param length must be a multiple of the expected byte-size of a CRC. + * @param bytesPerCrc bytesPerCrc. + * @throws IOException raised on errors performing I/O. */ public void update( byte[] crcBuffer, int offset, int length, long bytesPerCrc) @@ -125,6 +141,11 @@ public void update( * Composes {@code numChecksumsToRead} additional CRCs into the current digest * out of {@code checksumIn}, with each CRC expected to correspond to exactly * {@code bytesPerCrc} underlying data bytes. + * + * @param checksumIn checksumIn. + * @param numChecksumsToRead numChecksumsToRead. + * @param bytesPerCrc bytesPerCrc. + * @throws IOException raised on errors performing I/O. */ public void update( DataInputStream checksumIn, long numChecksumsToRead, long bytesPerCrc) @@ -138,6 +159,10 @@ public void update( /** * Updates with a single additional CRC which corresponds to an underlying * data size of {@code bytesPerCrc}. + * + * @param crcB crcB. + * @param bytesPerCrc bytesPerCrc. + * @throws IOException raised on errors performing I/O. */ public void update(int crcB, long bytesPerCrc) throws IOException { if (curCompositeCrc == 0) { @@ -173,6 +198,8 @@ public void update(int crcB, long bytesPerCrc) throws IOException { * total sum bytesPerCrc divided by stripeLength. If the sum of bytesPerCrc * is not a multiple of stripeLength, then the last CRC in the array * corresponds to totalLength % stripeLength underlying data bytes. + * + * @return byte representation of composed CRCs. */ public byte[] digest() { if (curPositionInStripe > 0) { diff --git a/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/util/CrcUtil.java b/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/util/CrcUtil.java index 42eaf148d64c3..c8183b042fb1a 100644 --- a/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/util/CrcUtil.java +++ b/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/util/CrcUtil.java @@ -44,6 +44,10 @@ private CrcUtil() { * Compute x^({@code lengthBytes} * 8) mod {@code mod}, where {@code mod} is * in "reversed" (little-endian) format such that {@code mod & 1} represents * x^31 and has an implicit term x^32. + * + * @param lengthBytes lengthBytes. + * @param mod mod. + * @return monomial. */ public static int getMonomial(long lengthBytes, int mod) { if (lengthBytes == 0) { @@ -73,7 +77,13 @@ public static int getMonomial(long lengthBytes, int mod) { } /** + * composeWithMonomial. + * + * @param crcA crcA. + * @param crcB crcB. * @param monomial Precomputed x^(lengthBInBytes * 8) mod {@code mod} + * @param mod mod. + * @return compose with monomial. */ public static int composeWithMonomial( int crcA, int crcB, int monomial, int mod) { @@ -81,7 +91,13 @@ public static int composeWithMonomial( } /** + * compose. + * + * @param crcA crcA. + * @param crcB crcB. * @param lengthB length of content corresponding to {@code crcB}, in bytes. + * @param mod mod. + * @return compose result. */ public static int compose(int crcA, int crcB, long lengthB, int mod) { int monomial = getMonomial(lengthB, mod); @@ -91,6 +107,8 @@ public static int compose(int crcA, int crcB, long lengthB, int mod) { /** * @return 4-byte array holding the big-endian representation of * {@code value}. + * + * @param value value. */ public static byte[] intToBytes(int value) { byte[] buf = new byte[4]; @@ -110,6 +128,11 @@ public static byte[] intToBytes(int value) { * Writes big-endian representation of {@code value} into {@code buf} * starting at {@code offset}. buf.length must be greater than or * equal to offset + 4. + * + * @param buf buf size. + * @param offset offset. + * @param value value. + * @throws IOException raised on errors performing I/O. */ public static void writeInt(byte[] buf, int offset, int value) throws IOException { @@ -127,6 +150,11 @@ public static void writeInt(byte[] buf, int offset, int value) /** * Reads 4-byte big-endian int value from {@code buf} starting at * {@code offset}. buf.length must be greater than or equal to offset + 4. + * + * @param offset offset. + * @param buf buf. + * @return int. + * @throws IOException raised on errors performing I/O. */ public static int readInt(byte[] buf, int offset) throws IOException { @@ -146,6 +174,10 @@ public static int readInt(byte[] buf, int offset) * For use with debug statements; verifies bytes.length on creation, * expecting it to represent exactly one CRC, and returns a hex * formatted value. + * + * @param bytes bytes. + * @throws IOException raised on errors performing I/O. + * @return a list of hex formatted values. */ public static String toSingleCrcString(final byte[] bytes) throws IOException { @@ -161,6 +193,10 @@ public static String toSingleCrcString(final byte[] bytes) * For use with debug statements; verifies bytes.length on creation, * expecting it to be divisible by CRC byte size, and returns a list of * hex formatted values. + * + * @param bytes bytes. + * @throws IOException raised on errors performing I/O. + * @return a list of hex formatted values. */ public static String toMultiCrcString(final byte[] bytes) throws IOException { diff --git a/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/util/Daemon.java b/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/util/Daemon.java index 3b95db6693e03..f735b82e4289b 100644 --- a/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/util/Daemon.java +++ b/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/util/Daemon.java @@ -52,14 +52,21 @@ public Daemon() { super(); } - /** Construct a daemon thread. */ + /** + * Construct a daemon thread. + * @param runnable runnable. + */ public Daemon(Runnable runnable) { super(runnable); this.runnable = runnable; this.setName(((Object)runnable).toString()); } - /** Construct a daemon thread to be part of a specified thread group. */ + /** + * Construct a daemon thread to be part of a specified thread group. + * @param group thread group. + * @param runnable runnable. + */ public Daemon(ThreadGroup group, Runnable runnable) { super(group, runnable); this.runnable = runnable; diff --git a/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/util/DataChecksum.java b/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/util/DataChecksum.java index 32a0adca1979a..1c37d5944c6f2 100644 --- a/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/util/DataChecksum.java +++ b/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/util/DataChecksum.java @@ -69,7 +69,12 @@ public enum Type { this.size = size; } - /** @return the type corresponding to the id. */ + /** + * the type corresponding to the id. + * + * @return the type corresponding to the id. + * @param id id. + */ public static Type valueOf(int id) { if (id < 0 || id >= values().length) { throw new IllegalArgumentException("id=" + id @@ -82,6 +87,8 @@ public static Type valueOf(int id) { /** * Create a Crc32 Checksum object. The implementation of the Crc32 algorithm * is chosen depending on the platform. + * + * @return Checksum. */ public static Checksum newCrc32() { return new CRC32(); @@ -105,6 +112,9 @@ static Checksum newCrc32C() { } /** + * getCrcPolynomialForType. + * + * @param type type. * @return the int representation of the polynomial associated with the * CRC {@code type}, suitable for use with further CRC arithmetic. * @throws IOException if there is no CRC polynomial applicable @@ -141,7 +151,11 @@ public static DataChecksum newDataChecksum(Type type, int bytesPerChecksum ) { /** * Creates a DataChecksum from HEADER_LEN bytes from arr[offset]. + * + * @param bytes bytes. + * @param offset offset. * @return DataChecksum of the type in the array or null in case of an error. + * @throws IOException raised on errors performing I/O. */ public static DataChecksum newDataChecksum(byte[] bytes, int offset) throws IOException { @@ -168,7 +182,12 @@ public static DataChecksum newDataChecksum(byte[] bytes, int offset) /** * This constructs a DataChecksum by reading HEADER_LEN bytes from input - * stream in + * stream in. + * + * @param in data input stream. + * @throws IOException raised on errors performing I/O. + * @return DataChecksum by reading HEADER_LEN + * bytes from input stream. */ public static DataChecksum newDataChecksum( DataInputStream in ) throws IOException { @@ -194,6 +213,9 @@ private static Type mapByteToChecksumType(int type) /** * Writes the checksum header to the output stream out. + * + * @param out output stream. + * @throws IOException raised on errors performing I/O. */ public void writeHeader( DataOutputStream out ) throws IOException { @@ -215,7 +237,11 @@ public byte[] getHeader() { /** * Writes the current checksum to the stream. * If reset is true, then resets the checksum. + * + * @param out out. + * @param reset reset. * @return number of bytes written. Will be equal to getChecksumSize(); + * @throws IOException raised on errors performing I/O. */ public int writeValue( DataOutputStream out, boolean reset ) throws IOException { @@ -239,7 +265,12 @@ public int writeValue( DataOutputStream out, boolean reset ) /** * Writes the current checksum to a buffer. * If reset is true, then resets the checksum. + * + * @param buf buf. + * @param offset offset. + * @param reset reset. * @return number of bytes written. Will be equal to getChecksumSize(); + * @throws IOException raised on errors performing I/O. */ public int writeValue( byte[] buf, int offset, boolean reset ) throws IOException { @@ -266,6 +297,9 @@ public int writeValue( byte[] buf, int offset, boolean reset ) /** * Compares the checksum located at buf[offset] with the current checksum. + * + * @param buf buf. + * @param offset offset. * @return true if the checksum matches and false otherwise. */ public boolean compare( byte buf[], int offset ) { @@ -295,12 +329,19 @@ public Type getChecksumType() { return type; } - /** @return the size for a checksum. */ + /** + * the size for a checksum. + * @return the size for a checksum. + */ public int getChecksumSize() { return type.size; } - /** @return the required checksum size given the data length. */ + /** + * the required checksum size given the data length. + * @param dataSize data size. + * @return the required checksum size given the data length. + */ public int getChecksumSize(int dataSize) { return ((dataSize - 1)/getBytesPerChecksum() + 1) * getChecksumSize(); } @@ -525,6 +566,12 @@ public void calculateChunkedSums(ByteBuffer data, ByteBuffer checksums) { /** * Implementation of chunked calculation specifically on byte arrays. This * is to avoid the copy when dealing with ByteBuffers that have array backing. + * + * @param data data. + * @param dataOffset dataOffset. + * @param dataLength dataLength. + * @param sums sums. + * @param sumsOffset sumsOffset. */ public void calculateChunkedSums( byte[] data, int dataOffset, int dataLength, diff --git a/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/util/DirectBufferPool.java b/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/util/DirectBufferPool.java index 3951ec2609c2c..8808e1067284a 100644 --- a/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/util/DirectBufferPool.java +++ b/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/util/DirectBufferPool.java @@ -53,6 +53,9 @@ public class DirectBufferPool { * Allocate a direct buffer of the specified size, in bytes. * If a pooled buffer is available, returns that. Otherwise * allocates a new one. + * + * @param size size. + * @return ByteBuffer. */ public ByteBuffer getBuffer(int size) { Queue> list = buffersBySize.get(size); diff --git a/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/util/DiskChecker.java b/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/util/DiskChecker.java index 446aad949a2f6..9fc18ca3bbf92 100644 --- a/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/util/DiskChecker.java +++ b/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/util/DiskChecker.java @@ -70,8 +70,8 @@ public DiskOutOfSpaceException(String msg) { * Create the directory if it doesn't exist and check that dir is readable, * writable and executable * - * @param dir - * @throws DiskErrorException + * @param dir dir. + * @throws DiskErrorException disk problem. */ public static void checkDir(File dir) throws DiskErrorException { checkDirInternal(dir); @@ -82,8 +82,8 @@ public static void checkDir(File dir) throws DiskErrorException { * readable, writable and executable. Perform some disk IO to * ensure that the disk is usable for writes. * - * @param dir - * @throws DiskErrorException + * @param dir dir. + * @throws DiskErrorException disk problem. */ public static void checkDirWithDiskIo(File dir) throws DiskErrorException { @@ -107,8 +107,8 @@ private static void checkDirInternal(File dir) * @param localFS local filesystem * @param dir directory * @param expected permission - * @throws DiskErrorException - * @throws IOException + * @throws DiskErrorException disk problem. + * @throws IOException raised on errors performing I/O. */ public static void checkDir(LocalFileSystem localFS, Path dir, FsPermission expected) @@ -125,8 +125,8 @@ public static void checkDir(LocalFileSystem localFS, Path dir, * @param localFS local filesystem * @param dir directory * @param expected permission - * @throws DiskErrorException - * @throws IOException + * @throws DiskErrorException disk problem. + * @throws IOException raised on errors performing I/O. */ public static void checkDirWithDiskIo(LocalFileSystem localFS, Path dir, FsPermission expected) diff --git a/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/util/DiskValidatorFactory.java b/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/util/DiskValidatorFactory.java index 67ded618d25b1..97d0c812667a2 100644 --- a/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/util/DiskValidatorFactory.java +++ b/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/util/DiskValidatorFactory.java @@ -40,6 +40,7 @@ private DiskValidatorFactory() { /** * Returns a {@link DiskValidator} instance corresponding to the passed clazz. * @param clazz a class extends {@link DiskValidator} + * @return disk validator. */ public static DiskValidator getInstance(Class clazz) { @@ -66,6 +67,7 @@ private DiskValidatorFactory() { * or "read-write" for {@link ReadWriteDiskValidator}. * @param diskValidator canonical class name, for example, "basic" * @throws DiskErrorException if the class cannot be located + * @return disk validator. */ @SuppressWarnings("unchecked") public static DiskValidator getInstance(String diskValidator) diff --git a/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/util/FileBasedIPList.java b/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/util/FileBasedIPList.java index 47aa9cc71a12e..31dfe594207be 100644 --- a/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/util/FileBasedIPList.java +++ b/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/util/FileBasedIPList.java @@ -58,7 +58,7 @@ public FileBasedIPList(String fileName) { lines = null; } if (lines != null) { - addressList = new MachineList(new HashSet(Arrays.asList(lines))); + addressList = new MachineList(new HashSet<>(Arrays.asList(lines))); } else { addressList = null; } diff --git a/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/util/FindClass.java b/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/util/FindClass.java index e51f7b14a60e0..268b4e166e8a3 100644 --- a/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/util/FindClass.java +++ b/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/util/FindClass.java @@ -20,7 +20,6 @@ import org.apache.hadoop.classification.VisibleForTesting; import org.apache.hadoop.conf.Configuration; import org.apache.hadoop.conf.Configured; -import org.apache.hadoop.util.StringUtils; import org.apache.hadoop.util.Tool; import org.apache.hadoop.util.ToolRunner; import java.io.IOException; diff --git a/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/util/GcTimeMonitor.java b/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/util/GcTimeMonitor.java index f189708692d8b..95d0d4d290ccd 100644 --- a/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/util/GcTimeMonitor.java +++ b/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/util/GcTimeMonitor.java @@ -54,6 +54,8 @@ public static class Builder { /** * Set observation window size in milliseconds. + * @param value value. + * @return window size in milliseconds. */ public Builder observationWindowMs(long value) { this.observationWindowMs = value; @@ -62,6 +64,8 @@ public Builder observationWindowMs(long value) { /** * Set sleep interval in milliseconds. + * @param value value. + * @return IntervalMs. */ public Builder sleepIntervalMs(long value) { this.sleepIntervalMs = value; @@ -70,6 +74,8 @@ public Builder sleepIntervalMs(long value) { /** * Set the max GC time percentage that triggers the alert handler. + * @param value value. + * @return max GC time percentage. */ public Builder maxGcTimePercentage(int value) { this.maxGcTimePercentage = value; @@ -78,6 +84,8 @@ public Builder maxGcTimePercentage(int value) { /** * Set the GC alert handler. + * @param value value. + * @return GC alert handler. */ public Builder gcTimeAlertHandler(GcTimeAlertHandler value) { this.handler = value; @@ -167,7 +175,10 @@ public void shutdown() { shouldRun = false; } - /** Returns a copy of the most recent data measured by this monitor. */ + /** + * Returns a copy of the most recent data measured by this monitor. + * @return a copy of the most recent data measured by this monitor + */ public GcData getLatestGcData() { return curData.clone(); } @@ -227,22 +238,34 @@ public static class GcData implements Cloneable { private long gcMonitorRunTime, totalGcTime, totalGcCount; private int gcTimePercentage; - /** Returns the absolute timestamp when this measurement was taken. */ + /** + * Returns the absolute timestamp when this measurement was taken. + * @return timestamp. + */ public long getTimestamp() { return timestamp; } - /** Returns the time since the start of the associated GcTimeMonitor. */ + /** + * Returns the time since the start of the associated GcTimeMonitor. + * @return GcMonitorRunTime. + */ public long getGcMonitorRunTime() { return gcMonitorRunTime; } - /** Returns accumulated GC time since this JVM started. */ + /** + * Returns accumulated GC time since this JVM started. + * @return AccumulatedGcTime. + */ public long getAccumulatedGcTime() { return totalGcTime; } - /** Returns the accumulated number of GC pauses since this JVM started. */ + /** + * Returns the accumulated number of GC pauses since this JVM started. + * @return AccumulatedGcCount. + */ public long getAccumulatedGcCount() { return totalGcCount; } @@ -250,6 +273,8 @@ public long getAccumulatedGcCount() { /** * Returns the percentage (0..100) of time that the JVM spent in GC pauses * within the observation window of the associated GcTimeMonitor. + * + * @return GcTimePercentage. */ public int getGcTimePercentage() { return gcTimePercentage; diff --git a/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/util/GenericOptionsParser.java b/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/util/GenericOptionsParser.java index 8ca7a904fdc84..81e1fb5d21234 100644 --- a/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/util/GenericOptionsParser.java +++ b/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/util/GenericOptionsParser.java @@ -57,8 +57,9 @@ * *

      Generic Options

      * - *

      The supported generic options are: - *

      + * 

      The supported generic options are:

      + *
      + *
        *     -conf <configuration file>     specify a configuration file
        *     -D <property=value>            use value for given property
        *     -fs <local|namenode:port>      specify a namenode
      @@ -69,13 +70,15 @@
        *                            jar files to include in the classpath.
        *     -archives <comma separated list of archives>    specify comma
        *             separated archives to be unarchived on the compute machines.
      -
      - * 

      + *

      + *
      * *

      The general command line syntax is:

      - *

      
      + * 
      + * 
        * bin/hadoop command [genericOptions] [commandOptions]
      - * 

      + * + *

      * *

      Generic command line arguments might modify * Configuration objects, given to constructors.

      @@ -83,7 +86,9 @@ *

      The functionality is implemented using Commons CLI.

      * *

      Examples:

      - *

      + *
      + * 
      + *
        * $ bin/hadoop dfs -fs darwin:8020 -ls /data
        * list /data directory in dfs with namenode darwin:8020
        * 
      @@ -105,7 +110,9 @@
        * $ bin/hadoop jar -libjars testlib.jar 
        * -archives test.tgz -files file.txt inputjar args
        * job submission with libjars, files and archives
      - * 

      + *

      + *
      + * * * @see Tool * @see ToolRunner @@ -124,7 +131,7 @@ public class GenericOptionsParser { * Create an options parser with the given options to parse the args. * @param opts the options * @param args the command line arguments - * @throws IOException + * @throws IOException raised on errors performing I/O. */ public GenericOptionsParser(Options opts, String[] args) throws IOException { @@ -134,7 +141,7 @@ public GenericOptionsParser(Options opts, String[] args) /** * Create an options parser to parse the args. * @param args the command line arguments - * @throws IOException + * @throws IOException raised on errors performing I/O. */ public GenericOptionsParser(String[] args) throws IOException { @@ -150,7 +157,7 @@ public GenericOptionsParser(String[] args) * * @param conf the Configuration to modify. * @param args command-line arguments. - * @throws IOException + * @throws IOException raised on errors performing I/O. */ public GenericOptionsParser(Configuration conf, String[] args) throws IOException { @@ -167,7 +174,7 @@ public GenericOptionsParser(Configuration conf, String[] args) * @param conf the configuration to modify * @param options options built by the caller * @param args User-specified arguments - * @throws IOException + * @throws IOException raised on errors performing I/O. */ public GenericOptionsParser(Configuration conf, Options options, String[] args) throws IOException { @@ -217,9 +224,10 @@ public boolean isParseSuccessful() { } /** - * Specify properties of each generic option. + * @return Specify properties of each generic option. * Important: as {@link OptionBuilder} is not thread safe, subclasses * must synchronize use on {@code OptionBuilder.class} + * @param opts input opts. */ @SuppressWarnings("static-access") protected Options buildGeneralOptions(Options opts) { @@ -359,9 +367,9 @@ private void processGeneralOptions(CommandLine line) throws IOException { /** * If libjars are set in the conf, parse the libjars. - * @param conf + * @param conf input Configuration. * @return libjar urls - * @throws IOException + * @throws IOException raised on errors performing I/O. */ public static URL[] getLibJars(Configuration conf) throws IOException { String jars = conf.get("tmpjars"); diff --git a/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/util/GenericsUtil.java b/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/util/GenericsUtil.java index 0aba34845a676..2d35b15bc5900 100644 --- a/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/util/GenericsUtil.java +++ b/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/util/GenericsUtil.java @@ -51,6 +51,8 @@ public static Class getClass(T t) { * T[]. * @param c the Class object of the items in the list * @param list the list to convert + * @param Generics Type T. + * @return T Array. */ public static T[] toArray(Class c, List list) { @@ -67,8 +69,10 @@ public static T[] toArray(Class c, List list) * Converts the given List<T> to a an array of * T[]. * @param list the list to convert + * @param Generics Type T. * @throws ArrayIndexOutOfBoundsException if the list is empty. * Use {@link #toArray(Class, List)} if the list may be empty. + * @return T Array. */ public static T[] toArray(List list) { return toArray(getClass(list.get(0)), list); diff --git a/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/util/HostsFileReader.java b/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/util/HostsFileReader.java index 5141740a3d23e..d94668356e261 100644 --- a/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/util/HostsFileReader.java +++ b/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/util/HostsFileReader.java @@ -135,7 +135,7 @@ public static void readFileToMapWithFileInputStream(String type, if (xmlInput) { readXmlFileToMapWithFileInputStream(type, filename, inputStream, map); } else { - HashSet nodes = new HashSet(); + HashSet nodes = new HashSet<>(); readFileToSetWithFileInputStream(type, filename, inputStream, nodes); for (String node : nodes) { map.put(node, null); diff --git a/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/util/IPList.java b/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/util/IPList.java index 3a2616376fbac..71cdcf11656ae 100644 --- a/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/util/IPList.java +++ b/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/util/IPList.java @@ -26,7 +26,7 @@ public interface IPList { /** * returns true if the ipAddress is in the IPList. - * @param ipAddress + * @param ipAddress ipAddress. * @return boolean value indicating whether the ipAddress is in the IPList */ public abstract boolean isIn(String ipAddress); diff --git a/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/util/IdGenerator.java b/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/util/IdGenerator.java index c14727a3771da..49ae8a8605e98 100644 --- a/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/util/IdGenerator.java +++ b/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/util/IdGenerator.java @@ -26,6 +26,9 @@ @InterfaceAudience.Private public interface IdGenerator { - /** Increment and then return the next value. */ + /** + * Increment and then return the next value. + * @return long value. + */ public long nextValue(); } diff --git a/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/util/IdentityHashStore.java b/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/util/IdentityHashStore.java index ecf099feff9d0..25e79f651b944 100644 --- a/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/util/IdentityHashStore.java +++ b/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/util/IdentityHashStore.java @@ -111,6 +111,9 @@ private void putInternal(Object k, Object v) { * Inserting a new (key, value) never overwrites a previous one. * In other words, you can insert the same key multiple times and it will * lead to multiple entries. + * + * @param k Generics Type k. + * @param v Generics Type v. */ public void put(K k, V v) { Preconditions.checkNotNull(k); @@ -142,6 +145,9 @@ private int getElementIndex(K k) { /** * Retrieve a value associated with a given key. + * + * @param k Generics Type k. + * @return Generics Type V. */ public V get(K k) { int index = getElementIndex(k); @@ -154,6 +160,9 @@ public V get(K k) { /** * Retrieve a value associated with a given key, and delete the * relevant entry. + * + * @param k Generics Type k. + * @return Generics Type V. */ public V remove(K k) { int index = getElementIndex(k); @@ -185,6 +194,8 @@ public interface Visitor { /** * Visit all key, value pairs in the IdentityHashStore. + * + * @param visitor visitor. */ public void visitAll(Visitor visitor) { int length = buffer == null ? 0 : buffer.length; diff --git a/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/util/IndexedSortable.java b/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/util/IndexedSortable.java index 1aa036e95b735..99472e18f1daf 100644 --- a/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/util/IndexedSortable.java +++ b/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/util/IndexedSortable.java @@ -31,11 +31,18 @@ public interface IndexedSortable { /** * Compare items at the given addresses consistent with the semantics of * {@link java.util.Comparator#compare(Object, Object)}. + * + * @param i(int). + * @param j(int). + * @return compare result. */ int compare(int i, int j); /** * Swap items at the given addresses. + * + * @param i i(int). + * @param j j(int). */ void swap(int i, int j); } diff --git a/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/util/IndexedSorter.java b/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/util/IndexedSorter.java index bdd024302c99f..252efe3e4de44 100644 --- a/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/util/IndexedSorter.java +++ b/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/util/IndexedSorter.java @@ -38,6 +38,10 @@ public interface IndexedSorter { * entry. * @see IndexedSortable#compare * @see IndexedSortable#swap + * + * @param r r. + * @param l l. + * @param s s. */ void sort(IndexedSortable s, int l, int r); @@ -45,6 +49,10 @@ public interface IndexedSorter { * Same as {@link #sort(IndexedSortable,int,int)}, but indicate progress * periodically. * @see #sort(IndexedSortable,int,int) + * @param s s. + * @param l l. + * @param r r. + * @param rep rep. */ void sort(IndexedSortable s, int l, int r, Progressable rep); diff --git a/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/util/InstrumentedLock.java b/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/util/InstrumentedLock.java index e83736cd3e35a..35d127af425c2 100644 --- a/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/util/InstrumentedLock.java +++ b/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/util/InstrumentedLock.java @@ -185,6 +185,7 @@ protected void startLockTiming() { * * @param acquireTime - timestamp just after acquiring the lock. * @param releaseTime - timestamp just before releasing the lock. + * @param checkLockHeld checkLockHeld. */ protected void check(long acquireTime, long releaseTime, boolean checkLockHeld) { diff --git a/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/util/IntrusiveCollection.java b/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/util/IntrusiveCollection.java index 21d8ad34a8784..c27a42d763ed1 100644 --- a/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/util/IntrusiveCollection.java +++ b/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/util/IntrusiveCollection.java @@ -49,38 +49,59 @@ public interface Element { /** * Insert this element into the list. This is the first thing that will * be called on the element. + * + * @param list list. + * @param prev prev. + * @param next next. */ void insertInternal(IntrusiveCollection list, Element prev, Element next); /** * Set the prev pointer of an element already in the list. + * + * @param list list. + * @param prev prev. */ void setPrev(IntrusiveCollection list, Element prev); /** * Set the next pointer of an element already in the list. + * + * @param list list. + * @param next next. */ void setNext(IntrusiveCollection list, Element next); /** * Remove an element from the list. This is the last thing that will be * called on an element. + * + * @param list list. */ void removeInternal(IntrusiveCollection list); /** * Get the prev pointer of an element. + * + * @param list list. + * @return Element. */ Element getPrev(IntrusiveCollection list); /** * Get the next pointer of an element. + * + * @param list list. + * @return Element. */ Element getNext(IntrusiveCollection list); /** * Returns true if this element is in the provided list. + * + * @param list list. + * @return if this element is in the provided list true, not false. */ boolean isInList(IntrusiveCollection list); } @@ -260,6 +281,7 @@ public T[] toArray(T[] array) { * Add an element to the end of the list. * * @param elem The new element to add. + * @return add result. */ @Override public boolean add(E elem) { @@ -281,6 +303,7 @@ public boolean add(E elem) { * Add an element to the front of the list. * * @param elem The new element to add. + * @return if addFirst success true, not false. */ public boolean addFirst(Element elem) { if (elem == null) { diff --git a/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/util/JsonSerialization.java b/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/util/JsonSerialization.java index 002c725490f4b..52c6c4505226a 100644 --- a/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/util/JsonSerialization.java +++ b/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/util/JsonSerialization.java @@ -287,7 +287,8 @@ public T load(FileSystem fs, Path path, @Nullable FileStatus status) * @param fs filesystem * @param path path * @param overwrite should any existing file be overwritten - * @throws IOException IO exception + * @param instance instance + * @throws IOException IO exception. */ public void save(FileSystem fs, Path path, T instance, boolean overwrite) throws @@ -296,11 +297,12 @@ public void save(FileSystem fs, Path path, T instance, } /** - * Write the JSON as bytes, then close the file. + * Write the JSON as bytes, then close the stream. + * @param instance instance to write * @param dataOutputStream an output stream that will always be closed * @throws IOException on any failure */ - private void writeJsonAsBytes(T instance, + public void writeJsonAsBytes(T instance, OutputStream dataOutputStream) throws IOException { try { dataOutputStream.write(toBytes(instance)); @@ -324,6 +326,7 @@ public byte[] toBytes(T instance) throws IOException { * @param bytes byte array * @throws IOException IO problems * @throws EOFException not enough data + * @return byte array. */ public T fromBytes(byte[] bytes) throws IOException { return fromJson(new String(bytes, 0, bytes.length, UTF_8)); diff --git a/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/util/JvmPauseMonitor.java b/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/util/JvmPauseMonitor.java index feb4f9b9d3f01..382266a99401f 100644 --- a/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/util/JvmPauseMonitor.java +++ b/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/util/JvmPauseMonitor.java @@ -214,6 +214,9 @@ public void run() { * This main function just leaks memory into a list. Running this class * with a 1GB heap will very quickly go into "GC hell" and result in * log messages about the GC pauses. + * + * @param args args. + * @throws Exception Exception. */ @SuppressWarnings("resource") public static void main(String []args) throws Exception { diff --git a/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/util/LightWeightCache.java b/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/util/LightWeightCache.java index f1897ea2368b2..0ce612ccca3e0 100644 --- a/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/util/LightWeightCache.java +++ b/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/util/LightWeightCache.java @@ -58,10 +58,13 @@ public class LightWeightCache extends LightWeightGSet { * Entries of {@link LightWeightCache}. */ public static interface Entry extends LinkedElement { - /** Set the expiration time. */ + /** + * Set the expiration time. + * @param timeNano input timeNano. + */ public void setExpirationTime(long timeNano); - /** Get the expiration time. */ + /** @return Get the expiration time. */ public long getExpirationTime(); } diff --git a/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/util/LightWeightGSet.java b/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/util/LightWeightGSet.java index d32d1f37b42fa..5151cdec5cd76 100644 --- a/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/util/LightWeightGSet.java +++ b/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/util/LightWeightGSet.java @@ -52,10 +52,13 @@ public class LightWeightGSet implements GSet { * Elements of {@link LightWeightGSet}. */ public interface LinkedElement { - /** Set the next element. */ + /** + * Set the next element. + * @param next inputNext. + */ void setNext(LinkedElement next); - /** Get the next element. */ + /** @return Get the next element. */ LinkedElement getNext(); } @@ -177,6 +180,8 @@ public E put(final E element) { * Remove the element corresponding to the key, * given key.hashCode() == index. * + * @param key key. + * @param index index. * @return If such element exists, return it. * Otherwise, return null. */ @@ -270,7 +275,11 @@ public String toString() { return b.toString(); } - /** Print detailed information of this object. */ + /** + * Print detailed information of this object. + * + * @param out out. + */ public void printDetails(final PrintStream out) { out.print(this + ", entries = ["); for(int i = 0; i < entries.length; i++) { @@ -357,6 +366,10 @@ public void setTrackModification(boolean trackModification) { * Let e = round(log_2 t). * Then, we choose capacity = 2^e/(size of reference), * unless it is outside the close interval [1, 2^30]. + * + * @param mapName mapName. + * @param percentage percentage. + * @return compute capacity. */ public static int computeCapacity(double percentage, String mapName) { return computeCapacity(Runtime.getRuntime().maxMemory(), percentage, diff --git a/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/util/LightWeightResizableGSet.java b/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/util/LightWeightResizableGSet.java index 7e7ececb32ee8..051e2680bc306 100644 --- a/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/util/LightWeightResizableGSet.java +++ b/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/util/LightWeightResizableGSet.java @@ -116,6 +116,8 @@ public synchronized void getIterator(Consumer> consumer) { /** * Resize the internal table to given capacity. + * + * @param cap capacity. */ @SuppressWarnings("unchecked") protected synchronized void resize(int cap) { diff --git a/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/util/LineReader.java b/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/util/LineReader.java index 520ddf6bdf401..08bd8102b1388 100644 --- a/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/util/LineReader.java +++ b/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/util/LineReader.java @@ -89,7 +89,7 @@ public LineReader(InputStream in, int bufferSize) { * Configuration. * @param in input stream * @param conf configuration - * @throws IOException + * @throws IOException raised on errors performing I/O. */ public LineReader(InputStream in, Configuration conf) throws IOException { this(in, conf.getInt(IO_FILE_BUFFER_SIZE_KEY, DEFAULT_BUFFER_SIZE)); @@ -133,7 +133,7 @@ public LineReader(InputStream in, int bufferSize, * @param in input stream * @param conf configuration * @param recordDelimiterBytes The delimiter - * @throws IOException + * @throws IOException raised on errors performing I/O. */ public LineReader(InputStream in, Configuration conf, byte[] recordDelimiterBytes) throws IOException { @@ -146,7 +146,7 @@ public LineReader(InputStream in, Configuration conf, /** * Close the underlying stream. - * @throws IOException + * @throws IOException raised on errors performing I/O. */ public void close() throws IOException { in.close(); diff --git a/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/util/Lists.java b/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/util/Lists.java index 5d9cc0502afaa..a9d0756e8e381 100644 --- a/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/util/Lists.java +++ b/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/util/Lists.java @@ -44,6 +44,9 @@ private Lists() { /** * Creates a mutable, empty {@code ArrayList} instance. + * + * @param Generics Type E. + * @return ArrayList Generics Type E. */ public static ArrayList newArrayList() { return new ArrayList<>(); @@ -59,6 +62,10 @@ public static ArrayList newArrayList() { * {@link Arrays#asList asList} * {@code (...))}, or for creating an empty list then calling * {@link Collections#addAll}. + * + * @param Generics Type E. + * @param elements elements. + * @return ArrayList Generics Type E. */ @SafeVarargs public static ArrayList newArrayList(E... elements) { @@ -76,6 +83,10 @@ public static ArrayList newArrayList(E... elements) { * Creates a mutable {@code ArrayList} instance containing the * given elements; a very thin shortcut for creating an empty list then * calling Iterables#addAll. + * + * @param Generics Type E. + * @param elements elements. + * @return ArrayList Generics Type E. */ public static ArrayList newArrayList(Iterable elements) { if (elements == null) { @@ -90,6 +101,10 @@ public static ArrayList newArrayList(Iterable elements) { * Creates a mutable {@code ArrayList} instance containing the * given elements; a very thin shortcut for creating an empty list * and then calling Iterators#addAll. + * + * @param Generics Type E. + * @param elements elements. + * @return ArrayList Generics Type E. */ public static ArrayList newArrayList(Iterator elements) { ArrayList list = newArrayList(); @@ -102,6 +117,7 @@ public static ArrayList newArrayList(Iterator elements) { * specified initial size; * simply delegates to {@link ArrayList#ArrayList(int)}. * + * @param Generics Type E. * @param initialArraySize the exact size of the initial backing array for * the returned array list * ({@code ArrayList} documentation calls this value the "capacity"). @@ -126,6 +142,8 @@ public static ArrayList newArrayListWithCapacity( * @return a new, empty {@code ArrayList}, sized appropriately to hold the * estimated number of elements. * @throws IllegalArgumentException if {@code estimatedSize} is negative. + * + * @param Generics Type E. */ public static ArrayList newArrayListWithExpectedSize( int estimatedSize) { @@ -140,7 +158,10 @@ public static ArrayList newArrayListWithExpectedSize( * outperform {@code LinkedList} except in certain rare and specific * situations. Unless you have * spent a lot of time benchmarking your specific needs, use one of those - * instead. + * instead.

      + * + * @param Generics Type E. + * @return Generics Type E List. */ public static LinkedList newLinkedList() { return new LinkedList<>(); @@ -155,7 +176,11 @@ public static LinkedList newLinkedList() { * {@link java.util.ArrayDeque} consistently * outperform {@code LinkedList} except in certain rare and specific * situations. Unless you have spent a lot of time benchmarking your - * specific needs, use one of those instead. + * specific needs, use one of those instead.

      + * + * @param elements elements. + * @param Generics Type E. + * @return Generics Type E List. */ public static LinkedList newLinkedList( Iterable elements) { @@ -238,6 +263,7 @@ private static boolean addAll(Collection addTo, * @param originalList original big list. * @param pageSize desired size of each sublist ( last one * may be smaller) + * @param Generics Type. * @return a list of sub lists. */ public static List> partition(List originalList, int pageSize) { diff --git a/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/util/MachineList.java b/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/util/MachineList.java index d07264c88ddd8..68c5c3ca47ff9 100644 --- a/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/util/MachineList.java +++ b/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/util/MachineList.java @@ -89,7 +89,7 @@ public MachineList(Collection hostEntries) { /** * Accepts a collection of ip/cidr/host addresses * - * @param hostEntries + * @param hostEntries hostEntries. * @param addressFactory addressFactory to convert host to InetAddress */ public MachineList(Collection hostEntries, @@ -139,7 +139,7 @@ public MachineList(Collection hostEntries, * {@link #includes(InetAddress)} should be preferred * to avoid possibly re-resolving the ip address. * - * @param ipAddress + * @param ipAddress ipAddress. * @return true if ipAddress is part of the list */ public boolean includes(String ipAddress) { @@ -161,7 +161,7 @@ public boolean includes(String ipAddress) { /** * Accepts an inet address and return true if address is in the list. - * @param address + * @param address address. * @return true if address is part of the list */ public boolean includes(InetAddress address) { diff --git a/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/util/NativeCodeLoader.java b/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/util/NativeCodeLoader.java index 11d1176f92a59..b5550f58ae218 100644 --- a/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/util/NativeCodeLoader.java +++ b/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/util/NativeCodeLoader.java @@ -76,16 +76,22 @@ public static boolean isNativeCodeLoaded() { /** * Returns true only if this build was compiled with support for ISA-L. + * + * @return if this build was compiled with support for ISA-L true, not false. */ public static native boolean buildSupportsIsal(); /** - * Returns true only if this build was compiled with support for ZStandard. + * Returns true only if this build was compiled with support for ZStandard. + * + * @return if this build was compiled with support for ZStandard true,not false. */ public static native boolean buildSupportsZstd(); /** * Returns true only if this build was compiled with support for openssl. + * + * @return if this build was compiled with support for openssl true,not false. */ public static native boolean buildSupportsOpenssl(); diff --git a/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/util/NativeLibraryChecker.java b/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/util/NativeLibraryChecker.java index 3847902e79743..9843a9d4057dd 100644 --- a/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/util/NativeLibraryChecker.java +++ b/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/util/NativeLibraryChecker.java @@ -39,7 +39,8 @@ public class NativeLibraryChecker { LoggerFactory.getLogger(NativeLibraryChecker.class); /** - * A tool to test native library availability, + * A tool to test native library availability. + * @param args args. */ public static void main(String[] args) { String usage = "NativeLibraryChecker [-a|-h]\n" diff --git a/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/util/OperationDuration.java b/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/util/OperationDuration.java index fdd25286a2300..1fb920e99f08e 100644 --- a/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/util/OperationDuration.java +++ b/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/util/OperationDuration.java @@ -95,9 +95,11 @@ public String toString() { /** * Get the duration in milliseconds. - *

      + * + *

      * This will be 0 until a call * to {@link #finished()} has been made. + *

      * @return the currently recorded duration. */ public long value() { diff --git a/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/util/Options.java b/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/util/Options.java index 23169e3af3533..ccd494e5e40d9 100644 --- a/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/util/Options.java +++ b/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/util/Options.java @@ -126,7 +126,7 @@ public Progressable getValue() { * @param cls the dynamic class to find * @param opts the list of options to look through * @return the first option that matches - * @throws IOException + * @throws IOException raised on errors performing I/O. */ @SuppressWarnings("unchecked") public static T getOption(Class cls, base [] opts diff --git a/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/util/PrintJarMainClass.java b/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/util/PrintJarMainClass.java index df571f35e2fcd..f7822e3f788d0 100644 --- a/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/util/PrintJarMainClass.java +++ b/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/util/PrintJarMainClass.java @@ -31,7 +31,7 @@ public class PrintJarMainClass { /** - * @param args + * @param args args. */ public static void main(String[] args) { try (JarFile jar_file = new JarFile(args[0])) { diff --git a/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/util/PriorityQueue.java b/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/util/PriorityQueue.java index ebb943bcb6285..d149d5d811914 100644 --- a/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/util/PriorityQueue.java +++ b/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/util/PriorityQueue.java @@ -31,11 +31,19 @@ public abstract class PriorityQueue { private int size; private int maxSize; - /** Determines the ordering of objects in this priority queue. Subclasses - must define this one method. */ + /** + * Determines the ordering of objects in this priority queue. Subclasses + must define this one method. + * @param a object a. + * @param b object b. + * @return if a less than b true, not false + */ protected abstract boolean lessThan(Object a, Object b); - /** Subclass constructors must call this. */ + /** + * Subclass constructors must call this. + * @param maxSize max size. + */ @SuppressWarnings("unchecked") protected final void initialize(int maxSize) { size = 0; @@ -48,6 +56,7 @@ protected final void initialize(int maxSize) { * Adds an Object to a PriorityQueue in log(size) time. * If one tries to add more objects than maxSize from initialize * a RuntimeException (ArrayIndexOutOfBound) is thrown. + * @param element element. */ public final void put(T element) { size++; @@ -58,7 +67,7 @@ public final void put(T element) { /** * Adds element to the PriorityQueue in log(size) time if either * the PriorityQueue is not full, or not lessThan(element, top()). - * @param element + * @param element element. * @return true if element is added, false otherwise. */ public boolean insert(T element){ @@ -75,7 +84,11 @@ else if (size > 0 && !lessThan(element, top())){ return false; } - /** Returns the least element of the PriorityQueue in constant time. */ + /** + * Returns the least element of the PriorityQueue in constant time. + * + * @return T Generics Type T. + */ public final T top() { if (size > 0) return heap[1]; @@ -83,8 +96,11 @@ public final T top() { return null; } - /** Removes and returns the least element of the PriorityQueue in log(size) - time. */ + /** + * Removes and returns the least element of the PriorityQueue in log(size) + time. + * @return T Generics Type T. + */ public final T pop() { if (size > 0) { T result = heap[1]; // save first value @@ -109,7 +125,11 @@ public final void adjustTop() { } - /** Returns the number of elements currently stored in the PriorityQueue. */ + /** + * Returns the number of elements currently stored in the PriorityQueue. + * + * @return size. + */ public final int size() { return size; } diff --git a/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/util/ProgramDriver.java b/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/util/ProgramDriver.java index 347e5087eaa37..e49cf57705344 100644 --- a/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/util/ProgramDriver.java +++ b/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/util/ProgramDriver.java @@ -91,12 +91,12 @@ private static void printUsage(Map programs) { } /** - * This is the method that adds the classed to the repository + * This is the method that adds the classed to the repository. * @param name The name of the string you want the class instance to be called with * @param mainClass The class that you want to add to the repository * @param description The description of the class - * @throws NoSuchMethodException - * @throws SecurityException + * @throws NoSuchMethodException when a particular method cannot be found. + * @throws SecurityException security manager to indicate a security violation. */ public void addClass(String name, Class mainClass, String description) throws Throwable { @@ -111,10 +111,10 @@ public void addClass(String name, Class mainClass, String description) * of the command line arguments. * @param args The argument from the user. args[0] is the command to run. * @return -1 on error, 0 on success - * @throws NoSuchMethodException - * @throws SecurityException - * @throws IllegalAccessException - * @throws IllegalArgumentException + * @throws NoSuchMethodException when a particular method cannot be found. + * @throws SecurityException security manager to indicate a security violation. + * @throws IllegalAccessException for backward compatibility. + * @throws IllegalArgumentException if the arg is invalid. * @throws Throwable Anything thrown by the example program's main */ public int run(String[] args) @@ -146,7 +146,11 @@ public int run(String[] args) } /** - * API compatible with Hadoop 1.x + * API compatible with Hadoop 1.x. + * + * @param argv argv. + * @throws Throwable Anything thrown + * by the example program's main */ public void driver(String[] argv) throws Throwable { if (run(argv) == -1) { diff --git a/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/util/Progress.java b/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/util/Progress.java index bd1c0f4a62a78..a839c04e99e63 100644 --- a/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/util/Progress.java +++ b/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/util/Progress.java @@ -53,14 +53,21 @@ public class Progress { /** Creates a new root node. */ public Progress() {} - /** Adds a named node to the tree. */ + /** + * Adds a named node to the tree. + * @param status status. + * @return Progress. + */ public Progress addPhase(String status) { Progress phase = addPhase(); phase.setStatus(status); return phase; } - /** Adds a node to the tree. Gives equal weightage to all phases */ + /** + * Adds a node to the tree. Gives equal weightage to all phases. + * @return Progress. + */ public synchronized Progress addPhase() { Progress phase = addNewPhase(); // set equal weightage for all phases @@ -77,7 +84,13 @@ private synchronized Progress addNewPhase() { return phase; } - /** Adds a named node with a specified progress weightage to the tree. */ + /** + * Adds a named node with a specified progress weightage to the tree. + * + * @param status status. + * @param weightage weightage. + * @return Progress. + */ public Progress addPhase(String status, float weightage) { Progress phase = addPhase(weightage); phase.setStatus(status); @@ -85,7 +98,12 @@ public Progress addPhase(String status, float weightage) { return phase; } - /** Adds a node with a specified progress weightage to the tree. */ + /** + * Adds a node with a specified progress weightage to the tree. + * + * @param weightage weightage. + * @return Progress. + */ public synchronized Progress addPhase(float weightage) { Progress phase = new Progress(); progressWeightagesForPhases.add(weightage); @@ -104,7 +122,11 @@ public synchronized Progress addPhase(float weightage) { return phase; } - /** Adds n nodes to the tree. Gives equal weightage to all phases */ + /** + * Adds n nodes to the tree. Gives equal weightage to all phases. + * + * @param n n. + */ public synchronized void addPhases(int n) { for (int i = 0; i < n; i++) { addNewPhase(); @@ -136,7 +158,10 @@ public synchronized void startNextPhase() { currentPhase++; } - /** Returns the current sub-node executing. */ + /** + * Returns the current sub-node executing. + * @return Progress. + */ public synchronized Progress phase() { return phases.get(currentPhase); } @@ -158,7 +183,10 @@ public void complete() { } } - /** Called during execution on a leaf node to set its progress. */ + /** + * Called during execution on a leaf node to set its progress. + * @param progress progress. + */ public synchronized void set(float progress) { if (Float.isNaN(progress)) { progress = 0; @@ -188,7 +216,10 @@ else if (progress == Float.POSITIVE_INFINITY) { this.progress = progress; } - /** Returns the overall progress of the root. */ + /** + * Returns the overall progress of the root. + * @return progress. + */ // this method probably does not need to be synchronized as getInternal() is // synchronized and the node's parent never changes. Still, it doesn't hurt. public synchronized float get() { @@ -202,6 +233,8 @@ public synchronized float get() { /** * Returns progress in this node. get() would give overall progress of the * root node(not just given current node). + * + * @return progress. */ public synchronized float getProgress() { return getInternal(); diff --git a/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/util/ProtoUtil.java b/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/util/ProtoUtil.java index 9807adc50d6d1..883c19c5e7750 100644 --- a/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/util/ProtoUtil.java +++ b/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/util/ProtoUtil.java @@ -83,6 +83,10 @@ public static int readRawVarint32(DataInput in) throws IOException { * as the old connection context as was done for writable where * the effective and real users are set based on the auth method. * + * @param protocol protocol. + * @param ugi ugi. + * @param authMethod authMethod. + * @return IpcConnectionContextProto. */ public static IpcConnectionContextProto makeIpcConnectionContext( final String protocol, diff --git a/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/util/QuickSort.java b/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/util/QuickSort.java index 73d8d90d42507..f3f8b839a0500 100644 --- a/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/util/QuickSort.java +++ b/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/util/QuickSort.java @@ -40,6 +40,9 @@ private static void fix(IndexedSortable s, int p, int r) { /** * Deepest recursion before giving up and doing a heapsort. * Returns 2 * ceil(log(n)). + * + * @param x x. + * @return MaxDepth. */ protected static int getMaxDepth(int x) { if (x <= 0) diff --git a/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/util/ReflectionUtils.java b/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/util/ReflectionUtils.java index 47e44c9e09f8d..155c4f9c5f498 100644 --- a/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/util/ReflectionUtils.java +++ b/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/util/ReflectionUtils.java @@ -21,7 +21,6 @@ import java.io.ByteArrayOutputStream; import java.io.IOException; import java.io.PrintStream; -import java.io.PrintWriter; import java.io.UnsupportedEncodingException; import java.lang.management.ManagementFactory; import java.lang.management.ThreadInfo; @@ -120,6 +119,7 @@ private static void setJobConf(Object theObject, Configuration conf) { * * @param theClass class of which an object is created * @param conf Configuration + * @param Generics Type T. * @return a new object */ @SuppressWarnings("unchecked") @@ -133,6 +133,7 @@ public static T newInstance(Class theClass, Configuration conf) { * @param conf Configuration * @param argTypes the types of the arguments * @param values the values of the arguments + * @param Generics Type. * @return a new object */ @SuppressWarnings("unchecked") @@ -284,6 +285,7 @@ public static void logThreadInfo(Logger log, * Return the correctly-typed {@link Class} of the given object. * * @param o object whose correctly-typed Class is to be obtained + * @param Generics Type T. * @return the correctly typed Class of the given object. */ @SuppressWarnings("unchecked") @@ -332,11 +334,13 @@ private static SerializationFactory getFactory(Configuration conf) { } /** - * Make a copy of the writable object using serialization to a buffer + * Make a copy of the writable object using serialization to a buffer. * @param src the object to copy from * @param dst the object to copy into, which is destroyed + * @param Generics Type. + * @param conf configuration. * @return dst param (the copy) - * @throws IOException + * @throws IOException raised on errors performing I/O. */ @SuppressWarnings("unchecked") public static T copy(Configuration conf, @@ -368,6 +372,9 @@ public static void cloneWritableInto(Writable dst, /** * Gets all the declared fields of a class including fields declared in * superclasses. + * + * @param clazz clazz + * @return field List */ public static List getDeclaredFieldsIncludingInherited(Class clazz) { List fields = new ArrayList(); @@ -390,6 +397,9 @@ public int compare(Field a, Field b) { /** * Gets all the declared methods of a class including methods declared in * superclasses. + * + * @param clazz clazz. + * @return Method List. */ public static List getDeclaredMethodsIncludingInherited(Class clazz) { List methods = new ArrayList(); diff --git a/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/util/RunJar.java b/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/util/RunJar.java index 50126002b7be7..c28e69f54611e 100644 --- a/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/util/RunJar.java +++ b/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/util/RunJar.java @@ -156,6 +156,7 @@ public static void unJar(InputStream inputStream, File toDir, * @param inputStream the jar stream to unpack * @param toDir the destination directory into which to unpack the jar * @param unpackRegex the pattern to match jar entries against + * @param name name. * * @throws IOException if an I/O error has occurred or toDir * cannot be created and does not already exist @@ -231,7 +232,11 @@ private static void ensureDirectory(File dir) throws IOException { } /** Run a Hadoop job jar. If the main class is not in the jar's manifest, - * then it must be provided on the command line. */ + * then it must be provided on the command line. + * + * @param args args. + * @throws Throwable error. + */ public static void main(String[] args) throws Throwable { new RunJar().run(args); } diff --git a/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/util/SequentialNumber.java b/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/util/SequentialNumber.java index 685e92d628136..c3c04493d8be5 100644 --- a/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/util/SequentialNumber.java +++ b/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/util/SequentialNumber.java @@ -30,7 +30,10 @@ public abstract class SequentialNumber implements IdGenerator { private final AtomicLong currentValue; - /** Create a new instance with the given initial value. */ + /** + * Create a new instance with the given initial value. + * @param initialValue initialValue. + */ protected SequentialNumber(final long initialValue) { currentValue = new AtomicLong(initialValue); } @@ -40,7 +43,10 @@ public long getCurrentValue() { return currentValue.get(); } - /** Set current value. */ + /** + * Set current value. + * @param value value. + */ public void setCurrentValue(long value) { currentValue.set(value); } @@ -63,7 +69,12 @@ public long nextValue() { return currentValue.incrementAndGet(); } - /** Skip to the new value. */ + /** + * Skip to the new value. + * @param newValue newValue. + * @throws IllegalStateException + * Cannot skip to less than the current value. + */ public void skipTo(long newValue) throws IllegalStateException { for(;;) { final long c = getCurrentValue(); diff --git a/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/util/ServletUtil.java b/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/util/ServletUtil.java index bb367278e537e..10cac5f49f7e3 100644 --- a/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/util/ServletUtil.java +++ b/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/util/ServletUtil.java @@ -30,7 +30,12 @@ @InterfaceStability.Unstable public class ServletUtil { /** - * Initial HTML header + * Initial HTML header. + * + * @param response response. + * @param title title. + * @throws IOException raised on errors performing I/O. + * @return PrintWriter. */ public static PrintWriter initHTML(ServletResponse response, String title ) throws IOException { @@ -47,6 +52,10 @@ public static PrintWriter initHTML(ServletResponse response, String title /** * Get a parameter from a ServletRequest. * Return null if the parameter contains only white spaces. + * + * @param request request. + * @param name name. + * @return get a parameter from a ServletRequest. */ public static String getParameter(ServletRequest request, String name) { String s = request.getParameter(name); @@ -58,8 +67,13 @@ public static String getParameter(ServletRequest request, String name) { } /** + * parseLongParam. + * + * @param request request. + * @param param param. * @return a long value as passed in the given parameter, throwing * an exception if it is not present or if it is not a valid number. + * @throws IOException raised on errors performing I/O. */ public static long parseLongParam(ServletRequest request, String param) throws IOException { diff --git a/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/util/Sets.java b/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/util/Sets.java index bddcbeb21f26a..8867900d0b692 100644 --- a/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/util/Sets.java +++ b/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/util/Sets.java @@ -55,7 +55,10 @@ private Sets() { * instead. If {@code E} is an {@link Enum} type, use {@link EnumSet#noneOf} * instead. Otherwise, strongly consider using a {@code LinkedHashSet} * instead, at the cost of increased memory footprint, to get - * deterministic iteration behavior. + * deterministic iteration behavior.

      + * + * @param Generics Type E. + * @return a new, empty {@code TreeSet} */ public static HashSet newHashSet() { return new HashSet(); @@ -66,8 +69,9 @@ public static HashSet newHashSet() { * natural sort ordering of its elements. * *

      Note: if mutability is not required, use ImmutableSortedSet#of() - * instead. + * instead.

      * + * @param Generics Type E * @return a new, empty {@code TreeSet} */ public static TreeSet newTreeSet() { @@ -83,11 +87,15 @@ public static TreeSet newTreeSet() { * instead. If {@code E} is an {@link Enum} type, use * {@link EnumSet#of(Enum, Enum[])} instead. Otherwise, strongly consider * using a {@code LinkedHashSet} instead, at the cost of increased memory - * footprint, to get deterministic iteration behavior. + * footprint, to get deterministic iteration behavior.

      * *

      This method is just a small convenience, either for * {@code newHashSet(}{@link Arrays#asList}{@code (...))}, or for creating an - * empty set then calling {@link Collections#addAll}. + * empty set then calling {@link Collections#addAll}.

      + * + * @param Generics Type E. + * @param elements the elements that the set should contain. + * @return a new, empty thread-safe {@code Set} */ @SafeVarargs public static HashSet newHashSet(E... elements) { @@ -103,10 +111,14 @@ public static HashSet newHashSet(E... elements) { * *

      Note: if mutability is not required and the elements are * non-null, use ImmutableSet#copyOf(Iterable) instead. (Or, change - * {@code elements} to be a FluentIterable and call {@code elements.toSet()}.) + * {@code elements} to be a FluentIterable and call {@code elements.toSet()}.)

      * *

      Note: if {@code E} is an {@link Enum} type, use - * newEnumSet(Iterable, Class) instead. + * newEnumSet(Iterable, Class) instead.

      + * + * @param Generics Type E. + * @param elements the elements that the set should contain. + * @return a new, empty thread-safe {@code Set}. */ public static HashSet newHashSet(Iterable elements) { return (elements instanceof Collection) @@ -135,6 +147,7 @@ public static HashSet newHashSet(Iterable elements) { * then calling Iterables#addAll. This method is not very useful and will * likely be deprecated in the future. * + * @param Generics Type E. * @param elements the elements that the set should contain * @return a new {@code TreeSet} containing those elements (minus duplicates) */ @@ -163,13 +176,17 @@ private static boolean addAll(TreeSet addTo, * calling Iterators#addAll. * *

      Note: if mutability is not required and the elements are - * non-null, use ImmutableSet#copyOf(Iterator) instead. + * non-null, use ImmutableSet#copyOf(Iterator) instead.

      * *

      Note: if {@code E} is an {@link Enum} type, you should create - * an {@link EnumSet} instead. + * an {@link EnumSet} instead.

      * *

      Overall, this method is not very useful and will likely be deprecated - * in the future. + * in the future.

      + * + * @param Generics Type E. + * @param elements elements. + * @return a new, empty thread-safe {@code Set}. */ public static HashSet newHashSet(Iterator elements) { HashSet set = newHashSet(); @@ -184,10 +201,11 @@ public static HashSet newHashSet(Iterator elements) { * expect it to do. * *

      This behavior can't be broadly guaranteed, but has been tested with - * OpenJDK 1.7 and 1.8. + * OpenJDK 1.7 and 1.8.

      * * @param expectedSize the number of elements you expect to add to the * returned set + * @param Generics Type E. * @return a new, empty hash set with enough capacity to hold * {@code expectedSize} elements without resizing * @throws IllegalArgumentException if {@code expectedSize} is negative @@ -223,6 +241,11 @@ private static boolean addAll(Collection addTo, *

      Results are undefined if {@code set1} and {@code set2} are sets based * on different equivalence relations (as {@code HashSet}, {@code TreeSet}, * and the keySet of an {@code IdentityHashMap} all are). + * + * @param set1 set1. + * @param set2 set2. + * @param Generics Type E. + * @return a new, empty thread-safe {@code Set}. */ public static Set intersection(final Set set1, final Set set2) { @@ -246,6 +269,11 @@ public static Set intersection(final Set set1, * based on different equivalence relations (as {@link HashSet}, * {@link TreeSet}, and the {@link Map#keySet} of an * {@code IdentityHashMap} all are). + * + * @param set1 set1. + * @param set2 set2. + * @param Generics Type E. + * @return a new, empty thread-safe {@code Set}. */ public static Set union( final Set set1, final Set set2) { @@ -272,6 +300,11 @@ public static Set union( * This method is used to find difference for HashSets. For TreeSets with * strict order requirement, recommended method is * {@link #differenceInTreeSets(Set, Set)}. + * + * @param set1 set1. + * @param set2 set2. + * @param Generics Type E. + * @return a new, empty thread-safe {@code Set}. */ public static Set difference( final Set set1, final Set set2) { @@ -297,6 +330,11 @@ public static Set difference( * * This method is used to find difference for TreeSets. For HashSets, * recommended method is {@link #difference(Set, Set)}. + * + * @param Generics Type E. + * @param set1 set1. + * @param set2 set2. + * @return a new, empty thread-safe {@code Set}. */ public static Set differenceInTreeSets( final Set set1, final Set set2) { @@ -320,6 +358,11 @@ public static Set differenceInTreeSets( *

      Results are undefined if {@code set1} and {@code set2} are sets based * on different equivalence relations (as {@code HashSet}, {@code TreeSet}, * and the keySet of an {@code IdentityHashMap} all are). + * + * @param set1 set1. + * @param set2 set2. + * @param Generics Type E. + * @return a new, empty thread-safe {@code Set}. */ public static Set symmetricDifference( final Set set1, final Set set2) { @@ -345,6 +388,7 @@ public static Set symmetricDifference( *

      Unlike {@code HashSet}, this class does NOT allow {@code null} to be * used as an element. The set is serializable. * + * @param Generics Type. * @return a new, empty thread-safe {@code Set} */ public static Set newConcurrentHashSet() { diff --git a/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/util/Shell.java b/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/util/Shell.java index 084e2b8f5e3b6..bd6bcb08d9c6d 100644 --- a/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/util/Shell.java +++ b/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/util/Shell.java @@ -122,6 +122,7 @@ public static boolean isJavaVersionAtLeast(int version) { * delimiters, no extra count will be added for delimiters. * * @param commands command parts, including any space delimiters + * @throws IOException raised on errors performing I/O. */ public static void checkWindowsCommandLineLength(String...commands) throws IOException { @@ -205,7 +206,11 @@ private static OSType getOSType() { public static final boolean PPC_64 = System.getProperties().getProperty("os.arch").contains("ppc64"); - /** a Unix command to get the current user's groups list. */ + /** + * a Unix command to get the current user's groups list. + * + * @return group command array. + */ public static String[] getGroupsCommand() { return (WINDOWS)? new String[]{"cmd", "/c", "groups"} : new String[]{"groups"}; @@ -216,6 +221,9 @@ public static String[] getGroupsCommand() { * If the OS is not WINDOWS, the command will get the user's primary group * first and finally get the groups list which includes the primary group. * i.e. the user's primary group will be included twice. + * + * @param user user. + * @return groups for user command. */ public static String[] getGroupsForUserCommand(final String user) { //'groups username' command return is inconsistent across different unixes @@ -235,6 +243,9 @@ public static String[] getGroupsForUserCommand(final String user) { * first and finally get the groups list which includes the primary group. * i.e. the user's primary group will be included twice. * This command does not support Windows and will only return group names. + * + * @param user user. + * @return groups id for user command. */ public static String[] getGroupsIDForUserCommand(final String user) { //'groups username' command return is inconsistent across different unixes @@ -248,19 +259,34 @@ public static String[] getGroupsIDForUserCommand(final String user) { } } - /** A command to get a given netgroup's user list. */ + /** + * A command to get a given netgroup's user list. + * + * @param netgroup net group. + * @return users for net group command. + */ public static String[] getUsersForNetgroupCommand(final String netgroup) { //'groups username' command return is non-consistent across different unixes return new String[] {"getent", "netgroup", netgroup}; } - /** Return a command to get permission information. */ + /** + * Return a command to get permission information. + * + * @return permission command. + */ public static String[] getGetPermissionCommand() { return (WINDOWS) ? new String[] { getWinUtilsPath(), "ls", "-F" } : new String[] { "ls", "-ld" }; } - /** Return a command to set permission. */ + /** + * Return a command to set permission. + * + * @param perm permission. + * @param recursive recursive. + * @return set permission command. + */ public static String[] getSetPermissionCommand(String perm, boolean recursive) { if (recursive) { return (WINDOWS) ? @@ -290,21 +316,37 @@ public static String[] getSetPermissionCommand(String perm, return cmdWithFile; } - /** Return a command to set owner. */ + /** + * Return a command to set owner. + * + * @param owner owner. + * @return set owner command. + */ public static String[] getSetOwnerCommand(String owner) { return (WINDOWS) ? new String[] { getWinUtilsPath(), "chown", "\"" + owner + "\"" } : new String[] { "chown", owner }; } - /** Return a command to create symbolic links. */ + /** + * Return a command to create symbolic links. + * + * @param target target. + * @param link link. + * @return symlink command. + */ public static String[] getSymlinkCommand(String target, String link) { return WINDOWS ? new String[] { getWinUtilsPath(), "symlink", link, target } : new String[] { "ln", "-s", target, link }; } - /** Return a command to read the target of the a symbolic link. */ + /** + * Return a command to read the target of the a symbolic link. + * + * @param link link. + * @return read link command. + */ public static String[] getReadlinkCommand(String link) { return WINDOWS ? new String[] { getWinUtilsPath(), "readlink", link } @@ -320,7 +362,13 @@ public static String[] getCheckProcessIsAliveCommand(String pid) { return getSignalKillCommand(0, pid); } - /** Return a command to send a signal to a given pid. */ + /** + * Return a command to send a signal to a given pid. + * + * @param code code. + * @param pid pid. + * @return signal kill command. + */ public static String[] getSignalKillCommand(int code, String pid) { // Code == 0 means check alive if (Shell.WINDOWS) { @@ -347,7 +395,11 @@ public static String[] getSignalKillCommand(int code, String pid) { /** Regular expression for environment variables: {@value}. */ public static final String ENV_NAME_REGEX = "[A-Za-z_][A-Za-z0-9_]*"; - /** Return a regular expression string that match environment variables. */ + /** + * Return a regular expression string that match environment variables. + * + * @return environment variable regex. + */ public static String getEnvironmentVariableRegex() { return (WINDOWS) ? "%(" + ENV_NAME_REGEX + "?)%" @@ -890,7 +942,11 @@ protected void setWorkingDirectory(File dir) { this.dir = dir; } - /** Check to see if a command needs to be executed and execute if needed. */ + /** + * Check to see if a command needs to be executed and execute if needed. + * + * @throws IOException raised on errors performing I/O. + */ protected void run() throws IOException { if (lastTime + interval > Time.monotonicNow()) { return; @@ -902,7 +958,11 @@ protected void run() throws IOException { runCommand(); } - /** Run the command. */ + /** + * Run the command. + * + * @throws IOException raised on errors performing I/O. + */ private void runCommand() throws IOException { ProcessBuilder builder = new ProcessBuilder(getExecString()); Timer timeOutTimer = null; @@ -1049,10 +1109,19 @@ private static void joinThread(Thread t) { } } - /** return an array containing the command name and its parameters. */ + /** + * return an array containing the command name and its parameters. + * + * @return exec string array. + */ protected abstract String[] getExecString(); - /** Parse the execution result */ + /** + * Parse the execution result. + * + * @param lines lines. + * @throws IOException raised on errors performing I/O. + * */ protected abstract void parseExecResult(BufferedReader lines) throws IOException; @@ -1283,6 +1352,7 @@ private void setTimedOut() { * the Shell interface. * @param cmd shell command to execute. * @return the output of the executed command. + * @throws IOException raised on errors performing I/O. */ public static String execCommand(String ... cmd) throws IOException { return execCommand(null, cmd, 0L); @@ -1367,6 +1437,8 @@ public static void destroyAllShellProcesses() { /** * Static method to return a Set of all Shell objects. + * + * @return all shells set. */ public static Set getAllShells() { synchronized (CHILD_SHELLS) { diff --git a/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/util/ShutdownHookManager.java b/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/util/ShutdownHookManager.java index fbdd33331b62b..e85f850514b16 100644 --- a/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/util/ShutdownHookManager.java +++ b/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/util/ShutdownHookManager.java @@ -249,7 +249,7 @@ TimeUnit getTimeUnit() { } private final Set hooks = - Collections.synchronizedSet(new HashSet()); + Collections.synchronizedSet(new HashSet<>()); private AtomicBoolean shutdownInProgress = new AtomicBoolean(false); diff --git a/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/util/ShutdownThreadsHelper.java b/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/util/ShutdownThreadsHelper.java index 16673129cb7fe..dc13697f158ad 100644 --- a/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/util/ShutdownThreadsHelper.java +++ b/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/util/ShutdownThreadsHelper.java @@ -68,10 +68,12 @@ public static boolean shutdownThread(Thread thread, } /** + * shutdownExecutorService. + * * @param service {@link ExecutorService to be shutdown} * @return true if the service is terminated, * false otherwise - * @throws InterruptedException + * @throws InterruptedException if the thread is interrupted. */ public static boolean shutdownExecutorService(ExecutorService service) throws InterruptedException { @@ -79,13 +81,15 @@ public static boolean shutdownExecutorService(ExecutorService service) } /** + * shutdownExecutorService. + * * @param service {@link ExecutorService to be shutdown} * @param timeoutInMs time to wait for {@link * ExecutorService#awaitTermination(long, java.util.concurrent.TimeUnit)} * calls in milli seconds. * @return true if the service is terminated, * false otherwise - * @throws InterruptedException + * @throws InterruptedException if the thread is interrupted. */ public static boolean shutdownExecutorService(ExecutorService service, long timeoutInMs) diff --git a/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/util/StopWatch.java b/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/util/StopWatch.java index c0eedf6110d7f..7ccaebedb5662 100644 --- a/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/util/StopWatch.java +++ b/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/util/StopWatch.java @@ -89,6 +89,9 @@ public StopWatch reset() { } /** + * now. + * + * @param timeUnit timeUnit. * @return current elapsed time in specified timeunit. */ public long now(TimeUnit timeUnit) { diff --git a/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/util/StringInterner.java b/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/util/StringInterner.java index 4a30ee2800aa9..2cbaa2ac1f1a6 100644 --- a/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/util/StringInterner.java +++ b/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/util/StringInterner.java @@ -74,6 +74,9 @@ public static String weakIntern(String sample) { /** * Interns all the strings in the given array in place, * returning the same array. + * + * @param strings strings. + * @return internStringsInArray. */ public static String[] internStringsInArray(String[] strings) { for (int i = 0; i < strings.length; i++) { diff --git a/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/util/StringUtils.java b/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/util/StringUtils.java index baae69f2e1791..b620ba73222ad 100644 --- a/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/util/StringUtils.java +++ b/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/util/StringUtils.java @@ -120,7 +120,12 @@ public static String humanReadableInt(long number) { return TraditionalBinaryPrefix.long2String(number, "", 1); } - /** The same as String.format(Locale.ENGLISH, format, objects). */ + /** + * The same as String.format(Locale.ENGLISH, format, objects). + * @param format format. + * @param objects objects. + * @return format string. + */ public static String format(final String format, final Object... objects) { return String.format(Locale.ENGLISH, format, objects); } @@ -156,7 +161,7 @@ public static String arrayToString(String[] strs) { /** * Given an array of bytes it will convert the bytes to a hex string * representation of the bytes - * @param bytes + * @param bytes bytes. * @param start start index, inclusively * @param end end index, exclusively * @return hex string representation of the byte array @@ -172,7 +177,11 @@ public static String byteToHexString(byte[] bytes, int start, int end) { return s.toString(); } - /** Same as byteToHexString(bytes, 0, bytes.length). */ + /** + * Same as byteToHexString(bytes, 0, bytes.length). + * @param bytes bytes. + * @return byteToHexString. + */ public static String byteToHexString(byte bytes[]) { return byteToHexString(bytes, 0, bytes.length); } @@ -203,8 +212,9 @@ public static byte[] hexStringToByte(String hex) { return bts; } /** - * - * @param uris + * uriToString. + * @param uris uris. + * @return uriToString. */ public static String uriToString(URI[] uris){ if (uris == null) { @@ -242,8 +252,9 @@ public static URI[] stringToURI(String[] str){ } /** - * - * @param str + * stringToPath. + * @param str str. + * @return path array. */ public static Path[] stringToPath(String[] str){ if (str == null) { @@ -263,6 +274,8 @@ public static Path[] stringToPath(String[] str){ * * @param finishTime finish time * @param startTime start time + * @return a String in the format Xhrs, Ymins, Z sec, + * for the time difference between two times. */ public static String formatTimeDiff(long finishTime, long startTime){ long timeDiff = finishTime - startTime; @@ -275,6 +288,7 @@ public static String formatTimeDiff(long finishTime, long startTime){ * String in the format Xhrs, Ymins, Z sec. * * @param timeDiff The time difference to format + * @return formatTime String. */ public static String formatTime(long timeDiff){ StringBuilder buf = new StringBuilder(); @@ -305,6 +319,7 @@ public static String formatTime(long timeDiff){ * more than 100 hours ,it is displayed as 99hrs, 59mins, 59sec. * * @param timeDiff The time difference to format + * @return format time sortable. */ public static String formatTimeSortable(long timeDiff) { StringBuilder buf = new StringBuilder(); @@ -563,6 +578,7 @@ public static String[] split( * @param escapeChar character used to escape * @param start from where to search * @param split used to pass back the extracted string + * @return index. */ public static int findNext(String str, char separator, char escapeChar, int start, StringBuilder split) { @@ -615,7 +631,12 @@ private static boolean hasChar(char[] chars, char character) { } /** + * escapeString. + * + * @param str str. + * @param escapeChar escapeChar. * @param charsToEscape array of characters to be escaped + * @return escapeString. */ public static String escapeString(String str, char escapeChar, char[] charsToEscape) { @@ -658,7 +679,11 @@ public static String unEscapeString( } /** + * unEscapeString. + * @param str str. + * @param escapeChar escapeChar. * @param charsToEscape array of characters to unescape + * @return escape string. */ public static String unEscapeString(String str, char escapeChar, char[] charsToEscape) { @@ -807,7 +832,10 @@ private TraditionalBinaryPrefix(int bitShift) { } /** - * @return The TraditionalBinaryPrefix object corresponding to the symbol. + * The TraditionalBinaryPrefix object corresponding to the symbol. + * + * @param symbol symbol. + * @return traditional binary prefix object. */ public static TraditionalBinaryPrefix valueOf(char symbol) { symbol = Character.toUpperCase(symbol); @@ -907,7 +935,7 @@ public static String long2String(long n, String unit, int decimalPlaces) { /** * Escapes HTML Special characters present in the string. - * @param string + * @param string param string. * @return HTML Escaped String representation */ public static String escapeHTML(String string) { @@ -942,13 +970,22 @@ public static String escapeHTML(String string) { } /** + * a byte description of the given long interger value. + * + * @param len len. * @return a byte description of the given long interger value. */ public static String byteDesc(long len) { return TraditionalBinaryPrefix.long2String(len, "B", 2); } - /** @deprecated use StringUtils.format("%.2f", d). */ + /** + * limitDecimalTo2. + * + * @param d double param. + * @return string value ("%.2f"). + * @deprecated use StringUtils.format("%.2f", d). + */ @Deprecated public static String limitDecimalTo2(double d) { return format("%.2f", d); @@ -959,6 +996,7 @@ public static String limitDecimalTo2(double d) { * * @param separator Separator to join with. * @param strings Strings to join. + * @return join string. */ public static String join(CharSequence separator, Iterable strings) { Iterator i = strings.iterator(); @@ -1054,6 +1092,8 @@ public static String replaceTokens(String template, Pattern pattern, /** * Get stack trace for a given thread. + * @param t thread. + * @return stack trace string. */ public static String getStackTrace(Thread t) { final StackTraceElement[] stackTrace = t.getStackTrace(); diff --git a/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/util/Time.java b/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/util/Time.java index 42005f0b09b3e..f0ce85bbac873 100644 --- a/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/util/Time.java +++ b/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/util/Time.java @@ -81,6 +81,8 @@ public static long monotonicNowNanos() { /** * Convert time in millisecond to human readable format. + * + * @param millis millisecond. * @return a human readable string for the input time */ public static String formatTime(long millis) { diff --git a/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/util/Tool.java b/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/util/Tool.java index a4fbce4ace86f..63c275a1b06bd 100644 --- a/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/util/Tool.java +++ b/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/util/Tool.java @@ -32,7 +32,7 @@ * and only handle its custom arguments.

      * *

      Here is how a typical Tool is implemented:

      - *

      + * 
        *     public class MyApp extends Configured implements Tool {
        *     
        *       public int run(String[] args) throws Exception {
      @@ -69,7 +69,7 @@
        *         System.exit(res);
        *       }
        *     }
      - * 

      + *

      * * @see GenericOptionsParser * @see ToolRunner @@ -82,7 +82,7 @@ public interface Tool extends Configurable { * * @param args command specific arguments. * @return exit code. - * @throws Exception + * @throws Exception command exception. */ int run(String [] args) throws Exception; } diff --git a/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/util/ToolRunner.java b/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/util/ToolRunner.java index 336700a6e276a..b2b57f233a7b2 100644 --- a/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/util/ToolRunner.java +++ b/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/util/ToolRunner.java @@ -57,6 +57,7 @@ public class ToolRunner { * @param tool Tool to run. * @param args command-line arguments to the tool. * @return exit code of the {@link Tool#run(String[])} method. + * @throws Exception Exception. */ public static int run(Configuration conf, Tool tool, String[] args) throws Exception{ @@ -89,6 +90,7 @@ public static int run(Configuration conf, Tool tool, String[] args) * @param tool Tool to run. * @param args command-line arguments to the tool. * @return exit code of the {@link Tool#run(String[])} method. + * @throws Exception exception. */ public static int run(Tool tool, String[] args) throws Exception{ @@ -107,7 +109,13 @@ public static void printGenericCommandUsage(PrintStream out) { /** * Print out a prompt to the user, and return true if the user - * responds with "y" or "yes". (case insensitive) + * responds with "y" or "yes". (case insensitive). + * + * @param prompt prompt. + * @throws IOException raised on errors performing I/O. + * @return if the user + * responds with "y" or "yes". (case insensitive) true, + * not false. */ public static boolean confirmPrompt(String prompt) throws IOException { while (true) { diff --git a/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/util/XMLUtils.java b/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/util/XMLUtils.java index 6d3894f5b4ca5..e2b9e414ad33b 100644 --- a/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/util/XMLUtils.java +++ b/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/util/XMLUtils.java @@ -39,8 +39,10 @@ public class XMLUtils { * @param styleSheet the style-sheet * @param xml input xml data * @param out output - * @throws TransformerConfigurationException - * @throws TransformerException + * @throws TransformerConfigurationException synopsis signals a problem + * creating a transformer object. + * @throws TransformerException this is used for throwing processor + * exceptions before the processing has started. */ public static void transform( InputStream styleSheet, InputStream xml, Writer out diff --git a/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/util/ZKUtil.java b/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/util/ZKUtil.java index 6d38c606c8c4c..17d8233342743 100644 --- a/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/util/ZKUtil.java +++ b/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/util/ZKUtil.java @@ -88,6 +88,7 @@ public static int removeSpecificPerms(int perms, int remove) { * Parse comma separated list of ACL entries to secure generated nodes, e.g. * sasl:hdfs/host1@MY.DOMAIN:cdrwa,sasl:hdfs/host2@MY.DOMAIN:cdrwa * + * @param aclString aclString. * @return ACL list * @throws BadAclFormatException if an ACL is invalid */ diff --git a/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/util/bloom/Key.java b/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/util/bloom/Key.java index e8ad18cfc87e3..e2299365f8e22 100644 --- a/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/util/bloom/Key.java +++ b/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/util/bloom/Key.java @@ -100,8 +100,8 @@ public Key(byte[] value, double weight) { } /** - * @param value - * @param weight + * @param value value. + * @param weight weight. */ public void set(byte[] value, double weight) { if (value == null) { diff --git a/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/util/concurrent/AsyncGet.java b/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/util/concurrent/AsyncGet.java index 9304b483952d0..35ec9115a5f4c 100644 --- a/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/util/concurrent/AsyncGet.java +++ b/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/util/concurrent/AsyncGet.java @@ -52,7 +52,13 @@ R get(long timeout, TimeUnit unit) /** Utility */ class Util { - /** Use {@link #get(long, TimeUnit)} timeout parameters to wait. */ + /** + * Use {@link #get(long, TimeUnit)} timeout parameters to wait. + * @param obj object. + * @param timeout timeout. + * @param unit unit. + * @throws InterruptedException if the thread is interrupted. + */ public static void wait(Object obj, long timeout, TimeUnit unit) throws InterruptedException { if (timeout < 0) { diff --git a/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/util/curator/ZKCuratorManager.java b/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/util/curator/ZKCuratorManager.java index ef9cec6677302..7d35977e5b00d 100644 --- a/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/util/curator/ZKCuratorManager.java +++ b/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/util/curator/ZKCuratorManager.java @@ -83,8 +83,11 @@ public void close() { /** * Utility method to fetch the ZK ACLs from the configuration. + * + * @param conf configuration. * @throws java.io.IOException if the Zookeeper ACLs configuration file * cannot be read + * @return acl list. */ public static List getZKAcls(Configuration conf) throws IOException { // Parse authentication from configuration. @@ -102,9 +105,12 @@ public static List getZKAcls(Configuration conf) throws IOException { /** * Utility method to fetch ZK auth info from the configuration. + * + * @param conf configuration. * @throws java.io.IOException if the Zookeeper ACLs configuration file * cannot be read * @throws ZKUtil.BadAuthFormatException if the auth format is invalid + * @return ZKAuthInfo List. */ public static List getZKAuths(Configuration conf) throws IOException { @@ -167,7 +173,7 @@ public void start(List authInfos) throws IOException { * Get ACLs for a ZNode. * @param path Path of the ZNode. * @return The list of ACLs. - * @throws Exception + * @throws Exception If it cannot contact Zookeeper. */ public List getACL(final String path) throws Exception { return curator.getACL().forPath(path); @@ -186,7 +192,7 @@ public byte[] getData(final String path) throws Exception { /** * Get the data in a ZNode. * @param path Path of the ZNode. - * @param stat + * @param stat stat. * @return The data in the ZNode. * @throws Exception If it cannot contact Zookeeper. */ @@ -363,7 +369,10 @@ public void safeCreate(String path, byte[] data, List acl, /** * Deletes the path. Checks for existence of path as well. + * * @param path Path to be deleted. + * @param fencingNodePath fencingNodePath. + * @param fencingACL fencingACL. * @throws Exception if any problem occurs while performing deletion. */ public void safeDelete(final String path, List fencingACL, diff --git a/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/util/functional/CloseableTaskPoolSubmitter.java b/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/util/functional/CloseableTaskPoolSubmitter.java index 26b687a3c5610..695da7e932279 100644 --- a/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/util/functional/CloseableTaskPoolSubmitter.java +++ b/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/util/functional/CloseableTaskPoolSubmitter.java @@ -34,7 +34,7 @@ */ @InterfaceAudience.Public @InterfaceStability.Unstable -public final class CloseableTaskPoolSubmitter implements TaskPool.Submitter, +public class CloseableTaskPoolSubmitter implements TaskPool.Submitter, Closeable { /** Executors. */ diff --git a/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/util/functional/CommonCallableSupplier.java b/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/util/functional/CommonCallableSupplier.java index 32e299b4d45b1..67299ef96aec6 100644 --- a/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/util/functional/CommonCallableSupplier.java +++ b/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/util/functional/CommonCallableSupplier.java @@ -89,7 +89,9 @@ public static CompletableFuture submit(final Executor executor, /** * Wait for a list of futures to complete. If the list is empty, * return immediately. + * * @param futures list of futures. + * @param Generics Type T. * @throws IOException if one of the called futures raised an IOE. * @throws RuntimeException if one of the futures raised one. */ @@ -105,6 +107,8 @@ public static void waitForCompletion( /** * Wait for a single of future to complete, extracting IOEs afterwards. + * + * @param Generics Type T. * @param future future to wait for. * @throws IOException if one of the called futures raised an IOE. * @throws RuntimeException if one of the futures raised one. @@ -124,6 +128,7 @@ public static void waitForCompletion(final CompletableFuture future) /** * Wait for a single of future to complete, ignoring exceptions raised. * @param future future to wait for. + * @param Generics Type T. */ public static void waitForCompletionIgnoringExceptions( @Nullable final CompletableFuture future) { diff --git a/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/util/functional/RemoteIterators.java b/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/util/functional/RemoteIterators.java index 5fdea4f5b747a..d2f7742d3d988 100644 --- a/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/util/functional/RemoteIterators.java +++ b/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/util/functional/RemoteIterators.java @@ -47,18 +47,18 @@ * with IOStatisticsSource passthrough, and of conversions of * the iterators to lists/arrays and of performing actions * on the values. - *

      + *

      * This aims to make it straightforward to use lambda-expressions to * transform the results of an iterator, without losing the statistics * in the process, and to chain the operations together. - *

      + *

      * The closeable operation will be passed through RemoteIterators which * wrap other RemoteIterators. This is to support any iterator which * can be closed to release held connections, file handles etc. * Unless client code is written to assume that RemoteIterator instances * may be closed, this is not likely to be broadly used. It is added * to make it possible to adopt this feature in a managed way. - *

      + *

      * One notable feature is that the * {@link #foreach(RemoteIterator, ConsumerRaisingIOE)} method will * LOG at debug any IOStatistics provided by the iterator, if such @@ -66,7 +66,7 @@ * if the LOG is not set to debug, so it is a zero cost feature unless * the logger {@code org.apache.hadoop.fs.functional.RemoteIterators} * is at DEBUG. - *

      + *

      * Based on the S3A Listing code, and some some work on moving other code * to using iterative listings so as to pick up the statistics. */ @@ -99,6 +99,7 @@ public static RemoteIterator remoteIteratorFromSingleton( /** * Create a remote iterator from a java.util.Iterator. * @param type + * @param iterator iterator. * @return a remote iterator */ public static RemoteIterator remoteIteratorFromIterator( @@ -110,6 +111,7 @@ public static RemoteIterator remoteIteratorFromIterator( * Create a remote iterator from a java.util.Iterable -e.g. a list * or other collection. * @param type + * @param iterable iterable. * @return a remote iterator */ public static RemoteIterator remoteIteratorFromIterable( @@ -120,6 +122,7 @@ public static RemoteIterator remoteIteratorFromIterable( /** * Create a remote iterator from an array. * @param type + * @param array array. * @return a remote iterator */ public static RemoteIterator remoteIteratorFromArray(T[] array) { @@ -158,10 +161,11 @@ public static RemoteIterator typeCastingRemoteIterator( * Create a RemoteIterator from a RemoteIterator and a filter * function which returns true for every element to be passed * through. - *

      + *

      * Elements are filtered in the hasNext() method; if not used * the filtering will be done on demand in the {@code next()} * call. + *

      * @param type * @param iterator source * @param filter filter @@ -218,16 +222,16 @@ public static T[] toArray(RemoteIterator source, /** * Apply an operation to all values of a RemoteIterator. - *

      + * * If the iterator is an IOStatisticsSource returning a non-null * set of statistics, and this classes log is set to DEBUG, * then the statistics of the operation are evaluated and logged at * debug. - *

      + *

      * The number of entries processed is returned, as it is useful to * know this, especially during tests or when reporting values * to users. - *

      + *

      * This does not close the iterator afterwards. * @param source iterator source * @param consumer consumer of the values. diff --git a/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/util/functional/package-info.java b/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/util/functional/package-info.java index 1c204bb9979a8..18d23b31ff6d7 100644 --- a/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/util/functional/package-info.java +++ b/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/util/functional/package-info.java @@ -18,14 +18,14 @@ /** * Support for functional programming within the Hadoop APIs. - *

      + * * Much of this is needed simply to cope with Java's checked exceptions and * the fact that the java.util.function can only throw runtime exceptions. - *

      + *

      * Pretty much all the Hadoop FS APIs raise IOExceptions, hence the need * for these classes. If Java had made a different decision about the * nature of exceptions, life would be better. - *

      + *

      * Do note that the {@link org.apache.hadoop.util.functional.RemoteIterators} * iterators go beyond that of the java ones, in terms of declaring themselves * Closeable and implementors of diff --git a/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/util/hash/JenkinsHash.java b/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/util/hash/JenkinsHash.java index 3f62aef00a5f6..595a09db3f824 100644 --- a/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/util/hash/JenkinsHash.java +++ b/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/util/hash/JenkinsHash.java @@ -247,7 +247,7 @@ public int hash(byte[] key, int nbytes, int initval) { /** * Compute the hash of the specified file * @param args name of file to compute hash of. - * @throws IOException + * @throws IOException raised on errors performing I/O. */ public static void main(String[] args) throws IOException { if (args.length != 1) { diff --git a/hadoop-common-project/hadoop-common/src/site/markdown/Metrics.md b/hadoop-common-project/hadoop-common/src/site/markdown/Metrics.md index 47786e473a52f..04cbd9fedf83c 100644 --- a/hadoop-common-project/hadoop-common/src/site/markdown/Metrics.md +++ b/hadoop-common-project/hadoop-common/src/site/markdown/Metrics.md @@ -104,6 +104,8 @@ The default timeunit used for RPC metrics is milliseconds (as per the below desc | `rpcLockWaitTime`*num*`s90thPercentileLatency` | Shows the 90th percentile of RPC lock wait time in milliseconds (*num* seconds granularity) if `rpc.metrics.quantile.enable` is set to true. *num* is specified by `rpc.metrics.percentiles.intervals`. | | `rpcLockWaitTime`*num*`s95thPercentileLatency` | Shows the 95th percentile of RPC lock wait time in milliseconds (*num* seconds granularity) if `rpc.metrics.quantile.enable` is set to true. *num* is specified by `rpc.metrics.percentiles.intervals`. | | `rpcLockWaitTime`*num*`s99thPercentileLatency` | Shows the 99th percentile of RPC lock wait time in milliseconds (*num* seconds granularity) if `rpc.metrics.quantile.enable` is set to true. *num* is specified by `rpc.metrics.percentiles.intervals`. | +| `TotalRequests` | Total num of requests served by the RPC server. | +| `TotalRequestsPerSeconds` | Total num of requests per second served by the RPC server. | RetryCache/NameNodeRetryCache ----------------------------- diff --git a/hadoop-common-project/hadoop-common/src/site/markdown/filesystem/fsdatainputstream.md b/hadoop-common-project/hadoop-common/src/site/markdown/filesystem/fsdatainputstream.md index 090696483be34..197b999c81f66 100644 --- a/hadoop-common-project/hadoop-common/src/site/markdown/filesystem/fsdatainputstream.md +++ b/hadoop-common-project/hadoop-common/src/site/markdown/filesystem/fsdatainputstream.md @@ -443,6 +443,45 @@ The semantics of this are exactly equivalent to That is, the buffer is filled entirely with the contents of the input source from position `position` +### `default void readVectored(List ranges, IntFunction allocate)` + +Read fully data for a list of ranges asynchronously. The default implementation +iterates through the ranges, tries to coalesce the ranges based on values of +`minSeekForVectorReads` and `maxReadSizeForVectorReads` and then read each merged +ranges synchronously, but the intent is sub classes can implement efficient +implementation. Reading in both direct and heap byte buffers are supported. +Also, clients are encouraged to use `WeakReferencedElasticByteBufferPool` for +allocating buffers such that even direct buffers are garbage collected when +they are no longer referenced. + +Note: Don't use direct buffers for reading from ChecksumFileSystem as that may +lead to memory fragmentation explained in HADOOP-18296. + + +#### Preconditions + +For each requested range: + + range.getOffset >= 0 else raise IllegalArgumentException + range.getLength >= 0 else raise EOFException + +#### Postconditions + +For each requested range: + + range.getData() returns CompletableFuture which will have data + from range.getOffset to range.getLength. + +### `minSeekForVectorReads()` + +The smallest reasonable seek. Two ranges won't be merged together if the difference between +end of first and start of next range is more than this value. + +### `maxReadSizeForVectorReads()` + +Maximum number of bytes which can be read in one go after merging the ranges. +Two ranges won't be merged if the combined data to be read is more than this value. +Essentially setting this to 0 will disable the merging of ranges. ## Consistency diff --git a/hadoop-common-project/hadoop-common/src/site/markdown/filesystem/fsdatainputstreambuilder.md b/hadoop-common-project/hadoop-common/src/site/markdown/filesystem/fsdatainputstreambuilder.md index db630e05c22d4..16a14150ef949 100644 --- a/hadoop-common-project/hadoop-common/src/site/markdown/filesystem/fsdatainputstreambuilder.md +++ b/hadoop-common-project/hadoop-common/src/site/markdown/filesystem/fsdatainputstreambuilder.md @@ -230,7 +230,7 @@ Note: some operations on the input stream, such as `seek()` may not attempt any at all. Such operations MAY NOT raise exceotions when interacting with nonexistent/unreadable files. -## Standard `openFile()` options since Hadoop 3.3.3 +## Standard `openFile()` options since hadoop branch-3.3 These are options which `FileSystem` and `FileContext` implementation MUST recognise and MAY support by changing the behavior of diff --git a/hadoop-common-project/hadoop-common/src/site/markdown/filesystem/fsdataoutputstreambuilder.md b/hadoop-common-project/hadoop-common/src/site/markdown/filesystem/fsdataoutputstreambuilder.md index 64dda2df8c63c..59a93c5887a1f 100644 --- a/hadoop-common-project/hadoop-common/src/site/markdown/filesystem/fsdataoutputstreambuilder.md +++ b/hadoop-common-project/hadoop-common/src/site/markdown/filesystem/fsdataoutputstreambuilder.md @@ -26,7 +26,7 @@ create a new file or open an existing file on `FileSystem` for write. ## Invariants The `FSDataOutputStreamBuilder` interface does not validate parameters -and modify the state of `FileSystem` until [`build()`](#Builder.build) is +and modify the state of `FileSystem` until `build()` is invoked. ## Implementation-agnostic parameters. @@ -110,7 +110,7 @@ of `FileSystem`. #### Implementation Notes The concrete `FileSystem` and/or `FSDataOutputStreamBuilder` implementation -MUST verify that implementation-agnostic parameters (i.e., "syncable") or +MUST verify that implementation-agnostic parameters (i.e., "syncable`) or implementation-specific parameters (i.e., "foofs:cache") are supported. `FileSystem` will satisfy optional parameters (via `opt(key, ...)`) on best effort. If the mandatory parameters (via `must(key, ...)`) can not be satisfied @@ -182,3 +182,58 @@ see `FileSystem#create(path, ...)` and `FileSystem#append()`. result = FSDataOutputStream The result is `FSDataOutputStream` to be used to write data to filesystem. + + +## S3A-specific options + +Here are the custom options which the S3A Connector supports. + +| Name | Type | Meaning | +|-----------------------------|-----------|----------------------------------------| +| `fs.s3a.create.performance` | `boolean` | create a file with maximum performance | +| `fs.s3a.create.header` | `string` | prefix for user supplied headers | + +### `fs.s3a.create.performance` + +Prioritize file creation performance over safety checks for filesystem consistency. + +This: +1. Skips the `LIST` call which makes sure a file is being created over a directory. + Risk: a file is created over a directory. +1. Ignores the overwrite flag. +1. Never issues a `DELETE` call to delete parent directory markers. + +It is possible to probe an S3A Filesystem instance for this capability through +the `hasPathCapability(path, "fs.s3a.create.performance")` check. + +Creating files with this option over existing directories is likely +to make S3A filesystem clients behave inconsistently. + +Operations optimized for directories (e.g. listing calls) are likely +to see the directory tree not the file; operations optimized for +files (`getFileStatus()`, `isFile()`) more likely to see the file. +The exact form of the inconsistencies, and which operations/parameters +trigger this are undefined and may change between even minor releases. + +Using this option is the equivalent of pressing and holding down the +"Electronic Stability Control" +button on a rear-wheel drive car for five seconds: the safety checks are off. +Things wil be faster if the driver knew what they were doing. +If they didn't, the fact they had held the button down will +be used as evidence at the inquest as proof that they made a +conscious decision to choose speed over safety and +that the outcome was their own fault. + +Accordingly: *Use if and only if you are confident that the conditions are met.* + +### `fs.s3a.create.header` User-supplied header support + +Options with the prefix `fs.s3a.create.header.` will be added to to the +S3 object metadata as "user defined metadata". +This metadata is visible to all applications. It can also be retrieved through the +FileSystem/FileContext `listXAttrs()` and `getXAttrs()` API calls with the prefix `header.` + +When an object is renamed, the metadata is propagated the copy created. + +It is possible to probe an S3A Filesystem instance for this capability through +the `hasPathCapability(path, "fs.s3a.create.header")` check. \ No newline at end of file diff --git a/hadoop-common-project/hadoop-common/src/site/markdown/release/2.10.2/CHANGELOG.2.10.2.md b/hadoop-common-project/hadoop-common/src/site/markdown/release/2.10.2/CHANGELOG.2.10.2.md new file mode 100644 index 0000000000000..a72f1ba6e9894 --- /dev/null +++ b/hadoop-common-project/hadoop-common/src/site/markdown/release/2.10.2/CHANGELOG.2.10.2.md @@ -0,0 +1,277 @@ + + +# "Apache Hadoop" Changelog + +## Release 2.10.2 - 2022-05-31 + + + +### IMPORTANT ISSUES: + +| JIRA | Summary | Priority | Component | Reporter | Contributor | +|:---- |:---- | :--- |:---- |:---- |:---- | +| [HADOOP-17338](https://issues.apache.org/jira/browse/HADOOP-17338) | Intermittent S3AInputStream failures: Premature end of Content-Length delimited message body etc | Major | fs/s3 | Yongjun Zhang | Yongjun Zhang | + + +### NEW FEATURES: + +| JIRA | Summary | Priority | Component | Reporter | Contributor | +|:---- |:---- | :--- |:---- |:---- |:---- | +| [HADOOP-12077](https://issues.apache.org/jira/browse/HADOOP-12077) | Provide a multi-URI replication Inode for ViewFs | Major | fs | Gera Shegalov | Gera Shegalov | +| [HADOOP-13055](https://issues.apache.org/jira/browse/HADOOP-13055) | Implement linkMergeSlash and linkFallback for ViewFileSystem | Major | fs, viewfs | Zhe Zhang | Manoj Govindassamy | + + +### IMPROVEMENTS: + +| JIRA | Summary | Priority | Component | Reporter | Contributor | +|:---- |:---- | :--- |:---- |:---- |:---- | +| [HADOOP-13722](https://issues.apache.org/jira/browse/HADOOP-13722) | Code cleanup -- ViewFileSystem and InodeTree | Minor | viewfs | Manoj Govindassamy | Manoj Govindassamy | +| [YARN-1151](https://issues.apache.org/jira/browse/YARN-1151) | Ability to configure auxiliary services from HDFS-based JAR files | Major | nodemanager | john lilley | Xuan Gong | +| [HADOOP-15584](https://issues.apache.org/jira/browse/HADOOP-15584) | move httpcomponents version in pom.xml | Minor | build | Brandon Scheller | Brandon Scheller | +| [HADOOP-16208](https://issues.apache.org/jira/browse/HADOOP-16208) | Do Not Log InterruptedException in Client | Minor | common | David Mollitor | David Mollitor | +| [HADOOP-16052](https://issues.apache.org/jira/browse/HADOOP-16052) | Remove Subversion and Forrest from Dockerfile | Minor | build | Akira Ajisaka | Xieming Li | +| [YARN-10036](https://issues.apache.org/jira/browse/YARN-10036) | Install yarnpkg and upgrade nodejs in Dockerfile | Major | buid, yarn-ui-v2 | Akira Ajisaka | Akira Ajisaka | +| [HADOOP-16811](https://issues.apache.org/jira/browse/HADOOP-16811) | Use JUnit TemporaryFolder Rule in TestFileUtils | Minor | common, test | David Mollitor | David Mollitor | +| [HADOOP-16054](https://issues.apache.org/jira/browse/HADOOP-16054) | Update Dockerfile to use Bionic | Major | build, test | Akira Ajisaka | Akira Ajisaka | +| [YARN-10451](https://issues.apache.org/jira/browse/YARN-10451) | RM (v1) UI NodesPage can NPE when yarn.io/gpu resource type is defined. | Major | . | Eric Payne | Eric Payne | +| [YARN-9667](https://issues.apache.org/jira/browse/YARN-9667) | Container-executor.c duplicates messages to stdout | Major | nodemanager, yarn | Adam Antal | Peter Bacsko | +| [MAPREDUCE-7301](https://issues.apache.org/jira/browse/MAPREDUCE-7301) | Expose Mini MR Cluster attribute for testing | Minor | test | Swaroopa Kadam | Swaroopa Kadam | +| [HDFS-15567](https://issues.apache.org/jira/browse/HDFS-15567) | [SBN Read] HDFS should expose msync() API to allow downstream applications call it explicitly. | Major | ha, hdfs-client | Konstantin Shvachko | Konstantin Shvachko | +| [YARN-10450](https://issues.apache.org/jira/browse/YARN-10450) | Add cpu and memory utilization per node and cluster-wide metrics | Minor | yarn | Jim Brennan | Jim Brennan | +| [HDFS-15652](https://issues.apache.org/jira/browse/HDFS-15652) | Make block size from NNThroughputBenchmark configurable | Minor | benchmarks | Hui Fei | Hui Fei | +| [HDFS-15665](https://issues.apache.org/jira/browse/HDFS-15665) | Balancer logging improvement | Major | balancer & mover | Konstantin Shvachko | Konstantin Shvachko | +| [HADOOP-17342](https://issues.apache.org/jira/browse/HADOOP-17342) | Creating a token identifier should not do kerberos name resolution | Major | common | Jim Brennan | Jim Brennan | +| [YARN-10479](https://issues.apache.org/jira/browse/YARN-10479) | RMProxy should retry on SocketTimeout Exceptions | Major | yarn | Jim Brennan | Jim Brennan | +| [HDFS-15623](https://issues.apache.org/jira/browse/HDFS-15623) | Respect configured values of rpc.engine | Major | hdfs | Hector Sandoval Chaverri | Hector Sandoval Chaverri | +| [HDFS-15717](https://issues.apache.org/jira/browse/HDFS-15717) | Improve fsck logging | Major | logging, namenode | Ahmed Hussein | Ahmed Hussein | +| [HDFS-15751](https://issues.apache.org/jira/browse/HDFS-15751) | Add documentation for msync() API to filesystem.md | Major | documentation | Konstantin Shvachko | Konstantin Shvachko | +| [YARN-10562](https://issues.apache.org/jira/browse/YARN-10562) | Follow up changes for YARN-9833 | Major | yarn | Jim Brennan | Jim Brennan | +| [HADOOP-17478](https://issues.apache.org/jira/browse/HADOOP-17478) | Improve the description of hadoop.http.authentication.signature.secret.file | Minor | documentation | Akira Ajisaka | Akira Ajisaka | +| [HADOOP-17501](https://issues.apache.org/jira/browse/HADOOP-17501) | Fix logging typo in ShutdownHookManager | Major | common | Konstantin Shvachko | Fengnan Li | +| [HADOOP-17354](https://issues.apache.org/jira/browse/HADOOP-17354) | Move Jenkinsfile outside of the root directory | Major | build | Akira Ajisaka | Akira Ajisaka | +| [MAPREDUCE-7319](https://issues.apache.org/jira/browse/MAPREDUCE-7319) | Log list of mappers at trace level in ShuffleHandler audit log | Minor | yarn | Jim Brennan | Jim Brennan | +| [YARN-10626](https://issues.apache.org/jira/browse/YARN-10626) | Log resource allocation in NM log at container start time | Major | . | Eric Badger | Eric Badger | +| [YARN-10613](https://issues.apache.org/jira/browse/YARN-10613) | Config to allow Intra- and Inter-queue preemption to enable/disable conservativeDRF | Minor | capacity scheduler, scheduler preemption | Eric Payne | Eric Payne | +| [MAPREDUCE-7324](https://issues.apache.org/jira/browse/MAPREDUCE-7324) | ClientHSSecurityInfo class is in wrong META-INF file | Major | . | Eric Badger | Eric Badger | +| [HADOOP-17546](https://issues.apache.org/jira/browse/HADOOP-17546) | Update Description of hadoop-http-auth-signature-secret in HttpAuthentication.md | Minor | . | Ravuri Sushma sree | Ravuri Sushma sree | +| [HADOOP-17570](https://issues.apache.org/jira/browse/HADOOP-17570) | Apply YETUS-1102 to re-enable GitHub comments | Major | build | Akira Ajisaka | Akira Ajisaka | +| [HADOOP-16870](https://issues.apache.org/jira/browse/HADOOP-16870) | Use spotbugs-maven-plugin instead of findbugs-maven-plugin | Major | build | Akira Ajisaka | Akira Ajisaka | +| [HDFS-15931](https://issues.apache.org/jira/browse/HDFS-15931) | Fix non-static inner classes for better memory management | Major | . | Viraj Jasani | Viraj Jasani | +| [HADOOP-17680](https://issues.apache.org/jira/browse/HADOOP-17680) | Allow ProtobufRpcEngine to be extensible | Major | common | Hector Sandoval Chaverri | Hector Sandoval Chaverri | +| [HADOOP-17756](https://issues.apache.org/jira/browse/HADOOP-17756) | Increase precommit job timeout from 20 hours to 24 hours. | Major | build | Takanobu Asanuma | Takanobu Asanuma | +| [HDFS-15150](https://issues.apache.org/jira/browse/HDFS-15150) | Introduce read write lock to Datanode | Major | datanode | Stephen O'Donnell | Stephen O'Donnell | +| [YARN-10834](https://issues.apache.org/jira/browse/YARN-10834) | Intra-queue preemption: apps that don't use defined custom resource won't be preempted. | Major | . | Eric Payne | Eric Payne | +| [HADOOP-17775](https://issues.apache.org/jira/browse/HADOOP-17775) | Remove JavaScript package from Docker environment | Major | build | Masatake Iwasaki | Masatake Iwasaki | +| [HADOOP-12665](https://issues.apache.org/jira/browse/HADOOP-12665) | Document hadoop.security.token.service.use\_ip | Major | documentation | Arpit Agarwal | Akira Ajisaka | +| [YARN-10456](https://issues.apache.org/jira/browse/YARN-10456) | RM PartitionQueueMetrics records are named QueueMetrics in Simon metrics registry | Major | resourcemanager | Eric Payne | Eric Payne | +| [YARN-10860](https://issues.apache.org/jira/browse/YARN-10860) | Make max container per heartbeat configs refreshable | Major | . | Eric Badger | Eric Badger | +| [HADOOP-17813](https://issues.apache.org/jira/browse/HADOOP-17813) | Checkstyle - Allow line length: 100 | Major | . | Akira Ajisaka | Viraj Jasani | +| [HADOOP-17819](https://issues.apache.org/jira/browse/HADOOP-17819) | Add extensions to ProtobufRpcEngine RequestHeaderProto | Major | common | Hector Sandoval Chaverri | Hector Sandoval Chaverri | +| [HADOOP-17897](https://issues.apache.org/jira/browse/HADOOP-17897) | Allow nested blocks in switch case in checkstyle settings | Minor | build | Masatake Iwasaki | Masatake Iwasaki | +| [HADOOP-17857](https://issues.apache.org/jira/browse/HADOOP-17857) | Check real user ACLs in addition to proxied user ACLs | Major | . | Eric Payne | Eric Payne | +| [HADOOP-17926](https://issues.apache.org/jira/browse/HADOOP-17926) | Maven-eclipse-plugin is no longer needed since Eclipse can import Maven projects by itself. | Minor | documentation | Rintaro Ikeda | Rintaro Ikeda | +| [YARN-10935](https://issues.apache.org/jira/browse/YARN-10935) | AM Total Queue Limit goes below per-user AM Limit if parent is full. | Major | capacity scheduler, capacityscheduler | Eric Payne | Eric Payne | +| [HDFS-16257](https://issues.apache.org/jira/browse/HDFS-16257) | [HDFS] [RBF] Guava cache performance issue in Router MountTableResolver | Major | . | Janus Chow | Janus Chow | +| [YARN-1115](https://issues.apache.org/jira/browse/YARN-1115) | Provide optional means for a scheduler to check real user ACLs | Major | capacity scheduler, scheduler | Eric Payne | | +| [HDFS-16294](https://issues.apache.org/jira/browse/HDFS-16294) | Remove invalid DataNode#CONFIG\_PROPERTY\_SIMULATED | Major | datanode | JiangHua Zhu | JiangHua Zhu | +| [HDFS-16301](https://issues.apache.org/jira/browse/HDFS-16301) | Improve BenchmarkThroughput#SIZE naming standardization | Minor | benchmarks, test | JiangHua Zhu | JiangHua Zhu | +| [HADOOP-18035](https://issues.apache.org/jira/browse/HADOOP-18035) | Skip unit test failures to run all the unit tests | Major | build | Akira Ajisaka | Akira Ajisaka | +| [HADOOP-18040](https://issues.apache.org/jira/browse/HADOOP-18040) | Use maven.test.failure.ignore instead of ignoreTestFailure | Major | build | Akira Ajisaka | Akira Ajisaka | +| [YARN-8234](https://issues.apache.org/jira/browse/YARN-8234) | Improve RM system metrics publisher's performance by pushing events to timeline server in batch | Critical | resourcemanager, timelineserver | Hu Ziqian | Ashutosh Gupta | +| [HADOOP-18093](https://issues.apache.org/jira/browse/HADOOP-18093) | Better exception handling for testFileStatusOnMountLink() in ViewFsBaseTest.java | Trivial | . | Xing Lin | Xing Lin | +| [HADOOP-18099](https://issues.apache.org/jira/browse/HADOOP-18099) | Upgrade bundled Tomcat to 8.5.75 | Major | httpfs, kms | Akira Ajisaka | Ashutosh Gupta | +| [HADOOP-18110](https://issues.apache.org/jira/browse/HADOOP-18110) | ViewFileSystem: Add Support for Localized Trash Root | Major | common | Xing Lin | Xing Lin | +| [HADOOP-18144](https://issues.apache.org/jira/browse/HADOOP-18144) | getTrashRoot/s in ViewFileSystem should return viewFS path, not targetFS path | Major | common | Xing Lin | Xing Lin | +| [HADOOP-18136](https://issues.apache.org/jira/browse/HADOOP-18136) | Verify FileUtils.unTar() handling of missing .tar files | Minor | test, util | Steve Loughran | Steve Loughran | +| [HDFS-16529](https://issues.apache.org/jira/browse/HDFS-16529) | Remove unnecessary setObserverRead in TestConsistentReadsObserver | Trivial | test | wangzhaohui | wangzhaohui | +| [HADOOP-18155](https://issues.apache.org/jira/browse/HADOOP-18155) | Refactor tests in TestFileUtil | Trivial | common | Gautham Banasandra | Gautham Banasandra | +| [HADOOP-18088](https://issues.apache.org/jira/browse/HADOOP-18088) | Replace log4j 1.x with reload4j | Major | . | Wei-Chiu Chuang | Wei-Chiu Chuang | +| [YARN-11125](https://issues.apache.org/jira/browse/YARN-11125) | Backport YARN-6483 to branch-2.10 | Major | resourcemanager | Ashutosh Gupta | Ashutosh Gupta | +| [HADOOP-18172](https://issues.apache.org/jira/browse/HADOOP-18172) | Change scope of getRootFallbackLink for InodeTree to make them accessible from outside package | Minor | . | Xing Lin | Xing Lin | + + +### BUG FIXES: + +| JIRA | Summary | Priority | Component | Reporter | Contributor | +|:---- |:---- | :--- |:---- |:---- |:---- | +| [HDFS-10650](https://issues.apache.org/jira/browse/HDFS-10650) | DFSClient#mkdirs and DFSClient#primitiveMkdir should use default directory permission | Minor | . | John Zhuge | John Zhuge | +| [HADOOP-13965](https://issues.apache.org/jira/browse/HADOOP-13965) | Groups should be consistent in using default group mapping class | Minor | security | Yiqun Lin | Yiqun Lin | +| [YARN-6977](https://issues.apache.org/jira/browse/YARN-6977) | Node information is not provided for non am containers in RM logs | Major | capacity scheduler | Sumana Sathish | Suma Shivaprasad | +| [HADOOP-15261](https://issues.apache.org/jira/browse/HADOOP-15261) | Upgrade commons-io from 2.4 to 2.5 | Major | minikdc | PandaMonkey | PandaMonkey | +| [HADOOP-15331](https://issues.apache.org/jira/browse/HADOOP-15331) | Fix a race condition causing parsing error of java.io.BufferedInputStream in class org.apache.hadoop.conf.Configuration | Major | common | Miklos Szegedi | Miklos Szegedi | +| [YARN-8222](https://issues.apache.org/jira/browse/YARN-8222) | Fix potential NPE when gets RMApp from RM context | Critical | . | Tao Yang | Tao Yang | +| [HDFS-13174](https://issues.apache.org/jira/browse/HDFS-13174) | hdfs mover -p /path times out after 20 min | Major | balancer & mover | István Fajth | István Fajth | +| [HDFS-13723](https://issues.apache.org/jira/browse/HDFS-13723) | Occasional "Should be different group" error in TestRefreshUserMappings#testGroupMappingRefresh | Major | security, test | Siyao Meng | Siyao Meng | +| [YARN-7266](https://issues.apache.org/jira/browse/YARN-7266) | Timeline Server event handler threads locked | Major | ATSv2, timelineserver | Venkata Puneet Ravuri | Prabhu Joseph | +| [HDFS-13677](https://issues.apache.org/jira/browse/HDFS-13677) | Dynamic refresh Disk configuration results in overwriting VolumeMap | Blocker | . | ZanderXu | ZanderXu | +| [HADOOP-16334](https://issues.apache.org/jira/browse/HADOOP-16334) | Fix yetus-wrapper not working when HADOOP\_YETUS\_VERSION \>= 0.9.0 | Major | yetus | Wanqiang Ji | Wanqiang Ji | +| [YARN-9594](https://issues.apache.org/jira/browse/YARN-9594) | Fix missing break statement in ContainerScheduler#handle | Major | . | lujie | lujie | +| [YARN-9744](https://issues.apache.org/jira/browse/YARN-9744) | RollingLevelDBTimelineStore.getEntityByTime fails with NPE | Major | timelineserver | Prabhu Joseph | Prabhu Joseph | +| [YARN-9785](https://issues.apache.org/jira/browse/YARN-9785) | Fix DominantResourceCalculator when one resource is zero | Blocker | . | Bilwa S T | Bilwa S T | +| [YARN-9833](https://issues.apache.org/jira/browse/YARN-9833) | Race condition when DirectoryCollection.checkDirs() runs during container launch | Major | . | Peter Bacsko | Peter Bacsko | +| [HDFS-14216](https://issues.apache.org/jira/browse/HDFS-14216) | NullPointerException happens in NamenodeWebHdfs | Critical | . | lujie | lujie | +| [YARN-9984](https://issues.apache.org/jira/browse/YARN-9984) | FSPreemptionThread can cause NullPointerException while app is unregistered with containers running on a node | Major | fairscheduler | Wilfred Spiegelenburg | Wilfred Spiegelenburg | +| [HADOOP-16841](https://issues.apache.org/jira/browse/HADOOP-16841) | The description of hadoop.http.authentication.signature.secret.file contains outdated information | Minor | documentation | Akira Ajisaka | Xieming Li | +| [HADOOP-16768](https://issues.apache.org/jira/browse/HADOOP-16768) | SnappyCompressor test cases wrongly assume that the compressed data is always smaller than the input data | Major | io, test | zhao bo | Akira Ajisaka | +| [HADOOP-17068](https://issues.apache.org/jira/browse/HADOOP-17068) | client fails forever when namenode ipaddr changed | Major | hdfs-client | Sean Chow | Sean Chow | +| [HADOOP-17116](https://issues.apache.org/jira/browse/HADOOP-17116) | Skip Retry INFO logging on first failover from a proxy | Major | ha | Hanisha Koneru | Hanisha Koneru | +| [MAPREDUCE-7294](https://issues.apache.org/jira/browse/MAPREDUCE-7294) | Only application master should upload resource to Yarn Shared Cache | Major | mrv2 | zhenzhao wang | zhenzhao wang | +| [YARN-10438](https://issues.apache.org/jira/browse/YARN-10438) | Handle null containerId in ClientRMService#getContainerReport() | Major | resourcemanager | Raghvendra Singh | Shubham Gupta | +| [MAPREDUCE-7289](https://issues.apache.org/jira/browse/MAPREDUCE-7289) | Fix wrong comment in LongLong.java | Trivial | documentation, examples | Akira Ajisaka | Wanqiang Ji | +| [YARN-10393](https://issues.apache.org/jira/browse/YARN-10393) | MR job live lock caused by completed state container leak in heartbeat between node manager and RM | Major | nodemanager, yarn | zhenzhao wang | Jim Brennan | +| [YARN-10455](https://issues.apache.org/jira/browse/YARN-10455) | TestNMProxy.testNMProxyRPCRetry is not consistent | Major | . | Ahmed Hussein | Ahmed Hussein | +| [HADOOP-17223](https://issues.apache.org/jira/browse/HADOOP-17223) | update org.apache.httpcomponents:httpclient to 4.5.13 and httpcore to 4.4.13 | Blocker | . | Pranav Bheda | Pranav Bheda | +| [HADOOP-17309](https://issues.apache.org/jira/browse/HADOOP-17309) | Javadoc warnings and errors are ignored in the precommit jobs | Major | build, documentation | Akira Ajisaka | Akira Ajisaka | +| [MAPREDUCE-7302](https://issues.apache.org/jira/browse/MAPREDUCE-7302) | Upgrading to JUnit 4.13 causes testcase TestFetcher.testCorruptedIFile() to fail | Major | test | Peter Bacsko | Peter Bacsko | +| [HDFS-15644](https://issues.apache.org/jira/browse/HDFS-15644) | Failed volumes can cause DNs to stop block reporting | Major | block placement, datanode | Ahmed Hussein | Ahmed Hussein | +| [YARN-10467](https://issues.apache.org/jira/browse/YARN-10467) | ContainerIdPBImpl objects can be leaked in RMNodeImpl.completedContainers | Major | resourcemanager | Haibo Chen | Haibo Chen | +| [HADOOP-17340](https://issues.apache.org/jira/browse/HADOOP-17340) | TestLdapGroupsMapping failing -string mismatch in exception validation | Major | test | Steve Loughran | Steve Loughran | +| [HADOOP-17352](https://issues.apache.org/jira/browse/HADOOP-17352) | Update PATCH\_NAMING\_RULE in the personality file | Minor | build | Akira Ajisaka | Akira Ajisaka | +| [HADOOP-17358](https://issues.apache.org/jira/browse/HADOOP-17358) | Improve excessive reloading of Configurations | Major | conf | Ahmed Hussein | Ahmed Hussein | +| [YARN-8558](https://issues.apache.org/jira/browse/YARN-8558) | NM recovery level db not cleaned up properly on container finish | Critical | . | Bibin Chundatt | Bibin Chundatt | +| [YARN-10498](https://issues.apache.org/jira/browse/YARN-10498) | Fix Yarn CapacityScheduler Markdown document | Trivial | documentation | zhaoshengjie | zhaoshengjie | +| [HDFS-15660](https://issues.apache.org/jira/browse/HDFS-15660) | StorageTypeProto is not compatiable between 3.x and 2.6 | Major | . | Ryan Wu | Ryan Wu | +| [HDFS-15725](https://issues.apache.org/jira/browse/HDFS-15725) | Lease Recovery never completes for a committed block which the DNs never finalize | Major | namenode | Stephen O'Donnell | Stephen O'Donnell | +| [HADOOP-17438](https://issues.apache.org/jira/browse/HADOOP-17438) | Increase docker memory limit in Jenkins | Major | build, scripts, test, yetus | Ahmed Hussein | Ahmed Hussein | +| [HADOOP-16947](https://issues.apache.org/jira/browse/HADOOP-16947) | Stale record should be remove when MutableRollingAverages generating aggregate data. | Major | . | Haibin Huang | Haibin Huang | +| [HDFS-15632](https://issues.apache.org/jira/browse/HDFS-15632) | AbstractContractDeleteTest should set recursive parameter to true for recursive test cases. | Major | . | Konstantin Shvachko | Anton Kutuzov | +| [HDFS-10498](https://issues.apache.org/jira/browse/HDFS-10498) | Intermittent test failure org.apache.hadoop.hdfs.server.namenode.snapshot.TestSnapshotFileLength.testSnapshotfileLength | Major | hdfs, snapshots | Hanisha Koneru | Jim Brennan | +| [HADOOP-17495](https://issues.apache.org/jira/browse/HADOOP-17495) | Backport HADOOP-16947 "Stale record should be remove when MutableRollingAverages generating aggregate data." to branch 2.10 | Major | . | Felix N | Felix N | +| [HDFS-15801](https://issues.apache.org/jira/browse/HDFS-15801) | Backport HDFS-14582 to branch-2.10 (Failed to start DN with ArithmeticException when NULL checksum used) | Major | . | Janus Chow | Janus Chow | +| [YARN-10428](https://issues.apache.org/jira/browse/YARN-10428) | Zombie applications in the YARN queue using FAIR + sizebasedweight | Critical | capacityscheduler | Guang Yang | Andras Gyori | +| [HDFS-15792](https://issues.apache.org/jira/browse/HDFS-15792) | ClasscastException while loading FSImage | Major | nn | Renukaprasad C | Renukaprasad C | +| [HADOOP-17516](https://issues.apache.org/jira/browse/HADOOP-17516) | Upgrade ant to 1.10.9 | Major | . | Akira Ajisaka | Akira Ajisaka | +| [YARN-10500](https://issues.apache.org/jira/browse/YARN-10500) | TestDelegationTokenRenewer fails intermittently | Major | test | Akira Ajisaka | Masatake Iwasaki | +| [MAPREDUCE-7323](https://issues.apache.org/jira/browse/MAPREDUCE-7323) | Remove job\_history\_summary.py | Major | . | Akira Ajisaka | Akira Ajisaka | +| [HADOOP-17510](https://issues.apache.org/jira/browse/HADOOP-17510) | Hadoop prints sensitive Cookie information. | Major | . | Renukaprasad C | Renukaprasad C | +| [HDFS-15422](https://issues.apache.org/jira/browse/HDFS-15422) | Reported IBR is partially replaced with stored info when queuing. | Critical | namenode | Kihwal Lee | Stephen O'Donnell | +| [YARN-10651](https://issues.apache.org/jira/browse/YARN-10651) | CapacityScheduler crashed with NPE in AbstractYarnScheduler.updateNodeResource() | Major | . | Haibo Chen | Haibo Chen | +| [MAPREDUCE-7320](https://issues.apache.org/jira/browse/MAPREDUCE-7320) | ClusterMapReduceTestCase does not clean directories | Major | . | Ahmed Hussein | Ahmed Hussein | +| [HDFS-15849](https://issues.apache.org/jira/browse/HDFS-15849) | ExpiredHeartbeats metric should be of Type.COUNTER | Major | metrics | Konstantin Shvachko | Qi Zhu | +| [HADOOP-17557](https://issues.apache.org/jira/browse/HADOOP-17557) | skip-dir option is not processed by Yetus | Major | build, precommit, yetus | Ahmed Hussein | Ahmed Hussein | +| [HADOOP-17572](https://issues.apache.org/jira/browse/HADOOP-17572) | [branch-2.10] Docker image build fails due to the removal of openjdk-7-jdk package | Blocker | build | Akira Ajisaka | Akira Ajisaka | +| [HADOOP-17582](https://issues.apache.org/jira/browse/HADOOP-17582) | Replace GitHub App Token with GitHub OAuth token | Major | build | Akira Ajisaka | Akira Ajisaka | +| [HADOOP-17586](https://issues.apache.org/jira/browse/HADOOP-17586) | Upgrade org.codehaus.woodstox:stax2-api to 4.2.1 | Major | . | Ayush Saxena | Ayush Saxena | +| [YARN-10588](https://issues.apache.org/jira/browse/YARN-10588) | Percentage of queue and cluster is zero in WebUI | Major | . | Bilwa S T | Bilwa S T | +| [MAPREDUCE-7322](https://issues.apache.org/jira/browse/MAPREDUCE-7322) | revisiting TestMRIntermediateDataEncryption | Major | job submission, security, test | Ahmed Hussein | Ahmed Hussein | +| [HADOOP-17592](https://issues.apache.org/jira/browse/HADOOP-17592) | Fix the wrong CIDR range example in Proxy User documentation | Minor | documentation | Kwangsun Noh | Kwangsun Noh | +| [MAPREDUCE-7325](https://issues.apache.org/jira/browse/MAPREDUCE-7325) | Intermediate data encryption is broken in LocalJobRunner | Major | job submission, security | Ahmed Hussein | Ahmed Hussein | +| [YARN-10697](https://issues.apache.org/jira/browse/YARN-10697) | Resources are displayed in bytes in UI for schedulers other than capacity | Major | . | Bilwa S T | Bilwa S T | +| [HADOOP-17602](https://issues.apache.org/jira/browse/HADOOP-17602) | Upgrade JUnit to 4.13.1 | Major | build, security, test | Ahmed Hussein | Ahmed Hussein | +| [MAPREDUCE-7332](https://issues.apache.org/jira/browse/MAPREDUCE-7332) | Fix SpillCallBackPathsFinder to use JDK7 on branch-2.10 | Minor | job submission, security | Ahmed Hussein | Ahmed Hussein | +| [YARN-10501](https://issues.apache.org/jira/browse/YARN-10501) | Can't remove all node labels after add node label without nodemanager port | Critical | yarn | caozhiqiang | caozhiqiang | +| [YARN-10716](https://issues.apache.org/jira/browse/YARN-10716) | Fix typo in ContainerRuntime | Trivial | documentation | Wanqiang Ji | xishuhai | +| [HADOOP-17603](https://issues.apache.org/jira/browse/HADOOP-17603) | Upgrade tomcat-embed-core to 7.0.108 | Major | build, security | Ahmed Hussein | Ahmed Hussein | +| [HADOOP-17601](https://issues.apache.org/jira/browse/HADOOP-17601) | Upgrade Jackson databind in branch-2.10 to 2.9.10.7 | Major | . | Ahmed Hussein | Ahmed Hussein | +| [YARN-10733](https://issues.apache.org/jira/browse/YARN-10733) | TimelineService Hbase tests are failing with timeout error on branch-2.10 | Major | test, timelineserver, yarn | Ahmed Hussein | Ahmed Hussein | +| [HDFS-15963](https://issues.apache.org/jira/browse/HDFS-15963) | Unreleased volume references cause an infinite loop | Critical | datanode | Shuyan Zhang | Shuyan Zhang | +| [YARN-10460](https://issues.apache.org/jira/browse/YARN-10460) | Upgrading to JUnit 4.13 causes tests in TestNodeStatusUpdater to fail | Major | nodemanager, test | Peter Bacsko | Peter Bacsko | +| [YARN-10749](https://issues.apache.org/jira/browse/YARN-10749) | Can't remove all node labels after add node label without nodemanager port, broken by YARN-10647 | Major | . | D M Murali Krishna Reddy | D M Murali Krishna Reddy | +| [YARN-10555](https://issues.apache.org/jira/browse/YARN-10555) | Missing access check before getAppAttempts | Critical | webapp | lujie | lujie | +| [HADOOP-17718](https://issues.apache.org/jira/browse/HADOOP-17718) | Explicitly set locale in the Dockerfile | Blocker | build | Wei-Chiu Chuang | Wei-Chiu Chuang | +| [YARN-10770](https://issues.apache.org/jira/browse/YARN-10770) | container-executor permission is wrong in SecureContainer.md | Major | documentation | Akira Ajisaka | Siddharth Ahuja | +| [HDFS-15915](https://issues.apache.org/jira/browse/HDFS-15915) | Race condition with async edits logging due to updating txId outside of the namesystem log | Major | hdfs, namenode | Konstantin Shvachko | Konstantin Shvachko | +| [HDFS-16040](https://issues.apache.org/jira/browse/HDFS-16040) | RpcQueueTime metric counts requeued calls as unique events. | Major | hdfs | Simbarashe Dzinamarira | Simbarashe Dzinamarira | +| [HDFS-16068](https://issues.apache.org/jira/browse/HDFS-16068) | WebHdfsFileSystem has a possible connection leak in connection with HttpFS | Major | . | Takanobu Asanuma | Takanobu Asanuma | +| [HDFS-15618](https://issues.apache.org/jira/browse/HDFS-15618) | Improve datanode shutdown latency | Major | datanode | Ahmed Hussein | Ahmed Hussein | +| [HADOOP-17769](https://issues.apache.org/jira/browse/HADOOP-17769) | Upgrade JUnit to 4.13.2 | Major | . | Ahmed Hussein | Ahmed Hussein | +| [YARN-10824](https://issues.apache.org/jira/browse/YARN-10824) | Title not set for JHS and NM webpages | Major | . | Rajshree Mishra | Bilwa S T | +| [MAPREDUCE-7353](https://issues.apache.org/jira/browse/MAPREDUCE-7353) | Mapreduce job fails when NM is stopped | Major | . | Bilwa S T | Bilwa S T | +| [HADOOP-17793](https://issues.apache.org/jira/browse/HADOOP-17793) | Better token validation | Major | . | Artem Smotrakov | Artem Smotrakov | +| [HDFS-16042](https://issues.apache.org/jira/browse/HDFS-16042) | DatanodeAdminMonitor scan should be delay based | Major | datanode | Ahmed Hussein | Ahmed Hussein | +| [HADOOP-17028](https://issues.apache.org/jira/browse/HADOOP-17028) | ViewFS should initialize target filesystems lazily | Major | client-mounts, fs, viewfs | Uma Maheswara Rao G | Abhishek Das | +| [HADOOP-17370](https://issues.apache.org/jira/browse/HADOOP-17370) | Upgrade commons-compress to 1.21 | Major | common | Dongjoon Hyun | Akira Ajisaka | +| [HADOOP-17886](https://issues.apache.org/jira/browse/HADOOP-17886) | Upgrade ant to 1.10.11 | Major | . | Ahmed Hussein | Ahmed Hussein | +| [HADOOP-17885](https://issues.apache.org/jira/browse/HADOOP-17885) | Upgrade JSON smart to 1.3.3 on branch-2.10 | Major | . | Ahmed Hussein | Ahmed Hussein | +| [HDFS-16207](https://issues.apache.org/jira/browse/HDFS-16207) | Remove NN logs stack trace for non-existent xattr query | Major | namenode | Ahmed Hussein | Ahmed Hussein | +| [HDFS-16198](https://issues.apache.org/jira/browse/HDFS-16198) | Short circuit read leaks Slot objects when InvalidToken exception is thrown | Major | . | Eungsop Yoo | Eungsop Yoo | +| [HDFS-16233](https://issues.apache.org/jira/browse/HDFS-16233) | Do not use exception handler to implement copy-on-write for EnumCounters | Major | namenode | Wei-Chiu Chuang | Wei-Chiu Chuang | +| [HDFS-16181](https://issues.apache.org/jira/browse/HDFS-16181) | [SBN Read] Fix metric of RpcRequestCacheMissAmount can't display when tailEditLog form JN | Critical | . | wangzhaohui | wangzhaohui | +| [YARN-8127](https://issues.apache.org/jira/browse/YARN-8127) | Resource leak when async scheduling is enabled | Critical | . | Weiwei Yang | Tao Yang | +| [HADOOP-17964](https://issues.apache.org/jira/browse/HADOOP-17964) | Increase Java heap size for running Maven in Dockerfile of branch-2.10 | Major | build | Masatake Iwasaki | Masatake Iwasaki | +| [HADOOP-16532](https://issues.apache.org/jira/browse/HADOOP-16532) | Fix TestViewFsTrash to use the correct homeDir. | Minor | test, viewfs | Steve Loughran | Xing Lin | +| [HADOOP-17965](https://issues.apache.org/jira/browse/HADOOP-17965) | Fix documentation build failure using JDK 7 on branch-2.10 | Major | build | Masatake Iwasaki | Masatake Iwasaki | +| [HDFS-7612](https://issues.apache.org/jira/browse/HDFS-7612) | TestOfflineEditsViewer.testStored() uses incorrect default value for cacheDir | Major | test | Konstantin Shvachko | Michael Kuchenbecker | +| [HADOOP-17978](https://issues.apache.org/jira/browse/HADOOP-17978) | Exclude ASF license check for pkg-resolver JSON | Major | build | Gautham Banasandra | Gautham Banasandra | +| [HADOOP-17880](https://issues.apache.org/jira/browse/HADOOP-17880) | Build Hadoop on Centos 7 | Major | build | baizhendong | | +| [HADOOP-17988](https://issues.apache.org/jira/browse/HADOOP-17988) | Disable JIRA plugin for YETUS on Hadoop | Critical | build | Gautham Banasandra | Gautham Banasandra | +| [HDFS-16311](https://issues.apache.org/jira/browse/HDFS-16311) | Metric metadataOperationRate calculation error in DataNodeVolumeMetrics | Major | . | Tao Li | Tao Li | +| [HADOOP-17999](https://issues.apache.org/jira/browse/HADOOP-17999) | No-op implementation of setWriteChecksum and setVerifyChecksum in ViewFileSystem | Major | . | Abhishek Das | Abhishek Das | +| [YARN-9063](https://issues.apache.org/jira/browse/YARN-9063) | ATS 1.5 fails to start if RollingLevelDb files are corrupt or missing | Major | timelineserver, timelineservice | Tarun Parimi | Ashutosh Gupta | +| [HADOOP-18049](https://issues.apache.org/jira/browse/HADOOP-18049) | Pin python lazy-object-proxy to 1.6.0 in Docker file as newer versions are incompatible with python2.7 | Major | build | Dhananjay Badaya | Dhananjay Badaya | +| [HADOOP-13500](https://issues.apache.org/jira/browse/HADOOP-13500) | Synchronizing iteration of Configuration properties object | Major | conf | Jason Darrell Lowe | Dhananjay Badaya | +| [YARN-10178](https://issues.apache.org/jira/browse/YARN-10178) | Global Scheduler async thread crash caused by 'Comparison method violates its general contract | Major | capacity scheduler | tuyu | Andras Gyori | +| [HDFS-16410](https://issues.apache.org/jira/browse/HDFS-16410) | Insecure Xml parsing in OfflineEditsXmlLoader | Minor | . | Ashutosh Gupta | Ashutosh Gupta | +| [HADOOP-18158](https://issues.apache.org/jira/browse/HADOOP-18158) | Fix failure of create-release script due to releasedocmaker changes in branch-2.10 | Major | build | Masatake Iwasaki | Masatake Iwasaki | +| [HADOOP-18129](https://issues.apache.org/jira/browse/HADOOP-18129) | Change URI[] in INodeLink to String[] to reduce memory footprint of ViewFileSystem | Major | . | Abhishek Das | Abhishek Das | +| [HDFS-16517](https://issues.apache.org/jira/browse/HDFS-16517) | In 2.10 the distance metric is wrong for non-DN machines | Major | . | Owen O'Malley | Owen O'Malley | +| [HDFS-16518](https://issues.apache.org/jira/browse/HDFS-16518) | KeyProviderCache close cached KeyProvider with Hadoop ShutdownHookManager | Major | hdfs | Lei Yang | Lei Yang | +| [HADOOP-18169](https://issues.apache.org/jira/browse/HADOOP-18169) | getDelegationTokens in ViewFs should also fetch the token from the fallback FS | Major | . | Xing Lin | Xing Lin | +| [YARN-10720](https://issues.apache.org/jira/browse/YARN-10720) | YARN WebAppProxyServlet should support connection timeout to prevent proxy server from hanging | Critical | . | Qi Zhu | Qi Zhu | +| [HDFS-11041](https://issues.apache.org/jira/browse/HDFS-11041) | Unable to unregister FsDatasetState MBean if DataNode is shutdown twice | Trivial | datanode | Wei-Chiu Chuang | Wei-Chiu Chuang | +| [MAPREDUCE-7246](https://issues.apache.org/jira/browse/MAPREDUCE-7246) | In MapredAppMasterRest#Mapreduce\_Application\_Master\_Info\_API, the datatype of appId should be "string". | Major | documentation | jenny | Ashutosh Gupta | +| [YARN-11126](https://issues.apache.org/jira/browse/YARN-11126) | ZKConfigurationStore Java deserialisation vulnerability | Major | yarn | Tamas Domok | Tamas Domok | +| [YARN-11162](https://issues.apache.org/jira/browse/YARN-11162) | Set the zk acl for nodes created by ZKConfigurationStore. | Major | resourcemanager | Owen O'Malley | Owen O'Malley | + + +### TESTS: + +| JIRA | Summary | Priority | Component | Reporter | Contributor | +|:---- |:---- | :--- |:---- |:---- |:---- | +| [MAPREDUCE-7011](https://issues.apache.org/jira/browse/MAPREDUCE-7011) | TestClientDistributedCacheManager::testDetermineCacheVisibilities assumes all parent dirs set other exec | Trivial | . | Christopher Douglas | Christopher Douglas | + + +### SUB-TASKS: + +| JIRA | Summary | Priority | Component | Reporter | Contributor | +|:---- |:---- | :--- |:---- |:---- |:---- | +| [HADOOP-15133](https://issues.apache.org/jira/browse/HADOOP-15133) | [JDK9] Ignore com.sun.javadoc.\* and com.sun.tools.\* in animal-sniffer-maven-plugin to compile with Java 9 | Major | . | Akira Ajisaka | Akira Ajisaka | +| [HADOOP-15293](https://issues.apache.org/jira/browse/HADOOP-15293) | TestLogLevel fails on Java 9 | Major | test | Akira Ajisaka | Takanobu Asanuma | +| [HADOOP-15513](https://issues.apache.org/jira/browse/HADOOP-15513) | Add additional test cases to cover some corner cases for FileUtil#symlink | Major | . | Giovanni Matteo Fumarola | Giovanni Matteo Fumarola | +| [HADOOP-15783](https://issues.apache.org/jira/browse/HADOOP-15783) | [JDK10] TestSFTPFileSystem.testGetModifyTime fails | Major | . | Akira Ajisaka | Akira Ajisaka | +| [HADOOP-16764](https://issues.apache.org/jira/browse/HADOOP-16764) | Rewrite Python example codes using Python3 | Minor | documentation | Kengo Seki | Kengo Seki | +| [HADOOP-16767](https://issues.apache.org/jira/browse/HADOOP-16767) | S3AInputStream reopening does not handle non IO exceptions properly | Major | . | Sergei Poganshev | Sergei Poganshev | +| [HADOOP-17336](https://issues.apache.org/jira/browse/HADOOP-17336) | Backport HADOOP-16005-"NativeAzureFileSystem does not support setXAttr" and HADOOP-16785. "Improve wasb and abfs resilience on double close() calls. followup to abfs close() fix." to branch-2.10 | Major | fs/azure | Sally Zuo | Sally Zuo | +| [HDFS-15716](https://issues.apache.org/jira/browse/HDFS-15716) | TestUpgradeDomainBlockPlacementPolicy flaky | Major | namenode, test | Ahmed Hussein | Ahmed Hussein | +| [HADOOP-16748](https://issues.apache.org/jira/browse/HADOOP-16748) | Migrate to Python 3 and upgrade Yetus to 0.13.0 | Major | . | Akira Ajisaka | Akira Ajisaka | +| [HDFS-13975](https://issues.apache.org/jira/browse/HDFS-13975) | TestBalancer#testMaxIterationTime fails sporadically | Major | . | Jason Darrell Lowe | Toshihiko Uchida | +| [HDFS-16072](https://issues.apache.org/jira/browse/HDFS-16072) | TestBlockRecovery fails consistently on Branch-2.10 | Major | datanode, test | Ahmed Hussein | Ahmed Hussein | +| [HDFS-15659](https://issues.apache.org/jira/browse/HDFS-15659) | Set dfs.namenode.redundancy.considerLoad to false in MiniDFSCluster | Major | test | Akira Ajisaka | Ahmed Hussein | +| [HADOOP-17952](https://issues.apache.org/jira/browse/HADOOP-17952) | Replace Guava VisibleForTesting by Hadoop's own annotation in hadoop-common-project modules | Major | . | Viraj Jasani | Viraj Jasani | +| [YARN-8546](https://issues.apache.org/jira/browse/YARN-8546) | Resource leak caused by a reserved container being released more than once under async scheduling | Major | capacity scheduler | Weiwei Yang | Tao Yang | +| [HDFS-13248](https://issues.apache.org/jira/browse/HDFS-13248) | RBF: Namenode need to choose block location for the client | Major | . | Wu Weiwei | Owen O'Malley | + + +### OTHER: + +| JIRA | Summary | Priority | Component | Reporter | Contributor | +|:---- |:---- | :--- |:---- |:---- |:---- | +| [HDFS-15808](https://issues.apache.org/jira/browse/HDFS-15808) | Add metrics for FSNamesystem read/write lock hold long time | Major | hdfs | Tao Li | Tao Li | +| [HDFS-16298](https://issues.apache.org/jira/browse/HDFS-16298) | Improve error msg for BlockMissingException | Minor | . | Tao Li | Tao Li | +| [HDFS-16312](https://issues.apache.org/jira/browse/HDFS-16312) | Fix typo for DataNodeVolumeMetrics and ProfilingFileIoEvents | Minor | . | Tao Li | Tao Li | +| [YARN-10540](https://issues.apache.org/jira/browse/YARN-10540) | Node page is broken in YARN UI1 and UI2 including RMWebService api for nodes | Critical | webapp | Sunil G | Jim Brennan | +| [HADOOP-17445](https://issues.apache.org/jira/browse/HADOOP-17445) | Update the year to 2021 | Major | . | Xiaoqiao He | Xiaoqiao He | +| [HADOOP-17571](https://issues.apache.org/jira/browse/HADOOP-17571) | Upgrade com.fasterxml.woodstox:woodstox-core for security reasons | Major | . | Viraj Jasani | Viraj Jasani | +| [HDFS-15895](https://issues.apache.org/jira/browse/HDFS-15895) | DFSAdmin#printOpenFiles has redundant String#format usage | Minor | . | Viraj Jasani | Viraj Jasani | +| [HADOOP-18061](https://issues.apache.org/jira/browse/HADOOP-18061) | Update the year to 2022 | Major | . | Ayush Saxena | Ayush Saxena | +| [HADOOP-18191](https://issues.apache.org/jira/browse/HADOOP-18191) | Log retry count while handling exceptions in RetryInvocationHandler | Minor | . | Viraj Jasani | Viraj Jasani | +| [HADOOP-18125](https://issues.apache.org/jira/browse/HADOOP-18125) | Utility to identify git commit / Jira fixVersion discrepancies for RC preparation | Major | . | Viraj Jasani | Viraj Jasani | +| [HDFS-16551](https://issues.apache.org/jira/browse/HDFS-16551) | Backport HADOOP-17588 to 3.3 and other active old branches. | Major | . | Renukaprasad C | Renukaprasad C | + + diff --git a/hadoop-common-project/hadoop-common/src/site/markdown/release/2.10.2/RELEASENOTES.2.10.2.md b/hadoop-common-project/hadoop-common/src/site/markdown/release/2.10.2/RELEASENOTES.2.10.2.md new file mode 100644 index 0000000000000..ab4936c18fa32 --- /dev/null +++ b/hadoop-common-project/hadoop-common/src/site/markdown/release/2.10.2/RELEASENOTES.2.10.2.md @@ -0,0 +1,102 @@ + + +# "Apache Hadoop" 2.10.2 Release Notes + +These release notes cover new developer and user-facing incompatibilities, important issues, features, and major improvements. + + +--- + +* [HDFS-10650](https://issues.apache.org/jira/browse/HDFS-10650) | *Minor* | **DFSClient#mkdirs and DFSClient#primitiveMkdir should use default directory permission** + +If the caller does not supply a permission, DFSClient#mkdirs and DFSClient#primitiveMkdir will create a new directory with the default directory permission 00777 now, instead of 00666. + + +--- + +* [HDFS-13174](https://issues.apache.org/jira/browse/HDFS-13174) | *Major* | **hdfs mover -p /path times out after 20 min** + +Mover could have fail after 20+ minutes if a block move was enqueued for this long, between two DataNodes due to an internal constant that was introduced for Balancer, but affected Mover as well. +The internal constant can be configured with the dfs.balancer.max-iteration-time parameter after the patch, and affects only the Balancer. Default is 20 minutes. + + +--- + +* [YARN-10036](https://issues.apache.org/jira/browse/YARN-10036) | *Major* | **Install yarnpkg and upgrade nodejs in Dockerfile** + +In the Dockerfile, nodejs is upgraded to 8.17.0 and yarn 1.12.1 is installed. + + +--- + +* [HADOOP-16054](https://issues.apache.org/jira/browse/HADOOP-16054) | *Major* | **Update Dockerfile to use Bionic** + +The build image has been upgraded to Bionic. + + +--- + +* [HADOOP-17338](https://issues.apache.org/jira/browse/HADOOP-17338) | *Major* | **Intermittent S3AInputStream failures: Premature end of Content-Length delimited message body etc** + +**WARNING: No release note provided for this change.** + + +--- + +* [HADOOP-16748](https://issues.apache.org/jira/browse/HADOOP-16748) | *Major* | **Migrate to Python 3 and upgrade Yetus to 0.13.0** + + +- Upgraded Yetus to 0.13.0. +- Removed determine-flaky-tests-hadoop.py. +- Temporarily disabled shelldocs check in the Jenkins jobs due to YETUS-1099. + + +--- + +* [HADOOP-16870](https://issues.apache.org/jira/browse/HADOOP-16870) | *Major* | **Use spotbugs-maven-plugin instead of findbugs-maven-plugin** + +Removed findbugs from the hadoop build images and added spotbugs instead. +Upgraded SpotBugs to 4.2.2 and spotbugs-maven-plugin to 4.2.0. + + +--- + +* [YARN-8234](https://issues.apache.org/jira/browse/YARN-8234) | *Critical* | **Improve RM system metrics publisher's performance by pushing events to timeline server in batch** + +When Timeline Service V1 or V1.5 is used, if "yarn.resourcemanager.system-metrics-publisher.timeline-server-v1.enable-batch" is set to true, ResourceManager sends timeline events in batch. The default value is false. If this functionality is enabled, the maximum number that events published in batch is configured by "yarn.resourcemanager.system-metrics-publisher.timeline-server-v1.batch-size". The default value is 1000. The interval of publishing events can be configured by "yarn.resourcemanager.system-metrics-publisher.timeline-server-v1.interval-seconds". By default, it is set to 60 seconds. + + +--- + +* [HADOOP-18088](https://issues.apache.org/jira/browse/HADOOP-18088) | *Major* | **Replace log4j 1.x with reload4j** + +log4j 1 was replaced with reload4j which is fork of log4j 1.2.17 with the goal of fixing pressing security issues. + +If you are depending on the hadoop artifacts in your build were explicitly excluding log4 artifacts, and now want to exclude the reload4j files, you will need to update your exclusion lists +\ + \org.slf4j\ + \slf4j-reload4j\ +\ +\ + \ch.qos.reload4j\ + \reload4j\ +\ + + + diff --git a/hadoop-common-project/hadoop-common/src/site/markdown/release/3.3.3/CHANGELOG.3.3.3.md b/hadoop-common-project/hadoop-common/src/site/markdown/release/3.3.3/CHANGELOG.3.3.3.md new file mode 100644 index 0000000000000..bf9498db08172 --- /dev/null +++ b/hadoop-common-project/hadoop-common/src/site/markdown/release/3.3.3/CHANGELOG.3.3.3.md @@ -0,0 +1,65 @@ + + +# Apache Hadoop Changelog + +## Release 3.3.3 - 2022-05-09 + + + +### IMPROVEMENTS: + +| JIRA | Summary | Priority | Component | Reporter | Contributor | +|:---- |:---- | :--- |:---- |:---- |:---- | +| [MAPREDUCE-7373](https://issues.apache.org/jira/browse/MAPREDUCE-7373) | Building MapReduce NativeTask fails on Fedora 34+ | Major | build, nativetask | Kengo Seki | Kengo Seki | +| [HDFS-16355](https://issues.apache.org/jira/browse/HDFS-16355) | Improve the description of dfs.block.scanner.volume.bytes.per.second | Minor | documentation, hdfs | guophilipse | guophilipse | +| [HADOOP-18155](https://issues.apache.org/jira/browse/HADOOP-18155) | Refactor tests in TestFileUtil | Trivial | common | Gautham Banasandra | Gautham Banasandra | +| [HADOOP-18088](https://issues.apache.org/jira/browse/HADOOP-18088) | Replace log4j 1.x with reload4j | Major | . | Wei-Chiu Chuang | Wei-Chiu Chuang | +| [HDFS-16501](https://issues.apache.org/jira/browse/HDFS-16501) | Print the exception when reporting a bad block | Major | datanode | qinyuren | qinyuren | +| [HADOOP-18214](https://issues.apache.org/jira/browse/HADOOP-18214) | Update BUILDING.txt | Minor | build, documentation | Steve Loughran | | +| [HDFS-16556](https://issues.apache.org/jira/browse/HDFS-16556) | Fix typos in distcp | Minor | documentation | guophilipse | guophilipse | + + +### BUG FIXES: + +| JIRA | Summary | Priority | Component | Reporter | Contributor | +|:---- |:---- | :--- |:---- |:---- |:---- | +| [HADOOP-17341](https://issues.apache.org/jira/browse/HADOOP-17341) | Upgrade commons-codec to 1.15 | Minor | . | Dongjoon Hyun | Dongjoon Hyun | +| [HADOOP-17650](https://issues.apache.org/jira/browse/HADOOP-17650) | Fails to build using Maven 3.8.1 | Major | build | Wei-Chiu Chuang | Viraj Jasani | +| [HADOOP-18178](https://issues.apache.org/jira/browse/HADOOP-18178) | Upgrade jackson to 2.13.2 and jackson-databind to 2.13.2.2 | Major | . | PJ Fanning | PJ Fanning | +| [HDFS-16535](https://issues.apache.org/jira/browse/HDFS-16535) | SlotReleaser should reuse the domain socket based on socket paths | Major | hdfs-client | Quanlong Huang | | +| [HADOOP-18109](https://issues.apache.org/jira/browse/HADOOP-18109) | Ensure that default permissions of directories under internal ViewFS directories are the same as directories on target filesystems | Major | viewfs | Chentao Yu | Chentao Yu | +| [HDFS-16422](https://issues.apache.org/jira/browse/HDFS-16422) | Fix thread safety of EC decoding during concurrent preads | Critical | dfsclient, ec, erasure-coding | daimin | daimin | +| [HDFS-16437](https://issues.apache.org/jira/browse/HDFS-16437) | ReverseXML processor doesn't accept XML files without the SnapshotDiffSection. | Critical | hdfs | yanbin.zhang | yanbin.zhang | +| [HDFS-16507](https://issues.apache.org/jira/browse/HDFS-16507) | [SBN read] Avoid purging edit log which is in progress | Critical | . | Tao Li | Tao Li | +| [YARN-10720](https://issues.apache.org/jira/browse/YARN-10720) | YARN WebAppProxyServlet should support connection timeout to prevent proxy server from hanging | Critical | . | Qi Zhu | Qi Zhu | +| [HDFS-16428](https://issues.apache.org/jira/browse/HDFS-16428) | Source path with storagePolicy cause wrong typeConsumed while rename | Major | hdfs, namenode | lei w | lei w | +| [YARN-11014](https://issues.apache.org/jira/browse/YARN-11014) | YARN incorrectly validates maximum capacity resources on the validation API | Major | . | Benjamin Teke | Benjamin Teke | +| [YARN-11075](https://issues.apache.org/jira/browse/YARN-11075) | Explicitly declare serialVersionUID in LogMutation class | Major | . | Benjamin Teke | Benjamin Teke | +| [HDFS-11041](https://issues.apache.org/jira/browse/HDFS-11041) | Unable to unregister FsDatasetState MBean if DataNode is shutdown twice | Trivial | datanode | Wei-Chiu Chuang | Wei-Chiu Chuang | +| [HADOOP-18160](https://issues.apache.org/jira/browse/HADOOP-18160) | \`org.wildfly.openssl\` should not be shaded by Hadoop build | Major | build | André F. | André F. | +| [HADOOP-18202](https://issues.apache.org/jira/browse/HADOOP-18202) | create-release fails fatal: unsafe repository ('/build/source' is owned by someone else) | Major | build | Steve Loughran | Steve Loughran | + + +### OTHER: + +| JIRA | Summary | Priority | Component | Reporter | Contributor | +|:---- |:---- | :--- |:---- |:---- |:---- | +| [HADOOP-18125](https://issues.apache.org/jira/browse/HADOOP-18125) | Utility to identify git commit / Jira fixVersion discrepancies for RC preparation | Major | . | Viraj Jasani | Viraj Jasani | + + diff --git a/hadoop-common-project/hadoop-common/src/site/markdown/release/3.3.3/RELEASENOTES.3.3.3.md b/hadoop-common-project/hadoop-common/src/site/markdown/release/3.3.3/RELEASENOTES.3.3.3.md new file mode 100644 index 0000000000000..bd9ea8a2e49d6 --- /dev/null +++ b/hadoop-common-project/hadoop-common/src/site/markdown/release/3.3.3/RELEASENOTES.3.3.3.md @@ -0,0 +1,48 @@ + + +# Apache Hadoop 3.3.3 Release Notes + +These release notes cover new developer and user-facing incompatibilities, important issues, features, and major improvements. + + +--- + +* [HADOOP-17650](https://issues.apache.org/jira/browse/HADOOP-17650) | *Major* | **Fails to build using Maven 3.8.1** + +In order to resolve build issues with Maven 3.8.1, we have to bump SolrJ to latest version 8.8.2 as of now. Solr is used by YARN application catalog. Hence, we would recommend upgrading Solr cluster accordingly before upgrading entire Hadoop cluster to 3.4.0 if the YARN application catalog service is used. + + +--- + +* [HADOOP-18088](https://issues.apache.org/jira/browse/HADOOP-18088) | *Major* | **Replace log4j 1.x with reload4j** + +log4j 1 was replaced with reload4j which is fork of log4j 1.2.17 with the goal of fixing pressing security issues. + +If you are depending on the hadoop artifacts in your build were explicitly excluding log4 artifacts, and now want to exclude the reload4j files, you will need to update your exclusion lists +\ + \org.slf4j\ + \slf4j-reload4j\ +\ +\ + \ch.qos.reload4j\ + \reload4j\ +\ + + + diff --git a/hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/cli/util/CommandExecutor.java b/hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/cli/util/CommandExecutor.java index 5ef129cdc87ed..2ccfcfebb27e3 100644 --- a/hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/cli/util/CommandExecutor.java +++ b/hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/cli/util/CommandExecutor.java @@ -23,7 +23,6 @@ import java.io.ByteArrayOutputStream; import java.io.File; import java.io.PrintStream; -import java.util.StringTokenizer; import java.util.regex.Matcher; import java.util.regex.Pattern; import java.util.ArrayList; diff --git a/hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/conf/TestCommonConfigurationFields.java b/hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/conf/TestCommonConfigurationFields.java index 9fcf4a5eb55a2..c31229ba9fcf1 100644 --- a/hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/conf/TestCommonConfigurationFields.java +++ b/hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/conf/TestCommonConfigurationFields.java @@ -21,21 +21,17 @@ import java.util.HashSet; import org.apache.hadoop.crypto.key.kms.KMSClientProvider; -import org.apache.hadoop.fs.AbstractFileSystem; import org.apache.hadoop.fs.CommonConfigurationKeys; import org.apache.hadoop.fs.CommonConfigurationKeysPublic; import org.apache.hadoop.fs.ftp.FtpConfigKeys; import org.apache.hadoop.fs.local.LocalConfigKeys; import org.apache.hadoop.ha.SshFenceByTcpPort; import org.apache.hadoop.ha.ZKFailoverController; -import org.apache.hadoop.http.HttpServer2; import org.apache.hadoop.io.erasurecode.CodecUtil; -import org.apache.hadoop.io.nativeio.NativeIO; import org.apache.hadoop.security.CompositeGroupsMapping; import org.apache.hadoop.security.HttpCrossOriginFilterInitializer; import org.apache.hadoop.security.LdapGroupsMapping; import org.apache.hadoop.security.RuleBasedLdapGroupsMapping; -import org.apache.hadoop.security.http.CrossOriginFilter; import org.apache.hadoop.security.ssl.SSLFactory; /** @@ -80,9 +76,9 @@ public void initializeMemberVariables() { }; // Initialize used variables - xmlPropsToSkipCompare = new HashSet(); - xmlPrefixToSkipCompare = new HashSet(); - configurationPropsToSkipCompare = new HashSet(); + xmlPropsToSkipCompare = new HashSet<>(); + xmlPrefixToSkipCompare = new HashSet<>(); + configurationPropsToSkipCompare = new HashSet<>(); // Set error modes errorIfMissingConfigProps = true; diff --git a/hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/conf/TestConfigurationDeprecation.java b/hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/conf/TestConfigurationDeprecation.java index 2c0d6025f2688..83837862ac47e 100644 --- a/hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/conf/TestConfigurationDeprecation.java +++ b/hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/conf/TestConfigurationDeprecation.java @@ -30,7 +30,6 @@ import java.util.LinkedList; import java.util.List; import java.util.Map; -import java.util.Random; import java.util.concurrent.Callable; import java.util.concurrent.CountDownLatch; import java.util.concurrent.Future; diff --git a/hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/crypto/TestCryptoStreamsForLocalFS.java b/hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/crypto/TestCryptoStreamsForLocalFS.java index 8453889b53a5a..072baf188de72 100644 --- a/hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/crypto/TestCryptoStreamsForLocalFS.java +++ b/hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/crypto/TestCryptoStreamsForLocalFS.java @@ -25,7 +25,6 @@ import java.io.OutputStream; import org.apache.hadoop.conf.Configuration; -import org.apache.hadoop.fs.CommonConfigurationKeysPublic; import org.apache.hadoop.fs.FileSystem; import org.apache.hadoop.fs.FileUtil; import org.apache.hadoop.fs.LocalFileSystem; diff --git a/hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/crypto/key/TestValueQueue.java b/hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/crypto/key/TestValueQueue.java index 5da973c6a761d..4805fca1d49f4 100644 --- a/hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/crypto/key/TestValueQueue.java +++ b/hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/crypto/key/TestValueQueue.java @@ -18,6 +18,8 @@ package org.apache.hadoop.crypto.key; import java.io.IOException; +import java.util.Arrays; +import java.util.HashSet; import java.util.Queue; import java.util.concurrent.LinkedBlockingQueue; import java.util.concurrent.TimeoutException; @@ -32,7 +34,6 @@ import org.junit.Assert; import org.junit.Test; -import org.apache.hadoop.util.Sets; public class TestValueQueue { Logger LOG = LoggerFactory.getLogger(TestValueQueue.class); @@ -103,10 +104,10 @@ public void testWarmUp() throws Exception { Assert.assertEquals(5, fillInfos[0].num); Assert.assertEquals(5, fillInfos[1].num); Assert.assertEquals(5, fillInfos[2].num); - Assert.assertEquals(Sets.newHashSet("k1", "k2", "k3"), - Sets.newHashSet(fillInfos[0].key, + Assert.assertEquals(new HashSet<>(Arrays.asList("k1", "k2", "k3")), + new HashSet<>(Arrays.asList(fillInfos[0].key, fillInfos[1].key, - fillInfos[2].key)); + fillInfos[2].key))); vq.shutdown(); } diff --git a/hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/crypto/key/kms/TestLoadBalancingKMSClientProvider.java b/hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/crypto/key/kms/TestLoadBalancingKMSClientProvider.java index 9122389bd6e54..3bc96c3e2fce0 100644 --- a/hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/crypto/key/kms/TestLoadBalancingKMSClientProvider.java +++ b/hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/crypto/key/kms/TestLoadBalancingKMSClientProvider.java @@ -31,6 +31,7 @@ import java.io.IOException; import java.net.ConnectException; import java.net.NoRouteToHostException; +import java.net.SocketException; import java.net.SocketTimeoutException; import java.net.URI; import java.net.UnknownHostException; @@ -38,9 +39,12 @@ import java.security.NoSuchAlgorithmException; import java.security.PrivilegedExceptionAction; import java.util.Arrays; +import java.util.Collections; +import java.util.HashSet; import java.util.List; import java.util.concurrent.TimeUnit; +import javax.net.ssl.SSLException; import javax.net.ssl.SSLHandshakeException; import org.apache.hadoop.conf.Configuration; @@ -63,7 +67,6 @@ import org.junit.rules.Timeout; import org.mockito.Mockito; -import org.apache.hadoop.util.Sets; public class TestLoadBalancingKMSClientProvider { @@ -84,8 +87,8 @@ public void testCreation() throws Exception { KMSClientProvider[] providers = ((LoadBalancingKMSClientProvider) kp).getProviders(); assertEquals(1, providers.length); - assertEquals(Sets.newHashSet("http://host1:9600/kms/foo/v1/"), - Sets.newHashSet(providers[0].getKMSUrl())); + assertEquals(new HashSet<>(Collections.singleton("http://host1:9600/kms/foo/v1/")), + new HashSet<>(Collections.singleton(providers[0].getKMSUrl()))); kp = new KMSClientProvider.Factory().createProvider(new URI( "kms://http@host1;host2;host3:9600/kms/foo"), conf); @@ -93,12 +96,12 @@ public void testCreation() throws Exception { providers = ((LoadBalancingKMSClientProvider) kp).getProviders(); assertEquals(3, providers.length); - assertEquals(Sets.newHashSet("http://host1:9600/kms/foo/v1/", + assertEquals(new HashSet<>(Arrays.asList("http://host1:9600/kms/foo/v1/", "http://host2:9600/kms/foo/v1/", - "http://host3:9600/kms/foo/v1/"), - Sets.newHashSet(providers[0].getKMSUrl(), + "http://host3:9600/kms/foo/v1/")), + new HashSet<>(Arrays.asList(providers[0].getKMSUrl(), providers[1].getKMSUrl(), - providers[2].getKMSUrl())); + providers[2].getKMSUrl()))); kp = new KMSClientProvider.Factory().createProvider(new URI( "kms://http@host1;host2;host3:9600/kms/foo"), conf); @@ -106,12 +109,12 @@ public void testCreation() throws Exception { providers = ((LoadBalancingKMSClientProvider) kp).getProviders(); assertEquals(3, providers.length); - assertEquals(Sets.newHashSet("http://host1:9600/kms/foo/v1/", + assertEquals(new HashSet<>(Arrays.asList("http://host1:9600/kms/foo/v1/", "http://host2:9600/kms/foo/v1/", - "http://host3:9600/kms/foo/v1/"), - Sets.newHashSet(providers[0].getKMSUrl(), + "http://host3:9600/kms/foo/v1/")), + new HashSet<>(Arrays.asList(providers[0].getKMSUrl(), providers[1].getKMSUrl(), - providers[2].getKMSUrl())); + providers[2].getKMSUrl()))); } @Test @@ -707,16 +710,18 @@ public void testClientRetriesWithSSLHandshakeExceptionSucceedsSecondTime() throws Exception { Configuration conf = new Configuration(); conf.setInt( - CommonConfigurationKeysPublic.KMS_CLIENT_FAILOVER_MAX_RETRIES_KEY, 3); + CommonConfigurationKeysPublic.KMS_CLIENT_FAILOVER_MAX_RETRIES_KEY, 5); final String keyName = "test"; KMSClientProvider p1 = mock(KMSClientProvider.class); when(p1.createKey(Mockito.anyString(), Mockito.any(Options.class))) .thenThrow(new SSLHandshakeException("p1")) + .thenThrow(new SSLException("p1")) .thenReturn(new KMSClientProvider.KMSKeyVersion(keyName, "v1", new byte[0])); KMSClientProvider p2 = mock(KMSClientProvider.class); when(p2.createKey(Mockito.anyString(), Mockito.any(Options.class))) - .thenThrow(new ConnectException("p2")); + .thenThrow(new ConnectException("p2")) + .thenThrow(new SocketException("p1")); when(p1.getKMSUrl()).thenReturn("p1"); when(p2.getKMSUrl()).thenReturn("p2"); @@ -725,9 +730,9 @@ public void testClientRetriesWithSSLHandshakeExceptionSucceedsSecondTime() new KMSClientProvider[] {p1, p2}, 0, conf); kp.createKey(keyName, new Options(conf)); - verify(p1, Mockito.times(2)).createKey(Mockito.eq(keyName), + verify(p1, Mockito.times(3)).createKey(Mockito.eq(keyName), Mockito.any(Options.class)); - verify(p2, Mockito.times(1)).createKey(Mockito.eq(keyName), + verify(p2, Mockito.times(2)).createKey(Mockito.eq(keyName), Mockito.any(Options.class)); } diff --git a/hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/crypto/random/TestOsSecureRandom.java b/hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/crypto/random/TestOsSecureRandom.java index 2ea45231a13a1..6448a9a2fba73 100644 --- a/hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/crypto/random/TestOsSecureRandom.java +++ b/hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/crypto/random/TestOsSecureRandom.java @@ -22,7 +22,7 @@ import org.apache.commons.lang3.SystemUtils; import org.apache.hadoop.conf.Configuration; -import org.apache.hadoop.util.Shell.ShellCommandExecutor; + import org.junit.Assume; import org.junit.Test; diff --git a/hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/fs/TestCommandFormat.java b/hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/fs/TestCommandFormat.java index 4b855c4940440..084c6a0aef83d 100644 --- a/hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/fs/TestCommandFormat.java +++ b/hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/fs/TestCommandFormat.java @@ -43,9 +43,9 @@ public class TestCommandFormat { @Before public void setUp() { - args = new ArrayList(); - expectedOpts = new HashSet(); - expectedArgs = new ArrayList(); + args = new ArrayList<>(); + expectedOpts = new HashSet<>(); + expectedArgs = new ArrayList<>(); } @Test @@ -205,6 +205,6 @@ private static List listOf(String ... objects) { } private static Set setOf(String ... objects) { - return new HashSet(listOf(objects)); + return new HashSet<>(listOf(objects)); } } diff --git a/hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/fs/TestHarFileSystemBasics.java b/hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/fs/TestHarFileSystemBasics.java index 6415df6310fc2..471d2458f4f46 100644 --- a/hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/fs/TestHarFileSystemBasics.java +++ b/hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/fs/TestHarFileSystemBasics.java @@ -246,7 +246,7 @@ public void testListLocatedStatus() throws Exception { // test.har has the following contents: // dir1/1.txt // dir1/2.txt - Set expectedFileNames = new HashSet(); + Set expectedFileNames = new HashSet<>(); expectedFileNames.add("1.txt"); expectedFileNames.add("2.txt"); diff --git a/hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/fs/TestListFiles.java b/hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/fs/TestListFiles.java index 44308ea6fc5ea..dce3b956d47ef 100644 --- a/hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/fs/TestListFiles.java +++ b/hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/fs/TestListFiles.java @@ -152,7 +152,7 @@ public void testDirectory() throws IOException { writeFile(fs, FILE1, FILE_LEN); writeFile(fs, FILE3, FILE_LEN); - Set filesToFind = new HashSet(); + Set filesToFind = new HashSet<>(); filesToFind.add(fs.makeQualified(FILE1)); filesToFind.add(fs.makeQualified(FILE2)); filesToFind.add(fs.makeQualified(FILE3)); diff --git a/hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/fs/TestTrash.java b/hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/fs/TestTrash.java index 72287782baac6..5b8c10b3fa6f9 100644 --- a/hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/fs/TestTrash.java +++ b/hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/fs/TestTrash.java @@ -747,7 +747,7 @@ public void testTrashEmptier() throws Exception { Path myPath = new Path(TEST_DIR, "test/mkdirs"); mkdir(fs, myPath); int fileIndex = 0; - Set checkpoints = new HashSet(); + Set checkpoints = new HashSet<>(); while (true) { // Create a file with a new name Path myFile = new Path(TEST_DIR, "test/mkdirs/myFile" + fileIndex++); diff --git a/hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/fs/TestVectoredReadUtils.java b/hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/fs/TestVectoredReadUtils.java new file mode 100644 index 0000000000000..5d08b02e113d5 --- /dev/null +++ b/hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/fs/TestVectoredReadUtils.java @@ -0,0 +1,371 @@ +/* + * 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.fs; + +import java.io.IOException; +import java.nio.ByteBuffer; +import java.nio.IntBuffer; +import java.util.Arrays; +import java.util.List; +import java.util.concurrent.CompletableFuture; +import java.util.function.IntFunction; + +import org.assertj.core.api.Assertions; +import org.junit.Test; +import org.mockito.ArgumentMatchers; +import org.mockito.Mockito; + +import org.apache.hadoop.fs.impl.CombinedFileRange; +import org.apache.hadoop.test.HadoopTestBase; + +import static org.apache.hadoop.fs.VectoredReadUtils.sortRanges; +import static org.apache.hadoop.test.MoreAsserts.assertFutureCompletedSuccessfully; +import static org.apache.hadoop.test.MoreAsserts.assertFutureFailedExceptionally; + +/** + * Test behavior of {@link VectoredReadUtils}. + */ +public class TestVectoredReadUtils extends HadoopTestBase { + + @Test + public void testSliceTo() { + final int size = 64 * 1024; + ByteBuffer buffer = ByteBuffer.allocate(size); + // fill the buffer with data + IntBuffer intBuffer = buffer.asIntBuffer(); + for(int i=0; i < size / Integer.BYTES; ++i) { + intBuffer.put(i); + } + // ensure we don't make unnecessary slices + ByteBuffer slice = VectoredReadUtils.sliceTo(buffer, 100, + FileRange.createFileRange(100, size)); + Assertions.assertThat(buffer) + .describedAs("Slicing on the same offset shouldn't " + + "create a new buffer") + .isEqualTo(slice); + + // try slicing a range + final int offset = 100; + final int sliceStart = 1024; + final int sliceLength = 16 * 1024; + slice = VectoredReadUtils.sliceTo(buffer, offset, + FileRange.createFileRange(offset + sliceStart, sliceLength)); + // make sure they aren't the same, but use the same backing data + Assertions.assertThat(buffer) + .describedAs("Slicing on new offset should " + + "create a new buffer") + .isNotEqualTo(slice); + Assertions.assertThat(buffer.array()) + .describedAs("Slicing should use the same underlying " + + "data") + .isEqualTo(slice.array()); + // test the contents of the slice + intBuffer = slice.asIntBuffer(); + for(int i=0; i < sliceLength / Integer.BYTES; ++i) { + assertEquals("i = " + i, i + sliceStart / Integer.BYTES, intBuffer.get()); + } + } + + @Test + public void testRounding() { + for(int i=5; i < 10; ++i) { + assertEquals("i = "+ i, 5, VectoredReadUtils.roundDown(i, 5)); + assertEquals("i = "+ i, 10, VectoredReadUtils.roundUp(i+1, 5)); + } + assertEquals("Error while roundDown", 13, VectoredReadUtils.roundDown(13, 1)); + assertEquals("Error while roundUp", 13, VectoredReadUtils.roundUp(13, 1)); + } + + @Test + public void testMerge() { + FileRange base = FileRange.createFileRange(2000, 1000); + CombinedFileRange mergeBase = new CombinedFileRange(2000, 3000, base); + + // test when the gap between is too big + assertFalse("Large gap ranges shouldn't get merged", mergeBase.merge(5000, 6000, + FileRange.createFileRange(5000, 1000), 2000, 4000)); + assertEquals("Number of ranges in merged range shouldn't increase", + 1, mergeBase.getUnderlying().size()); + assertEquals("post merge offset", 2000, mergeBase.getOffset()); + assertEquals("post merge length", 1000, mergeBase.getLength()); + + // test when the total size gets exceeded + assertFalse("Large size ranges shouldn't get merged", mergeBase.merge(5000, 6000, + FileRange.createFileRange(5000, 1000), 2001, 3999)); + assertEquals("Number of ranges in merged range shouldn't increase", + 1, mergeBase.getUnderlying().size()); + assertEquals("post merge offset", 2000, mergeBase.getOffset()); + assertEquals("post merge length", 1000, mergeBase.getLength()); + + // test when the merge works + assertTrue("ranges should get merged ", mergeBase.merge(5000, 6000, + FileRange.createFileRange(5000, 1000), 2001, 4000)); + assertEquals("post merge size", 2, mergeBase.getUnderlying().size()); + assertEquals("post merge offset", 2000, mergeBase.getOffset()); + assertEquals("post merge length", 4000, mergeBase.getLength()); + + // reset the mergeBase and test with a 10:1 reduction + mergeBase = new CombinedFileRange(200, 300, base); + assertEquals(200, mergeBase.getOffset()); + assertEquals(100, mergeBase.getLength()); + assertTrue("ranges should get merged ", mergeBase.merge(500, 600, + FileRange.createFileRange(5000, 1000), 201, 400)); + assertEquals("post merge size", 2, mergeBase.getUnderlying().size()); + assertEquals("post merge offset", 200, mergeBase.getOffset()); + assertEquals("post merge length", 400, mergeBase.getLength()); + } + + @Test + public void testSortAndMerge() { + List input = Arrays.asList( + FileRange.createFileRange(3000, 100), + FileRange.createFileRange(2100, 100), + FileRange.createFileRange(1000, 100) + ); + assertFalse("Ranges are non disjoint", VectoredReadUtils.isOrderedDisjoint(input, 100, 800)); + List outputList = VectoredReadUtils.mergeSortedRanges( + Arrays.asList(sortRanges(input)), 100, 1001, 2500); + Assertions.assertThat(outputList) + .describedAs("merged range size") + .hasSize(1); + CombinedFileRange output = outputList.get(0); + Assertions.assertThat(output.getUnderlying()) + .describedAs("merged range underlying size") + .hasSize(3); + assertEquals("range[1000,3100)", output.toString()); + assertTrue("merged output ranges are disjoint", + VectoredReadUtils.isOrderedDisjoint(outputList, 100, 800)); + + // the minSeek doesn't allow the first two to merge + assertFalse("Ranges are non disjoint", + VectoredReadUtils.isOrderedDisjoint(input, 100, 1000)); + outputList = VectoredReadUtils.mergeSortedRanges(Arrays.asList(sortRanges(input)), + 100, 1000, 2100); + Assertions.assertThat(outputList) + .describedAs("merged range size") + .hasSize(2); + assertEquals("range[1000,1100)", outputList.get(0).toString()); + assertEquals("range[2100,3100)", outputList.get(1).toString()); + assertTrue("merged output ranges are disjoint", + VectoredReadUtils.isOrderedDisjoint(outputList, 100, 1000)); + + // the maxSize doesn't allow the third range to merge + assertFalse("Ranges are non disjoint", + VectoredReadUtils.isOrderedDisjoint(input, 100, 800)); + outputList = VectoredReadUtils.mergeSortedRanges(Arrays.asList(sortRanges(input)), + 100, 1001, 2099); + Assertions.assertThat(outputList) + .describedAs("merged range size") + .hasSize(2); + assertEquals("range[1000,2200)", outputList.get(0).toString()); + assertEquals("range[3000,3100)", outputList.get(1).toString()); + assertTrue("merged output ranges are disjoint", + VectoredReadUtils.isOrderedDisjoint(outputList, 100, 800)); + + // test the round up and round down (the maxSize doesn't allow any merges) + assertFalse("Ranges are non disjoint", + VectoredReadUtils.isOrderedDisjoint(input, 16, 700)); + outputList = VectoredReadUtils.mergeSortedRanges(Arrays.asList(sortRanges(input)), + 16, 1001, 100); + Assertions.assertThat(outputList) + .describedAs("merged range size") + .hasSize(3); + assertEquals("range[992,1104)", outputList.get(0).toString()); + assertEquals("range[2096,2208)", outputList.get(1).toString()); + assertEquals("range[2992,3104)", outputList.get(2).toString()); + assertTrue("merged output ranges are disjoint", + VectoredReadUtils.isOrderedDisjoint(outputList, 16, 700)); + } + + @Test + public void testSortAndMergeMoreCases() throws Exception { + List input = Arrays.asList( + FileRange.createFileRange(3000, 110), + FileRange.createFileRange(3000, 100), + FileRange.createFileRange(2100, 100), + FileRange.createFileRange(1000, 100) + ); + assertFalse("Ranges are non disjoint", + VectoredReadUtils.isOrderedDisjoint(input, 100, 800)); + List outputList = VectoredReadUtils.mergeSortedRanges( + Arrays.asList(sortRanges(input)), 1, 1001, 2500); + Assertions.assertThat(outputList) + .describedAs("merged range size") + .hasSize(1); + CombinedFileRange output = outputList.get(0); + Assertions.assertThat(output.getUnderlying()) + .describedAs("merged range underlying size") + .hasSize(4); + assertEquals("range[1000,3110)", output.toString()); + assertTrue("merged output ranges are disjoint", + VectoredReadUtils.isOrderedDisjoint(outputList, 1, 800)); + + outputList = VectoredReadUtils.mergeSortedRanges( + Arrays.asList(sortRanges(input)), 100, 1001, 2500); + Assertions.assertThat(outputList) + .describedAs("merged range size") + .hasSize(1); + output = outputList.get(0); + Assertions.assertThat(output.getUnderlying()) + .describedAs("merged range underlying size") + .hasSize(4); + assertEquals("range[1000,3200)", output.toString()); + assertTrue("merged output ranges are disjoint", + VectoredReadUtils.isOrderedDisjoint(outputList, 1, 800)); + + } + + @Test + public void testMaxSizeZeroDisablesMering() throws Exception { + List randomRanges = Arrays.asList( + FileRange.createFileRange(3000, 110), + FileRange.createFileRange(3000, 100), + FileRange.createFileRange(2100, 100) + ); + assertEqualRangeCountsAfterMerging(randomRanges, 1, 1, 0); + assertEqualRangeCountsAfterMerging(randomRanges, 1, 0, 0); + assertEqualRangeCountsAfterMerging(randomRanges, 1, 100, 0); + } + + private void assertEqualRangeCountsAfterMerging(List inputRanges, + int chunkSize, + int minimumSeek, + int maxSize) { + List combinedFileRanges = VectoredReadUtils + .mergeSortedRanges(inputRanges, chunkSize, minimumSeek, maxSize); + Assertions.assertThat(combinedFileRanges) + .describedAs("Mismatch in number of ranges post merging") + .hasSize(inputRanges.size()); + } + + interface Stream extends PositionedReadable, ByteBufferPositionedReadable { + // nothing + } + + static void fillBuffer(ByteBuffer buffer) { + byte b = 0; + while (buffer.remaining() > 0) { + buffer.put(b++); + } + } + + @Test + public void testReadRangeFromByteBufferPositionedReadable() throws Exception { + Stream stream = Mockito.mock(Stream.class); + Mockito.doAnswer(invocation -> { + fillBuffer(invocation.getArgument(1)); + return null; + }).when(stream).readFully(ArgumentMatchers.anyLong(), + ArgumentMatchers.any(ByteBuffer.class)); + CompletableFuture result = + VectoredReadUtils.readRangeFrom(stream, FileRange.createFileRange(1000, 100), + ByteBuffer::allocate); + assertFutureCompletedSuccessfully(result); + ByteBuffer buffer = result.get(); + assertEquals("Size of result buffer", 100, buffer.remaining()); + byte b = 0; + while (buffer.remaining() > 0) { + assertEquals("remain = " + buffer.remaining(), b++, buffer.get()); + } + + // test an IOException + Mockito.reset(stream); + Mockito.doThrow(new IOException("foo")) + .when(stream).readFully(ArgumentMatchers.anyLong(), + ArgumentMatchers.any(ByteBuffer.class)); + result = + VectoredReadUtils.readRangeFrom(stream, FileRange.createFileRange(1000, 100), + ByteBuffer::allocate); + assertFutureFailedExceptionally(result); + } + + static void runReadRangeFromPositionedReadable(IntFunction allocate) + throws Exception { + PositionedReadable stream = Mockito.mock(PositionedReadable.class); + Mockito.doAnswer(invocation -> { + byte b=0; + byte[] buffer = invocation.getArgument(1); + for(int i=0; i < buffer.length; ++i) { + buffer[i] = b++; + } + return null; + }).when(stream).readFully(ArgumentMatchers.anyLong(), + ArgumentMatchers.any(), ArgumentMatchers.anyInt(), + ArgumentMatchers.anyInt()); + CompletableFuture result = + VectoredReadUtils.readRangeFrom(stream, FileRange.createFileRange(1000, 100), + allocate); + assertFutureCompletedSuccessfully(result); + ByteBuffer buffer = result.get(); + assertEquals("Size of result buffer", 100, buffer.remaining()); + byte b = 0; + while (buffer.remaining() > 0) { + assertEquals("remain = " + buffer.remaining(), b++, buffer.get()); + } + + // test an IOException + Mockito.reset(stream); + Mockito.doThrow(new IOException("foo")) + .when(stream).readFully(ArgumentMatchers.anyLong(), + ArgumentMatchers.any(), ArgumentMatchers.anyInt(), + ArgumentMatchers.anyInt()); + result = + VectoredReadUtils.readRangeFrom(stream, FileRange.createFileRange(1000, 100), + ByteBuffer::allocate); + assertFutureFailedExceptionally(result); + } + + @Test + public void testReadRangeArray() throws Exception { + runReadRangeFromPositionedReadable(ByteBuffer::allocate); + } + + @Test + public void testReadRangeDirect() throws Exception { + runReadRangeFromPositionedReadable(ByteBuffer::allocateDirect); + } + + static void validateBuffer(String message, ByteBuffer buffer, int start) { + byte expected = (byte) start; + while (buffer.remaining() > 0) { + assertEquals(message + " remain: " + buffer.remaining(), expected++, + buffer.get()); + } + } + + @Test + public void testReadVectored() throws Exception { + List input = Arrays.asList(FileRange.createFileRange(0, 100), + FileRange.createFileRange(100_000, 100), + FileRange.createFileRange(200_000, 100)); + Stream stream = Mockito.mock(Stream.class); + Mockito.doAnswer(invocation -> { + fillBuffer(invocation.getArgument(1)); + return null; + }).when(stream).readFully(ArgumentMatchers.anyLong(), + ArgumentMatchers.any(ByteBuffer.class)); + // should not merge the ranges + VectoredReadUtils.readVectored(stream, input, ByteBuffer::allocate); + Mockito.verify(stream, Mockito.times(3)) + .readFully(ArgumentMatchers.anyLong(), ArgumentMatchers.any(ByteBuffer.class)); + for(int b=0; b < input.size(); ++b) { + validateBuffer("buffer " + b, input.get(b).getData().get(), 0); + } + } +} diff --git a/hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/fs/contract/AbstractContractCreateTest.java b/hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/fs/contract/AbstractContractCreateTest.java index 85e1f849998c3..91d19ecad1ec6 100644 --- a/hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/fs/contract/AbstractContractCreateTest.java +++ b/hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/fs/contract/AbstractContractCreateTest.java @@ -264,13 +264,8 @@ public void testCreatedFileIsEventuallyVisible() throws Throwable { describe("verify a written to file is visible after the stream is closed"); Path path = path("testCreatedFileIsEventuallyVisible"); FileSystem fs = getFileSystem(); - try( - FSDataOutputStream out = fs.create(path, - false, - 4096, - (short) 1, - 1024) - ) { + try(FSDataOutputStream out = fs.create(path, false, 4096, (short) 1, + 1024)) { out.write(0x01); out.close(); getFileStatusEventually(fs, path, CREATE_TIMEOUT); @@ -568,14 +563,14 @@ protected void validateSyncableSemantics(final FileSystem fs, in.close(); } catch (FileNotFoundException e) { - // that's OK if it's an object store, but not if its a real - // FS - if (!isSupported(IS_BLOBSTORE)) { + // that's OK if it's an object store or some file systems that newly created files + // are not immediately visible, but not if its a real FS + if (!isSupported(IS_BLOBSTORE) && !isSupported(CREATE_VISIBILITY_DELAYED)) { throw e; } else { LOG.warn( - "Output file was not created; this is an object store with different" - + " visibility semantics"); + "Output file was not created; this is an object store or " + + "a file system with different visibility semantics"); } } } diff --git a/hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/fs/contract/AbstractContractDeleteTest.java b/hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/fs/contract/AbstractContractDeleteTest.java index 08df1d4d883a6..605ea45649a16 100644 --- a/hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/fs/contract/AbstractContractDeleteTest.java +++ b/hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/fs/contract/AbstractContractDeleteTest.java @@ -19,7 +19,7 @@ package org.apache.hadoop.fs.contract; import org.apache.hadoop.fs.Path; -import org.apache.hadoop.fs.FileSystem; + import org.junit.Test; import java.io.IOException; diff --git a/hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/fs/contract/AbstractContractVectoredReadTest.java b/hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/fs/contract/AbstractContractVectoredReadTest.java new file mode 100644 index 0000000000000..77bcc496ff4a2 --- /dev/null +++ b/hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/fs/contract/AbstractContractVectoredReadTest.java @@ -0,0 +1,406 @@ +/* + * 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.fs.contract; + +import java.io.EOFException; +import java.nio.ByteBuffer; +import java.util.ArrayList; +import java.util.Arrays; +import java.util.List; +import java.util.concurrent.CompletableFuture; +import java.util.concurrent.ExecutionException; +import java.util.function.IntFunction; + +import org.assertj.core.api.Assertions; +import org.junit.Assert; +import org.junit.Test; +import org.junit.runner.RunWith; +import org.junit.runners.Parameterized; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + +import org.apache.hadoop.fs.FSDataInputStream; +import org.apache.hadoop.fs.FileRange; +import org.apache.hadoop.fs.FileStatus; +import org.apache.hadoop.fs.StreamCapabilities; +import org.apache.hadoop.fs.FileSystem; +import org.apache.hadoop.fs.Path; +import org.apache.hadoop.fs.impl.FutureIOSupport; +import org.apache.hadoop.io.WeakReferencedElasticByteBufferPool; +import org.apache.hadoop.test.LambdaTestUtils; + +import static org.apache.hadoop.fs.contract.ContractTestUtils.assertCapabilities; +import static org.apache.hadoop.fs.contract.ContractTestUtils.assertDatasetEquals; +import static org.apache.hadoop.fs.contract.ContractTestUtils.createFile; +import static org.apache.hadoop.fs.contract.ContractTestUtils.returnBuffersToPoolPostRead; +import static org.apache.hadoop.fs.contract.ContractTestUtils.validateVectoredReadResult; + +@RunWith(Parameterized.class) +public abstract class AbstractContractVectoredReadTest extends AbstractFSContractTestBase { + + private static final Logger LOG = LoggerFactory.getLogger(AbstractContractVectoredReadTest.class); + + public static final int DATASET_LEN = 64 * 1024; + protected static final byte[] DATASET = ContractTestUtils.dataset(DATASET_LEN, 'a', 32); + protected static final String VECTORED_READ_FILE_NAME = "vectored_file.txt"; + + private final IntFunction allocate; + + private final WeakReferencedElasticByteBufferPool pool = + new WeakReferencedElasticByteBufferPool(); + + private final String bufferType; + + @Parameterized.Parameters(name = "Buffer type : {0}") + public static List params() { + return Arrays.asList("direct", "array"); + } + + public AbstractContractVectoredReadTest(String bufferType) { + this.bufferType = bufferType; + this.allocate = value -> { + boolean isDirect = !"array".equals(bufferType); + return pool.getBuffer(isDirect, value); + }; + } + + public IntFunction getAllocate() { + return allocate; + } + + @Override + public void setup() throws Exception { + super.setup(); + Path path = path(VECTORED_READ_FILE_NAME); + FileSystem fs = getFileSystem(); + createFile(fs, path, true, DATASET); + } + + @Override + public void teardown() throws Exception { + super.teardown(); + pool.release(); + } + + @Test + public void testVectoredReadCapability() throws Exception { + FileSystem fs = getFileSystem(); + String[] vectoredReadCapability = new String[]{StreamCapabilities.VECTOREDIO}; + try (FSDataInputStream in = fs.open(path(VECTORED_READ_FILE_NAME))) { + assertCapabilities(in, vectoredReadCapability, null); + } + } + + @Test + public void testVectoredReadMultipleRanges() throws Exception { + FileSystem fs = getFileSystem(); + List fileRanges = new ArrayList<>(); + for (int i = 0; i < 10; i++) { + FileRange fileRange = FileRange.createFileRange(i * 100, 100); + fileRanges.add(fileRange); + } + try (FSDataInputStream in = fs.open(path(VECTORED_READ_FILE_NAME))) { + in.readVectored(fileRanges, allocate); + CompletableFuture[] completableFutures = new CompletableFuture[fileRanges.size()]; + int i = 0; + for (FileRange res : fileRanges) { + completableFutures[i++] = res.getData(); + } + CompletableFuture combinedFuture = CompletableFuture.allOf(completableFutures); + combinedFuture.get(); + + validateVectoredReadResult(fileRanges, DATASET); + returnBuffersToPoolPostRead(fileRanges, pool); + } + } + + @Test + public void testVectoredReadAndReadFully() throws Exception { + FileSystem fs = getFileSystem(); + List fileRanges = new ArrayList<>(); + fileRanges.add(FileRange.createFileRange(100, 100)); + try (FSDataInputStream in = fs.open(path(VECTORED_READ_FILE_NAME))) { + in.readVectored(fileRanges, allocate); + byte[] readFullRes = new byte[100]; + in.readFully(100, readFullRes); + ByteBuffer vecRes = FutureIOSupport.awaitFuture(fileRanges.get(0).getData()); + Assertions.assertThat(vecRes) + .describedAs("Result from vectored read and readFully must match") + .isEqualByComparingTo(ByteBuffer.wrap(readFullRes)); + returnBuffersToPoolPostRead(fileRanges, pool); + } + } + + /** + * As the minimum seek value is 4*1024,none of the below ranges + * will get merged. + */ + @Test + public void testDisjointRanges() throws Exception { + FileSystem fs = getFileSystem(); + List fileRanges = new ArrayList<>(); + fileRanges.add(FileRange.createFileRange(0, 100)); + fileRanges.add(FileRange.createFileRange(4_000 + 101, 100)); + fileRanges.add(FileRange.createFileRange(16_000 + 101, 100)); + try (FSDataInputStream in = fs.open(path(VECTORED_READ_FILE_NAME))) { + in.readVectored(fileRanges, allocate); + validateVectoredReadResult(fileRanges, DATASET); + returnBuffersToPoolPostRead(fileRanges, pool); + } + } + + /** + * As the minimum seek value is 4*1024, all the below ranges + * will get merged into one. + */ + @Test + public void testAllRangesMergedIntoOne() throws Exception { + FileSystem fs = getFileSystem(); + List fileRanges = new ArrayList<>(); + fileRanges.add(FileRange.createFileRange(0, 100)); + fileRanges.add(FileRange.createFileRange(4_000 - 101, 100)); + fileRanges.add(FileRange.createFileRange(8_000 - 101, 100)); + try (FSDataInputStream in = fs.open(path(VECTORED_READ_FILE_NAME))) { + in.readVectored(fileRanges, allocate); + validateVectoredReadResult(fileRanges, DATASET); + returnBuffersToPoolPostRead(fileRanges, pool); + } + } + + /** + * As the minimum seek value is 4*1024, the first three ranges will be + * merged into and other two will remain as it is. + */ + @Test + public void testSomeRangesMergedSomeUnmerged() throws Exception { + FileSystem fs = getFileSystem(); + List fileRanges = new ArrayList<>(); + fileRanges.add(FileRange.createFileRange(8 * 1024, 100)); + fileRanges.add(FileRange.createFileRange(14 * 1024, 100)); + fileRanges.add(FileRange.createFileRange(10 * 1024, 100)); + fileRanges.add(FileRange.createFileRange(2 * 1024 - 101, 100)); + fileRanges.add(FileRange.createFileRange(40 * 1024, 1024)); + FileStatus fileStatus = fs.getFileStatus(path(VECTORED_READ_FILE_NAME)); + CompletableFuture builder = + fs.openFile(path(VECTORED_READ_FILE_NAME)) + .withFileStatus(fileStatus) + .build(); + try (FSDataInputStream in = builder.get()) { + in.readVectored(fileRanges, allocate); + validateVectoredReadResult(fileRanges, DATASET); + returnBuffersToPoolPostRead(fileRanges, pool); + } + } + + @Test + public void testOverlappingRanges() throws Exception { + FileSystem fs = getFileSystem(); + List fileRanges = getSampleOverlappingRanges(); + FileStatus fileStatus = fs.getFileStatus(path(VECTORED_READ_FILE_NAME)); + CompletableFuture builder = + fs.openFile(path(VECTORED_READ_FILE_NAME)) + .withFileStatus(fileStatus) + .build(); + try (FSDataInputStream in = builder.get()) { + in.readVectored(fileRanges, allocate); + validateVectoredReadResult(fileRanges, DATASET); + returnBuffersToPoolPostRead(fileRanges, pool); + } + } + + @Test + public void testSameRanges() throws Exception { + // Same ranges are special case of overlapping only. + FileSystem fs = getFileSystem(); + List fileRanges = getSampleSameRanges(); + CompletableFuture builder = + fs.openFile(path(VECTORED_READ_FILE_NAME)) + .build(); + try (FSDataInputStream in = builder.get()) { + in.readVectored(fileRanges, allocate); + validateVectoredReadResult(fileRanges, DATASET); + returnBuffersToPoolPostRead(fileRanges, pool); + } + } + + @Test + public void testSomeRandomNonOverlappingRanges() throws Exception { + FileSystem fs = getFileSystem(); + List fileRanges = new ArrayList<>(); + fileRanges.add(FileRange.createFileRange(500, 100)); + fileRanges.add(FileRange.createFileRange(1000, 200)); + fileRanges.add(FileRange.createFileRange(50, 10)); + fileRanges.add(FileRange.createFileRange(10, 5)); + try (FSDataInputStream in = fs.open(path(VECTORED_READ_FILE_NAME))) { + in.readVectored(fileRanges, allocate); + validateVectoredReadResult(fileRanges, DATASET); + returnBuffersToPoolPostRead(fileRanges, pool); + } + } + + @Test + public void testConsecutiveRanges() throws Exception { + FileSystem fs = getFileSystem(); + List fileRanges = new ArrayList<>(); + fileRanges.add(FileRange.createFileRange(500, 100)); + fileRanges.add(FileRange.createFileRange(600, 200)); + fileRanges.add(FileRange.createFileRange(800, 100)); + try (FSDataInputStream in = fs.open(path(VECTORED_READ_FILE_NAME))) { + in.readVectored(fileRanges, allocate); + validateVectoredReadResult(fileRanges, DATASET); + returnBuffersToPoolPostRead(fileRanges, pool); + } + } + + @Test + public void testEOFRanges() throws Exception { + FileSystem fs = getFileSystem(); + List fileRanges = new ArrayList<>(); + fileRanges.add(FileRange.createFileRange(DATASET_LEN, 100)); + try (FSDataInputStream in = fs.open(path(VECTORED_READ_FILE_NAME))) { + in.readVectored(fileRanges, allocate); + for (FileRange res : fileRanges) { + CompletableFuture data = res.getData(); + try { + ByteBuffer buffer = data.get(); + // Shouldn't reach here. + Assert.fail("EOFException must be thrown while reading EOF"); + } catch (ExecutionException ex) { + // ignore as expected. + } catch (Exception ex) { + LOG.error("Exception while running vectored read ", ex); + Assert.fail("Exception while running vectored read " + ex); + } + } + } + } + + @Test + public void testNegativeLengthRange() throws Exception { + FileSystem fs = getFileSystem(); + List fileRanges = new ArrayList<>(); + fileRanges.add(FileRange.createFileRange(0, -50)); + verifyExceptionalVectoredRead(fs, fileRanges, IllegalArgumentException.class); + } + + @Test + public void testNegativeOffsetRange() throws Exception { + FileSystem fs = getFileSystem(); + List fileRanges = new ArrayList<>(); + fileRanges.add(FileRange.createFileRange(-1, 50)); + verifyExceptionalVectoredRead(fs, fileRanges, EOFException.class); + } + + @Test + public void testNormalReadAfterVectoredRead() throws Exception { + FileSystem fs = getFileSystem(); + List fileRanges = createSampleNonOverlappingRanges(); + try (FSDataInputStream in = fs.open(path(VECTORED_READ_FILE_NAME))) { + in.readVectored(fileRanges, allocate); + // read starting 200 bytes + byte[] res = new byte[200]; + in.read(res, 0, 200); + ByteBuffer buffer = ByteBuffer.wrap(res); + assertDatasetEquals(0, "normal_read", buffer, 200, DATASET); + Assertions.assertThat(in.getPos()) + .describedAs("Vectored read shouldn't change file pointer.") + .isEqualTo(200); + validateVectoredReadResult(fileRanges, DATASET); + returnBuffersToPoolPostRead(fileRanges, pool); + } + } + + @Test + public void testVectoredReadAfterNormalRead() throws Exception { + FileSystem fs = getFileSystem(); + List fileRanges = createSampleNonOverlappingRanges(); + try (FSDataInputStream in = fs.open(path(VECTORED_READ_FILE_NAME))) { + // read starting 200 bytes + byte[] res = new byte[200]; + in.read(res, 0, 200); + ByteBuffer buffer = ByteBuffer.wrap(res); + assertDatasetEquals(0, "normal_read", buffer, 200, DATASET); + Assertions.assertThat(in.getPos()) + .describedAs("Vectored read shouldn't change file pointer.") + .isEqualTo(200); + in.readVectored(fileRanges, allocate); + validateVectoredReadResult(fileRanges, DATASET); + returnBuffersToPoolPostRead(fileRanges, pool); + } + } + + @Test + public void testMultipleVectoredReads() throws Exception { + FileSystem fs = getFileSystem(); + List fileRanges1 = createSampleNonOverlappingRanges(); + List fileRanges2 = createSampleNonOverlappingRanges(); + try (FSDataInputStream in = fs.open(path(VECTORED_READ_FILE_NAME))) { + in.readVectored(fileRanges1, allocate); + in.readVectored(fileRanges2, allocate); + validateVectoredReadResult(fileRanges2, DATASET); + validateVectoredReadResult(fileRanges1, DATASET); + returnBuffersToPoolPostRead(fileRanges1, pool); + returnBuffersToPoolPostRead(fileRanges2, pool); + } + } + + protected List createSampleNonOverlappingRanges() { + List fileRanges = new ArrayList<>(); + fileRanges.add(FileRange.createFileRange(0, 100)); + fileRanges.add(FileRange.createFileRange(110, 50)); + return fileRanges; + } + + protected List getSampleSameRanges() { + List fileRanges = new ArrayList<>(); + fileRanges.add(FileRange.createFileRange(8_000, 1000)); + fileRanges.add(FileRange.createFileRange(8_000, 1000)); + fileRanges.add(FileRange.createFileRange(8_000, 1000)); + return fileRanges; + } + + protected List getSampleOverlappingRanges() { + List fileRanges = new ArrayList<>(); + fileRanges.add(FileRange.createFileRange(100, 500)); + fileRanges.add(FileRange.createFileRange(400, 500)); + return fileRanges; + } + + /** + * Validate that exceptions must be thrown during a vectored + * read operation with specific input ranges. + * @param fs FileSystem instance. + * @param fileRanges input file ranges. + * @param clazz type of exception expected. + * @throws Exception any other IOE. + */ + protected void verifyExceptionalVectoredRead( + FileSystem fs, + List fileRanges, + Class clazz) throws Exception { + + CompletableFuture builder = + fs.openFile(path(VECTORED_READ_FILE_NAME)) + .build(); + try (FSDataInputStream in = builder.get()) { + LambdaTestUtils.intercept(clazz, + () -> in.readVectored(fileRanges, allocate)); + } + } +} diff --git a/hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/fs/contract/ContractTestUtils.java b/hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/fs/contract/ContractTestUtils.java index eb56d957d9a1a..b61abddd43426 100644 --- a/hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/fs/contract/ContractTestUtils.java +++ b/hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/fs/contract/ContractTestUtils.java @@ -21,6 +21,7 @@ import org.apache.hadoop.fs.FSDataInputStream; import org.apache.hadoop.fs.FSDataOutputStream; import org.apache.hadoop.fs.FileContext; +import org.apache.hadoop.fs.FileRange; import org.apache.hadoop.fs.FileStatus; import org.apache.hadoop.fs.FileSystem; import org.apache.hadoop.fs.LocatedFileStatus; @@ -28,7 +29,11 @@ import org.apache.hadoop.fs.PathCapabilities; import org.apache.hadoop.fs.RemoteIterator; import org.apache.hadoop.fs.StreamCapabilities; +import org.apache.hadoop.io.ByteBufferPool; import org.apache.hadoop.io.IOUtils; +import org.apache.hadoop.util.functional.RemoteIterators; +import org.apache.hadoop.util.functional.FutureIO; + import org.junit.Assert; import org.junit.AssumptionViolatedException; import org.slf4j.Logger; @@ -39,6 +44,7 @@ import java.io.IOException; import java.io.InputStream; import java.io.OutputStream; +import java.nio.ByteBuffer; import java.util.ArrayList; import java.util.Arrays; import java.util.Collection; @@ -49,6 +55,9 @@ import java.util.Properties; import java.util.Set; import java.util.UUID; +import java.util.concurrent.CompletableFuture; +import java.util.concurrent.TimeUnit; +import java.util.concurrent.TimeoutException; import static org.apache.hadoop.fs.CommonConfigurationKeysPublic.IO_FILE_BUFFER_SIZE_DEFAULT; import static org.apache.hadoop.fs.CommonConfigurationKeysPublic.IO_FILE_BUFFER_SIZE_KEY; @@ -68,6 +77,11 @@ public class ContractTestUtils extends Assert { public static final String IO_CHUNK_MODULUS_SIZE = "io.chunk.modulus.size"; public static final int DEFAULT_IO_CHUNK_MODULUS_SIZE = 128; + /** + * Timeout in seconds for vectored read operation in tests : {@value}. + */ + public static final int VECTORED_READ_OPERATION_TEST_TIMEOUT_SECONDS = 5 * 60; + /** * Assert that a property in the property set matches the expected value. * @param props property set @@ -1095,6 +1109,78 @@ public static void validateFileContent(byte[] concat, byte[][] bytes) { mismatch); } + /** + * Utility to validate vectored read results. + * @param fileRanges input ranges. + * @param originalData original data. + * @throws IOException any ioe. + */ + public static void validateVectoredReadResult(List fileRanges, + byte[] originalData) + throws IOException, TimeoutException { + CompletableFuture[] completableFutures = new CompletableFuture[fileRanges.size()]; + int i = 0; + for (FileRange res : fileRanges) { + completableFutures[i++] = res.getData(); + } + CompletableFuture combinedFuture = CompletableFuture.allOf(completableFutures); + FutureIO.awaitFuture(combinedFuture, + VECTORED_READ_OPERATION_TEST_TIMEOUT_SECONDS, + TimeUnit.SECONDS); + + for (FileRange res : fileRanges) { + CompletableFuture data = res.getData(); + ByteBuffer buffer = FutureIO.awaitFuture(data, + VECTORED_READ_OPERATION_TEST_TIMEOUT_SECONDS, + TimeUnit.SECONDS); + assertDatasetEquals((int) res.getOffset(), "vecRead", + buffer, res.getLength(), originalData); + } + } + + /** + * Utility to return buffers back to the pool once all + * data has been read for each file range. + * @param fileRanges list of file range. + * @param pool buffer pool. + * @throws IOException any IOE + * @throws TimeoutException ideally this should never occur. + */ + public static void returnBuffersToPoolPostRead(List fileRanges, + ByteBufferPool pool) + throws IOException, TimeoutException { + for (FileRange range : fileRanges) { + ByteBuffer buffer = FutureIO.awaitFuture(range.getData(), + VECTORED_READ_OPERATION_TEST_TIMEOUT_SECONDS, + TimeUnit.SECONDS); + pool.putBuffer(buffer); + } + } + + + /** + * Assert that the data read matches the dataset at the given offset. + * This helps verify that the seek process is moving the read pointer + * to the correct location in the file. + * @param readOffset the offset in the file where the read began. + * @param operation operation name for the assertion. + * @param data data read in. + * @param length length of data to check. + * @param originalData original data. + */ + public static void assertDatasetEquals( + final int readOffset, + final String operation, + final ByteBuffer data, + int length, byte[] originalData) { + for (int i = 0; i < length; i++) { + int o = readOffset + i; + assertEquals(operation + " with read offset " + readOffset + + ": data[" + i + "] != DATASET[" + o + "]", + originalData[o], data.get()); + } + } + /** * Receives test data from the given input file and checks the size of the * data as well as the pattern inside the received data. @@ -1446,11 +1532,7 @@ public static TreeScanResults treeWalk(FileSystem fs, Path path) */ public static List toList( RemoteIterator iterator) throws IOException { - ArrayList list = new ArrayList<>(); - while (iterator.hasNext()) { - list.add(iterator.next()); - } - return list; + return RemoteIterators.toList(iterator); } /** @@ -1464,11 +1546,7 @@ public static List toList( */ public static List iteratorToList( RemoteIterator iterator) throws IOException { - List list = new ArrayList<>(); - while (iterator.hasNext()) { - list.add(iterator.next()); - } - return list; + return RemoteIterators.toList(iterator); } diff --git a/hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/fs/contract/ftp/FTPContract.java b/hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/fs/contract/ftp/FTPContract.java index 1efd7fc4e95d4..62648ec58bcc7 100644 --- a/hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/fs/contract/ftp/FTPContract.java +++ b/hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/fs/contract/ftp/FTPContract.java @@ -22,7 +22,6 @@ import org.apache.hadoop.fs.FileSystem; import org.apache.hadoop.fs.Path; import org.apache.hadoop.fs.contract.AbstractBondedFSContract; -import org.junit.Assert; import java.net.URI; diff --git a/hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/fs/contract/localfs/TestLocalFSContractVectoredRead.java b/hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/fs/contract/localfs/TestLocalFSContractVectoredRead.java new file mode 100644 index 0000000000000..5d6ca3f8f0c90 --- /dev/null +++ b/hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/fs/contract/localfs/TestLocalFSContractVectoredRead.java @@ -0,0 +1,86 @@ +/* + * 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.fs.contract.localfs; + +import java.util.ArrayList; +import java.util.List; +import java.util.concurrent.CompletableFuture; + +import org.assertj.core.api.Assertions; +import org.junit.Test; + +import org.apache.hadoop.conf.Configuration; +import org.apache.hadoop.fs.ChecksumException; +import org.apache.hadoop.fs.FSDataInputStream; +import org.apache.hadoop.fs.FSDataOutputStream; +import org.apache.hadoop.fs.FileRange; +import org.apache.hadoop.fs.LocalFileSystem; +import org.apache.hadoop.fs.Path; +import org.apache.hadoop.fs.contract.AbstractContractVectoredReadTest; +import org.apache.hadoop.fs.contract.AbstractFSContract; +import org.apache.hadoop.fs.contract.ContractTestUtils; + +import static org.apache.hadoop.fs.contract.ContractTestUtils.validateVectoredReadResult; +import static org.apache.hadoop.test.LambdaTestUtils.intercept; + +public class TestLocalFSContractVectoredRead extends AbstractContractVectoredReadTest { + + public TestLocalFSContractVectoredRead(String bufferType) { + super(bufferType); + } + + @Override + protected AbstractFSContract createContract(Configuration conf) { + return new LocalFSContract(conf); + } + + @Test + public void testChecksumValidationDuringVectoredRead() throws Exception { + Path testPath = path("big_range_checksum"); + LocalFileSystem localFs = (LocalFileSystem) getFileSystem(); + final byte[] datasetCorrect = ContractTestUtils.dataset(DATASET_LEN, 'a', 32); + try (FSDataOutputStream out = localFs.create(testPath, true)){ + out.write(datasetCorrect); + } + Path checksumPath = localFs.getChecksumFile(testPath); + Assertions.assertThat(localFs.exists(checksumPath)) + .describedAs("Checksum file should be present") + .isTrue(); + CompletableFuture fis = localFs.openFile(testPath).build(); + List someRandomRanges = new ArrayList<>(); + someRandomRanges.add(FileRange.createFileRange(10, 1024)); + someRandomRanges.add(FileRange.createFileRange(1025, 1024)); + try (FSDataInputStream in = fis.get()){ + in.readVectored(someRandomRanges, getAllocate()); + validateVectoredReadResult(someRandomRanges, datasetCorrect); + } + final byte[] datasetCorrupted = ContractTestUtils.dataset(DATASET_LEN, 'a', 64); + try (FSDataOutputStream out = localFs.getRaw().create(testPath, true)){ + out.write(datasetCorrupted); + } + CompletableFuture fisN = localFs.openFile(testPath).build(); + try (FSDataInputStream in = fisN.get()){ + in.readVectored(someRandomRanges, getAllocate()); + // Expect checksum exception when data is updated directly through + // raw local fs instance. + intercept(ChecksumException.class, + () -> validateVectoredReadResult(someRandomRanges, datasetCorrupted)); + } + } +} diff --git a/hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/fs/contract/rawlocal/TestRawLocalContractUnderlyingFileBehavior.java b/hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/fs/contract/rawlocal/TestRawLocalContractUnderlyingFileBehavior.java index 2cb5414caa4c7..6eb24985f4ff3 100644 --- a/hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/fs/contract/rawlocal/TestRawLocalContractUnderlyingFileBehavior.java +++ b/hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/fs/contract/rawlocal/TestRawLocalContractUnderlyingFileBehavior.java @@ -19,7 +19,7 @@ package org.apache.hadoop.fs.contract.rawlocal; import org.apache.hadoop.conf.Configuration; -import org.apache.hadoop.fs.contract.ContractTestUtils; + import org.junit.Assert; import org.junit.BeforeClass; import org.junit.Test; diff --git a/hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/fs/contract/rawlocal/TestRawLocalContractVectoredRead.java b/hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/fs/contract/rawlocal/TestRawLocalContractVectoredRead.java new file mode 100644 index 0000000000000..cbb31ffe27a59 --- /dev/null +++ b/hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/fs/contract/rawlocal/TestRawLocalContractVectoredRead.java @@ -0,0 +1,35 @@ +/* + * 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.fs.contract.rawlocal; + +import org.apache.hadoop.conf.Configuration; +import org.apache.hadoop.fs.contract.AbstractContractVectoredReadTest; +import org.apache.hadoop.fs.contract.AbstractFSContract; + +public class TestRawLocalContractVectoredRead extends AbstractContractVectoredReadTest { + + public TestRawLocalContractVectoredRead(String bufferType) { + super(bufferType); + } + + @Override + protected AbstractFSContract createContract(Configuration conf) { + return new RawlocalFSContract(conf); + } +} diff --git a/hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/fs/contract/rawlocal/TestRawlocalContractPathHandle.java b/hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/fs/contract/rawlocal/TestRawlocalContractPathHandle.java index 3c088d278e536..c34269708ddcb 100644 --- a/hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/fs/contract/rawlocal/TestRawlocalContractPathHandle.java +++ b/hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/fs/contract/rawlocal/TestRawlocalContractPathHandle.java @@ -21,7 +21,6 @@ import org.apache.hadoop.fs.Options; import org.apache.hadoop.fs.contract.AbstractContractPathHandleTest; import org.apache.hadoop.fs.contract.AbstractFSContract; -import org.apache.hadoop.fs.contract.localfs.LocalFSContract; import org.apache.hadoop.fs.contract.rawlocal.RawlocalFSContract; public class TestRawlocalContractPathHandle diff --git a/hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/fs/shell/TestPathData.java b/hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/fs/shell/TestPathData.java index 921fc18131a5c..130ee5edee768 100644 --- a/hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/fs/shell/TestPathData.java +++ b/hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/fs/shell/TestPathData.java @@ -24,7 +24,6 @@ import java.io.File; import java.io.IOException; -import java.net.URI; import java.util.Arrays; import org.apache.hadoop.conf.Configuration; diff --git a/hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/fs/shell/TestTextCommand.java b/hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/fs/shell/TestTextCommand.java index c99b97e6e4021..4eb1d433bee45 100644 --- a/hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/fs/shell/TestTextCommand.java +++ b/hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/fs/shell/TestTextCommand.java @@ -25,15 +25,12 @@ import java.io.IOException; import java.io.InputStream; import java.io.StringWriter; -import java.lang.reflect.Method; import java.net.URI; import java.nio.charset.StandardCharsets; import java.nio.file.Files; -import java.nio.file.Paths; import org.apache.commons.io.IOUtils; import org.apache.hadoop.conf.Configuration; -import org.apache.hadoop.fs.Path; import org.apache.hadoop.test.GenericTestUtils; import org.junit.Test; diff --git a/hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/fs/viewfs/TestNestedMountPoint.java b/hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/fs/viewfs/TestNestedMountPoint.java new file mode 100644 index 0000000000000..4a7aafd0a2b25 --- /dev/null +++ b/hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/fs/viewfs/TestNestedMountPoint.java @@ -0,0 +1,365 @@ +/** + * 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.fs.viewfs; + +import java.net.URI; +import java.util.List; +import java.util.function.Function; +import org.apache.hadoop.conf.Configuration; +import org.apache.hadoop.fs.FileSystem; +import org.apache.hadoop.fs.FsConstants; +import org.apache.hadoop.fs.Path; +import org.junit.After; +import org.junit.Assert; +import org.junit.Before; +import org.junit.Test; + + +/** + * Unit test of nested mount point support in INodeTree + */ +public class TestNestedMountPoint { + private InodeTree inodeTree; + private Configuration conf; + private String mtName; + private URI fsUri; + + static class TestNestMountPointFileSystem { + public URI getUri() { + return uri; + } + + private URI uri; + + TestNestMountPointFileSystem(URI uri) { + this.uri = uri; + } + } + + static class TestNestMountPointInternalFileSystem extends TestNestMountPointFileSystem { + TestNestMountPointInternalFileSystem(URI uri) { + super(uri); + } + } + + private static final URI LINKFALLBACK_TARGET = URI.create("hdfs://nn00"); + private static final URI NN1_TARGET = URI.create("hdfs://nn01/a/b"); + private static final URI NN2_TARGET = URI.create("hdfs://nn02/a/b/e"); + private static final URI NN3_TARGET = URI.create("hdfs://nn03/a/b/c/d"); + private static final URI NN4_TARGET = URI.create("hdfs://nn04/a/b/c/d/e"); + private static final URI NN5_TARGET = URI.create("hdfs://nn05/b/c/d/e"); + private static final URI NN6_TARGET = URI.create("hdfs://nn06/b/c/d/e/f"); + + @Before + public void setUp() throws Exception { + conf = new Configuration(); + mtName = TestNestedMountPoint.class.getName(); + ConfigUtil.setIsNestedMountPointSupported(conf, true); + ConfigUtil.addLink(conf, mtName, "/a/b", NN1_TARGET); + ConfigUtil.addLink(conf, mtName, "/a/b/e", NN2_TARGET); + ConfigUtil.addLink(conf, mtName, "/a/b/c/d", NN3_TARGET); + ConfigUtil.addLink(conf, mtName, "/a/b/c/d/e", NN4_TARGET); + ConfigUtil.addLink(conf, mtName, "/b/c/d/e", NN5_TARGET); + ConfigUtil.addLink(conf, mtName, "/b/c/d/e/f", NN6_TARGET); + ConfigUtil.addLinkFallback(conf, mtName, LINKFALLBACK_TARGET); + + fsUri = new URI(FsConstants.VIEWFS_SCHEME, mtName, "/", null, null); + + inodeTree = new InodeTree(conf, + mtName, fsUri, false) { + @Override + protected Function initAndGetTargetFs() { + return new Function() { + @Override + public TestNestedMountPoint.TestNestMountPointFileSystem apply(URI uri) { + return new TestNestMountPointFileSystem(uri); + } + }; + } + + // For intenral dir fs + @Override + protected TestNestedMountPoint.TestNestMountPointInternalFileSystem getTargetFileSystem( + final INodeDir dir) { + return new TestNestMountPointInternalFileSystem(fsUri); + } + + @Override + protected TestNestedMountPoint.TestNestMountPointInternalFileSystem getTargetFileSystem( + final String settings, final URI[] mergeFsURIList) { + return new TestNestMountPointInternalFileSystem(null); + } + }; + } + + @After + public void tearDown() throws Exception { + inodeTree = null; + } + + @Test + public void testPathResolveToLink() throws Exception { + // /a/b/c/d/e/f resolves to /a/b/c/d/e and /f + InodeTree.ResolveResult resolveResult = inodeTree.resolve("/a/b/c/d/e/f", true); + Assert.assertEquals(InodeTree.ResultKind.EXTERNAL_DIR, resolveResult.kind); + Assert.assertEquals("/a/b/c/d/e", resolveResult.resolvedPath); + Assert.assertEquals(new Path("/f"), resolveResult.remainingPath); + Assert.assertTrue(resolveResult.targetFileSystem instanceof TestNestMountPointFileSystem); + Assert.assertEquals(NN4_TARGET, ((TestNestMountPointFileSystem) resolveResult.targetFileSystem).getUri()); + Assert.assertTrue(resolveResult.isLastInternalDirLink()); + + // /a/b/c/d/e resolves to /a/b/c/d/e and / + InodeTree.ResolveResult resolveResult2 = inodeTree.resolve("/a/b/c/d/e", true); + Assert.assertEquals(InodeTree.ResultKind.EXTERNAL_DIR, resolveResult2.kind); + Assert.assertEquals("/a/b/c/d/e", resolveResult2.resolvedPath); + Assert.assertEquals(new Path("/"), resolveResult2.remainingPath); + Assert.assertTrue(resolveResult2.targetFileSystem instanceof TestNestMountPointFileSystem); + Assert.assertEquals(NN4_TARGET, ((TestNestMountPointFileSystem) resolveResult2.targetFileSystem).getUri()); + Assert.assertTrue(resolveResult2.isLastInternalDirLink()); + + // /a/b/c/d/e/f/g/h/i resolves to /a/b/c/d/e and /f/g/h/i + InodeTree.ResolveResult resolveResult3 = inodeTree.resolve("/a/b/c/d/e/f/g/h/i", true); + Assert.assertEquals(InodeTree.ResultKind.EXTERNAL_DIR, resolveResult3.kind); + Assert.assertEquals("/a/b/c/d/e", resolveResult3.resolvedPath); + Assert.assertEquals(new Path("/f/g/h/i"), resolveResult3.remainingPath); + Assert.assertTrue(resolveResult3.targetFileSystem instanceof TestNestMountPointFileSystem); + Assert.assertEquals(NN4_TARGET, ((TestNestMountPointFileSystem) resolveResult3.targetFileSystem).getUri()); + Assert.assertTrue(resolveResult3.isLastInternalDirLink()); + } + + @Test + public void testPathResolveToLinkNotResolveLastComponent() throws Exception { + // /a/b/c/d/e/f resolves to /a/b/c/d/e and /f + InodeTree.ResolveResult resolveResult = inodeTree.resolve("/a/b/c/d/e/f", false); + Assert.assertEquals(InodeTree.ResultKind.EXTERNAL_DIR, resolveResult.kind); + Assert.assertEquals("/a/b/c/d/e", resolveResult.resolvedPath); + Assert.assertEquals(new Path("/f"), resolveResult.remainingPath); + Assert.assertTrue(resolveResult.targetFileSystem instanceof TestNestMountPointFileSystem); + Assert.assertEquals(NN4_TARGET, ((TestNestMountPointFileSystem) resolveResult.targetFileSystem).getUri()); + Assert.assertTrue(resolveResult.isLastInternalDirLink()); + + // /a/b/c/d/e resolves to /a/b/c/d and /e + InodeTree.ResolveResult resolveResult2 = inodeTree.resolve("/a/b/c/d/e", false); + Assert.assertEquals(InodeTree.ResultKind.EXTERNAL_DIR, resolveResult2.kind); + Assert.assertEquals("/a/b/c/d", resolveResult2.resolvedPath); + Assert.assertEquals(new Path("/e"), resolveResult2.remainingPath); + Assert.assertTrue(resolveResult2.targetFileSystem instanceof TestNestMountPointFileSystem); + Assert.assertEquals(NN3_TARGET, ((TestNestMountPointFileSystem) resolveResult2.targetFileSystem).getUri()); + Assert.assertTrue(resolveResult2.isLastInternalDirLink()); + + // /a/b/c/d/e/f/g/h/i resolves to /a/b/c/d/e and /f/g/h/i + InodeTree.ResolveResult resolveResult3 = inodeTree.resolve("/a/b/c/d/e/f/g/h/i", false); + Assert.assertEquals(InodeTree.ResultKind.EXTERNAL_DIR, resolveResult3.kind); + Assert.assertEquals("/a/b/c/d/e", resolveResult3.resolvedPath); + Assert.assertEquals(new Path("/f/g/h/i"), resolveResult3.remainingPath); + Assert.assertTrue(resolveResult3.targetFileSystem instanceof TestNestMountPointFileSystem); + Assert.assertEquals(NN4_TARGET, ((TestNestMountPointFileSystem) resolveResult3.targetFileSystem).getUri()); + Assert.assertTrue(resolveResult3.isLastInternalDirLink()); + } + + @Test + public void testPathResolveToDirLink() throws Exception { + // /a/b/c/d/f resolves to /a/b/c/d, /f + InodeTree.ResolveResult resolveResult = inodeTree.resolve("/a/b/c/d/f", true); + Assert.assertEquals(InodeTree.ResultKind.EXTERNAL_DIR, resolveResult.kind); + Assert.assertEquals("/a/b/c/d", resolveResult.resolvedPath); + Assert.assertEquals(new Path("/f"), resolveResult.remainingPath); + Assert.assertTrue(resolveResult.targetFileSystem instanceof TestNestMountPointFileSystem); + Assert.assertEquals(NN3_TARGET, ((TestNestMountPointFileSystem) resolveResult.targetFileSystem).getUri()); + Assert.assertTrue(resolveResult.isLastInternalDirLink()); + + // /a/b/c/d resolves to /a/b/c/d and / + InodeTree.ResolveResult resolveResult2 = inodeTree.resolve("/a/b/c/d", true); + Assert.assertEquals(InodeTree.ResultKind.EXTERNAL_DIR, resolveResult2.kind); + Assert.assertEquals("/a/b/c/d", resolveResult2.resolvedPath); + Assert.assertEquals(new Path("/"), resolveResult2.remainingPath); + Assert.assertTrue(resolveResult2.targetFileSystem instanceof TestNestMountPointFileSystem); + Assert.assertEquals(NN3_TARGET, ((TestNestMountPointFileSystem) resolveResult2.targetFileSystem).getUri()); + Assert.assertTrue(resolveResult2.isLastInternalDirLink()); + + // /a/b/c/d/f/g/h/i resolves to /a/b/c/d and /f/g/h/i + InodeTree.ResolveResult resolveResult3 = inodeTree.resolve("/a/b/c/d/f/g/h/i", true); + Assert.assertEquals(InodeTree.ResultKind.EXTERNAL_DIR, resolveResult3.kind); + Assert.assertEquals("/a/b/c/d", resolveResult3.resolvedPath); + Assert.assertEquals(new Path("/f/g/h/i"), resolveResult3.remainingPath); + Assert.assertTrue(resolveResult3.targetFileSystem instanceof TestNestMountPointFileSystem); + Assert.assertEquals(NN3_TARGET, ((TestNestMountPointFileSystem) resolveResult3.targetFileSystem).getUri()); + Assert.assertTrue(resolveResult3.isLastInternalDirLink()); + } + + @Test + public void testPathResolveToDirLinkNotResolveLastComponent() throws Exception { + // /a/b/c/d/f resolves to /a/b/c/d, /f + InodeTree.ResolveResult resolveResult = inodeTree.resolve("/a/b/c/d/f", false); + Assert.assertEquals(InodeTree.ResultKind.EXTERNAL_DIR, resolveResult.kind); + Assert.assertEquals("/a/b/c/d", resolveResult.resolvedPath); + Assert.assertEquals(new Path("/f"), resolveResult.remainingPath); + Assert.assertTrue(resolveResult.targetFileSystem instanceof TestNestMountPointFileSystem); + Assert.assertEquals(NN3_TARGET, ((TestNestMountPointFileSystem) resolveResult.targetFileSystem).getUri()); + Assert.assertTrue(resolveResult.isLastInternalDirLink()); + + // /a/b/c/d resolves to /a/b and /c/d + InodeTree.ResolveResult resolveResult2 = inodeTree.resolve("/a/b/c/d", false); + Assert.assertEquals(InodeTree.ResultKind.EXTERNAL_DIR, resolveResult2.kind); + Assert.assertEquals("/a/b", resolveResult2.resolvedPath); + Assert.assertEquals(new Path("/c/d"), resolveResult2.remainingPath); + Assert.assertTrue(resolveResult2.targetFileSystem instanceof TestNestMountPointFileSystem); + Assert.assertEquals(NN1_TARGET, ((TestNestMountPointFileSystem) resolveResult2.targetFileSystem).getUri()); + Assert.assertTrue(resolveResult2.isLastInternalDirLink()); + + // /a/b/c/d/f/g/h/i resolves to /a/b/c/d and /f/g/h/i + InodeTree.ResolveResult resolveResult3 = inodeTree.resolve("/a/b/c/d/f/g/h/i", false); + Assert.assertEquals(InodeTree.ResultKind.EXTERNAL_DIR, resolveResult3.kind); + Assert.assertEquals("/a/b/c/d", resolveResult3.resolvedPath); + Assert.assertEquals(new Path("/f/g/h/i"), resolveResult3.remainingPath); + Assert.assertTrue(resolveResult3.targetFileSystem instanceof TestNestMountPointFileSystem); + Assert.assertEquals(NN3_TARGET, ((TestNestMountPointFileSystem) resolveResult3.targetFileSystem).getUri()); + Assert.assertTrue(resolveResult3.isLastInternalDirLink()); + } + + @Test + public void testMultiNestedMountPointsPathResolveToDirLink() throws Exception { + // /a/b/f resolves to /a/b and /f + InodeTree.ResolveResult resolveResult = inodeTree.resolve("/a/b/f", true); + Assert.assertEquals(InodeTree.ResultKind.EXTERNAL_DIR, resolveResult.kind); + Assert.assertEquals("/a/b", resolveResult.resolvedPath); + Assert.assertEquals(new Path("/f"), resolveResult.remainingPath); + Assert.assertTrue(resolveResult.targetFileSystem instanceof TestNestMountPointFileSystem); + Assert.assertEquals(NN1_TARGET, ((TestNestMountPointFileSystem) resolveResult.targetFileSystem).getUri()); + Assert.assertTrue(resolveResult.isLastInternalDirLink()); + + // /a/b resolves to /a/b and / + InodeTree.ResolveResult resolveResult2 = inodeTree.resolve("/a/b", true); + Assert.assertEquals(InodeTree.ResultKind.EXTERNAL_DIR, resolveResult2.kind); + Assert.assertEquals("/a/b", resolveResult2.resolvedPath); + Assert.assertEquals(new Path("/"), resolveResult2.remainingPath); + Assert.assertTrue(resolveResult2.targetFileSystem instanceof TestNestMountPointFileSystem); + Assert.assertEquals(NN1_TARGET, ((TestNestMountPointFileSystem) resolveResult2.targetFileSystem).getUri()); + Assert.assertTrue(resolveResult2.isLastInternalDirLink()); + } + + @Test + public void testMultiNestedMountPointsPathResolveToDirLinkNotResolveLastComponent() throws Exception { + // /a/b/f resolves to /a/b and /f + InodeTree.ResolveResult resolveResult = inodeTree.resolve("/a/b/f", false); + Assert.assertEquals(InodeTree.ResultKind.EXTERNAL_DIR, resolveResult.kind); + Assert.assertEquals("/a/b", resolveResult.resolvedPath); + Assert.assertEquals(new Path("/f"), resolveResult.remainingPath); + Assert.assertTrue(resolveResult.targetFileSystem instanceof TestNestMountPointFileSystem); + Assert.assertEquals(NN1_TARGET, ((TestNestMountPointFileSystem) resolveResult.targetFileSystem).getUri()); + Assert.assertTrue(resolveResult.isLastInternalDirLink()); + + // /a/b resolves to /a and /b + InodeTree.ResolveResult resolveResult2 = inodeTree.resolve("/a/b", false); + Assert.assertEquals(InodeTree.ResultKind.INTERNAL_DIR, resolveResult2.kind); + Assert.assertEquals("/a", resolveResult2.resolvedPath); + Assert.assertEquals(new Path("/b"), resolveResult2.remainingPath); + Assert.assertTrue(resolveResult2.targetFileSystem instanceof TestNestMountPointInternalFileSystem); + Assert.assertEquals(fsUri, ((TestNestMountPointInternalFileSystem) resolveResult2.targetFileSystem).getUri()); + Assert.assertFalse(resolveResult2.isLastInternalDirLink()); + } + + @Test + public void testPathResolveToDirLinkLastComponentInternalDir() throws Exception { + // /a/b/c resolves to /a/b and /c + InodeTree.ResolveResult resolveResult = inodeTree.resolve("/a/b/c", true); + Assert.assertEquals(InodeTree.ResultKind.EXTERNAL_DIR, resolveResult.kind); + Assert.assertEquals("/a/b", resolveResult.resolvedPath); + Assert.assertEquals(new Path("/c"), resolveResult.remainingPath); + Assert.assertTrue(resolveResult.targetFileSystem instanceof TestNestMountPointFileSystem); + Assert.assertEquals(NN1_TARGET, ((TestNestMountPointFileSystem) resolveResult.targetFileSystem).getUri()); + Assert.assertTrue(resolveResult.isLastInternalDirLink()); + } + + @Test + public void testPathResolveToDirLinkLastComponentInternalDirNotResolveLastComponent() throws Exception { + // /a/b/c resolves to /a/b and /c + InodeTree.ResolveResult resolveResult = inodeTree.resolve("/a/b/c", false); + Assert.assertEquals(InodeTree.ResultKind.EXTERNAL_DIR, resolveResult.kind); + Assert.assertEquals("/a/b", resolveResult.resolvedPath); + Assert.assertEquals(new Path("/c"), resolveResult.remainingPath); + Assert.assertTrue(resolveResult.targetFileSystem instanceof TestNestMountPointFileSystem); + Assert.assertEquals(NN1_TARGET, ((TestNestMountPointFileSystem) resolveResult.targetFileSystem).getUri()); + Assert.assertTrue(resolveResult.isLastInternalDirLink()); + } + + @Test + public void testPathResolveToLinkFallBack() throws Exception { + // /a/e resolves to linkfallback + InodeTree.ResolveResult resolveResult = inodeTree.resolve("/a/e", true); + Assert.assertEquals(InodeTree.ResultKind.EXTERNAL_DIR, resolveResult.kind); + Assert.assertEquals("/", resolveResult.resolvedPath); + Assert.assertEquals(new Path("/a/e"), resolveResult.remainingPath); + Assert.assertTrue(resolveResult.targetFileSystem instanceof TestNestMountPointFileSystem); + Assert.assertEquals(LINKFALLBACK_TARGET, ((TestNestMountPointFileSystem) resolveResult.targetFileSystem).getUri()); + Assert.assertFalse(resolveResult.isLastInternalDirLink()); + } + + @Test + public void testPathNotResolveToLinkFallBackNotResolveLastComponent() throws Exception { + // /a/e resolves to internalDir instead of linkfallback + InodeTree.ResolveResult resolveResult = inodeTree.resolve("/a/e", false); + Assert.assertEquals(InodeTree.ResultKind.INTERNAL_DIR, resolveResult.kind); + Assert.assertEquals("/a", resolveResult.resolvedPath); + Assert.assertEquals(new Path("/e"), resolveResult.remainingPath); + Assert.assertTrue(resolveResult.targetFileSystem instanceof TestNestMountPointInternalFileSystem); + Assert.assertEquals(fsUri, ((TestNestMountPointInternalFileSystem) resolveResult.targetFileSystem).getUri()); + Assert.assertFalse(resolveResult.isLastInternalDirLink()); + } + + @Test + public void testPathResolveToInternalDir() throws Exception { + // /b/c resolves to internal dir + InodeTree.ResolveResult resolveResult = inodeTree.resolve("/b/c", true); + Assert.assertEquals(InodeTree.ResultKind.INTERNAL_DIR, resolveResult.kind); + Assert.assertEquals("/b/c", resolveResult.resolvedPath); + Assert.assertEquals(new Path("/"), resolveResult.remainingPath); + Assert.assertTrue(resolveResult.targetFileSystem instanceof TestNestMountPointInternalFileSystem); + Assert.assertEquals(fsUri, ((TestNestMountPointInternalFileSystem) resolveResult.targetFileSystem).getUri()); + Assert.assertFalse(resolveResult.isLastInternalDirLink()); + } + + @Test + public void testPathResolveToInternalDirNotResolveLastComponent() throws Exception { + // /b/c resolves to internal dir + InodeTree.ResolveResult resolveResult = inodeTree.resolve("/b/c", false); + Assert.assertEquals(InodeTree.ResultKind.INTERNAL_DIR, resolveResult.kind); + Assert.assertEquals("/b", resolveResult.resolvedPath); + Assert.assertEquals(new Path("/c"), resolveResult.remainingPath); + Assert.assertTrue(resolveResult.targetFileSystem instanceof TestNestMountPointInternalFileSystem); + Assert.assertEquals(fsUri, ((TestNestMountPointInternalFileSystem) resolveResult.targetFileSystem).getUri()); + Assert.assertFalse(resolveResult.isLastInternalDirLink()); + } + + @Test + public void testSlashResolveToInternalDir() throws Exception { + // / resolves to internal dir + InodeTree.ResolveResult resolveResult = inodeTree.resolve("/", true); + Assert.assertEquals(InodeTree.ResultKind.INTERNAL_DIR, resolveResult.kind); + Assert.assertEquals("/", resolveResult.resolvedPath); + Assert.assertEquals(new Path("/"), resolveResult.remainingPath); + Assert.assertTrue(resolveResult.targetFileSystem instanceof TestNestMountPointInternalFileSystem); + Assert.assertFalse(resolveResult.isLastInternalDirLink()); + } + + @Test + public void testInodeTreeMountPoints() throws Exception { + List> mountPoints = inodeTree.getMountPoints(); + Assert.assertEquals(6, mountPoints.size()); + } +} diff --git a/hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/fs/viewfs/TestViewFileSystemWithAuthorityLocalFileSystem.java b/hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/fs/viewfs/TestViewFileSystemWithAuthorityLocalFileSystem.java index 9223338f34bf5..f2452279bc7fc 100644 --- a/hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/fs/viewfs/TestViewFileSystemWithAuthorityLocalFileSystem.java +++ b/hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/fs/viewfs/TestViewFileSystemWithAuthorityLocalFileSystem.java @@ -22,7 +22,6 @@ import org.apache.hadoop.conf.Configuration; import org.apache.hadoop.fs.FileSystem; -import org.apache.hadoop.fs.FileSystemTestHelper; import org.apache.hadoop.fs.FsConstants; import org.apache.hadoop.fs.Path; import static org.apache.hadoop.fs.FileSystem.TRASH_PREFIX; diff --git a/hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/fs/viewfs/TestViewFsConfig.java b/hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/fs/viewfs/TestViewFsConfig.java index 7c318654ecf1c..9d7c58f8197b3 100644 --- a/hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/fs/viewfs/TestViewFsConfig.java +++ b/hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/fs/viewfs/TestViewFsConfig.java @@ -32,6 +32,7 @@ public class TestViewFsConfig { @Test(expected = FileAlreadyExistsException.class) public void testInvalidConfig() throws IOException, URISyntaxException { Configuration conf = new Configuration(); + ConfigUtil.setIsNestedMountPointSupported(conf, false); ConfigUtil.addLink(conf, "/internalDir/linkToDir2", new Path("file:///dir2").toUri()); ConfigUtil.addLink(conf, "/internalDir/linkToDir2/linkToDir3", diff --git a/hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/fs/viewfs/ViewFileSystemBaseTest.java b/hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/fs/viewfs/ViewFileSystemBaseTest.java index 6eb0570f9c60b..0f3c8aacab7cd 100644 --- a/hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/fs/viewfs/ViewFileSystemBaseTest.java +++ b/hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/fs/viewfs/ViewFileSystemBaseTest.java @@ -460,6 +460,148 @@ public void testRenameAcrossMounts4() throws IOException { .assertIsFile(fsTarget, new Path(targetTestRoot, "data/fooBar")); } + + // rename across nested mount points that point to same target also fail + @Test + public void testRenameAcrossNestedMountPointSameTarget() throws IOException { + setUpNestedMountPoint(); + fileSystemTestHelper.createFile(fsView, "/user/foo"); + try { + // Nested mount points point to the same target should fail + // /user -> /user + // /user/userA -> /user + // Rename strategy: SAME_MOUNTPOINT + fsView.rename(new Path("/user/foo"), new Path("/user/userA/foo")); + ContractTestUtils.fail("IOException is not thrown on rename operation"); + } catch (IOException e) { + GenericTestUtils + .assertExceptionContains("Renames across Mount points not supported", + e); + } + } + + + // rename across nested mount points fail if the mount link targets are different + // even if the targets are part of the same target FS + @Test + public void testRenameAcrossMountPointDifferentTarget() throws IOException { + setUpNestedMountPoint(); + fileSystemTestHelper.createFile(fsView, "/data/foo"); + // /data -> /data + // /data/dataA -> /dataA + // Rename strategy: SAME_MOUNTPOINT + try { + fsView.rename(new Path("/data/foo"), new Path("/data/dataA/fooBar")); + ContractTestUtils.fail("IOException is not thrown on rename operation"); + } catch (IOException e) { + GenericTestUtils + .assertExceptionContains("Renames across Mount points not supported", + e); + } + } + + // RenameStrategy SAME_TARGET_URI_ACROSS_MOUNTPOINT enabled + // to rename across nested mount points that point to same target URI + @Test + public void testRenameAcrossNestedMountPointSameTargetUriAcrossMountPoint() throws IOException { + setUpNestedMountPoint(); + // /user/foo -> /user + // /user/userA/fooBarBar -> /user + // Rename strategy: SAME_TARGET_URI_ACROSS_MOUNTPOINT + Configuration conf2 = new Configuration(conf); + conf2.set(Constants.CONFIG_VIEWFS_RENAME_STRATEGY, + ViewFileSystem.RenameStrategy.SAME_TARGET_URI_ACROSS_MOUNTPOINT + .toString()); + FileSystem fsView2 = FileSystem.newInstance(FsConstants.VIEWFS_URI, conf2); + fileSystemTestHelper.createFile(fsView2, "/user/foo"); + fsView2.rename(new Path("/user/foo"), new Path("/user/userA/fooBarBar")); + ContractTestUtils.assertPathDoesNotExist(fsView2, "src should not exist after rename", + new Path("/user/foo")); + ContractTestUtils.assertPathDoesNotExist(fsTarget, "src should not exist after rename", + new Path(targetTestRoot, "user/foo")); + ContractTestUtils.assertIsFile(fsView2, fileSystemTestHelper.getTestRootPath(fsView2, "/user/userA/fooBarBar")); + ContractTestUtils.assertIsFile(fsTarget, new Path(targetTestRoot, "user/fooBarBar")); + } + + // RenameStrategy SAME_FILESYSTEM_ACROSS_MOUNTPOINT enabled + // to rename across mount points where the mount link targets are different + // but are part of the same target FS + @Test + public void testRenameAcrossNestedMountPointSameFileSystemAcrossMountPoint() throws IOException { + setUpNestedMountPoint(); + // /data/foo -> /data + // /data/dataA/fooBar -> /dataA + // Rename strategy: SAME_FILESYSTEM_ACROSS_MOUNTPOINT + Configuration conf2 = new Configuration(conf); + conf2.set(Constants.CONFIG_VIEWFS_RENAME_STRATEGY, + ViewFileSystem.RenameStrategy.SAME_FILESYSTEM_ACROSS_MOUNTPOINT + .toString()); + FileSystem fsView2 = FileSystem.newInstance(FsConstants.VIEWFS_URI, conf2); + fileSystemTestHelper.createFile(fsView2, "/data/foo"); + fsView2.rename(new Path("/data/foo"), new Path("/data/dataB/fooBar")); + ContractTestUtils + .assertPathDoesNotExist(fsView2, "src should not exist after rename", + new Path("/data/foo")); + ContractTestUtils + .assertPathDoesNotExist(fsTarget, "src should not exist after rename", + new Path(targetTestRoot, "data/foo")); + ContractTestUtils.assertIsFile(fsView2, + fileSystemTestHelper.getTestRootPath(fsView2, "/user/fooBar")); + ContractTestUtils + .assertIsFile(fsTarget, new Path(targetTestRoot, "user/fooBar")); + } + + @Test + public void testOperationsThroughNestedMountPointsInternal() + throws IOException { + setUpNestedMountPoint(); + // Create file with nested mount point + fileSystemTestHelper.createFile(fsView, "/user/userB/foo"); + Assert.assertTrue("Created file should be type file", + fsView.getFileStatus(new Path("/user/userB/foo")).isFile()); + Assert.assertTrue("Target of created file should be type file", + fsTarget.getFileStatus(new Path(targetTestRoot,"userB/foo")).isFile()); + + // Delete the created file with nested mount point + Assert.assertTrue("Delete should succeed", + fsView.delete(new Path("/user/userB/foo"), false)); + Assert.assertFalse("File should not exist after delete", + fsView.exists(new Path("/user/userB/foo"))); + Assert.assertFalse("Target File should not exist after delete", + fsTarget.exists(new Path(targetTestRoot,"userB/foo"))); + + // Create file with a 2 component dirs with nested mount point + fileSystemTestHelper.createFile(fsView, "/internalDir/linkToDir2/linkToDir2/foo"); + Assert.assertTrue("Created file should be type file", + fsView.getFileStatus(new Path("/internalDir/linkToDir2/linkToDir2/foo")).isFile()); + Assert.assertTrue("Target of created file should be type file", + fsTarget.getFileStatus(new Path(targetTestRoot,"linkToDir2/foo")).isFile()); + + // Delete the created file with nested mount point + Assert.assertTrue("Delete should succeed", + fsView.delete(new Path("/internalDir/linkToDir2/linkToDir2/foo"), false)); + Assert.assertFalse("File should not exist after delete", + fsView.exists(new Path("/internalDir/linkToDir2/linkToDir2/foo"))); + Assert.assertFalse("Target File should not exist after delete", + fsTarget.exists(new Path(targetTestRoot,"linkToDir2/foo"))); + } + + private void setUpNestedMountPoint() throws IOException { + // Enable nested mount point, ViewFilesystem should support both non-nested and nested mount points + ConfigUtil.setIsNestedMountPointSupported(conf, true); + ConfigUtil.addLink(conf, "/user/userA", + new Path(targetTestRoot, "user").toUri()); + ConfigUtil.addLink(conf, "/user/userB", + new Path(targetTestRoot, "userB").toUri()); + ConfigUtil.addLink(conf, "/data/dataA", + new Path(targetTestRoot, "dataA").toUri()); + ConfigUtil.addLink(conf, "/data/dataB", + new Path(targetTestRoot, "user").toUri()); + ConfigUtil.addLink(conf, "/internalDir/linkToDir2/linkToDir2", + new Path(targetTestRoot,"linkToDir2").toUri()); + fsView = FileSystem.get(FsConstants.VIEWFS_URI, conf); + } + static protected boolean SupportsBlocks = false; // local fs use 1 block // override for HDFS @Test diff --git a/hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/fs/viewfs/ViewFileSystemTestSetup.java b/hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/fs/viewfs/ViewFileSystemTestSetup.java index 866c03ecda9d2..5713f532be7e8 100644 --- a/hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/fs/viewfs/ViewFileSystemTestSetup.java +++ b/hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/fs/viewfs/ViewFileSystemTestSetup.java @@ -20,7 +20,6 @@ import java.net.URI; import org.apache.hadoop.conf.Configuration; -import org.apache.hadoop.fs.FileContext; import org.apache.hadoop.fs.FileSystem; import org.apache.hadoop.fs.FileSystemTestHelper; import org.apache.hadoop.fs.FsConstants; diff --git a/hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/http/TestHttpServerLifecycle.java b/hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/http/TestHttpServerLifecycle.java index 757ea0c05e7c0..4ae1190abd5af 100644 --- a/hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/http/TestHttpServerLifecycle.java +++ b/hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/http/TestHttpServerLifecycle.java @@ -17,8 +17,6 @@ */ package org.apache.hadoop.http; -import org.apache.hadoop.test.GenericTestUtils; -import org.apache.log4j.Logger; import org.junit.Test; public class TestHttpServerLifecycle extends HttpServerFunctionalTest { diff --git a/hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/io/TestIOUtils.java b/hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/io/TestIOUtils.java index fca72d9c65a6a..51f207f97ad29 100644 --- a/hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/io/TestIOUtils.java +++ b/hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/io/TestIOUtils.java @@ -275,7 +275,7 @@ public void testListDirectory() throws IOException { File dir = new File("testListDirectory"); Files.createDirectory(dir.toPath()); try { - Set entries = new HashSet(); + Set entries = new HashSet<>(); entries.add("entry1"); entries.add("entry2"); entries.add("entry3"); diff --git a/hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/io/TestMoreWeakReferencedElasticByteBufferPool.java b/hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/io/TestMoreWeakReferencedElasticByteBufferPool.java new file mode 100644 index 0000000000000..6ca380ef0e46b --- /dev/null +++ b/hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/io/TestMoreWeakReferencedElasticByteBufferPool.java @@ -0,0 +1,97 @@ +/** + * 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.io; + +import java.nio.BufferOverflowException; +import java.nio.ByteBuffer; + +import org.assertj.core.api.Assertions; +import org.junit.Test; + +import org.apache.hadoop.test.HadoopTestBase; + +import static org.apache.hadoop.test.LambdaTestUtils.intercept; + +/** + * Non parameterized tests for {@code WeakReferencedElasticByteBufferPool}. + */ +public class TestMoreWeakReferencedElasticByteBufferPool + extends HadoopTestBase { + + @Test + public void testMixedBuffersInPool() { + WeakReferencedElasticByteBufferPool pool = new WeakReferencedElasticByteBufferPool(); + ByteBuffer buffer1 = pool.getBuffer(true, 5); + ByteBuffer buffer2 = pool.getBuffer(true, 10); + ByteBuffer buffer3 = pool.getBuffer(false, 5); + ByteBuffer buffer4 = pool.getBuffer(false, 10); + ByteBuffer buffer5 = pool.getBuffer(true, 15); + + assertBufferCounts(pool, 0, 0); + pool.putBuffer(buffer1); + pool.putBuffer(buffer2); + assertBufferCounts(pool, 2, 0); + pool.putBuffer(buffer3); + assertBufferCounts(pool, 2, 1); + pool.putBuffer(buffer5); + assertBufferCounts(pool, 3, 1); + pool.putBuffer(buffer4); + assertBufferCounts(pool, 3, 2); + pool.release(); + assertBufferCounts(pool, 0, 0); + + } + + @Test + public void testUnexpectedBufferSizes() throws Exception { + WeakReferencedElasticByteBufferPool pool = new WeakReferencedElasticByteBufferPool(); + ByteBuffer buffer1 = pool.getBuffer(true, 0); + + // try writing a random byte in a 0 length buffer. + // Expected exception as buffer requested is of size 0. + intercept(BufferOverflowException.class, + () -> buffer1.put(new byte[1])); + + // Expected IllegalArgumentException as negative length buffer is requested. + intercept(IllegalArgumentException.class, + () -> pool.getBuffer(true, -5)); + + // test returning null buffer to the pool. + intercept(NullPointerException.class, + () -> pool.putBuffer(null)); + } + + /** + * Utility method to assert counts of direct and heap buffers in + * the given buffer pool. + * @param pool buffer pool. + * @param numDirectBuffersExpected expected number of direct buffers. + * @param numHeapBuffersExpected expected number of heap buffers. + */ + private void assertBufferCounts(WeakReferencedElasticByteBufferPool pool, + int numDirectBuffersExpected, + int numHeapBuffersExpected) { + Assertions.assertThat(pool.getCurrentBuffersCount(true)) + .describedAs("Number of direct buffers in pool") + .isEqualTo(numDirectBuffersExpected); + Assertions.assertThat(pool.getCurrentBuffersCount(false)) + .describedAs("Number of heap buffers in pool") + .isEqualTo(numHeapBuffersExpected); + } +} diff --git a/hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/io/TestWeakReferencedElasticByteBufferPool.java b/hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/io/TestWeakReferencedElasticByteBufferPool.java new file mode 100644 index 0000000000000..1434010ffa652 --- /dev/null +++ b/hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/io/TestWeakReferencedElasticByteBufferPool.java @@ -0,0 +1,232 @@ +/** + * 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.io; + +import java.nio.ByteBuffer; +import java.util.Arrays; +import java.util.List; +import java.util.Random; + +import org.assertj.core.api.Assertions; +import org.junit.Test; +import org.junit.runner.RunWith; +import org.junit.runners.Parameterized; + +import org.apache.hadoop.test.HadoopTestBase; + +/** + * Unit tests for {@code WeakReferencedElasticByteBufferPool}. + */ +@RunWith(Parameterized.class) +public class TestWeakReferencedElasticByteBufferPool + extends HadoopTestBase { + + private final boolean isDirect; + + private final String type; + + @Parameterized.Parameters(name = "Buffer type : {0}") + public static List params() { + return Arrays.asList("direct", "array"); + } + + public TestWeakReferencedElasticByteBufferPool(String type) { + this.type = type; + this.isDirect = !"array".equals(type); + } + + @Test + public void testGetAndPutBasic() { + WeakReferencedElasticByteBufferPool pool = new WeakReferencedElasticByteBufferPool(); + int bufferSize = 5; + ByteBuffer buffer = pool.getBuffer(isDirect, bufferSize); + Assertions.assertThat(buffer.isDirect()) + .describedAs("Buffered returned should be of correct type {}", type) + .isEqualTo(isDirect); + Assertions.assertThat(buffer.capacity()) + .describedAs("Initial capacity of returned buffer from pool") + .isEqualTo(bufferSize); + Assertions.assertThat(buffer.position()) + .describedAs("Initial position of returned buffer from pool") + .isEqualTo(0); + + byte[] arr = createByteArray(bufferSize); + buffer.put(arr, 0, arr.length); + buffer.flip(); + validateBufferContent(buffer, arr); + Assertions.assertThat(buffer.position()) + .describedAs("Buffer's position after filling bytes in it") + .isEqualTo(bufferSize); + // releasing buffer to the pool. + pool.putBuffer(buffer); + Assertions.assertThat(buffer.position()) + .describedAs("Position should be reset to 0 after returning buffer to the pool") + .isEqualTo(0); + + } + + @Test + public void testPoolingWithDifferentSizes() { + WeakReferencedElasticByteBufferPool pool = new WeakReferencedElasticByteBufferPool(); + ByteBuffer buffer = pool.getBuffer(isDirect, 5); + ByteBuffer buffer1 = pool.getBuffer(isDirect, 10); + ByteBuffer buffer2 = pool.getBuffer(isDirect, 15); + + Assertions.assertThat(pool.getCurrentBuffersCount(isDirect)) + .describedAs("Number of buffers in the pool") + .isEqualTo(0); + + pool.putBuffer(buffer1); + pool.putBuffer(buffer2); + Assertions.assertThat(pool.getCurrentBuffersCount(isDirect)) + .describedAs("Number of buffers in the pool") + .isEqualTo(2); + ByteBuffer buffer3 = pool.getBuffer(isDirect, 12); + Assertions.assertThat(buffer3.capacity()) + .describedAs("Pooled buffer should have older capacity") + .isEqualTo(15); + Assertions.assertThat(pool.getCurrentBuffersCount(isDirect)) + .describedAs("Number of buffers in the pool") + .isEqualTo(1); + pool.putBuffer(buffer); + ByteBuffer buffer4 = pool.getBuffer(isDirect, 6); + Assertions.assertThat(buffer4.capacity()) + .describedAs("Pooled buffer should have older capacity") + .isEqualTo(10); + Assertions.assertThat(pool.getCurrentBuffersCount(isDirect)) + .describedAs("Number of buffers in the pool") + .isEqualTo(1); + + pool.release(); + Assertions.assertThat(pool.getCurrentBuffersCount(isDirect)) + .describedAs("Number of buffers in the pool post release") + .isEqualTo(0); + } + + @Test + public void testPoolingWithDifferentInsertionTime() { + WeakReferencedElasticByteBufferPool pool = new WeakReferencedElasticByteBufferPool(); + ByteBuffer buffer = pool.getBuffer(isDirect, 10); + ByteBuffer buffer1 = pool.getBuffer(isDirect, 10); + ByteBuffer buffer2 = pool.getBuffer(isDirect, 10); + + Assertions.assertThat(pool.getCurrentBuffersCount(isDirect)) + .describedAs("Number of buffers in the pool") + .isEqualTo(0); + + pool.putBuffer(buffer1); + pool.putBuffer(buffer2); + Assertions.assertThat(pool.getCurrentBuffersCount(isDirect)) + .describedAs("Number of buffers in the pool") + .isEqualTo(2); + ByteBuffer buffer3 = pool.getBuffer(isDirect, 10); + // As buffer1 is returned to the pool before buffer2, it should + // be returned when buffer of same size is asked again from + // the pool. Memory references must match not just content + // that is why {@code Assertions.isSameAs} is used here rather + // than usual {@code Assertions.isEqualTo}. + Assertions.assertThat(buffer3) + .describedAs("Buffers should be returned in order of their " + + "insertion time") + .isSameAs(buffer1); + pool.putBuffer(buffer); + ByteBuffer buffer4 = pool.getBuffer(isDirect, 10); + Assertions.assertThat(buffer4) + .describedAs("Buffers should be returned in order of their " + + "insertion time") + .isSameAs(buffer2); + } + + @Test + public void testGarbageCollection() { + WeakReferencedElasticByteBufferPool pool = new WeakReferencedElasticByteBufferPool(); + ByteBuffer buffer = pool.getBuffer(isDirect, 5); + ByteBuffer buffer1 = pool.getBuffer(isDirect, 10); + ByteBuffer buffer2 = pool.getBuffer(isDirect, 15); + Assertions.assertThat(pool.getCurrentBuffersCount(isDirect)) + .describedAs("Number of buffers in the pool") + .isEqualTo(0); + pool.putBuffer(buffer1); + pool.putBuffer(buffer2); + Assertions.assertThat(pool.getCurrentBuffersCount(isDirect)) + .describedAs("Number of buffers in the pool") + .isEqualTo(2); + // Before GC. + ByteBuffer buffer4 = pool.getBuffer(isDirect, 12); + Assertions.assertThat(buffer4.capacity()) + .describedAs("Pooled buffer should have older capacity") + .isEqualTo(15); + pool.putBuffer(buffer4); + // Removing the references + buffer1 = null; + buffer2 = null; + buffer4 = null; + System.gc(); + ByteBuffer buffer3 = pool.getBuffer(isDirect, 12); + Assertions.assertThat(buffer3.capacity()) + .describedAs("After garbage collection new buffer should be " + + "returned with fixed capacity") + .isEqualTo(12); + } + + @Test + public void testWeakReferencesPruning() { + WeakReferencedElasticByteBufferPool pool = new WeakReferencedElasticByteBufferPool(); + ByteBuffer buffer1 = pool.getBuffer(isDirect, 5); + ByteBuffer buffer2 = pool.getBuffer(isDirect, 10); + ByteBuffer buffer3 = pool.getBuffer(isDirect, 15); + + pool.putBuffer(buffer2); + pool.putBuffer(buffer3); + Assertions.assertThat(pool.getCurrentBuffersCount(isDirect)) + .describedAs("Number of buffers in the pool") + .isEqualTo(2); + + // marking only buffer2 to be garbage collected. + buffer2 = null; + System.gc(); + ByteBuffer buffer4 = pool.getBuffer(isDirect, 10); + // Number of buffers in the pool is 0 as one got garbage + // collected and other got returned in above call. + Assertions.assertThat(pool.getCurrentBuffersCount(isDirect)) + .describedAs("Number of buffers in the pool") + .isEqualTo(0); + Assertions.assertThat(buffer4.capacity()) + .describedAs("After gc, pool should return next greater than " + + "available buffer") + .isEqualTo(15); + + } + + private void validateBufferContent(ByteBuffer buffer, byte[] arr) { + for (int i=0; i compressors = new HashSet(); + Set compressors = new HashSet<>(); for (int i = 0; i < 10; ++i) { compressors.add(CodecPool.getCompressor(codec)); } @@ -180,7 +180,7 @@ public void testDecompressorNotReturnSameInstance() { Decompressor decomp = CodecPool.getDecompressor(codec); CodecPool.returnDecompressor(decomp); CodecPool.returnDecompressor(decomp); - Set decompressors = new HashSet(); + Set decompressors = new HashSet<>(); for (int i = 0; i < 10; ++i) { decompressors.add(CodecPool.getDecompressor(codec)); } diff --git a/hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/io/compress/bzip2/TestBzip2CompressorDecompressor.java b/hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/io/compress/bzip2/TestBzip2CompressorDecompressor.java index c585a463e46b1..fae5ce6de40a4 100644 --- a/hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/io/compress/bzip2/TestBzip2CompressorDecompressor.java +++ b/hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/io/compress/bzip2/TestBzip2CompressorDecompressor.java @@ -18,9 +18,6 @@ package org.apache.hadoop.io.compress.bzip2; import org.apache.hadoop.conf.Configuration; -import org.apache.hadoop.io.DataInputBuffer; -import org.apache.hadoop.io.DataOutputBuffer; -import org.apache.hadoop.io.compress.*; import org.apache.hadoop.io.compress.bzip2.Bzip2Compressor; import org.apache.hadoop.io.compress.bzip2.Bzip2Decompressor; import org.apache.hadoop.test.MultithreadedTestUtil; @@ -32,7 +29,6 @@ import static org.junit.Assert.*; import static org.junit.Assume.*; -import static org.junit.Assume.assumeTrue; public class TestBzip2CompressorDecompressor { diff --git a/hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/io/compress/zlib/TestZlibCompressorDecompressor.java b/hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/io/compress/zlib/TestZlibCompressorDecompressor.java index ac9ea5e8a8468..25da4fe2375ed 100644 --- a/hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/io/compress/zlib/TestZlibCompressorDecompressor.java +++ b/hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/io/compress/zlib/TestZlibCompressorDecompressor.java @@ -28,7 +28,6 @@ import java.util.zip.DeflaterOutputStream; import org.apache.hadoop.conf.Configuration; -import org.apache.hadoop.fs.CommonConfigurationKeys; import org.apache.hadoop.io.DataInputBuffer; import org.apache.hadoop.io.compress.CompressDecompressTester; import org.apache.hadoop.io.compress.Compressor; diff --git a/hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/io/file/tfile/TestCompression.java b/hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/io/file/tfile/TestCompression.java index b1bf0774974da..6b4c698551359 100644 --- a/hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/io/file/tfile/TestCompression.java +++ b/hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/io/file/tfile/TestCompression.java @@ -17,15 +17,12 @@ */ package org.apache.hadoop.io.file.tfile; -import org.apache.hadoop.io.compress.CompressionCodec; import org.apache.hadoop.test.LambdaTestUtils; import org.junit.*; import java.io.IOException; import static org.junit.Assert.assertEquals; -import static org.junit.Assert.assertTrue; -import static org.junit.Assert.fail; public class TestCompression { diff --git a/hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/ipc/MiniRPCBenchmark.java b/hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/ipc/MiniRPCBenchmark.java index 2290270bfba1a..70ae639091421 100644 --- a/hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/ipc/MiniRPCBenchmark.java +++ b/hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/ipc/MiniRPCBenchmark.java @@ -36,7 +36,6 @@ import org.apache.hadoop.security.SecurityUtil; import org.apache.hadoop.security.UserGroupInformation; import org.apache.hadoop.security.authorize.DefaultImpersonationProvider; -import org.apache.hadoop.security.authorize.ProxyUsers; import org.apache.hadoop.security.token.Token; import org.apache.hadoop.security.token.TokenInfo; import org.apache.hadoop.security.token.delegation.AbstractDelegationTokenSelector; diff --git a/hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/ipc/TestRPC.java b/hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/ipc/TestRPC.java index 5fc9cb5410b30..5caabd22a88c6 100644 --- a/hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/ipc/TestRPC.java +++ b/hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/ipc/TestRPC.java @@ -19,6 +19,8 @@ package org.apache.hadoop.ipc; import org.apache.hadoop.ipc.metrics.RpcMetrics; + +import org.apache.hadoop.thirdparty.com.google.common.util.concurrent.ThreadFactoryBuilder; import org.apache.hadoop.thirdparty.protobuf.ServiceException; import org.apache.hadoop.HadoopIllegalArgumentException; import org.apache.hadoop.conf.Configuration; @@ -84,6 +86,7 @@ import java.util.concurrent.ThreadLocalRandom; import java.util.concurrent.TimeUnit; import java.util.concurrent.atomic.AtomicBoolean; +import java.util.concurrent.atomic.AtomicInteger; import java.util.concurrent.atomic.AtomicReference; import static org.assertj.core.api.Assertions.assertThat; @@ -1697,6 +1700,61 @@ public void testRpcMetricsInNanos() throws Exception { } } + @Test + public void testNumTotalRequestsMetrics() throws Exception { + UserGroupInformation ugi = UserGroupInformation. + createUserForTesting("userXyz", new String[0]); + + final Server server = setupTestServer(conf, 1); + + ExecutorService executorService = null; + try { + RpcMetrics rpcMetrics = server.getRpcMetrics(); + assertEquals(0, rpcMetrics.getTotalRequests()); + assertEquals(0, rpcMetrics.getTotalRequestsPerSecond()); + + List> externalCallList = new ArrayList<>(); + + executorService = Executors.newSingleThreadExecutor( + new ThreadFactoryBuilder().setDaemon(true).setNameFormat("testNumTotalRequestsMetrics") + .build()); + AtomicInteger rps = new AtomicInteger(0); + CountDownLatch countDownLatch = new CountDownLatch(1); + executorService.submit(() -> { + while (true) { + int numRps = (int) rpcMetrics.getTotalRequestsPerSecond(); + rps.getAndSet(numRps); + if (rps.get() > 0) { + countDownLatch.countDown(); + break; + } + } + }); + + for (int i = 0; i < 100000; i++) { + externalCallList.add(newExtCall(ugi, () -> null)); + } + for (ExternalCall externalCall : externalCallList) { + server.queueCall(externalCall); + } + for (ExternalCall externalCall : externalCallList) { + externalCall.get(); + } + + assertEquals(100000, rpcMetrics.getTotalRequests()); + if (countDownLatch.await(10, TimeUnit.SECONDS)) { + assertTrue(rps.get() > 10); + } else { + throw new AssertionError("total requests per seconds are still 0"); + } + } finally { + if (executorService != null) { + executorService.shutdown(); + } + server.stop(); + } + } + public static void main(String[] args) throws Exception { new TestRPC().testCallsInternal(conf); diff --git a/hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/net/TestDNS.java b/hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/net/TestDNS.java index 0e1ac1deb9612..2504a6401a8d9 100644 --- a/hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/net/TestDNS.java +++ b/hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/net/TestDNS.java @@ -18,8 +18,6 @@ package org.apache.hadoop.net; -import java.lang.reflect.Field; -import java.lang.reflect.Modifier; import java.net.NetworkInterface; import java.net.SocketException; import java.net.UnknownHostException; diff --git a/hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/net/TestTableMapping.java b/hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/net/TestTableMapping.java index 86870e1257119..697b0bad43757 100644 --- a/hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/net/TestTableMapping.java +++ b/hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/net/TestTableMapping.java @@ -30,7 +30,7 @@ import java.util.List; import org.apache.hadoop.conf.Configuration; -import org.junit.Before; + import org.junit.Test; public class TestTableMapping { diff --git a/hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/security/TestKDiagNoKDC.java b/hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/security/TestKDiagNoKDC.java index 2e266bba1f97a..03d953b5f3cc3 100644 --- a/hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/security/TestKDiagNoKDC.java +++ b/hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/security/TestKDiagNoKDC.java @@ -19,8 +19,7 @@ package org.apache.hadoop.security; import org.apache.hadoop.conf.Configuration; -import org.apache.hadoop.minikdc.MiniKdc; -import org.junit.AfterClass; + import org.junit.Assert; import org.junit.Before; import org.junit.BeforeClass; @@ -31,20 +30,12 @@ import org.slf4j.Logger; import org.slf4j.LoggerFactory; -import java.io.File; -import java.util.Properties; import java.util.concurrent.TimeUnit; import static org.apache.hadoop.fs.CommonConfigurationKeysPublic.HADOOP_TOKEN_FILES; -import static org.apache.hadoop.fs.CommonConfigurationKeysPublic.HADOOP_SECURITY_AUTHENTICATION; import static org.apache.hadoop.security.KDiag.ARG_KEYLEN; -import static org.apache.hadoop.security.KDiag.ARG_KEYTAB; import static org.apache.hadoop.security.KDiag.ARG_NOFAIL; import static org.apache.hadoop.security.KDiag.ARG_NOLOGIN; -import static org.apache.hadoop.security.KDiag.ARG_PRINCIPAL; -import static org.apache.hadoop.security.KDiag.ARG_SECURE; -import static org.apache.hadoop.security.KDiag.CAT_CONFIG; -import static org.apache.hadoop.security.KDiag.CAT_KERBEROS; import static org.apache.hadoop.security.KDiag.CAT_LOGIN; import static org.apache.hadoop.security.KDiag.CAT_TOKEN; import static org.apache.hadoop.security.KDiag.KerberosDiagsFailure; diff --git a/hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/security/TestUGIWithMiniKdc.java b/hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/security/TestUGIWithMiniKdc.java index de74d17863668..f04fbe1e08926 100644 --- a/hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/security/TestUGIWithMiniKdc.java +++ b/hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/security/TestUGIWithMiniKdc.java @@ -19,23 +19,18 @@ import org.apache.hadoop.conf.Configuration; import org.apache.hadoop.minikdc.MiniKdc; -import org.apache.hadoop.security.authentication.util.KerberosUtil; import org.apache.hadoop.test.GenericTestUtils; import org.apache.hadoop.test.LambdaTestUtils; -import org.apache.hadoop.util.PlatformName; + import org.junit.After; import org.junit.Test; import org.slf4j.event.Level; -import javax.security.auth.Subject; import javax.security.auth.kerberos.KerberosPrincipal; -import javax.security.auth.login.AppConfigurationEntry; import javax.security.auth.login.LoginContext; import java.io.File; import java.security.Principal; -import java.util.HashMap; import java.util.HashSet; -import java.util.Map; import java.util.Properties; import java.util.Set; diff --git a/hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/security/http/TestCrossOriginFilter.java b/hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/security/http/TestCrossOriginFilter.java index b9662b8c6a328..0b396be48f983 100644 --- a/hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/security/http/TestCrossOriginFilter.java +++ b/hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/security/http/TestCrossOriginFilter.java @@ -36,9 +36,6 @@ import org.junit.Test; import org.mockito.Mockito; -import static org.mockito.Mockito.mock; -import static org.mockito.Mockito.verify; - public class TestCrossOriginFilter { @Test diff --git a/hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/service/launcher/TestServiceLauncher.java b/hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/service/launcher/TestServiceLauncher.java index f40051b0d178f..72757e4b1c182 100644 --- a/hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/service/launcher/TestServiceLauncher.java +++ b/hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/service/launcher/TestServiceLauncher.java @@ -29,8 +29,6 @@ import org.apache.hadoop.service.launcher.testservices.StoppingInStartLaunchableService; import org.apache.hadoop.service.launcher.testservices.StringConstructorOnlyService; -import static org.apache.hadoop.service.launcher.LauncherArguments.*; - import static org.apache.hadoop.test.GenericTestUtils.*; import static org.apache.hadoop.service.launcher.testservices.ExceptionInExecuteLaunchableService.*; diff --git a/hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/service/launcher/testservices/NoArgsAllowedService.java b/hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/service/launcher/testservices/NoArgsAllowedService.java index 602cb157ed5d8..9245b1844f792 100644 --- a/hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/service/launcher/testservices/NoArgsAllowedService.java +++ b/hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/service/launcher/testservices/NoArgsAllowedService.java @@ -26,7 +26,6 @@ import org.slf4j.LoggerFactory; import java.util.List; -import java.util.Map; /** * service that does not allow any arguments. diff --git a/hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/test/GenericTestUtils.java b/hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/test/GenericTestUtils.java index f1bf4bb91e668..61d5938494c22 100644 --- a/hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/test/GenericTestUtils.java +++ b/hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/test/GenericTestUtils.java @@ -39,6 +39,7 @@ import java.util.Random; import java.util.Set; import java.util.Enumeration; +import java.util.TreeSet; import java.util.concurrent.CompletableFuture; import java.util.concurrent.CountDownLatch; import java.util.concurrent.TimeUnit; @@ -72,7 +73,6 @@ import org.slf4j.LoggerFactory; import org.apache.hadoop.thirdparty.com.google.common.base.Joiner; -import org.apache.hadoop.util.Sets; import static org.apache.hadoop.fs.contract.ContractTestUtils.createFile; import static org.apache.hadoop.util.functional.CommonCallableSupplier.submit; @@ -344,13 +344,13 @@ public static void assertExists(File f) { public static void assertGlobEquals(File dir, String pattern, String ... expectedMatches) throws IOException { - Set found = Sets.newTreeSet(); + Set found = new TreeSet<>(); for (File f : FileUtil.listFiles(dir)) { if (f.getName().matches(pattern)) { found.add(f.getName()); } } - Set expectedSet = Sets.newTreeSet( + Set expectedSet = new TreeSet<>( Arrays.asList(expectedMatches)); Assert.assertEquals("Bad files matching " + pattern + " in " + dir, Joiner.on(",").join(expectedSet), diff --git a/hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/test/MoreAsserts.java b/hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/test/MoreAsserts.java index 142669b78682e..f6e6055d78e2c 100644 --- a/hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/test/MoreAsserts.java +++ b/hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/test/MoreAsserts.java @@ -19,6 +19,9 @@ package org.apache.hadoop.test; import java.util.Iterator; +import java.util.concurrent.CompletableFuture; + +import org.assertj.core.api.Assertions; import org.junit.Assert; /** @@ -28,17 +31,18 @@ public class MoreAsserts { /** * Assert equivalence for array and iterable - * @param the type of the elements - * @param s the name/message for the collection - * @param expected the expected array of elements - * @param actual the actual iterable of elements + * + * @param the type of the elements + * @param s the name/message for the collection + * @param expected the expected array of elements + * @param actual the actual iterable of elements */ public static void assertEquals(String s, T[] expected, Iterable actual) { Iterator it = actual.iterator(); int i = 0; for (; i < expected.length && it.hasNext(); ++i) { - Assert.assertEquals("Element "+ i +" for "+ s, expected[i], it.next()); + Assert.assertEquals("Element " + i + " for " + s, expected[i], it.next()); } Assert.assertTrue("Expected more elements", i == expected.length); Assert.assertTrue("Expected less elements", !it.hasNext()); @@ -46,7 +50,8 @@ public static void assertEquals(String s, T[] expected, /** * Assert equality for two iterables - * @param the type of the elements + * + * @param the type of the elements * @param s * @param expected * @param actual @@ -57,10 +62,40 @@ public static void assertEquals(String s, Iterable expected, Iterator ita = actual.iterator(); int i = 0; while (ite.hasNext() && ita.hasNext()) { - Assert.assertEquals("Element "+ i +" for "+s, ite.next(), ita.next()); + Assert.assertEquals("Element " + i + " for " + s, ite.next(), ita.next()); } Assert.assertTrue("Expected more elements", !ite.hasNext()); Assert.assertTrue("Expected less elements", !ita.hasNext()); } + + public static void assertFutureCompletedSuccessfully(CompletableFuture future) { + Assertions.assertThat(future.isDone()) + .describedAs("This future is supposed to be " + + "completed successfully") + .isTrue(); + Assertions.assertThat(future.isCompletedExceptionally()) + .describedAs("This future is supposed to be " + + "completed successfully") + .isFalse(); + } + + public static void assertFutureFailedExceptionally(CompletableFuture future) { + Assertions.assertThat(future.isCompletedExceptionally()) + .describedAs("This future is supposed to be " + + "completed exceptionally") + .isTrue(); + } + + /** + * Assert two same type of values. + * @param actual actual value. + * @param expected expected value. + * @param message error message to print in case of mismatch. + */ + public static void assertEqual(T actual, T expected, String message) { + Assertions.assertThat(actual) + .describedAs("Mismatch in %s", message) + .isEqualTo(expected); + } } diff --git a/hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/test/MultithreadedTestUtil.java b/hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/test/MultithreadedTestUtil.java index 217c2f84eba4b..e270ee68000eb 100644 --- a/hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/test/MultithreadedTestUtil.java +++ b/hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/test/MultithreadedTestUtil.java @@ -70,8 +70,8 @@ public abstract class MultithreadedTestUtil { public static class TestContext { private Throwable err = null; private boolean stopped = false; - private Set testThreads = new HashSet(); - private Set finishedThreads = new HashSet(); + private Set testThreads = new HashSet<>(); + private Set finishedThreads = new HashSet<>(); /** * Check if the context can run threads. diff --git a/hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/test/TestTimedOutTestsListener.java b/hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/test/TestTimedOutTestsListener.java index 1334f1c95f407..42ed8c8775570 100644 --- a/hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/test/TestTimedOutTestsListener.java +++ b/hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/test/TestTimedOutTestsListener.java @@ -19,7 +19,6 @@ import java.io.PrintWriter; import java.io.StringWriter; -import java.util.concurrent.BrokenBarrierException; import java.util.concurrent.CyclicBarrier; import java.util.concurrent.locks.Lock; import java.util.concurrent.locks.ReentrantLock; diff --git a/hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/tools/TestCommandShell.java b/hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/tools/TestCommandShell.java index 606791801fe13..e9c5950b729c6 100644 --- a/hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/tools/TestCommandShell.java +++ b/hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/tools/TestCommandShell.java @@ -24,7 +24,6 @@ import org.apache.hadoop.tools.CommandShell; import static org.junit.Assert.assertEquals; -import static org.junit.Assert.assertFalse; import static org.junit.Assert.assertTrue; import org.junit.Before; import org.junit.Test; diff --git a/hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/util/TestDiskCheckerWithDiskIo.java b/hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/util/TestDiskCheckerWithDiskIo.java index 082672ccd33d2..552d1319312c6 100644 --- a/hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/util/TestDiskCheckerWithDiskIo.java +++ b/hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/util/TestDiskCheckerWithDiskIo.java @@ -23,15 +23,12 @@ import org.junit.Rule; import org.junit.Test; import org.junit.rules.Timeout; -import org.slf4j.Logger; -import org.slf4j.LoggerFactory; import java.io.File; import java.io.FileNotFoundException; import java.io.FileOutputStream; import java.io.IOException; import java.nio.file.Files; -import java.nio.file.attribute.PosixFilePermission; import java.nio.file.attribute.PosixFilePermissions; import java.util.concurrent.TimeUnit; import java.util.concurrent.atomic.AtomicInteger; diff --git a/hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/util/TestShell.java b/hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/util/TestShell.java index 6b7154b83b523..9ae52ff95cb91 100644 --- a/hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/util/TestShell.java +++ b/hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/util/TestShell.java @@ -20,7 +20,7 @@ import java.util.concurrent.TimeUnit; import java.util.function.Supplier; import org.apache.commons.io.FileUtils; -import org.apache.hadoop.security.alias.AbstractJavaKeyStoreProvider; + import org.junit.Assert; import java.io.BufferedReader; @@ -51,7 +51,7 @@ public class TestShell extends Assert { /** - * Set the timeout for every test + * Set the timeout for every test. */ @Rule public Timeout testTimeout = new Timeout(30000, TimeUnit.MILLISECONDS); diff --git a/hadoop-common-project/hadoop-kms/src/main/java/org/apache/hadoop/crypto/key/kms/server/KMSAudit.java b/hadoop-common-project/hadoop-kms/src/main/java/org/apache/hadoop/crypto/key/kms/server/KMSAudit.java index 14c2ae907b14d..d71172e1b93ef 100644 --- a/hadoop-common-project/hadoop-kms/src/main/java/org/apache/hadoop/crypto/key/kms/server/KMSAudit.java +++ b/hadoop-common-project/hadoop-kms/src/main/java/org/apache/hadoop/crypto/key/kms/server/KMSAudit.java @@ -36,9 +36,9 @@ import org.apache.hadoop.thirdparty.com.google.common.cache.CacheBuilder; import org.apache.hadoop.thirdparty.com.google.common.cache.RemovalListener; import org.apache.hadoop.thirdparty.com.google.common.cache.RemovalNotification; -import org.apache.hadoop.util.Sets; import org.apache.hadoop.thirdparty.com.google.common.util.concurrent.ThreadFactoryBuilder; +import java.util.Arrays; import java.util.HashSet; import java.util.concurrent.Callable; import java.util.concurrent.ExecutionException; @@ -56,10 +56,10 @@ */ public class KMSAudit { @VisibleForTesting - static final Set AGGREGATE_OPS_WHITELIST = Sets.newHashSet( + static final Set AGGREGATE_OPS_WHITELIST = new HashSet<>(Arrays.asList( KMS.KMSOp.GET_KEY_VERSION, KMS.KMSOp.GET_CURRENT_KEY, KMS.KMSOp.DECRYPT_EEK, KMS.KMSOp.GENERATE_EEK, KMS.KMSOp.REENCRYPT_EEK - ); + )); private Cache cache; diff --git a/hadoop-common-project/hadoop-kms/src/test/java/org/apache/hadoop/crypto/key/kms/server/TestKMS.java b/hadoop-common-project/hadoop-kms/src/test/java/org/apache/hadoop/crypto/key/kms/server/TestKMS.java index a0a58ff3567f5..f4c7fbe0b3c3c 100644 --- a/hadoop-common-project/hadoop-kms/src/test/java/org/apache/hadoop/crypto/key/kms/server/TestKMS.java +++ b/hadoop-common-project/hadoop-kms/src/test/java/org/apache/hadoop/crypto/key/kms/server/TestKMS.java @@ -17,6 +17,7 @@ */ package org.apache.hadoop.crypto.key.kms.server; +import org.apache.commons.lang3.reflect.FieldUtils; import org.apache.hadoop.thirdparty.com.google.common.cache.LoadingCache; import org.apache.curator.test.TestingServer; import org.apache.hadoop.conf.Configuration; @@ -48,7 +49,6 @@ import org.apache.hadoop.security.token.TokenIdentifier; import org.apache.hadoop.security.token.delegation.web.DelegationTokenIdentifier; import org.apache.hadoop.test.GenericTestUtils; -import org.apache.hadoop.test.Whitebox; import org.apache.hadoop.util.Time; import org.apache.http.client.utils.URIBuilder; import org.junit.After; @@ -929,6 +929,7 @@ public Void call() throws Exception { } @Test + @SuppressWarnings("unchecked") public void testKMSProviderCaching() throws Exception { Configuration conf = new Configuration(); File confDir = getTestDir(); @@ -946,11 +947,12 @@ public Void call() throws Exception { KMSClientProvider kmscp = createKMSClientProvider(uri, conf); // get the reference to the internal cache, to test invalidation. - ValueQueue vq = - (ValueQueue) Whitebox.getInternalState(kmscp, "encKeyVersionQueue"); + ValueQueue vq = (ValueQueue) FieldUtils.getField(KMSClientProvider.class, + "encKeyVersionQueue", true).get(kmscp); LoadingCache> kq = - ((LoadingCache>) - Whitebox.getInternalState(vq, "keyQueues")); + (LoadingCache>) + FieldUtils.getField(ValueQueue.class, "keyQueues", true).get(vq); + EncryptedKeyVersion mockEKV = Mockito.mock(EncryptedKeyVersion.class); when(mockEKV.getEncryptionKeyName()).thenReturn(keyName); when(mockEKV.getEncryptionKeyVersionName()).thenReturn(mockVersionName); diff --git a/hadoop-common-project/hadoop-kms/src/test/java/org/apache/hadoop/crypto/key/kms/server/TestKMSAudit.java b/hadoop-common-project/hadoop-kms/src/test/java/org/apache/hadoop/crypto/key/kms/server/TestKMSAudit.java index 2f47ed794ac84..3d0fd7de6428d 100644 --- a/hadoop-common-project/hadoop-kms/src/test/java/org/apache/hadoop/crypto/key/kms/server/TestKMSAudit.java +++ b/hadoop-common-project/hadoop-kms/src/test/java/org/apache/hadoop/crypto/key/kms/server/TestKMSAudit.java @@ -24,13 +24,14 @@ import java.io.OutputStream; import java.io.PrintStream; import java.util.List; +import java.util.concurrent.TimeUnit; +import org.apache.commons.lang3.reflect.FieldUtils; import org.apache.hadoop.conf.Configuration; import org.apache.hadoop.crypto.key.kms.server.KMS.KMSOp; import org.apache.hadoop.io.IOUtils; import org.apache.hadoop.security.UserGroupInformation; import org.apache.hadoop.test.GenericTestUtils; -import org.apache.hadoop.test.Whitebox; import org.apache.hadoop.util.ThreadUtil; import org.apache.log4j.LogManager; import org.apache.log4j.PropertyConfigurator; @@ -63,7 +64,7 @@ public void setOutputStream(OutputStream out) { } @Rule - public final Timeout testTimeout = new Timeout(180000); + public final Timeout testTimeout = new Timeout(180000L, TimeUnit.MILLISECONDS); @Before public void setUp() throws IOException { @@ -207,8 +208,9 @@ public void testAuditLogFormat() throws Exception { @Test public void testInitAuditLoggers() throws Exception { // Default should be the simple logger - List loggers = (List) Whitebox - .getInternalState(kmsAudit, "auditLoggers"); + List loggers = (List) FieldUtils. + getField(KMSAudit.class, "auditLoggers", true).get(kmsAudit); + Assert.assertEquals(1, loggers.size()); Assert.assertEquals(SimpleKMSAuditLogger.class, loggers.get(0).getClass()); @@ -218,8 +220,8 @@ public void testInitAuditLoggers() throws Exception { SimpleKMSAuditLogger.class.getName() + ", " + SimpleKMSAuditLogger.class.getName()); final KMSAudit audit = new KMSAudit(conf); - loggers = - (List) Whitebox.getInternalState(audit, "auditLoggers"); + loggers = (List) FieldUtils. + getField(KMSAudit.class, "auditLoggers", true).get(kmsAudit); Assert.assertEquals(1, loggers.size()); Assert.assertEquals(SimpleKMSAuditLogger.class, loggers.get(0).getClass()); diff --git a/hadoop-common-project/hadoop-nfs/src/main/java/org/apache/hadoop/nfs/nfs3/response/FSINFO3Response.java b/hadoop-common-project/hadoop-nfs/src/main/java/org/apache/hadoop/nfs/nfs3/response/FSINFO3Response.java index ebd54fe08cdbe..4ec4cec610732 100644 --- a/hadoop-common-project/hadoop-nfs/src/main/java/org/apache/hadoop/nfs/nfs3/response/FSINFO3Response.java +++ b/hadoop-common-project/hadoop-nfs/src/main/java/org/apache/hadoop/nfs/nfs3/response/FSINFO3Response.java @@ -18,7 +18,6 @@ package org.apache.hadoop.nfs.nfs3.response; import org.apache.hadoop.nfs.NfsTime; -import org.apache.hadoop.nfs.nfs3.FileHandle; import org.apache.hadoop.nfs.nfs3.Nfs3FileAttributes; import org.apache.hadoop.nfs.nfs3.Nfs3Status; import org.apache.hadoop.oncrpc.XDR; diff --git a/hadoop-common-project/hadoop-nfs/src/main/java/org/apache/hadoop/nfs/nfs3/response/FSSTAT3Response.java b/hadoop-common-project/hadoop-nfs/src/main/java/org/apache/hadoop/nfs/nfs3/response/FSSTAT3Response.java index c0d1a8a38ef06..bcc95861a9608 100644 --- a/hadoop-common-project/hadoop-nfs/src/main/java/org/apache/hadoop/nfs/nfs3/response/FSSTAT3Response.java +++ b/hadoop-common-project/hadoop-nfs/src/main/java/org/apache/hadoop/nfs/nfs3/response/FSSTAT3Response.java @@ -17,7 +17,6 @@ */ package org.apache.hadoop.nfs.nfs3.response; -import org.apache.hadoop.nfs.nfs3.FileHandle; import org.apache.hadoop.nfs.nfs3.Nfs3FileAttributes; import org.apache.hadoop.nfs.nfs3.Nfs3Status; import org.apache.hadoop.oncrpc.XDR; diff --git a/hadoop-common-project/hadoop-nfs/src/main/java/org/apache/hadoop/nfs/nfs3/response/LINK3Response.java b/hadoop-common-project/hadoop-nfs/src/main/java/org/apache/hadoop/nfs/nfs3/response/LINK3Response.java index 3893aa10b2f3e..c810030c2616e 100644 --- a/hadoop-common-project/hadoop-nfs/src/main/java/org/apache/hadoop/nfs/nfs3/response/LINK3Response.java +++ b/hadoop-common-project/hadoop-nfs/src/main/java/org/apache/hadoop/nfs/nfs3/response/LINK3Response.java @@ -17,8 +17,6 @@ */ package org.apache.hadoop.nfs.nfs3.response; -import org.apache.hadoop.nfs.nfs3.Nfs3FileAttributes; -import org.apache.hadoop.nfs.nfs3.Nfs3Status; import org.apache.hadoop.oncrpc.XDR; import org.apache.hadoop.oncrpc.security.Verifier; diff --git a/hadoop-common-project/hadoop-nfs/src/main/java/org/apache/hadoop/nfs/nfs3/response/READDIR3Response.java b/hadoop-common-project/hadoop-nfs/src/main/java/org/apache/hadoop/nfs/nfs3/response/READDIR3Response.java index 5802b7544e6a8..f47acce57f435 100644 --- a/hadoop-common-project/hadoop-nfs/src/main/java/org/apache/hadoop/nfs/nfs3/response/READDIR3Response.java +++ b/hadoop-common-project/hadoop-nfs/src/main/java/org/apache/hadoop/nfs/nfs3/response/READDIR3Response.java @@ -17,7 +17,6 @@ */ package org.apache.hadoop.nfs.nfs3.response; -import java.nio.ByteBuffer; import java.util.ArrayList; import java.util.Arrays; import java.util.Collections; diff --git a/hadoop-common-project/hadoop-nfs/src/main/java/org/apache/hadoop/nfs/nfs3/response/READDIRPLUS3Response.java b/hadoop-common-project/hadoop-nfs/src/main/java/org/apache/hadoop/nfs/nfs3/response/READDIRPLUS3Response.java index f1bfd56fd3908..cc27397d77c1d 100644 --- a/hadoop-common-project/hadoop-nfs/src/main/java/org/apache/hadoop/nfs/nfs3/response/READDIRPLUS3Response.java +++ b/hadoop-common-project/hadoop-nfs/src/main/java/org/apache/hadoop/nfs/nfs3/response/READDIRPLUS3Response.java @@ -25,8 +25,6 @@ import org.apache.hadoop.nfs.nfs3.FileHandle; import org.apache.hadoop.nfs.nfs3.Nfs3FileAttributes; import org.apache.hadoop.nfs.nfs3.Nfs3Status; -import org.apache.hadoop.nfs.nfs3.response.READDIR3Response.DirList3; -import org.apache.hadoop.nfs.nfs3.response.READDIR3Response.Entry3; import org.apache.hadoop.oncrpc.XDR; import org.apache.hadoop.oncrpc.security.Verifier; diff --git a/hadoop-common-project/hadoop-nfs/src/main/java/org/apache/hadoop/nfs/nfs3/response/REMOVE3Response.java b/hadoop-common-project/hadoop-nfs/src/main/java/org/apache/hadoop/nfs/nfs3/response/REMOVE3Response.java index f0fcb3d705c5d..a0b51111a6368 100644 --- a/hadoop-common-project/hadoop-nfs/src/main/java/org/apache/hadoop/nfs/nfs3/response/REMOVE3Response.java +++ b/hadoop-common-project/hadoop-nfs/src/main/java/org/apache/hadoop/nfs/nfs3/response/REMOVE3Response.java @@ -17,8 +17,6 @@ */ package org.apache.hadoop.nfs.nfs3.response; -import org.apache.hadoop.nfs.nfs3.Nfs3FileAttributes; -import org.apache.hadoop.nfs.nfs3.Nfs3Status; import org.apache.hadoop.oncrpc.XDR; import org.apache.hadoop.oncrpc.security.Verifier; diff --git a/hadoop-common-project/hadoop-nfs/src/main/java/org/apache/hadoop/nfs/nfs3/response/WRITE3Response.java b/hadoop-common-project/hadoop-nfs/src/main/java/org/apache/hadoop/nfs/nfs3/response/WRITE3Response.java index 8d4b4d909f640..3f2552a8eca2a 100644 --- a/hadoop-common-project/hadoop-nfs/src/main/java/org/apache/hadoop/nfs/nfs3/response/WRITE3Response.java +++ b/hadoop-common-project/hadoop-nfs/src/main/java/org/apache/hadoop/nfs/nfs3/response/WRITE3Response.java @@ -17,9 +17,7 @@ */ package org.apache.hadoop.nfs.nfs3.response; -import org.apache.hadoop.nfs.nfs3.FileHandle; import org.apache.hadoop.nfs.nfs3.Nfs3Constant; -import org.apache.hadoop.nfs.nfs3.Nfs3FileAttributes; import org.apache.hadoop.nfs.nfs3.Nfs3Status; import org.apache.hadoop.nfs.nfs3.Nfs3Constant.WriteStableHow; import org.apache.hadoop.oncrpc.XDR; diff --git a/hadoop-common-project/hadoop-nfs/src/main/java/org/apache/hadoop/oncrpc/SimpleTcpClientHandler.java b/hadoop-common-project/hadoop-nfs/src/main/java/org/apache/hadoop/oncrpc/SimpleTcpClientHandler.java index 1acefc857f830..163e26b823d04 100644 --- a/hadoop-common-project/hadoop-nfs/src/main/java/org/apache/hadoop/oncrpc/SimpleTcpClientHandler.java +++ b/hadoop-common-project/hadoop-nfs/src/main/java/org/apache/hadoop/oncrpc/SimpleTcpClientHandler.java @@ -18,11 +18,8 @@ package org.apache.hadoop.oncrpc; import io.netty.buffer.ByteBuf; -import io.netty.channel.ChannelFuture; -import io.netty.channel.ChannelFutureListener; import io.netty.channel.ChannelHandlerContext; import io.netty.channel.ChannelInboundHandlerAdapter; -import io.netty.util.ReferenceCountUtil; import org.slf4j.Logger; import org.slf4j.LoggerFactory; diff --git a/hadoop-common-project/hadoop-nfs/src/main/java/org/apache/hadoop/oncrpc/security/SysSecurityHandler.java b/hadoop-common-project/hadoop-nfs/src/main/java/org/apache/hadoop/oncrpc/security/SysSecurityHandler.java index 884bebc97561a..1ca4dbe2cca0c 100644 --- a/hadoop-common-project/hadoop-nfs/src/main/java/org/apache/hadoop/oncrpc/security/SysSecurityHandler.java +++ b/hadoop-common-project/hadoop-nfs/src/main/java/org/apache/hadoop/oncrpc/security/SysSecurityHandler.java @@ -17,7 +17,6 @@ */ package org.apache.hadoop.oncrpc.security; -import org.apache.hadoop.nfs.nfs3.Nfs3Constant; import org.apache.hadoop.oncrpc.RpcCall; import org.apache.hadoop.security.IdMappingConstant; import org.apache.hadoop.security.IdMappingServiceProvider; diff --git a/hadoop-common-project/hadoop-registry/src/main/java/org/apache/hadoop/registry/client/binding/RegistryPathUtils.java b/hadoop-common-project/hadoop-registry/src/main/java/org/apache/hadoop/registry/client/binding/RegistryPathUtils.java index 9fa4b8d84d85e..b0b30bdea5faa 100644 --- a/hadoop-common-project/hadoop-registry/src/main/java/org/apache/hadoop/registry/client/binding/RegistryPathUtils.java +++ b/hadoop-common-project/hadoop-registry/src/main/java/org/apache/hadoop/registry/client/binding/RegistryPathUtils.java @@ -24,7 +24,6 @@ import org.apache.hadoop.fs.PathNotFoundException; import org.apache.hadoop.registry.client.exceptions.InvalidPathnameException; import org.apache.hadoop.registry.client.impl.zk.RegistryInternalConstants; -import org.apache.hadoop.registry.server.dns.BaseServiceRecordProcessor; import org.apache.zookeeper.common.PathUtils; import java.net.IDN; diff --git a/hadoop-common-project/hadoop-registry/src/main/java/org/apache/hadoop/registry/client/binding/RegistryUtils.java b/hadoop-common-project/hadoop-registry/src/main/java/org/apache/hadoop/registry/client/binding/RegistryUtils.java index cff70a613783a..f451792355c50 100644 --- a/hadoop-common-project/hadoop-registry/src/main/java/org/apache/hadoop/registry/client/binding/RegistryUtils.java +++ b/hadoop-common-project/hadoop-registry/src/main/java/org/apache/hadoop/registry/client/binding/RegistryUtils.java @@ -43,7 +43,6 @@ import java.util.Collection; import java.util.HashMap; import java.util.List; -import java.util.Locale; import java.util.Map; /** diff --git a/hadoop-common-project/hadoop-registry/src/main/java/org/apache/hadoop/registry/client/impl/zk/RegistryOperationsService.java b/hadoop-common-project/hadoop-registry/src/main/java/org/apache/hadoop/registry/client/impl/zk/RegistryOperationsService.java index 49ea16bba7d62..96cf911fd9541 100644 --- a/hadoop-common-project/hadoop-registry/src/main/java/org/apache/hadoop/registry/client/impl/zk/RegistryOperationsService.java +++ b/hadoop-common-project/hadoop-registry/src/main/java/org/apache/hadoop/registry/client/impl/zk/RegistryOperationsService.java @@ -28,7 +28,6 @@ import org.apache.hadoop.registry.client.binding.RegistryUtils; import org.apache.hadoop.registry.client.binding.RegistryPathUtils; import org.apache.hadoop.registry.client.exceptions.InvalidPathnameException; -import org.apache.hadoop.registry.client.exceptions.NoRecordException; import org.apache.hadoop.registry.client.types.RegistryPathStatus; import org.apache.hadoop.registry.client.types.ServiceRecord; import org.apache.zookeeper.CreateMode; diff --git a/hadoop-common-project/hadoop-registry/src/main/java/org/apache/hadoop/registry/client/impl/zk/RegistrySecurity.java b/hadoop-common-project/hadoop-registry/src/main/java/org/apache/hadoop/registry/client/impl/zk/RegistrySecurity.java index bcca604182bfd..e500ba6617b5f 100644 --- a/hadoop-common-project/hadoop-registry/src/main/java/org/apache/hadoop/registry/client/impl/zk/RegistrySecurity.java +++ b/hadoop-common-project/hadoop-registry/src/main/java/org/apache/hadoop/registry/client/impl/zk/RegistrySecurity.java @@ -33,7 +33,6 @@ import org.apache.zookeeper.Environment; import org.apache.zookeeper.ZooDefs; import org.apache.zookeeper.client.ZKClientConfig; -import org.apache.zookeeper.client.ZooKeeperSaslClient; import org.apache.zookeeper.data.ACL; import org.apache.zookeeper.data.Id; import org.apache.zookeeper.server.auth.DigestAuthenticationProvider; diff --git a/hadoop-common-project/pom.xml b/hadoop-common-project/pom.xml index b36dbf30610ff..f167a079a9b0c 100644 --- a/hadoop-common-project/pom.xml +++ b/hadoop-common-project/pom.xml @@ -56,5 +56,4 @@ - diff --git a/hadoop-hdfs-project/hadoop-hdfs-client/dev-support/findbugsExcludeFile.xml b/hadoop-hdfs-project/hadoop-hdfs-client/dev-support/findbugsExcludeFile.xml index c96b3a99bd1c4..508388aa4812e 100644 --- a/hadoop-hdfs-project/hadoop-hdfs-client/dev-support/findbugsExcludeFile.xml +++ b/hadoop-hdfs-project/hadoop-hdfs-client/dev-support/findbugsExcludeFile.xml @@ -94,4 +94,17 @@ + + + + + + + + + + + + + diff --git a/hadoop-hdfs-project/hadoop-hdfs-client/pom.xml b/hadoop-hdfs-project/hadoop-hdfs-client/pom.xml index c4e65ef811dbf..9bb0932d32867 100644 --- a/hadoop-hdfs-project/hadoop-hdfs-client/pom.xml +++ b/hadoop-hdfs-project/hadoop-hdfs-client/pom.xml @@ -35,9 +35,17 @@ https://maven.apache.org/xsd/maven-4.0.0.xsd"> - com.squareup.okhttp + com.squareup.okhttp3 okhttp + + org.jetbrains.kotlin + kotlin-stdlib + + + org.jetbrains.kotlin + kotlin-stdlib-common + org.apache.hadoop hadoop-common diff --git a/hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/hdfs/client/HdfsClientConfigKeys.java b/hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/hdfs/client/HdfsClientConfigKeys.java index 7750c48ae6bfe..8e9a5b62490d0 100755 --- a/hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/hdfs/client/HdfsClientConfigKeys.java +++ b/hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/hdfs/client/HdfsClientConfigKeys.java @@ -273,6 +273,14 @@ public interface HdfsClientConfigKeys { String DFS_LEASE_HARDLIMIT_KEY = "dfs.namenode.lease-hard-limit-sec"; long DFS_LEASE_HARDLIMIT_DEFAULT = 20 * 60; + String DFS_CLIENT_FSCK_CONNECT_TIMEOUT = + "dfs.client.fsck.connect.timeout"; + int DFS_CLIENT_FSCK_CONNECT_TIMEOUT_DEFAULT = 60 * 1000; + + String DFS_CLIENT_FSCK_READ_TIMEOUT = + "dfs.client.fsck.read.timeout"; + int DFS_CLIENT_FSCK_READ_TIMEOUT_DEFAULT = 60 * 1000; + /** * These are deprecated config keys to client code. */ diff --git a/hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/hdfs/server/protocol/OutlierMetrics.java b/hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/hdfs/server/protocol/OutlierMetrics.java new file mode 100644 index 0000000000000..452885d0c0538 --- /dev/null +++ b/hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/hdfs/server/protocol/OutlierMetrics.java @@ -0,0 +1,90 @@ +/** + * 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.hdfs.server.protocol; + +import org.apache.commons.lang3.builder.EqualsBuilder; +import org.apache.commons.lang3.builder.HashCodeBuilder; +import org.apache.hadoop.classification.InterfaceAudience; + +/** + * Outlier detection metrics - median, median absolute deviation, upper latency limit, + * actual latency etc. + */ +@InterfaceAudience.Private +public class OutlierMetrics { + + private final Double median; + private final Double mad; + private final Double upperLimitLatency; + private final Double actualLatency; + + public OutlierMetrics(Double median, Double mad, Double upperLimitLatency, + Double actualLatency) { + this.median = median; + this.mad = mad; + this.upperLimitLatency = upperLimitLatency; + this.actualLatency = actualLatency; + } + + public Double getMedian() { + return median; + } + + public Double getMad() { + return mad; + } + + public Double getUpperLimitLatency() { + return upperLimitLatency; + } + + public Double getActualLatency() { + return actualLatency; + } + + @Override + public boolean equals(Object o) { + if (this == o) { + return true; + } + + if (o == null || getClass() != o.getClass()) { + return false; + } + + OutlierMetrics that = (OutlierMetrics) o; + + return new EqualsBuilder() + .append(median, that.median) + .append(mad, that.mad) + .append(upperLimitLatency, that.upperLimitLatency) + .append(actualLatency, that.actualLatency) + .isEquals(); + } + + @Override + public int hashCode() { + return new HashCodeBuilder(17, 37) + .append(median) + .append(mad) + .append(upperLimitLatency) + .append(actualLatency) + .toHashCode(); + } +} diff --git a/hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/hdfs/server/protocol/SlowPeerReports.java b/hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/hdfs/server/protocol/SlowPeerReports.java index a3b3445d5a743..586e2f4931e28 100644 --- a/hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/hdfs/server/protocol/SlowPeerReports.java +++ b/hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/hdfs/server/protocol/SlowPeerReports.java @@ -51,7 +51,7 @@ public final class SlowPeerReports { * meaningful and must be avoided. */ @Nonnull - private final Map slowPeers; + private final Map slowPeers; /** * An object representing a SlowPeerReports with no entries. Should @@ -61,19 +61,19 @@ public final class SlowPeerReports { public static final SlowPeerReports EMPTY_REPORT = new SlowPeerReports(ImmutableMap.of()); - private SlowPeerReports(Map slowPeers) { + private SlowPeerReports(Map slowPeers) { this.slowPeers = slowPeers; } public static SlowPeerReports create( - @Nullable Map slowPeers) { + @Nullable Map slowPeers) { if (slowPeers == null || slowPeers.isEmpty()) { return EMPTY_REPORT; } return new SlowPeerReports(slowPeers); } - public Map getSlowPeers() { + public Map getSlowPeers() { return slowPeers; } diff --git a/hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/hdfs/web/WebHdfsFileSystem.java b/hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/hdfs/web/WebHdfsFileSystem.java index ebc12ff04b919..5afb526675116 100644 --- a/hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/hdfs/web/WebHdfsFileSystem.java +++ b/hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/hdfs/web/WebHdfsFileSystem.java @@ -138,8 +138,6 @@ import org.apache.hadoop.thirdparty.com.google.common.base.Charsets; import org.apache.hadoop.util.Preconditions; -import static org.apache.hadoop.fs.impl.PathCapabilitiesSupport.validatePathCapabilityArgs; - /** A FileSystem for HDFS over the web. */ public class WebHdfsFileSystem extends FileSystem implements DelegationTokenRenewer.Renewable, diff --git a/hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/hdfs/web/oauth2/ConfRefreshTokenBasedAccessTokenProvider.java b/hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/hdfs/web/oauth2/ConfRefreshTokenBasedAccessTokenProvider.java index 3e3fbfbd91380..e944e8c1c8d77 100644 --- a/hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/hdfs/web/oauth2/ConfRefreshTokenBasedAccessTokenProvider.java +++ b/hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/hdfs/web/oauth2/ConfRefreshTokenBasedAccessTokenProvider.java @@ -18,10 +18,15 @@ */ package org.apache.hadoop.hdfs.web.oauth2; -import com.squareup.okhttp.OkHttpClient; -import com.squareup.okhttp.Request; -import com.squareup.okhttp.RequestBody; -import com.squareup.okhttp.Response; +import java.io.IOException; +import java.util.Map; +import java.util.concurrent.TimeUnit; + +import okhttp3.OkHttpClient; +import okhttp3.Request; +import okhttp3.RequestBody; +import okhttp3.Response; + import org.apache.hadoop.classification.InterfaceAudience; import org.apache.hadoop.classification.InterfaceStability; import org.apache.hadoop.conf.Configuration; @@ -30,10 +35,6 @@ import org.apache.hadoop.util.Timer; import org.apache.http.HttpStatus; -import java.io.IOException; -import java.util.Map; -import java.util.concurrent.TimeUnit; - import static org.apache.hadoop.hdfs.client.HdfsClientConfigKeys.OAUTH_CLIENT_ID_KEY; import static org.apache.hadoop.hdfs.client.HdfsClientConfigKeys.OAUTH_REFRESH_URL_KEY; import static org.apache.hadoop.hdfs.web.oauth2.OAuth2Constants.ACCESS_TOKEN; @@ -102,37 +103,34 @@ public synchronized String getAccessToken() throws IOException { } void refresh() throws IOException { - try { - OkHttpClient client = new OkHttpClient(); - client.setConnectTimeout(URLConnectionFactory.DEFAULT_SOCKET_TIMEOUT, - TimeUnit.MILLISECONDS); - client.setReadTimeout(URLConnectionFactory.DEFAULT_SOCKET_TIMEOUT, - TimeUnit.MILLISECONDS); - - String bodyString = Utils.postBody(GRANT_TYPE, REFRESH_TOKEN, - REFRESH_TOKEN, refreshToken, - CLIENT_ID, clientId); - - RequestBody body = RequestBody.create(URLENCODED, bodyString); - - Request request = new Request.Builder() - .url(refreshURL) - .post(body) - .build(); - Response responseBody = client.newCall(request).execute(); - - if (responseBody.code() != HttpStatus.SC_OK) { - throw new IllegalArgumentException("Received invalid http response: " - + responseBody.code() + ", text = " + responseBody.toString()); + OkHttpClient client = + new OkHttpClient.Builder().connectTimeout(URLConnectionFactory.DEFAULT_SOCKET_TIMEOUT, + TimeUnit.MILLISECONDS) + .readTimeout(URLConnectionFactory.DEFAULT_SOCKET_TIMEOUT, TimeUnit.MILLISECONDS) + .build(); + + String bodyString = + Utils.postBody(GRANT_TYPE, REFRESH_TOKEN, REFRESH_TOKEN, refreshToken, CLIENT_ID, clientId); + + RequestBody body = RequestBody.create(bodyString, URLENCODED); + + Request request = new Request.Builder().url(refreshURL).post(body).build(); + try (Response response = client.newCall(request).execute()) { + if (!response.isSuccessful()) { + throw new IOException("Unexpected code " + response); + } + if (response.code() != HttpStatus.SC_OK) { + throw new IllegalArgumentException( + "Received invalid http response: " + response.code() + ", text = " + + response.toString()); } - Map response = JsonSerialization.mapReader().readValue( - responseBody.body().string()); + Map responseBody = JsonSerialization.mapReader().readValue(response.body().string()); - String newExpiresIn = response.get(EXPIRES_IN).toString(); + String newExpiresIn = responseBody.get(EXPIRES_IN).toString(); accessTokenTimer.setExpiresIn(newExpiresIn); - accessToken = response.get(ACCESS_TOKEN).toString(); + accessToken = responseBody.get(ACCESS_TOKEN).toString(); } catch (Exception e) { throw new IOException("Exception while refreshing access token", e); } diff --git a/hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/hdfs/web/oauth2/CredentialBasedAccessTokenProvider.java b/hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/hdfs/web/oauth2/CredentialBasedAccessTokenProvider.java index bfd7055990e81..25ceb8846092b 100644 --- a/hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/hdfs/web/oauth2/CredentialBasedAccessTokenProvider.java +++ b/hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/hdfs/web/oauth2/CredentialBasedAccessTokenProvider.java @@ -18,10 +18,15 @@ */ package org.apache.hadoop.hdfs.web.oauth2; -import com.squareup.okhttp.OkHttpClient; -import com.squareup.okhttp.Request; -import com.squareup.okhttp.RequestBody; -import com.squareup.okhttp.Response; +import java.io.IOException; +import java.util.Map; +import java.util.concurrent.TimeUnit; + +import okhttp3.OkHttpClient; +import okhttp3.Request; +import okhttp3.RequestBody; +import okhttp3.Response; + import org.apache.hadoop.classification.InterfaceAudience; import org.apache.hadoop.classification.InterfaceStability; import org.apache.hadoop.conf.Configuration; @@ -30,10 +35,6 @@ import org.apache.hadoop.util.Timer; import org.apache.http.HttpStatus; -import java.io.IOException; -import java.util.Map; -import java.util.concurrent.TimeUnit; - import static org.apache.hadoop.hdfs.client.HdfsClientConfigKeys.OAUTH_CLIENT_ID_KEY; import static org.apache.hadoop.hdfs.client.HdfsClientConfigKeys.OAUTH_REFRESH_URL_KEY; import static org.apache.hadoop.hdfs.web.oauth2.OAuth2Constants.ACCESS_TOKEN; @@ -96,38 +97,38 @@ public synchronized String getAccessToken() throws IOException { } void refresh() throws IOException { - try { - OkHttpClient client = new OkHttpClient(); - client.setConnectTimeout(URLConnectionFactory.DEFAULT_SOCKET_TIMEOUT, - TimeUnit.MILLISECONDS); - client.setReadTimeout(URLConnectionFactory.DEFAULT_SOCKET_TIMEOUT, - TimeUnit.MILLISECONDS); - - String bodyString = Utils.postBody(CLIENT_SECRET, getCredential(), - GRANT_TYPE, CLIENT_CREDENTIALS, - CLIENT_ID, clientId); - - RequestBody body = RequestBody.create(URLENCODED, bodyString); - - Request request = new Request.Builder() - .url(refreshURL) - .post(body) - .build(); - Response responseBody = client.newCall(request).execute(); - - if (responseBody.code() != HttpStatus.SC_OK) { + OkHttpClient client = new OkHttpClient.Builder() + .connectTimeout(URLConnectionFactory.DEFAULT_SOCKET_TIMEOUT, TimeUnit.MILLISECONDS) + .readTimeout(URLConnectionFactory.DEFAULT_SOCKET_TIMEOUT, TimeUnit.MILLISECONDS) + .build(); + + String bodyString = Utils.postBody(CLIENT_SECRET, getCredential(), + GRANT_TYPE, CLIENT_CREDENTIALS, + CLIENT_ID, clientId); + + RequestBody body = RequestBody.create(bodyString, URLENCODED); + + Request request = new Request.Builder() + .url(refreshURL) + .post(body) + .build(); + try (Response response = client.newCall(request).execute()) { + if (!response.isSuccessful()) { + throw new IOException("Unexpected code " + response); + } + + if (response.code() != HttpStatus.SC_OK) { throw new IllegalArgumentException("Received invalid http response: " - + responseBody.code() + ", text = " + responseBody.toString()); + + response.code() + ", text = " + response.toString()); } - Map response = JsonSerialization.mapReader().readValue( - responseBody.body().string()); + Map responseBody = JsonSerialization.mapReader().readValue( + response.body().string()); - String newExpiresIn = response.get(EXPIRES_IN).toString(); + String newExpiresIn = responseBody.get(EXPIRES_IN).toString(); timer.setExpiresIn(newExpiresIn); - accessToken = response.get(ACCESS_TOKEN).toString(); - + accessToken = responseBody.get(ACCESS_TOKEN).toString(); } catch (Exception e) { throw new IOException("Unable to obtain access token from credential", e); } diff --git a/hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/hdfs/web/oauth2/OAuth2Constants.java b/hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/hdfs/web/oauth2/OAuth2Constants.java index 3f091c2945bcc..2f28b65e40e92 100644 --- a/hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/hdfs/web/oauth2/OAuth2Constants.java +++ b/hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/hdfs/web/oauth2/OAuth2Constants.java @@ -18,7 +18,7 @@ */ package org.apache.hadoop.hdfs.web.oauth2; -import com.squareup.okhttp.MediaType; +import okhttp3.MediaType; import org.apache.hadoop.classification.InterfaceAudience; import org.apache.hadoop.classification.InterfaceStability; diff --git a/hadoop-hdfs-project/hadoop-hdfs-client/src/test/java/org/apache/hadoop/hdfs/server/namenode/ha/TestConfiguredFailoverProxyProvider.java b/hadoop-hdfs-project/hadoop-hdfs-client/src/test/java/org/apache/hadoop/hdfs/server/namenode/ha/TestConfiguredFailoverProxyProvider.java index f28bfa65b59f5..c198536d01a2b 100644 --- a/hadoop-hdfs-project/hadoop-hdfs-client/src/test/java/org/apache/hadoop/hdfs/server/namenode/ha/TestConfiguredFailoverProxyProvider.java +++ b/hadoop-hdfs-project/hadoop-hdfs-client/src/test/java/org/apache/hadoop/hdfs/server/namenode/ha/TestConfiguredFailoverProxyProvider.java @@ -35,7 +35,6 @@ import org.slf4j.event.Level; import java.io.IOException; -import java.net.InetAddress; import java.net.InetSocketAddress; import java.net.URI; import java.net.URISyntaxException; diff --git a/hadoop-hdfs-project/hadoop-hdfs-httpfs/src/main/java/org/apache/hadoop/fs/http/client/HttpFSFileSystem.java b/hadoop-hdfs-project/hadoop-hdfs-httpfs/src/main/java/org/apache/hadoop/fs/http/client/HttpFSFileSystem.java index 0b4de72844403..10dc787fa12b9 100644 --- a/hadoop-hdfs-project/hadoop-hdfs-httpfs/src/main/java/org/apache/hadoop/fs/http/client/HttpFSFileSystem.java +++ b/hadoop-hdfs-project/hadoop-hdfs-httpfs/src/main/java/org/apache/hadoop/fs/http/client/HttpFSFileSystem.java @@ -93,7 +93,6 @@ import java.security.PrivilegedExceptionAction; import java.text.MessageFormat; import java.util.HashMap; -import java.util.Locale; import java.util.Map; import static org.apache.hadoop.fs.impl.PathCapabilitiesSupport.validatePathCapabilityArgs; diff --git a/hadoop-hdfs-project/hadoop-hdfs-httpfs/src/test/java/org/apache/hadoop/fs/http/client/TestHttpFSFileSystemLocalFileSystem.java b/hadoop-hdfs-project/hadoop-hdfs-httpfs/src/test/java/org/apache/hadoop/fs/http/client/TestHttpFSFileSystemLocalFileSystem.java index 83bcb2efacc3c..955529ef9816d 100644 --- a/hadoop-hdfs-project/hadoop-hdfs-httpfs/src/test/java/org/apache/hadoop/fs/http/client/TestHttpFSFileSystemLocalFileSystem.java +++ b/hadoop-hdfs-project/hadoop-hdfs-httpfs/src/test/java/org/apache/hadoop/fs/http/client/TestHttpFSFileSystemLocalFileSystem.java @@ -32,8 +32,6 @@ import org.junit.runners.Parameterized; import java.io.File; -import java.net.URI; -import java.net.URISyntaxException; @RunWith(value = Parameterized.class) public class TestHttpFSFileSystemLocalFileSystem extends BaseTestHttpFSWith { diff --git a/hadoop-hdfs-project/hadoop-hdfs-httpfs/src/test/java/org/apache/hadoop/fs/http/server/TestHttpFSAccessControlled.java b/hadoop-hdfs-project/hadoop-hdfs-httpfs/src/test/java/org/apache/hadoop/fs/http/server/TestHttpFSAccessControlled.java index 7a010e034d2f0..1411cbf78a180 100644 --- a/hadoop-hdfs-project/hadoop-hdfs-httpfs/src/test/java/org/apache/hadoop/fs/http/server/TestHttpFSAccessControlled.java +++ b/hadoop-hdfs-project/hadoop-hdfs-httpfs/src/test/java/org/apache/hadoop/fs/http/server/TestHttpFSAccessControlled.java @@ -21,7 +21,6 @@ import org.apache.hadoop.fs.CommonConfigurationKeysPublic; import org.apache.hadoop.fs.FileSystem; import org.apache.hadoop.fs.Path; -import org.apache.hadoop.hdfs.DFSConfigKeys; import org.apache.hadoop.hdfs.MiniDFSCluster; import org.apache.hadoop.security.authentication.server.AuthenticationFilter; import org.apache.hadoop.test.HTestCase; diff --git a/hadoop-hdfs-project/hadoop-hdfs-httpfs/src/test/java/org/apache/hadoop/lib/server/TestServer.java b/hadoop-hdfs-project/hadoop-hdfs-httpfs/src/test/java/org/apache/hadoop/lib/server/TestServer.java index a6a139f23e16b..ff1d1ca0ad55c 100644 --- a/hadoop-hdfs-project/hadoop-hdfs-httpfs/src/test/java/org/apache/hadoop/lib/server/TestServer.java +++ b/hadoop-hdfs-project/hadoop-hdfs-httpfs/src/test/java/org/apache/hadoop/lib/server/TestServer.java @@ -42,7 +42,6 @@ import org.apache.hadoop.test.TestDir; import org.apache.hadoop.test.TestDirHelper; import org.apache.hadoop.test.TestException; -import org.apache.hadoop.util.Shell; import org.apache.hadoop.util.StringUtils; import org.junit.Test; diff --git a/hadoop-hdfs-project/hadoop-hdfs-native-client/src/main/native/fuse-dfs/fuse_dfs.h b/hadoop-hdfs-project/hadoop-hdfs-native-client/src/main/native/fuse-dfs/fuse_dfs.h index 4554dbdbea584..5ab8ac4131b15 100644 --- a/hadoop-hdfs-project/hadoop-hdfs-native-client/src/main/native/fuse-dfs/fuse_dfs.h +++ b/hadoop-hdfs-project/hadoop-hdfs-native-client/src/main/native/fuse-dfs/fuse_dfs.h @@ -25,7 +25,6 @@ #include #include #include -#include #include #include diff --git a/hadoop-hdfs-project/hadoop-hdfs-native-client/src/main/native/fuse-dfs/fuse_trash.c b/hadoop-hdfs-project/hadoop-hdfs-native-client/src/main/native/fuse-dfs/fuse_trash.c index 02f1b5f282cd3..23a00a28898ce 100644 --- a/hadoop-hdfs-project/hadoop-hdfs-native-client/src/main/native/fuse-dfs/fuse_trash.c +++ b/hadoop-hdfs-project/hadoop-hdfs-native-client/src/main/native/fuse-dfs/fuse_trash.c @@ -21,7 +21,6 @@ #include #include #include -#include #include "fuse_context_handle.h" #include "fuse_dfs.h" diff --git a/hadoop-hdfs-project/hadoop-hdfs-native-client/src/main/native/libhdfs-tests/CMakeLists.txt b/hadoop-hdfs-project/hadoop-hdfs-native-client/src/main/native/libhdfs-tests/CMakeLists.txt index 44bc87d17c67d..36055cc009041 100644 --- a/hadoop-hdfs-project/hadoop-hdfs-native-client/src/main/native/libhdfs-tests/CMakeLists.txt +++ b/hadoop-hdfs-project/hadoop-hdfs-native-client/src/main/native/libhdfs-tests/CMakeLists.txt @@ -35,6 +35,8 @@ add_library(native_mini_dfs ../libhdfs/jni_helper.c ${OS_DIR}/mutexes.c ${OS_DIR}/thread_local_storage.c + $ + $ ) add_executable(test_native_mini_dfs test_native_mini_dfs.c) diff --git a/hadoop-hdfs-project/hadoop-hdfs-native-client/src/main/native/libhdfs/CMakeLists.txt b/hadoop-hdfs-project/hadoop-hdfs-native-client/src/main/native/libhdfs/CMakeLists.txt index 22d18708d6e93..958bdcf2a41c9 100644 --- a/hadoop-hdfs-project/hadoop-hdfs-native-client/src/main/native/libhdfs/CMakeLists.txt +++ b/hadoop-hdfs-project/hadoop-hdfs-native-client/src/main/native/libhdfs/CMakeLists.txt @@ -39,6 +39,8 @@ hadoop_add_dual_library(hdfs jclasses.c ${OS_DIR}/mutexes.c ${OS_DIR}/thread_local_storage.c + $ + $ ) if(NEED_LINK_DL) set(LIB_DL dl) diff --git a/hadoop-hdfs-project/hadoop-hdfs-native-client/src/main/native/libhdfs/jni_helper.c b/hadoop-hdfs-project/hadoop-hdfs-native-client/src/main/native/libhdfs/jni_helper.c index c834c74e4cd09..8f00a08b0a98b 100644 --- a/hadoop-hdfs-project/hadoop-hdfs-native-client/src/main/native/libhdfs/jni_helper.c +++ b/hadoop-hdfs-project/hadoop-hdfs-native-client/src/main/native/libhdfs/jni_helper.c @@ -23,10 +23,10 @@ #include "platform.h" #include "os/mutexes.h" #include "os/thread_local_storage.h" +#include "x-platform/c-api/dirent.h" #include "x-platform/types.h" #include -#include #include #include diff --git a/hadoop-hdfs-project/hadoop-hdfs-native-client/src/main/native/libhdfspp/CMakeLists-gtest.txt.in b/hadoop-hdfs-project/hadoop-hdfs-native-client/src/main/native/libhdfspp/CMakeLists-gtest.txt.in deleted file mode 100644 index 10a414588d59f..0000000000000 --- a/hadoop-hdfs-project/hadoop-hdfs-native-client/src/main/native/libhdfspp/CMakeLists-gtest.txt.in +++ /dev/null @@ -1,33 +0,0 @@ -# -# 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. -# - -cmake_minimum_required(VERSION 3.1 FATAL_ERROR) - -project(googletest-download NONE) - -include(ExternalProject) -ExternalProject_Add(googletest - GIT_REPOSITORY https://github.com/google/googletest.git - GIT_TAG release-1.10.0 - SOURCE_DIR "${CMAKE_CURRENT_BINARY_DIR}/googletest-src" - BINARY_DIR "${CMAKE_CURRENT_BINARY_DIR}/googletest-build" - CONFIGURE_COMMAND "" - BUILD_COMMAND "" - INSTALL_COMMAND "" - TEST_COMMAND "" -) diff --git a/hadoop-hdfs-project/hadoop-hdfs-native-client/src/main/native/libhdfspp/CMakeLists.txt b/hadoop-hdfs-project/hadoop-hdfs-native-client/src/main/native/libhdfspp/CMakeLists.txt index d23f84941db9b..078e392d4da47 100644 --- a/hadoop-hdfs-project/hadoop-hdfs-native-client/src/main/native/libhdfspp/CMakeLists.txt +++ b/hadoop-hdfs-project/hadoop-hdfs-native-client/src/main/native/libhdfspp/CMakeLists.txt @@ -52,31 +52,20 @@ find_package(Threads) include(CheckCXXSourceCompiles) include(CheckSymbolExists) - -# Download and build gtest -configure_file(CMakeLists-gtest.txt.in googletest-download/CMakeLists.txt) -execute_process(COMMAND ${CMAKE_COMMAND} -G "${CMAKE_GENERATOR}" . - RESULT_VARIABLE result - WORKING_DIRECTORY ${CMAKE_CURRENT_BINARY_DIR}/googletest-download) -if(result) - message(FATAL_ERROR "CMake step for googletest failed: ${result}") -endif() -execute_process(COMMAND ${CMAKE_COMMAND} --build . - RESULT_VARIABLE result - WORKING_DIRECTORY ${CMAKE_CURRENT_BINARY_DIR}/googletest-download) -if(result) - message(FATAL_ERROR "Build step for googletest failed: ${result}") -endif() - -# Prevent overriding the parent project's compiler/linker -# settings on Windows +include(FetchContent) + +# Install googletest +# As per the approach documented in the GoogleTest repo - +# https://github.com/google/googletest/blob/e649993a402d96afe25fbf3413749adf0f2947f6/googletest/README.md#incorporating-into-an-existing-cmake-project +FetchContent_Declare( + googletest + URL https://github.com/google/googletest/archive/703bd9caab50b139428cea1aaff9974ebee5742e.zip +) +# The commit SHA 703bd9caab50b139428cea1aaff9974ebee5742e in the URL above corresponds to the +# tag release-1.10.0 in the googletest repository. +# For Windows: Prevent overriding the parent project's compiler/linker settings set(gtest_force_shared_crt ON CACHE BOOL "" FORCE) - -# Add googletest directly to our build. This defines -# the gtest and gtest_main targets. -add_subdirectory(${CMAKE_CURRENT_BINARY_DIR}/googletest-src - ${CMAKE_CURRENT_BINARY_DIR}/googletest-build - EXCLUDE_FROM_ALL) +FetchContent_MakeAvailable(googletest) # Check if thread_local is supported unset (THREAD_LOCAL_SUPPORTED CACHE) diff --git a/hadoop-hdfs-project/hadoop-hdfs-native-client/src/main/native/libhdfspp/lib/common/configuration.cc b/hadoop-hdfs-project/hadoop-hdfs-native-client/src/main/native/libhdfspp/lib/common/configuration.cc index 947214bdbd50a..8219740ef05d7 100644 --- a/hadoop-hdfs-project/hadoop-hdfs-native-client/src/main/native/libhdfspp/lib/common/configuration.cc +++ b/hadoop-hdfs-project/hadoop-hdfs-native-client/src/main/native/libhdfspp/lib/common/configuration.cc @@ -35,7 +35,6 @@ #include "hdfspp/uri.h" #include "x-platform/syscall.h" -#include #include #include #include diff --git a/hadoop-hdfs-project/hadoop-hdfs-native-client/src/main/native/libhdfspp/lib/common/configuration_loader.cc b/hadoop-hdfs-project/hadoop-hdfs-native-client/src/main/native/libhdfspp/lib/common/configuration_loader.cc index 5301137505af1..7d88550a81f71 100644 --- a/hadoop-hdfs-project/hadoop-hdfs-native-client/src/main/native/libhdfspp/lib/common/configuration_loader.cc +++ b/hadoop-hdfs-project/hadoop-hdfs-native-client/src/main/native/libhdfspp/lib/common/configuration_loader.cc @@ -21,7 +21,6 @@ #include "x-platform/syscall.h" #include -#include #include #include #include diff --git a/hadoop-hdfs-project/hadoop-hdfs-native-client/src/main/native/libhdfspp/lib/proto/CMakeLists.txt b/hadoop-hdfs-project/hadoop-hdfs-native-client/src/main/native/libhdfspp/lib/proto/CMakeLists.txt index d47b0886c1ba9..35a38258f5f49 100644 --- a/hadoop-hdfs-project/hadoop-hdfs-native-client/src/main/native/libhdfspp/lib/proto/CMakeLists.txt +++ b/hadoop-hdfs-project/hadoop-hdfs-native-client/src/main/native/libhdfspp/lib/proto/CMakeLists.txt @@ -67,10 +67,10 @@ function(GEN_HRPC SRCS) add_custom_command( OUTPUT "${CMAKE_CURRENT_BINARY_DIR}/${FIL_WE}.hrpc.inl" COMMAND ${PROTOBUF_PROTOC_EXECUTABLE} - ARGS --plugin=protoc-gen-hrpc=${CMAKE_CURRENT_BINARY_DIR}/protoc-gen-hrpc --hrpc_out=${CMAKE_CURRENT_BINARY_DIR} ${_protobuf_include_path} ${ABS_FIL} + ARGS --plugin=protoc-gen-hrpc=$ --hrpc_out=${CMAKE_CURRENT_BINARY_DIR} ${_protobuf_include_path} ${ABS_FIL} DEPENDS ${ABS_FIL} ${PROTOBUF_PROTOC_EXECUTABLE} protoc-gen-hrpc COMMENT "Running HRPC protocol buffer compiler on ${FIL}" - VERBATIM ) + VERBATIM) endforeach() set_source_files_properties(${${SRCS}} PROPERTIES GENERATED TRUE) diff --git a/hadoop-hdfs-project/hadoop-hdfs-native-client/src/main/native/libhdfspp/lib/x-platform/CMakeLists.txt b/hadoop-hdfs-project/hadoop-hdfs-native-client/src/main/native/libhdfspp/lib/x-platform/CMakeLists.txt index 77d0216d4877a..44e2943aef3ce 100644 --- a/hadoop-hdfs-project/hadoop-hdfs-native-client/src/main/native/libhdfspp/lib/x-platform/CMakeLists.txt +++ b/hadoop-hdfs-project/hadoop-hdfs-native-client/src/main/native/libhdfspp/lib/x-platform/CMakeLists.txt @@ -22,5 +22,6 @@ else() set(SYSCALL_SRC syscall_linux.cc) endif() -add_library(x_platform_obj OBJECT ${SYSCALL_SRC} utils.cc) -add_library(x_platform_obj_c_api OBJECT $ c-api/syscall.cc) +add_library(x_platform_obj OBJECT ${SYSCALL_SRC} utils.cc dirent.cc) +add_library(x_platform_obj_c_api OBJECT $ c-api/syscall.cc c-api/dirent.cc) +target_compile_definitions(x_platform_obj_c_api PRIVATE USE_X_PLATFORM_DIRENT) diff --git a/hadoop-hdfs-project/hadoop-hdfs-native-client/src/main/native/libhdfspp/lib/x-platform/c-api/core/dirent.h b/hadoop-hdfs-project/hadoop-hdfs-native-client/src/main/native/libhdfspp/lib/x-platform/c-api/core/dirent.h new file mode 100644 index 0000000000000..1a592dbeeb58f --- /dev/null +++ b/hadoop-hdfs-project/hadoop-hdfs-native-client/src/main/native/libhdfspp/lib/x-platform/c-api/core/dirent.h @@ -0,0 +1,68 @@ +/** + * 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. + */ + +#ifndef NATIVE_LIBHDFSPP_LIB_CROSS_PLATFORM_C_API_CORE_DIRENT_H +#define NATIVE_LIBHDFSPP_LIB_CROSS_PLATFORM_C_API_CORE_DIRENT_H + +/** + * DIR struct holds the pointer to XPlatform::Dirent instance. Since this will + * be used in C, we can't hold the pointer to XPlatform::Dirent. We're working + * around this by using a void pointer and casting it to XPlatform::Dirent when + * needed in C++. + */ +typedef struct DIR { + void *x_platform_dirent_ptr; +} DIR; + +/** + * dirent struct contains the name of the file/folder while iterating through + * the directory's children. + */ +struct dirent { + char d_name[256]; +}; + +/** + * Opens a directory for iteration. Internally, it instantiates DIR struct for + * the given path. closedir must be called on the returned pointer to DIR struct + * when done. + * + * @param dir_path The path to the directory to iterate through. + * @return A pointer to the DIR struct. + */ +DIR *opendir(const char *dir_path); + +/** + * For iterating through the children of the directory pointed to by the DIR + * struct pointer. + * + * @param dir The pointer to the DIR struct. + * @return A pointer to dirent struct containing the name of the current child + * file/folder. + */ +struct dirent *readdir(DIR *dir); + +/** + * De-allocates the XPlatform::Dirent instance pointed to by the DIR pointer. + * + * @param dir The pointer to DIR struct to close. + * @return 0 if successful. + */ +int closedir(DIR *dir); + +#endif \ No newline at end of file diff --git a/hadoop-hdfs-project/hadoop-hdfs-native-client/src/main/native/libhdfspp/lib/x-platform/c-api/dirent.cc b/hadoop-hdfs-project/hadoop-hdfs-native-client/src/main/native/libhdfspp/lib/x-platform/c-api/dirent.cc new file mode 100644 index 0000000000000..24cebc1192d60 --- /dev/null +++ b/hadoop-hdfs-project/hadoop-hdfs-native-client/src/main/native/libhdfspp/lib/x-platform/c-api/dirent.cc @@ -0,0 +1,92 @@ +/** + * 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. + */ + +#include +#include +#include +#include +#include +#include + +#include "x-platform/c-api/dirent.h" +#include "x-platform/dirent.h" + +DIR *opendir(const char *dir_path) { + const auto dir = new DIR; + dir->x_platform_dirent_ptr = new XPlatform::Dirent(dir_path); + return dir; +} + +struct dirent *readdir(DIR *dir) { + /* + * We will use a static variable to hold the dirent, so that we align with the + * readdir's implementation in dirent.h header file in Linux. + */ + static struct dirent static_dir_entry; + + // Get the XPlatform::Dirent instance and move the iterator. + const auto x_platform_dirent = + static_cast(dir->x_platform_dirent_ptr); + const auto dir_entry = x_platform_dirent->NextFile(); + + // End of iteration. + if (std::holds_alternative(dir_entry)) { + return nullptr; + } + + // Error in iteration. + if (std::holds_alternative(dir_entry)) { + const auto err = std::get(dir_entry); + errno = err.value(); + +#ifdef X_PLATFORM_C_API_DIRENT_DEBUG + std::cerr << "Error in listing directory: " << err.message() << std::endl; +#endif + + return nullptr; + } + + // Return the current child file/folder's name. + if (std::holds_alternative(dir_entry)) { + const auto entry = std::get(dir_entry); + const auto filename = entry.path().filename().string(); + + // The file name's length shouldn't exceed 256. + if (filename.length() >= 256) { + errno = 1; + return nullptr; + } + + std::fill(std::begin(static_dir_entry.d_name), + std::end(static_dir_entry.d_name), '\0'); + std::copy(filename.begin(), filename.end(), + std::begin(static_dir_entry.d_name)); + } + return &static_dir_entry; +} + +int closedir(DIR *dir) { + const auto x_platform_dirent = + static_cast(dir->x_platform_dirent_ptr); + delete x_platform_dirent; + delete dir; + + // We can't use the void return type for closedir since we want to align the + // closedir method's signature in dirent.h header file in Linux. + return 0; +} diff --git a/hadoop-hdfs-project/hadoop-hdfs-native-client/src/main/native/libhdfspp/lib/x-platform/c-api/dirent.h b/hadoop-hdfs-project/hadoop-hdfs-native-client/src/main/native/libhdfspp/lib/x-platform/c-api/dirent.h new file mode 100644 index 0000000000000..23fa9d997ae11 --- /dev/null +++ b/hadoop-hdfs-project/hadoop-hdfs-native-client/src/main/native/libhdfspp/lib/x-platform/c-api/dirent.h @@ -0,0 +1,39 @@ +/** + * 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. + */ + +#ifndef NATIVE_LIBHDFSPP_LIB_CROSS_PLATFORM_C_API_DIRENT_H +#define NATIVE_LIBHDFSPP_LIB_CROSS_PLATFORM_C_API_DIRENT_H + +#if !(defined(WIN32) || defined(USE_X_PLATFORM_DIRENT)) + +/* + * For non-Windows environments, we use the dirent.h header itself. + */ +#include + +#else + +/* + * If it's a Windows environment or if the macro USE_X_PLATFORM_DIRENT is + * defined, we switch to using dirent from the XPlatform library. + */ +#include "x-platform/c-api/extern/dirent.h" + +#endif + +#endif diff --git a/hadoop-hdfs-project/hadoop-hdfs-rbf/src/main/java/org/apache/hadoop/hdfs/protocolPB/package-info.java b/hadoop-hdfs-project/hadoop-hdfs-native-client/src/main/native/libhdfspp/lib/x-platform/c-api/extern/dirent.h similarity index 69% rename from hadoop-hdfs-project/hadoop-hdfs-rbf/src/main/java/org/apache/hadoop/hdfs/protocolPB/package-info.java rename to hadoop-hdfs-project/hadoop-hdfs-native-client/src/main/native/libhdfspp/lib/x-platform/c-api/extern/dirent.h index 6233024467df7..1845a831deb92 100644 --- a/hadoop-hdfs-project/hadoop-hdfs-rbf/src/main/java/org/apache/hadoop/hdfs/protocolPB/package-info.java +++ b/hadoop-hdfs-project/hadoop-hdfs-native-client/src/main/native/libhdfspp/lib/x-platform/c-api/extern/dirent.h @@ -15,4 +15,21 @@ * See the License for the specific language governing permissions and * limitations under the License. */ -package org.apache.hadoop.hdfs.protocolPB; \ No newline at end of file + +#ifndef NATIVE_LIBHDFSPP_LIB_CROSS_PLATFORM_C_API_EXTERN_DIRENT_H +#define NATIVE_LIBHDFSPP_LIB_CROSS_PLATFORM_C_API_EXTERN_DIRENT_H + +/* + * We will use extern "C" only on Windows. + */ +#if defined(WIN32) && defined(__cplusplus) +extern "C" { +#endif + +#include "x-platform/c-api/core/dirent.h" + +#if defined(WIN32) && defined(__cplusplus) +} +#endif + +#endif diff --git a/hadoop-hdfs-project/hadoop-hdfs-native-client/src/main/native/libhdfspp/lib/x-platform/dirent.cc b/hadoop-hdfs-project/hadoop-hdfs-native-client/src/main/native/libhdfspp/lib/x-platform/dirent.cc new file mode 100644 index 0000000000000..c53e7986979f1 --- /dev/null +++ b/hadoop-hdfs-project/hadoop-hdfs-native-client/src/main/native/libhdfspp/lib/x-platform/dirent.cc @@ -0,0 +1,38 @@ +/** + * 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. + */ + +#include +#include +#include + +#include "dirent.h" + +std::variant +XPlatform::Dirent::NextFile() { + if (dir_it_err_) { + return dir_it_err_; + } + + if (dir_it_ == std::filesystem::end(dir_it_)) { + return std::monostate(); + } + + const std::filesystem::directory_entry dir_entry = *dir_it_; + dir_it_ = dir_it_.increment(dir_it_err_); + return dir_entry; +} diff --git a/hadoop-hdfs-project/hadoop-hdfs-native-client/src/main/native/libhdfspp/lib/x-platform/dirent.h b/hadoop-hdfs-project/hadoop-hdfs-native-client/src/main/native/libhdfspp/lib/x-platform/dirent.h new file mode 100644 index 0000000000000..91b586d921baa --- /dev/null +++ b/hadoop-hdfs-project/hadoop-hdfs-native-client/src/main/native/libhdfspp/lib/x-platform/dirent.h @@ -0,0 +1,77 @@ +/** + * 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. + */ + +#ifndef NATIVE_LIBHDFSPP_LIB_CROSS_PLATFORM_DIRENT +#define NATIVE_LIBHDFSPP_LIB_CROSS_PLATFORM_DIRENT + +#include +#include +#include +#include + +namespace XPlatform { +/** + * {@class XPlatform::Dirent} provides the functionality to perform a one-time + * iteration per {@link XPlatform::Dirent} through the child files or folders + * under a given path. + */ +class Dirent { +public: + Dirent(const std::string &path) + : dir_it_{std::filesystem::path{path}, dir_it_err_} {} + + // Abiding to the Rule of 5 + Dirent(const Dirent &) = default; + Dirent(Dirent &&) = default; + Dirent &operator=(const Dirent &) = default; + Dirent &operator=(Dirent &&) = default; + ~Dirent() = default; + + /** + * Advances the iterator {@link XPlatform::Dirent#dir_it_} to the next file in + * the given path. + * + * @return An {@link std::variant} comprising of any one of the following + * types: + * 1. {@link std::monostate} which indicates the end of iteration of all the + * files in the given path. + * 2. {@link std::filesystem::directory_entry} which is the directory entry of + * the current file. + * 3. {@link std::error_code} which corresponds to the error in retrieving the + * file. + */ + std::variant + NextFile(); + +private: + /** + * Indicates the error corresponding to the most recent invocation of + * directory iteration by {@link XPlatform::Dirent#dir_it_}. + */ + std::error_code dir_it_err_{}; + + /** + * The iterator used for iterating through the files or folders under the + * given path. + */ + std::filesystem::directory_iterator dir_it_; +}; +} // namespace XPlatform + +#endif diff --git a/hadoop-hdfs-project/hadoop-hdfs-native-client/src/main/native/libhdfspp/lib/x-platform/types.h b/hadoop-hdfs-project/hadoop-hdfs-native-client/src/main/native/libhdfspp/lib/x-platform/types.h index 6df5b96f39337..0719fe90dae54 100644 --- a/hadoop-hdfs-project/hadoop-hdfs-native-client/src/main/native/libhdfspp/lib/x-platform/types.h +++ b/hadoop-hdfs-project/hadoop-hdfs-native-client/src/main/native/libhdfspp/lib/x-platform/types.h @@ -19,10 +19,10 @@ #ifndef NATIVE_LIBHDFSPP_LIB_CROSS_PLATFORM_TYPES #define NATIVE_LIBHDFSPP_LIB_CROSS_PLATFORM_TYPES -#if _WIN64 +#if defined(_WIN64) // Windows 64-bit. typedef long int ssize_t; -#elif _WIN32 +#elif defined(_WIN32) // Windows 32-bit. typedef int ssize_t; #else diff --git a/hadoop-hdfs-project/hadoop-hdfs-native-client/src/main/native/libhdfspp/tests/tools/hdfs-chmod-mock.cc b/hadoop-hdfs-project/hadoop-hdfs-native-client/src/main/native/libhdfspp/tests/tools/hdfs-chmod-mock.cc index e0df32cc2a16a..3c4ea24e0db79 100644 --- a/hadoop-hdfs-project/hadoop-hdfs-native-client/src/main/native/libhdfspp/tests/tools/hdfs-chmod-mock.cc +++ b/hadoop-hdfs-project/hadoop-hdfs-native-client/src/main/native/libhdfspp/tests/tools/hdfs-chmod-mock.cc @@ -55,6 +55,15 @@ void ChmodMock::SetExpectations( .WillOnce(testing::Return(true)); } + if (*test_case_func == &PassInvalidPermissionsAndAPath) { + const auto arg1 = args[0]; + const auto arg2 = args[1]; + + EXPECT_CALL(*this, HandlePath(arg1, false, arg2)) + .Times(1) + .WillOnce(testing::Return(false)); + } + if (*test_case_func == &PassRecursivePermissionsAndAPath) { const auto arg1 = args[1]; const auto arg2 = args[2]; diff --git a/hadoop-hdfs-project/hadoop-hdfs-native-client/src/main/native/libhdfspp/tests/tools/hdfs-tool-tests.h b/hadoop-hdfs-project/hadoop-hdfs-native-client/src/main/native/libhdfspp/tests/tools/hdfs-tool-tests.h index 4fef261d0e72f..8a1ae8cf0b772 100644 --- a/hadoop-hdfs-project/hadoop-hdfs-native-client/src/main/native/libhdfspp/tests/tools/hdfs-tool-tests.h +++ b/hadoop-hdfs-project/hadoop-hdfs-native-client/src/main/native/libhdfspp/tests/tools/hdfs-tool-tests.h @@ -175,6 +175,19 @@ template std::unique_ptr PassPermissionsAndAPath() { return hdfs_tool; } +template std::unique_ptr PassInvalidPermissionsAndAPath() { + constexpr auto argc = 3; + static std::string exe("hdfs_tool_name"); + static std::string arg1("123456789123456789123456789"); + static std::string arg2("g/h/i"); + + static char *argv[] = {exe.data(), arg1.data(), arg2.data()}; + + auto hdfs_tool = std::make_unique(argc, argv); + hdfs_tool->SetExpectations(PassInvalidPermissionsAndAPath, {arg1, arg2}); + return hdfs_tool; +} + template std::unique_ptr PassRecursivePermissionsAndAPath() { constexpr auto argc = 4; static std::string exe("hdfs_tool_name"); diff --git a/hadoop-hdfs-project/hadoop-hdfs-native-client/src/main/native/libhdfspp/tests/x-platform/CMakeLists.txt b/hadoop-hdfs-project/hadoop-hdfs-native-client/src/main/native/libhdfspp/tests/x-platform/CMakeLists.txt index e481ebc31e862..565346e088376 100644 --- a/hadoop-hdfs-project/hadoop-hdfs-native-client/src/main/native/libhdfspp/tests/x-platform/CMakeLists.txt +++ b/hadoop-hdfs-project/hadoop-hdfs-native-client/src/main/native/libhdfspp/tests/x-platform/CMakeLists.txt @@ -36,3 +36,15 @@ add_executable(x_platform_types_test types_test.cc) target_include_directories(x_platform_types_test PRIVATE ${LIBHDFSPP_LIB_DIR}) target_link_libraries(x_platform_types_test gtest_main) add_test(x_platform_types_test x_platform_types_test) + +add_library(x_platform_dirent_test_obj OBJECT $ dirent_test.cc) +add_executable(x_platform_dirent_test $ $) +target_include_directories(x_platform_dirent_test PRIVATE ${LIBHDFSPP_LIB_DIR}) +target_link_libraries(x_platform_dirent_test PRIVATE gtest_main) +add_test(x_platform_dirent_test x_platform_dirent_test) + +add_executable(x_platform_dirent_c_test $ $ $ c-api/dirent_test.cc) +target_compile_definitions(x_platform_dirent_c_test PRIVATE USE_X_PLATFORM_DIRENT) +target_include_directories(x_platform_dirent_c_test PRIVATE ${LIBHDFSPP_LIB_DIR} ../) +target_link_libraries(x_platform_dirent_c_test PRIVATE gtest_main) +add_test(x_platform_dirent_c_test x_platform_dirent_c_test) diff --git a/hadoop-hdfs-project/hadoop-hdfs-native-client/src/main/native/libhdfspp/tests/x-platform/c-api/dirent_test.cc b/hadoop-hdfs-project/hadoop-hdfs-native-client/src/main/native/libhdfspp/tests/x-platform/c-api/dirent_test.cc new file mode 100644 index 0000000000000..6456b02e84d09 --- /dev/null +++ b/hadoop-hdfs-project/hadoop-hdfs-native-client/src/main/native/libhdfspp/tests/x-platform/c-api/dirent_test.cc @@ -0,0 +1,100 @@ +/** + * 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. + */ + +#include +#include +#include +#include +#include +#include + +#include "x-platform/c-api/dirent.h" +#include "x-platform/c-api/dirent_test.h" + +std::unordered_set +DirentCApiTest::ListDirAndFiles(const std::string &path) const { + std::stringstream err_msg; + std::unordered_set paths; + + DIR *dir = opendir(path.c_str()); + if (dir == nullptr) { + err_msg << "Unable to open directory " << path; + throw std::runtime_error(err_msg.str()); + } + + errno = 0; + for (struct dirent *file; (file = readdir(dir)) != nullptr; errno = 0) { + std::filesystem::path absolute_path{path}; + absolute_path = absolute_path / file->d_name; + paths.emplace(absolute_path.string()); + } + + if (errno != 0) { + err_msg << "Expected errno to be 0, instead it is " << errno; + throw std::runtime_error(err_msg.str()); + } + + if (const auto result = closedir(dir); result != 0) { + err_msg << "Expected the return value of closedir() to be 0, instead it is " + << result; + throw std::runtime_error(err_msg.str()); + } + return paths; +} + +TEST_F(DirentCApiTest, TestEmptyFolder) { + const auto expected = CreateTempDirAndFiles(0, 0); + const auto actual = ListDirAndFiles(tmp_root_.string()); + EXPECT_EQ(expected, actual); +} + +TEST_F(DirentCApiTest, TestOneFolder) { + const auto expected = CreateTempDirAndFiles(1, 0); + const auto actual = ListDirAndFiles(tmp_root_.string()); + EXPECT_EQ(expected, actual); +} + +TEST_F(DirentCApiTest, TestOneFile) { + const auto expected = CreateTempDirAndFiles(0, 1); + const auto actual = ListDirAndFiles(tmp_root_.string()); + EXPECT_EQ(expected, actual); +} + +TEST_F(DirentCApiTest, TestMultipleFolders) { + const auto expected = CreateTempDirAndFiles(10, 0); + const auto actual = ListDirAndFiles(tmp_root_.string()); + EXPECT_EQ(expected, actual); +} + +TEST_F(DirentCApiTest, TestMultipleFiles) { + const auto expected = CreateTempDirAndFiles(0, 10); + const auto actual = ListDirAndFiles(tmp_root_.string()); + EXPECT_EQ(expected, actual); +} + +TEST_F(DirentCApiTest, TestOneFileAndFolder) { + const auto expected = CreateTempDirAndFiles(1, 1); + const auto actual = ListDirAndFiles(tmp_root_.string()); + EXPECT_EQ(expected, actual); +} + +TEST_F(DirentCApiTest, TestMultipleFilesAndFolders) { + const auto expected = CreateTempDirAndFiles(10, 10); + const auto actual = ListDirAndFiles(tmp_root_.string()); + EXPECT_EQ(expected, actual); +} diff --git a/hadoop-hdfs-project/hadoop-hdfs-native-client/src/main/native/libhdfspp/tests/x-platform/c-api/dirent_test.h b/hadoop-hdfs-project/hadoop-hdfs-native-client/src/main/native/libhdfspp/tests/x-platform/c-api/dirent_test.h new file mode 100644 index 0000000000000..6e7054c78cb28 --- /dev/null +++ b/hadoop-hdfs-project/hadoop-hdfs-native-client/src/main/native/libhdfspp/tests/x-platform/c-api/dirent_test.h @@ -0,0 +1,37 @@ +/** + * 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. + */ + +#ifndef LIBHDFSPP_CROSS_PLATFORM_C_API_DIRENT_TEST +#define LIBHDFSPP_CROSS_PLATFORM_C_API_DIRENT_TEST + +#include "x-platform/dirent.h" +#include "x-platform/dirent_test.h" + +/** + * Test fixture for testing {@link XPlatform::Dirent}'s C APIs. + */ +class DirentCApiTest : public DirentTest { +protected: + /** + * {@inheritdoc} + */ + [[nodiscard]] std::unordered_set + ListDirAndFiles(const std::string &path) const override; +}; + +#endif \ No newline at end of file diff --git a/hadoop-hdfs-project/hadoop-hdfs-native-client/src/main/native/libhdfspp/tests/x-platform/dirent_test.cc b/hadoop-hdfs-project/hadoop-hdfs-native-client/src/main/native/libhdfspp/tests/x-platform/dirent_test.cc new file mode 100644 index 0000000000000..d0c9f98b0ec1e --- /dev/null +++ b/hadoop-hdfs-project/hadoop-hdfs-native-client/src/main/native/libhdfspp/tests/x-platform/dirent_test.cc @@ -0,0 +1,151 @@ +/** + * 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. + */ + +#include +#include +#include +#include +#include +#include +#include +#include +#include +#include +#include + +#include + +#include "dirent_test.h" +#include "x-platform/dirent.h" +#include "x-platform/syscall.h" + +void DirentTest::SetUp() { + tmp_root_ = std::filesystem::temp_directory_path() / GetTempName(); + if (!std::filesystem::create_directories(tmp_root_)) { + std::stringstream err_msg; + err_msg << "Unable to create temp directory " << tmp_root_.string(); + throw std::runtime_error(err_msg.str()); + } +} + +void DirentTest::TearDown() { std::filesystem::remove_all(tmp_root_); } + +std::string DirentTest::GetTempName(const std::string &pattern) const { + std::vector pattern_raw(pattern.begin(), pattern.end()); + if (!XPlatform::Syscall::CreateTempDir(pattern_raw)) { + std::stringstream err_msg; + err_msg << "Creating temp dir failed" << std::endl; + throw std::runtime_error(err_msg.str()); + } + + std::string tmp_dir_path(pattern_raw.data()); + std::filesystem::remove(tmp_dir_path); + return tmp_dir_path; +} + +std::unordered_set +DirentTest::CreateTempDirAndFiles(std::size_t num_dirs, + std::size_t num_files) const { + std::unordered_set paths; + for (std::size_t i = 0; i < num_dirs; ++i) { + const auto tmp_dir_absolute_path = tmp_root_ / std::to_string(i); + if (std::error_code err; + !std::filesystem::create_directories(tmp_dir_absolute_path, err)) { + std::stringstream err_msg; + err_msg << "Unable to create the temp dir " + << tmp_dir_absolute_path.string() << " reason: " << err.message(); + throw std::runtime_error(err_msg.str()); + } + paths.emplace(tmp_dir_absolute_path.string()); + } + + for (std::size_t i = 0; i < num_files; ++i) { + std::stringstream tmp_filename; + tmp_filename << i << ".txt"; + const auto tmp_file_absolute_path = tmp_root_ / tmp_filename.str(); + std::ofstream{tmp_file_absolute_path}; + paths.emplace(tmp_file_absolute_path.string()); + } + return paths; +} + +std::unordered_set +DirentTest::ListDirAndFiles(const std::string &path) const { + std::unordered_set paths; + + XPlatform::Dirent dirent(path); + for (auto dir_entry = dirent.NextFile(); + !std::holds_alternative(dir_entry); + dir_entry = dirent.NextFile()) { + if (std::holds_alternative(dir_entry)) { + std::stringstream err_msg; + const auto err = std::get(dir_entry); + err_msg << "Error in listing directory " << path + << " cause: " << err.message(); + throw std::runtime_error(err_msg.str()); + } + + if (std::holds_alternative(dir_entry)) { + const auto entry = std::get(dir_entry); + paths.emplace(entry.path().string()); + } + } + return paths; +} + +TEST_F(DirentTest, TestEmptyFolder) { + const auto expected = CreateTempDirAndFiles(0, 0); + const auto actual = ListDirAndFiles(tmp_root_.string()); + EXPECT_EQ(expected, actual); +} + +TEST_F(DirentTest, TestOneFolder) { + const auto expected = CreateTempDirAndFiles(1, 0); + const auto actual = ListDirAndFiles(tmp_root_.string()); + EXPECT_EQ(expected, actual); +} + +TEST_F(DirentTest, TestOneFile) { + const auto expected = CreateTempDirAndFiles(0, 1); + const auto actual = ListDirAndFiles(tmp_root_.string()); + EXPECT_EQ(expected, actual); +} + +TEST_F(DirentTest, TestMultipleFolders) { + const auto expected = CreateTempDirAndFiles(10, 0); + const auto actual = ListDirAndFiles(tmp_root_.string()); + EXPECT_EQ(expected, actual); +} + +TEST_F(DirentTest, TestMultipleFiles) { + const auto expected = CreateTempDirAndFiles(0, 10); + const auto actual = ListDirAndFiles(tmp_root_.string()); + EXPECT_EQ(expected, actual); +} + +TEST_F(DirentTest, TestOneFileAndFolder) { + const auto expected = CreateTempDirAndFiles(1, 1); + const auto actual = ListDirAndFiles(tmp_root_.string()); + EXPECT_EQ(expected, actual); +} + +TEST_F(DirentTest, TestMultipleFilesAndFolders) { + const auto expected = CreateTempDirAndFiles(10, 10); + const auto actual = ListDirAndFiles(tmp_root_.string()); + EXPECT_EQ(expected, actual); +} diff --git a/hadoop-hdfs-project/hadoop-hdfs-native-client/src/main/native/libhdfspp/tests/x-platform/dirent_test.h b/hadoop-hdfs-project/hadoop-hdfs-native-client/src/main/native/libhdfspp/tests/x-platform/dirent_test.h new file mode 100644 index 0000000000000..4b79c30eea4e7 --- /dev/null +++ b/hadoop-hdfs-project/hadoop-hdfs-native-client/src/main/native/libhdfspp/tests/x-platform/dirent_test.h @@ -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 + * + * 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. + */ + +#ifndef LIBHDFSPP_CROSS_PLATFORM_DIRENT_TEST +#define LIBHDFSPP_CROSS_PLATFORM_DIRENT_TEST + +#include +#include +#include + +#include + +#include "x-platform/dirent.h" + +/** + * Test fixture for testing {@link XPlatform::Dirent}. + */ +class DirentTest : public ::testing::Test { +protected: + void SetUp() override; + void TearDown() override; + + /** + * Gets a name for creating temporary file or folder. This also ensures that + * the temporary file or folder does not exist. + * + * @param pattern The pattern to use for naming the temporary directory. + * @return The temporary file or folder name that can be used for creating the + * same. + */ + [[nodiscard]] std::string + GetTempName(const std::string &pattern = "test_XXXXXX") const; + + /** + * Creates the given number of temporary files and directories under the + * {@link DirentTest#tmp_root_}. + * + * @param num_dirs The number of temporary directories to create. + * @param num_files The number of temporary files to create. + * @return An {@link std::unordered_set> of the absolute paths of all the + * temporary files and folders that were created. + */ + [[nodiscard]] std::unordered_set + CreateTempDirAndFiles(std::size_t num_dirs, std::size_t num_files) const; + + /** + * Lists all the children of the given path. + * + * @param path The path whose children must be listed. + * @return An {@link std::unordered_set} containing the absolute paths of all + * the children of the given path. + */ + [[nodiscard]] virtual std::unordered_set + ListDirAndFiles(const std::string &path) const; + + /** + * The root in temp folder under which the rest of all the temporary files and + * folders will be created for the purpose of testing. + */ + std::filesystem::path tmp_root_; +}; + +#endif \ No newline at end of file diff --git a/hadoop-hdfs-project/hadoop-hdfs-native-client/src/main/native/libhdfspp/tools/hdfs-chmod/hdfs-chmod.cc b/hadoop-hdfs-project/hadoop-hdfs-native-client/src/main/native/libhdfspp/tools/hdfs-chmod/hdfs-chmod.cc index 4775860fb4483..cd5aefabfcf4b 100644 --- a/hadoop-hdfs-project/hadoop-hdfs-native-client/src/main/native/libhdfspp/tools/hdfs-chmod/hdfs-chmod.cc +++ b/hadoop-hdfs-project/hadoop-hdfs-native-client/src/main/native/libhdfspp/tools/hdfs-chmod/hdfs-chmod.cc @@ -140,8 +140,21 @@ bool Chmod::HandlePath(const std::string &permissions, const bool recursive, /* * strtol is reading the value with base 8, NULL because we are reading in * just one value. + * + * The strtol function may result in errors so check for that before + * typecasting. */ - auto perm = static_cast(strtol(permissions.c_str(), nullptr, 8)); + errno = 0; + long result = strtol(permissions.c_str(), nullptr, 8); + bool all_0_in_permission = std::all_of(permissions.begin(), permissions.end(), + [](char c) { return c == '0'; }); + /* + * The errno is set to ERANGE incase the string doesn't fit in long + * Also, the result is set to 0, in case conversion is not possible + */ + if ((errno == ERANGE) || (!all_0_in_permission && result == 0)) + return false; + auto perm = static_cast(result); if (!recursive) { fs->SetPermission(uri.get_path(), perm, handler); } else { diff --git a/hadoop-hdfs-project/hadoop-hdfs-nfs/src/main/java/org/apache/hadoop/hdfs/nfs/nfs3/Nfs3HttpServer.java b/hadoop-hdfs-project/hadoop-hdfs-nfs/src/main/java/org/apache/hadoop/hdfs/nfs/nfs3/Nfs3HttpServer.java index c37a21e7d83f5..ad410d4d388f3 100644 --- a/hadoop-hdfs-project/hadoop-hdfs-nfs/src/main/java/org/apache/hadoop/hdfs/nfs/nfs3/Nfs3HttpServer.java +++ b/hadoop-hdfs-project/hadoop-hdfs-nfs/src/main/java/org/apache/hadoop/hdfs/nfs/nfs3/Nfs3HttpServer.java @@ -25,7 +25,6 @@ import org.apache.hadoop.hdfs.DFSUtil; import org.apache.hadoop.hdfs.nfs.conf.NfsConfigKeys; import org.apache.hadoop.hdfs.nfs.conf.NfsConfiguration; -import org.apache.hadoop.hdfs.server.common.JspHelper; import org.apache.hadoop.http.HttpConfig; import org.apache.hadoop.http.HttpServer2; import org.apache.hadoop.net.NetUtils; diff --git a/hadoop-hdfs-project/hadoop-hdfs-nfs/src/test/java/org/apache/hadoop/hdfs/nfs/nfs3/TestNfs3Utils.java b/hadoop-hdfs-project/hadoop-hdfs-nfs/src/test/java/org/apache/hadoop/hdfs/nfs/nfs3/TestNfs3Utils.java index 77646af2c5ed9..8380c3c9bb3ef 100644 --- a/hadoop-hdfs-project/hadoop-hdfs-nfs/src/test/java/org/apache/hadoop/hdfs/nfs/nfs3/TestNfs3Utils.java +++ b/hadoop-hdfs-project/hadoop-hdfs-nfs/src/test/java/org/apache/hadoop/hdfs/nfs/nfs3/TestNfs3Utils.java @@ -18,8 +18,6 @@ package org.apache.hadoop.hdfs.nfs.nfs3; import static org.junit.Assert.assertEquals; -import static org.junit.Assert.assertTrue; -import static org.junit.Assert.fail; import org.junit.Test; diff --git a/hadoop-hdfs-project/hadoop-hdfs-rbf/src/main/java/org/apache/hadoop/hdfs/rbfbalance/RouterFedBalance.java b/hadoop-hdfs-project/hadoop-hdfs-rbf/src/main/java/org/apache/hadoop/hdfs/rbfbalance/RouterFedBalance.java index 4161ab503fe77..0cb4b54bfc431 100644 --- a/hadoop-hdfs-project/hadoop-hdfs-rbf/src/main/java/org/apache/hadoop/hdfs/rbfbalance/RouterFedBalance.java +++ b/hadoop-hdfs-project/hadoop-hdfs-rbf/src/main/java/org/apache/hadoop/hdfs/rbfbalance/RouterFedBalance.java @@ -63,7 +63,7 @@ * dst sub-namespace with distcp. * * 1. Move data from the source path to the destination path with distcp. - * 2. Update the the mount entry. + * 2. Update the mount entry. * 3. Delete the source path to trash. */ public class RouterFedBalance extends Configured implements Tool { @@ -77,7 +77,7 @@ public class RouterFedBalance extends Configured implements Tool { private static final String TRASH_PROCEDURE = "trash-procedure"; /** - * This class helps building the balance job. + * This class helps to build the balance job. */ private class Builder { /* Force close all open files while there is no diff. */ diff --git a/hadoop-hdfs-project/hadoop-hdfs-rbf/src/main/java/org/apache/hadoop/hdfs/server/federation/metrics/FederationMBean.java b/hadoop-hdfs-project/hadoop-hdfs-rbf/src/main/java/org/apache/hadoop/hdfs/server/federation/metrics/FederationMBean.java index e8b00d0b5dcfb..ed3069af83633 100644 --- a/hadoop-hdfs-project/hadoop-hdfs-rbf/src/main/java/org/apache/hadoop/hdfs/server/federation/metrics/FederationMBean.java +++ b/hadoop-hdfs-project/hadoop-hdfs-rbf/src/main/java/org/apache/hadoop/hdfs/server/federation/metrics/FederationMBean.java @@ -50,7 +50,7 @@ public interface FederationMBean { /** * Get the latest state of all routers. - * @return JSON with all of the known routers or null if failure. + * @return JSON with all the known routers or null if failure. */ String getRouters(); diff --git a/hadoop-hdfs-project/hadoop-hdfs-rbf/src/main/java/org/apache/hadoop/hdfs/server/federation/metrics/FederationRPCPerformanceMonitor.java b/hadoop-hdfs-project/hadoop-hdfs-rbf/src/main/java/org/apache/hadoop/hdfs/server/federation/metrics/FederationRPCPerformanceMonitor.java index 5c6dac465fb7c..159d08e26a161 100644 --- a/hadoop-hdfs-project/hadoop-hdfs-rbf/src/main/java/org/apache/hadoop/hdfs/server/federation/metrics/FederationRPCPerformanceMonitor.java +++ b/hadoop-hdfs-project/hadoop-hdfs-rbf/src/main/java/org/apache/hadoop/hdfs/server/federation/metrics/FederationRPCPerformanceMonitor.java @@ -52,7 +52,7 @@ public class FederationRPCPerformanceMonitor implements RouterRpcMonitor { /** Time for an operation to be received in the Router. */ private static final ThreadLocal START_TIME = new ThreadLocal<>(); - /** Time for an operation to be send to the Namenode. */ + /** Time for an operation to be sent to the Namenode. */ private static final ThreadLocal PROXY_TIME = new ThreadLocal<>(); /** Configuration for the performance monitor. */ diff --git a/hadoop-hdfs-project/hadoop-hdfs-rbf/src/main/java/org/apache/hadoop/hdfs/server/federation/metrics/RBFMetrics.java b/hadoop-hdfs-project/hadoop-hdfs-rbf/src/main/java/org/apache/hadoop/hdfs/server/federation/metrics/RBFMetrics.java index d5eabd1a3da82..be88069b49166 100644 --- a/hadoop-hdfs-project/hadoop-hdfs-rbf/src/main/java/org/apache/hadoop/hdfs/server/federation/metrics/RBFMetrics.java +++ b/hadoop-hdfs-project/hadoop-hdfs-rbf/src/main/java/org/apache/hadoop/hdfs/server/federation/metrics/RBFMetrics.java @@ -290,7 +290,7 @@ public String getMountTable() { // Dump mount table entries information into JSON for (MountTable entry : orderedMounts) { - // Sumarize destinations + // Summarize destinations Set nameservices = new LinkedHashSet<>(); Set paths = new LinkedHashSet<>(); for (RemoteLocation location : entry.getDestinations()) { diff --git a/hadoop-hdfs-project/hadoop-hdfs-rbf/src/main/java/org/apache/hadoop/hdfs/server/federation/resolver/MountTableResolver.java b/hadoop-hdfs-project/hadoop-hdfs-rbf/src/main/java/org/apache/hadoop/hdfs/server/federation/resolver/MountTableResolver.java index 9c0f33763f208..2888d8cc501a2 100644 --- a/hadoop-hdfs-project/hadoop-hdfs-rbf/src/main/java/org/apache/hadoop/hdfs/server/federation/resolver/MountTableResolver.java +++ b/hadoop-hdfs-project/hadoop-hdfs-rbf/src/main/java/org/apache/hadoop/hdfs/server/federation/resolver/MountTableResolver.java @@ -359,7 +359,7 @@ public static boolean isTrashPath(String path) throws IOException { public static String getTrashRoot() throws IOException { // Gets the Trash directory for the current user. return FileSystem.USER_HOME_PREFIX + "/" + - RouterRpcServer.getRemoteUser().getUserName() + "/" + + RouterRpcServer.getRemoteUser().getShortUserName() + "/" + FileSystem.TRASH_PREFIX; } diff --git a/hadoop-hdfs-project/hadoop-hdfs-rbf/src/main/java/org/apache/hadoop/hdfs/server/federation/router/ConnectionContext.java b/hadoop-hdfs-project/hadoop-hdfs-rbf/src/main/java/org/apache/hadoop/hdfs/server/federation/router/ConnectionContext.java index 9a5434b91ce2f..bd2d8c9d69719 100644 --- a/hadoop-hdfs-project/hadoop-hdfs-rbf/src/main/java/org/apache/hadoop/hdfs/server/federation/router/ConnectionContext.java +++ b/hadoop-hdfs-project/hadoop-hdfs-rbf/src/main/java/org/apache/hadoop/hdfs/server/federation/router/ConnectionContext.java @@ -124,7 +124,7 @@ public synchronized void release() { */ public synchronized void close(boolean force) { if (!force && this.numThreads > 0) { - // this is an erroneous case but we have to close the connection + // this is an erroneous case, but we have to close the connection // anyway since there will be connection leak if we don't do so // the connection has been moved out of the pool LOG.error("Active connection with {} handlers will be closed", @@ -132,7 +132,7 @@ public synchronized void close(boolean force) { } this.closed = true; Object proxy = this.client.getProxy(); - // Nobody should be using this anymore so it should close right away + // Nobody should be using this anymore, so it should close right away RPC.stopProxy(proxy); } diff --git a/hadoop-hdfs-project/hadoop-hdfs-rbf/src/main/java/org/apache/hadoop/hdfs/server/federation/router/ConnectionManager.java b/hadoop-hdfs-project/hadoop-hdfs-rbf/src/main/java/org/apache/hadoop/hdfs/server/federation/router/ConnectionManager.java index aad272f5831fc..5fe797bf5ce2c 100644 --- a/hadoop-hdfs-project/hadoop-hdfs-rbf/src/main/java/org/apache/hadoop/hdfs/server/federation/router/ConnectionManager.java +++ b/hadoop-hdfs-project/hadoop-hdfs-rbf/src/main/java/org/apache/hadoop/hdfs/server/federation/router/ConnectionManager.java @@ -135,12 +135,12 @@ public void start() { this.creator.start(); // Schedule a task to remove stale connection pools and sockets - long recyleTimeMs = Math.min( + long recycleTimeMs = Math.min( poolCleanupPeriodMs, connectionCleanupPeriodMs); LOG.info("Cleaning every {} seconds", - TimeUnit.MILLISECONDS.toSeconds(recyleTimeMs)); + TimeUnit.MILLISECONDS.toSeconds(recycleTimeMs)); this.cleaner.scheduleAtFixedRate( - new CleanupTask(), 0, recyleTimeMs, TimeUnit.MILLISECONDS); + new CleanupTask(), 0, recycleTimeMs, TimeUnit.MILLISECONDS); // Mark the manager as running this.running = true; @@ -364,9 +364,9 @@ void cleanup(ConnectionPool pool) { long timeSinceLastActive = Time.now() - pool.getLastActiveTime(); int total = pool.getNumConnections(); // Active is a transient status in many cases for a connection since - // the handler thread uses the connection very quickly. Thus the number + // the handler thread uses the connection very quickly. Thus, the number // of connections with handlers using at the call time is constantly low. - // Recently active is more lasting status and it shows how many + // Recently active is more lasting status, and it shows how many // connections have been used with a recent time period. (i.e. 30 seconds) int active = pool.getNumActiveConnectionsRecently(); float poolMinActiveRatio = pool.getMinActiveRatio(); @@ -376,9 +376,9 @@ void cleanup(ConnectionPool pool) { // The number should at least be 1 int targetConnectionsCount = Math.max(1, (int)(poolMinActiveRatio * total) - active); - List conns = + List connections = pool.removeConnections(targetConnectionsCount); - for (ConnectionContext conn : conns) { + for (ConnectionContext conn : connections) { conn.close(); } LOG.debug("Removed connection {} used {} seconds ago. " + diff --git a/hadoop-hdfs-project/hadoop-hdfs-rbf/src/main/java/org/apache/hadoop/hdfs/server/federation/router/FederationUtil.java b/hadoop-hdfs-project/hadoop-hdfs-rbf/src/main/java/org/apache/hadoop/hdfs/server/federation/router/FederationUtil.java index 7ff853946d700..e593e888c9ac4 100644 --- a/hadoop-hdfs-project/hadoop-hdfs-rbf/src/main/java/org/apache/hadoop/hdfs/server/federation/router/FederationUtil.java +++ b/hadoop-hdfs-project/hadoop-hdfs-rbf/src/main/java/org/apache/hadoop/hdfs/server/federation/router/FederationUtil.java @@ -231,7 +231,7 @@ public static ActiveNamenodeResolver newActiveNamenodeResolver( } /** - * Add the the number of children for an existing HdfsFileStatus object. + * Add the number of children for an existing HdfsFileStatus object. * @param dirStatus HdfsfileStatus object. * @param children number of children to be added. * @return HdfsFileStatus with the number of children specified. diff --git a/hadoop-hdfs-project/hadoop-hdfs-rbf/src/main/java/org/apache/hadoop/hdfs/server/federation/router/MountTableRefresherThread.java b/hadoop-hdfs-project/hadoop-hdfs-rbf/src/main/java/org/apache/hadoop/hdfs/server/federation/router/MountTableRefresherThread.java index a077c4b3f45a4..40ff843fa1dfe 100644 --- a/hadoop-hdfs-project/hadoop-hdfs-rbf/src/main/java/org/apache/hadoop/hdfs/server/federation/router/MountTableRefresherThread.java +++ b/hadoop-hdfs-project/hadoop-hdfs-rbf/src/main/java/org/apache/hadoop/hdfs/server/federation/router/MountTableRefresherThread.java @@ -50,14 +50,14 @@ public MountTableRefresherThread(MountTableManager manager, /** * Refresh mount table cache of local and remote routers. Local and remote - * routers will be refreshed differently. Lets understand what are the + * routers will be refreshed differently. Let's understand what are the * local and remote routers and refresh will be done differently on these * routers. Suppose there are three routers R1, R2 and R3. User want to add * new mount table entry. He will connect to only one router, not all the * routers. Suppose He connects to R1 and calls add mount table entry through * API or CLI. Now in this context R1 is local router, R2 and R3 are remote * routers. Because add mount table entry is invoked on R1, R1 will update the - * cache locally it need not to make RPC call. But R1 will make RPC calls to + * cache locally it need not make RPC call. But R1 will make RPC calls to * update cache on R2 and R3. */ @Override diff --git a/hadoop-hdfs-project/hadoop-hdfs-rbf/src/main/java/org/apache/hadoop/hdfs/server/federation/router/RBFConfigKeys.java b/hadoop-hdfs-project/hadoop-hdfs-rbf/src/main/java/org/apache/hadoop/hdfs/server/federation/router/RBFConfigKeys.java index 741e470c6fc3f..c0a9e3f294cd8 100644 --- a/hadoop-hdfs-project/hadoop-hdfs-rbf/src/main/java/org/apache/hadoop/hdfs/server/federation/router/RBFConfigKeys.java +++ b/hadoop-hdfs-project/hadoop-hdfs-rbf/src/main/java/org/apache/hadoop/hdfs/server/federation/router/RBFConfigKeys.java @@ -255,7 +255,7 @@ public class RBFConfigKeys extends CommonConfigurationKeysPublic { TimeUnit.MINUTES.toMillis(1); /** * Remote router mount table cache is updated through RouterClient(RPC call). - * To improve performance, RouterClient connections are cached but it should + * To improve performance, RouterClient connections are cached, but it should * not be kept in cache forever. This property defines the max time a * connection can be cached. */ diff --git a/hadoop-hdfs-project/hadoop-hdfs-rbf/src/main/java/org/apache/hadoop/hdfs/server/federation/router/RemoteMethod.java b/hadoop-hdfs-project/hadoop-hdfs-rbf/src/main/java/org/apache/hadoop/hdfs/server/federation/router/RemoteMethod.java index 6f1121ef9fd66..e5df4893a9155 100644 --- a/hadoop-hdfs-project/hadoop-hdfs-rbf/src/main/java/org/apache/hadoop/hdfs/server/federation/router/RemoteMethod.java +++ b/hadoop-hdfs-project/hadoop-hdfs-rbf/src/main/java/org/apache/hadoop/hdfs/server/federation/router/RemoteMethod.java @@ -36,7 +36,7 @@ public class RemoteMethod { private static final Logger LOG = LoggerFactory.getLogger(RemoteMethod.class); - /** List of parameters: static and dynamic values, matchings types. */ + /** List of parameters: static and dynamic values, matching types. */ private final Object[] params; /** List of method parameters types, matches parameters. */ private final Class[] types; diff --git a/hadoop-hdfs-project/hadoop-hdfs-rbf/src/main/java/org/apache/hadoop/hdfs/server/federation/router/RouterAdminServer.java b/hadoop-hdfs-project/hadoop-hdfs-rbf/src/main/java/org/apache/hadoop/hdfs/server/federation/router/RouterAdminServer.java index 127470a1264ed..db1922ad80811 100644 --- a/hadoop-hdfs-project/hadoop-hdfs-rbf/src/main/java/org/apache/hadoop/hdfs/server/federation/router/RouterAdminServer.java +++ b/hadoop-hdfs-project/hadoop-hdfs-rbf/src/main/java/org/apache/hadoop/hdfs/server/federation/router/RouterAdminServer.java @@ -104,7 +104,7 @@ import org.apache.hadoop.thirdparty.protobuf.BlockingService; /** - * This class is responsible for handling all of the Admin calls to the HDFS + * This class is responsible for handling all the Admin calls to the HDFS * router. It is created, started, and stopped by {@link Router}. */ public class RouterAdminServer extends AbstractService diff --git a/hadoop-hdfs-project/hadoop-hdfs-rbf/src/main/java/org/apache/hadoop/hdfs/server/federation/router/RouterClientProtocol.java b/hadoop-hdfs-project/hadoop-hdfs-rbf/src/main/java/org/apache/hadoop/hdfs/server/federation/router/RouterClientProtocol.java index 1bd7d65836dbe..c1dafec92203b 100644 --- a/hadoop-hdfs-project/hadoop-hdfs-rbf/src/main/java/org/apache/hadoop/hdfs/server/federation/router/RouterClientProtocol.java +++ b/hadoop-hdfs-project/hadoop-hdfs-rbf/src/main/java/org/apache/hadoop/hdfs/server/federation/router/RouterClientProtocol.java @@ -305,7 +305,7 @@ public HdfsFileStatus create(String src, FsPermission masked, * Check if an exception is caused by an unavailable subcluster or not. It * also checks the causes. * @param ioe IOException to check. - * @return If caused by an unavailable subcluster. False if the should not be + * @return If caused by an unavailable subcluster. False if they should not be * retried (e.g., NSQuotaExceededException). */ protected static boolean isUnavailableSubclusterException( @@ -1903,6 +1903,8 @@ private ContentSummary aggregateContentSummary( long quota = 0; long spaceConsumed = 0; long spaceQuota = 0; + long snapshotDirectoryCount = 0; + long snapshotFileCount = 0; String ecPolicy = ""; for (ContentSummary summary : summaries) { @@ -1912,6 +1914,8 @@ private ContentSummary aggregateContentSummary( quota = summary.getQuota(); spaceConsumed += summary.getSpaceConsumed(); spaceQuota = summary.getSpaceQuota(); + snapshotDirectoryCount += summary.getSnapshotDirectoryCount(); + snapshotFileCount += summary.getSnapshotFileCount(); // We return from the first response as we assume that the EC policy // of each sub-cluster is same. if (ecPolicy.isEmpty()) { @@ -1927,6 +1931,8 @@ private ContentSummary aggregateContentSummary( .spaceConsumed(spaceConsumed) .spaceQuota(spaceQuota) .erasureCodingPolicy(ecPolicy) + .snapshotDirectoryCount(snapshotDirectoryCount) + .snapshotFileCount(snapshotFileCount) .build(); return ret; } diff --git a/hadoop-hdfs-project/hadoop-hdfs-rbf/src/main/java/org/apache/hadoop/hdfs/server/federation/router/RouterQuotaUpdateService.java b/hadoop-hdfs-project/hadoop-hdfs-rbf/src/main/java/org/apache/hadoop/hdfs/server/federation/router/RouterQuotaUpdateService.java index a4a7d9e9ddc2d..e9b780d5bca29 100644 --- a/hadoop-hdfs-project/hadoop-hdfs-rbf/src/main/java/org/apache/hadoop/hdfs/server/federation/router/RouterQuotaUpdateService.java +++ b/hadoop-hdfs-project/hadoop-hdfs-rbf/src/main/java/org/apache/hadoop/hdfs/server/federation/router/RouterQuotaUpdateService.java @@ -200,7 +200,7 @@ private List getMountTableEntries() throws IOException { * During this time, the quota usage cache will also be updated by * quota manager: * 1. Stale paths (entries) will be removed. - * 2. Existing entries will be override and updated. + * 2. Existing entries will be overridden and updated. * @return List of mount tables which quota was set. * @throws IOException */ diff --git a/hadoop-hdfs-project/hadoop-hdfs-rbf/src/main/java/org/apache/hadoop/hdfs/server/federation/router/RouterRpcClient.java b/hadoop-hdfs-project/hadoop-hdfs-rbf/src/main/java/org/apache/hadoop/hdfs/server/federation/router/RouterRpcClient.java index 34a2c47c3ef29..ff90854ebb7ec 100644 --- a/hadoop-hdfs-project/hadoop-hdfs-rbf/src/main/java/org/apache/hadoop/hdfs/server/federation/router/RouterRpcClient.java +++ b/hadoop-hdfs-project/hadoop-hdfs-rbf/src/main/java/org/apache/hadoop/hdfs/server/federation/router/RouterRpcClient.java @@ -139,7 +139,7 @@ public class RouterRpcClient { /** * Create a router RPC client to manage remote procedure calls to NNs. * - * @param conf Hdfs Configuation. + * @param conf Hdfs Configuration. * @param router A router using this RPC client. * @param resolver A NN resolver to determine the currently active NN in HA. * @param monitor Optional performance monitor. @@ -444,7 +444,7 @@ private RetryDecision shouldRetry(final IOException ioe, final int retryCount, * @param ugi User group information. * @param namenodes A prioritized list of namenodes within the same * nameservice. - * @param method Remote ClientProtcol method to invoke. + * @param method Remote ClientProtocol method to invoke. * @param params Variable list of parameters matching the method. * @return The result of invoking the method. * @throws ConnectException If it cannot connect to any Namenode. @@ -1027,7 +1027,7 @@ public RemoteResult invokeSequential( } catch (Exception e) { // Unusual error, ClientProtocol calls always use IOException (or // RemoteException). Re-wrap in IOException for compatibility with - // ClientProtcol. + // ClientProtocol. LOG.error("Unexpected exception {} proxying {} to {}", e.getClass(), m.getName(), ns, e); IOException ioe = new IOException( @@ -1449,7 +1449,7 @@ public Map invokeConcurrent( results.add(new RemoteResult<>(location, ioe)); } catch (ExecutionException ex) { Throwable cause = ex.getCause(); - LOG.debug("Canot execute {} in {}: {}", + LOG.debug("Cannot execute {} in {}: {}", m.getName(), location, cause.getMessage()); // Convert into IOException if needed diff --git a/hadoop-hdfs-project/hadoop-hdfs-rbf/src/main/java/org/apache/hadoop/hdfs/server/federation/router/RouterRpcMonitor.java b/hadoop-hdfs-project/hadoop-hdfs-rbf/src/main/java/org/apache/hadoop/hdfs/server/federation/router/RouterRpcMonitor.java index 388fc5a0da496..039b40ae2e585 100644 --- a/hadoop-hdfs-project/hadoop-hdfs-rbf/src/main/java/org/apache/hadoop/hdfs/server/federation/router/RouterRpcMonitor.java +++ b/hadoop-hdfs-project/hadoop-hdfs-rbf/src/main/java/org/apache/hadoop/hdfs/server/federation/router/RouterRpcMonitor.java @@ -54,7 +54,7 @@ void init( /** * Start proxying an operation to the Namenode. - * @return Id of the thread doing the proxying. + * @return id of the thread doing the proxying. */ long proxyOp(); diff --git a/hadoop-hdfs-project/hadoop-hdfs-rbf/src/main/java/org/apache/hadoop/hdfs/server/federation/router/RouterRpcServer.java b/hadoop-hdfs-project/hadoop-hdfs-rbf/src/main/java/org/apache/hadoop/hdfs/server/federation/router/RouterRpcServer.java index 2bec2c726a116..58181dcc346cf 100644 --- a/hadoop-hdfs-project/hadoop-hdfs-rbf/src/main/java/org/apache/hadoop/hdfs/server/federation/router/RouterRpcServer.java +++ b/hadoop-hdfs-project/hadoop-hdfs-rbf/src/main/java/org/apache/hadoop/hdfs/server/federation/router/RouterRpcServer.java @@ -49,7 +49,6 @@ import java.util.Map; import java.util.Map.Entry; import java.util.Set; -import java.util.concurrent.Callable; import java.util.concurrent.ExecutionException; import java.util.concurrent.Executors; import java.util.concurrent.ThreadFactory; @@ -2061,12 +2060,7 @@ public DatanodeInfo[] load(DatanodeReportType type) throws Exception { public ListenableFuture reload( final DatanodeReportType type, DatanodeInfo[] oldValue) throws Exception { - return executorService.submit(new Callable() { - @Override - public DatanodeInfo[] call() throws Exception { - return load(type); - } - }); + return executorService.submit(() -> load(type)); } } } diff --git a/hadoop-hdfs-project/hadoop-hdfs-rbf/src/main/java/org/apache/hadoop/hdfs/server/federation/router/RouterWebHdfsMethods.java b/hadoop-hdfs-project/hadoop-hdfs-rbf/src/main/java/org/apache/hadoop/hdfs/server/federation/router/RouterWebHdfsMethods.java index accec4627eda8..a66953b1bd752 100644 --- a/hadoop-hdfs-project/hadoop-hdfs-rbf/src/main/java/org/apache/hadoop/hdfs/server/federation/router/RouterWebHdfsMethods.java +++ b/hadoop-hdfs-project/hadoop-hdfs-rbf/src/main/java/org/apache/hadoop/hdfs/server/federation/router/RouterWebHdfsMethods.java @@ -410,7 +410,7 @@ protected Response get( * @param path Path to check. * @param op Operation to perform. * @param openOffset Offset for opening a file. - * @param excludeDatanodes Blocks to excluded. + * @param excludeDatanodes Blocks to exclude. * @param parameters Other parameters. * @return Redirection URI. * @throws URISyntaxException If it cannot parse the URI. diff --git a/hadoop-hdfs-project/hadoop-hdfs-rbf/src/main/java/org/apache/hadoop/hdfs/server/federation/store/CachedRecordStore.java b/hadoop-hdfs-project/hadoop-hdfs-rbf/src/main/java/org/apache/hadoop/hdfs/server/federation/store/CachedRecordStore.java index 7b28c03a529ad..2b693aa936f00 100644 --- a/hadoop-hdfs-project/hadoop-hdfs-rbf/src/main/java/org/apache/hadoop/hdfs/server/federation/store/CachedRecordStore.java +++ b/hadoop-hdfs-project/hadoop-hdfs-rbf/src/main/java/org/apache/hadoop/hdfs/server/federation/store/CachedRecordStore.java @@ -85,7 +85,7 @@ protected CachedRecordStore(Class clazz, StateStoreDriver driver) { * * @param clazz Class of the record to store. * @param driver State Store driver. - * @param over If the entries should be override if they expire + * @param over If the entries should be overridden if they expire */ protected CachedRecordStore( Class clazz, StateStoreDriver driver, boolean over) { @@ -153,7 +153,7 @@ public boolean loadCache(boolean force) throws IOException { } /** - * Check if it's time to update the cache. Update it it was never updated. + * Check if it's time to update the cache. Update it was never updated. * * @return If it's time to update this cache. */ @@ -206,7 +206,7 @@ public void overrideExpiredRecords(QueryResult query) throws IOException { * Updates the state store with any record overrides we detected, such as an * expired state. * - * @param record Record record to be updated. + * @param record record to be updated. * @throws IOException If the values cannot be updated. */ public void overrideExpiredRecord(R record) throws IOException { diff --git a/hadoop-hdfs-project/hadoop-hdfs-rbf/src/main/java/org/apache/hadoop/hdfs/server/federation/store/StateStoreService.java b/hadoop-hdfs-project/hadoop-hdfs-rbf/src/main/java/org/apache/hadoop/hdfs/server/federation/store/StateStoreService.java index 507c83786a8f4..201c7a325f105 100644 --- a/hadoop-hdfs-project/hadoop-hdfs-rbf/src/main/java/org/apache/hadoop/hdfs/server/federation/store/StateStoreService.java +++ b/hadoop-hdfs-project/hadoop-hdfs-rbf/src/main/java/org/apache/hadoop/hdfs/server/federation/store/StateStoreService.java @@ -331,7 +331,7 @@ public StateStoreDriver getDriver() { } /** - * Fetch a unique identifier for this state store instance. Typically it is + * Fetch a unique identifier for this state store instance. Typically, it is * the address of the router. * * @return Unique identifier for this store. diff --git a/hadoop-hdfs-project/hadoop-hdfs-rbf/src/main/java/org/apache/hadoop/hdfs/server/federation/store/driver/StateStoreDriver.java b/hadoop-hdfs-project/hadoop-hdfs-rbf/src/main/java/org/apache/hadoop/hdfs/server/federation/store/driver/StateStoreDriver.java index d595a97d01717..a4e9c1ce82bbb 100644 --- a/hadoop-hdfs-project/hadoop-hdfs-rbf/src/main/java/org/apache/hadoop/hdfs/server/federation/store/driver/StateStoreDriver.java +++ b/hadoop-hdfs-project/hadoop-hdfs-rbf/src/main/java/org/apache/hadoop/hdfs/server/federation/store/driver/StateStoreDriver.java @@ -97,7 +97,7 @@ protected Configuration getConf() { } /** - * Gets a unique identifier for the running task/process. Typically the + * Gets a unique identifier for the running task/process. Typically, the * router address. * * @return Unique identifier for the running task. diff --git a/hadoop-hdfs-project/hadoop-hdfs-rbf/src/main/java/org/apache/hadoop/hdfs/server/federation/store/protocol/GetRouterRegistrationsRequest.java b/hadoop-hdfs-project/hadoop-hdfs-rbf/src/main/java/org/apache/hadoop/hdfs/server/federation/store/protocol/GetRouterRegistrationsRequest.java index b70cccf593ae9..10b172eb50a21 100644 --- a/hadoop-hdfs-project/hadoop-hdfs-rbf/src/main/java/org/apache/hadoop/hdfs/server/federation/store/protocol/GetRouterRegistrationsRequest.java +++ b/hadoop-hdfs-project/hadoop-hdfs-rbf/src/main/java/org/apache/hadoop/hdfs/server/federation/store/protocol/GetRouterRegistrationsRequest.java @@ -22,7 +22,7 @@ import org.apache.hadoop.hdfs.server.federation.store.driver.StateStoreSerializer; /** - * API request for retrieving a all non-expired router registrations present in + * API request for retrieving an all non-expired router registrations present in * the state store. */ public abstract class GetRouterRegistrationsRequest { diff --git a/hadoop-hdfs-project/hadoop-hdfs-rbf/src/main/java/org/apache/hadoop/hdfs/server/federation/store/protocol/impl/pb/FederationProtocolPBTranslator.java b/hadoop-hdfs-project/hadoop-hdfs-rbf/src/main/java/org/apache/hadoop/hdfs/server/federation/store/protocol/impl/pb/FederationProtocolPBTranslator.java index 8422a8c4b6d15..98988f8bb2d86 100644 --- a/hadoop-hdfs-project/hadoop-hdfs-rbf/src/main/java/org/apache/hadoop/hdfs/server/federation/store/protocol/impl/pb/FederationProtocolPBTranslator.java +++ b/hadoop-hdfs-project/hadoop-hdfs-rbf/src/main/java/org/apache/hadoop/hdfs/server/federation/store/protocol/impl/pb/FederationProtocolPBTranslator.java @@ -92,7 +92,7 @@ public B getBuilder() { /** * Get the serialized proto object. If the translator was created from a byte - * stream, returns the intitial byte stream. Otherwise creates a new byte + * stream, returns the initial byte stream. Otherwise, creates a new byte * stream from the cached builder. * * @return Protobuf message object diff --git a/hadoop-hdfs-project/hadoop-hdfs-rbf/src/main/java/org/apache/hadoop/hdfs/tools/federation/RouterAdmin.java b/hadoop-hdfs-project/hadoop-hdfs-rbf/src/main/java/org/apache/hadoop/hdfs/tools/federation/RouterAdmin.java index b8e7c796a147d..d7fcf862fb6e9 100644 --- a/hadoop-hdfs-project/hadoop-hdfs-rbf/src/main/java/org/apache/hadoop/hdfs/tools/federation/RouterAdmin.java +++ b/hadoop-hdfs-project/hadoop-hdfs-rbf/src/main/java/org/apache/hadoop/hdfs/tools/federation/RouterAdmin.java @@ -193,7 +193,7 @@ private String getUsage(String cmd) { /** * Usage: validates the maximum number of arguments for a command. - * @param arg List of of command line parameters. + * @param arg List of command line parameters. */ private void validateMax(String[] arg) { if (arg[0].equals("-ls")) { @@ -407,7 +407,7 @@ public int run(String[] argv) throws Exception { System.err.println(cmd.substring(1) + ": " + arge.getLocalizedMessage()); printUsage(cmd); } catch (RemoteException e) { - // This is a error returned by the server. + // This is an error returned by the server. // Print out the first line of the error message, ignore the stack trace. exitCode = -1; debugException = e; @@ -807,7 +807,7 @@ public void listMounts(String[] argv, int i) throws IOException { } else if (argv[i].equals("-d")) { // Check if -d parameter is specified. detail = true; if (argv.length == 2) { - path = "/"; // If no path is provide with -ls -d. + path = "/"; // If no path is provided with -ls -d. } else { path = argv[++i]; } diff --git a/hadoop-hdfs-project/hadoop-hdfs-rbf/src/test/java/org/apache/hadoop/hdfs/server/federation/MiniRouterDFSCluster.java b/hadoop-hdfs-project/hadoop-hdfs-rbf/src/test/java/org/apache/hadoop/hdfs/server/federation/MiniRouterDFSCluster.java index ac6ecd4398cba..87b99e5d9523c 100644 --- a/hadoop-hdfs-project/hadoop-hdfs-rbf/src/test/java/org/apache/hadoop/hdfs/server/federation/MiniRouterDFSCluster.java +++ b/hadoop-hdfs-project/hadoop-hdfs-rbf/src/test/java/org/apache/hadoop/hdfs/server/federation/MiniRouterDFSCluster.java @@ -789,7 +789,7 @@ public void startCluster(Configuration overrideConf) { Configuration nnConf = generateNamenodeConfiguration(ns0); if (overrideConf != null) { nnConf.addResource(overrideConf); - // Router also uses this configurations as initial values. + // Router also uses these configurations as initial values. routerConf = new Configuration(overrideConf); } diff --git a/hadoop-hdfs-project/hadoop-hdfs-rbf/src/test/java/org/apache/hadoop/hdfs/server/federation/MockResolver.java b/hadoop-hdfs-project/hadoop-hdfs-rbf/src/test/java/org/apache/hadoop/hdfs/server/federation/MockResolver.java index 43efd85228d72..1519bad74b5c1 100644 --- a/hadoop-hdfs-project/hadoop-hdfs-rbf/src/test/java/org/apache/hadoop/hdfs/server/federation/MockResolver.java +++ b/hadoop-hdfs-project/hadoop-hdfs-rbf/src/test/java/org/apache/hadoop/hdfs/server/federation/MockResolver.java @@ -137,7 +137,7 @@ public void updateActiveNamenode( break; } } - // This operation modifies the list so we need to be careful + // This operation modifies the list, so we need to be careful synchronized(namenodes) { Collections.sort(namenodes, new NamenodePriorityComparator()); } diff --git a/hadoop-hdfs-project/hadoop-hdfs-rbf/src/test/java/org/apache/hadoop/hdfs/server/federation/resolver/TestNamenodeResolver.java b/hadoop-hdfs-project/hadoop-hdfs-rbf/src/test/java/org/apache/hadoop/hdfs/server/federation/resolver/TestNamenodeResolver.java index df80037c69917..ed10a3a87317d 100644 --- a/hadoop-hdfs-project/hadoop-hdfs-rbf/src/test/java/org/apache/hadoop/hdfs/server/federation/resolver/TestNamenodeResolver.java +++ b/hadoop-hdfs-project/hadoop-hdfs-rbf/src/test/java/org/apache/hadoop/hdfs/server/federation/resolver/TestNamenodeResolver.java @@ -297,7 +297,7 @@ public void testCacheUpdateOnNamenodeStateUpdate() throws IOException { String rpcAddr = namenode.getRpcAddress(); InetSocketAddress inetAddr = getInetSocketAddress(rpcAddr); - // If the namenode state changes and it serves request, + // If the namenode state changes, and it serves request, // RouterRpcClient calls updateActiveNamenode to update the state to active, // Check whether correct updated state is returned post update. namenodeResolver.updateActiveNamenode(NAMESERVICES[0], inetAddr); diff --git a/hadoop-hdfs-project/hadoop-hdfs-rbf/src/test/java/org/apache/hadoop/hdfs/server/federation/router/TestRouter.java b/hadoop-hdfs-project/hadoop-hdfs-rbf/src/test/java/org/apache/hadoop/hdfs/server/federation/router/TestRouter.java index 0464877d3cd80..9b8fb67e68122 100644 --- a/hadoop-hdfs-project/hadoop-hdfs-rbf/src/test/java/org/apache/hadoop/hdfs/server/federation/router/TestRouter.java +++ b/hadoop-hdfs-project/hadoop-hdfs-rbf/src/test/java/org/apache/hadoop/hdfs/server/federation/router/TestRouter.java @@ -45,7 +45,7 @@ import org.junit.Test; /** - * The the safe mode for the {@link Router} controlled by + * The safe mode for the {@link Router} controlled by * {@link SafeModeTimer}. */ public class TestRouter { diff --git a/hadoop-hdfs-project/hadoop-hdfs-rbf/src/test/java/org/apache/hadoop/hdfs/server/federation/router/TestRouterAdminCLI.java b/hadoop-hdfs-project/hadoop-hdfs-rbf/src/test/java/org/apache/hadoop/hdfs/server/federation/router/TestRouterAdminCLI.java index b9dff65b28a09..677f3b5e947e3 100644 --- a/hadoop-hdfs-project/hadoop-hdfs-rbf/src/test/java/org/apache/hadoop/hdfs/server/federation/router/TestRouterAdminCLI.java +++ b/hadoop-hdfs-project/hadoop-hdfs-rbf/src/test/java/org/apache/hadoop/hdfs/server/federation/router/TestRouterAdminCLI.java @@ -742,7 +742,7 @@ public void testMountTablePermissions() throws Exception { * @param mount * target mount table * @param canRead - * whether can list mount tables under specified mount + * whether you can list mount tables under specified mount * @param addCommandCode * expected return code of add command executed for specified mount * @param rmCommandCode @@ -1467,7 +1467,7 @@ public void testUpdateErrorCase() throws Exception { err.toString().contains("update: /noMount doesn't exist.")); err.reset(); - // Check update if non true/false value is passed for readonly. + // Check update if no true/false value is passed for readonly. argv = new String[] {"-update", src, "-readonly", "check"}; assertEquals(-1, ToolRunner.run(admin, argv)); assertTrue(err.toString(), err.toString().contains("update: " diff --git a/hadoop-hdfs-project/hadoop-hdfs-rbf/src/test/java/org/apache/hadoop/hdfs/server/federation/router/TestRouterFaultTolerant.java b/hadoop-hdfs-project/hadoop-hdfs-rbf/src/test/java/org/apache/hadoop/hdfs/server/federation/router/TestRouterFaultTolerant.java index bf571e2ff790c..ef5322ba218ed 100644 --- a/hadoop-hdfs-project/hadoop-hdfs-rbf/src/test/java/org/apache/hadoop/hdfs/server/federation/router/TestRouterFaultTolerant.java +++ b/hadoop-hdfs-project/hadoop-hdfs-rbf/src/test/java/org/apache/hadoop/hdfs/server/federation/router/TestRouterFaultTolerant.java @@ -582,7 +582,7 @@ public String toString() { } /** - * Asserts that the results are the expected amount and it has both success + * Asserts that the results are the expected amount, and it has both success * and failure. * @param msg Message to show when the assertion fails. * @param expected Expected number of results. diff --git a/hadoop-hdfs-project/hadoop-hdfs-rbf/src/test/java/org/apache/hadoop/hdfs/server/federation/router/TestRouterMountTableCacheRefresh.java b/hadoop-hdfs-project/hadoop-hdfs-rbf/src/test/java/org/apache/hadoop/hdfs/server/federation/router/TestRouterMountTableCacheRefresh.java index c90e614a5cd40..82bc7d905e606 100644 --- a/hadoop-hdfs-project/hadoop-hdfs-rbf/src/test/java/org/apache/hadoop/hdfs/server/federation/router/TestRouterMountTableCacheRefresh.java +++ b/hadoop-hdfs-project/hadoop-hdfs-rbf/src/test/java/org/apache/hadoop/hdfs/server/federation/router/TestRouterMountTableCacheRefresh.java @@ -308,9 +308,9 @@ public void run() { TimeUnit.SECONDS); mountTableRefresherService.init(config); // One router is not responding for 1 minute, still refresh should - // finished in 5 second as cache update timeout is set 5 second. + // finish in 5 second as cache update timeout is set 5 second. mountTableRefresherService.refresh(); - // Test case timeout is assert for this test case. + // Test case timeout is asserted for this test case. } /** @@ -349,7 +349,7 @@ protected RouterClient createRouterClient( mountTableRefresherService.refresh(); assertNotEquals("No RouterClient is created.", 0, createCounter.get()); /* - * Wait for clients to expire. Lets wait triple the cache eviction period. + * Wait for clients to expire. Let's wait triple the cache eviction period. * After cache eviction period all created client must be removed and * closed. */ diff --git a/hadoop-hdfs-project/hadoop-hdfs-rbf/src/test/java/org/apache/hadoop/hdfs/server/federation/router/TestRouterQuotaManager.java b/hadoop-hdfs-project/hadoop-hdfs-rbf/src/test/java/org/apache/hadoop/hdfs/server/federation/router/TestRouterQuotaManager.java index 4a1dd2e091bc7..d24ef8ebc3c4b 100644 --- a/hadoop-hdfs-project/hadoop-hdfs-rbf/src/test/java/org/apache/hadoop/hdfs/server/federation/router/TestRouterQuotaManager.java +++ b/hadoop-hdfs-project/hadoop-hdfs-rbf/src/test/java/org/apache/hadoop/hdfs/server/federation/router/TestRouterQuotaManager.java @@ -75,11 +75,11 @@ public void testGetChildrenPaths() { public void testGetQuotaUsage() { RouterQuotaUsage quotaGet; - // test case1: get quota with an non-exist path + // test case1: get quota with a non-exist path quotaGet = manager.getQuotaUsage("/non-exist-path"); assertNull(quotaGet); - // test case2: get quota from an no-quota set path + // test case2: get quota from a no-quota set path RouterQuotaUsage.Builder quota = new RouterQuotaUsage.Builder() .quota(HdfsConstants.QUOTA_RESET) .spaceQuota(HdfsConstants.QUOTA_RESET); @@ -88,7 +88,7 @@ public void testGetQuotaUsage() { // it should return null assertNull(quotaGet); - // test case3: get quota from an quota-set path + // test case3: get quota from a quota-set path quota.quota(1); quota.spaceQuota(HdfsConstants.QUOTA_RESET); manager.put("/hasQuotaSet", quota.build()); @@ -96,24 +96,24 @@ public void testGetQuotaUsage() { assertEquals(1, quotaGet.getQuota()); assertEquals(HdfsConstants.QUOTA_RESET, quotaGet.getSpaceQuota()); - // test case4: get quota with an non-exist child path + // test case4: get quota with a non-exist child path quotaGet = manager.getQuotaUsage("/hasQuotaSet/file"); // it will return the nearest ancestor which quota was set assertEquals(1, quotaGet.getQuota()); assertEquals(HdfsConstants.QUOTA_RESET, quotaGet.getSpaceQuota()); - // test case5: get quota with an child path which its parent + // test case5: get quota with a child path which its parent // wasn't quota set quota.quota(HdfsConstants.QUOTA_RESET); quota.spaceQuota(HdfsConstants.QUOTA_RESET); manager.put("/hasQuotaSet/noQuotaSet", quota.build()); - // here should returns the quota of path /hasQuotaSet + // here should return the quota of path /hasQuotaSet // (the nearest ancestor which quota was set) quotaGet = manager.getQuotaUsage("/hasQuotaSet/noQuotaSet/file"); assertEquals(1, quotaGet.getQuota()); assertEquals(HdfsConstants.QUOTA_RESET, quotaGet.getSpaceQuota()); - // test case6: get quota with an child path which its parent was quota set + // test case6: get quota with a child path which its parent was quota set quota.quota(2); quota.spaceQuota(HdfsConstants.QUOTA_RESET); manager.put("/hasQuotaSet/hasQuotaSet", quota.build()); diff --git a/hadoop-hdfs-project/hadoop-hdfs-rbf/src/test/java/org/apache/hadoop/hdfs/server/federation/router/TestRouterRPCClientRetries.java b/hadoop-hdfs-project/hadoop-hdfs-rbf/src/test/java/org/apache/hadoop/hdfs/server/federation/router/TestRouterRPCClientRetries.java index a16f9d10c85fc..73803d9805203 100644 --- a/hadoop-hdfs-project/hadoop-hdfs-rbf/src/test/java/org/apache/hadoop/hdfs/server/federation/router/TestRouterRPCClientRetries.java +++ b/hadoop-hdfs-project/hadoop-hdfs-rbf/src/test/java/org/apache/hadoop/hdfs/server/federation/router/TestRouterRPCClientRetries.java @@ -68,7 +68,7 @@ public class TestRouterRPCClientRetries { private static ClientProtocol routerProtocol; @Rule - public final Timeout testTimeout = new Timeout(100000); + public final Timeout testTimeout = new Timeout(100000L, TimeUnit.MILLISECONDS); @Before public void setUp() throws Exception { diff --git a/hadoop-hdfs-project/hadoop-hdfs-rbf/src/test/java/org/apache/hadoop/hdfs/server/federation/router/TestRouterRPCMultipleDestinationMountTableResolver.java b/hadoop-hdfs-project/hadoop-hdfs-rbf/src/test/java/org/apache/hadoop/hdfs/server/federation/router/TestRouterRPCMultipleDestinationMountTableResolver.java index 238d1b0301180..b05337443f659 100644 --- a/hadoop-hdfs-project/hadoop-hdfs-rbf/src/test/java/org/apache/hadoop/hdfs/server/federation/router/TestRouterRPCMultipleDestinationMountTableResolver.java +++ b/hadoop-hdfs-project/hadoop-hdfs-rbf/src/test/java/org/apache/hadoop/hdfs/server/federation/router/TestRouterRPCMultipleDestinationMountTableResolver.java @@ -511,7 +511,7 @@ public void testIsMultiDestDir() throws Exception { */ @Test public void testSnapshotPathResolution() throws Exception { - // Create a mount entry with non isPathAll order, so as to call + // Create a mount entry with non isPathAll order, to call // invokeSequential. Map destMap = new HashMap<>(); destMap.put("ns0", "/tmp_ns0"); diff --git a/hadoop-hdfs-project/hadoop-hdfs-rbf/src/test/java/org/apache/hadoop/hdfs/server/federation/router/TestRouterRpc.java b/hadoop-hdfs-project/hadoop-hdfs-rbf/src/test/java/org/apache/hadoop/hdfs/server/federation/router/TestRouterRpc.java index 96e77d580073b..4aeb2ec9b8f30 100644 --- a/hadoop-hdfs-project/hadoop-hdfs-rbf/src/test/java/org/apache/hadoop/hdfs/server/federation/router/TestRouterRpc.java +++ b/hadoop-hdfs-project/hadoop-hdfs-rbf/src/test/java/org/apache/hadoop/hdfs/server/federation/router/TestRouterRpc.java @@ -56,6 +56,7 @@ import org.apache.hadoop.conf.Configuration; import org.apache.hadoop.crypto.CryptoProtocolVersion; +import org.apache.hadoop.fs.ContentSummary; import org.apache.hadoop.fs.CreateFlag; import org.apache.hadoop.fs.FileContext; import org.apache.hadoop.fs.FileStatus; @@ -2004,4 +2005,46 @@ public void testAddClientIpPortToCallerContext() throws IOException { assertFalse(auditLog.getOutput().contains("clientIp:1.1.1.1")); assertFalse(auditLog.getOutput().contains("clientPort:1234")); } + + @Test + public void testContentSummaryWithSnapshot() throws Exception { + DistributedFileSystem routerDFS = (DistributedFileSystem) routerFS; + Path dirPath = new Path("/testdir"); + Path subdirPath = new Path(dirPath, "subdir"); + Path filePath1 = new Path(dirPath, "file"); + Path filePath2 = new Path(subdirPath, "file2"); + + // Create directories. + routerDFS.mkdirs(dirPath); + routerDFS.mkdirs(subdirPath); + + // Create files. + createFile(routerDFS, filePath1.toString(), 32); + createFile(routerDFS, filePath2.toString(), 16); + + // Allow & Create snapshot. + routerDFS.allowSnapshot(dirPath); + routerDFS.createSnapshot(dirPath, "s1"); + + try { + // Check content summary, snapshot count should be 0 + ContentSummary contentSummary = routerDFS.getContentSummary(dirPath); + assertEquals(0, contentSummary.getSnapshotDirectoryCount()); + assertEquals(0, contentSummary.getSnapshotFileCount()); + + // Delete the file & subdir(Total 2 files deleted & 1 directory) + routerDFS.delete(filePath1, true); + routerDFS.delete(subdirPath, true); + + // Get the Content Summary + contentSummary = routerDFS.getContentSummary(dirPath); + assertEquals(1, contentSummary.getSnapshotDirectoryCount()); + assertEquals(2, contentSummary.getSnapshotFileCount()); + } finally { + // Cleanup + routerDFS.deleteSnapshot(dirPath, "s1"); + routerDFS.disallowSnapshot(dirPath); + routerDFS.delete(dirPath, true); + } + } } diff --git a/hadoop-hdfs-project/hadoop-hdfs-rbf/src/test/java/org/apache/hadoop/hdfs/server/federation/router/TestRouterRpcMultiDestination.java b/hadoop-hdfs-project/hadoop-hdfs-rbf/src/test/java/org/apache/hadoop/hdfs/server/federation/router/TestRouterRpcMultiDestination.java index 370a1250a7c11..6ade57326e316 100644 --- a/hadoop-hdfs-project/hadoop-hdfs-rbf/src/test/java/org/apache/hadoop/hdfs/server/federation/router/TestRouterRpcMultiDestination.java +++ b/hadoop-hdfs-project/hadoop-hdfs-rbf/src/test/java/org/apache/hadoop/hdfs/server/federation/router/TestRouterRpcMultiDestination.java @@ -75,7 +75,7 @@ import org.junit.Test; /** - * The the RPC interface of the {@link getRouter()} implemented by + * The RPC interface of the {@link getRouter()} implemented by * {@link RouterRpcServer}. */ public class TestRouterRpcMultiDestination extends TestRouterRpc { diff --git a/hadoop-hdfs-project/hadoop-hdfs-rbf/src/test/java/org/apache/hadoop/hdfs/server/federation/router/TestRouterRpcStoragePolicySatisfier.java b/hadoop-hdfs-project/hadoop-hdfs-rbf/src/test/java/org/apache/hadoop/hdfs/server/federation/router/TestRouterRpcStoragePolicySatisfier.java index fa1079a4edeb4..57518c811197f 100644 --- a/hadoop-hdfs-project/hadoop-hdfs-rbf/src/test/java/org/apache/hadoop/hdfs/server/federation/router/TestRouterRpcStoragePolicySatisfier.java +++ b/hadoop-hdfs-project/hadoop-hdfs-rbf/src/test/java/org/apache/hadoop/hdfs/server/federation/router/TestRouterRpcStoragePolicySatisfier.java @@ -31,7 +31,6 @@ import org.apache.hadoop.hdfs.server.common.HdfsServerConstants; import org.apache.hadoop.hdfs.server.federation.MiniRouterDFSCluster; import org.apache.hadoop.hdfs.server.federation.RouterConfigBuilder; -import org.apache.hadoop.hdfs.server.federation.metrics.NamenodeBeanMetrics; import org.apache.hadoop.hdfs.server.namenode.sps.Context; import org.apache.hadoop.hdfs.server.namenode.sps.StoragePolicySatisfier; import org.apache.hadoop.hdfs.server.sps.ExternalSPSContext; diff --git a/hadoop-hdfs-project/hadoop-hdfs-rbf/src/test/java/org/apache/hadoop/hdfs/server/federation/router/TestRouterTrash.java b/hadoop-hdfs-project/hadoop-hdfs-rbf/src/test/java/org/apache/hadoop/hdfs/server/federation/router/TestRouterTrash.java index acd7b87a14b4f..dfb8c33c72d4b 100644 --- a/hadoop-hdfs-project/hadoop-hdfs-rbf/src/test/java/org/apache/hadoop/hdfs/server/federation/router/TestRouterTrash.java +++ b/hadoop-hdfs-project/hadoop-hdfs-rbf/src/test/java/org/apache/hadoop/hdfs/server/federation/router/TestRouterTrash.java @@ -189,6 +189,44 @@ public void testMoveToTrashNoMountPoint() throws IOException, assertEquals(2, fileStatuses.length); } + @Test + public void testMoveToTrashWithKerberosUser() throws IOException, + URISyntaxException, InterruptedException { + //Constructs the structure of the KerBoers user name + String kerberosUser = "randomUser/dev@HADOOP.COM"; + UserGroupInformation ugi = UserGroupInformation.createRemoteUser(kerberosUser); + MountTable addEntry = MountTable.newInstance(MOUNT_POINT, + Collections.singletonMap(ns1, MOUNT_POINT)); + assertTrue(addMountTable(addEntry)); + // current user client + MiniRouterDFSCluster.NamenodeContext nn1Context = cluster.getNamenode(ns1, null); + DFSClient currentUserClientNs0 = nnContext.getClient(); + DFSClient currentUserClientNs1 = nn1Context.getClient(); + + currentUserClientNs0.setOwner("/", ugi.getShortUserName(), ugi.getShortUserName()); + currentUserClientNs1.setOwner("/", ugi.getShortUserName(), ugi.getShortUserName()); + + // test user client + DFSClient testUserClientNs1 = nn1Context.getClient(ugi); + testUserClientNs1.mkdirs(MOUNT_POINT, new FsPermission("777"), true); + assertTrue(testUserClientNs1.exists(MOUNT_POINT)); + // create test file + testUserClientNs1.create(FILE, true); + Path filePath = new Path(FILE); + + FileStatus[] fileStatuses = routerFs.listStatus(filePath); + assertEquals(1, fileStatuses.length); + assertEquals(ugi.getShortUserName(), fileStatuses[0].getOwner()); + // move to Trash + Configuration routerConf = routerContext.getConf(); + FileSystem fs = DFSTestUtil.getFileSystemAs(ugi, routerConf); + Trash trash = new Trash(fs, routerConf); + assertTrue(trash.moveToTrash(filePath)); + fileStatuses = fs.listStatus( + new Path("/user/" + ugi.getShortUserName() + "/.Trash/Current" + MOUNT_POINT)); + assertEquals(1, fileStatuses.length); + } + @Test public void testDeleteToTrashExistMountPoint() throws IOException, URISyntaxException, InterruptedException { diff --git a/hadoop-hdfs-project/hadoop-hdfs-rbf/src/test/java/org/apache/hadoop/hdfs/server/federation/router/TestRouterUserMappings.java b/hadoop-hdfs-project/hadoop-hdfs-rbf/src/test/java/org/apache/hadoop/hdfs/server/federation/router/TestRouterUserMappings.java index 707a2f7baa348..2be7b7b1f49c2 100644 --- a/hadoop-hdfs-project/hadoop-hdfs-rbf/src/test/java/org/apache/hadoop/hdfs/server/federation/router/TestRouterUserMappings.java +++ b/hadoop-hdfs-project/hadoop-hdfs-rbf/src/test/java/org/apache/hadoop/hdfs/server/federation/router/TestRouterUserMappings.java @@ -39,7 +39,6 @@ import org.apache.hadoop.test.GenericTestUtils; import org.apache.hadoop.test.LambdaTestUtils; import org.apache.hadoop.tools.GetUserMappingsProtocol; -import org.apache.hadoop.util.Sets; import org.junit.After; import org.junit.Before; import org.junit.Test; @@ -57,6 +56,8 @@ import java.net.URL; import java.net.URLDecoder; import java.util.ArrayList; +import java.util.Arrays; +import java.util.HashSet; import java.util.LinkedHashSet; import java.util.List; import java.util.Set; @@ -120,7 +121,7 @@ public Set getGroupsSet(String user) throws IOException { LOG.info("Getting groups in MockUnixGroupsMapping"); String g1 = user + (10 * i + 1); String g2 = user + (10 * i + 2); - Set s = Sets.newHashSet(g1, g2); + Set s = new HashSet<>(Arrays.asList(g1, g2)); i++; return s; } diff --git a/hadoop-hdfs-project/hadoop-hdfs-rbf/src/test/java/org/apache/hadoop/hdfs/server/federation/security/token/TestZKDelegationTokenSecretManagerImpl.java b/hadoop-hdfs-project/hadoop-hdfs-rbf/src/test/java/org/apache/hadoop/hdfs/server/federation/security/token/TestZKDelegationTokenSecretManagerImpl.java index 3c7f8e88a91d1..e4c293091b045 100644 --- a/hadoop-hdfs-project/hadoop-hdfs-rbf/src/test/java/org/apache/hadoop/hdfs/server/federation/security/token/TestZKDelegationTokenSecretManagerImpl.java +++ b/hadoop-hdfs-project/hadoop-hdfs-rbf/src/test/java/org/apache/hadoop/hdfs/server/federation/security/token/TestZKDelegationTokenSecretManagerImpl.java @@ -110,7 +110,7 @@ public void testMultiNodeTokenRemovalShortSyncWithoutWatch() conf.setBoolean(ZK_DTSM_TOKEN_WATCHER_ENABLED, false); // make sync quick conf.setInt(ZK_DTSM_ROUTER_TOKEN_SYNC_INTERVAL, 3); - // set the renew window and removal interval to be a + // set the renewal window and removal interval to be a // short time to trigger the background cleanup conf.setInt(RENEW_INTERVAL, 10); conf.setInt(REMOVAL_SCAN_INTERVAL, 10); @@ -170,7 +170,7 @@ public void testMultiNodeTokenRemovalLongSyncWithoutWatch() conf.setBoolean(ZK_DTSM_TOKEN_WATCHER_ENABLED, false); // make sync quick conf.setInt(ZK_DTSM_ROUTER_TOKEN_SYNC_INTERVAL, 20); - // set the renew window and removal interval to be a + // set the renewal window and removal interval to be a // short time to trigger the background cleanup conf.setInt(RENEW_INTERVAL, 10); conf.setInt(REMOVAL_SCAN_INTERVAL, 10); diff --git a/hadoop-hdfs-project/hadoop-hdfs-rbf/src/test/java/org/apache/hadoop/hdfs/server/federation/store/TestStateStoreMembershipState.java b/hadoop-hdfs-project/hadoop-hdfs-rbf/src/test/java/org/apache/hadoop/hdfs/server/federation/store/TestStateStoreMembershipState.java index 63bc6235a6116..45a240b866b85 100644 --- a/hadoop-hdfs-project/hadoop-hdfs-rbf/src/test/java/org/apache/hadoop/hdfs/server/federation/store/TestStateStoreMembershipState.java +++ b/hadoop-hdfs-project/hadoop-hdfs-rbf/src/test/java/org/apache/hadoop/hdfs/server/federation/store/TestStateStoreMembershipState.java @@ -186,7 +186,7 @@ public void testRegistrationMajorityQuorum() // 1) ns0:nn0 - Standby (newest) // 2) ns0:nn0 - Active (oldest) // 3) ns0:nn0 - Active (2nd oldest) - // 4) ns0:nn0 - Active (3nd oldest element, newest active element) + // 4) ns0:nn0 - Active (3rd oldest element, newest active element) // Verify the selected entry is the newest majority opinion (4) String ns = "ns0"; String nn = "nn0"; diff --git a/hadoop-hdfs-project/hadoop-hdfs-rbf/src/test/java/org/apache/hadoop/hdfs/server/federation/store/driver/TestStateStoreDriverBase.java b/hadoop-hdfs-project/hadoop-hdfs-rbf/src/test/java/org/apache/hadoop/hdfs/server/federation/store/driver/TestStateStoreDriverBase.java index fe1b9a5bfa04c..b8bb7c4d2d115 100644 --- a/hadoop-hdfs-project/hadoop-hdfs-rbf/src/test/java/org/apache/hadoop/hdfs/server/federation/store/driver/TestStateStoreDriverBase.java +++ b/hadoop-hdfs-project/hadoop-hdfs-rbf/src/test/java/org/apache/hadoop/hdfs/server/federation/store/driver/TestStateStoreDriverBase.java @@ -556,7 +556,7 @@ private static Map> getFields(BaseRecord record) { } /** - * Get the type of a field. + * Get the type of field. * * @param fieldName * @return Field type @@ -601,7 +601,7 @@ private static Method locateGetter(BaseRecord record, String fieldName) { } /** - * Expands a data object from the store into an record object. Default store + * Expands a data object from the store into a record object. Default store * data type is a String. Override if additional serialization is required. * * @param data Object containing the serialized data. Only string is diff --git a/hadoop-hdfs-project/hadoop-hdfs/dev-support/jdiff/Apache_Hadoop_HDFS_2.10.2.xml b/hadoop-hdfs-project/hadoop-hdfs/dev-support/jdiff/Apache_Hadoop_HDFS_2.10.2.xml new file mode 100644 index 0000000000000..8ab450692ddf3 --- /dev/null +++ b/hadoop-hdfs-project/hadoop-hdfs/dev-support/jdiff/Apache_Hadoop_HDFS_2.10.2.xml @@ -0,0 +1,312 @@ + + + + + + + + + + + A distributed implementation of {@link +org.apache.hadoop.fs.FileSystem}. This is loosely modelled after +Google's GFS.

      + +

      The most important difference is that unlike GFS, Hadoop DFS files +have strictly one writer at any one time. Bytes are always appended +to the end of the writer's stream. There is no notion of "record appends" +or "mutations" that are then checked or reordered. Writers simply emit +a byte stream. That byte stream is guaranteed to be stored in the +order written.

      ]]> + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + This method must return as quickly as possible, since it's called + in a critical section of the NameNode's operation. + + @param succeeded Whether authorization succeeded. + @param userName Name of the user executing the request. + @param addr Remote address of the request. + @param cmd The requested command. + @param src Path of affected source file. + @param dst Path of affected destination file (if any). + @param stat File information for operations that change the file's + metadata (permissions, owner, times, etc).]]> + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + diff --git a/hadoop-hdfs-project/hadoop-hdfs/dev-support/jdiff/Apache_Hadoop_HDFS_3.3.3.xml b/hadoop-hdfs-project/hadoop-hdfs/dev-support/jdiff/Apache_Hadoop_HDFS_3.3.3.xml new file mode 100644 index 0000000000000..0eaf5019aa6d2 --- /dev/null +++ b/hadoop-hdfs-project/hadoop-hdfs/dev-support/jdiff/Apache_Hadoop_HDFS_3.3.3.xml @@ -0,0 +1,835 @@ + + + + + + + + + + + A distributed implementation of {@link +org.apache.hadoop.fs.FileSystem}. This is loosely modelled after +Google's GFS.

      + +

      The most important difference is that unlike GFS, Hadoop DFS files +have strictly one writer at any one time. Bytes are always appended +to the end of the writer's stream. There is no notion of "record appends" +or "mutations" that are then checked or reordered. Writers simply emit +a byte stream. That byte stream is guaranteed to be stored in the +order written.

      ]]> +
      +
      + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + This method must return as quickly as possible, since it's called + in a critical section of the NameNode's operation. + + @param succeeded Whether authorization succeeded. + @param userName Name of the user executing the request. + @param addr Remote address of the request. + @param cmd The requested command. + @param src Path of affected source file. + @param dst Path of affected destination file (if any). + @param stat File information for operations that change the file's + metadata (permissions, owner, times, etc).]]> + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + +
      diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/DFSConfigKeys.java b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/DFSConfigKeys.java index 962b0dce833c5..9e1333f95295b 100755 --- a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/DFSConfigKeys.java +++ b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/DFSConfigKeys.java @@ -1404,6 +1404,10 @@ public class DFSConfigKeys extends CommonConfigurationKeys { "dfs.journalnode.edits.dir.perm"; public static final String DFS_JOURNAL_EDITS_DIR_PERMISSION_DEFAULT = "700"; + public static final String DFS_JOURNALNODE_HANDLER_COUNT_KEY = + "dfs.journalnode.handler.count"; + public static final int DFS_JOURNALNODE_HANDLER_COUNT_DEFAULT = 5; + public static final String DFS_JOURNALNODE_HTTP_ADDRESS_KEY = "dfs.journalnode.http-address"; public static final int DFS_JOURNALNODE_HTTP_PORT_DEFAULT = 8480; diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/DFSUtil.java b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/DFSUtil.java index 50c947b05941a..7237489e7bfef 100644 --- a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/DFSUtil.java +++ b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/DFSUtil.java @@ -76,7 +76,6 @@ import org.apache.hadoop.net.DomainNameResolverFactory; import org.apache.hadoop.security.AccessControlException; import org.apache.hadoop.util.Lists; -import org.apache.hadoop.util.Sets; import org.apache.hadoop.thirdparty.com.google.common.collect.Maps; import org.slf4j.Logger; import org.slf4j.LoggerFactory; @@ -725,8 +724,9 @@ private static Collection getParentNameServices(Configuration conf) } else { // Ensure that the internal service is indeed in the list of all available // nameservices. - Set availableNameServices = Sets.newHashSet(conf - .getTrimmedStringCollection(DFSConfigKeys.DFS_NAMESERVICES)); + Collection namespaces = conf + .getTrimmedStringCollection(DFSConfigKeys.DFS_NAMESERVICES); + Set availableNameServices = new HashSet<>(namespaces); for (String nsId : parentNameServices) { if (!availableNameServices.contains(nsId)) { throw new IOException("Unknown nameservice: " + nsId); diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/protocol/SnapshotInfo.java b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/protocol/SnapshotInfo.java index ef547788f1ac7..626bf2678883c 100644 --- a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/protocol/SnapshotInfo.java +++ b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/protocol/SnapshotInfo.java @@ -20,7 +20,6 @@ import org.apache.hadoop.classification.InterfaceAudience; import org.apache.hadoop.classification.InterfaceStability; import org.apache.hadoop.hdfs.protocol.proto.AclProtos.FsPermissionProto; -import org.apache.hadoop.hdfs.server.namenode.snapshot.Snapshot; /** * SnapshotInfo maintains information for a snapshot diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/protocol/datatransfer/Receiver.java b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/protocol/datatransfer/Receiver.java index ab767e3b93810..8bcfb199ff5a9 100644 --- a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/protocol/datatransfer/Receiver.java +++ b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/protocol/datatransfer/Receiver.java @@ -32,7 +32,6 @@ import org.apache.hadoop.hdfs.protocol.proto.DataTransferProtos.BaseHeaderProto; import org.apache.hadoop.hdfs.protocol.proto.DataTransferProtos.CachingStrategyProto; import org.apache.hadoop.hdfs.protocol.proto.DataTransferProtos.ClientOperationHeaderProto; -import org.apache.hadoop.hdfs.protocol.proto.DataTransferProtos.DataTransferTraceInfoProto; import org.apache.hadoop.hdfs.protocol.proto.DataTransferProtos.OpBlockChecksumProto; import org.apache.hadoop.hdfs.protocol.proto.DataTransferProtos.OpBlockGroupChecksumProto; import org.apache.hadoop.hdfs.protocol.proto.DataTransferProtos.OpCopyBlockProto; diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/protocolPB/PBHelper.java b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/protocolPB/PBHelper.java index 690ad0c279019..ce27342729ba3 100644 --- a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/protocolPB/PBHelper.java +++ b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/protocolPB/PBHelper.java @@ -112,6 +112,7 @@ import org.apache.hadoop.hdfs.server.protocol.NamenodeCommand; import org.apache.hadoop.hdfs.server.protocol.NamenodeRegistration; import org.apache.hadoop.hdfs.server.protocol.NamespaceInfo; +import org.apache.hadoop.hdfs.server.protocol.OutlierMetrics; import org.apache.hadoop.hdfs.server.protocol.ReceivedDeletedBlockInfo; import org.apache.hadoop.hdfs.server.protocol.ReceivedDeletedBlockInfo.BlockStatus; import org.apache.hadoop.hdfs.server.protocol.RegisterCommand; @@ -853,11 +854,15 @@ public static List convertSlowPeerInfo( List slowPeerInfoProtos = new ArrayList<>(slowPeers.getSlowPeers().size()); - for (Map.Entry entry : - slowPeers.getSlowPeers().entrySet()) { - slowPeerInfoProtos.add(SlowPeerReportProto.newBuilder() + for (Map.Entry entry : slowPeers.getSlowPeers().entrySet()) { + OutlierMetrics outlierMetrics = entry.getValue(); + slowPeerInfoProtos.add( + SlowPeerReportProto.newBuilder() .setDataNodeId(entry.getKey()) - .setAggregateLatency(entry.getValue()) + .setAggregateLatency(outlierMetrics.getActualLatency()) + .setMedian(outlierMetrics.getMedian()) + .setMad(outlierMetrics.getMad()) + .setUpperLimitLatency(outlierMetrics.getUpperLimitLatency()) .build()); } return slowPeerInfoProtos; @@ -871,15 +876,19 @@ public static SlowPeerReports convertSlowPeerInfo( return SlowPeerReports.EMPTY_REPORT; } - Map slowPeersMap = new HashMap<>(slowPeerProtos.size()); + Map slowPeersMap = new HashMap<>(slowPeerProtos.size()); for (SlowPeerReportProto proto : slowPeerProtos) { if (!proto.hasDataNodeId()) { // The DataNodeId should be reported. continue; } - slowPeersMap.put( - proto.getDataNodeId(), - proto.hasAggregateLatency() ? proto.getAggregateLatency() : 0.0); + Double aggregateLatency = proto.hasAggregateLatency() ? proto.getAggregateLatency() : 0.0; + Double medianLatency = proto.hasMedian() ? proto.getMedian() : 0.0; + Double madLatency = proto.hasMad() ? proto.getMad() : 0.0; + Double upperLimitLatency = proto.hasUpperLimitLatency() ? proto.getUpperLimitLatency() : 0.0; + OutlierMetrics outlierMetrics = + new OutlierMetrics(medianLatency, madLatency, upperLimitLatency, aggregateLatency); + slowPeersMap.put(proto.getDataNodeId(), outlierMetrics); } return SlowPeerReports.create(slowPeersMap); } diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/qjournal/server/JNStorage.java b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/qjournal/server/JNStorage.java index 7f82bff3376f0..78ef6acc9af4d 100644 --- a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/qjournal/server/JNStorage.java +++ b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/qjournal/server/JNStorage.java @@ -123,10 +123,6 @@ File getSyncLogTemporaryFile(long segmentTxId, long epoch) { return new File(sd.getCurrentDir(), name); } - File getCurrentDir() { - return sd.getCurrentDir(); - } - /** * Directory {@code edits.sync} temporarily holds the log segments * downloaded through {@link JournalNodeSyncer} before they are moved to diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/qjournal/server/JournalNodeRpcServer.java b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/qjournal/server/JournalNodeRpcServer.java index 46820d38662be..ad67cf481ae70 100644 --- a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/qjournal/server/JournalNodeRpcServer.java +++ b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/qjournal/server/JournalNodeRpcServer.java @@ -55,6 +55,8 @@ import java.net.InetSocketAddress; import java.net.URL; +import static org.apache.hadoop.hdfs.DFSConfigKeys.DFS_JOURNALNODE_HANDLER_COUNT_DEFAULT; +import static org.apache.hadoop.hdfs.DFSConfigKeys.DFS_JOURNALNODE_HANDLER_COUNT_KEY; import static org.apache.hadoop.hdfs.DFSConfigKeys.DFS_JOURNALNODE_RPC_BIND_HOST_KEY; @@ -63,9 +65,9 @@ public class JournalNodeRpcServer implements QJournalProtocol, InterQJournalProtocol { private static final Logger LOG = JournalNode.LOG; - private static final int HANDLER_COUNT = 5; private final JournalNode jn; private Server server; + private final int handlerCount; JournalNodeRpcServer(Configuration conf, JournalNode jn) throws IOException { this.jn = jn; @@ -90,13 +92,25 @@ public class JournalNodeRpcServer implements QJournalProtocol, new QJournalProtocolServerSideTranslatorPB(this); BlockingService service = QJournalProtocolService .newReflectiveBlockingService(translator); + int confHandlerCount = conf.getInt(DFS_JOURNALNODE_HANDLER_COUNT_KEY, + DFS_JOURNALNODE_HANDLER_COUNT_DEFAULT); + if (confHandlerCount <= 0) { + LOG.warn("Invalid value for: {} = {}, Should be > 0," + + " will use default value of: {}.", + DFS_JOURNALNODE_HANDLER_COUNT_KEY, confHandlerCount, + DFS_JOURNALNODE_HANDLER_COUNT_DEFAULT); + confHandlerCount = DFS_JOURNALNODE_HANDLER_COUNT_DEFAULT; + } + this.handlerCount = confHandlerCount; + LOG.info("The number of JournalNodeRpcServer handlers is {}.", + this.handlerCount); this.server = new RPC.Builder(confCopy) .setProtocol(QJournalProtocolPB.class) .setInstance(service) .setBindAddress(bindHost) .setPort(addr.getPort()) - .setNumHandlers(HANDLER_COUNT) + .setNumHandlers(this.handlerCount) .setVerbose(false) .build(); @@ -121,6 +135,11 @@ public class JournalNodeRpcServer implements QJournalProtocol, this.server.setTracer(jn.tracer); } + @VisibleForTesting + protected int getHandlerCount() { + return this.handlerCount; + } + void start() { this.server.start(); } diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/qjournal/server/JournalNodeSyncer.java b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/qjournal/server/JournalNodeSyncer.java index ff46aa751d7ba..fd29c849dfcb3 100644 --- a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/qjournal/server/JournalNodeSyncer.java +++ b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/qjournal/server/JournalNodeSyncer.java @@ -39,7 +39,6 @@ import org.apache.hadoop.security.UserGroupInformation; import org.apache.hadoop.util.Daemon; import org.apache.hadoop.util.Lists; -import org.apache.hadoop.util.Sets; import org.slf4j.Logger; import org.slf4j.LoggerFactory; @@ -51,6 +50,7 @@ import java.net.URISyntaxException; import java.net.URL; import java.security.PrivilegedExceptionAction; +import java.util.Arrays; import java.util.Collection; import java.util.HashSet; import java.util.List; @@ -273,7 +273,7 @@ private List getOtherJournalNodeAddrs() { } if (uriStr == null || uriStr.isEmpty()) { - HashSet sharedEditsUri = Sets.newHashSet(); + HashSet sharedEditsUri = new HashSet<>(); if (nameServiceId != null) { Collection nnIds = DFSUtilClient.getNameNodeIds( conf, nameServiceId); @@ -315,7 +315,7 @@ private List getJournalAddrList(String uriStr) throws IOException { URI uri = new URI(uriStr); return Util.getLoggerAddresses(uri, - Sets.newHashSet(jn.getBoundIpcAddress()), conf); + new HashSet<>(Arrays.asList(jn.getBoundIpcAddress())), conf); } private void getMissingLogSegments(List thisJournalEditLogs, diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/balancer/Dispatcher.java b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/balancer/Dispatcher.java index 612f4c028eff5..5c66d66912072 100644 --- a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/balancer/Dispatcher.java +++ b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/balancer/Dispatcher.java @@ -241,6 +241,7 @@ public class PendingMove { private DDatanode proxySource; private StorageGroup target; + @VisibleForTesting PendingMove(Source source, StorageGroup target) { this.source = source; this.target = target; @@ -282,6 +283,7 @@ private boolean chooseBlockAndProxy() { /** * @return true if the given block is good for the tentative move. */ + @VisibleForTesting boolean markMovedIfGoodBlock(DBlock block, StorageType targetStorageType) { synchronized (block) { synchronized (movedBlocks) { @@ -1361,6 +1363,7 @@ public static boolean checkForSuccess( * 2. the block does not have a replica/internalBlock on the target; * 3. doing the move does not reduce the number of racks that the block has */ + @VisibleForTesting boolean isGoodBlockCandidate(StorageGroup source, StorageGroup target, StorageType targetStorageType, DBlock block) { if (source.equals(target)) { diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/balancer/NameNodeConnector.java b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/balancer/NameNodeConnector.java index 238457bcb867a..34be025203d47 100644 --- a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/balancer/NameNodeConnector.java +++ b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/balancer/NameNodeConnector.java @@ -21,6 +21,7 @@ import java.io.IOException; import java.io.OutputStream; import java.net.InetAddress; +import java.net.InetSocketAddress; import java.net.URI; import java.util.ArrayList; import java.util.Arrays; @@ -260,13 +261,15 @@ public BlocksWithLocations getBlocks(DatanodeInfo datanode, long size, long } boolean isRequestStandby = false; NamenodeProtocol nnProxy = null; + InetSocketAddress standbyAddress = null; try { ProxyPair proxyPair = getProxy(); isRequestStandby = proxyPair.isRequestStandby; ClientProtocol proxy = proxyPair.clientProtocol; if (isRequestStandby) { + standbyAddress = RPC.getServerAddress(proxy); nnProxy = NameNodeProxies.createNonHAProxy( - config, RPC.getServerAddress(proxy), NamenodeProtocol.class, + config, standbyAddress, NamenodeProtocol.class, UserGroupInformation.getCurrentUser(), false).getProxy(); } else { nnProxy = namenode; @@ -274,7 +277,8 @@ public BlocksWithLocations getBlocks(DatanodeInfo datanode, long size, long return nnProxy.getBlocks(datanode, size, minBlockSize, timeInterval); } finally { if (isRequestStandby) { - LOG.info("Request #getBlocks to Standby NameNode success."); + LOG.info("Request #getBlocks to Standby NameNode success. " + + "remoteAddress: {}", standbyAddress.getHostString()); } } } @@ -297,15 +301,19 @@ public boolean isUpgrading() throws IOException { public DatanodeStorageReport[] getLiveDatanodeStorageReport() throws IOException { boolean isRequestStandby = false; + InetSocketAddress standbyAddress = null; try { ProxyPair proxyPair = getProxy(); isRequestStandby = proxyPair.isRequestStandby; ClientProtocol proxy = proxyPair.clientProtocol; + if (isRequestStandby) { + standbyAddress = RPC.getServerAddress(proxy); + } return proxy.getDatanodeStorageReport(DatanodeReportType.LIVE); } finally { if (isRequestStandby) { LOG.info("Request #getLiveDatanodeStorageReport to Standby " + - "NameNode success."); + "NameNode success. remoteAddress: {}", standbyAddress.getHostString()); } } } diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/blockmanagement/BlockManager.java b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/blockmanagement/BlockManager.java index c522e2604e70f..51e12ec43372c 100644 --- a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/blockmanagement/BlockManager.java +++ b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/blockmanagement/BlockManager.java @@ -1916,23 +1916,29 @@ private void markBlockAsCorrupt(BlockToMarkCorrupt b, b.getReasonCode(), b.getStored().isStriped()); NumberReplicas numberOfReplicas = countNodes(b.getStored()); - boolean hasEnoughLiveReplicas = numberOfReplicas.liveReplicas() >= + final int numUsableReplicas = numberOfReplicas.liveReplicas() + + numberOfReplicas.decommissioning() + + numberOfReplicas.liveEnteringMaintenanceReplicas(); + boolean hasEnoughLiveReplicas = numUsableReplicas >= expectedRedundancies; boolean minReplicationSatisfied = hasMinStorage(b.getStored(), - numberOfReplicas.liveReplicas()); + numUsableReplicas); boolean hasMoreCorruptReplicas = minReplicationSatisfied && (numberOfReplicas.liveReplicas() + numberOfReplicas.corruptReplicas()) > expectedRedundancies; boolean corruptedDuringWrite = minReplicationSatisfied && b.isCorruptedDuringWrite(); - // case 1: have enough number of live replicas - // case 2: corrupted replicas + live replicas > Replication factor + // case 1: have enough number of usable replicas + // case 2: corrupted replicas + usable replicas > Replication factor // case 3: Block is marked corrupt due to failure while writing. In this // case genstamp will be different than that of valid block. // In all these cases we can delete the replica. - // In case of 3, rbw block will be deleted and valid block can be replicated + // In case 3, rbw block will be deleted and valid block can be replicated. + // Note NN only becomes aware of corrupt blocks when the block report is sent, + // this means that by default it can take up to 6 hours for a corrupt block to + // be invalidated, after which the valid block can be replicated. if (hasEnoughLiveReplicas || hasMoreCorruptReplicas || corruptedDuringWrite) { if (b.getStored().isStriped()) { @@ -3656,7 +3662,7 @@ private Block addStoredBlock(final BlockInfo block, ". blockMap has {} but corrupt replicas map has {}", storedBlock, numCorruptNodes, corruptReplicasCount); } - if ((corruptReplicasCount > 0) && (numLiveReplicas >= fileRedundancy)) { + if ((corruptReplicasCount > 0) && (numUsableReplicas >= fileRedundancy)) { invalidateCorruptReplicas(storedBlock, reportedBlock, num); } return storedBlock; diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/blockmanagement/DatanodeAdminBackoffMonitor.java b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/blockmanagement/DatanodeAdminBackoffMonitor.java index 9b38f2353fbd7..a7d72d019bde5 100644 --- a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/blockmanagement/DatanodeAdminBackoffMonitor.java +++ b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/blockmanagement/DatanodeAdminBackoffMonitor.java @@ -32,8 +32,6 @@ import java.util.List; import java.util.Iterator; import java.util.LinkedList; -import java.util.ArrayDeque; -import java.util.Queue; import java.util.stream.Collectors; /** @@ -71,12 +69,6 @@ public class DatanodeAdminBackoffMonitor extends DatanodeAdminMonitorBase private HashMap> outOfServiceNodeBlocks = new HashMap<>(); - /** - * Any nodes where decommission or maintenance has been cancelled are added - * to this queue for later processing. - */ - private final Queue cancelledNodes = new ArrayDeque<>(); - /** * The numbe of blocks to process when moving blocks to pendingReplication * before releasing and reclaiming the namenode lock. @@ -151,7 +143,7 @@ protected void processConf() { @Override public void stopTrackingNode(DatanodeDescriptor dn) { getPendingNodes().remove(dn); - cancelledNodes.add(dn); + getCancelledNodes().add(dn); } @Override @@ -232,7 +224,7 @@ public void run() { "in maintenance or transitioning state. {} nodes pending. {} " + "nodes waiting to be cancelled.", numBlocksChecked, outOfServiceNodeBlocks.size(), getPendingNodes().size(), - cancelledNodes.size()); + getCancelledNodes().size()); } } @@ -259,8 +251,8 @@ private void processPendingNodes() { * write lock to prevent the cancelledNodes list being modified externally. */ private void processCancelledNodes() { - while(!cancelledNodes.isEmpty()) { - DatanodeDescriptor dn = cancelledNodes.poll(); + while(!getCancelledNodes().isEmpty()) { + DatanodeDescriptor dn = getCancelledNodes().poll(); outOfServiceNodeBlocks.remove(dn); pendingRep.remove(dn); } diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/blockmanagement/DatanodeAdminDefaultMonitor.java b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/blockmanagement/DatanodeAdminDefaultMonitor.java index 4149d198c72f8..3ff360466c98f 100644 --- a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/blockmanagement/DatanodeAdminDefaultMonitor.java +++ b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/blockmanagement/DatanodeAdminDefaultMonitor.java @@ -124,7 +124,7 @@ private boolean exceededNumBlocksPerCheck() { @Override public void stopTrackingNode(DatanodeDescriptor dn) { getPendingNodes().remove(dn); - outOfServiceNodeBlocks.remove(dn); + getCancelledNodes().add(dn); } @Override @@ -152,6 +152,7 @@ public void run() { // Check decommission or maintenance progress. namesystem.writeLock(); try { + processCancelledNodes(); processPendingNodes(); check(); } catch (Exception e) { @@ -180,6 +181,20 @@ private void processPendingNodes() { } } + /** + * Process any nodes which have had their decommission or maintenance mode + * cancelled by an administrator. + * + * This method must be executed under the write lock to prevent the + * internal structures being modified concurrently. + */ + private void processCancelledNodes() { + while(!getCancelledNodes().isEmpty()) { + DatanodeDescriptor dn = getCancelledNodes().poll(); + outOfServiceNodeBlocks.remove(dn); + } + } + private void check() { final Iterator>> it = new CyclicIteration<>(outOfServiceNodeBlocks, diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/blockmanagement/DatanodeAdminMonitorBase.java b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/blockmanagement/DatanodeAdminMonitorBase.java index f9761c2dfcb4e..5aab1b4a8a18c 100644 --- a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/blockmanagement/DatanodeAdminMonitorBase.java +++ b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/blockmanagement/DatanodeAdminMonitorBase.java @@ -24,6 +24,7 @@ import org.slf4j.Logger; import org.slf4j.LoggerFactory; +import java.util.ArrayDeque; import java.util.Comparator; import java.util.List; import java.util.PriorityQueue; @@ -53,6 +54,12 @@ public abstract class DatanodeAdminMonitorBase private final PriorityQueue pendingNodes = new PriorityQueue<>( PENDING_NODES_QUEUE_COMPARATOR); + /** + * Any nodes where decommission or maintenance has been cancelled are added + * to this queue for later processing. + */ + private final Queue cancelledNodes = new ArrayDeque<>(); + /** * The maximum number of nodes to track in outOfServiceNodeBlocks. * A value of 0 means no limit. @@ -163,6 +170,11 @@ public Queue getPendingNodes() { return pendingNodes; } + @Override + public Queue getCancelledNodes() { + return cancelledNodes; + } + /** * If node "is dead while in Decommission In Progress", it cannot be decommissioned * until it becomes healthy again. If there are more pendingNodes than can be tracked diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/blockmanagement/DatanodeAdminMonitorInterface.java b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/blockmanagement/DatanodeAdminMonitorInterface.java index f34c00587c78d..89673a759eaed 100644 --- a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/blockmanagement/DatanodeAdminMonitorInterface.java +++ b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/blockmanagement/DatanodeAdminMonitorInterface.java @@ -32,6 +32,7 @@ public interface DatanodeAdminMonitorInterface extends Runnable { int getTrackedNodeCount(); int getNumNodesChecked(); Queue getPendingNodes(); + Queue getCancelledNodes(); void setBlockManager(BlockManager bm); void setDatanodeAdminManager(DatanodeAdminManager dnm); diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/blockmanagement/DatanodeManager.java b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/blockmanagement/DatanodeManager.java index b1dbbdde3b9f8..237daed0960ff 100644 --- a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/blockmanagement/DatanodeManager.java +++ b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/blockmanagement/DatanodeManager.java @@ -1790,8 +1790,8 @@ private void addCacheCommands(String blockPoolId, DatanodeDescriptor nodeinfo, /** Handle heartbeat from datanodes. */ public DatanodeCommand[] handleHeartbeat(DatanodeRegistration nodeReg, StorageReport[] reports, final String blockPoolId, - long cacheCapacity, long cacheUsed, int xceiverCount, - int maxTransfers, int failedVolumes, + long cacheCapacity, long cacheUsed, int xceiverCount, + int xmitsInProgress, int failedVolumes, VolumeFailureSummary volumeFailureSummary, @Nonnull SlowPeerReports slowPeers, @Nonnull SlowDiskReports slowDisks) throws IOException { @@ -1835,6 +1835,14 @@ public DatanodeCommand[] handleHeartbeat(DatanodeRegistration nodeReg, int totalECBlocks = nodeinfo.getNumberOfBlocksToBeErasureCoded(); int totalBlocks = totalReplicateBlocks + totalECBlocks; if (totalBlocks > 0) { + int maxTransfers; + if (nodeinfo.isDecommissionInProgress()) { + maxTransfers = blockManager.getReplicationStreamsHardLimit() + - xmitsInProgress; + } else { + maxTransfers = blockManager.getMaxReplicationStreams() + - xmitsInProgress; + } int numReplicationTasks = (int) Math.ceil( (double) (totalReplicateBlocks * maxTransfers) / totalBlocks); int numECTasks = (int) Math.ceil( @@ -1896,13 +1904,14 @@ public DatanodeCommand[] handleHeartbeat(DatanodeRegistration nodeReg, Preconditions.checkNotNull(slowPeerTracker, "slowPeerTracker should not be un-assigned"); if (slowPeerTracker.isSlowPeerTrackerEnabled()) { - final Map slowPeersMap = slowPeers.getSlowPeers(); + final Map slowPeersMap = slowPeers.getSlowPeers(); if (!slowPeersMap.isEmpty()) { if (LOG.isDebugEnabled()) { LOG.debug("DataNode " + nodeReg + " reported slow peers: " + slowPeersMap); } - for (String slowNodeId : slowPeersMap.keySet()) { - slowPeerTracker.addReport(slowNodeId, nodeReg.getIpcAddr(false)); + for (Map.Entry slowNodeEntry : slowPeersMap.entrySet()) { + slowPeerTracker.addReport(slowNodeEntry.getKey(), nodeReg.getIpcAddr(false), + slowNodeEntry.getValue()); } } } @@ -2247,5 +2256,9 @@ public DatanodeStorageReport[] getDatanodeStorageReport( public Map getDatanodeMap() { return datanodeMap; } -} + public void setMaxSlowPeersToReport(int maxSlowPeersToReport) { + Preconditions.checkNotNull(slowPeerTracker, "slowPeerTracker should not be un-assigned"); + slowPeerTracker.setMaxSlowPeersToReport(maxSlowPeersToReport); + } +} diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/blockmanagement/SlowPeerDisabledTracker.java b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/blockmanagement/SlowPeerDisabledTracker.java index 2f6be2f5a7b39..ac109e0c90ae3 100644 --- a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/blockmanagement/SlowPeerDisabledTracker.java +++ b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/blockmanagement/SlowPeerDisabledTracker.java @@ -32,6 +32,7 @@ import org.apache.hadoop.classification.InterfaceAudience; import org.apache.hadoop.conf.Configuration; import org.apache.hadoop.hdfs.DFSConfigKeys; +import org.apache.hadoop.hdfs.server.protocol.OutlierMetrics; import org.apache.hadoop.util.Preconditions; import org.apache.hadoop.util.Timer; @@ -58,20 +59,20 @@ public boolean isSlowPeerTrackerEnabled() { } @Override - public void addReport(String slowNode, String reportingNode) { + public void addReport(String slowNode, String reportingNode, OutlierMetrics slowNodeMetrics) { LOG.trace("Adding slow peer report is disabled. To enable it, please enable config {}.", DFSConfigKeys.DFS_DATANODE_PEER_STATS_ENABLED_KEY); } @Override - public Set getReportsForNode(String slowNode) { + public Set getReportsForNode(String slowNode) { LOG.trace("Retrieval of slow peer report is disabled. To enable it, please enable config {}.", DFSConfigKeys.DFS_DATANODE_PEER_STATS_ENABLED_KEY); return ImmutableSet.of(); } @Override - public Map> getReportsForAllDataNodes() { + public Map> getReportsForAllDataNodes() { LOG.trace("Retrieval of slow peer report for all nodes is disabled. " + "To enable it, please enable config {}.", DFSConfigKeys.DFS_DATANODE_PEER_STATS_ENABLED_KEY); diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/blockmanagement/SlowPeerJsonReport.java b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/blockmanagement/SlowPeerJsonReport.java new file mode 100644 index 0000000000000..b9b741e9d611c --- /dev/null +++ b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/blockmanagement/SlowPeerJsonReport.java @@ -0,0 +1,84 @@ +/** + * 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.hdfs.server.blockmanagement; + +import java.util.SortedSet; + +import com.fasterxml.jackson.annotation.JsonProperty; + +import org.apache.commons.lang3.builder.EqualsBuilder; +import org.apache.commons.lang3.builder.HashCodeBuilder; +import org.apache.hadoop.classification.InterfaceAudience; + +/** + * This structure is a thin wrapper over slow peer reports to make Json + * [de]serialization easy. + */ +@InterfaceAudience.Private +final class SlowPeerJsonReport { + + @JsonProperty("SlowNode") + private final String slowNode; + + @JsonProperty("SlowPeerLatencyWithReportingNodes") + private final SortedSet slowPeerLatencyWithReportingNodes; + + SlowPeerJsonReport( + @JsonProperty("SlowNode") + String slowNode, + @JsonProperty("SlowPeerLatencyWithReportingNodes") + SortedSet slowPeerLatencyWithReportingNodes) { + this.slowNode = slowNode; + this.slowPeerLatencyWithReportingNodes = slowPeerLatencyWithReportingNodes; + } + + public String getSlowNode() { + return slowNode; + } + + public SortedSet getSlowPeerLatencyWithReportingNodes() { + return slowPeerLatencyWithReportingNodes; + } + + @Override + public boolean equals(Object o) { + if (this == o) { + return true; + } + + if (o == null || getClass() != o.getClass()) { + return false; + } + + SlowPeerJsonReport that = (SlowPeerJsonReport) o; + + return new EqualsBuilder() + .append(slowNode, that.slowNode) + .append(slowPeerLatencyWithReportingNodes, that.slowPeerLatencyWithReportingNodes) + .isEquals(); + } + + @Override + public int hashCode() { + return new HashCodeBuilder(17, 37) + .append(slowNode) + .append(slowPeerLatencyWithReportingNodes) + .toHashCode(); + } +} diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/blockmanagement/SlowPeerLatencyWithReportingNode.java b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/blockmanagement/SlowPeerLatencyWithReportingNode.java new file mode 100644 index 0000000000000..b90f809f1ed8c --- /dev/null +++ b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/blockmanagement/SlowPeerLatencyWithReportingNode.java @@ -0,0 +1,124 @@ +/** + * 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.hdfs.server.blockmanagement; + +import com.fasterxml.jackson.annotation.JsonProperty; + +import org.apache.commons.lang3.builder.EqualsBuilder; +import org.apache.commons.lang3.builder.HashCodeBuilder; +import org.apache.hadoop.classification.InterfaceAudience; + +/** + * This class represents the reporting node and the slow node's latency as observed by the + * reporting node. This class is used by SlowPeerJsonReport class. + */ +@InterfaceAudience.Private +final class SlowPeerLatencyWithReportingNode + implements Comparable { + + @JsonProperty("ReportingNode") + private final String reportingNode; + + @JsonProperty("ReportedLatency") + private final Double reportedLatency; + + @JsonProperty("MedianLatency") + private final Double medianLatency; + + @JsonProperty("MadLatency") + private final Double madLatency; + + @JsonProperty("UpperLimitLatency") + private final Double upperLimitLatency; + + SlowPeerLatencyWithReportingNode( + @JsonProperty("ReportingNode") + String reportingNode, + @JsonProperty("ReportedLatency") + Double reportedLatency, + @JsonProperty("MedianLatency") + Double medianLatency, + @JsonProperty("MadLatency") + Double madLatency, + @JsonProperty("UpperLimitLatency") + Double upperLimitLatency) { + this.reportingNode = reportingNode; + this.reportedLatency = reportedLatency; + this.medianLatency = medianLatency; + this.madLatency = madLatency; + this.upperLimitLatency = upperLimitLatency; + } + + public String getReportingNode() { + return reportingNode; + } + + public Double getReportedLatency() { + return reportedLatency; + } + + public Double getMedianLatency() { + return medianLatency; + } + + public Double getMadLatency() { + return madLatency; + } + + public Double getUpperLimitLatency() { + return upperLimitLatency; + } + + @Override + public int compareTo(SlowPeerLatencyWithReportingNode o) { + return this.reportingNode.compareTo(o.getReportingNode()); + } + + @Override + public boolean equals(Object o) { + if (this == o) { + return true; + } + + if (o == null || getClass() != o.getClass()) { + return false; + } + + SlowPeerLatencyWithReportingNode that = (SlowPeerLatencyWithReportingNode) o; + + return new EqualsBuilder() + .append(reportingNode, that.reportingNode) + .append(reportedLatency, that.reportedLatency) + .append(medianLatency, that.medianLatency) + .append(madLatency, that.madLatency) + .append(upperLimitLatency, that.upperLimitLatency) + .isEquals(); + } + + @Override + public int hashCode() { + return new HashCodeBuilder(17, 37) + .append(reportingNode) + .append(reportedLatency) + .append(medianLatency) + .append(madLatency) + .append(upperLimitLatency) + .toHashCode(); + } +} diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/blockmanagement/SlowPeerTracker.java b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/blockmanagement/SlowPeerTracker.java index c674bc19f8a29..e4feb4815eee4 100644 --- a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/blockmanagement/SlowPeerTracker.java +++ b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/blockmanagement/SlowPeerTracker.java @@ -18,7 +18,6 @@ package org.apache.hadoop.hdfs.server.blockmanagement; -import com.fasterxml.jackson.annotation.JsonProperty; import com.fasterxml.jackson.core.JsonProcessingException; import com.fasterxml.jackson.databind.ObjectMapper; import com.fasterxml.jackson.databind.ObjectWriter; @@ -29,6 +28,7 @@ import org.apache.hadoop.classification.InterfaceStability; import org.apache.hadoop.conf.Configuration; import org.apache.hadoop.hdfs.DFSConfigKeys; +import org.apache.hadoop.hdfs.server.protocol.OutlierMetrics; import org.apache.hadoop.hdfs.server.protocol.SlowPeerReports; import org.apache.hadoop.util.Timer; import org.slf4j.Logger; @@ -37,7 +37,6 @@ import java.util.ArrayList; import java.util.Collection; import java.util.Collections; -import java.util.Comparator; import java.util.HashMap; import java.util.List; import java.util.Map; @@ -81,7 +80,7 @@ public class SlowPeerTracker { * Number of nodes to include in JSON report. We will return nodes with * the highest number of votes from peers. */ - private final int maxNodesToReport; + private volatile int maxNodesToReport; /** * Information about peers that have reported a node as being slow. @@ -95,7 +94,7 @@ public class SlowPeerTracker { * Stale reports are not evicted proactively and can potentially * hang around forever. */ - private final ConcurrentMap> + private final ConcurrentMap> allReports; public SlowPeerTracker(Configuration conf, Timer timer) { @@ -105,9 +104,8 @@ public SlowPeerTracker(Configuration conf, Timer timer) { DFSConfigKeys.DFS_DATANODE_OUTLIERS_REPORT_INTERVAL_KEY, DFSConfigKeys.DFS_DATANODE_OUTLIERS_REPORT_INTERVAL_DEFAULT, TimeUnit.MILLISECONDS) * 3; - this.maxNodesToReport = conf.getInt( - DFSConfigKeys.DFS_DATANODE_MAX_NODES_TO_REPORT_KEY, - DFSConfigKeys.DFS_DATANODE_MAX_NODES_TO_REPORT_DEFAULT); + this.setMaxSlowPeersToReport(conf.getInt(DFSConfigKeys.DFS_DATANODE_MAX_NODES_TO_REPORT_KEY, + DFSConfigKeys.DFS_DATANODE_MAX_NODES_TO_REPORT_DEFAULT)); } /** @@ -123,12 +121,13 @@ public boolean isSlowPeerTrackerEnabled() { * Add a new report. DatanodeIds can be the DataNodeIds or addresses * We don't care as long as the caller is consistent. * - * @param reportingNode DataNodeId of the node reporting on its peer. * @param slowNode DataNodeId of the peer suspected to be slow. + * @param reportingNode DataNodeId of the node reporting on its peer. + * @param slowNodeMetrics Aggregate latency metrics of slownode as reported by the + * reporting node. */ - public void addReport(String slowNode, - String reportingNode) { - ConcurrentMap nodeEntries = allReports.get(slowNode); + public void addReport(String slowNode, String reportingNode, OutlierMetrics slowNodeMetrics) { + ConcurrentMap nodeEntries = allReports.get(slowNode); if (nodeEntries == null) { // putIfAbsent guards against multiple writers. @@ -137,7 +136,8 @@ public void addReport(String slowNode, } // Replace the existing entry from this node, if any. - nodeEntries.put(reportingNode, timer.monotonicNow()); + nodeEntries.put(reportingNode, + new LatencyWithLastReportTime(timer.monotonicNow(), slowNodeMetrics)); } /** @@ -147,8 +147,8 @@ public void addReport(String slowNode, * @param slowNode target node Id. * @return set of reports which implicate the target node as being slow. */ - public Set getReportsForNode(String slowNode) { - final ConcurrentMap nodeEntries = + public Set getReportsForNode(String slowNode) { + final ConcurrentMap nodeEntries = allReports.get(slowNode); if (nodeEntries == null || nodeEntries.isEmpty()) { @@ -163,17 +163,19 @@ public Set getReportsForNode(String slowNode) { * * @return map from SlowNodeId {@literal ->} (set of nodes reporting peers). */ - public Map> getReportsForAllDataNodes() { + public Map> getReportsForAllDataNodes() { if (allReports.isEmpty()) { return ImmutableMap.of(); } - final Map> allNodesValidReports = new HashMap<>(); + final Map> allNodesValidReports = + new HashMap<>(); final long now = timer.monotonicNow(); - for (Map.Entry> entry : - allReports.entrySet()) { - SortedSet validReports = filterNodeReports(entry.getValue(), now); + for (Map.Entry> entry + : allReports.entrySet()) { + SortedSet validReports = + filterNodeReports(entry.getValue(), now); if (!validReports.isEmpty()) { allNodesValidReports.put(entry.getKey(), validReports); } @@ -184,17 +186,21 @@ public Map> getReportsForAllDataNodes() { /** * Filter the given reports to return just the valid ones. * - * @param reports - * @param now - * @return + * @param reports Current set of reports. + * @param now Current time. + * @return Set of valid reports that were created within last reportValidityMs millis. */ - private SortedSet filterNodeReports( - ConcurrentMap reports, long now) { - final SortedSet validReports = new TreeSet<>(); - - for (Map.Entry entry : reports.entrySet()) { - if (now - entry.getValue() < reportValidityMs) { - validReports.add(entry.getKey()); + private SortedSet filterNodeReports( + ConcurrentMap reports, long now) { + final SortedSet validReports = new TreeSet<>(); + + for (Map.Entry entry : reports.entrySet()) { + if (now - entry.getValue().getTime() < reportValidityMs) { + OutlierMetrics outlierMetrics = entry.getValue().getLatency(); + validReports.add( + new SlowPeerLatencyWithReportingNode(entry.getKey(), outlierMetrics.getActualLatency(), + outlierMetrics.getMedian(), outlierMetrics.getMad(), + outlierMetrics.getUpperLimitLatency())); } } return validReports; @@ -206,7 +212,7 @@ private SortedSet filterNodeReports( * serialization failed. */ public String getJson() { - Collection validReports = getJsonReports( + Collection validReports = getJsonReports( maxNodesToReport); try { return WRITER.writeValueAsString(validReports); @@ -217,42 +223,15 @@ public String getJson() { } } - /** - * This structure is a thin wrapper over reports to make Json - * [de]serialization easy. - */ - public static class ReportForJson { - @JsonProperty("SlowNode") - final private String slowNode; - - @JsonProperty("ReportingNodes") - final private SortedSet reportingNodes; - - public ReportForJson( - @JsonProperty("SlowNode") String slowNode, - @JsonProperty("ReportingNodes") SortedSet reportingNodes) { - this.slowNode = slowNode; - this.reportingNodes = reportingNodes; - } - - public String getSlowNode() { - return slowNode; - } - - public SortedSet getReportingNodes() { - return reportingNodes; - } - } - /** * Returns all tracking slow peers. * @param numNodes * @return */ public List getSlowNodes(int numNodes) { - Collection jsonReports = getJsonReports(numNodes); + Collection jsonReports = getJsonReports(numNodes); ArrayList slowNodes = new ArrayList<>(); - for (ReportForJson jsonReport : jsonReports) { + for (SlowPeerJsonReport jsonReport : jsonReports) { slowNodes.add(jsonReport.getSlowNode()); } if (!slowNodes.isEmpty()) { @@ -267,35 +246,30 @@ public List getSlowNodes(int numNodes) { * @param numNodes number of nodes to return. This is to limit the * size of the generated JSON. */ - private Collection getJsonReports(int numNodes) { + private Collection getJsonReports(int numNodes) { if (allReports.isEmpty()) { return Collections.emptyList(); } - final PriorityQueue topNReports = - new PriorityQueue<>(allReports.size(), - new Comparator() { - @Override - public int compare(ReportForJson o1, ReportForJson o2) { - return Ints.compare(o1.reportingNodes.size(), - o2.reportingNodes.size()); - } - }); + final PriorityQueue topNReports = new PriorityQueue<>(allReports.size(), + (o1, o2) -> Ints.compare(o1.getSlowPeerLatencyWithReportingNodes().size(), + o2.getSlowPeerLatencyWithReportingNodes().size())); final long now = timer.monotonicNow(); - for (Map.Entry> entry : - allReports.entrySet()) { - SortedSet validReports = filterNodeReports( - entry.getValue(), now); + for (Map.Entry> entry + : allReports.entrySet()) { + SortedSet validReports = + filterNodeReports(entry.getValue(), now); if (!validReports.isEmpty()) { if (topNReports.size() < numNodes) { - topNReports.add(new ReportForJson(entry.getKey(), validReports)); - } else if (topNReports.peek().getReportingNodes().size() < - validReports.size()){ + topNReports.add(new SlowPeerJsonReport(entry.getKey(), validReports)); + } else if (topNReports.peek() != null + && topNReports.peek().getSlowPeerLatencyWithReportingNodes().size() + < validReports.size()) { // Remove the lowest element topNReports.poll(); - topNReports.add(new ReportForJson(entry.getKey(), validReports)); + topNReports.add(new SlowPeerJsonReport(entry.getKey(), validReports)); } } } @@ -306,4 +280,27 @@ public int compare(ReportForJson o1, ReportForJson o2) { long getReportValidityMs() { return reportValidityMs; } + + public synchronized void setMaxSlowPeersToReport(int maxSlowPeersToReport) { + this.maxNodesToReport = maxSlowPeersToReport; + } + + private static class LatencyWithLastReportTime { + private final Long time; + private final OutlierMetrics latency; + + LatencyWithLastReportTime(Long time, OutlierMetrics latency) { + this.time = time; + this.latency = latency; + } + + public Long getTime() { + return time; + } + + public OutlierMetrics getLatency() { + return latency; + } + } + } diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/common/IncorrectVersionException.java b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/common/IncorrectVersionException.java index 43b8c3c05d102..7a14e38f5505f 100644 --- a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/common/IncorrectVersionException.java +++ b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/common/IncorrectVersionException.java @@ -21,7 +21,6 @@ import org.apache.hadoop.classification.InterfaceAudience; import org.apache.hadoop.classification.InterfaceStability; -import org.apache.hadoop.hdfs.protocol.HdfsConstants; /** * The exception is thrown when external version does not match diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/common/blockaliasmap/impl/TextFileRegionAliasMap.java b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/common/blockaliasmap/impl/TextFileRegionAliasMap.java index 183853739aa29..e94e48ed0f7b3 100644 --- a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/common/blockaliasmap/impl/TextFileRegionAliasMap.java +++ b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/common/blockaliasmap/impl/TextFileRegionAliasMap.java @@ -26,7 +26,6 @@ import java.io.InputStreamReader; import java.io.OutputStream; import java.io.OutputStreamWriter; -import java.nio.charset.Charset; import java.util.ArrayList; import java.util.Base64; import java.util.Iterator; diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/datanode/BPOfferService.java b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/datanode/BPOfferService.java index 51a6f115ccdea..a990e1915de6f 100644 --- a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/datanode/BPOfferService.java +++ b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/datanode/BPOfferService.java @@ -40,6 +40,7 @@ import java.net.InetSocketAddress; import java.util.ArrayList; import java.util.Collection; +import java.util.HashSet; import java.util.List; import java.util.Set; import java.util.concurrent.CopyOnWriteArrayList; @@ -143,11 +144,11 @@ void writeUnlock() { void refreshNNList(String serviceId, List nnIds, ArrayList addrs, ArrayList lifelineAddrs) throws IOException { - Set oldAddrs = Sets.newHashSet(); + Set oldAddrs = new HashSet<>(); for (BPServiceActor actor : bpServices) { oldAddrs.add(actor.getNNSocketAddress()); } - Set newAddrs = Sets.newHashSet(addrs); + Set newAddrs = new HashSet<>(addrs); // Process added NNs Set addedNNs = Sets.difference(newAddrs, oldAddrs); diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/datanode/BPServiceActor.java b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/datanode/BPServiceActor.java index 838259d7f6a0b..844b67ce1a877 100755 --- a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/datanode/BPServiceActor.java +++ b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/datanode/BPServiceActor.java @@ -816,7 +816,7 @@ void register(NamespaceInfo nsInfo) throws IOException { // off disk - so update the bpRegistration object from that info DatanodeRegistration newBpRegistration = bpos.createRegistration(); - LOG.info(this + " beginning handshake with NN"); + LOG.info("{} beginning handshake with NN: {}.", this, nnAddr); while (shouldRun()) { try { @@ -826,15 +826,14 @@ void register(NamespaceInfo nsInfo) throws IOException { bpRegistration = newBpRegistration; break; } catch(EOFException e) { // namenode might have just restarted - LOG.info("Problem connecting to server: " + nnAddr + " :" - + e.getLocalizedMessage()); + LOG.info("Problem connecting to server: {} : {}.", nnAddr, e.getLocalizedMessage()); } catch(SocketTimeoutException e) { // namenode is busy - LOG.info("Problem connecting to server: " + nnAddr); + LOG.info("Problem connecting to server: {}.", nnAddr); } catch(RemoteException e) { - LOG.warn("RemoteException in register", e); + LOG.warn("RemoteException in register to server: {}.", nnAddr, e); throw e; } catch(IOException e) { - LOG.warn("Problem connecting to server: " + nnAddr); + LOG.warn("Problem connecting to server: {}.", nnAddr); } // Try again in a second sleepAndLogInterrupts(1000, "connecting to server"); @@ -844,7 +843,7 @@ void register(NamespaceInfo nsInfo) throws IOException { throw new IOException("DN shut down before block pool registered"); } - LOG.info(this + " successfully registered with NN"); + LOG.info("{} successfully registered with NN: {}.", this, nnAddr); bpos.registrationSucceeded(this, bpRegistration); // reset lease id whenever registered to NN. @@ -1345,7 +1344,8 @@ long getHeartbeatWaitTime() { } long getLifelineWaitTime() { - return nextLifelineTime - monotonicNow(); + long waitTime = nextLifelineTime - monotonicNow(); + return waitTime > 0 ? waitTime : 0; } @VisibleForTesting diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/datanode/BlockPoolManager.java b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/datanode/BlockPoolManager.java index 2ce81f593e33a..073576546c790 100644 --- a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/datanode/BlockPoolManager.java +++ b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/datanode/BlockPoolManager.java @@ -196,8 +196,8 @@ private void doRefreshNamenodes( // Step 2. Any nameservices we currently have but are no longer present // need to be removed. - toRemove = Sets.newHashSet(Sets.difference( - bpByNameserviceId.keySet(), addrMap.keySet())); + toRemove = Sets.difference( + bpByNameserviceId.keySet(), addrMap.keySet()); assert toRefresh.size() + toAdd.size() == addrMap.size() : diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/datanode/BlockReceiver.java b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/datanode/BlockReceiver.java index 9b3a899323642..77e0be6c7b32e 100644 --- a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/datanode/BlockReceiver.java +++ b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/datanode/BlockReceiver.java @@ -307,6 +307,17 @@ Replica getReplica() { return replicaInfo; } + public void releaseAnyRemainingReservedSpace() { + if (replicaInfo != null) { + if (replicaInfo.getReplicaInfo().getBytesReserved() > 0) { + LOG.warn("Block {} has not released the reserved bytes. " + + "Releasing {} bytes as part of close.", replicaInfo.getBlockId(), + replicaInfo.getReplicaInfo().getBytesReserved()); + replicaInfo.releaseAllBytesReserved(); + } + } + } + /** * close files and release volume reference. */ diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/datanode/DataXceiver.java b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/datanode/DataXceiver.java index 9ad3e7cf32613..770410230162a 100644 --- a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/datanode/DataXceiver.java +++ b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/datanode/DataXceiver.java @@ -951,6 +951,9 @@ public void writeBlock(final ExtendedBlock block, IOUtils.closeStream(mirrorIn); IOUtils.closeStream(replyOut); IOUtils.closeSocket(mirrorSock); + if (blockReceiver != null) { + blockReceiver.releaseAnyRemainingReservedSpace(); + } IOUtils.closeStream(blockReceiver); setCurrentBlockReceiver(null); } diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/datanode/LocalReplicaInPipeline.java b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/datanode/LocalReplicaInPipeline.java index 99d2fc8e04ea8..24b6bd550e7b5 100644 --- a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/datanode/LocalReplicaInPipeline.java +++ b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/datanode/LocalReplicaInPipeline.java @@ -174,6 +174,10 @@ public void releaseAllBytesReserved() { getVolume().releaseLockedMemory(bytesReserved); bytesReserved = 0; } + @Override + public void releaseReplicaInfoBytesReserved() { + bytesReserved = 0; + } @Override public void setLastChecksumAndDataLen(long dataLength, byte[] checksum) { diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/datanode/ReplicaInPipeline.java b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/datanode/ReplicaInPipeline.java index 174827b5a20eb..65da42d3a205a 100644 --- a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/datanode/ReplicaInPipeline.java +++ b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/datanode/ReplicaInPipeline.java @@ -51,6 +51,11 @@ public interface ReplicaInPipeline extends Replica { */ public void releaseAllBytesReserved(); + /** + * Release the reserved space from the ReplicaInfo. + */ + void releaseReplicaInfoBytesReserved(); + /** * store the checksum for the last chunk along with the data length * @param dataLength number of bytes on disk diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/datanode/checker/AbstractFuture.java b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/datanode/checker/AbstractFuture.java index 4855c5de85fde..1cd97800e33cb 100644 --- a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/datanode/checker/AbstractFuture.java +++ b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/datanode/checker/AbstractFuture.java @@ -29,7 +29,6 @@ import org.apache.hadoop.thirdparty.com.google.common.util.concurrent.Uninterruptibles; import org.apache.hadoop.util.Preconditions; -import edu.umd.cs.findbugs.annotations.SuppressFBWarnings; import static java.util.concurrent.atomic.AtomicReferenceFieldUpdater .newUpdater; diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/datanode/fsdataset/impl/FsDatasetAsyncDiskService.java b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/datanode/fsdataset/impl/FsDatasetAsyncDiskService.java index db4987e25ac98..7bfc24197a740 100644 --- a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/datanode/fsdataset/impl/FsDatasetAsyncDiskService.java +++ b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/datanode/fsdataset/impl/FsDatasetAsyncDiskService.java @@ -73,7 +73,6 @@ class FsDatasetAsyncDiskService { private final DataNode datanode; private final FsDatasetImpl fsdatasetImpl; - private final ThreadGroup threadGroup; private Map executors = new HashMap(); private Map> deletedBlockIds @@ -91,7 +90,6 @@ class FsDatasetAsyncDiskService { FsDatasetAsyncDiskService(DataNode datanode, FsDatasetImpl fsdatasetImpl) { this.datanode = datanode; this.fsdatasetImpl = fsdatasetImpl; - this.threadGroup = new ThreadGroup(getClass().getSimpleName()); maxNumThreadsPerVolume = datanode.getConf().getInt( DFSConfigKeys.DFS_DATANODE_FSDATASETASYNCDISK_MAX_THREADS_PER_VOLUME_KEY, DFSConfigKeys.DFS_DATANODE_FSDATASETASYNCDISK_MAX_THREADS_PER_VOLUME_DEFAULT); @@ -110,7 +108,7 @@ public Thread newThread(Runnable r) { synchronized (this) { thisIndex = counter++; } - Thread t = new Thread(threadGroup, r); + Thread t = new Thread(r); t.setName("Async disk worker #" + thisIndex + " for volume " + volume); return t; @@ -218,16 +216,20 @@ synchronized void shutdown() { } } - public void submitSyncFileRangeRequest(FsVolumeImpl volume, - final ReplicaOutputStreams streams, final long offset, final long nbytes, - final int flags) { - execute(volume, new Runnable() { - @Override - public void run() { + public void submitSyncFileRangeRequest(FsVolumeImpl volume, final ReplicaOutputStreams streams, + final long offset, final long nbytes, final int flags) { + execute(volume, () -> { + try { + streams.syncFileRangeIfPossible(offset, nbytes, flags); + } catch (NativeIOException e) { try { - streams.syncFileRangeIfPossible(offset, nbytes, flags); - } catch (NativeIOException e) { - LOG.warn("sync_file_range error", e); + LOG.warn("sync_file_range error. Volume: {}, Capacity: {}, Available space: {}, " + + "File range offset: {}, length: {}, flags: {}", volume, volume.getCapacity(), + volume.getAvailable(), offset, nbytes, flags, e); + } catch (IOException ioe) { + LOG.warn("sync_file_range error. Volume: {}, Capacity: {}, " + + "File range offset: {}, length: {}, flags: {}", volume, volume.getCapacity(), + offset, nbytes, flags, e); } } }); diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/datanode/fsdataset/impl/FsDatasetImpl.java b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/datanode/fsdataset/impl/FsDatasetImpl.java index df24f9890db04..633eeab03e9cc 100644 --- a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/datanode/fsdataset/impl/FsDatasetImpl.java +++ b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/datanode/fsdataset/impl/FsDatasetImpl.java @@ -121,7 +121,6 @@ import org.apache.hadoop.util.DiskChecker.DiskOutOfSpaceException; import org.apache.hadoop.util.Lists; import org.apache.hadoop.util.ReflectionUtils; -import org.apache.hadoop.util.Sets; import org.apache.hadoop.util.Time; import org.apache.hadoop.util.Timer; @@ -404,11 +403,7 @@ public LengthInputStream getMetaDataInputStream(ExtendedBlock b) */ private static List getInitialVolumeFailureInfos( Collection dataLocations, DataStorage storage) { - Set failedLocationSet = Sets.newHashSetWithExpectedSize( - dataLocations.size()); - for (StorageLocation sl: dataLocations) { - failedLocationSet.add(sl); - } + Set failedLocationSet = new HashSet<>(dataLocations); for (Iterator it = storage.dirIterator(); it.hasNext(); ) { Storage.StorageDirectory sd = it.next(); @@ -906,6 +901,12 @@ ReplicaInfo getReplicaInfo(String bpid, long blkid) return info; } + String getStorageUuidForLock(ExtendedBlock b) + throws ReplicaNotFoundException { + return getReplicaInfo(b.getBlockPoolId(), b.getBlockId()) + .getStorageUuid(); + } + /** * Returns handles to the block file and its metadata file */ @@ -913,7 +914,7 @@ ReplicaInfo getReplicaInfo(String bpid, long blkid) public ReplicaInputStreams getTmpInputStreams(ExtendedBlock b, long blkOffset, long metaOffset) throws IOException { try (AutoCloseDataSetLock l = lockManager.readLock(LockLevel.VOLUME, - b.getBlockPoolId(), getReplicaInfo(b).getStorageUuid())) { + b.getBlockPoolId(), getStorageUuidForLock(b))) { ReplicaInfo info = getReplicaInfo(b); FsVolumeReference ref = info.getVolume().obtainReference(); try { @@ -1379,7 +1380,7 @@ static void computeChecksum(ReplicaInfo srcReplica, File dstMeta, public ReplicaHandler append(ExtendedBlock b, long newGS, long expectedBlockLen) throws IOException { try (AutoCloseableLock lock = lockManager.writeLock(LockLevel.VOLUME, - b.getBlockPoolId(), getReplicaInfo(b).getStorageUuid())) { + b.getBlockPoolId(), getStorageUuidForLock(b))) { // If the block was successfully finalized because all packets // were successfully processed at the Datanode but the ack for // some of the packets were not received by the client. The client @@ -1562,7 +1563,7 @@ public Replica recoverClose(ExtendedBlock b, long newGS, while (true) { try { try (AutoCloseableLock lock = lockManager.writeLock(LockLevel.VOLUME, - b.getBlockPoolId(), getReplicaInfo(b).getStorageUuid())) { + b.getBlockPoolId(), getStorageUuidForLock(b))) { // check replica's state ReplicaInfo replicaInfo = recoverCheck(b, newGS, expectedBlockLen); // bump the replica's GS @@ -1665,7 +1666,7 @@ public ReplicaHandler recoverRbw( while (true) { try { try (AutoCloseableLock lock = lockManager.writeLock(LockLevel.VOLUME, - b.getBlockPoolId(), getReplicaInfo(b).getStorageUuid())) { + b.getBlockPoolId(), getStorageUuidForLock(b))) { ReplicaInfo replicaInfo = getReplicaInfo(b.getBlockPoolId(), b.getBlockId()); // check the replica's state @@ -1697,7 +1698,7 @@ private ReplicaHandler recoverRbwImpl(ReplicaInPipeline rbw, ExtendedBlock b, long newGS, long minBytesRcvd, long maxBytesRcvd) throws IOException { try (AutoCloseableLock lock = lockManager.writeLock(LockLevel.VOLUME, - b.getBlockPoolId(), getReplicaInfo(b).getStorageUuid())) { + b.getBlockPoolId(), getStorageUuidForLock(b))) { // check generation stamp long replicaGenerationStamp = rbw.getGenerationStamp(); if (replicaGenerationStamp < b.getGenerationStamp() || @@ -1759,7 +1760,7 @@ public ReplicaInPipeline convertTemporaryToRbw( final ExtendedBlock b) throws IOException { long startTimeMs = Time.monotonicNow(); try (AutoCloseableLock lock = lockManager.writeLock(LockLevel.VOLUME, - b.getBlockPoolId(), getReplicaInfo(b).getStorageUuid())) { + b.getBlockPoolId(), getStorageUuidForLock(b))) { final long blockId = b.getBlockId(); final long expectedGs = b.getGenerationStamp(); final long visible = b.getNumBytes(); @@ -1957,7 +1958,7 @@ public void finalizeBlock(ExtendedBlock b, boolean fsyncDir) ReplicaInfo finalizedReplicaInfo = null; long startTimeMs = Time.monotonicNow(); try (AutoCloseableLock lock = lockManager.writeLock(LockLevel.VOLUME, - b.getBlockPoolId(), getReplicaInfo(b).getStorageUuid())) { + b.getBlockPoolId(), getStorageUuidForLock(b))) { if (Thread.interrupted()) { // Don't allow data modifications from interrupted threads throw new IOException("Cannot finalize block from Interrupted Thread"); @@ -2016,6 +2017,9 @@ private ReplicaInfo finalizeReplica(String bpid, ReplicaInfo replicaInfo) newReplicaInfo = v.addFinalizedBlock( bpid, replicaInfo, replicaInfo, replicaInfo.getBytesReserved()); + if (replicaInfo instanceof ReplicaInPipeline) { + ((ReplicaInPipeline) replicaInfo).releaseReplicaInfoBytesReserved(); + } if (v.isTransientStorage()) { releaseLockedMemory( replicaInfo.getOriginalBytesReserved() @@ -2041,7 +2045,7 @@ private ReplicaInfo finalizeReplica(String bpid, ReplicaInfo replicaInfo) public void unfinalizeBlock(ExtendedBlock b) throws IOException { long startTimeMs = Time.monotonicNow(); try (AutoCloseableLock lock = lockManager.writeLock(LockLevel.VOLUME, - b.getBlockPoolId(), getReplicaInfo(b).getStorageUuid())) { + b.getBlockPoolId(), getStorageUuidForLock(b))) { ReplicaInfo replicaInfo = volumeMap.get(b.getBlockPoolId(), b.getLocalBlock()); if (replicaInfo != null && @@ -2992,7 +2996,7 @@ public Replica updateReplicaUnderRecovery( final long newlength) throws IOException { long startTimeMs = Time.monotonicNow(); try (AutoCloseableLock lock = lockManager.writeLock(LockLevel.VOLUME, - oldBlock.getBlockPoolId(), getReplicaInfo(oldBlock).getStorageUuid())) { + oldBlock.getBlockPoolId(), getStorageUuidForLock(oldBlock))) { //get replica final String bpid = oldBlock.getBlockPoolId(); final ReplicaInfo replica = volumeMap.get(bpid, oldBlock.getBlockId()); @@ -3522,28 +3526,32 @@ public void evictBlocks(long bytesNeeded) throws IOException { ReplicaInfo replicaInfo, newReplicaInfo; final String bpid = replicaState.getBlockPoolId(); + final FsVolumeImpl lazyPersistVolume = replicaState.getLazyPersistVolume(); - try (AutoCloseableLock lock = lockManager.writeLock(LockLevel.BLOCK_POOl, bpid)) { + try (AutoCloseableLock lock = lockManager.readLock(LockLevel.BLOCK_POOl, bpid)) { replicaInfo = getReplicaInfo(replicaState.getBlockPoolId(), replicaState.getBlockId()); Preconditions.checkState(replicaInfo.getVolume().isTransientStorage()); ramDiskReplicaTracker.discardReplica(replicaState.getBlockPoolId(), replicaState.getBlockId(), false); - // Move the replica from lazyPersist/ to finalized/ on - // the target volume - newReplicaInfo = - replicaState.getLazyPersistVolume().activateSavedReplica(bpid, - replicaInfo, replicaState); - // Update the volumeMap entry. - volumeMap.add(bpid, newReplicaInfo); - - // Update metrics - datanode.getMetrics().incrRamDiskBlocksEvicted(); - datanode.getMetrics().addRamDiskBlocksEvictionWindowMs( - Time.monotonicNow() - replicaState.getCreationTime()); - if (replicaState.getNumReads() == 0) { - datanode.getMetrics().incrRamDiskBlocksEvictedWithoutRead(); + try (AutoCloseableLock lock1 = lockManager.writeLock(LockLevel.VOLUME, + bpid, lazyPersistVolume.getStorageID())) { + // Move the replica from lazyPersist/ to finalized/ on + // the target volume + newReplicaInfo = + replicaState.getLazyPersistVolume().activateSavedReplica(bpid, + replicaInfo, replicaState); + // Update the volumeMap entry. + volumeMap.add(bpid, newReplicaInfo); + + // Update metrics + datanode.getMetrics().incrRamDiskBlocksEvicted(); + datanode.getMetrics().addRamDiskBlocksEvictionWindowMs( + Time.monotonicNow() - replicaState.getCreationTime()); + if (replicaState.getNumReads() == 0) { + datanode.getMetrics().incrRamDiskBlocksEvictedWithoutRead(); + } } // Delete the block+meta files from RAM disk and release locked diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/datanode/metrics/DataNodePeerMetrics.java b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/datanode/metrics/DataNodePeerMetrics.java index 2e456b67ca149..a77c3ba0643a2 100644 --- a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/datanode/metrics/DataNodePeerMetrics.java +++ b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/datanode/metrics/DataNodePeerMetrics.java @@ -23,6 +23,7 @@ import org.apache.hadoop.classification.InterfaceStability; import org.apache.hadoop.classification.VisibleForTesting; import org.apache.hadoop.conf.Configuration; +import org.apache.hadoop.hdfs.server.protocol.OutlierMetrics; import org.apache.hadoop.metrics2.MetricsJsonBuilder; import org.apache.hadoop.metrics2.lib.MutableRollingAverages; import org.apache.hadoop.util.Preconditions; @@ -55,7 +56,7 @@ public class DataNodePeerMetrics { private final String name; // Strictly to be used by test code only. Source code is not supposed to use this. - private Map testOutlier = null; + private Map testOutlier = null; private final OutlierDetector slowNodeDetector; @@ -143,7 +144,7 @@ public void collectThreadLocalStates() { * Retrieve the set of dataNodes that look significantly slower * than their peers. */ - public Map getOutliers() { + public Map getOutliers() { // outlier must be null for source code. if (testOutlier == null) { // This maps the metric name to the aggregate latency. @@ -151,7 +152,7 @@ public Map getOutliers() { final Map stats = sendPacketDownstreamRollingAverages.getStats(minOutlierDetectionSamples); LOG.trace("DataNodePeerMetrics: Got stats: {}", stats); - return slowNodeDetector.getOutliers(stats); + return slowNodeDetector.getOutlierMetrics(stats); } else { // this happens only for test code. return testOutlier; @@ -164,7 +165,7 @@ public Map getOutliers() { * * @param outlier outlier directly set by tests. */ - public void setTestOutliers(Map outlier) { + public void setTestOutliers(Map outlier) { this.testOutlier = outlier; } diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/datanode/metrics/OutlierDetector.java b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/datanode/metrics/OutlierDetector.java index 39feca03d665e..e13cf27517852 100644 --- a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/datanode/metrics/OutlierDetector.java +++ b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/datanode/metrics/OutlierDetector.java @@ -22,6 +22,8 @@ import org.apache.hadoop.thirdparty.com.google.common.collect.ImmutableMap; import org.apache.hadoop.classification.InterfaceAudience; import org.apache.hadoop.classification.InterfaceStability; +import org.apache.hadoop.hdfs.server.protocol.OutlierMetrics; + import org.slf4j.Logger; import org.slf4j.LoggerFactory; @@ -108,6 +110,26 @@ public OutlierDetector(long minNumResources, long lowThresholdMs) { * @return */ public Map getOutliers(Map stats) { + final Map slowResources = new HashMap<>(); + Map slowResourceMetrics = getOutlierMetrics(stats); + slowResourceMetrics.forEach( + (node, outlierMetrics) -> slowResources.put(node, outlierMetrics.getActualLatency())); + return slowResources; + } + + /** + * Return a set of nodes whose latency is much higher than + * their counterparts. The input is a map of (resource {@literal ->} aggregate + * latency) entries. + * + * The aggregate may be an arithmetic mean or a percentile e.g. + * 90th percentile. Percentiles are a better choice than median + * since latency is usually not a normal distribution. + * + * @param stats map of aggregate latency entries. + * @return map of outlier nodes to outlier metrics. + */ + public Map getOutlierMetrics(Map stats) { if (stats.size() < minNumResources) { LOG.debug("Skipping statistical outlier detection as we don't have " + "latency data for enough resources. Have {}, need at least {}", @@ -124,19 +146,20 @@ public Map getOutliers(Map stats) { upperLimitLatency = Math.max( upperLimitLatency, median + (DEVIATION_MULTIPLIER * mad)); - final Map slowResources = new HashMap<>(); + final Map slowResources = new HashMap<>(); - LOG.trace("getOutliers: List={}, MedianLatency={}, " + - "MedianAbsoluteDeviation={}, upperLimitLatency={}", - sorted, median, mad, upperLimitLatency); + LOG.trace("getOutliers: List={}, MedianLatency={}, " + + "MedianAbsoluteDeviation={}, upperLimitLatency={}", sorted, median, mad, + upperLimitLatency); // Find resources whose latency exceeds the threshold. for (Map.Entry entry : stats.entrySet()) { if (entry.getValue() > upperLimitLatency) { - slowResources.put(entry.getKey(), entry.getValue()); + OutlierMetrics outlierMetrics = + new OutlierMetrics(median, mad, upperLimitLatency, entry.getValue()); + slowResources.put(entry.getKey(), outlierMetrics); } } - return slowResources; } diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/datanode/web/DatanodeHttpServer.java b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/datanode/web/DatanodeHttpServer.java index 46656c178a9c5..bb2757287fbe0 100644 --- a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/datanode/web/DatanodeHttpServer.java +++ b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/datanode/web/DatanodeHttpServer.java @@ -17,8 +17,8 @@ */ package org.apache.hadoop.hdfs.server.datanode.web; -import io.netty.bootstrap.ChannelFactory; import io.netty.bootstrap.ServerBootstrap; +import io.netty.channel.ChannelFactory; import io.netty.channel.ChannelFuture; import io.netty.channel.ChannelHandler; import io.netty.channel.ChannelInitializer; @@ -28,6 +28,7 @@ import io.netty.channel.nio.NioEventLoopGroup; import io.netty.channel.socket.SocketChannel; import io.netty.channel.socket.nio.NioServerSocketChannel; +import io.netty.channel.WriteBufferWaterMark; import io.netty.handler.codec.http.HttpRequestDecoder; import io.netty.handler.codec.http.HttpResponseEncoder; import io.netty.handler.ssl.SslHandler; @@ -168,15 +169,13 @@ protected void initChannel(SocketChannel ch) throws Exception { }); this.httpServer.childOption( - ChannelOption.WRITE_BUFFER_HIGH_WATER_MARK, - conf.getInt( - DFSConfigKeys.DFS_WEBHDFS_NETTY_HIGH_WATERMARK, - DFSConfigKeys.DFS_WEBHDFS_NETTY_HIGH_WATERMARK_DEFAULT)); - this.httpServer.childOption( - ChannelOption.WRITE_BUFFER_LOW_WATER_MARK, - conf.getInt( - DFSConfigKeys.DFS_WEBHDFS_NETTY_LOW_WATERMARK, - DFSConfigKeys.DFS_WEBHDFS_NETTY_LOW_WATERMARK_DEFAULT)); + ChannelOption.WRITE_BUFFER_WATER_MARK, + new WriteBufferWaterMark(conf.getInt( + DFSConfigKeys.DFS_WEBHDFS_NETTY_LOW_WATERMARK, + DFSConfigKeys.DFS_WEBHDFS_NETTY_LOW_WATERMARK_DEFAULT), + conf.getInt( + DFSConfigKeys.DFS_WEBHDFS_NETTY_HIGH_WATERMARK, + DFSConfigKeys.DFS_WEBHDFS_NETTY_HIGH_WATERMARK_DEFAULT))); if (externalHttpChannel == null) { httpServer.channel(NioServerSocketChannel.class); @@ -303,18 +302,7 @@ public InetSocketAddress getHttpsAddress() { public void start() throws IOException { if (httpServer != null) { InetSocketAddress infoAddr = DataNode.getInfoAddr(conf); - ChannelFuture f = httpServer.bind(infoAddr); - try { - f.syncUninterruptibly(); - } catch (Throwable e) { - if (e instanceof BindException) { - throw NetUtils.wrapException(null, 0, infoAddr.getHostName(), - infoAddr.getPort(), (SocketException) e); - } else { - throw e; - } - } - httpAddress = (InetSocketAddress) f.channel().localAddress(); + httpAddress = getChannelLocalAddress(httpServer, infoAddr); LOG.info("Listening HTTP traffic on " + httpAddress); } @@ -323,21 +311,25 @@ public void start() throws IOException { NetUtils.createSocketAddr(conf.getTrimmed( DFS_DATANODE_HTTPS_ADDRESS_KEY, DFS_DATANODE_HTTPS_ADDRESS_DEFAULT)); - ChannelFuture f = httpsServer.bind(secInfoSocAddr); + httpsAddress = getChannelLocalAddress(httpsServer, secInfoSocAddr); + LOG.info("Listening HTTPS traffic on " + httpsAddress); + } + } - try { - f.syncUninterruptibly(); - } catch (Throwable e) { - if (e instanceof BindException) { - throw NetUtils.wrapException(null, 0, secInfoSocAddr.getHostName(), - secInfoSocAddr.getPort(), (SocketException) e); - } else { - throw e; - } + private InetSocketAddress getChannelLocalAddress( + ServerBootstrap server, InetSocketAddress address) throws IOException { + ChannelFuture f = server.bind(address); + try { + f.syncUninterruptibly(); + } catch (Throwable e) { + if (e instanceof BindException) { + throw NetUtils.wrapException(null, 0, address.getHostName(), + address.getPort(), (SocketException) e); + } else { + throw e; } - httpsAddress = (InetSocketAddress) f.channel().localAddress(); - LOG.info("Listening HTTPS traffic on " + httpsAddress); } + return (InetSocketAddress) f.channel().localAddress(); } @Override diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/datanode/web/HostRestrictingAuthorizationFilterHandler.java b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/datanode/web/HostRestrictingAuthorizationFilterHandler.java index 798def0c716b7..8c5d538d23384 100644 --- a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/datanode/web/HostRestrictingAuthorizationFilterHandler.java +++ b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/datanode/web/HostRestrictingAuthorizationFilterHandler.java @@ -197,7 +197,7 @@ public String getRemoteAddr() { @Override public String getQueryString() { try { - return (new URI(req.getUri()).getQuery()); + return (new URI(req.uri()).getQuery()); } catch (URISyntaxException e) { return null; } @@ -205,7 +205,7 @@ public String getQueryString() { @Override public String getRequestURI() { - String uri = req.getUri(); + String uri = req.uri(); // Netty's getUri includes the query string, while Servlet's does not return (uri.substring(0, uri.indexOf("?") >= 0 ? uri.indexOf("?") : uri.length())); diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/datanode/web/SimpleHttpProxyHandler.java b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/datanode/web/SimpleHttpProxyHandler.java index 9a2e0b71a4482..51aeab024b81d 100644 --- a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/datanode/web/SimpleHttpProxyHandler.java +++ b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/datanode/web/SimpleHttpProxyHandler.java @@ -98,7 +98,7 @@ public void exceptionCaught(ChannelHandlerContext ctx, Throwable cause) { @Override public void channelRead0 (final ChannelHandlerContext ctx, final HttpRequest req) { - uri = req.getUri(); + uri = req.uri(); final Channel client = ctx.channel(); Bootstrap proxiedServer = new Bootstrap() .group(client.eventLoop()) @@ -117,8 +117,7 @@ protected void initChannel(SocketChannel ch) throws Exception { public void operationComplete(ChannelFuture future) throws Exception { if (future.isSuccess()) { ctx.channel().pipeline().remove(HttpResponseEncoder.class); - HttpRequest newReq = new DefaultFullHttpRequest(HTTP_1_1, - req.getMethod(), req.getUri()); + HttpRequest newReq = new DefaultFullHttpRequest(HTTP_1_1, req.method(), req.uri()); newReq.headers().add(req.headers()); newReq.headers().set(CONNECTION, Values.CLOSE); future.channel().writeAndFlush(newReq); diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/datanode/web/URLDispatcher.java b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/datanode/web/URLDispatcher.java index 8ec5bf6f64c84..4545f733a8cf3 100644 --- a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/datanode/web/URLDispatcher.java +++ b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/datanode/web/URLDispatcher.java @@ -43,7 +43,7 @@ class URLDispatcher extends SimpleChannelInboundHandler { @Override protected void channelRead0(ChannelHandlerContext ctx, HttpRequest req) throws Exception { - String uri = req.getUri(); + String uri = req.uri(); ChannelPipeline p = ctx.pipeline(); if (uri.startsWith(WEBHDFS_PREFIX)) { WebHdfsHandler h = new WebHdfsHandler(conf, confForCreate); diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/datanode/web/webhdfs/WebHdfsHandler.java b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/datanode/web/webhdfs/WebHdfsHandler.java index f2ced88aaa30e..7567162840893 100644 --- a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/datanode/web/webhdfs/WebHdfsHandler.java +++ b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/datanode/web/webhdfs/WebHdfsHandler.java @@ -122,8 +122,8 @@ public WebHdfsHandler(Configuration conf, Configuration confForCreate) @Override public void channelRead0(final ChannelHandlerContext ctx, final HttpRequest req) throws Exception { - Preconditions.checkArgument(req.getUri().startsWith(WEBHDFS_PREFIX)); - QueryStringDecoder queryString = new QueryStringDecoder(req.getUri()); + Preconditions.checkArgument(req.uri().startsWith(WEBHDFS_PREFIX)); + QueryStringDecoder queryString = new QueryStringDecoder(req.uri()); params = new ParameterParser(queryString, conf); DataNodeUGIProvider ugiProvider = new DataNodeUGIProvider(params); ugi = ugiProvider.ugi(); @@ -144,7 +144,7 @@ public Void run() throws Exception { LOG.warn("Error retrieving hostname: ", e); host = "unknown"; } - REQLOG.info(host + " " + req.getMethod() + " " + req.getUri() + " " + + REQLOG.info(host + " " + req.method() + " " + req.uri() + " " + getResponseCode()); } return null; @@ -154,13 +154,13 @@ public Void run() throws Exception { int getResponseCode() { return (resp == null) ? INTERNAL_SERVER_ERROR.code() : - resp.getStatus().code(); + resp.status().code(); } public void handle(ChannelHandlerContext ctx, HttpRequest req) throws IOException, URISyntaxException { String op = params.op(); - HttpMethod method = req.getMethod(); + HttpMethod method = req.method(); if (PutOpParam.Op.CREATE.name().equalsIgnoreCase(op) && method == PUT) { onCreate(ctx); diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/AclEntryStatusFormat.java b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/AclEntryStatusFormat.java index a610aee48496c..a6e7d00f848d9 100644 --- a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/AclEntryStatusFormat.java +++ b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/AclEntryStatusFormat.java @@ -26,8 +26,6 @@ import org.apache.hadoop.fs.permission.FsAction; import org.apache.hadoop.hdfs.util.LongBitFormat; -import org.apache.hadoop.thirdparty.com.google.common.collect.ImmutableList; - /** * Class to pack an AclEntry into an integer.
      * An ACL entry is represented by a 32-bit integer in Big Endian format.
      diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/FSNamesystem.java b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/FSNamesystem.java index ab3c49fc2641f..13894b4fecf8c 100644 --- a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/FSNamesystem.java +++ b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/FSNamesystem.java @@ -4393,11 +4393,9 @@ HeartbeatResponse handleHeartbeat(DatanodeRegistration nodeReg, readLock(); try { //get datanode commands - final int maxTransfer = blockManager.getMaxReplicationStreams() - - xmitsInProgress; DatanodeCommand[] cmds = blockManager.getDatanodeManager().handleHeartbeat( nodeReg, reports, getBlockPoolId(), cacheCapacity, cacheUsed, - xceiverCount, maxTransfer, failedVolumes, volumeFailureSummary, + xceiverCount, xmitsInProgress, failedVolumes, volumeFailureSummary, slowPeers, slowDisks); long blockReportLeaseId = 0; if (requestFullBlockReportLease) { diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/JournalSet.java b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/JournalSet.java index 4729cd99f305d..19efc0d631120 100644 --- a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/JournalSet.java +++ b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/JournalSet.java @@ -31,6 +31,7 @@ import java.util.Map; import java.util.PriorityQueue; import java.util.SortedSet; +import java.util.TreeSet; import java.util.concurrent.CopyOnWriteArrayList; import org.slf4j.Logger; @@ -42,7 +43,6 @@ import org.apache.hadoop.hdfs.server.protocol.RemoteEditLog; import org.apache.hadoop.hdfs.server.protocol.RemoteEditLogManifest; import org.apache.hadoop.util.Lists; -import org.apache.hadoop.util.Sets; import org.apache.hadoop.classification.VisibleForTesting; import org.apache.hadoop.util.Preconditions; @@ -677,7 +677,7 @@ public synchronized RemoteEditLogManifest getEditLogManifest(long fromTxId) { // storage directory with ancient logs. Clear out any logs we've // accumulated so far, and then skip to the next segment of logs // after the gap. - SortedSet startTxIds = Sets.newTreeSet(logsByStartTxId.keySet()); + SortedSet startTxIds = new TreeSet<>(logsByStartTxId.keySet()); startTxIds = startTxIds.tailSet(curStartTxId); if (startTxIds.isEmpty()) { break; diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/NameNode.java b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/NameNode.java index f57135a7fc664..c3371eefacb04 100644 --- a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/NameNode.java +++ b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/NameNode.java @@ -124,6 +124,8 @@ import static org.apache.hadoop.fs.CommonConfigurationKeysPublic.FS_TRASH_INTERVAL_DEFAULT; import static org.apache.hadoop.fs.CommonConfigurationKeysPublic.FS_TRASH_INTERVAL_KEY; import static org.apache.hadoop.hdfs.DFSConfigKeys.DFS_BLOCK_INVALIDATE_LIMIT_KEY; +import static org.apache.hadoop.hdfs.DFSConfigKeys.DFS_DATANODE_MAX_NODES_TO_REPORT_DEFAULT; +import static org.apache.hadoop.hdfs.DFSConfigKeys.DFS_DATANODE_MAX_NODES_TO_REPORT_KEY; import static org.apache.hadoop.hdfs.DFSConfigKeys.DFS_DATANODE_PEER_STATS_ENABLED_DEFAULT; import static org.apache.hadoop.hdfs.DFSConfigKeys.DFS_DATANODE_PEER_STATS_ENABLED_KEY; import static org.apache.hadoop.hdfs.DFSConfigKeys.DFS_HA_NN_NOT_BECOME_ACTIVE_IN_SAFEMODE; @@ -344,7 +346,8 @@ public enum OperationCategory { DFS_NAMENODE_BLOCKPLACEMENTPOLICY_EXCLUDE_SLOW_NODES_ENABLED_KEY, DFS_NAMENODE_MAX_SLOWPEER_COLLECT_NODES_KEY, DFS_BLOCK_INVALIDATE_LIMIT_KEY, - DFS_DATANODE_PEER_STATS_ENABLED_KEY)); + DFS_DATANODE_PEER_STATS_ENABLED_KEY, + DFS_DATANODE_MAX_NODES_TO_REPORT_KEY)); private static final String USAGE = "Usage: hdfs namenode [" + StartupOption.BACKUP.getName() + "] | \n\t[" @@ -2216,7 +2219,8 @@ protected String reconfigurePropertyImpl(String property, String newVal) } else if (property.equals(DFS_NAMENODE_AVOID_SLOW_DATANODE_FOR_READ_KEY) || (property.equals( DFS_NAMENODE_BLOCKPLACEMENTPOLICY_EXCLUDE_SLOW_NODES_ENABLED_KEY)) || (property.equals( DFS_NAMENODE_MAX_SLOWPEER_COLLECT_NODES_KEY)) || (property.equals( - DFS_DATANODE_PEER_STATS_ENABLED_KEY))) { + DFS_DATANODE_PEER_STATS_ENABLED_KEY)) || property.equals( + DFS_DATANODE_MAX_NODES_TO_REPORT_KEY)) { return reconfigureSlowNodesParameters(datanodeManager, property, newVal); } else if (property.equals(DFS_BLOCK_INVALIDATE_LIMIT_KEY)) { return reconfigureBlockInvalidateLimit(datanodeManager, property, newVal); @@ -2450,6 +2454,13 @@ String reconfigureSlowNodesParameters(final DatanodeManager datanodeManager, datanodeManager.initSlowPeerTracker(getConf(), timer, peerStatsEnabled); break; } + case DFS_DATANODE_MAX_NODES_TO_REPORT_KEY: { + int maxSlowPeersToReport = (newVal == null + ? DFS_DATANODE_MAX_NODES_TO_REPORT_DEFAULT : Integer.parseInt(newVal)); + result = Integer.toString(maxSlowPeersToReport); + datanodeManager.setMaxSlowPeersToReport(maxSlowPeersToReport); + break; + } default: { throw new IllegalArgumentException( "Unexpected property " + property + " in reconfigureSlowNodesParameters"); diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/SerialNumberMap.java b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/SerialNumberMap.java index ee6b8eb80ce04..d9a41428b5584 100644 --- a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/SerialNumberMap.java +++ b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/SerialNumberMap.java @@ -18,7 +18,6 @@ package org.apache.hadoop.hdfs.server.namenode; import java.util.HashSet; -import java.util.Iterator; import java.util.Map; import java.util.Set; import java.util.concurrent.ConcurrentHashMap; diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/ha/RemoteNameNodeInfo.java b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/ha/RemoteNameNodeInfo.java index 2960c95c1cd95..784d19660f104 100644 --- a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/ha/RemoteNameNodeInfo.java +++ b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/ha/RemoteNameNodeInfo.java @@ -24,7 +24,6 @@ import java.util.Collections; import java.util.List; -import org.apache.hadoop.thirdparty.com.google.common.base.Objects; import org.apache.hadoop.conf.Configuration; import org.apache.hadoop.hdfs.DFSUtil; import org.apache.hadoop.hdfs.HAUtil; diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/startupprogress/package-info.java b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/startupprogress/package-info.java index e6a8b97fb3faa..1ba0b8332d183 100644 --- a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/startupprogress/package-info.java +++ b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/startupprogress/package-info.java @@ -17,7 +17,7 @@ */ /** - * This package provides a mechanism for tracking {@link NameNode} startup + * This package provides a mechanism for tracking NameNode startup * progress. The package models NameNode startup as a series of {@link Phase}s, * with each phase further sub-divided into multiple {@link Step}s. All phases * are coarse-grained and typically known in advance, implied by the structure of @@ -40,4 +40,3 @@ package org.apache.hadoop.hdfs.server.namenode.startupprogress; import org.apache.hadoop.classification.InterfaceAudience; -import org.apache.hadoop.hdfs.server.namenode.NameNode; diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/sps/metrics/ExternalSPSBeanMetrics.java b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/sps/metrics/ExternalSPSBeanMetrics.java index 75546386f9da3..adab0e40328a9 100644 --- a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/sps/metrics/ExternalSPSBeanMetrics.java +++ b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/sps/metrics/ExternalSPSBeanMetrics.java @@ -95,6 +95,7 @@ public int getAttemptedItemsCount() { @VisibleForTesting public void updateAttemptedItemsCount() { storagePolicySatisfier.getAttemptedItemsMonitor().getStorageMovementAttemptedItems() - .add(new StoragePolicySatisfier.AttemptedItemInfo(0, 1, 1, new HashSet<>(), 1)); + .add(new StoragePolicySatisfier.AttemptedItemInfo(0, 1, + 1, new HashSet<>(), 1)); } } diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/tools/DFSAdmin.java b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/tools/DFSAdmin.java index d435bb6da3ebf..1d3e8da77a3a9 100644 --- a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/tools/DFSAdmin.java +++ b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/tools/DFSAdmin.java @@ -35,7 +35,6 @@ import java.util.List; import java.util.Map; import java.util.Optional; -import java.util.TreeSet; import java.util.concurrent.ExecutorService; import java.util.concurrent.Executors; import java.util.concurrent.TimeUnit; @@ -904,7 +903,7 @@ public int saveNamespace(String[] argv) throws IOException { + proxy.getAddress()); } }catch (IOException ioe){ - System.out.println("Save namespace failed for " + + System.err.println("Save namespace failed for " + proxy.getAddress()); exceptions.add(ioe); } @@ -961,7 +960,7 @@ public int restoreFailedStorage(String arg) throws IOException { System.out.println("restoreFailedStorage is set to " + res + " for " + proxy.getAddress()); } catch (IOException ioe){ - System.out.println("restoreFailedStorage failed for " + System.err.println("restoreFailedStorage failed for " + proxy.getAddress()); exceptions.add(ioe); } @@ -1004,7 +1003,7 @@ public int refreshNodes() throws IOException { System.out.println("Refresh nodes successful for " + proxy.getAddress()); }catch (IOException ioe){ - System.out.println("Refresh nodes failed for " + + System.err.println("Refresh nodes failed for " + proxy.getAddress()); exceptions.add(ioe); } @@ -1060,7 +1059,7 @@ public int listOpenFiles(String[] argv) throws IOException { openFilesRemoteIterator = dfs.listOpenFiles(openFilesTypes, path); printOpenFiles(openFilesRemoteIterator); } catch (IOException ioe){ - System.out.println("List open files failed."); + System.err.println("List open files failed."); throw ioe; } return 0; @@ -1494,7 +1493,7 @@ public int finalizeUpgrade() throws IOException { System.out.println("Finalize upgrade successful for " + proxy.getAddress()); }catch (IOException ioe){ - System.out.println("Finalize upgrade failed for " + + System.err.println("Finalize upgrade failed for " + proxy.getAddress()); exceptions.add(ioe); } @@ -1623,7 +1622,7 @@ public int metaSave(String[] argv, int idx) throws IOException { throw re; } } catch (IOException ioe) { - System.out.println("Created metasave file " + pathname + System.err.println("Created metasave file " + pathname + " in the log directory of namenode " + proxy.getAddress() + " failed"); exceptions.add(ioe); @@ -1648,40 +1647,45 @@ public int metaSave(String[] argv, int idx) throws IOException { * @throws IOException If an error while getting datanode report */ public int printTopology() throws IOException { - DistributedFileSystem dfs = getDFS(); - final DatanodeInfo[] report = dfs.getDataNodeStats(); - - // Build a map of rack -> nodes from the datanode report - HashMap > tree = new HashMap>(); - for(DatanodeInfo dni : report) { - String location = dni.getNetworkLocation(); - String name = dni.getName(); - - if(!tree.containsKey(location)) { - tree.put(location, new TreeSet()); - } + DistributedFileSystem dfs = getDFS(); + final DatanodeInfo[] report = dfs.getDataNodeStats(); + + // Build a map of rack -> nodes from the datanode report + Map> map = new HashMap<>(); + for(DatanodeInfo dni : report) { + String location = dni.getNetworkLocation(); + String name = dni.getName(); + String dnState = dni.getAdminState().toString(); - tree.get(location).add(name); + if(!map.containsKey(location)) { + map.put(location, new HashMap<>()); } + + Map node = map.get(location); + node.put(name, dnState); + } - // Sort the racks (and nodes) alphabetically, display in order - ArrayList racks = new ArrayList(tree.keySet()); - Collections.sort(racks); + // Sort the racks (and nodes) alphabetically, display in order + List racks = new ArrayList<>(map.keySet()); + Collections.sort(racks); - for(String r : racks) { - System.out.println("Rack: " + r); - TreeSet nodes = tree.get(r); - - for(String n : nodes) { - System.out.print(" " + n); - String hostname = NetUtils.getHostNameOfIP(n); - if(hostname != null) - System.out.print(" (" + hostname + ")"); - System.out.println(); + for(String r : racks) { + System.out.println("Rack: " + r); + Map nodes = map.get(r); + + for(Map.Entry entry : nodes.entrySet()) { + String n = entry.getKey(); + System.out.print(" " + n); + String hostname = NetUtils.getHostNameOfIP(n); + if(hostname != null) { + System.out.print(" (" + hostname + ")"); } - + System.out.print(" " + entry.getValue()); System.out.println(); } + + System.out.println(); + } return 0; } @@ -1722,7 +1726,7 @@ public int refreshServiceAcl() throws IOException { System.out.println("Refresh service acl successful for " + proxy.getAddress()); }catch (IOException ioe){ - System.out.println("Refresh service acl failed for " + System.err.println("Refresh service acl failed for " + proxy.getAddress()); exceptions.add(ioe); } @@ -1775,7 +1779,7 @@ public int refreshUserToGroupsMappings() throws IOException { System.out.println("Refresh user to groups mapping successful for " + proxy.getAddress()); }catch (IOException ioe){ - System.out.println("Refresh user to groups mapping failed for " + System.err.println("Refresh user to groups mapping failed for " + proxy.getAddress()); exceptions.add(ioe); } @@ -1830,7 +1834,7 @@ public int refreshSuperUserGroupsConfiguration() throws IOException { System.out.println("Refresh super user groups configuration " + "successful for " + proxy.getAddress()); }catch (IOException ioe){ - System.out.println("Refresh super user groups configuration " + + System.err.println("Refresh super user groups configuration " + "failed for " + proxy.getAddress()); exceptions.add(ioe); } @@ -1879,7 +1883,7 @@ public int refreshCallQueue() throws IOException { System.out.println("Refresh call queue successful for " + proxy.getAddress()); }catch (IOException ioe){ - System.out.println("Refresh call queue failed for " + System.err.println("Refresh call queue failed for " + proxy.getAddress()); exceptions.add(ioe); } diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/tools/DFSck.java b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/tools/DFSck.java index db30133d0c028..acb370e421a30 100644 --- a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/tools/DFSck.java +++ b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/tools/DFSck.java @@ -27,6 +27,7 @@ import java.net.URLConnection; import java.net.URLEncoder; import java.security.PrivilegedExceptionAction; +import java.util.concurrent.TimeUnit; import org.apache.hadoop.classification.InterfaceAudience; import org.apache.hadoop.conf.Configuration; @@ -37,6 +38,7 @@ import org.apache.hadoop.hdfs.DistributedFileSystem; import org.apache.hadoop.hdfs.HAUtil; import org.apache.hadoop.hdfs.HdfsConfiguration; +import org.apache.hadoop.hdfs.client.HdfsClientConfigKeys; import org.apache.hadoop.hdfs.server.namenode.NamenodeFsck; import org.apache.hadoop.hdfs.web.URLConnectionFactory; import org.apache.hadoop.security.UserGroupInformation; @@ -137,8 +139,17 @@ public DFSck(Configuration conf, PrintStream out) throws IOException { super(conf); this.ugi = UserGroupInformation.getCurrentUser(); this.out = out; + int connectTimeout = (int) conf.getTimeDuration( + HdfsClientConfigKeys.DFS_CLIENT_FSCK_CONNECT_TIMEOUT, + HdfsClientConfigKeys.DFS_CLIENT_FSCK_CONNECT_TIMEOUT_DEFAULT, + TimeUnit.MILLISECONDS); + int readTimeout = (int) conf.getTimeDuration( + HdfsClientConfigKeys.DFS_CLIENT_FSCK_READ_TIMEOUT, + HdfsClientConfigKeys.DFS_CLIENT_FSCK_READ_TIMEOUT_DEFAULT, + TimeUnit.MILLISECONDS); + this.connectionFactory = URLConnectionFactory - .newDefaultURLConnectionFactory(conf); + .newDefaultURLConnectionFactory(connectTimeout, readTimeout, conf); this.isSpnegoEnabled = UserGroupInformation.isSecurityEnabled(); } diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/tools/offlineImageViewer/FSImageHandler.java b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/tools/offlineImageViewer/FSImageHandler.java index f14ee5f930fe0..bfa35aaf1e8be 100644 --- a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/tools/offlineImageViewer/FSImageHandler.java +++ b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/tools/offlineImageViewer/FSImageHandler.java @@ -76,7 +76,7 @@ public void channelActive(ChannelHandlerContext ctx) throws Exception { @Override public void channelRead0(ChannelHandlerContext ctx, HttpRequest request) throws Exception { - if (request.getMethod() != HttpMethod.GET) { + if (request.method() != HttpMethod.GET) { DefaultHttpResponse resp = new DefaultHttpResponse(HTTP_1_1, METHOD_NOT_ALLOWED); resp.headers().set(CONNECTION, CLOSE); @@ -84,7 +84,7 @@ public void channelRead0(ChannelHandlerContext ctx, HttpRequest request) return; } - QueryStringDecoder decoder = new QueryStringDecoder(request.getUri()); + QueryStringDecoder decoder = new QueryStringDecoder(request.uri()); // check path. throw exception if path doesn't start with WEBHDFS_PREFIX String path = getPath(decoder); final String op = getOp(decoder); @@ -140,7 +140,7 @@ public void channelReadComplete(ChannelHandlerContext ctx) throws Exception { public void exceptionCaught(ChannelHandlerContext ctx, Throwable cause) throws Exception { Exception e = cause instanceof Exception ? (Exception) cause : new - Exception(cause); + Exception(cause); final String output = JsonUtil.toJsonString(e); ByteBuf content = Unpooled.wrappedBuffer(output.getBytes(Charsets.UTF_8)); final DefaultFullHttpResponse resp = new DefaultFullHttpResponse( diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/tools/snapshot/LsSnapshottableDir.java b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/tools/snapshot/LsSnapshottableDir.java index 2f030b4734ce9..640065bb5c663 100644 --- a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/tools/snapshot/LsSnapshottableDir.java +++ b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/tools/snapshot/LsSnapshottableDir.java @@ -20,7 +20,6 @@ import java.io.IOException; import org.apache.hadoop.classification.InterfaceAudience; -import org.apache.hadoop.conf.Configuration; import org.apache.hadoop.conf.Configured; import org.apache.hadoop.fs.FileSystem; import org.apache.hadoop.hdfs.DistributedFileSystem; diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/main/proto/DatanodeProtocol.proto b/hadoop-hdfs-project/hadoop-hdfs/src/main/proto/DatanodeProtocol.proto index 48a3855c0381e..c537ce3ae494c 100644 --- a/hadoop-hdfs-project/hadoop-hdfs/src/main/proto/DatanodeProtocol.proto +++ b/hadoop-hdfs-project/hadoop-hdfs/src/main/proto/DatanodeProtocol.proto @@ -408,6 +408,9 @@ message CommitBlockSynchronizationResponseProto { message SlowPeerReportProto { optional string dataNodeId = 1; optional double aggregateLatency = 2; + optional double median = 3; + optional double mad = 4; + optional double upperLimitLatency = 5; } /** diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/main/resources/hdfs-default.xml b/hadoop-hdfs-project/hadoop-hdfs/src/main/resources/hdfs-default.xml index bffde8cf7482a..33ffd07c8de2b 100755 --- a/hadoop-hdfs-project/hadoop-hdfs/src/main/resources/hdfs-default.xml +++ b/hadoop-hdfs-project/hadoop-hdfs/src/main/resources/hdfs-default.xml @@ -6253,6 +6253,15 @@ + + dfs.journalnode.handler.count + 5 + + The number of JournalNode RPC server threads that listen to + requests from clients. + + + dfs.namenode.lease-hard-limit-sec 1200 @@ -6420,4 +6429,21 @@ problem. In produce default set false, because it's have little performance loss. + + dfs.client.fsck.connect.timeout + 60000ms + + The amount of time the fsck client will wait to connect to the namenode + before timing out. + + + + dfs.client.fsck.read.timeout + 60000ms + + The amount of time the fsck client will wait to read from the namenode + before timing out. If the namenode does not report progress more + frequently than this time, the client will give up waiting. + + diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/TestGenericRefresh.java b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/TestGenericRefresh.java index dcd91c7d848bb..70f9af9841708 100644 --- a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/TestGenericRefresh.java +++ b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/TestGenericRefresh.java @@ -19,9 +19,7 @@ package org.apache.hadoop; import static org.junit.Assert.assertEquals; -import static org.junit.Assert.assertFalse; import static org.junit.Assert.assertTrue; -import static org.junit.Assert.fail; import org.apache.hadoop.conf.Configuration; import org.apache.hadoop.fs.FileSystem; diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/fs/TestResolveHdfsSymlink.java b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/fs/TestResolveHdfsSymlink.java index 05060af4a90d8..aefb4835b14d4 100644 --- a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/fs/TestResolveHdfsSymlink.java +++ b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/fs/TestResolveHdfsSymlink.java @@ -37,7 +37,6 @@ import org.apache.hadoop.security.UserGroupInformation; import org.apache.hadoop.security.token.Token; import org.apache.hadoop.security.token.delegation.AbstractDelegationTokenIdentifier; -import org.apache.hadoop.test.PathUtils; import org.apache.hadoop.test.GenericTestUtils; import org.junit.AfterClass; import org.junit.Assert; diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/fs/TestSWebHdfsFileContextMainOperations.java b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/fs/TestSWebHdfsFileContextMainOperations.java index 84bd98bf67c78..061858c3d2863 100644 --- a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/fs/TestSWebHdfsFileContextMainOperations.java +++ b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/fs/TestSWebHdfsFileContextMainOperations.java @@ -17,7 +17,6 @@ */ package org.apache.hadoop.fs; -import org.apache.hadoop.conf.Configuration; import org.apache.hadoop.hdfs.DFSConfigKeys; import org.apache.hadoop.hdfs.HdfsConfiguration; import org.apache.hadoop.hdfs.MiniDFSCluster; diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/fs/TestSymlinkHdfsFileSystem.java b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/fs/TestSymlinkHdfsFileSystem.java index fba9c42858d61..cd0155389c001 100644 --- a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/fs/TestSymlinkHdfsFileSystem.java +++ b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/fs/TestSymlinkHdfsFileSystem.java @@ -17,7 +17,6 @@ */ package org.apache.hadoop.fs; -import static org.junit.Assert.assertEquals; import static org.junit.Assert.assertTrue; import java.io.IOException; diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/MiniDFSCluster.java b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/MiniDFSCluster.java index d8d633f2c861e..484958e3c302c 100644 --- a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/MiniDFSCluster.java +++ b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/MiniDFSCluster.java @@ -71,6 +71,7 @@ import java.util.ArrayList; import java.util.Arrays; import java.util.Collection; +import java.util.HashSet; import java.util.List; import java.util.Map; import java.util.Set; @@ -87,7 +88,6 @@ import org.apache.hadoop.http.HttpConfig; import org.apache.hadoop.security.ssl.KeyStoreTestUtil; import org.apache.hadoop.util.Lists; -import org.apache.hadoop.util.Sets; import org.slf4j.Logger; import org.slf4j.LoggerFactory; import org.apache.hadoop.classification.InterfaceAudience; @@ -653,7 +653,7 @@ public DataNode getDatanode() { private boolean federation; private boolean checkExitOnShutdown = true; protected final int storagesPerDatanode; - private Set fileSystems = Sets.newHashSet(); + private Set fileSystems = new HashSet<>(); private List storageCap = Lists.newLinkedList(); diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/TestAbandonBlock.java b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/TestAbandonBlock.java index e7d8b38aed9ce..02298165402e9 100644 --- a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/TestAbandonBlock.java +++ b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/TestAbandonBlock.java @@ -68,7 +68,7 @@ public void tearDown() throws Exception { } @Test - /** Abandon a block while creating a file */ + /* Abandon a block while creating a file */ public void testAbandonBlock() throws IOException { String src = FILE_NAME_PREFIX + "foo"; @@ -105,7 +105,7 @@ public void testAbandonBlock() throws IOException { } @Test - /** Make sure that the quota is decremented correctly when a block is abandoned */ + /* Make sure that the quota is decremented correctly when a block is abandoned */ public void testQuotaUpdatedWhenBlockAbandoned() throws IOException { // Setting diskspace quota to 3MB fs.setQuota(new Path("/"), HdfsConstants.QUOTA_DONT_SET, 3 * 1024 * 1024); diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/TestClientProtocolForPipelineRecovery.java b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/TestClientProtocolForPipelineRecovery.java index 3138c23bac5ee..70aa9d7d635a4 100644 --- a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/TestClientProtocolForPipelineRecovery.java +++ b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/TestClientProtocolForPipelineRecovery.java @@ -448,7 +448,7 @@ public void testPipelineRecoveryOnRestartFailure() throws Exception { * rolling upgrades. The client should be able to retry pipeline recovery * more times than the default. * (in a row for the same packet, including the heartbeat packet) - * (See{@link DataStreamer#pipelineRecoveryCount}) + * (See{@link DataStreamer#getPipelineRecoveryCount}) */ @Test(timeout = 60000) public void testPipelineRecoveryOnDatanodeUpgrade() throws Exception { diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/TestDFSShell.java b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/TestDFSShell.java index 44c06e39ea904..a00f21ecc942d 100644 --- a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/TestDFSShell.java +++ b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/TestDFSShell.java @@ -1410,7 +1410,7 @@ private void confirmOwner(String owner, String group, } } - @Test (timeout = 30000) + @Test (timeout = 60000) public void testFilePermissions() throws IOException { Configuration conf = new HdfsConfiguration(); diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/TestDFSUtil.java b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/TestDFSUtil.java index 9a024c3084586..e6ce29316c5b6 100644 --- a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/TestDFSUtil.java +++ b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/TestDFSUtil.java @@ -53,6 +53,7 @@ import java.net.URISyntaxException; import java.util.Arrays; import java.util.Collection; +import java.util.HashSet; import java.util.Iterator; import java.util.List; import java.util.Map; @@ -80,7 +81,6 @@ import org.apache.hadoop.security.alias.JavaKeyStoreProvider; import org.apache.hadoop.test.GenericTestUtils; import org.apache.hadoop.test.LambdaTestUtils; -import org.apache.hadoop.util.Sets; import org.junit.Assert; import org.junit.Before; import org.junit.Test; @@ -1042,10 +1042,10 @@ public void testGetNNServiceRpcAddressesForNsIds() throws IOException { { Collection internal = DFSUtil.getInternalNameServices(conf); - assertEquals(Sets.newHashSet("nn1"), internal); + assertEquals(new HashSet<>(Arrays.asList("nn1")), internal); Collection all = DFSUtilClient.getNameServiceIds(conf); - assertEquals(Sets.newHashSet("nn1", "nn2"), all); + assertEquals(new HashSet<>(Arrays.asList("nn1", "nn2")), all); } Map> nnMap = DFSUtil diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/TestDecommission.java b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/TestDecommission.java index 670ca5fd9a6fe..0133d3aec37b1 100644 --- a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/TestDecommission.java +++ b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/TestDecommission.java @@ -52,6 +52,7 @@ import org.apache.hadoop.fs.FileSystem; import org.apache.hadoop.fs.Path; import org.apache.hadoop.fs.BatchedRemoteIterator.BatchedEntries; +import org.apache.hadoop.hdfs.client.HdfsClientConfigKeys; import org.apache.hadoop.hdfs.client.HdfsDataInputStream; import org.apache.hadoop.hdfs.client.HdfsDataOutputStream; import org.apache.hadoop.hdfs.protocol.Block; @@ -1902,4 +1903,285 @@ private void createClusterWithDeadNodesDecommissionInProgress(final int numLiveN !BlockManagerTestUtil.isNodeHealthyForDecommissionOrMaintenance(blockManager, node) && !node.isAlive()), 500, 20000); } + + /* + This test reproduces a scenario where an under-replicated block on a decommissioning node + cannot be replicated to some datanodes because they have a corrupt replica of the block. + The test ensures that the corrupt replicas are eventually invalidated so that the + under-replicated block can be replicated to sufficient datanodes & the decommissioning + node can be decommissioned. + */ + @Test(timeout = 60000) + public void testDeleteCorruptReplicaForUnderReplicatedBlock() throws Exception { + // Constants + final Path file = new Path("/test-file"); + final int numDatanode = 3; + final short replicationFactor = 2; + final int numStoppedNodes = 2; + final int numDecommNodes = 1; + assertEquals(numDatanode, numStoppedNodes + numDecommNodes); + + // Run monitor every 5 seconds to speed up decommissioning & make the test faster + final int datanodeAdminMonitorFixedRateSeconds = 5; + getConf().setInt(MiniDFSCluster.DFS_NAMENODE_DECOMMISSION_INTERVAL_TESTING_KEY, + datanodeAdminMonitorFixedRateSeconds); + // Set block report interval to 6 hours to avoid unexpected block reports. + // The default block report interval is different for a MiniDFSCluster + getConf().setLong(DFSConfigKeys.DFS_BLOCKREPORT_INTERVAL_MSEC_KEY, + DFSConfigKeys.DFS_BLOCKREPORT_INTERVAL_MSEC_DEFAULT); + // Run the BlockManager RedundancyMonitor every 3 seconds such that the Namenode + // sends under-replication blocks for replication frequently + getConf().setLong(DFSConfigKeys.DFS_NAMENODE_REDUNDANCY_INTERVAL_SECONDS_KEY, + DFSConfigKeys.DFS_NAMENODE_REDUNDANCY_INTERVAL_SECONDS_DEFAULT); + // Ensure that the DataStreamer client will replace the bad datanode on append failure + getConf().set(HdfsClientConfigKeys.BlockWrite.ReplaceDatanodeOnFailure.POLICY_KEY, "ALWAYS"); + // Avoid having the DataStreamer client fail the append operation if datanode replacement fails + getConf() + .setBoolean(HdfsClientConfigKeys.BlockWrite.ReplaceDatanodeOnFailure.BEST_EFFORT_KEY, true); + + // References to datanodes in the cluster + // - 2 datanode will be stopped to generate corrupt block replicas & then + // restarted later to validate the corrupt replicas are invalidated + // - 1 datanode will start decommissioning to make the block under replicated + final List allNodes = new ArrayList<>(); + final List stoppedNodes = new ArrayList<>(); + final DatanodeDescriptor decommNode; + + // Create MiniDFSCluster + startCluster(1, numDatanode); + getCluster().waitActive(); + final FSNamesystem namesystem = getCluster().getNamesystem(); + final BlockManager blockManager = namesystem.getBlockManager(); + final DatanodeManager datanodeManager = blockManager.getDatanodeManager(); + final DatanodeAdminManager decomManager = datanodeManager.getDatanodeAdminManager(); + final FileSystem fs = getCluster().getFileSystem(); + + // Get DatanodeDescriptors + for (final DataNode node : getCluster().getDataNodes()) { + allNodes.add(getDatanodeDesriptor(namesystem, node.getDatanodeUuid())); + } + + // Create block with 2 FINALIZED replicas + // Note that: + // - calling hflush leaves block in state ReplicaBeingWritten + // - calling close leaves the block in state FINALIZED + // - amount of data is kept small because flush is not synchronous + LOG.info("Creating Initial Block with {} FINALIZED replicas", replicationFactor); + FSDataOutputStream out = fs.create(file, replicationFactor); + for (int i = 0; i < 512; i++) { + out.write(i); + } + out.close(); + + // Validate the block exists with expected number of replicas + assertEquals(1, blockManager.getTotalBlocks()); + BlockLocation[] blocksInFile = fs.getFileBlockLocations(file, 0, 0); + assertEquals(1, blocksInFile.length); + List replicasInBlock = Arrays.asList(blocksInFile[0].getNames()); + assertEquals(replicationFactor, replicasInBlock.size()); + + // Identify the DatanodeDescriptors associated with the 2 nodes with replicas. + // Each of nodes with a replica will be stopped later to corrupt the replica + DatanodeDescriptor decommNodeTmp = null; + for (DatanodeDescriptor node : allNodes) { + if (replicasInBlock.contains(node.getName())) { + stoppedNodes.add(node); + } else { + decommNodeTmp = node; + } + } + assertEquals(numStoppedNodes, stoppedNodes.size()); + assertNotNull(decommNodeTmp); + decommNode = decommNodeTmp; + final DatanodeDescriptor firstStoppedNode = stoppedNodes.get(0); + final DatanodeDescriptor secondStoppedNode = stoppedNodes.get(1); + LOG.info("Detected 2 nodes with replicas : {} , {}", firstStoppedNode.getXferAddr(), + secondStoppedNode.getXferAddr()); + LOG.info("Detected 1 node without replica : {}", decommNode.getXferAddr()); + + // Stop firstStoppedNode & the append to the block pipeline such that DataStreamer client: + // - detects firstStoppedNode as bad link in block pipeline + // - replaces the firstStoppedNode with decommNode in block pipeline + // The result is that: + // - secondStoppedNode & decommNode have a live block replica + // - firstStoppedNode has a corrupt replica (corrupt because of old GenStamp) + LOG.info("Stopping first node with replica {}", firstStoppedNode.getXferAddr()); + final List stoppedNodeProps = new ArrayList<>(); + MiniDFSCluster.DataNodeProperties stoppedNodeProp = + getCluster().stopDataNode(firstStoppedNode.getXferAddr()); + stoppedNodeProps.add(stoppedNodeProp); + firstStoppedNode.setLastUpdate(213); // Set last heartbeat to be in the past + // Wait for NN to detect the datanode as dead + GenericTestUtils.waitFor( + () -> 2 == datanodeManager.getNumLiveDataNodes() && 1 == datanodeManager + .getNumDeadDataNodes(), 500, 30000); + // Append to block pipeline + appendBlock(fs, file, 2); + + // Stop secondStoppedNode & the append to the block pipeline such that DataStreamer client: + // - detects secondStoppedNode as bad link in block pipeline + // - attempts to replace secondStoppedNode but cannot because there are no more live nodes + // - appends to the block pipeline containing just decommNode + // The result is that: + // - decommNode has a live block replica + // - firstStoppedNode & secondStoppedNode both have a corrupt replica + LOG.info("Stopping second node with replica {}", secondStoppedNode.getXferAddr()); + stoppedNodeProp = getCluster().stopDataNode(secondStoppedNode.getXferAddr()); + stoppedNodeProps.add(stoppedNodeProp); + secondStoppedNode.setLastUpdate(213); // Set last heartbeat to be in the past + // Wait for NN to detect the datanode as dead + GenericTestUtils.waitFor(() -> numDecommNodes == datanodeManager.getNumLiveDataNodes() + && numStoppedNodes == datanodeManager.getNumDeadDataNodes(), 500, 30000); + // Append to block pipeline + appendBlock(fs, file, 1); + + // Validate block replica locations + blocksInFile = fs.getFileBlockLocations(file, 0, 0); + assertEquals(1, blocksInFile.length); + replicasInBlock = Arrays.asList(blocksInFile[0].getNames()); + assertEquals(numDecommNodes, replicasInBlock.size()); + assertTrue(replicasInBlock.contains(decommNode.getName())); + LOG.info("Block now has 2 corrupt replicas on [{} , {}] and 1 live replica on {}", + firstStoppedNode.getXferAddr(), secondStoppedNode.getXferAddr(), decommNode.getXferAddr()); + + LOG.info("Decommission node {} with the live replica", decommNode.getXferAddr()); + final ArrayList decommissionedNodes = new ArrayList<>(); + takeNodeOutofService(0, decommNode.getDatanodeUuid(), 0, decommissionedNodes, + AdminStates.DECOMMISSION_INPROGRESS); + + // Wait for the datanode to start decommissioning + try { + GenericTestUtils.waitFor(() -> decomManager.getNumTrackedNodes() == 0 + && decomManager.getNumPendingNodes() == numDecommNodes && decommNode.getAdminState() + .equals(AdminStates.DECOMMISSION_INPROGRESS), 500, 30000); + } catch (Exception e) { + blocksInFile = fs.getFileBlockLocations(file, 0, 0); + assertEquals(1, blocksInFile.length); + replicasInBlock = Arrays.asList(blocksInFile[0].getNames()); + String errMsg = String.format("Node %s failed to start decommissioning." + + " numTrackedNodes=%d , numPendingNodes=%d , adminState=%s , nodesWithReplica=[%s]", + decommNode.getXferAddr(), decomManager.getNumTrackedNodes(), + decomManager.getNumPendingNodes(), decommNode.getAdminState(), + String.join(", ", replicasInBlock)); + LOG.error(errMsg); // Do not log generic timeout exception + fail(errMsg); + } + + // Validate block replica locations + blocksInFile = fs.getFileBlockLocations(file, 0, 0); + assertEquals(1, blocksInFile.length); + replicasInBlock = Arrays.asList(blocksInFile[0].getNames()); + assertEquals(numDecommNodes, replicasInBlock.size()); + assertEquals(replicasInBlock.get(0), decommNode.getName()); + LOG.info("Block now has 2 corrupt replicas on [{} , {}] and 1 decommissioning replica on {}", + firstStoppedNode.getXferAddr(), secondStoppedNode.getXferAddr(), decommNode.getXferAddr()); + + // Restart the 2 stopped datanodes + LOG.info("Restarting stopped nodes {} , {}", firstStoppedNode.getXferAddr(), + secondStoppedNode.getXferAddr()); + for (final MiniDFSCluster.DataNodeProperties stoppedNode : stoppedNodeProps) { + assertTrue(getCluster().restartDataNode(stoppedNode)); + } + for (final MiniDFSCluster.DataNodeProperties stoppedNode : stoppedNodeProps) { + try { + getCluster().waitDatanodeFullyStarted(stoppedNode.getDatanode(), 30000); + LOG.info("Node {} Restarted", stoppedNode.getDatanode().getXferAddress()); + } catch (Exception e) { + String errMsg = String.format("Node %s Failed to Restart within 30 seconds", + stoppedNode.getDatanode().getXferAddress()); + LOG.error(errMsg); // Do not log generic timeout exception + fail(errMsg); + } + } + + // Trigger block reports for the 2 restarted nodes to ensure their corrupt + // block replicas are identified by the namenode + for (MiniDFSCluster.DataNodeProperties dnProps : stoppedNodeProps) { + DataNodeTestUtils.triggerBlockReport(dnProps.getDatanode()); + } + + // Validate the datanode is eventually decommissioned + // Some changes are needed to ensure replication/decommissioning occur in a timely manner: + // - if the namenode sends a DNA_TRANSFER before sending the DNA_INVALIDATE's then: + // - the block will enter the pendingReconstruction queue + // - this prevent the block from being sent for transfer again for some time + // - solution is to call "clearQueues" so that DNA_TRANSFER is sent again after DNA_INVALIDATE + // - need to run the check less frequently than DatanodeAdminMonitor + // such that in between "clearQueues" calls 2 things can occur: + // - DatanodeAdminMonitor runs which sets the block as neededReplication + // - datanode heartbeat is received which sends the DNA_TRANSFER to the node + final int checkEveryMillis = datanodeAdminMonitorFixedRateSeconds * 2 * 1000; + try { + GenericTestUtils.waitFor(() -> { + blockManager.clearQueues(); // Clear pendingReconstruction queue + return decomManager.getNumTrackedNodes() == 0 && decomManager.getNumPendingNodes() == 0 + && decommNode.getAdminState().equals(AdminStates.DECOMMISSIONED); + }, checkEveryMillis, 40000); + } catch (Exception e) { + blocksInFile = fs.getFileBlockLocations(file, 0, 0); + assertEquals(1, blocksInFile.length); + replicasInBlock = Arrays.asList(blocksInFile[0].getNames()); + String errMsg = String.format("Node %s failed to complete decommissioning." + + " numTrackedNodes=%d , numPendingNodes=%d , adminState=%s , nodesWithReplica=[%s]", + decommNode.getXferAddr(), decomManager.getNumTrackedNodes(), + decomManager.getNumPendingNodes(), decommNode.getAdminState(), + String.join(", ", replicasInBlock)); + LOG.error(errMsg); // Do not log generic timeout exception + fail(errMsg); + } + + // Validate block replica locations. + // Note that in order for decommissioning to complete the block must be + // replicated to both of the restarted datanodes; this implies that the + // corrupt replicas were invalidated on both of the restarted datanodes. + blocksInFile = fs.getFileBlockLocations(file, 0, 0); + assertEquals(1, blocksInFile.length); + replicasInBlock = Arrays.asList(blocksInFile[0].getNames()); + assertEquals(numDatanode, replicasInBlock.size()); + assertTrue(replicasInBlock.contains(decommNode.getName())); + for (final DatanodeDescriptor node : stoppedNodes) { + assertTrue(replicasInBlock.contains(node.getName())); + } + LOG.info("Block now has 2 live replicas on [{} , {}] and 1 decommissioned replica on {}", + firstStoppedNode.getXferAddr(), secondStoppedNode.getXferAddr(), decommNode.getXferAddr()); + } + + void appendBlock(final FileSystem fs, final Path file, int expectedReplicas) throws IOException { + LOG.info("Appending to the block pipeline"); + boolean failed = false; + Exception failedReason = null; + try { + FSDataOutputStream out = fs.append(file); + for (int i = 0; i < 512; i++) { + out.write(i); + } + out.close(); + } catch (Exception e) { + failed = true; + failedReason = e; + } finally { + BlockLocation[] blocksInFile = fs.getFileBlockLocations(file, 0, 0); + assertEquals(1, blocksInFile.length); + List replicasInBlock = Arrays.asList(blocksInFile[0].getNames()); + if (failed) { + String errMsg = String.format( + "Unexpected exception appending to the block pipeline." + + " nodesWithReplica=[%s]", String.join(", ", replicasInBlock)); + LOG.error(errMsg, failedReason); // Do not swallow the exception + fail(errMsg); + } else if (expectedReplicas != replicasInBlock.size()) { + String errMsg = String.format("Expecting %d replicas in block pipeline," + + " unexpectedly found %d replicas. nodesWithReplica=[%s]", expectedReplicas, + replicasInBlock.size(), String.join(", ", replicasInBlock)); + LOG.error(errMsg); + fail(errMsg); + } else { + String infoMsg = String.format( + "Successfully appended block pipeline with %d replicas." + + " nodesWithReplica=[%s]", + replicasInBlock.size(), String.join(", ", replicasInBlock)); + LOG.info(infoMsg); + } + } + } } diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/TestFileAppend3.java b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/TestFileAppend3.java index 3e9adcac7e7e0..a7663b0ba9aa0 100644 --- a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/TestFileAppend3.java +++ b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/TestFileAppend3.java @@ -26,7 +26,6 @@ import java.util.List; import org.apache.hadoop.fs.CreateFlag; -import org.apache.hadoop.hdfs.server.datanode.FsDatasetTestUtils.MaterializedReplica; import org.apache.hadoop.test.GenericTestUtils; import org.mockito.invocation.InvocationOnMock; import static org.mockito.Mockito.spy; diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/TestFileCreation.java b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/TestFileCreation.java index 406e1e6699179..2c41c410ca3f3 100644 --- a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/TestFileCreation.java +++ b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/TestFileCreation.java @@ -273,7 +273,7 @@ public void testServerDefaultsWithMinimalCaching() throws Exception { defaults.getDefaultStoragePolicyId()); doReturn(newDefaults).when(spyNamesystem).getServerDefaults(); - // Verify that the value is updated correctly. Wait for 3 seconds. + // Verify that the value is updated correctly. Wait for 6 seconds. GenericTestUtils.waitFor(()->{ try { FsServerDefaults currDef = dfsClient.getServerDefaults(); @@ -282,7 +282,7 @@ public void testServerDefaultsWithMinimalCaching() throws Exception { // do nothing; return false; } - }, 1, 3000); + }, 1, 6000); } finally { cluster.shutdown(); diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/TestHDFSPolicyProvider.java b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/TestHDFSPolicyProvider.java index 0a74a2d025659..6f43f858c1d96 100644 --- a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/TestHDFSPolicyProvider.java +++ b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/TestHDFSPolicyProvider.java @@ -19,9 +19,7 @@ import static org.junit.Assert.*; -import java.util.ArrayList; import java.util.Arrays; -import java.util.Collections; import java.util.Comparator; import java.util.HashSet; import java.util.List; diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/TestModTime.java b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/TestModTime.java index 3978444c6a217..0b73157e71898 100644 --- a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/TestModTime.java +++ b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/TestModTime.java @@ -26,8 +26,6 @@ import java.util.Random; import org.apache.hadoop.conf.Configuration; -import org.apache.hadoop.fs.CommonConfigurationKeys; -import org.apache.hadoop.fs.FSDataOutputStream; import org.apache.hadoop.fs.FileStatus; import org.apache.hadoop.fs.FileSystem; import org.apache.hadoop.fs.Path; diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/TestRollingUpgradeDowngrade.java b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/TestRollingUpgradeDowngrade.java index 189b5f575a48f..ff17c2f61740d 100644 --- a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/TestRollingUpgradeDowngrade.java +++ b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/TestRollingUpgradeDowngrade.java @@ -28,7 +28,6 @@ import org.apache.hadoop.hdfs.protocol.HdfsConstants.SafeModeAction; import org.apache.hadoop.hdfs.protocol.RollingUpgradeInfo; import org.apache.hadoop.hdfs.qjournal.MiniQJMHACluster; -import org.apache.hadoop.hdfs.server.common.IncorrectVersionException; import org.apache.hadoop.hdfs.server.namenode.NNStorage; import org.apache.hadoop.hdfs.server.namenode.NameNodeLayoutVersion; import org.junit.Assert; diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/TestSeekBug.java b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/TestSeekBug.java index 9dd2987f55f64..c7a7e7a8d11c3 100644 --- a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/TestSeekBug.java +++ b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/TestSeekBug.java @@ -20,7 +20,6 @@ import static org.junit.Assert.assertEquals; import static org.junit.Assert.assertTrue; -import java.io.DataOutputStream; import java.io.IOException; import java.util.Random; diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/TestSetTimes.java b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/TestSetTimes.java index 74cde2a2a95ec..7039a6ba692f1 100644 --- a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/TestSetTimes.java +++ b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/TestSetTimes.java @@ -42,7 +42,6 @@ import org.apache.hadoop.util.Time; import org.junit.Assert; import org.junit.Test; -import org.mockito.Mockito; /** * This class tests the access time on files. diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/TestSlowDatanodeReport.java b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/TestSlowDatanodeReport.java index 583c3159d5c57..187919c39b9aa 100644 --- a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/TestSlowDatanodeReport.java +++ b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/TestSlowDatanodeReport.java @@ -33,6 +33,7 @@ import org.apache.hadoop.conf.Configuration; import org.apache.hadoop.hdfs.protocol.DatanodeInfo; import org.apache.hadoop.hdfs.server.datanode.DataNode; +import org.apache.hadoop.hdfs.server.protocol.OutlierMetrics; import org.apache.hadoop.test.GenericTestUtils; import static org.apache.hadoop.hdfs.DFSConfigKeys.DFS_DATANODE_MIN_OUTLIER_DETECTION_NODES_KEY; @@ -72,8 +73,10 @@ public void tearDown() throws Exception { public void testSingleNodeReport() throws Exception { List dataNodes = cluster.getDataNodes(); DataNode slowNode = dataNodes.get(1); + OutlierMetrics outlierMetrics = new OutlierMetrics(1.245, 2.69375, 4.5667, 15.5); dataNodes.get(0).getPeerMetrics().setTestOutliers( - ImmutableMap.of(slowNode.getDatanodeHostname() + ":" + slowNode.getIpcPort(), 15.5)); + ImmutableMap.of(slowNode.getDatanodeHostname() + ":" + slowNode.getIpcPort(), + outlierMetrics)); DistributedFileSystem distributedFileSystem = cluster.getFileSystem(); Assert.assertEquals(3, distributedFileSystem.getDataNodeStats().length); GenericTestUtils.waitFor(() -> { @@ -86,15 +89,27 @@ public void testSingleNodeReport() throws Exception { return false; } }, 2000, 180000, "Slow nodes could not be detected"); + LOG.info("Slow peer report: {}", cluster.getNameNode().getSlowPeersReport()); + Assert.assertTrue(cluster.getNameNode().getSlowPeersReport().length() > 0); + Assert.assertTrue( + cluster.getNameNode().getSlowPeersReport().contains(slowNode.getDatanodeHostname())); + Assert.assertTrue(cluster.getNameNode().getSlowPeersReport().contains("15.5")); + Assert.assertTrue(cluster.getNameNode().getSlowPeersReport().contains("1.245")); + Assert.assertTrue(cluster.getNameNode().getSlowPeersReport().contains("2.69375")); + Assert.assertTrue(cluster.getNameNode().getSlowPeersReport().contains("4.5667")); } @Test public void testMultiNodesReport() throws Exception { List dataNodes = cluster.getDataNodes(); + OutlierMetrics outlierMetrics1 = new OutlierMetrics(2.498237, 19.2495, 23.568204, 14.5); + OutlierMetrics outlierMetrics2 = new OutlierMetrics(3.2535, 22.4945, 44.5667, 18.7); dataNodes.get(0).getPeerMetrics().setTestOutliers(ImmutableMap.of( - dataNodes.get(1).getDatanodeHostname() + ":" + dataNodes.get(1).getIpcPort(), 15.5)); + dataNodes.get(1).getDatanodeHostname() + ":" + dataNodes.get(1).getIpcPort(), + outlierMetrics1)); dataNodes.get(1).getPeerMetrics().setTestOutliers(ImmutableMap.of( - dataNodes.get(2).getDatanodeHostname() + ":" + dataNodes.get(2).getIpcPort(), 18.7)); + dataNodes.get(2).getDatanodeHostname() + ":" + dataNodes.get(2).getIpcPort(), + outlierMetrics2)); DistributedFileSystem distributedFileSystem = cluster.getFileSystem(); Assert.assertEquals(3, distributedFileSystem.getDataNodeStats().length); GenericTestUtils.waitFor(() -> { @@ -107,6 +122,16 @@ public void testMultiNodesReport() throws Exception { return false; } }, 2000, 200000, "Slow nodes could not be detected"); + LOG.info("Slow peer report: {}", cluster.getNameNode().getSlowPeersReport()); + Assert.assertTrue(cluster.getNameNode().getSlowPeersReport().length() > 0); + Assert.assertTrue(cluster.getNameNode().getSlowPeersReport() + .contains(dataNodes.get(1).getDatanodeHostname())); + Assert.assertTrue(cluster.getNameNode().getSlowPeersReport() + .contains(dataNodes.get(2).getDatanodeHostname())); + Assert.assertTrue(cluster.getNameNode().getSlowPeersReport().contains("14.5")); + Assert.assertTrue(cluster.getNameNode().getSlowPeersReport().contains("18.7")); + Assert.assertTrue(cluster.getNameNode().getSlowPeersReport().contains("23.568204")); + Assert.assertTrue(cluster.getNameNode().getSlowPeersReport().contains("22.4945")); } } diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/TestSnapshotCommands.java b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/TestSnapshotCommands.java index 52bc12f4d8098..97256b6524e9f 100644 --- a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/TestSnapshotCommands.java +++ b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/TestSnapshotCommands.java @@ -25,7 +25,7 @@ import org.apache.hadoop.fs.Path; import org.apache.hadoop.hdfs.protocol.SnapshotDiffReport; import org.apache.hadoop.hdfs.tools.snapshot.SnapshotDiff; -import org.apache.hadoop.util.ChunkedArrayList; + import org.junit.After; import org.junit.AfterClass; import org.junit.Before; diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/net/TestDFSNetworkTopology.java b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/net/TestDFSNetworkTopology.java index 1d0024d44472e..6681b2e627d78 100644 --- a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/net/TestDFSNetworkTopology.java +++ b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/net/TestDFSNetworkTopology.java @@ -26,7 +26,6 @@ import org.apache.hadoop.hdfs.server.blockmanagement.DatanodeDescriptor; import org.apache.hadoop.hdfs.server.blockmanagement.DatanodeStorageInfo; import org.apache.hadoop.net.Node; -import org.apache.hadoop.util.Sets; import org.junit.Before; import org.junit.Rule; import org.junit.Test; @@ -34,6 +33,7 @@ import org.slf4j.Logger; import org.slf4j.LoggerFactory; +import java.util.Arrays; import java.util.EnumMap; import java.util.HashMap; import java.util.HashSet; @@ -309,11 +309,11 @@ public void testChooseRandomWithStorageType() throws Exception { // test the choose random can return desired storage type nodes without // exclude Set diskUnderL1 = - Sets.newHashSet("host2", "host4", "host5", "host6"); - Set archiveUnderL1 = Sets.newHashSet("host1", "host3"); - Set ramdiskUnderL1 = Sets.newHashSet("host7"); - Set ssdUnderL1 = Sets.newHashSet("host8"); - Set nvdimmUnderL1 = Sets.newHashSet("host9"); + new HashSet<>(Arrays.asList("host2", "host4", "host5", "host6")); + Set archiveUnderL1 = new HashSet<>(Arrays.asList("host1", "host3")); + Set ramdiskUnderL1 = new HashSet<>(Arrays.asList("host7")); + Set ssdUnderL1 = new HashSet<>(Arrays.asList("host8")); + Set nvdimmUnderL1 = new HashSet<>(Arrays.asList("host9")); for (int i = 0; i < 10; i++) { n = CLUSTER.chooseRandomWithStorageType("/l1", null, null, StorageType.DISK); @@ -396,7 +396,7 @@ public void testChooseRandomWithStorageTypeWithExcluded() throws Exception { assertEquals("host6", dd.getHostName()); // exclude the host on r4 (since there is only one host, no randomness here) excluded.add(n); - Set expectedSet = Sets.newHashSet("host4", "host5"); + Set expectedSet = new HashSet<>(Arrays.asList("host4", "host5")); for (int i = 0; i < 10; i++) { // under l1, there are four hosts with DISK: // /l1/d1/r1/host2, /l1/d1/r2/host4, /l1/d1/r2/host5 and /l1/d2/r3/host6 diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/protocolPB/TestPBHelper.java b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/protocolPB/TestPBHelper.java index e9bcef527085d..a1b2e633edd3f 100644 --- a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/protocolPB/TestPBHelper.java +++ b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/protocolPB/TestPBHelper.java @@ -21,6 +21,7 @@ import org.apache.hadoop.thirdparty.protobuf.UninitializedMessageException; import org.apache.hadoop.hdfs.protocol.AddErasureCodingPolicyResponse; import org.apache.hadoop.hdfs.protocol.SystemErasureCodingPolicies; +import org.apache.hadoop.hdfs.server.protocol.OutlierMetrics; import org.apache.hadoop.hdfs.server.protocol.SlowDiskReports; import static org.hamcrest.CoreMatchers.is; @@ -805,8 +806,14 @@ public void testDataNodeInfoPBHelper() { @Test public void testSlowPeerInfoPBHelper() { // Test with a map that has a few slow peer entries. + OutlierMetrics outlierMetrics1 = new OutlierMetrics(0.0, 0.0, 0.0, 0.0); + OutlierMetrics outlierMetrics2 = new OutlierMetrics(0.0, 0.0, 0.0, 1.0); + OutlierMetrics outlierMetrics3 = new OutlierMetrics(0.0, 0.0, 0.0, 2.0); final SlowPeerReports slowPeers = SlowPeerReports.create( - ImmutableMap.of("peer1", 0.0, "peer2", 1.0, "peer3", 2.0)); + ImmutableMap.of( + "peer1", outlierMetrics1, + "peer2", outlierMetrics2, + "peer3", outlierMetrics3)); SlowPeerReports slowPeersConverted1 = PBHelper.convertSlowPeerInfo( PBHelper.convertSlowPeerInfo(slowPeers)); assertTrue( diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/qjournal/client/TestQJMWithFaults.java b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/qjournal/client/TestQJMWithFaults.java index 519d94af10afa..c6e9fb7aa034c 100644 --- a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/qjournal/client/TestQJMWithFaults.java +++ b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/qjournal/client/TestQJMWithFaults.java @@ -34,6 +34,7 @@ import java.util.Map; import java.util.Random; import java.util.SortedSet; +import java.util.TreeSet; import java.util.concurrent.Callable; import java.util.concurrent.ExecutorService; @@ -51,7 +52,6 @@ import org.apache.hadoop.io.IOUtils; import org.apache.hadoop.ipc.ProtobufRpcEngine2; import org.apache.hadoop.test.GenericTestUtils; -import org.apache.hadoop.util.Sets; import org.junit.Rule; import org.junit.Test; import org.junit.rules.ExpectedException; @@ -108,7 +108,7 @@ private static long determineMaxIpcNumber() throws Exception { qjm.format(FAKE_NSINFO, false); doWorkload(cluster, qjm); - SortedSet ipcCounts = Sets.newTreeSet(); + SortedSet ipcCounts = new TreeSet<>(); for (AsyncLogger l : qjm.getLoggerSetForTests().getLoggersForTests()) { InvocationCountingChannel ch = (InvocationCountingChannel)l; ch.waitForAllPendingCalls(); diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/qjournal/server/TestJournalNode.java b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/qjournal/server/TestJournalNode.java index a8d4442bdff28..064dd9e5dd86e 100644 --- a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/qjournal/server/TestJournalNode.java +++ b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/qjournal/server/TestJournalNode.java @@ -142,6 +142,10 @@ public void setup() throws Exception { "qjournal://journalnode0:9900;journalnode1:9901/" + journalId); conf.set(DFSConfigKeys.DFS_NAMENODE_SHARED_EDITS_DIR_KEY +".ns1" +".nn2", "qjournal://journalnode0:9902;journalnode1:9903/" + journalId); + } else if (testName.getMethodName().equals("testConfAbnormalHandlerNumber")) { + conf.setInt(DFSConfigKeys.DFS_JOURNALNODE_HANDLER_COUNT_KEY, -1); + } else if (testName.getMethodName().equals("testConfNormalHandlerNumber")) { + conf.setInt(DFSConfigKeys.DFS_JOURNALNODE_HANDLER_COUNT_KEY, 10); } jn = new JournalNode(); jn.setConf(conf); @@ -672,4 +676,26 @@ private void setupStaticHostResolution(int journalNodeCount, } } + @Test + public void testConfNormalHandlerNumber() { + int confHandlerNumber = jn.getConf().getInt( + DFSConfigKeys.DFS_JOURNALNODE_HANDLER_COUNT_KEY, + DFSConfigKeys.DFS_JOURNALNODE_HANDLER_COUNT_DEFAULT); + assertTrue(confHandlerNumber > 0); + int handlerCount = jn.getRpcServer().getHandlerCount(); + assertEquals(confHandlerNumber, handlerCount); + assertEquals(10, handlerCount); + } + + @Test + public void testConfAbnormalHandlerNumber() { + int confHandlerCount = jn.getConf().getInt( + DFSConfigKeys.DFS_JOURNALNODE_HANDLER_COUNT_KEY, + DFSConfigKeys.DFS_JOURNALNODE_HANDLER_COUNT_DEFAULT); + assertTrue(confHandlerCount <= 0); + int handlerCount = jn.getRpcServer().getHandlerCount(); + assertEquals( + DFSConfigKeys.DFS_JOURNALNODE_HANDLER_COUNT_DEFAULT, + handlerCount); + } } diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/qjournal/server/TestJournalNodeSync.java b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/qjournal/server/TestJournalNodeSync.java index bc4cf3a6ee7b1..1564e41031aba 100644 --- a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/qjournal/server/TestJournalNodeSync.java +++ b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/qjournal/server/TestJournalNodeSync.java @@ -33,6 +33,8 @@ import org.apache.hadoop.hdfs.server.namenode.FileJournalManager.EditLogFile; import static org.apache.hadoop.hdfs.server.namenode.FileJournalManager .getLogFile; +import static org.assertj.core.api.Assertions.assertThat; + import org.apache.hadoop.hdfs.server.protocol.NamespaceInfo; import org.apache.hadoop.test.GenericTestUtils; import org.apache.hadoop.util.Lists; @@ -106,6 +108,8 @@ public void testJournalNodeSync() throws Exception { File firstJournalDir = jCluster.getJournalDir(0, jid); File firstJournalCurrentDir = new StorageDirectory(firstJournalDir) .getCurrentDir(); + assertThat(jCluster.getJournalNode(0).getRpcServer().getRpcServer().getRpcMetrics() + .getTotalRequests()).isGreaterThan(20); // Generate some edit logs and delete one. long firstTxId = generateEditLog(); diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/balancer/TestBalancerWithHANameNodes.java b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/balancer/TestBalancerWithHANameNodes.java index 9b4ba5ccb73a0..d69051c8d7af7 100644 --- a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/balancer/TestBalancerWithHANameNodes.java +++ b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/balancer/TestBalancerWithHANameNodes.java @@ -202,15 +202,19 @@ public void testBalancerRequestSBNWithHA() throws Exception { try { cluster.waitActive(); cluster.transitionToActive(0); + String standbyNameNode = cluster.getNameNode(1). + getNameNodeAddress().getHostString(); Thread.sleep(500); client = NameNodeProxies.createProxy(conf, FileSystem.getDefaultUri(conf), ClientProtocol.class).getProxy(); doTest(conf); // Check getBlocks request to Standby NameNode. assertTrue(log.getOutput().contains( - "Request #getBlocks to Standby NameNode success.")); + "Request #getBlocks to Standby NameNode success. remoteAddress: " + + standbyNameNode)); assertTrue(log.getOutput().contains( - "Request #getLiveDatanodeStorageReport to Standby NameNode success")); + "Request #getLiveDatanodeStorageReport to Standby NameNode success. " + + "remoteAddress: " + standbyNameNode)); } finally { cluster.shutdown(); } diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/blockmanagement/TestDatanodeManager.java b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/blockmanagement/TestDatanodeManager.java index 232424d4404ec..35ff36a856ba8 100644 --- a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/blockmanagement/TestDatanodeManager.java +++ b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/blockmanagement/TestDatanodeManager.java @@ -86,6 +86,10 @@ public class TestDatanodeManager { private static DatanodeManager mockDatanodeManager( FSNamesystem fsn, Configuration conf) throws IOException { BlockManager bm = Mockito.mock(BlockManager.class); + Mockito.when(bm.getMaxReplicationStreams()).thenReturn( + conf.getInt(DFSConfigKeys.DFS_NAMENODE_REPLICATION_MAX_STREAMS_KEY, 2)); + Mockito.when(bm.getReplicationStreamsHardLimit()).thenReturn( + conf.getInt(DFSConfigKeys.DFS_NAMENODE_REPLICATION_STREAMS_HARD_LIMIT_KEY, 2)); BlockReportLeaseManager blm = new BlockReportLeaseManager(conf); Mockito.when(bm.getBlockReportLeaseManager()).thenReturn(blm); DatanodeManager dm = new DatanodeManager(bm, fsn, conf); @@ -965,25 +969,33 @@ public void testRemoveIncludedNode() throws IOException { * @param numReplicationBlocks the number of replication blocks in the queue. * @param numECBlocks number of EC blocks in the queue. * @param maxTransfers the maxTransfer value. + * @param maxTransfersHardLimit the maxTransfer hard limit value. * @param numReplicationTasks the number of replication tasks polled from * the queue. * @param numECTasks the number of EC tasks polled from the queue. + * @param isDecommissioning if the node is in the decommissioning process. * * @throws IOException */ private void verifyPendingRecoveryTasks( int numReplicationBlocks, int numECBlocks, - int maxTransfers, int numReplicationTasks, int numECTasks) + int maxTransfers, int maxTransfersHardLimit, + int numReplicationTasks, int numECTasks, boolean isDecommissioning) throws IOException { FSNamesystem fsn = Mockito.mock(FSNamesystem.class); Mockito.when(fsn.hasWriteLock()).thenReturn(true); Configuration conf = new Configuration(); + conf.setInt(DFSConfigKeys.DFS_NAMENODE_REPLICATION_MAX_STREAMS_KEY, maxTransfers); + conf.setInt(DFSConfigKeys.DFS_NAMENODE_REPLICATION_STREAMS_HARD_LIMIT_KEY, + maxTransfersHardLimit); DatanodeManager dm = Mockito.spy(mockDatanodeManager(fsn, conf)); DatanodeDescriptor nodeInfo = Mockito.mock(DatanodeDescriptor.class); Mockito.when(nodeInfo.isRegistered()).thenReturn(true); Mockito.when(nodeInfo.getStorageInfos()) .thenReturn(new DatanodeStorageInfo[0]); + Mockito.when(nodeInfo.isDecommissionInProgress()) + .thenReturn(isDecommissioning); if (numReplicationBlocks > 0) { Mockito.when(nodeInfo.getNumberOfReplicateBlocks()) @@ -1010,7 +1022,7 @@ private void verifyPendingRecoveryTasks( DatanodeRegistration dnReg = Mockito.mock(DatanodeRegistration.class); Mockito.when(dm.getDatanode(dnReg)).thenReturn(nodeInfo); DatanodeCommand[] cmds = dm.handleHeartbeat( - dnReg, new StorageReport[1], "bp-123", 0, 0, 10, maxTransfers, 0, null, + dnReg, new StorageReport[1], "bp-123", 0, 0, 10, 0, 0, null, SlowPeerReports.EMPTY_REPORT, SlowDiskReports.EMPTY_REPORT); long expectedNumCmds = Arrays.stream( @@ -1042,11 +1054,14 @@ private void verifyPendingRecoveryTasks( @Test public void testPendingRecoveryTasks() throws IOException { // Tasks are slitted according to the ratio between queue lengths. - verifyPendingRecoveryTasks(20, 20, 20, 10, 10); - verifyPendingRecoveryTasks(40, 10, 20, 16, 4); + verifyPendingRecoveryTasks(20, 20, 20, 30, 10, 10, false); + verifyPendingRecoveryTasks(40, 10, 20, 30, 16, 4, false); // Approximately load tasks if the ratio between queue length is large. - verifyPendingRecoveryTasks(400, 1, 20, 20, 1); + verifyPendingRecoveryTasks(400, 1, 20, 30, 20, 1, false); + + // Tasks use dfs.namenode.replication.max-streams-hard-limit for decommissioning node + verifyPendingRecoveryTasks(30, 30, 20, 30, 15, 15, true); } @Test diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/blockmanagement/TestReplicationPolicyExcludeSlowNodes.java b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/blockmanagement/TestReplicationPolicyExcludeSlowNodes.java index f2c24a646b84a..05af3f9bb56cb 100644 --- a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/blockmanagement/TestReplicationPolicyExcludeSlowNodes.java +++ b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/blockmanagement/TestReplicationPolicyExcludeSlowNodes.java @@ -22,6 +22,8 @@ import org.apache.hadoop.hdfs.DFSTestUtil; import org.apache.hadoop.hdfs.TestBlockStoragePolicy; import org.apache.hadoop.hdfs.server.namenode.NameNode; +import org.apache.hadoop.hdfs.server.protocol.OutlierMetrics; + import org.junit.Test; import org.junit.runner.RunWith; import org.junit.runners.Parameterized; @@ -88,12 +90,18 @@ public void testChooseTargetExcludeSlowNodes() throws Exception { // mock slow nodes SlowPeerTracker tracker = dnManager.getSlowPeerTracker(); - tracker.addReport(dataNodes[0].getInfoAddr(), dataNodes[3].getInfoAddr()); - tracker.addReport(dataNodes[0].getInfoAddr(), dataNodes[4].getInfoAddr()); - tracker.addReport(dataNodes[1].getInfoAddr(), dataNodes[4].getInfoAddr()); - tracker.addReport(dataNodes[1].getInfoAddr(), dataNodes[5].getInfoAddr()); - tracker.addReport(dataNodes[2].getInfoAddr(), dataNodes[3].getInfoAddr()); - tracker.addReport(dataNodes[2].getInfoAddr(), dataNodes[5].getInfoAddr()); + OutlierMetrics outlierMetrics1 = new OutlierMetrics(0.0, 0.0, 0.0, 1.29463); + tracker.addReport(dataNodes[0].getInfoAddr(), dataNodes[3].getInfoAddr(), outlierMetrics1); + OutlierMetrics outlierMetrics2 = new OutlierMetrics(0.0, 0.0, 0.0, 2.9576); + tracker.addReport(dataNodes[0].getInfoAddr(), dataNodes[4].getInfoAddr(), outlierMetrics2); + OutlierMetrics outlierMetrics3 = new OutlierMetrics(0.0, 0.0, 0.0, 3.59674); + tracker.addReport(dataNodes[1].getInfoAddr(), dataNodes[4].getInfoAddr(), outlierMetrics3); + OutlierMetrics outlierMetrics4 = new OutlierMetrics(0.0, 0.0, 0.0, 4.238456); + tracker.addReport(dataNodes[1].getInfoAddr(), dataNodes[5].getInfoAddr(), outlierMetrics4); + OutlierMetrics outlierMetrics5 = new OutlierMetrics(0.0, 0.0, 0.0, 5.18375); + tracker.addReport(dataNodes[2].getInfoAddr(), dataNodes[3].getInfoAddr(), outlierMetrics5); + OutlierMetrics outlierMetrics6 = new OutlierMetrics(0.0, 0.0, 0.0, 6.39576); + tracker.addReport(dataNodes[2].getInfoAddr(), dataNodes[5].getInfoAddr(), outlierMetrics6); // waiting for slow nodes collector run Thread.sleep(3000); diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/blockmanagement/TestSlowPeerTracker.java b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/blockmanagement/TestSlowPeerTracker.java index fb2928cc4861e..67fb5a6a48c04 100644 --- a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/blockmanagement/TestSlowPeerTracker.java +++ b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/blockmanagement/TestSlowPeerTracker.java @@ -23,8 +23,7 @@ import com.fasterxml.jackson.databind.ObjectReader; import org.apache.hadoop.conf.Configuration; import org.apache.hadoop.hdfs.HdfsConfiguration; -import org.apache.hadoop.hdfs.server.blockmanagement.SlowPeerTracker - .ReportForJson; +import org.apache.hadoop.hdfs.server.protocol.OutlierMetrics; import org.apache.hadoop.util.FakeTimer; import org.junit.Before; import org.junit.Rule; @@ -37,6 +36,7 @@ import java.util.Set; import static org.hamcrest.core.Is.is; +import static org.junit.Assert.assertEquals; import static org.junit.Assert.assertFalse; import static org.junit.Assert.assertThat; import static org.junit.Assert.assertTrue; @@ -45,8 +45,7 @@ * Tests for {@link SlowPeerTracker}. */ public class TestSlowPeerTracker { - public static final Logger LOG = LoggerFactory.getLogger( - TestSlowPeerTracker.class); + private static final Logger LOG = LoggerFactory.getLogger(TestSlowPeerTracker.class); /** * Set a timeout for every test case. @@ -59,7 +58,7 @@ public class TestSlowPeerTracker { private FakeTimer timer; private long reportValidityMs; private static final ObjectReader READER = - new ObjectMapper().readerFor(new TypeReference>() {}); + new ObjectMapper().readerFor(new TypeReference>() {}); @Before public void setup() { @@ -80,9 +79,9 @@ public void testEmptyReports() { @Test public void testReportsAreRetrieved() { - tracker.addReport("node2", "node1"); - tracker.addReport("node3", "node1"); - tracker.addReport("node3", "node2"); + tracker.addReport("node2", "node1", new OutlierMetrics(0.0, 0.0, 0.0, 1.2)); + tracker.addReport("node3", "node1", new OutlierMetrics(0.0, 0.0, 0.0, 2.1)); + tracker.addReport("node3", "node2", new OutlierMetrics(0.0, 0.0, 0.0, 1.22)); assertThat(tracker.getReportsForAllDataNodes().size(), is(2)); assertThat(tracker.getReportsForNode("node2").size(), is(1)); @@ -95,9 +94,9 @@ public void testReportsAreRetrieved() { */ @Test public void testAllReportsAreExpired() { - tracker.addReport("node2", "node1"); - tracker.addReport("node3", "node2"); - tracker.addReport("node1", "node3"); + tracker.addReport("node2", "node1", new OutlierMetrics(0.0, 0.0, 0.0, 0.123)); + tracker.addReport("node3", "node2", new OutlierMetrics(0.0, 0.0, 0.0, 0.2334)); + tracker.addReport("node1", "node3", new OutlierMetrics(0.0, 0.0, 0.0, 1.234)); // No reports should expire after 1ms. timer.advance(1); @@ -117,13 +116,14 @@ public void testAllReportsAreExpired() { */ @Test public void testSomeReportsAreExpired() { - tracker.addReport("node3", "node1"); - tracker.addReport("node3", "node2"); + tracker.addReport("node3", "node1", new OutlierMetrics(0.0, 0.0, 0.0, 1.234)); + tracker.addReport("node3", "node2", new OutlierMetrics(0.0, 0.0, 0.0, 1.222)); timer.advance(reportValidityMs); - tracker.addReport("node3", "node4"); + tracker.addReport("node3", "node4", new OutlierMetrics(0.0, 0.0, 0.0, 1.20)); assertThat(tracker.getReportsForAllDataNodes().size(), is(1)); assertThat(tracker.getReportsForNode("node3").size(), is(1)); - assertTrue(tracker.getReportsForNode("node3").contains("node4")); + assertEquals(1, tracker.getReportsForNode("node3").stream() + .filter(e -> e.getReportingNode().equals("node4")).count()); } /** @@ -131,24 +131,30 @@ public void testSomeReportsAreExpired() { */ @Test public void testReplacement() { - tracker.addReport("node2", "node1"); + OutlierMetrics outlierMetrics1 = new OutlierMetrics(0.0, 0.0, 0.0, 2.1); + tracker.addReport("node2", "node1", outlierMetrics1); timer.advance(reportValidityMs); // Expire the report. assertThat(tracker.getReportsForAllDataNodes().size(), is(0)); // This should replace the expired report with a newer valid one. - tracker.addReport("node2", "node1"); + OutlierMetrics outlierMetrics2 = new OutlierMetrics(0.0, 0.0, 0.0, 0.001); + tracker.addReport("node2", "node1", outlierMetrics2); assertThat(tracker.getReportsForAllDataNodes().size(), is(1)); assertThat(tracker.getReportsForNode("node2").size(), is(1)); } @Test public void testGetJson() throws IOException { - tracker.addReport("node1", "node2"); - tracker.addReport("node2", "node3"); - tracker.addReport("node2", "node1"); - tracker.addReport("node4", "node1"); + OutlierMetrics outlierMetrics1 = new OutlierMetrics(0.0, 0.0, 0.0, 1.1); + tracker.addReport("node1", "node2", outlierMetrics1); + OutlierMetrics outlierMetrics2 = new OutlierMetrics(0.0, 0.0, 0.0, 1.23); + tracker.addReport("node2", "node3", outlierMetrics2); + OutlierMetrics outlierMetrics3 = new OutlierMetrics(0.0, 0.0, 0.0, 2.13); + tracker.addReport("node2", "node1", outlierMetrics3); + OutlierMetrics outlierMetrics4 = new OutlierMetrics(0.0, 0.0, 0.0, 1.244); + tracker.addReport("node4", "node1", outlierMetrics4); - final Set reports = getAndDeserializeJson(); + final Set reports = getAndDeserializeJson(); // And ensure its contents are what we expect. assertThat(reports.size(), is(3)); @@ -161,19 +167,19 @@ public void testGetJson() throws IOException { @Test public void testGetJsonSizeIsLimited() throws IOException { - tracker.addReport("node1", "node2"); - tracker.addReport("node1", "node3"); - tracker.addReport("node2", "node3"); - tracker.addReport("node2", "node4"); - tracker.addReport("node3", "node4"); - tracker.addReport("node3", "node5"); - tracker.addReport("node4", "node6"); - tracker.addReport("node5", "node6"); - tracker.addReport("node5", "node7"); - tracker.addReport("node6", "node7"); - tracker.addReport("node6", "node8"); - - final Set reports = getAndDeserializeJson(); + tracker.addReport("node1", "node2", new OutlierMetrics(0.0, 0.0, 0.0, 1.634)); + tracker.addReport("node1", "node3", new OutlierMetrics(0.0, 0.0, 0.0, 2.3566)); + tracker.addReport("node2", "node3", new OutlierMetrics(0.0, 0.0, 0.0, 3.869)); + tracker.addReport("node2", "node4", new OutlierMetrics(0.0, 0.0, 0.0, 4.1356)); + tracker.addReport("node3", "node4", new OutlierMetrics(0.0, 0.0, 0.0, 1.73057)); + tracker.addReport("node3", "node5", new OutlierMetrics(0.0, 0.0, 0.0, 2.4956730)); + tracker.addReport("node4", "node6", new OutlierMetrics(0.0, 0.0, 0.0, 3.29847)); + tracker.addReport("node5", "node6", new OutlierMetrics(0.0, 0.0, 0.0, 4.13444)); + tracker.addReport("node5", "node7", new OutlierMetrics(0.0, 0.0, 0.0, 5.10845)); + tracker.addReport("node6", "node8", new OutlierMetrics(0.0, 0.0, 0.0, 2.37464)); + tracker.addReport("node6", "node7", new OutlierMetrics(0.0, 0.0, 0.0, 1.29475656)); + + final Set reports = getAndDeserializeJson(); // Ensure that node4 is not in the list since it was // tagged by just one peer and we already have 5 other nodes. @@ -185,22 +191,49 @@ public void testGetJsonSizeIsLimited() throws IOException { assertTrue(isNodeInReports(reports, "node3")); assertTrue(isNodeInReports(reports, "node5")); assertTrue(isNodeInReports(reports, "node6")); + + assertEquals(1, reports.stream().filter( + e -> e.getSlowNode().equals("node1") && e.getSlowPeerLatencyWithReportingNodes().size() == 2 + && e.getSlowPeerLatencyWithReportingNodes().first().getReportedLatency().equals(1.634) + && e.getSlowPeerLatencyWithReportingNodes().last().getReportedLatency().equals(2.3566)) + .count()); + + assertEquals(1, reports.stream().filter( + e -> e.getSlowNode().equals("node2") && e.getSlowPeerLatencyWithReportingNodes().size() == 2 + && e.getSlowPeerLatencyWithReportingNodes().first().getReportedLatency().equals(3.869) + && e.getSlowPeerLatencyWithReportingNodes().last().getReportedLatency().equals(4.1356)) + .count()); + + assertEquals(1, reports.stream().filter( + e -> e.getSlowNode().equals("node3") && e.getSlowPeerLatencyWithReportingNodes().size() == 2 + && e.getSlowPeerLatencyWithReportingNodes().first().getReportedLatency().equals(1.73057) + && e.getSlowPeerLatencyWithReportingNodes().last().getReportedLatency() + .equals(2.4956730)).count()); + + assertEquals(1, reports.stream().filter( + e -> e.getSlowNode().equals("node6") && e.getSlowPeerLatencyWithReportingNodes().size() == 2 + && e.getSlowPeerLatencyWithReportingNodes().first().getReportedLatency() + .equals(1.29475656) && e.getSlowPeerLatencyWithReportingNodes().last() + .getReportedLatency().equals(2.37464)).count()); } @Test public void testLowRankedElementsIgnored() throws IOException { // Insert 5 nodes with 2 peer reports each. for (int i = 0; i < 5; ++i) { - tracker.addReport("node" + i, "reporter1"); - tracker.addReport("node" + i, "reporter2"); + OutlierMetrics outlierMetrics1 = new OutlierMetrics(0.0, 0.0, 0.0, 1.295673); + tracker.addReport("node" + i, "reporter1", outlierMetrics1); + OutlierMetrics outlierMetrics2 = new OutlierMetrics(0.0, 0.0, 0.0, 2.38560); + tracker.addReport("node" + i, "reporter2", outlierMetrics2); } // Insert 10 nodes with 1 peer report each. for (int i = 10; i < 20; ++i) { - tracker.addReport("node" + i, "reporter1"); + OutlierMetrics outlierMetrics = new OutlierMetrics(0.0, 0.0, 0.0, 3.4957); + tracker.addReport("node" + i, "reporter1", outlierMetrics); } - final Set reports = getAndDeserializeJson(); + final Set reports = getAndDeserializeJson(); // Ensure that only the first 5 nodes with two reports each were // included in the JSON. @@ -210,8 +243,8 @@ public void testLowRankedElementsIgnored() throws IOException { } private boolean isNodeInReports( - Set reports, String node) { - for (ReportForJson report : reports) { + Set reports, String node) { + for (SlowPeerJsonReport report : reports) { if (report.getSlowNode().equalsIgnoreCase(node)) { return true; } @@ -219,7 +252,7 @@ private boolean isNodeInReports( return false; } - private Set getAndDeserializeJson() + private Set getAndDeserializeJson() throws IOException { final String json = tracker.getJson(); LOG.info("Got JSON: {}", json); diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/datanode/SimulatedFSDataset.java b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/datanode/SimulatedFSDataset.java index 29eb051cb0210..e66b62e4e51fe 100644 --- a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/datanode/SimulatedFSDataset.java +++ b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/datanode/SimulatedFSDataset.java @@ -356,6 +356,10 @@ synchronized public void setBytesAcked(long bytesAcked) { public void releaseAllBytesReserved() { } + @Override + public void releaseReplicaInfoBytesReserved() { + } + @Override synchronized public long getBytesOnDisk() { if (finalized) { @@ -418,7 +422,6 @@ public void waitForMinLength(long minLength, long time, TimeUnit unit) } while (deadLine > System.currentTimeMillis()); throw new IOException("Minimum length was not achieved within timeout"); } - @Override public FsVolumeSpi getVolume() { return getStorage(theBlock).getVolume(); diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/datanode/TestBpServiceActorScheduler.java b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/datanode/TestBpServiceActorScheduler.java index f8406ed7a3fae..166ae118f784e 100644 --- a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/datanode/TestBpServiceActorScheduler.java +++ b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/datanode/TestBpServiceActorScheduler.java @@ -18,6 +18,7 @@ package org.apache.hadoop.hdfs.server.datanode; +import org.apache.hadoop.util.Time; import org.slf4j.Logger; import org.slf4j.LoggerFactory; import org.apache.hadoop.hdfs.server.datanode.BPServiceActor.Scheduler; @@ -204,6 +205,18 @@ public void testScheduleLifeline() { } } + @Test + public void testScheduleLifelineScheduleTime() { + Scheduler mockScheduler = spy(new Scheduler( + HEARTBEAT_INTERVAL_MS, LIFELINE_INTERVAL_MS, + BLOCK_REPORT_INTERVAL_MS, OUTLIER_REPORT_INTERVAL_MS)); + long now = Time.monotonicNow(); + mockScheduler.scheduleNextLifeline(now); + long mockMonotonicNow = now + LIFELINE_INTERVAL_MS * 2; + doReturn(mockMonotonicNow).when(mockScheduler).monotonicNow(); + assertTrue(mockScheduler.getLifelineWaitTime() >= 0); + } + @Test public void testOutlierReportScheduling() { for (final long now : getTimestamps()) { diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/datanode/TestDataNodeInitStorage.java b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/datanode/TestDataNodeInitStorage.java index df8be2220693d..9d6958ea2ab94 100644 --- a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/datanode/TestDataNodeInitStorage.java +++ b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/datanode/TestDataNodeInitStorage.java @@ -29,10 +29,6 @@ import org.apache.hadoop.hdfs.server.datanode.fsdataset.FsDatasetSpi; import org.junit.Test; -import static org.hamcrest.core.IsNot.not; -import static org.junit.Assert.*; - - /** * Test to verify that the DataNode Uuid is correctly initialized before * FsDataSet initialization. diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/datanode/TestDatanodeRegister.java b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/datanode/TestDatanodeRegister.java index 7c1c27b4dd4ed..772452decc0c2 100644 --- a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/datanode/TestDatanodeRegister.java +++ b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/datanode/TestDatanodeRegister.java @@ -76,7 +76,7 @@ public void setUp() throws IOException { actor = new BPServiceActor("test", "test", INVALID_ADDR, null, mockBPOS); fakeNsInfo = mock(NamespaceInfo.class); - // Return a a good software version. + // Return a good software version. doReturn(VersionInfo.getVersion()).when(fakeNsInfo).getSoftwareVersion(); // Return a good layout version for now. doReturn(HdfsServerConstants.NAMENODE_LAYOUT_VERSION).when(fakeNsInfo) @@ -144,7 +144,7 @@ public void testDNShutdwonBeforeRegister() throws Exception { DataNode dn = new DataNode(conf, locations, null, null); BPOfferService bpos = new BPOfferService("test_ns", Lists.newArrayList("nn0"), Lists.newArrayList(nnADDR), - Collections.nCopies(1, null), dn); + Collections.nCopies(1, null), dn); DatanodeProtocolClientSideTranslatorPB fakeDnProt = mock(DatanodeProtocolClientSideTranslatorPB.class); when(fakeDnProt.versionRequest()).thenReturn(fakeNsInfo); diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/datanode/TestRefreshNamenodes.java b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/datanode/TestRefreshNamenodes.java index 60d4cca059ac9..1f18b7bde0f4f 100644 --- a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/datanode/TestRefreshNamenodes.java +++ b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/datanode/TestRefreshNamenodes.java @@ -23,6 +23,7 @@ import java.io.IOException; import java.net.InetSocketAddress; +import java.util.HashSet; import java.util.Set; import org.apache.hadoop.conf.Configuration; @@ -73,13 +74,13 @@ public void testRefreshNamenodes() throws IOException { // Ensure a BPOfferService in the datanodes corresponds to // a namenode in the cluster - Set nnAddrsFromCluster = Sets.newHashSet(); + Set nnAddrsFromCluster = new HashSet<>(); for (int i = 0; i < 4; i++) { assertTrue(nnAddrsFromCluster.add( cluster.getNameNode(i).getNameNodeAddress())); } - Set nnAddrsFromDN = Sets.newHashSet(); + Set nnAddrsFromDN = new HashSet<>(); for (BPOfferService bpos : dn.getAllBpOs()) { for (BPServiceActor bpsa : bpos.getBPServiceActors()) { assertTrue(nnAddrsFromDN.add(bpsa.getNNSocketAddress())); diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/datanode/extdataset/ExternalReplicaInPipeline.java b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/datanode/extdataset/ExternalReplicaInPipeline.java index 084caf038c3db..460d1c1eb7d4e 100644 --- a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/datanode/extdataset/ExternalReplicaInPipeline.java +++ b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/datanode/extdataset/ExternalReplicaInPipeline.java @@ -45,6 +45,10 @@ public long getBytesAcked() { public void setBytesAcked(long bytesAcked) { } + @Override + public void releaseReplicaInfoBytesReserved() { + } + @Override public void releaseAllBytesReserved() { } diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/datanode/fsdataset/impl/TestCacheByPmemMappableBlockLoader.java b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/datanode/fsdataset/impl/TestCacheByPmemMappableBlockLoader.java index 9575028944997..baea8f5de2bbb 100644 --- a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/datanode/fsdataset/impl/TestCacheByPmemMappableBlockLoader.java +++ b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/datanode/fsdataset/impl/TestCacheByPmemMappableBlockLoader.java @@ -66,6 +66,7 @@ import org.apache.hadoop.thirdparty.com.google.common.primitives.Ints; import static org.apache.hadoop.hdfs.DFSConfigKeys.DFS_DATANODE_FSDATASETCACHE_MAX_THREADS_PER_VOLUME_KEY; +import static org.junit.Assume.assumeTrue; /** * Tests HDFS persistent memory cache by PmemMappableBlockLoader. @@ -104,6 +105,8 @@ public class TestCacheByPmemMappableBlockLoader { @BeforeClass public static void setUpClass() throws Exception { + assumeTrue("Requires PMDK", NativeIO.POSIX.isPmdkAvailable()); + oldInjector = DataNodeFaultInjector.get(); DataNodeFaultInjector.set(new DataNodeFaultInjector() { @Override diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/datanode/fsdataset/impl/TestDatanodeRestart.java b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/datanode/fsdataset/impl/TestDatanodeRestart.java index 0afee5fa57f27..ac9587e54f7ee 100644 --- a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/datanode/fsdataset/impl/TestDatanodeRestart.java +++ b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/datanode/fsdataset/impl/TestDatanodeRestart.java @@ -18,12 +18,8 @@ package org.apache.hadoop.hdfs.server.datanode.fsdataset.impl; import java.io.File; -import java.io.FileInputStream; -import java.io.FileOutputStream; import java.io.IOException; import java.io.RandomAccessFile; -import java.util.Collection; -import java.util.Iterator; import java.util.Random; import org.apache.hadoop.conf.Configuration; @@ -40,7 +36,6 @@ import org.apache.hadoop.hdfs.server.datanode.DataNode; import org.apache.hadoop.hdfs.server.datanode.DataNodeFaultInjector; import org.apache.hadoop.hdfs.server.datanode.DataNodeTestUtils; -import org.apache.hadoop.hdfs.server.datanode.DatanodeUtil; import org.apache.hadoop.hdfs.server.datanode.ReplicaInfo; import org.apache.hadoop.hdfs.server.datanode.fsdataset.FsDatasetSpi; import org.apache.hadoop.hdfs.server.datanode.fsdataset.FsVolumeSpi; diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/datanode/fsdataset/impl/TestPmemCacheRecovery.java b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/datanode/fsdataset/impl/TestPmemCacheRecovery.java index d3232c8362949..6ce420adeb197 100644 --- a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/datanode/fsdataset/impl/TestPmemCacheRecovery.java +++ b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/datanode/fsdataset/impl/TestPmemCacheRecovery.java @@ -27,6 +27,7 @@ import static org.junit.Assert.assertNotNull; import static org.junit.Assert.assertTrue; import static org.junit.Assert.fail; +import static org.junit.Assume.assumeTrue; import java.io.File; import java.io.IOException; @@ -103,6 +104,8 @@ public class TestPmemCacheRecovery { @BeforeClass public static void setUpClass() throws Exception { + assumeTrue("Requires PMDK", NativeIO.POSIX.isPmdkAvailable()); + oldInjector = DataNodeFaultInjector.get(); DataNodeFaultInjector.set(new DataNodeFaultInjector() { @Override diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/datanode/fsdataset/impl/TestSpaceReservation.java b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/datanode/fsdataset/impl/TestSpaceReservation.java index a702cec7cb0ca..de4d236617374 100644 --- a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/datanode/fsdataset/impl/TestSpaceReservation.java +++ b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/datanode/fsdataset/impl/TestSpaceReservation.java @@ -18,8 +18,14 @@ package org.apache.hadoop.hdfs.server.datanode.fsdataset.impl; +import java.util.Collection; +import java.util.EnumSet; import java.util.function.Supplier; +import org.apache.hadoop.fs.CreateFlag; +import org.apache.hadoop.fs.permission.FsPermission; +import org.apache.hadoop.hdfs.protocol.ExtendedBlock; +import org.apache.hadoop.hdfs.server.datanode.ReplicaInfo; import org.apache.hadoop.io.IOUtils; import org.slf4j.Logger; import org.slf4j.LoggerFactory; @@ -746,4 +752,48 @@ public Boolean get() { }, 500, 30000); checkReservedSpace(0); } + + /** + * Ensure that bytes reserved of ReplicaInfo gets cleared + * during finalize. + * + * @throws IOException + */ + @Test(timeout = 300000) + public void testReplicaInfoBytesReservedReleasedOnFinalize() throws IOException { + short replication = 3; + int bufferLength = 4096; + startCluster(BLOCK_SIZE, replication, -1); + + String methodName = GenericTestUtils.getMethodName(); + Path path = new Path("/" + methodName + ".01.dat"); + + FSDataOutputStream fos = + fs.create(path, FsPermission.getFileDefault(), EnumSet.of(CreateFlag.CREATE), bufferLength, + replication, BLOCK_SIZE, null); + // Allocate a block. + fos.write(new byte[bufferLength]); + fos.hsync(); + + DataNode dataNode = cluster.getDataNodes().get(0); + FsDatasetImpl fsDataSetImpl = (FsDatasetImpl) dataNode.getFSDataset(); + long expectedReservedSpace = BLOCK_SIZE - bufferLength; + + String bpid = cluster.getNamesystem().getBlockPoolId(); + Collection replicas = FsDatasetTestUtil.getReplicas(fsDataSetImpl, bpid); + ReplicaInfo r = replicas.iterator().next(); + + // Verify Initial Bytes Reserved for Replica and Volume are correct + assertEquals(fsDataSetImpl.getVolumeList().get(0).getReservedForReplicas(), + expectedReservedSpace); + assertEquals(r.getBytesReserved(), expectedReservedSpace); + + // Verify Bytes Reserved for Replica and Volume are correct after finalize + fsDataSetImpl.finalizeNewReplica(r, new ExtendedBlock(bpid, r)); + + assertEquals(fsDataSetImpl.getVolumeList().get(0).getReservedForReplicas(), 0L); + assertEquals(r.getBytesReserved(), 0L); + + fos.close(); + } } diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/datanode/fsdataset/impl/TestWriteToReplica.java b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/datanode/fsdataset/impl/TestWriteToReplica.java index fe78a0f2a41a0..202fb190f3d64 100644 --- a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/datanode/fsdataset/impl/TestWriteToReplica.java +++ b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/datanode/fsdataset/impl/TestWriteToReplica.java @@ -245,8 +245,10 @@ private void testAppend(String bpid, FsDatasetSpi dataSet, Assert.fail("Should not have appended to a non-existent replica " + blocks[NON_EXISTENT]); } catch (ReplicaNotFoundException e) { - Assert.assertEquals(ReplicaNotFoundException.NON_EXISTENT_REPLICA + - blocks[NON_EXISTENT], e.getMessage()); + String expectMessage = ReplicaNotFoundException.NON_EXISTENT_REPLICA + + blocks[NON_EXISTENT].getBlockPoolId() + ":" + + blocks[NON_EXISTENT].getBlockId(); + Assert.assertEquals(expectMessage, e.getMessage()); } newGS = blocks[FINALIZED].getGenerationStamp()+1; diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/datanode/metrics/TestDataNodeOutlierDetectionViaMetrics.java b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/datanode/metrics/TestDataNodeOutlierDetectionViaMetrics.java index 1faddb362ecd1..0042bcb042646 100644 --- a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/datanode/metrics/TestDataNodeOutlierDetectionViaMetrics.java +++ b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/datanode/metrics/TestDataNodeOutlierDetectionViaMetrics.java @@ -21,6 +21,7 @@ import java.util.function.Supplier; import org.apache.hadoop.conf.Configuration; import org.apache.hadoop.hdfs.HdfsConfiguration; +import org.apache.hadoop.hdfs.server.protocol.OutlierMetrics; import org.apache.hadoop.metrics2.lib.MetricsTestHelper; import org.apache.hadoop.test.GenericTestUtils; import org.junit.Before; @@ -100,7 +101,7 @@ public Boolean get() { } }, 500, 100_000); - final Map outliers = peerMetrics.getOutliers(); + final Map outliers = peerMetrics.getOutliers(); LOG.info("Got back outlier nodes: {}", outliers); assertThat(outliers.size(), is(1)); assertTrue(outliers.containsKey(slowNodeName)); diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/datanode/web/TestHostRestrictingAuthorizationFilterHandler.java b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/datanode/web/TestHostRestrictingAuthorizationFilterHandler.java index 031ac0aa29d35..a918a081e7f6a 100644 --- a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/datanode/web/TestHostRestrictingAuthorizationFilterHandler.java +++ b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/datanode/web/TestHostRestrictingAuthorizationFilterHandler.java @@ -48,7 +48,7 @@ public class TestHostRestrictingAuthorizationFilterHandler { * Test running in with no ACL rules (restrict all) */ @Test - public void testRejectAll() throws Exception { + public void testRejectAll() { EmbeddedChannel channel = new CustomEmbeddedChannel("127.0.0.1", 1006, new HostRestrictingAuthorizationFilterHandler()); FullHttpRequest httpRequest = @@ -61,7 +61,7 @@ public void testRejectAll() throws Exception { DefaultHttpResponse channelResponse = (DefaultHttpResponse) channel.outboundMessages().poll(); assertNotNull("Expected response to exist.", channelResponse); - assertEquals(HttpResponseStatus.FORBIDDEN, channelResponse.getStatus()); + assertEquals(HttpResponseStatus.FORBIDDEN, channelResponse.status()); assertFalse(channel.isOpen()); } @@ -70,7 +70,7 @@ public void testRejectAll() throws Exception { * reused */ @Test - public void testMultipleAcceptedGETsOneChannel() throws Exception { + public void testMultipleAcceptedGETsOneChannel() { Configuration conf = new Configuration(); conf.set(CONFNAME, "*,*,/allowed"); HostRestrictingAuthorizationFilter filter = @@ -102,7 +102,7 @@ public void testMultipleAcceptedGETsOneChannel() throws Exception { * single filter instance */ @Test - public void testMultipleChannels() throws Exception { + public void testMultipleChannels() { Configuration conf = new Configuration(); conf.set(CONFNAME, "*,*,/allowed"); HostRestrictingAuthorizationFilter filter = @@ -140,7 +140,7 @@ public void testMultipleChannels() throws Exception { * Test accepting a GET request for the file checksum */ @Test - public void testAcceptGETFILECHECKSUM() throws Exception { + public void testAcceptGETFILECHECKSUM() { EmbeddedChannel channel = new CustomEmbeddedChannel("127.0.0.1", 1006, new HostRestrictingAuthorizationFilterHandler()); FullHttpRequest httpRequest = @@ -158,7 +158,7 @@ public void testAcceptGETFILECHECKSUM() throws Exception { */ protected static class CustomEmbeddedChannel extends EmbeddedChannel { - private InetSocketAddress socketAddress; + private final InetSocketAddress socketAddress; /* * A normal @{EmbeddedChannel} constructor which takes the remote client diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/FSImageTestUtil.java b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/FSImageTestUtil.java index 7f4a0ce54197d..0334cb8a45f37 100644 --- a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/FSImageTestUtil.java +++ b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/FSImageTestUtil.java @@ -416,7 +416,7 @@ public static void assertFileContentsSame(File... files) throws Exception { if (files.length < 2) return; Map md5s = getFileMD5s(files); - if (Sets.newHashSet(md5s.values()).size() > 1) { + if (new HashSet<>(md5s.values()).size() > 1) { fail("File contents differed:\n " + Joiner.on("\n ") .withKeyValueSeparator("=") @@ -433,7 +433,8 @@ public static void assertFileContentsDifferent( File... files) throws Exception { Map md5s = getFileMD5s(files); - if (Sets.newHashSet(md5s.values()).size() != expectedUniqueHashes) { + int uniqueHashes = new HashSet<>(md5s.values()).size(); + if (uniqueHashes != expectedUniqueHashes) { fail("Expected " + expectedUniqueHashes + " different hashes, got:\n " + Joiner.on("\n ") .withKeyValueSeparator("=") diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/TestBlockUnderConstruction.java b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/TestBlockUnderConstruction.java index 49e79da674e5d..cab87e014a756 100644 --- a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/TestBlockUnderConstruction.java +++ b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/TestBlockUnderConstruction.java @@ -45,9 +45,6 @@ import org.junit.BeforeClass; import org.junit.Test; -import org.apache.hadoop.hdfs.protocol.proto.HdfsProtos.LocatedBlockProto; -import org.apache.hadoop.hdfs.protocol.ExtendedBlock; - public class TestBlockUnderConstruction { static final String BASE_DIR = "/test/TestBlockUnderConstruction"; static final int BLOCK_SIZE = 8192; // same as TestFileCreation.blocksize diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/TestCommitBlockWithInvalidGenStamp.java b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/TestCommitBlockWithInvalidGenStamp.java index f7b1ea5bac351..d57e00ef3967e 100644 --- a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/TestCommitBlockWithInvalidGenStamp.java +++ b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/TestCommitBlockWithInvalidGenStamp.java @@ -20,14 +20,12 @@ import org.apache.hadoop.conf.Configuration; import org.apache.hadoop.fs.FSDataOutputStream; import org.apache.hadoop.fs.Path; -import org.apache.hadoop.fs.StorageType; import org.apache.hadoop.hdfs.DFSConfigKeys; import org.apache.hadoop.hdfs.DFSTestUtil; import org.apache.hadoop.hdfs.DistributedFileSystem; import org.apache.hadoop.hdfs.MiniDFSCluster; import org.apache.hadoop.hdfs.protocol.Block; import org.apache.hadoop.hdfs.protocol.ExtendedBlock; -import org.apache.hadoop.hdfs.protocol.HdfsConstants; import org.apache.hadoop.io.IOUtils; import org.junit.After; import org.junit.Assert; diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/TestFileContextXAttr.java b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/TestFileContextXAttr.java index da09298b777dd..3ae6f4f7baccd 100644 --- a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/TestFileContextXAttr.java +++ b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/TestFileContextXAttr.java @@ -29,9 +29,7 @@ import org.apache.hadoop.fs.FileSystem; import org.apache.hadoop.fs.Path; import org.apache.hadoop.fs.XAttrSetFlag; -import org.apache.hadoop.hdfs.DFSConfigKeys; import org.apache.hadoop.hdfs.DistributedFileSystem; -import org.junit.BeforeClass; /** * Tests of XAttr operations using FileContext APIs. diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/TestFileLimit.java b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/TestFileLimit.java index d5a04c8e4ffe5..02818c952dabf 100644 --- a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/TestFileLimit.java +++ b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/TestFileLimit.java @@ -30,7 +30,6 @@ import org.apache.hadoop.hdfs.MiniDFSCluster; import org.apache.hadoop.hdfs.client.HdfsDataOutputStream; import org.apache.hadoop.hdfs.server.datanode.SimulatedFSDataset; -import org.apache.hadoop.ipc.RemoteException; import org.apache.hadoop.test.GenericTestUtils; import org.junit.Test; diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/TestFsck.java b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/TestFsck.java index ae1021da0db4d..7bb288809dea9 100644 --- a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/TestFsck.java +++ b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/TestFsck.java @@ -50,6 +50,7 @@ import java.util.ArrayList; import java.util.Arrays; import java.util.HashMap; +import java.util.HashSet; import java.util.List; import java.util.Map; import java.util.Random; @@ -118,7 +119,6 @@ import org.apache.hadoop.security.AccessControlException; import org.apache.hadoop.security.UserGroupInformation; import org.apache.hadoop.test.GenericTestUtils; -import org.apache.hadoop.util.Sets; import org.apache.hadoop.util.ToolRunner; import org.apache.log4j.Level; import org.apache.log4j.Logger; @@ -386,15 +386,15 @@ public void testFsckMove() throws Exception { cluster.getNameNodePort()), conf); String[] fileNames = util.getFileNames(topDir); CorruptedTestFile[] ctFiles = new CorruptedTestFile[]{ - new CorruptedTestFile(fileNames[0], Sets.newHashSet(0), + new CorruptedTestFile(fileNames[0], new HashSet<>(Arrays.asList(0)), dfsClient, numDatanodes, dfsBlockSize), - new CorruptedTestFile(fileNames[1], Sets.newHashSet(2, 3), + new CorruptedTestFile(fileNames[1], new HashSet<>(Arrays.asList(2, 3)), dfsClient, numDatanodes, dfsBlockSize), - new CorruptedTestFile(fileNames[2], Sets.newHashSet(4), + new CorruptedTestFile(fileNames[2], new HashSet<>(Arrays.asList(4)), dfsClient, numDatanodes, dfsBlockSize), - new CorruptedTestFile(fileNames[3], Sets.newHashSet(0, 1, 2, 3), + new CorruptedTestFile(fileNames[3], new HashSet<>(Arrays.asList(0, 1, 2, 3)), dfsClient, numDatanodes, dfsBlockSize), - new CorruptedTestFile(fileNames[4], Sets.newHashSet(1, 2, 3, 4), + new CorruptedTestFile(fileNames[4], new HashSet<>(Arrays.asList(1, 2, 3, 4)), dfsClient, numDatanodes, dfsBlockSize) }; int totalMissingBlocks = 0; @@ -2215,7 +2215,7 @@ public void testFsckMoveAfterCorruption() throws Exception { new InetSocketAddress("localhost", cluster.getNameNodePort()), conf); final String blockFileToCorrupt = fileNames[0]; final CorruptedTestFile ctf = new CorruptedTestFile(blockFileToCorrupt, - Sets.newHashSet(0), dfsClient, numDatanodes, dfsBlockSize); + new HashSet<>(Arrays.asList(0)), dfsClient, numDatanodes, dfsBlockSize); ctf.corruptBlocks(cluster); // Wait for fsck to discover all the missing blocks diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/TestFsckWithMultipleNameNodes.java b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/TestFsckWithMultipleNameNodes.java index 327c51ca765fa..5870fbb549884 100644 --- a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/TestFsckWithMultipleNameNodes.java +++ b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/TestFsckWithMultipleNameNodes.java @@ -28,7 +28,6 @@ import org.apache.hadoop.fs.FileSystem; import org.apache.hadoop.fs.Path; import org.apache.hadoop.fs.viewfs.ConfigUtil; -import org.apache.hadoop.fs.viewfs.ViewFileSystem; import org.apache.hadoop.hdfs.DFSConfigKeys; import org.apache.hadoop.hdfs.DFSTestUtil; import org.apache.hadoop.hdfs.HdfsConfiguration; diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/TestNameNodeReconfigure.java b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/TestNameNodeReconfigure.java index 7a3b9910553ab..d048429814656 100644 --- a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/TestNameNodeReconfigure.java +++ b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/TestNameNodeReconfigure.java @@ -19,11 +19,14 @@ package org.apache.hadoop.hdfs.server.namenode; import java.io.IOException; +import java.util.ArrayList; +import java.util.List; import org.junit.Test; import org.junit.Before; import org.junit.After; +import static org.apache.hadoop.hdfs.DFSConfigKeys.DFS_DATANODE_MAX_NODES_TO_REPORT_KEY; import static org.apache.hadoop.hdfs.DFSConfigKeys.DFS_IMAGE_PARALLEL_LOAD_KEY; import static org.junit.Assert.*; @@ -40,7 +43,9 @@ import org.apache.hadoop.hdfs.HdfsConfiguration; import org.apache.hadoop.hdfs.server.blockmanagement.DatanodeManager; import org.apache.hadoop.hdfs.server.blockmanagement.BlockManager; +import org.apache.hadoop.hdfs.server.blockmanagement.SlowPeerTracker; import org.apache.hadoop.hdfs.server.namenode.sps.StoragePolicySatisfyManager; +import org.apache.hadoop.hdfs.server.protocol.OutlierMetrics; import org.apache.hadoop.ipc.RemoteException; import org.apache.hadoop.test.GenericTestUtils; @@ -513,6 +518,55 @@ public void testSlowPeerTrackerEnabled() throws Exception { } + @Test + public void testSlowPeerMaxNodesToReportReconf() throws Exception { + final NameNode nameNode = cluster.getNameNode(); + final DatanodeManager datanodeManager = nameNode.namesystem.getBlockManager() + .getDatanodeManager(); + nameNode.reconfigurePropertyImpl(DFS_DATANODE_PEER_STATS_ENABLED_KEY, "true"); + assertTrue("SlowNode tracker is still disabled. Reconfiguration could not be successful", + datanodeManager.getSlowPeerTracker().isSlowPeerTrackerEnabled()); + + SlowPeerTracker tracker = datanodeManager.getSlowPeerTracker(); + + OutlierMetrics outlierMetrics1 = new OutlierMetrics(0.0, 0.0, 0.0, 1.1); + tracker.addReport("node1", "node70", outlierMetrics1); + OutlierMetrics outlierMetrics2 = new OutlierMetrics(0.0, 0.0, 0.0, 1.23); + tracker.addReport("node2", "node71", outlierMetrics2); + OutlierMetrics outlierMetrics3 = new OutlierMetrics(0.0, 0.0, 0.0, 2.13); + tracker.addReport("node3", "node72", outlierMetrics3); + OutlierMetrics outlierMetrics4 = new OutlierMetrics(0.0, 0.0, 0.0, 1.244); + tracker.addReport("node4", "node73", outlierMetrics4); + OutlierMetrics outlierMetrics5 = new OutlierMetrics(0.0, 0.0, 0.0, 0.2); + tracker.addReport("node5", "node74", outlierMetrics4); + OutlierMetrics outlierMetrics6 = new OutlierMetrics(0.0, 0.0, 0.0, 1.244); + tracker.addReport("node6", "node75", outlierMetrics4); + + String jsonReport = tracker.getJson(); + LOG.info("Retrieved slow peer json report: {}", jsonReport); + + List containReport = validatePeerReport(jsonReport); + assertEquals(1, containReport.stream().filter(reportVal -> !reportVal).count()); + + nameNode.reconfigurePropertyImpl(DFS_DATANODE_MAX_NODES_TO_REPORT_KEY, "2"); + jsonReport = tracker.getJson(); + LOG.info("Retrieved slow peer json report: {}", jsonReport); + + containReport = validatePeerReport(jsonReport); + assertEquals(4, containReport.stream().filter(reportVal -> !reportVal).count()); + } + + private List validatePeerReport(String jsonReport) { + List containReport = new ArrayList<>(); + containReport.add(jsonReport.contains("node1")); + containReport.add(jsonReport.contains("node2")); + containReport.add(jsonReport.contains("node3")); + containReport.add(jsonReport.contains("node4")); + containReport.add(jsonReport.contains("node5")); + containReport.add(jsonReport.contains("node6")); + return containReport; + } + @After public void shutDown() throws IOException { if (cluster != null) { diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/TestNameNodeRecovery.java b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/TestNameNodeRecovery.java index 51389c8336f16..8a701a31b9fb9 100644 --- a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/TestNameNodeRecovery.java +++ b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/TestNameNodeRecovery.java @@ -28,6 +28,7 @@ import java.io.IOException; import java.io.RandomAccessFile; import java.util.ArrayList; +import java.util.Arrays; import java.util.Collection; import java.util.HashSet; import java.util.Set; @@ -49,7 +50,6 @@ import org.apache.hadoop.io.IOUtils; import org.apache.hadoop.test.GenericTestUtils; import org.apache.hadoop.test.PathUtils; -import org.apache.hadoop.util.Sets; import org.apache.hadoop.util.StringUtils; import org.junit.Test; import org.junit.runner.RunWith; @@ -299,7 +299,7 @@ public long getLastValidTxId() { @Override public Set getValidTxIds() { - return Sets.newHashSet(0L); + return new HashSet<>(Arrays.asList(0L)); } public int getMaxOpSize() { @@ -341,7 +341,7 @@ public long getLastValidTxId() { @Override public Set getValidTxIds() { - return Sets.newHashSet(0L); + return new HashSet<>(Arrays.asList(0L)); } } @@ -387,7 +387,7 @@ public long getLastValidTxId() { @Override public Set getValidTxIds() { - return Sets.newHashSet(1L , 2L, 3L, 5L, 6L, 7L, 8L, 9L, 10L); + return new HashSet<>(Arrays.asList(1L, 2L, 3L, 5L, 6L, 7L, 8L, 9L, 10L)); } } diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/TestSecondaryWebUi.java b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/TestSecondaryWebUi.java index 9473cb3ed950d..1d3187dffe0b1 100644 --- a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/TestSecondaryWebUi.java +++ b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/TestSecondaryWebUi.java @@ -21,7 +21,6 @@ import org.apache.hadoop.conf.Configuration; import org.apache.hadoop.hdfs.DFSConfigKeys; -import org.apache.hadoop.hdfs.DFSTestUtil; import org.apache.hadoop.hdfs.MiniDFSCluster; import org.junit.AfterClass; import org.junit.Assert; @@ -31,7 +30,6 @@ import javax.management.*; import java.io.IOException; import java.lang.management.ManagementFactory; -import java.net.URL; public class TestSecondaryWebUi { diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/TestSecureNameNode.java b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/TestSecureNameNode.java index c90a91c67d4c0..e67e5cf720c4a 100644 --- a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/TestSecureNameNode.java +++ b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/TestSecureNameNode.java @@ -19,7 +19,6 @@ import static org.junit.Assert.assertEquals; import static org.junit.Assert.assertNotNull; -import static org.junit.Assert.fail; import java.io.IOException; import java.lang.management.ManagementFactory; diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/TestSecureNameNodeWithExternalKdc.java b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/TestSecureNameNodeWithExternalKdc.java index d36b9c997e2b9..60601a6ab9133 100644 --- a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/TestSecureNameNodeWithExternalKdc.java +++ b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/TestSecureNameNodeWithExternalKdc.java @@ -19,7 +19,6 @@ import static org.junit.Assert.assertEquals; import static org.junit.Assert.assertNotNull; -import static org.junit.Assert.assertTrue; import static org.junit.Assert.fail; import java.io.IOException; diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/ha/TestConsistentReadsObserver.java b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/ha/TestConsistentReadsObserver.java index decf85c06a46f..ff6c2288b538b 100644 --- a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/ha/TestConsistentReadsObserver.java +++ b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/ha/TestConsistentReadsObserver.java @@ -20,6 +20,7 @@ import static org.apache.hadoop.hdfs.DFSConfigKeys.DFS_NAMENODE_STATE_CONTEXT_ENABLED_KEY; import static org.apache.hadoop.test.MetricsAsserts.getLongCounter; import static org.apache.hadoop.test.MetricsAsserts.getMetrics; +import static org.assertj.core.api.Assertions.assertThat; import static org.junit.Assert.assertEquals; import static org.junit.Assert.assertTrue; import static org.junit.Assert.fail; @@ -123,6 +124,7 @@ public void testRequeueCall() throws Exception { + CommonConfigurationKeys.IPC_BACKOFF_ENABLE, true); NameNodeAdapter.getRpcServer(nn).refreshCallQueue(configuration); + assertThat(NameNodeAdapter.getRpcServer(nn).getTotalRequests()).isGreaterThan(0); dfs.create(testPath, (short)1).close(); assertSentTo(0); @@ -132,6 +134,7 @@ public void testRequeueCall() throws Exception { // be triggered and client should retry active NN. dfs.getFileStatus(testPath); assertSentTo(0); + assertThat(NameNodeAdapter.getRpcServer(nn).getTotalRequests()).isGreaterThan(1); // reset the original call queue NameNodeAdapter.getRpcServer(nn).refreshCallQueue(originalConf); } diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/snapshot/TestXAttrWithSnapshot.java b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/snapshot/TestXAttrWithSnapshot.java index badff6821243e..03931d6c6630d 100644 --- a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/snapshot/TestXAttrWithSnapshot.java +++ b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/snapshot/TestXAttrWithSnapshot.java @@ -34,7 +34,6 @@ import org.apache.hadoop.hdfs.DistributedFileSystem; import org.apache.hadoop.hdfs.MiniDFSCluster; import org.apache.hadoop.hdfs.protocol.HdfsConstants; -import org.apache.hadoop.hdfs.protocol.NSQuotaExceededException; import org.apache.hadoop.hdfs.protocol.SnapshotAccessControlException; import org.apache.hadoop.hdfs.protocol.SnapshotDiffReport; import org.apache.hadoop.hdfs.server.namenode.NameNode; diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/startupprogress/TestStartupProgressMetrics.java b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/startupprogress/TestStartupProgressMetrics.java index 4fe3d15c4a82b..ef2004f6eb6e1 100644 --- a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/startupprogress/TestStartupProgressMetrics.java +++ b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/startupprogress/TestStartupProgressMetrics.java @@ -17,9 +17,7 @@ */ package org.apache.hadoop.hdfs.server.namenode.startupprogress; -import static org.apache.hadoop.hdfs.server.namenode.startupprogress.Phase.*; import static org.apache.hadoop.hdfs.server.namenode.startupprogress.StartupProgressTestHelper.*; -import static org.apache.hadoop.hdfs.server.namenode.startupprogress.StepType.*; import static org.apache.hadoop.test.MetricsAsserts.*; import static org.junit.Assert.*; diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/sps/TestExternalStoragePolicySatisfier.java b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/sps/TestExternalStoragePolicySatisfier.java index 8012f30134366..58d72f14d73ee 100644 --- a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/sps/TestExternalStoragePolicySatisfier.java +++ b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/sps/TestExternalStoragePolicySatisfier.java @@ -1351,7 +1351,7 @@ public void testSPSWhenFileHasLowRedundancyBlocks() throws Exception { * 4. Set policy and call satisfyStoragePolicy for file. * 5. Block should be moved successfully. */ - @Test(timeout = 300000) + @Test(timeout = 600000) public void testSPSWhenFileHasExcessRedundancyBlocks() throws Exception { try { config.set(DFSConfigKeys diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/shortcircuit/TestShortCircuitCache.java b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/shortcircuit/TestShortCircuitCache.java index 965ac0ac98b0f..7b11ec30b43d2 100644 --- a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/shortcircuit/TestShortCircuitCache.java +++ b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/shortcircuit/TestShortCircuitCache.java @@ -21,7 +21,6 @@ import static org.apache.hadoop.hdfs.client.HdfsClientConfigKeys.DFS_CLIENT_CONTEXT; import static org.apache.hadoop.hdfs.client.HdfsClientConfigKeys.DFS_CLIENT_DOMAIN_SOCKET_DATA_TRAFFIC; import static org.apache.hadoop.hdfs.DFSConfigKeys.DFS_DOMAIN_SOCKET_PATH_KEY; -import static org.apache.hadoop.hdfs.client.HdfsClientConfigKeys.DFS_CLIENT_SOCKET_TIMEOUT_KEY; import static org.hamcrest.CoreMatchers.equalTo; import java.io.DataOutputStream; diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/tools/TestDFSAdmin.java b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/tools/TestDFSAdmin.java index 0f0d7824edf16..3df873a51ceae 100644 --- a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/tools/TestDFSAdmin.java +++ b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/tools/TestDFSAdmin.java @@ -33,6 +33,7 @@ import static org.apache.hadoop.fs.CommonConfigurationKeysPublic.IPC_CLIENT_CONNECT_MAX_RETRIES_KEY; import static org.apache.hadoop.hdfs.DFSConfigKeys.DFS_BLOCK_INVALIDATE_LIMIT_KEY; import static org.apache.hadoop.hdfs.DFSConfigKeys.DFS_DATANODE_DATA_DIR_KEY; +import static org.apache.hadoop.hdfs.DFSConfigKeys.DFS_DATANODE_MAX_NODES_TO_REPORT_KEY; import static org.apache.hadoop.hdfs.DFSConfigKeys.DFS_DATANODE_PEER_STATS_ENABLED_KEY; import static org.apache.hadoop.hdfs.DFSConfigKeys.DFS_HEARTBEAT_INTERVAL_DEFAULT; import static org.apache.hadoop.hdfs.DFSConfigKeys.DFS_HEARTBEAT_INTERVAL_KEY; @@ -70,6 +71,8 @@ import org.apache.hadoop.hdfs.protocol.SystemErasureCodingPolicies; import org.apache.hadoop.hdfs.server.blockmanagement.BlockManager; import org.apache.hadoop.hdfs.server.blockmanagement.BlockManagerTestUtil; +import org.apache.hadoop.hdfs.server.blockmanagement.DatanodeDescriptor; +import org.apache.hadoop.hdfs.server.blockmanagement.DatanodeManager; import org.apache.hadoop.hdfs.server.common.Storage; import org.apache.hadoop.hdfs.server.datanode.DataNode; import org.apache.hadoop.hdfs.server.datanode.StorageLocation; @@ -435,18 +438,19 @@ public void testNameNodeGetReconfigurableProperties() throws IOException, Interr final List outs = Lists.newArrayList(); final List errs = Lists.newArrayList(); getReconfigurableProperties("namenode", address, outs, errs); - assertEquals(18, outs.size()); + assertEquals(19, outs.size()); assertTrue(outs.get(0).contains("Reconfigurable properties:")); assertEquals(DFS_BLOCK_INVALIDATE_LIMIT_KEY, outs.get(1)); assertEquals(DFS_BLOCK_PLACEMENT_EC_CLASSNAME_KEY, outs.get(2)); assertEquals(DFS_BLOCK_REPLICATOR_CLASSNAME_KEY, outs.get(3)); - assertEquals(DFS_DATANODE_PEER_STATS_ENABLED_KEY, outs.get(4)); - assertEquals(DFS_HEARTBEAT_INTERVAL_KEY, outs.get(5)); - assertEquals(DFS_IMAGE_PARALLEL_LOAD_KEY, outs.get(6)); - assertEquals(DFS_NAMENODE_AVOID_SLOW_DATANODE_FOR_READ_KEY, outs.get(7)); - assertEquals(DFS_NAMENODE_BLOCKPLACEMENTPOLICY_EXCLUDE_SLOW_NODES_ENABLED_KEY, outs.get(8)); - assertEquals(DFS_NAMENODE_HEARTBEAT_RECHECK_INTERVAL_KEY, outs.get(9)); - assertEquals(DFS_NAMENODE_MAX_SLOWPEER_COLLECT_NODES_KEY, outs.get(10)); + assertEquals(DFS_DATANODE_MAX_NODES_TO_REPORT_KEY, outs.get(4)); + assertEquals(DFS_DATANODE_PEER_STATS_ENABLED_KEY, outs.get(5)); + assertEquals(DFS_HEARTBEAT_INTERVAL_KEY, outs.get(6)); + assertEquals(DFS_IMAGE_PARALLEL_LOAD_KEY, outs.get(7)); + assertEquals(DFS_NAMENODE_AVOID_SLOW_DATANODE_FOR_READ_KEY, outs.get(8)); + assertEquals(DFS_NAMENODE_BLOCKPLACEMENTPOLICY_EXCLUDE_SLOW_NODES_ENABLED_KEY, outs.get(9)); + assertEquals(DFS_NAMENODE_HEARTBEAT_RECHECK_INTERVAL_KEY, outs.get(10)); + assertEquals(DFS_NAMENODE_MAX_SLOWPEER_COLLECT_NODES_KEY, outs.get(11)); assertEquals(errs.size(), 0); } @@ -520,6 +524,52 @@ public void testPrintTopology() throws Exception { } } + @Test(timeout = 30000) + public void testPrintTopologyWithStatus() throws Exception { + redirectStream(); + final Configuration dfsConf = new HdfsConfiguration(); + final File baseDir = new File( + PathUtils.getTestDir(getClass()), + GenericTestUtils.getMethodName()); + dfsConf.set(MiniDFSCluster.HDFS_MINIDFS_BASEDIR, baseDir.getAbsolutePath()); + + final int numDn = 4; + final String[] racks = { + "/d1/r1", "/d1/r2", + "/d2/r1", "/d2/r2"}; + + try (MiniDFSCluster miniCluster = new MiniDFSCluster.Builder(dfsConf) + .numDataNodes(numDn).racks(racks).build()) { + miniCluster.waitActive(); + assertEquals(numDn, miniCluster.getDataNodes().size()); + + DatanodeManager dm = miniCluster.getNameNode().getNamesystem(). + getBlockManager().getDatanodeManager(); + DatanodeDescriptor maintenanceNode = dm.getDatanode( + miniCluster.getDataNodes().get(1).getDatanodeId()); + maintenanceNode.setInMaintenance(); + DatanodeDescriptor demissionNode = dm.getDatanode( + miniCluster.getDataNodes().get(2).getDatanodeId()); + demissionNode.setDecommissioned(); + + final DFSAdmin dfsAdmin = new DFSAdmin(dfsConf); + + resetStream(); + final int ret = ToolRunner.run(dfsAdmin, new String[] {"-printTopology"}); + + /* collect outputs */ + final List outs = Lists.newArrayList(); + scanIntoList(out, outs); + + /* verify results */ + assertEquals(0, ret); + assertTrue(outs.get(1).contains(DatanodeInfo.AdminStates.NORMAL.toString())); + assertTrue(outs.get(4).contains(DatanodeInfo.AdminStates.IN_MAINTENANCE.toString())); + assertTrue(outs.get(7).contains(DatanodeInfo.AdminStates.DECOMMISSIONED.toString())); + assertTrue(outs.get(10).contains(DatanodeInfo.AdminStates.NORMAL.toString())); + } + } + @Test(timeout = 30000) public void testNameNodeGetReconfigurationStatus() throws IOException, InterruptedException, TimeoutException { @@ -1205,9 +1255,10 @@ public void testAllDatanodesReconfig() LOG.info("dfsadmin -status -livenodes output:"); outs.forEach(s -> LOG.info("{}", s)); assertTrue(outs.get(0).startsWith("Reconfiguring status for node")); - assertEquals("SUCCESS: Changed property dfs.datanode.peer.stats.enabled", outs.get(2)); - assertEquals("\tFrom: \"false\"", outs.get(3)); - assertEquals("\tTo: \"true\"", outs.get(4)); + assertTrue("SUCCESS: Changed property dfs.datanode.peer.stats.enabled".equals(outs.get(2)) + || "SUCCESS: Changed property dfs.datanode.peer.stats.enabled".equals(outs.get(1))); + assertTrue("\tFrom: \"false\"".equals(outs.get(3)) || "\tFrom: \"false\"".equals(outs.get(2))); + assertTrue("\tTo: \"true\"".equals(outs.get(4)) || "\tTo: \"true\"".equals(outs.get(3))); assertEquals("SUCCESS: Changed property dfs.datanode.peer.stats.enabled", outs.get(5)); assertEquals("\tFrom: \"false\"", outs.get(6)); assertEquals("\tTo: \"true\"", outs.get(7)); diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/tools/TestDFSAdminWithHA.java b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/tools/TestDFSAdminWithHA.java index 6b8657ccce3b4..5681f3bdb25eb 100644 --- a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/tools/TestDFSAdminWithHA.java +++ b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/tools/TestDFSAdminWithHA.java @@ -69,6 +69,20 @@ private void assertOutputMatches(String string) { err.reset(); } + private void assertOutputMatches(String outMessage, String errMessage) { + String errOutput = new String(err.toByteArray(), Charsets.UTF_8); + String output = new String(out.toByteArray(), Charsets.UTF_8); + + if (!errOutput.matches(errMessage) || !output.matches(outMessage)) { + fail("Expected output to match '" + outMessage + " and " + errMessage + + "' but err_output was:\n" + errOutput + "\n and output was: \n" + + output); + } + + out.reset(); + err.reset(); + } + private void setHAConf(Configuration conf, String nn1Addr, String nn2Addr) { conf.set(CommonConfigurationKeysPublic.FS_DEFAULT_NAME_KEY, "hdfs://" + NSID); @@ -186,9 +200,9 @@ public void testSaveNamespaceNN1UpNN2Down() throws Exception { // exitCode = admin.run(new String[] {"-saveNamespace"}); assertNotEquals(err.toString().trim(), 0, exitCode); - message = "Save namespace successful for.*" + newLine - + "Save namespace failed for.*" + newLine; - assertOutputMatches(message); + String outMessage = "Save namespace successful for.*" + newLine; + String errMessage = "Save namespace failed for ([\\s\\S]*)" + newLine; + assertOutputMatches(outMessage, errMessage); } @Test (timeout = 30000) @@ -204,9 +218,9 @@ public void testSaveNamespaceNN1DownNN2Up() throws Exception { exitCode = admin.run(new String[] {"-saveNamespace"}); assertNotEquals(err.toString().trim(), 0, exitCode); - message = "Save namespace failed for.*" + newLine - + "Save namespace successful for.*" + newLine; - assertOutputMatches(message); + String errMessage = "Save namespace failed for ([\\s\\S]*)" + newLine; + String outMessage = "Save namespace successful for.*" + newLine; + assertOutputMatches(outMessage, errMessage); } @Test (timeout = 30000) @@ -223,7 +237,7 @@ public void testSaveNamespaceNN1DownNN2Down() throws Exception { exitCode = admin.run(new String[] {"-saveNamespace"}); assertNotEquals(err.toString().trim(), 0, exitCode); - message = "Save namespace failed for.*"; + message = "Save namespace failed for ([\\s\\S]*)"; assertOutputMatches(message + newLine + message + newLine); } @@ -253,22 +267,22 @@ public void testRestoreFailedStorageNN1UpNN2Down() throws Exception { cluster.getDfsCluster().shutdownNameNode(1); int exitCode = admin.run(new String[] {"-restoreFailedStorage", "check"}); assertNotEquals(err.toString().trim(), 0, exitCode); - String message = "restoreFailedStorage is set to false for.*" + newLine - + "restoreFailedStorage failed for.*" + newLine; + String outMessage = "restoreFailedStorage is set to false for.*" + newLine; + String errMessage = "restoreFailedStorage failed for ([\\s\\S]*)" + newLine; // Default is false - assertOutputMatches(message); + assertOutputMatches(outMessage, errMessage); exitCode = admin.run(new String[] {"-restoreFailedStorage", "true"}); assertNotEquals(err.toString().trim(), 0, exitCode); - message = "restoreFailedStorage is set to true for.*" + newLine - + "restoreFailedStorage failed for.*" + newLine; - assertOutputMatches(message); + outMessage = "restoreFailedStorage is set to true for.*" + newLine; + errMessage = "restoreFailedStorage failed for ([\\s\\S]*)" + newLine; + assertOutputMatches(outMessage, errMessage); exitCode = admin.run(new String[] {"-restoreFailedStorage", "false"}); assertNotEquals(err.toString().trim(), 0, exitCode); - message = "restoreFailedStorage is set to false for.*" + newLine - + "restoreFailedStorage failed for.*" + newLine; - assertOutputMatches(message); + outMessage = "restoreFailedStorage is set to false for.*" + newLine; + errMessage = "restoreFailedStorage failed for ([\\s\\S]*)" + newLine; + assertOutputMatches(outMessage, errMessage); } @Test (timeout = 30000) @@ -277,22 +291,22 @@ public void testRestoreFailedStorageNN1DownNN2Up() throws Exception { cluster.getDfsCluster().shutdownNameNode(0); int exitCode = admin.run(new String[] {"-restoreFailedStorage", "check"}); assertNotEquals(err.toString().trim(), 0, exitCode); - String message = "restoreFailedStorage failed for.*" + newLine - + "restoreFailedStorage is set to false for.*" + newLine; + String errMessage = "restoreFailedStorage failed for ([\\s\\S]*)" + newLine; + String outMessage = "restoreFailedStorage is set to false for.*" + newLine; // Default is false - assertOutputMatches(message); + assertOutputMatches(outMessage, errMessage); exitCode = admin.run(new String[] {"-restoreFailedStorage", "true"}); assertNotEquals(err.toString().trim(), 0, exitCode); - message = "restoreFailedStorage failed for.*" + newLine - + "restoreFailedStorage is set to true for.*" + newLine; - assertOutputMatches(message); + errMessage = "restoreFailedStorage failed for ([\\s\\S]*)" + newLine; + outMessage = "restoreFailedStorage is set to true for.*" + newLine; + assertOutputMatches(outMessage, errMessage); exitCode = admin.run(new String[] {"-restoreFailedStorage", "false"}); assertNotEquals(err.toString().trim(), 0, exitCode); - message = "restoreFailedStorage failed for.*" + newLine - + "restoreFailedStorage is set to false for.*" + newLine; - assertOutputMatches(message); + errMessage = "restoreFailedStorage failed for ([\\s\\S]*)" + newLine; + outMessage = "restoreFailedStorage is set to false for.*" + newLine; + assertOutputMatches(outMessage, errMessage); } @Test (timeout = 30000) @@ -302,18 +316,18 @@ public void testRestoreFailedStorageNN1DownNN2Down() throws Exception { cluster.getDfsCluster().shutdownNameNode(1); int exitCode = admin.run(new String[] {"-restoreFailedStorage", "check"}); assertNotEquals(err.toString().trim(), 0, exitCode); - String message = "restoreFailedStorage failed for.*"; + String message = "restoreFailedStorage failed for ([\\s\\S]*)"; // Default is false assertOutputMatches(message + newLine + message + newLine); exitCode = admin.run(new String[] {"-restoreFailedStorage", "true"}); assertNotEquals(err.toString().trim(), 0, exitCode); - message = "restoreFailedStorage failed for.*"; + message = "restoreFailedStorage failed for ([\\s\\S]*)"; assertOutputMatches(message + newLine + message + newLine); exitCode = admin.run(new String[] {"-restoreFailedStorage", "false"}); assertNotEquals(err.toString().trim(), 0, exitCode); - message = "restoreFailedStorage failed for.*"; + message = "restoreFailedStorage failed for ([\\s\\S]*)"; assertOutputMatches(message + newLine + message + newLine); } @@ -332,9 +346,9 @@ public void testRefreshNodesNN1UpNN2Down() throws Exception { cluster.getDfsCluster().shutdownNameNode(1); int exitCode = admin.run(new String[] {"-refreshNodes"}); assertNotEquals(err.toString().trim(), 0, exitCode); - String message = "Refresh nodes successful for.*" + newLine - + "Refresh nodes failed for.*" + newLine; - assertOutputMatches(message); + String outMessage = "Refresh nodes successful for .*" + newLine; + String errMessage = "Refresh nodes failed for ([\\s\\S]*)" + newLine; + assertOutputMatches(outMessage, errMessage); } @Test (timeout = 30000) @@ -343,9 +357,9 @@ public void testRefreshNodesNN1DownNN2Up() throws Exception { cluster.getDfsCluster().shutdownNameNode(0); int exitCode = admin.run(new String[] {"-refreshNodes"}); assertNotEquals(err.toString().trim(), 0, exitCode); - String message = "Refresh nodes failed for.*" + newLine - + "Refresh nodes successful for.*" + newLine; - assertOutputMatches(message); + String errMessage = "Refresh nodes failed for ([\\s\\S]*)" + newLine; + String outMessage = "Refresh nodes successful for .*" + newLine; + assertOutputMatches(outMessage, errMessage); } @Test (timeout = 30000) @@ -355,7 +369,7 @@ public void testRefreshNodesNN1DownNN2Down() throws Exception { cluster.getDfsCluster().shutdownNameNode(1); int exitCode = admin.run(new String[] {"-refreshNodes"}); assertNotEquals(err.toString().trim(), 0, exitCode); - String message = "Refresh nodes failed for.*"; + String message = "Refresh nodes failed for ([\\s\\S]*)"; assertOutputMatches(message + newLine + message + newLine); } @@ -432,11 +446,11 @@ public void testMetaSaveNN1UpNN2Down() throws Exception { cluster.getDfsCluster().shutdownNameNode(1); int exitCode = admin.run(new String[] {"-metasave", "dfs.meta"}); assertNotEquals(err.toString().trim(), 0, exitCode); - String message = "Created metasave file dfs.meta in the log directory" - + " of namenode.*" + newLine - + "Created metasave file dfs.meta in the log directory" - + " of namenode.*failed" + newLine; - assertOutputMatches(message); + String outMessage = "Created metasave file dfs.meta in the log " + + "directory of namenode.*" + newLine; + String errMessage = "Created metasave file dfs.meta in the log " + + "directory of namenode.*failed" + newLine + ".*" + newLine; + assertOutputMatches(outMessage, errMessage); } @Test (timeout = 30000) @@ -446,11 +460,11 @@ public void testMetaSaveNN1DownNN2Up() throws Exception { cluster.getDfsCluster().shutdownNameNode(0); int exitCode = admin.run(new String[] {"-metasave", "dfs.meta"}); assertNotEquals(err.toString().trim(), 0, exitCode); - String message = "Created metasave file dfs.meta in the log directory" - + " of namenode.*failed" + newLine - + "Created metasave file dfs.meta in the log directory" - + " of namenode.*" + newLine; - assertOutputMatches(message); + String errMessage = "Created metasave file dfs.meta in the log " + + "directory of namenode.*failed" + newLine + ".*" + newLine; + String outMessage = "Created metasave file dfs.meta in the log " + + "directory of namenode.*" + newLine; + assertOutputMatches(outMessage, errMessage); } @Test (timeout = 30000) @@ -460,9 +474,8 @@ public void testMetaSaveNN1DownNN2Down() throws Exception { cluster.getDfsCluster().shutdownNameNode(1); int exitCode = admin.run(new String[] {"-metasave", "dfs.meta"}); assertNotEquals(err.toString().trim(), 0, exitCode); - String message = "Created metasave file dfs.meta in the log directory" - + " of namenode.*failed"; - assertOutputMatches(message + newLine + message + newLine); + String message = "([\\s\\S]*)2 exceptions([\\s\\S]*)"; + assertOutputMatches(message + newLine); } @Test (timeout = 30000) @@ -480,9 +493,9 @@ public void testRefreshServiceAclNN1UpNN2Down() throws Exception { cluster.getDfsCluster().shutdownNameNode(1); int exitCode = admin.run(new String[] {"-refreshServiceAcl"}); assertNotEquals(err.toString().trim(), 0, exitCode); - String message = "Refresh service acl successful for.*" + newLine - + "Refresh service acl failed for.*" + newLine; - assertOutputMatches(message); + String outMessage = "Refresh service acl successful for.*" + newLine; + String errMessage = "Refresh service acl failed for([\\s\\S]*)" + newLine; + assertOutputMatches(outMessage, errMessage); } @Test (timeout = 30000) @@ -491,9 +504,9 @@ public void testRefreshServiceAclNN1DownNN2Up() throws Exception { cluster.getDfsCluster().shutdownNameNode(0); int exitCode = admin.run(new String[] {"-refreshServiceAcl"}); assertNotEquals(err.toString().trim(), 0, exitCode); - String message = "Refresh service acl failed for.*" + newLine - + "Refresh service acl successful for.*" + newLine; - assertOutputMatches(message); + String errMessage = "Refresh service acl failed for([\\s\\S]*)" + newLine; + String outMessage = "Refresh service acl successful for.*" + newLine; + assertOutputMatches(outMessage, errMessage); } @Test (timeout = 30000) @@ -503,8 +516,8 @@ public void testRefreshServiceAclNN1DownNN2Down() throws Exception { cluster.getDfsCluster().shutdownNameNode(1); int exitCode = admin.run(new String[] {"-refreshServiceAcl"}); assertNotEquals(err.toString().trim(), 0, exitCode); - String message = "Refresh service acl failed for.*"; - assertOutputMatches(message + newLine + message + newLine); + String message = "([\\s\\S]*)2 exceptions([\\s\\S]*)"; + assertOutputMatches(message + newLine); } @@ -523,11 +536,9 @@ public void testRefreshUserToGroupsMappingsNN1UpNN2Down() throws Exception { cluster.getDfsCluster().shutdownNameNode(1); int exitCode = admin.run(new String[] {"-refreshUserToGroupsMappings"}); assertNotEquals(err.toString().trim(), 0, exitCode); - String message = "Refresh user to groups mapping successful for.*" - + newLine - + "Refresh user to groups mapping failed for.*" - + newLine; - assertOutputMatches(message); + String outMessage = "Refresh user to groups mapping successful for.*" + newLine; + String errMessage = "Refresh user to groups mapping failed for([\\s\\S]*)" + newLine; + assertOutputMatches(outMessage, errMessage); } @Test (timeout = 30000) @@ -536,11 +547,9 @@ public void testRefreshUserToGroupsMappingsNN1DownNN2Up() throws Exception { cluster.getDfsCluster().shutdownNameNode(0); int exitCode = admin.run(new String[] {"-refreshUserToGroupsMappings"}); assertNotEquals(err.toString().trim(), 0, exitCode); - String message = "Refresh user to groups mapping failed for.*" - + newLine - + "Refresh user to groups mapping successful for.*" - + newLine; - assertOutputMatches(message); + String errMessage = "Refresh user to groups mapping failed for([\\s\\S]*)" + newLine; + String outMessage = "Refresh user to groups mapping successful for.*" + newLine; + assertOutputMatches(outMessage, errMessage); } @Test (timeout = 30000) @@ -550,8 +559,8 @@ public void testRefreshUserToGroupsMappingsNN1DownNN2Down() throws Exception { cluster.getDfsCluster().shutdownNameNode(1); int exitCode = admin.run(new String[] {"-refreshUserToGroupsMappings"}); assertNotEquals(err.toString().trim(), 0, exitCode); - String message = "Refresh user to groups mapping failed for.*"; - assertOutputMatches(message + newLine + message + newLine); + String message = "([\\s\\S]*)2 exceptions([\\s\\S]*)"; + assertOutputMatches(message + newLine); } @Test (timeout = 30000) @@ -572,11 +581,11 @@ public void testRefreshSuperUserGroupsConfigurationNN1UpNN2Down() int exitCode = admin.run( new String[] {"-refreshSuperUserGroupsConfiguration"}); assertNotEquals(err.toString().trim(), 0, exitCode); - String message = "Refresh super user groups configuration successful for.*" - + newLine - + "Refresh super user groups configuration failed for.*" - + newLine; - assertOutputMatches(message); + String outMessage = "Refresh super user groups configuration successful for.*" + + newLine; + String errMessage = "Refresh super user groups configuration failed for([\\s\\S]*)" + + newLine; + assertOutputMatches(outMessage, errMessage); } @Test (timeout = 30000) @@ -587,11 +596,11 @@ public void testRefreshSuperUserGroupsConfigurationNN1DownNN2Up() int exitCode = admin.run( new String[] {"-refreshSuperUserGroupsConfiguration"}); assertNotEquals(err.toString().trim(), 0, exitCode); - String message = "Refresh super user groups configuration failed for.*" - + newLine - + "Refresh super user groups configuration successful for.*" - + newLine; - assertOutputMatches(message); + String errMessage = "Refresh super user groups configuration failed for([\\s\\S]*)" + + newLine; + String outMessage = "Refresh super user groups configuration successful for.*" + + newLine; + assertOutputMatches(outMessage, errMessage); } @Test (timeout = 30000) @@ -603,8 +612,8 @@ public void testRefreshSuperUserGroupsConfigurationNN1DownNN2Down() int exitCode = admin.run( new String[] {"-refreshSuperUserGroupsConfiguration"}); assertNotEquals(err.toString().trim(), 0, exitCode); - String message = "Refresh super user groups configuration failed for.*"; - assertOutputMatches(message + newLine + message + newLine); + String message = "([\\s\\S]*)2 exceptions([\\s\\S]*)"; + assertOutputMatches(message + newLine); } @Test (timeout = 30000) @@ -622,9 +631,9 @@ public void testRefreshCallQueueNN1UpNN2Down() throws Exception { cluster.getDfsCluster().shutdownNameNode(1); int exitCode = admin.run(new String[] {"-refreshCallQueue"}); assertNotEquals(err.toString().trim(), 0, exitCode); - String message = "Refresh call queue successful for.*" + newLine - + "Refresh call queue failed for.*" + newLine; - assertOutputMatches(message); + String outMessage = "Refresh call queue successful for.*" + newLine; + String errMessage = "Refresh call queue failed for([\\s\\S]*)" + newLine; + assertOutputMatches(outMessage, errMessage); } @Test (timeout = 30000) @@ -633,9 +642,9 @@ public void testRefreshCallQueueNN1DownNN2Up() throws Exception { cluster.getDfsCluster().shutdownNameNode(0); int exitCode = admin.run(new String[] {"-refreshCallQueue"}); assertNotEquals(err.toString().trim(), 0, exitCode); - String message = "Refresh call queue failed for.*" + newLine - + "Refresh call queue successful for.*" + newLine; - assertOutputMatches(message); + String errMessage = "Refresh call queue failed for([\\s\\S]*)" + newLine; + String outMessage = "Refresh call queue successful for.*" + newLine; + assertOutputMatches(outMessage, errMessage); } @Test (timeout = 30000) @@ -645,8 +654,8 @@ public void testRefreshCallQueueNN1DownNN2Down() throws Exception { cluster.getDfsCluster().shutdownNameNode(1); int exitCode = admin.run(new String[] {"-refreshCallQueue"}); assertNotEquals(err.toString().trim(), 0, exitCode); - String message = "Refresh call queue failed for.*"; - assertOutputMatches(message + newLine + message + newLine); + String message = "([\\s\\S]*)2 exceptions([\\s\\S]*)"; + assertOutputMatches(message + newLine); } @Test (timeout = 30000) @@ -671,9 +680,9 @@ public void testFinalizeUpgradeNN1UpNN2Down() throws Exception { cluster.getDfsCluster().transitionToActive(0); int exitCode = admin.run(new String[] {"-finalizeUpgrade"}); assertNotEquals(err.toString().trim(), 0, exitCode); - String message = "Finalize upgrade successful for .*" + newLine - + "Finalize upgrade failed for .*" + newLine; - assertOutputMatches(message); + String outMessage = "Finalize upgrade successful for .*" + newLine; + String errMessage = "Finalize upgrade failed for ([\\s\\S]*)" + newLine; + assertOutputMatches(outMessage, errMessage); } @Test (timeout = 30000) @@ -683,9 +692,9 @@ public void testFinalizeUpgradeNN1DownNN2Up() throws Exception { cluster.getDfsCluster().transitionToActive(1); int exitCode = admin.run(new String[] {"-finalizeUpgrade"}); assertNotEquals(err.toString().trim(), 0, exitCode); - String message = "Finalize upgrade failed for .*" + newLine - + "Finalize upgrade successful for .*" + newLine; - assertOutputMatches(message); + String errMessage = "Finalize upgrade failed for ([\\s\\S]*)" + newLine; + String outMessage = "Finalize upgrade successful for .*" + newLine; + assertOutputMatches(outMessage, errMessage); } @Test (timeout = 30000) @@ -788,7 +797,8 @@ public void testListOpenFilesNN1DownNN2Down() throws Exception{ cluster.getDfsCluster().shutdownNameNode(1); int exitCode = admin.run(new String[] {"-listOpenFiles"}); assertNotEquals(err.toString().trim(), 0, exitCode); - String message = ".*" + newLine + "List open files failed." + newLine; + String message = "List open files failed." + newLine + + ".*" + newLine; assertOutputMatches(message); } } diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/tools/offlineImageViewer/TestOfflineImageViewerForAcl.java b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/tools/offlineImageViewer/TestOfflineImageViewerForAcl.java index b23ddf4afbcfb..31dec3f5e5c25 100644 --- a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/tools/offlineImageViewer/TestOfflineImageViewerForAcl.java +++ b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/tools/offlineImageViewer/TestOfflineImageViewerForAcl.java @@ -21,7 +21,6 @@ import java.io.ByteArrayInputStream; import java.io.ByteArrayOutputStream; import java.io.File; -import java.io.FileNotFoundException; import java.io.IOException; import java.io.InputStreamReader; import java.io.PrintStream; @@ -29,11 +28,9 @@ import java.io.StringReader; import java.net.HttpURLConnection; import java.net.URI; -import java.net.URISyntaxException; import java.net.URL; import java.util.HashMap; -import javax.xml.parsers.ParserConfigurationException; import javax.xml.parsers.SAXParser; import javax.xml.parsers.SAXParserFactory; @@ -54,7 +51,6 @@ import org.junit.BeforeClass; import org.junit.Test; import org.xml.sax.InputSource; -import org.xml.sax.SAXException; import org.xml.sax.helpers.DefaultHandler; import org.apache.hadoop.thirdparty.com.google.common.collect.Maps; diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/security/TestRefreshUserMappings.java b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/security/TestRefreshUserMappings.java index d410d3b045500..6806b6715b159 100644 --- a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/security/TestRefreshUserMappings.java +++ b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/security/TestRefreshUserMappings.java @@ -34,6 +34,8 @@ import java.net.URL; import java.net.URLDecoder; import java.util.ArrayList; +import java.util.Arrays; +import java.util.HashSet; import java.util.LinkedHashSet; import java.util.List; import java.util.Set; @@ -47,7 +49,6 @@ import org.apache.hadoop.security.authorize.DefaultImpersonationProvider; import org.apache.hadoop.security.authorize.ProxyUsers; import org.apache.hadoop.test.GenericTestUtils; -import org.apache.hadoop.util.Sets; import org.slf4j.Logger; import org.slf4j.LoggerFactory; import org.slf4j.event.Level; @@ -93,7 +94,7 @@ public Set getGroupsSet(String user) { LOG.info("Getting groups in MockUnixGroupsMapping"); String g1 = user + (10 * i + 1); String g2 = user + (10 * i + 2); - Set s = Sets.newHashSet(g1, g2); + Set s = new HashSet<>(Arrays.asList(g1, g2)); i++; return s; } diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/tools/TestJMXGet.java b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/tools/TestJMXGet.java index 2fe6102dee123..acd29655071cb 100644 --- a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/tools/TestJMXGet.java +++ b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/tools/TestJMXGet.java @@ -29,7 +29,6 @@ import java.io.PipedOutputStream; import java.io.PrintStream; import java.lang.management.ManagementFactory; -import java.util.Random; import java.util.Set; import java.util.concurrent.TimeoutException; @@ -37,9 +36,6 @@ import javax.management.ObjectName; import org.apache.hadoop.conf.Configuration; -import org.apache.hadoop.fs.CommonConfigurationKeys; -import org.apache.hadoop.fs.FSDataOutputStream; -import org.apache.hadoop.fs.FileSystem; import org.apache.hadoop.fs.FileUtil; import org.apache.hadoop.fs.Path; import org.apache.hadoop.hdfs.DFSTestUtil; diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/test/resources/testHDFSConf.xml b/hadoop-hdfs-project/hadoop-hdfs/src/test/resources/testHDFSConf.xml index 6a897aa609215..e6bf314a23d9c 100644 --- a/hadoop-hdfs-project/hadoop-hdfs/src/test/resources/testHDFSConf.xml +++ b/hadoop-hdfs-project/hadoop-hdfs/src/test/resources/testHDFSConf.xml @@ -16723,19 +16723,19 @@ RegexpAcrossOutputComparator - ^Rack: \/rack1\s*127\.0\.0\.1:\d+\s\([-.a-zA-Z0-9]+\)\s*127\.0\.0\.1:\d+\s\([-.a-zA-Z0-9]+\) + ^Rack: \/rack1\s*127\.0\.0\.1:\d+\s\([-.a-zA-Z0-9]+\)\sIn Service\s*127\.0\.0\.1:\d+\s\([-.a-zA-Z0-9]+\)\sIn Service RegexpAcrossOutputComparator - Rack: \/rack2\s*127\.0\.0\.1:\d+\s\([-.a-zA-Z0-9]+\)\s*127\.0\.0\.1:\d+\s\([-.a-zA-Z0-9]+\)\s*127\.0\.0\.1:\d+\s\([-.a-zA-Z0-9]+\) + Rack: \/rack2\s*127\.0\.0\.1:\d+\s\([-.a-zA-Z0-9]+\)\sIn Service\s*127\.0\.0\.1:\d+\s\([-.a-zA-Z0-9]+\)\sIn Service\s*127\.0\.0\.1:\d+\s\([-.a-zA-Z0-9]+\)\sIn Service RegexpAcrossOutputComparator - Rack: \/rack3\s*127\.0\.0\.1:\d+\s\([-.a-zA-Z0-9]+\) + Rack: \/rack3\s*127\.0\.0\.1:\d+\s\([-.a-zA-Z0-9]+\)\sIn Service RegexpAcrossOutputComparator - Rack: \/rack4\s*127\.0\.0\.1:\d+\s\([-.a-zA-Z0-9]+\)\s*127\.0\.0\.1:\d+\s\([-.a-zA-Z0-9]+\) + Rack: \/rack4\s*127\.0\.0\.1:\d+\s\([-.a-zA-Z0-9]+\)\sIn Service\s*127\.0\.0\.1:\d+\s\([-.a-zA-Z0-9]+\)\sIn Service diff --git a/hadoop-mapreduce-project/dev-support/jdiff/Apache_Hadoop_MapReduce_Common_3.3.3.xml b/hadoop-mapreduce-project/dev-support/jdiff/Apache_Hadoop_MapReduce_Common_3.3.3.xml new file mode 100644 index 0000000000000..9236c16299dea --- /dev/null +++ b/hadoop-mapreduce-project/dev-support/jdiff/Apache_Hadoop_MapReduce_Common_3.3.3.xml @@ -0,0 +1,113 @@ + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + diff --git a/hadoop-mapreduce-project/dev-support/jdiff/Apache_Hadoop_MapReduce_Core_3.3.3.xml b/hadoop-mapreduce-project/dev-support/jdiff/Apache_Hadoop_MapReduce_Core_3.3.3.xml new file mode 100644 index 0000000000000..ebe4c64b65fb0 --- /dev/null +++ b/hadoop-mapreduce-project/dev-support/jdiff/Apache_Hadoop_MapReduce_Core_3.3.3.xml @@ -0,0 +1,28087 @@ + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + FileStatus of a given cache file on hdfs + @throws IOException]]> + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + DistributedCache is a facility provided by the Map-Reduce + framework to cache files (text, archives, jars etc.) needed by applications. +

      + +

      Applications specify the files, via urls (hdfs:// or http://) to be cached + via the {@link org.apache.hadoop.mapred.JobConf}. The + DistributedCache assumes that the files specified via urls are + already present on the {@link FileSystem} at the path specified by the url + and are accessible by every machine in the cluster.

      + +

      The framework will copy the necessary files on to the worker node before + any tasks for the job are executed on that node. Its efficiency stems from + the fact that the files are only copied once per job and the ability to + cache archives which are un-archived on the workers.

      + +

      DistributedCache can be used to distribute simple, read-only + data/text files and/or more complex types such as archives, jars etc. + Archives (zip, tar and tgz/tar.gz files) are un-archived at the worker nodes. + Jars may be optionally added to the classpath of the tasks, a rudimentary + software distribution mechanism. Files have execution permissions. + In older version of Hadoop Map/Reduce users could optionally ask for symlinks + to be created in the working directory of the child task. In the current + version symlinks are always created. If the URL does not have a fragment + the name of the file or directory will be used. If multiple files or + directories map to the same link name, the last one added, will be used. All + others will not even be downloaded.

      + +

      DistributedCache tracks modification timestamps of the cache + files. Clearly the cache files should not be modified by the application + or externally while the job is executing.

      + +

      Here is an illustrative example on how to use the + DistributedCache:

      +

      +     // Setting up the cache for the application
      +
      +     1. Copy the requisite files to the FileSystem:
      +
      +     $ bin/hadoop fs -copyFromLocal lookup.dat /myapp/lookup.dat
      +     $ bin/hadoop fs -copyFromLocal map.zip /myapp/map.zip
      +     $ bin/hadoop fs -copyFromLocal mylib.jar /myapp/mylib.jar
      +     $ bin/hadoop fs -copyFromLocal mytar.tar /myapp/mytar.tar
      +     $ bin/hadoop fs -copyFromLocal mytgz.tgz /myapp/mytgz.tgz
      +     $ bin/hadoop fs -copyFromLocal mytargz.tar.gz /myapp/mytargz.tar.gz
      +
      +     2. Setup the application's JobConf:
      +
      +     JobConf job = new JobConf();
      +     DistributedCache.addCacheFile(new URI("/myapp/lookup.dat#lookup.dat"),
      +                                   job);
      +     DistributedCache.addCacheArchive(new URI("/myapp/map.zip"), job);
      +     DistributedCache.addFileToClassPath(new Path("/myapp/mylib.jar"), job);
      +     DistributedCache.addCacheArchive(new URI("/myapp/mytar.tar"), job);
      +     DistributedCache.addCacheArchive(new URI("/myapp/mytgz.tgz"), job);
      +     DistributedCache.addCacheArchive(new URI("/myapp/mytargz.tar.gz"), job);
      +
      +     3. Use the cached files in the {@link org.apache.hadoop.mapred.Mapper}
      +     or {@link org.apache.hadoop.mapred.Reducer}:
      +
      +     public static class MapClass extends MapReduceBase
      +     implements Mapper<K, V, K, V> {
      +
      +       private Path[] localArchives;
      +       private Path[] localFiles;
      +
      +       public void configure(JobConf job) {
      +         // Get the cached archives/files
      +         File f = new File("./map.zip/some/file/in/zip.txt");
      +       }
      +
      +       public void map(K key, V value,
      +                       OutputCollector<K, V> output, Reporter reporter)
      +       throws IOException {
      +         // Use data from the cached archives/files here
      +         // ...
      +         // ...
      +         output.collect(k, v);
      +       }
      +     }
      +
      + 
      + + It is also very common to use the DistributedCache by using + {@link org.apache.hadoop.util.GenericOptionsParser}. + + This class includes methods that should be used by users + (specifically those mentioned in the example above, as well + as {@link DistributedCache#addArchiveToClassPath(Path, Configuration)}), + as well as methods intended for use by the MapReduce framework + (e.g., {@link org.apache.hadoop.mapred.JobClient}). + + @see org.apache.hadoop.mapred.JobConf + @see org.apache.hadoop.mapred.JobClient + @see org.apache.hadoop.mapreduce.Job]]> + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + JobTracker, + as {@link JobTracker.State} + + {@link JobTracker.State} should no longer be used on M/R 2.x. The function + is kept to be compatible with M/R 1.x applications. + + @return the invalid state of the JobTracker.]]> + + + + + + + + + + + + + + ClusterStatus provides clients with information such as: +
        +
      1. + Size of the cluster. +
      2. +
      3. + Name of the trackers. +
      4. +
      5. + Task capacity of the cluster. +
      6. +
      7. + The number of currently running map and reduce tasks. +
      8. +
      9. + State of the JobTracker. +
      10. +
      11. + Details regarding black listed trackers. +
      12. +
      + +

      Clients can query for the latest ClusterStatus, via + {@link JobClient#getClusterStatus()}.

      + + @see JobClient]]> +
      +
      + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + Counters represent global counters, defined either by the + Map-Reduce framework or applications. Each Counter can be of + any {@link Enum} type.

      + +

      Counters are bunched into {@link Group}s, each comprising of + counters from a particular Enum class.]]> + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + Group of counters, comprising of counters from a particular + counter {@link Enum} class. + +

      Grouphandles localization of the class name and the + counter names.

      ]]> +
      +
      + + + + + + + + + + + + + + + + + + + + + + + + + FileInputFormat always returns + true. Implementations that may deal with non-splittable files must + override this method. + + FileInputFormat implementations can override this and return + false to ensure that individual input files are never split-up + so that {@link Mapper}s process entire files. + + @param fs the file system that the file is on + @param filename the file name to check + @return is this file splitable?]]> + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + FileInputFormat is the base class for all file-based + InputFormats. This provides a generic implementation of + {@link #getSplits(JobConf, int)}. + + Implementations of FileInputFormat can also override the + {@link #isSplitable(FileSystem, Path)} method to prevent input files + from being split-up in certain situations. Implementations that may + deal with non-splittable files must override this method, since + the default implementation assumes splitting is always possible.]]> + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + true if the job output should be compressed, + false otherwise]]> + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + Tasks' Side-Effect Files + +

      Note: The following is valid only if the {@link OutputCommitter} + is {@link FileOutputCommitter}. If OutputCommitter is not + a FileOutputCommitter, the task's temporary output + directory is same as {@link #getOutputPath(JobConf)} i.e. + ${mapreduce.output.fileoutputformat.outputdir}$

      + +

      Some applications need to create/write-to side-files, which differ from + the actual job-outputs. + +

      In such cases there could be issues with 2 instances of the same TIP + (running simultaneously e.g. speculative tasks) trying to open/write-to the + same file (path) on HDFS. Hence the application-writer will have to pick + unique names per task-attempt (e.g. using the attemptid, say + attempt_200709221812_0001_m_000000_0), not just per TIP.

      + +

      To get around this the Map-Reduce framework helps the application-writer + out by maintaining a special + ${mapreduce.output.fileoutputformat.outputdir}/_temporary/_${taskid} + sub-directory for each task-attempt on HDFS where the output of the + task-attempt goes. On successful completion of the task-attempt the files + in the ${mapreduce.output.fileoutputformat.outputdir}/_temporary/_${taskid} (only) + are promoted to ${mapreduce.output.fileoutputformat.outputdir}. Of course, the + framework discards the sub-directory of unsuccessful task-attempts. This + is completely transparent to the application.

      + +

      The application-writer can take advantage of this by creating any + side-files required in ${mapreduce.task.output.dir} during execution + of his reduce-task i.e. via {@link #getWorkOutputPath(JobConf)}, and the + framework will move them out similarly - thus she doesn't have to pick + unique paths per task-attempt.

      + +

      Note: the value of ${mapreduce.task.output.dir} during + execution of a particular task-attempt is actually + ${mapreduce.output.fileoutputformat.outputdir}/_temporary/_{$taskid}, and this value is + set by the map-reduce framework. So, just create any side-files in the + path returned by {@link #getWorkOutputPath(JobConf)} from map/reduce + task to take advantage of this feature.

      + +

      The entire discussion holds true for maps of jobs with + reducer=NONE (i.e. 0 reduces) since output of the map, in that case, + goes directly to HDFS.

      + + @return the {@link Path} to the task's temporary output directory + for the map-reduce job.]]> +
      +
      + + + + + + + + + + + + + The generated name can be used to create custom files from within the + different tasks for the job, the names for different tasks will not collide + with each other.

      + +

      The given name is postfixed with the task type, 'm' for maps, 'r' for + reduces and the task partition number. For example, give a name 'test' + running on the first map o the job the generated name will be + 'test-m-00000'.

      + + @param conf the configuration for the job. + @param name the name to make unique. + @return a unique name accross all tasks of the job.]]> +
      +
      + + + + + The path can be used to create custom files from within the map and + reduce tasks. The path name will be unique for each task. The path parent + will be the job output directory.

      ls + +

      This method uses the {@link #getUniqueName} method to make the file name + unique for the task.

      + + @param conf the configuration for the job. + @param name the name for the file. + @return a unique path accross all tasks of the job.]]> +
      +
      + + + +
      + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + +
      or + conf.setInt(FixedLengthInputFormat.FIXED_RECORD_LENGTH, recordLength); +

      + @see FixedLengthRecordReader]]> +
      +
      + + + + + + + + + + + + + + + + + + + + + + + Each {@link InputSplit} is then assigned to an individual {@link Mapper} + for processing.

      + +

      Note: The split is a logical split of the inputs and the + input files are not physically split into chunks. For e.g. a split could + be <input-file-path, start, offset> tuple. + + @param job job configuration. + @param numSplits the desired number of splits, a hint. + @return an array of {@link InputSplit}s for the job.]]> + + + + + + + + + It is the responsibility of the RecordReader to respect + record boundaries while processing the logical split to present a + record-oriented view to the individual task.

      + + @param split the {@link InputSplit} + @param job the job that this split belongs to + @return a {@link RecordReader}]]> +
      +
      + + InputFormat describes the input-specification for a + Map-Reduce job. + +

      The Map-Reduce framework relies on the InputFormat of the + job to:

      +

        +
      1. + Validate the input-specification of the job. +
      2. + Split-up the input file(s) into logical {@link InputSplit}s, each of + which is then assigned to an individual {@link Mapper}. +
      3. +
      4. + Provide the {@link RecordReader} implementation to be used to glean + input records from the logical InputSplit for processing by + the {@link Mapper}. +
      5. +
      + +

      The default behavior of file-based {@link InputFormat}s, typically + sub-classes of {@link FileInputFormat}, is to split the + input into logical {@link InputSplit}s based on the total size, in + bytes, of the input files. However, the {@link FileSystem} blocksize of + the input files is treated as an upper bound for input splits. A lower bound + on the split size can be set via + + mapreduce.input.fileinputformat.split.minsize.

      + +

      Clearly, logical splits based on input-size is insufficient for many + applications since record boundaries are to be respected. In such cases, the + application has to also implement a {@link RecordReader} on whom lies the + responsibilty to respect record-boundaries and present a record-oriented + view of the logical InputSplit to the individual task. + + @see InputSplit + @see RecordReader + @see JobClient + @see FileInputFormat]]> + + + + + + + + + + InputSplit. + + @return the number of bytes in the input split. + @throws IOException]]> + + + + + + InputSplit is + located as an array of Strings. + @throws IOException]]> + + + + InputSplit represents the data to be processed by an + individual {@link Mapper}. + +

      Typically, it presents a byte-oriented view on the input and is the + responsibility of {@link RecordReader} of the job to process this and present + a record-oriented view. + + @see InputFormat + @see RecordReader]]> + + + + + + + + + + SplitLocationInfos describing how the split + data is stored at each location. A null value indicates that all the + locations have the data stored on disk. + @throws IOException]]> + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + JobClient.]]> + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + jobid doesn't correspond to any known job. + @throws IOException]]> + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + JobClient is the primary interface for the user-job to interact + with the cluster. + + JobClient provides facilities to submit jobs, track their + progress, access component-tasks' reports/logs, get the Map-Reduce cluster + status information etc. + +

      The job submission process involves: +

        +
      1. + Checking the input and output specifications of the job. +
      2. +
      3. + Computing the {@link InputSplit}s for the job. +
      4. +
      5. + Setup the requisite accounting information for the {@link DistributedCache} + of the job, if necessary. +
      6. +
      7. + Copying the job's jar and configuration to the map-reduce system directory + on the distributed file-system. +
      8. +
      9. + Submitting the job to the cluster and optionally monitoring + it's status. +
      10. +
      + + Normally the user creates the application, describes various facets of the + job via {@link JobConf} and then uses the JobClient to submit + the job and monitor its progress. + +

      Here is an example on how to use JobClient:

      +

      +     // Create a new JobConf
      +     JobConf job = new JobConf(new Configuration(), MyJob.class);
      +     
      +     // Specify various job-specific parameters     
      +     job.setJobName("myjob");
      +     
      +     job.setInputPath(new Path("in"));
      +     job.setOutputPath(new Path("out"));
      +     
      +     job.setMapperClass(MyJob.MyMapper.class);
      +     job.setReducerClass(MyJob.MyReducer.class);
      +
      +     // Submit the job, then poll for progress until the job is complete
      +     JobClient.runJob(job);
      + 
      + + Job Control + +

      At times clients would chain map-reduce jobs to accomplish complex tasks + which cannot be done via a single map-reduce job. This is fairly easy since + the output of the job, typically, goes to distributed file-system and that + can be used as the input for the next job.

      + +

      However, this also means that the onus on ensuring jobs are complete + (success/failure) lies squarely on the clients. In such situations the + various job-control options are: +

        +
      1. + {@link #runJob(JobConf)} : submits the job and returns only after + the job has completed. +
      2. +
      3. + {@link #submitJob(JobConf)} : only submits the job, then poll the + returned handle to the {@link RunningJob} to query status and make + scheduling decisions. +
      4. +
      5. + {@link JobConf#setJobEndNotificationURI(String)} : setup a notification + on job-completion, thus avoiding polling. +
      6. +
      + + @see JobConf + @see ClusterStatus + @see Tool + @see DistributedCache]]> +
      + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + If the parameter {@code loadDefaults} is false, the new instance + will not load resources from the default files. + + @param loadDefaults specifies whether to load from the default files]]> + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + true if framework should keep the intermediate files + for failed tasks, false otherwise.]]> + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + true if the outputs of the maps are to be compressed, + false otherwise.]]> + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + This comparator should be provided if the equivalence rules for keys + for sorting the intermediates are different from those for grouping keys + before each call to + {@link Reducer#reduce(Object, java.util.Iterator, OutputCollector, Reporter)}.

      + +

      For key-value pairs (K1,V1) and (K2,V2), the values (V1, V2) are passed + in a single call to the reduce function if K1 and K2 compare as equal.

      + +

      Since {@link #setOutputKeyComparatorClass(Class)} can be used to control + how keys are sorted, this can be used in conjunction to simulate + secondary sort on values.

      + +

      Note: This is not a guarantee of the combiner sort being + stable in any sense. (In any case, with the order of available + map-outputs to the combiner being non-deterministic, it wouldn't make + that much sense.)

      + + @param theClass the comparator class to be used for grouping keys for the + combiner. It should implement RawComparator. + @see #setOutputKeyComparatorClass(Class)]]> +
      +
      + + + + This comparator should be provided if the equivalence rules for keys + for sorting the intermediates are different from those for grouping keys + before each call to + {@link Reducer#reduce(Object, java.util.Iterator, OutputCollector, Reporter)}.

      + +

      For key-value pairs (K1,V1) and (K2,V2), the values (V1, V2) are passed + in a single call to the reduce function if K1 and K2 compare as equal.

      + +

      Since {@link #setOutputKeyComparatorClass(Class)} can be used to control + how keys are sorted, this can be used in conjunction to simulate + secondary sort on values.

      + +

      Note: This is not a guarantee of the reduce sort being + stable in any sense. (In any case, with the order of available + map-outputs to the reduce being non-deterministic, it wouldn't make + that much sense.)

      + + @param theClass the comparator class to be used for grouping keys. + It should implement RawComparator. + @see #setOutputKeyComparatorClass(Class) + @see #setCombinerKeyGroupingComparator(Class)]]> +
      +
      + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + combiner class used to combine map-outputs + before being sent to the reducers. Typically the combiner is same as the + the {@link Reducer} for the job i.e. {@link #getReducerClass()}. + + @return the user-defined combiner class used to combine map-outputs.]]> + + + + + + combiner class used to combine map-outputs + before being sent to the reducers. + +

      The combiner is an application-specified aggregation operation, which + can help cut down the amount of data transferred between the + {@link Mapper} and the {@link Reducer}, leading to better performance.

      + +

      The framework may invoke the combiner 0, 1, or multiple times, in both + the mapper and reducer tasks. In general, the combiner is called as the + sort/merge result is written to disk. The combiner must: +

        +
      • be side-effect free
      • +
      • have the same input and output key types and the same input and + output value types
      • +
      + +

      Typically the combiner is same as the Reducer for the + job i.e. {@link #setReducerClass(Class)}.

      + + @param theClass the user-defined combiner class used to combine + map-outputs.]]> +
      +
      + + + true. + + @return true if speculative execution be used for this job, + false otherwise.]]> + + + + + + true if speculative execution + should be turned on, else false.]]> + + + + + true. + + @return true if speculative execution be + used for this job for map tasks, + false otherwise.]]> + + + + + + true if speculative execution + should be turned on for map tasks, + else false.]]> + + + + + true. + + @return true if speculative execution be used + for reduce tasks for this job, + false otherwise.]]> + + + + + + true if speculative execution + should be turned on for reduce tasks, + else false.]]> + + + + + 1. + + @return the number of map tasks for this job.]]> + + + + + + Note: This is only a hint to the framework. The actual + number of spawned map tasks depends on the number of {@link InputSplit}s + generated by the job's {@link InputFormat#getSplits(JobConf, int)}. + + A custom {@link InputFormat} is typically used to accurately control + the number of map tasks for the job.

      + + How many maps? + +

      The number of maps is usually driven by the total size of the inputs + i.e. total number of blocks of the input files.

      + +

      The right level of parallelism for maps seems to be around 10-100 maps + per-node, although it has been set up to 300 or so for very cpu-light map + tasks. Task setup takes awhile, so it is best if the maps take at least a + minute to execute.

      + +

      The default behavior of file-based {@link InputFormat}s is to split the + input into logical {@link InputSplit}s based on the total size, in + bytes, of input files. However, the {@link FileSystem} blocksize of the + input files is treated as an upper bound for input splits. A lower bound + on the split size can be set via + + mapreduce.input.fileinputformat.split.minsize.

      + +

      Thus, if you expect 10TB of input data and have a blocksize of 128MB, + you'll end up with 82,000 maps, unless {@link #setNumMapTasks(int)} is + used to set it even higher.

      + + @param n the number of map tasks for this job. + @see InputFormat#getSplits(JobConf, int) + @see FileInputFormat + @see FileSystem#getDefaultBlockSize() + @see FileStatus#getBlockSize()]]> +
      +
      + + + 1. + + @return the number of reduce tasks for this job.]]> + + + + + + How many reduces? + +

      The right number of reduces seems to be 0.95 or + 1.75 multiplied by ( + available memory for reduce tasks + (The value of this should be smaller than + numNodes * yarn.nodemanager.resource.memory-mb + since the resource of memory is shared by map tasks and other + applications) / + + mapreduce.reduce.memory.mb). +

      + +

      With 0.95 all of the reduces can launch immediately and + start transfering map outputs as the maps finish. With 1.75 + the faster nodes will finish their first round of reduces and launch a + second wave of reduces doing a much better job of load balancing.

      + +

      Increasing the number of reduces increases the framework overhead, but + increases load balancing and lowers the cost of failures.

      + +

      The scaling factors above are slightly less than whole numbers to + reserve a few reduce slots in the framework for speculative-tasks, failures + etc.

      + + Reducer NONE + +

      It is legal to set the number of reduce-tasks to zero.

      + +

      In this case the output of the map-tasks directly go to distributed + file-system, to the path set by + {@link FileOutputFormat#setOutputPath(JobConf, Path)}. Also, the + framework doesn't sort the map-outputs before writing it out to HDFS.

      + + @param n the number of reduce tasks for this job.]]> +
      +
      + + + mapreduce.map.maxattempts + property. If this property is not already set, the default is 4 attempts. + + @return the max number of attempts per map task.]]> + + + + + + + + + + + mapreduce.reduce.maxattempts + property. If this property is not already set, the default is 4 attempts. + + @return the max number of attempts per reduce task.]]> + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + noFailures, the + tasktracker is blacklisted for this job. + + @param noFailures maximum no. of failures of a given job per tasktracker.]]> + + + + + blacklisted for this job. + + @return the maximum no. of failures of a given job per tasktracker.]]> + + + + + failed. + + Defaults to zero, i.e. any failed map-task results in + the job being declared as {@link JobStatus#FAILED}. + + @return the maximum percentage of map tasks that can fail without + the job being aborted.]]> + + + + + + failed. + + @param percent the maximum percentage of map tasks that can fail without + the job being aborted.]]> + + + + + failed. + + Defaults to zero, i.e. any failed reduce-task results + in the job being declared as {@link JobStatus#FAILED}. + + @return the maximum percentage of reduce tasks that can fail without + the job being aborted.]]> + + + + + + failed. + + @param percent the maximum percentage of reduce tasks that can fail without + the job being aborted.]]> + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + The debug script can aid debugging of failed map tasks. The script is + given task's stdout, stderr, syslog, jobconf files as arguments.

      + +

      The debug command, run on the node where the map failed, is:

      +

      + $script $stdout $stderr $syslog $jobconf.
      + 
      + +

      The script file is distributed through {@link DistributedCache} + APIs. The script needs to be symlinked.

      + +

      Here is an example on how to submit a script +

      + job.setMapDebugScript("./myscript");
      + DistributedCache.createSymlink(job);
      + DistributedCache.addCacheFile("/debug/scripts/myscript#myscript");
      + 
      + + @param mDbgScript the script name]]> +
      +
      + + + + + + + + + The debug script can aid debugging of failed reduce tasks. The script + is given task's stdout, stderr, syslog, jobconf files as arguments.

      + +

      The debug command, run on the node where the map failed, is:

      +

      + $script $stdout $stderr $syslog $jobconf.
      + 
      + +

      The script file is distributed through {@link DistributedCache} + APIs. The script file needs to be symlinked

      + +

      Here is an example on how to submit a script +

      + job.setReduceDebugScript("./myscript");
      + DistributedCache.createSymlink(job);
      + DistributedCache.addCacheFile("/debug/scripts/myscript#myscript");
      + 
      + + @param rDbgScript the script name]]> +
      +
      + + + + + + + + null if it hasn't + been set. + @see #setJobEndNotificationURI(String)]]> + + + + + + The uri can contain 2 special parameters: $jobId and + $jobStatus. Those, if present, are replaced by the job's + identifier and completion-status respectively.

      + +

      This is typically used by application-writers to implement chaining of + Map-Reduce jobs in an asynchronous manner.

      + + @param uri the job end notification uri + @see JobStatus]]> +
      +
      + + + + + + + + + + + + + + + When a job starts, a shared directory is created at location + + ${mapreduce.cluster.local.dir}/taskTracker/$user/jobcache/$jobid/work/ . + This directory is exposed to the users through + mapreduce.job.local.dir . + So, the tasks can use this space + as scratch space and share files among them.

      + This value is available as System property also. + + @return The localized job specific shared directory]]> +
      +
      + + + + For backward compatibility, if the job configuration sets the + key {@link #MAPRED_TASK_MAXVMEM_PROPERTY} to a value different + from {@link #DISABLED_MEMORY_LIMIT}, that value will be used + after converting it from bytes to MB. + @return memory required to run a map task of the job, in MB,]]> + + + + + + + + + For backward compatibility, if the job configuration sets the + key {@link #MAPRED_TASK_MAXVMEM_PROPERTY} to a value different + from {@link #DISABLED_MEMORY_LIMIT}, that value will be used + after converting it from bytes to MB. + @return memory required to run a reduce task of the job, in MB.]]> + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + This method is deprecated. Now, different memory limits can be + set for map and reduce tasks of a job, in MB. +

      + For backward compatibility, if the job configuration sets the + key {@link #MAPRED_TASK_MAXVMEM_PROPERTY}, that value is returned. + Otherwise, this method will return the larger of the values returned by + {@link #getMemoryForMapTask()} and {@link #getMemoryForReduceTask()} + after converting them into bytes. + + @return Memory required to run a task of this job, in bytes. + @see #setMaxVirtualMemoryForTask(long) + @deprecated Use {@link #getMemoryForMapTask()} and + {@link #getMemoryForReduceTask()}]]> + + + + + + + mapred.task.maxvmem is split into + mapreduce.map.memory.mb + and mapreduce.map.memory.mb,mapred + each of the new key are set + as mapred.task.maxvmem / 1024 + as new values are in MB + + @param vmem Maximum amount of virtual memory in bytes any task of this job + can use. + @see #getMaxVirtualMemoryForTask() + @deprecated + Use {@link #setMemoryForMapTask(long mem)} and + Use {@link #setMemoryForReduceTask(long mem)}]]> + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + k1=v1,k2=v2. Further it can + reference existing environment variables via $key on + Linux or %key% on Windows. + + Example: +

        +
      • A=foo - This will set the env variable A to foo.
      • +
      + + @deprecated Use {@link #MAPRED_MAP_TASK_ENV} or + {@link #MAPRED_REDUCE_TASK_ENV}]]> +
      +
      + + + k1=v1,k2=v2. Further it can + reference existing environment variables via $key on + Linux or %key% on Windows. + + Example: +
        +
      • A=foo - This will set the env variable A to foo.
      • +
      + + You can also add environment variables individually by appending + .VARNAME to this configuration key, where VARNAME is + the name of the environment variable. + + Example: +
        +
      • mapreduce.map.env.VARNAME=value
      • +
      ]]> +
      +
      + + + k1=v1,k2=v2. Further it can + reference existing environment variables via $key on + Linux or %key% on Windows. + + Example: +
        +
      • A=foo - This will set the env variable A to foo.
      • +
      + + You can also add environment variables individually by appending + .VARNAME to this configuration key, where VARNAME is + the name of the environment variable. + + Example: +
        +
      • mapreduce.reduce.env.VARNAME=value
      • +
      ]]> +
      +
      + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + JobConf is the primary interface for a user to describe a + map-reduce job to the Hadoop framework for execution. The framework tries to + faithfully execute the job as-is described by JobConf, however: +
        +
      1. + Some configuration parameters might have been marked as + + final by administrators and hence cannot be altered. +
      2. +
      3. + While some job parameters are straight-forward to set + (e.g. {@link #setNumReduceTasks(int)}), some parameters interact subtly + with the rest of the framework and/or job-configuration and is relatively + more complex for the user to control finely + (e.g. {@link #setNumMapTasks(int)}). +
      4. +
      + +

      JobConf typically specifies the {@link Mapper}, combiner + (if any), {@link Partitioner}, {@link Reducer}, {@link InputFormat} and + {@link OutputFormat} implementations to be used etc. + +

      Optionally JobConf is used to specify other advanced facets + of the job such as Comparators to be used, files to be put in + the {@link DistributedCache}, whether or not intermediate and/or job outputs + are to be compressed (and how), debugability via user-provided scripts + ( {@link #setMapDebugScript(String)}/{@link #setReduceDebugScript(String)}), + for doing post-processing on task logs, task's stdout, stderr, syslog. + and etc.

      + +

      Here is an example on how to configure a job via JobConf:

      +

      +     // Create a new JobConf
      +     JobConf job = new JobConf(new Configuration(), MyJob.class);
      +     
      +     // Specify various job-specific parameters     
      +     job.setJobName("myjob");
      +     
      +     FileInputFormat.setInputPaths(job, new Path("in"));
      +     FileOutputFormat.setOutputPath(job, new Path("out"));
      +     
      +     job.setMapperClass(MyJob.MyMapper.class);
      +     job.setCombinerClass(MyJob.MyReducer.class);
      +     job.setReducerClass(MyJob.MyReducer.class);
      +     
      +     job.setInputFormat(SequenceFileInputFormat.class);
      +     job.setOutputFormat(SequenceFileOutputFormat.class);
      + 
      + + @see JobClient + @see ClusterStatus + @see Tool + @see DistributedCache]]> +
      + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + any job + run on the jobtracker started at 200707121733, we would use : +
       
      + JobID.getTaskIDsPattern("200707121733", null);
      + 
      + which will return : +
       "job_200707121733_[0-9]*" 
      + @param jtIdentifier jobTracker identifier, or null + @param jobId job number, or null + @return a regex pattern matching JobIDs]]> +
      +
      + + + An example JobID is : + job_200707121733_0003 , which represents the third job + running at the jobtracker started at 200707121733. +

      + Applications should never construct or parse JobID strings, but rather + use appropriate constructors or {@link #forName(String)} method. + + @see TaskID + @see TaskAttemptID]]> + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + Output pairs need not be of the same types as input pairs. A given + input pair may map to zero or many output pairs. Output pairs are + collected with calls to + {@link OutputCollector#collect(Object,Object)}.

      + +

      Applications can use the {@link Reporter} provided to report progress + or just indicate that they are alive. In scenarios where the application + takes significant amount of time to process individual key/value + pairs, this is crucial since the framework might assume that the task has + timed-out and kill that task. The other way of avoiding this is to set + + mapreduce.task.timeout to a high-enough value (or even zero for no + time-outs).

      + + @param key the input key. + @param value the input value. + @param output collects mapped keys and values. + @param reporter facility to report progress.]]> +
      + + + Maps are the individual tasks which transform input records into a + intermediate records. The transformed intermediate records need not be of + the same type as the input records. A given input pair may map to zero or + many output pairs.

      + +

      The Hadoop Map-Reduce framework spawns one map task for each + {@link InputSplit} generated by the {@link InputFormat} for the job. + Mapper implementations can access the {@link JobConf} for the + job via the {@link JobConfigurable#configure(JobConf)} and initialize + themselves. Similarly they can use the {@link Closeable#close()} method for + de-initialization.

      + +

      The framework then calls + {@link #map(Object, Object, OutputCollector, Reporter)} + for each key/value pair in the InputSplit for that task.

      + +

      All intermediate values associated with a given output key are + subsequently grouped by the framework, and passed to a {@link Reducer} to + determine the final output. Users can control the grouping by specifying + a Comparator via + {@link JobConf#setOutputKeyComparatorClass(Class)}.

      + +

      The grouped Mapper outputs are partitioned per + Reducer. Users can control which keys (and hence records) go to + which Reducer by implementing a custom {@link Partitioner}. + +

      Users can optionally specify a combiner, via + {@link JobConf#setCombinerClass(Class)}, to perform local aggregation of the + intermediate outputs, which helps to cut down the amount of data transferred + from the Mapper to the Reducer. + +

      The intermediate, grouped outputs are always stored in + {@link SequenceFile}s. Applications can specify if and how the intermediate + outputs are to be compressed and which {@link CompressionCodec}s are to be + used via the JobConf.

      + +

      If the job has + zero + reduces then the output of the Mapper is directly written + to the {@link FileSystem} without grouping by keys.

      + +

      Example:

      +

      +     public class MyMapper<K extends WritableComparable, V extends Writable> 
      +     extends MapReduceBase implements Mapper<K, V, K, V> {
      +     
      +       static enum MyCounters { NUM_RECORDS }
      +       
      +       private String mapTaskId;
      +       private String inputFile;
      +       private int noRecords = 0;
      +       
      +       public void configure(JobConf job) {
      +         mapTaskId = job.get(JobContext.TASK_ATTEMPT_ID);
      +         inputFile = job.get(JobContext.MAP_INPUT_FILE);
      +       }
      +       
      +       public void map(K key, V val,
      +                       OutputCollector<K, V> output, Reporter reporter)
      +       throws IOException {
      +         // Process the <key, value> pair (assume this takes a while)
      +         // ...
      +         // ...
      +         
      +         // Let the framework know that we are alive, and kicking!
      +         // reporter.progress();
      +         
      +         // Process some more
      +         // ...
      +         // ...
      +         
      +         // Increment the no. of <key, value> pairs processed
      +         ++noRecords;
      +
      +         // Increment counters
      +         reporter.incrCounter(NUM_RECORDS, 1);
      +        
      +         // Every 100 records update application-level status
      +         if ((noRecords%100) == 0) {
      +           reporter.setStatus(mapTaskId + " processed " + noRecords + 
      +                              " from input-file: " + inputFile); 
      +         }
      +         
      +         // Output the result
      +         output.collect(key, val);
      +       }
      +     }
      + 
      + +

      Applications may write a custom {@link MapRunnable} to exert greater + control on map processing e.g. multi-threaded Mappers etc.

      + + @see JobConf + @see InputFormat + @see Partitioner + @see Reducer + @see MapReduceBase + @see MapRunnable + @see SequenceFile]]> +
      + + + + + + + + + + + + + + + + + + + + + + Provides default no-op implementations for a few methods, most non-trivial + applications need to override some of them.

      ]]> +
      +
      + + + + + + + + + + + <key, value> pairs. + +

      Mapping of input records to output records is complete when this method + returns.

      + + @param input the {@link RecordReader} to read the input records. + @param output the {@link OutputCollector} to collect the outputrecords. + @param reporter {@link Reporter} to report progress, status-updates etc. + @throws IOException]]> +
      +
      + + Custom implementations of MapRunnable can exert greater + control on map processing e.g. multi-threaded, asynchronous mappers etc.

      + + @see Mapper]]> +
      +
      + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + nearly + equal content length.
      + Subclasses implement {@link #getRecordReader(InputSplit, JobConf, Reporter)} + to construct RecordReader's for MultiFileSplit's. + @see MultiFileSplit]]> +
      +
      + + + + + + + + + + + + + MultiFileSplit can be used to implement {@link RecordReader}'s, with + reading one record per file. + @see FileSplit + @see MultiFileInputFormat]]> + + + + + + + + + + + + + + + <key, value> pairs output by {@link Mapper}s + and {@link Reducer}s. + +

      OutputCollector is the generalization of the facility + provided by the Map-Reduce framework to collect data output by either the + Mapper or the Reducer i.e. intermediate outputs + or the output of the job.

      ]]> +
      +
      + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + true if task output recovery is supported, + false otherwise + @throws IOException + @see #recoverTask(TaskAttemptContext)]]> + + + + + + + true repeatable job commit is supported, + false otherwise + @throws IOException]]> + + + + + + + + + + + OutputCommitter. This is called from the application master + process, but it is called individually for each task. + + If an exception is thrown the task will be attempted again. + + @param taskContext Context of the task whose output is being recovered + @throws IOException]]> + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + OutputCommitter describes the commit of task output for a + Map-Reduce job. + +

      The Map-Reduce framework relies on the OutputCommitter of + the job to:

      +

        +
      1. + Setup the job during initialization. For example, create the temporary + output directory for the job during the initialization of the job. +
      2. +
      3. + Cleanup the job after the job completion. For example, remove the + temporary output directory after the job completion. +
      4. +
      5. + Setup the task temporary output. +
      6. +
      7. + Check whether a task needs a commit. This is to avoid the commit + procedure if a task does not need commit. +
      8. +
      9. + Commit of the task output. +
      10. +
      11. + Discard the task commit. +
      12. +
      + The methods in this class can be called from several different processes and + from several different contexts. It is important to know which process and + which context each is called from. Each method should be marked accordingly + in its documentation. It is also important to note that not all methods are + guaranteed to be called once and only once. If a method is not guaranteed to + have this property the output committer needs to handle this appropriately. + Also note it will only be in rare situations where they may be called + multiple times for the same task. + + @see FileOutputCommitter + @see JobContext + @see TaskAttemptContext]]> +
      +
      + + + + + + + + + + + + + + + + + + + This is to validate the output specification for the job when it is + a job is submitted. Typically checks that it does not already exist, + throwing an exception when it already exists, so that output is not + overwritten.

      + + Implementations which write to filesystems which support delegation + tokens usually collect the tokens for the destination path(s) + and attach them to the job configuration. + @param ignored + @param job job configuration. + @throws IOException when output should not be attempted]]> +
      +
      + + OutputFormat describes the output-specification for a + Map-Reduce job. + +

      The Map-Reduce framework relies on the OutputFormat of the + job to:

      +

        +
      1. + Validate the output-specification of the job. For e.g. check that the + output directory doesn't already exist. +
      2. + Provide the {@link RecordWriter} implementation to be used to write out + the output files of the job. Output files are stored in a + {@link FileSystem}. +
      3. +
      + + @see RecordWriter + @see JobConf]]> +
      +
      + + + + + + + + + + + + + + + + + + + + + + + Typically a hash function on a all or a subset of the key.

      + + @param key the key to be paritioned. + @param value the entry value. + @param numPartitions the total number of partitions. + @return the partition number for the key.]]> +
      +
      + + Partitioner controls the partitioning of the keys of the + intermediate map-outputs. The key (or a subset of the key) is used to derive + the partition, typically by a hash function. The total number of partitions + is the same as the number of reduce tasks for the job. Hence this controls + which of the m reduce tasks the intermediate key (and hence the + record) is sent for reduction.

      + +

      Note: A Partitioner is created only when there are multiple + reducers.

      + + @see Reducer]]> +
      +
      + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + 0.0 to 1.0. + @throws IOException]]> + + + + RecordReader reads <key, value> pairs from an + {@link InputSplit}. + +

      RecordReader, typically, converts the byte-oriented view of + the input, provided by the InputSplit, and presents a + record-oriented view for the {@link Mapper} and {@link Reducer} tasks for + processing. It thus assumes the responsibility of processing record + boundaries and presenting the tasks with keys and values.

      + + @see InputSplit + @see InputFormat]]> +
      +
      + + + + + + + + + + + + + + + + RecordWriter to future operations. + + @param reporter facility to report progress. + @throws IOException]]> + + + + RecordWriter writes the output <key, value> pairs + to an output file. + +

      RecordWriter implementations write the job outputs to the + {@link FileSystem}. + + @see OutputFormat]]> + + + + + + + + + + + + + + + Reduces values for a given key. + +

      The framework calls this method for each + <key, (list of values)> pair in the grouped inputs. + Output values must be of the same type as input values. Input keys must + not be altered. The framework will reuse the key and value objects + that are passed into the reduce, therefore the application should clone + the objects they want to keep a copy of. In many cases, all values are + combined into zero or one value. +

      + +

      Output pairs are collected with calls to + {@link OutputCollector#collect(Object,Object)}.

      + +

      Applications can use the {@link Reporter} provided to report progress + or just indicate that they are alive. In scenarios where the application + takes a significant amount of time to process individual key/value + pairs, this is crucial since the framework might assume that the task has + timed-out and kill that task. The other way of avoiding this is to set + + mapreduce.task.timeout to a high-enough value (or even zero for no + time-outs).

      + + @param key the key. + @param values the list of values to reduce. + @param output to collect keys and combined values. + @param reporter facility to report progress.]]> +
      + + + The number of Reducers for the job is set by the user via + {@link JobConf#setNumReduceTasks(int)}. Reducer implementations + can access the {@link JobConf} for the job via the + {@link JobConfigurable#configure(JobConf)} method and initialize themselves. + Similarly they can use the {@link Closeable#close()} method for + de-initialization.

      + +

      Reducer has 3 primary phases:

      +
        +
      1. + + Shuffle + +

        Reducer is input the grouped output of a {@link Mapper}. + In the phase the framework, for each Reducer, fetches the + relevant partition of the output of all the Mappers, via HTTP. +

        +
      2. + +
      3. + Sort + +

        The framework groups Reducer inputs by keys + (since different Mappers may have output the same key) in this + stage.

        + +

        The shuffle and sort phases occur simultaneously i.e. while outputs are + being fetched they are merged.

        + + SecondarySort + +

        If equivalence rules for keys while grouping the intermediates are + different from those for grouping keys before reduction, then one may + specify a Comparator via + {@link JobConf#setOutputValueGroupingComparator(Class)}.Since + {@link JobConf#setOutputKeyComparatorClass(Class)} can be used to + control how intermediate keys are grouped, these can be used in conjunction + to simulate secondary sort on values.

        + + + For example, say that you want to find duplicate web pages and tag them + all with the url of the "best" known example. You would set up the job + like: +
          +
        • Map Input Key: url
        • +
        • Map Input Value: document
        • +
        • Map Output Key: document checksum, url pagerank
        • +
        • Map Output Value: url
        • +
        • Partitioner: by checksum
        • +
        • OutputKeyComparator: by checksum and then decreasing pagerank
        • +
        • OutputValueGroupingComparator: by checksum
        • +
        +
      4. + +
      5. + Reduce + +

        In this phase the + {@link #reduce(Object, Iterator, OutputCollector, Reporter)} + method is called for each <key, (list of values)> pair in + the grouped inputs.

        +

        The output of the reduce task is typically written to the + {@link FileSystem} via + {@link OutputCollector#collect(Object, Object)}.

        +
      6. +
      + +

      The output of the Reducer is not re-sorted.

      + +

      Example:

      +

      +     public class MyReducer<K extends WritableComparable, V extends Writable> 
      +     extends MapReduceBase implements Reducer<K, V, K, V> {
      +     
      +       static enum MyCounters { NUM_RECORDS }
      +        
      +       private String reduceTaskId;
      +       private int noKeys = 0;
      +       
      +       public void configure(JobConf job) {
      +         reduceTaskId = job.get(JobContext.TASK_ATTEMPT_ID);
      +       }
      +       
      +       public void reduce(K key, Iterator<V> values,
      +                          OutputCollector<K, V> output, 
      +                          Reporter reporter)
      +       throws IOException {
      +       
      +         // Process
      +         int noValues = 0;
      +         while (values.hasNext()) {
      +           V value = values.next();
      +           
      +           // Increment the no. of values for this key
      +           ++noValues;
      +           
      +           // Process the <key, value> pair (assume this takes a while)
      +           // ...
      +           // ...
      +           
      +           // Let the framework know that we are alive, and kicking!
      +           if ((noValues%10) == 0) {
      +             reporter.progress();
      +           }
      +         
      +           // Process some more
      +           // ...
      +           // ...
      +           
      +           // Output the <key, value> 
      +           output.collect(key, value);
      +         }
      +         
      +         // Increment the no. of <key, list of values> pairs processed
      +         ++noKeys;
      +         
      +         // Increment counters
      +         reporter.incrCounter(NUM_RECORDS, 1);
      +         
      +         // Every 100 keys update application-level status
      +         if ((noKeys%100) == 0) {
      +           reporter.setStatus(reduceTaskId + " processed " + noKeys);
      +         }
      +       }
      +     }
      + 
      + + @see Mapper + @see Partitioner + @see Reporter + @see MapReduceBase]]> +
      +
      + + + + + + + + + + + + + + Counter of the given group/name.]]> + + + + + + + Counter of the given group/name.]]> + + + + + + + Enum. + @param amount A non-negative amount by which the counter is to + be incremented.]]> + + + + + + + + + + + + + + InputSplit that the map is reading from. + @throws UnsupportedOperationException if called outside a mapper]]> + + + + + + + + + + + + + + {@link Mapper} and {@link Reducer} can use the Reporter + provided to report progress or just indicate that they are alive. In + scenarios where the application takes significant amount of time to + process individual key/value pairs, this is crucial since the framework + might assume that the task has timed-out and kill that task. + +

      Applications can also update {@link Counters} via the provided + Reporter .

      + + @see Progressable + @see Counters]]> +
      +
      + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + progress of the job's map-tasks, as a float between 0.0 + and 1.0. When all map tasks have completed, the function returns 1.0. + + @return the progress of the job's map-tasks. + @throws IOException]]> + + + + + + progress of the job's reduce-tasks, as a float between 0.0 + and 1.0. When all reduce tasks have completed, the function returns 1.0. + + @return the progress of the job's reduce-tasks. + @throws IOException]]> + + + + + + progress of the job's cleanup-tasks, as a float between 0.0 + and 1.0. When all cleanup tasks have completed, the function returns 1.0. + + @return the progress of the job's cleanup-tasks. + @throws IOException]]> + + + + + + progress of the job's setup-tasks, as a float between 0.0 + and 1.0. When all setup tasks have completed, the function returns 1.0. + + @return the progress of the job's setup-tasks. + @throws IOException]]> + + + + + + true if the job is complete, else false. + @throws IOException]]> + + + + + + true if the job succeeded, else false. + @throws IOException]]> + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + true if the job retired, else false. + @throws IOException]]> + + + + + + + + + + RunningJob is the user-interface to query for details on a + running Map-Reduce job. + +

      Clients can get hold of RunningJob via the {@link JobClient} + and then query the running-job for details such as name, configuration, + progress etc.

      + + @see JobClient]]> +
      +
      + + + + + + + + + + + + + + + + + + + + + + + + + This allows the user to specify the key class to be different + from the actual class ({@link BytesWritable}) used for writing

      + + @param conf the {@link JobConf} to modify + @param theClass the SequenceFile output key class.]]> +
      +
      + + + + + This allows the user to specify the value class to be different + from the actual class ({@link BytesWritable}) used for writing

      + + @param conf the {@link JobConf} to modify + @param theClass the SequenceFile output key class.]]> +
      +
      + + + + + + + + + + + + + + + + + + + + + + + + + + + +
      + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + true if auto increment + {@link SkipBadRecords#COUNTER_MAP_PROCESSED_RECORDS}. + false otherwise.]]> + + + + + + + + + + + + + true if auto increment + {@link SkipBadRecords#COUNTER_REDUCE_PROCESSED_GROUPS}. + false otherwise.]]> + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + Hadoop provides an optional mode of execution in which the bad records + are detected and skipped in further attempts. + +

      This feature can be used when map/reduce tasks crashes deterministically on + certain input. This happens due to bugs in the map/reduce function. The usual + course would be to fix these bugs. But sometimes this is not possible; + perhaps the bug is in third party libraries for which the source code is + not available. Due to this, the task never reaches to completion even with + multiple attempts and complete data for that task is lost.

      + +

      With this feature, only a small portion of data is lost surrounding + the bad record, which may be acceptable for some user applications. + see {@link SkipBadRecords#setMapperMaxSkipRecords(Configuration, long)}

      + +

      The skipping mode gets kicked off after certain no of failures + see {@link SkipBadRecords#setAttemptsToStartSkipping(Configuration, int)}

      + +

      In the skipping mode, the map/reduce task maintains the record range which + is getting processed at all times. Before giving the input to the + map/reduce function, it sends this record range to the Task tracker. + If task crashes, the Task tracker knows which one was the last reported + range. On further attempts that range get skipped.

      ]]> +
      +
      + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + all task attempt IDs + of any jobtracker, in any job, of the first + map task, we would use : +
       
      + TaskAttemptID.getTaskAttemptIDsPattern(null, null, true, 1, null);
      + 
      + which will return : +
       "attempt_[^_]*_[0-9]*_m_000001_[0-9]*" 
      + @param jtIdentifier jobTracker identifier, or null + @param jobId job number, or null + @param isMap whether the tip is a map, or null + @param taskId taskId number, or null + @param attemptId the task attempt number, or null + @return a regex pattern matching TaskAttemptIDs]]> +
      +
      + + + + + + + + all task attempt IDs + of any jobtracker, in any job, of the first + map task, we would use : +
       
      + TaskAttemptID.getTaskAttemptIDsPattern(null, null, TaskType.MAP, 1, null);
      + 
      + which will return : +
       "attempt_[^_]*_[0-9]*_m_000001_[0-9]*" 
      + @param jtIdentifier jobTracker identifier, or null + @param jobId job number, or null + @param type the {@link TaskType} + @param taskId taskId number, or null + @param attemptId the task attempt number, or null + @return a regex pattern matching TaskAttemptIDs]]> +
      +
      + + + An example TaskAttemptID is : + attempt_200707121733_0003_m_000005_0 , which represents the + zeroth task attempt for the fifth map task in the third job + running at the jobtracker started at 200707121733. +

      + Applications should never construct or parse TaskAttemptID strings + , but rather use appropriate constructors or {@link #forName(String)} + method. + + @see JobID + @see TaskID]]> + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + the first map task + of any jobtracker, of any job, we would use : +

       
      + TaskID.getTaskIDsPattern(null, null, true, 1);
      + 
      + which will return : +
       "task_[^_]*_[0-9]*_m_000001*" 
      + @param jtIdentifier jobTracker identifier, or null + @param jobId job number, or null + @param isMap whether the tip is a map, or null + @param taskId taskId number, or null + @return a regex pattern matching TaskIDs + @deprecated Use {@link TaskID#getTaskIDsPattern(String, Integer, TaskType, + Integer)}]]> +
      + + + + + + + + the first map task + of any jobtracker, of any job, we would use : +
       
      + TaskID.getTaskIDsPattern(null, null, true, 1);
      + 
      + which will return : +
       "task_[^_]*_[0-9]*_m_000001*" 
      + @param jtIdentifier jobTracker identifier, or null + @param jobId job number, or null + @param type the {@link TaskType}, or null + @param taskId taskId number, or null + @return a regex pattern matching TaskIDs]]> +
      +
      + + + + + + + An example TaskID is : + task_200707121733_0003_m_000005 , which represents the + fifth map task in the third job running at the jobtracker + started at 200707121733. +

      + Applications should never construct or parse TaskID strings + , but rather use appropriate constructors or {@link #forName(String)} + method. + + @see JobID + @see TaskAttemptID]]> + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + true if the Job was added.]]> + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + ([,]*) + func ::= tbl(,"") + class ::= @see java.lang.Class#forName(java.lang.String) + path ::= @see org.apache.hadoop.fs.Path#Path(java.lang.String) + } + Reads expression from the mapred.join.expr property and + user-supplied join types from mapred.join.define.<ident> + types. Paths supplied to tbl are given as input paths to the + InputFormat class listed. + @see #compose(java.lang.String, java.lang.Class, java.lang.String...)]]> + + + + + + + + + + + + + + + + + + + + + + + + + + + + + ,

      ) }]]> + + + + + + + + (tbl(,),tbl(,),...,tbl(,)) }]]> + + + + + + + + (tbl(,),tbl(,),...,tbl(,)) }]]> + + + + mapred.join.define.<ident> to a classname. In the expression + mapred.join.expr, the identifier will be assumed to be a + ComposableRecordReader. + mapred.join.keycomparator can be a classname used to compare keys + in the join. + @see #setFormat + @see JoinRecordReader + @see MultiFilterRecordReader]]> + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + ...... + }]]> + + + + + + + + + + + + + + + + + + + + + capacity children to position + id in the parent reader. + The id of a root CompositeRecordReader is -1 by convention, but relying + on this is not recommended.]]> + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + override(S1,S2,S3) will prefer values + from S3 over S2, and values from S2 over S1 for all keys + emitted from all sources.]]> + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + It has to be specified how key and values are passed from one element of + the chain to the next, by value or by reference. If a Mapper leverages the + assumed semantics that the key and values are not modified by the collector + 'by value' must be used. If the Mapper does not expect this semantics, as + an optimization to avoid serialization and deserialization 'by reference' + can be used. +

      + For the added Mapper the configuration given for it, + mapperConf, have precedence over the job's JobConf. This + precedence is in effect when the task is running. +

      + IMPORTANT: There is no need to specify the output key/value classes for the + ChainMapper, this is done by the addMapper for the last mapper in the chain +

      + + @param job job's JobConf to add the Mapper class. + @param klass the Mapper class to add. + @param inputKeyClass mapper input key class. + @param inputValueClass mapper input value class. + @param outputKeyClass mapper output key class. + @param outputValueClass mapper output value class. + @param byValue indicates if key/values should be passed by value + to the next Mapper in the chain, if any. + @param mapperConf a JobConf with the configuration for the Mapper + class. It is recommended to use a JobConf without default values using the + JobConf(boolean loadDefaults) constructor with FALSE.]]> + + + + + + + If this method is overriden super.configure(...) should be + invoked at the beginning of the overwriter method.]]> + + + + + + + + + + map(...) methods of the Mappers in the chain.]]> + + + + + + + If this method is overriden super.close() should be + invoked at the end of the overwriter method.]]> + + + + + The Mapper classes are invoked in a chained (or piped) fashion, the output of + the first becomes the input of the second, and so on until the last Mapper, + the output of the last Mapper will be written to the task's output. +

      + The key functionality of this feature is that the Mappers in the chain do not + need to be aware that they are executed in a chain. This enables having + reusable specialized Mappers that can be combined to perform composite + operations within a single task. +

      + Special care has to be taken when creating chains that the key/values output + by a Mapper are valid for the following Mapper in the chain. It is assumed + all Mappers and the Reduce in the chain use maching output and input key and + value classes as no conversion is done by the chaining code. +

      + Using the ChainMapper and the ChainReducer classes is possible to compose + Map/Reduce jobs that look like [MAP+ / REDUCE MAP*]. And + immediate benefit of this pattern is a dramatic reduction in disk IO. +

      + IMPORTANT: There is no need to specify the output key/value classes for the + ChainMapper, this is done by the addMapper for the last mapper in the chain. +

      + ChainMapper usage pattern: +

      +

      + ...
      + conf.setJobName("chain");
      + conf.setInputFormat(TextInputFormat.class);
      + conf.setOutputFormat(TextOutputFormat.class);
      +
      + JobConf mapAConf = new JobConf(false);
      + ...
      + ChainMapper.addMapper(conf, AMap.class, LongWritable.class, Text.class,
      +   Text.class, Text.class, true, mapAConf);
      +
      + JobConf mapBConf = new JobConf(false);
      + ...
      + ChainMapper.addMapper(conf, BMap.class, Text.class, Text.class,
      +   LongWritable.class, Text.class, false, mapBConf);
      +
      + JobConf reduceConf = new JobConf(false);
      + ...
      + ChainReducer.setReducer(conf, XReduce.class, LongWritable.class, Text.class,
      +   Text.class, Text.class, true, reduceConf);
      +
      + ChainReducer.addMapper(conf, CMap.class, Text.class, Text.class,
      +   LongWritable.class, Text.class, false, null);
      +
      + ChainReducer.addMapper(conf, DMap.class, LongWritable.class, Text.class,
      +   LongWritable.class, LongWritable.class, true, null);
      +
      + FileInputFormat.setInputPaths(conf, inDir);
      + FileOutputFormat.setOutputPath(conf, outDir);
      + ...
      +
      + JobClient jc = new JobClient(conf);
      + RunningJob job = jc.submitJob(conf);
      + ...
      + 
      ]]> +
      +
      + + + + + + + + + + + + + + + + + + + + + It has to be specified how key and values are passed from one element of + the chain to the next, by value or by reference. If a Reducer leverages the + assumed semantics that the key and values are not modified by the collector + 'by value' must be used. If the Reducer does not expect this semantics, as + an optimization to avoid serialization and deserialization 'by reference' + can be used. +

      + For the added Reducer the configuration given for it, + reducerConf, have precedence over the job's JobConf. This + precedence is in effect when the task is running. +

      + IMPORTANT: There is no need to specify the output key/value classes for the + ChainReducer, this is done by the setReducer or the addMapper for the last + element in the chain. + + @param job job's JobConf to add the Reducer class. + @param klass the Reducer class to add. + @param inputKeyClass reducer input key class. + @param inputValueClass reducer input value class. + @param outputKeyClass reducer output key class. + @param outputValueClass reducer output value class. + @param byValue indicates if key/values should be passed by value + to the next Mapper in the chain, if any. + @param reducerConf a JobConf with the configuration for the Reducer + class. It is recommended to use a JobConf without default values using the + JobConf(boolean loadDefaults) constructor with FALSE.]]> + + + + + + + + + + + + + + It has to be specified how key and values are passed from one element of + the chain to the next, by value or by reference. If a Mapper leverages the + assumed semantics that the key and values are not modified by the collector + 'by value' must be used. If the Mapper does not expect this semantics, as + an optimization to avoid serialization and deserialization 'by reference' + can be used. +

      + For the added Mapper the configuration given for it, + mapperConf, have precedence over the job's JobConf. This + precedence is in effect when the task is running. +

      + IMPORTANT: There is no need to specify the output key/value classes for the + ChainMapper, this is done by the addMapper for the last mapper in the chain + . + + @param job chain job's JobConf to add the Mapper class. + @param klass the Mapper class to add. + @param inputKeyClass mapper input key class. + @param inputValueClass mapper input value class. + @param outputKeyClass mapper output key class. + @param outputValueClass mapper output value class. + @param byValue indicates if key/values should be passed by value + to the next Mapper in the chain, if any. + @param mapperConf a JobConf with the configuration for the Mapper + class. It is recommended to use a JobConf without default values using the + JobConf(boolean loadDefaults) constructor with FALSE.]]> + + + + + + + If this method is overriden super.configure(...) should be + invoked at the beginning of the overwriter method.]]> + + + + + + + + + + reduce(...) method of the Reducer with the + map(...) methods of the Mappers in the chain.]]> + + + + + + + If this method is overriden super.close() should be + invoked at the end of the overwriter method.]]> + + + + + For each record output by the Reducer, the Mapper classes are invoked in a + chained (or piped) fashion, the output of the first becomes the input of the + second, and so on until the last Mapper, the output of the last Mapper will + be written to the task's output. +

      + The key functionality of this feature is that the Mappers in the chain do not + need to be aware that they are executed after the Reducer or in a chain. + This enables having reusable specialized Mappers that can be combined to + perform composite operations within a single task. +

      + Special care has to be taken when creating chains that the key/values output + by a Mapper are valid for the following Mapper in the chain. It is assumed + all Mappers and the Reduce in the chain use maching output and input key and + value classes as no conversion is done by the chaining code. +

      + Using the ChainMapper and the ChainReducer classes is possible to compose + Map/Reduce jobs that look like [MAP+ / REDUCE MAP*]. And + immediate benefit of this pattern is a dramatic reduction in disk IO. +

      + IMPORTANT: There is no need to specify the output key/value classes for the + ChainReducer, this is done by the setReducer or the addMapper for the last + element in the chain. +

      + ChainReducer usage pattern: +

      +

      + ...
      + conf.setJobName("chain");
      + conf.setInputFormat(TextInputFormat.class);
      + conf.setOutputFormat(TextOutputFormat.class);
      +
      + JobConf mapAConf = new JobConf(false);
      + ...
      + ChainMapper.addMapper(conf, AMap.class, LongWritable.class, Text.class,
      +   Text.class, Text.class, true, mapAConf);
      +
      + JobConf mapBConf = new JobConf(false);
      + ...
      + ChainMapper.addMapper(conf, BMap.class, Text.class, Text.class,
      +   LongWritable.class, Text.class, false, mapBConf);
      +
      + JobConf reduceConf = new JobConf(false);
      + ...
      + ChainReducer.setReducer(conf, XReduce.class, LongWritable.class, Text.class,
      +   Text.class, Text.class, true, reduceConf);
      +
      + ChainReducer.addMapper(conf, CMap.class, Text.class, Text.class,
      +   LongWritable.class, Text.class, false, null);
      +
      + ChainReducer.addMapper(conf, DMap.class, LongWritable.class, Text.class,
      +   LongWritable.class, LongWritable.class, true, null);
      +
      + FileInputFormat.setInputPaths(conf, inDir);
      + FileOutputFormat.setOutputPath(conf, outDir);
      + ...
      +
      + JobClient jc = new JobClient(conf);
      + RunningJob job = jc.submitJob(conf);
      + ...
      + 
      ]]> +
      +
      + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + RecordReader's for CombineFileSplit's. + @see CombineFileSplit]]> + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + CombineFileRecordReader. + + Subclassing is needed to get a concrete record reader wrapper because of the + constructor requirement. + + @see CombineFileRecordReader + @see CombineFileInputFormat]]> + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + CombineFileInputFormat-equivalent for + SequenceFileInputFormat. + + @see CombineFileInputFormat]]> + + + + + + + + + + + + + + + CombineFileInputFormat-equivalent for + TextInputFormat. + + @see CombineFileInputFormat]]> + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + true if the name output is multi, false + if it is single. If the name output is not defined it returns + false]]> + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + By default these counters are disabled. +

      + MultipleOutputs supports counters, by default the are disabled. + The counters group is the {@link MultipleOutputs} class name. +

      + The names of the counters are the same as the named outputs. For multi + named outputs the name of the counter is the concatenation of the named + output, and underscore '_' and the multiname. + + @param conf job conf to enableadd the named output. + @param enabled indicates if the counters will be enabled or not.]]> +
      +
      + + + + + By default these counters are disabled. +

      + MultipleOutputs supports counters, by default the are disabled. + The counters group is the {@link MultipleOutputs} class name. +

      + The names of the counters are the same as the named outputs. For multi + named outputs the name of the counter is the concatenation of the named + output, and underscore '_' and the multiname. + + + @param conf job conf to enableadd the named output. + @return TRUE if the counters are enabled, FALSE if they are disabled.]]> +
      +
      + + + + + + + + + + + + + + + + + + + + + + + + + + + If overriden subclasses must invoke super.close() at the + end of their close() + + @throws java.io.IOException thrown if any of the MultipleOutput files + could not be closed properly.]]> + + + + OutputCollector passed to + the map() and reduce() methods of the + Mapper and Reducer implementations. +

      + Each additional output, or named output, may be configured with its own + OutputFormat, with its own key class and with its own value + class. +

      + A named output can be a single file or a multi file. The later is referred as + a multi named output. +

      + A multi named output is an unbound set of files all sharing the same + OutputFormat, key class and value class configuration. +

      + When named outputs are used within a Mapper implementation, + key/values written to a name output are not part of the reduce phase, only + key/values written to the job OutputCollector are part of the + reduce phase. +

      + MultipleOutputs supports counters, by default the are disabled. The counters + group is the {@link MultipleOutputs} class name. +

      + The names of the counters are the same as the named outputs. For multi + named outputs the name of the counter is the concatenation of the named + output, and underscore '_' and the multiname. +

      + Job configuration usage pattern is: +

      +
      + JobConf conf = new JobConf();
      +
      + conf.setInputPath(inDir);
      + FileOutputFormat.setOutputPath(conf, outDir);
      +
      + conf.setMapperClass(MOMap.class);
      + conf.setReducerClass(MOReduce.class);
      + ...
      +
      + // Defines additional single text based output 'text' for the job
      + MultipleOutputs.addNamedOutput(conf, "text", TextOutputFormat.class,
      + LongWritable.class, Text.class);
      +
      + // Defines additional multi sequencefile based output 'sequence' for the
      + // job
      + MultipleOutputs.addMultiNamedOutput(conf, "seq",
      +   SequenceFileOutputFormat.class,
      +   LongWritable.class, Text.class);
      + ...
      +
      + JobClient jc = new JobClient();
      + RunningJob job = jc.submitJob(conf);
      +
      + ...
      + 
      +

      + Job configuration usage pattern is: +

      +
      + public class MOReduce implements
      +   Reducer<WritableComparable, Writable> {
      + private MultipleOutputs mos;
      +
      + public void configure(JobConf conf) {
      + ...
      + mos = new MultipleOutputs(conf);
      + }
      +
      + public void reduce(WritableComparable key, Iterator<Writable> values,
      + OutputCollector output, Reporter reporter)
      + throws IOException {
      + ...
      + mos.getCollector("text", reporter).collect(key, new Text("Hello"));
      + mos.getCollector("seq", "A", reporter).collect(key, new Text("Bye"));
      + mos.getCollector("seq", "B", reporter).collect(key, new Text("Chau"));
      + ...
      + }
      +
      + public void close() throws IOException {
      + mos.close();
      + ...
      + }
      +
      + }
      + 
      ]]> +
      +
      + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + It can be used instead of the default implementation, + of {@link org.apache.hadoop.mapred.MapRunner}, when the Map + operation is not CPU bound in order to improve throughput. +

      + Map implementations using this MapRunnable must be thread-safe. +

      + The Map-Reduce job has to be configured to use this MapRunnable class (using + the JobConf.setMapRunnerClass method) and + the number of threads the thread-pool can use with the + mapred.map.multithreadedrunner.threads property, its default + value is 10 threads. +

      ]]> + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + R reduces, there are R-1 + keys in the SequenceFile. + @deprecated Use + {@link #setPartitionFile(Configuration, Path)} + instead]]> + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + Cluster. + @throws IOException]]> + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + ClusterMetrics provides clients with information such as: +

        +
      1. + Size of the cluster. +
      2. +
      3. + Number of blacklisted and decommissioned trackers. +
      4. +
      5. + Slot capacity of the cluster. +
      6. +
      7. + The number of currently occupied/reserved map and reduce slots. +
      8. +
      9. + The number of currently running map and reduce tasks. +
      10. +
      11. + The number of job submissions. +
      12. +
      + +

      Clients can query for the latest ClusterMetrics, via + {@link Cluster#getClusterStatus()}.

      + + @see Cluster]]> +
      +
      + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + Counters represent global counters, defined either by the + Map-Reduce framework or applications. Each Counter is named by + an {@link Enum} and has a long for the value.

      + +

      Counters are bunched into Groups, each comprising of + counters from a particular Enum class.]]> + + + + + + + + + + + + + + + + + + + + + the type of counter + @param the type of counter group + @param counters the old counters object]]> + + + + Counters holds per job/task counters, defined either by the + Map-Reduce framework or applications. Each Counter can be of + any {@link Enum} type.

      + +

      Counters are bunched into {@link CounterGroup}s, each + comprising of counters from a particular Enum class.]]> + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + Each {@link InputSplit} is then assigned to an individual {@link Mapper} + for processing.

      + +

      Note: The split is a logical split of the inputs and the + input files are not physically split into chunks. For e.g. a split could + be <input-file-path, start, offset> tuple. The InputFormat + also creates the {@link RecordReader} to read the {@link InputSplit}. + + @param context job configuration. + @return an array of {@link InputSplit}s for the job.]]> + + + + + + + + + + + + + InputFormat describes the input-specification for a + Map-Reduce job. + +

      The Map-Reduce framework relies on the InputFormat of the + job to:

      +

        +
      1. + Validate the input-specification of the job. +
      2. + Split-up the input file(s) into logical {@link InputSplit}s, each of + which is then assigned to an individual {@link Mapper}. +
      3. +
      4. + Provide the {@link RecordReader} implementation to be used to glean + input records from the logical InputSplit for processing by + the {@link Mapper}. +
      5. +
      + +

      The default behavior of file-based {@link InputFormat}s, typically + sub-classes of {@link FileInputFormat}, is to split the + input into logical {@link InputSplit}s based on the total size, in + bytes, of the input files. However, the {@link FileSystem} blocksize of + the input files is treated as an upper bound for input splits. A lower bound + on the split size can be set via + + mapreduce.input.fileinputformat.split.minsize.

      + +

      Clearly, logical splits based on input-size is insufficient for many + applications since record boundaries are to respected. In such cases, the + application has to also implement a {@link RecordReader} on whom lies the + responsibility to respect record-boundaries and present a record-oriented + view of the logical InputSplit to the individual task. + + @see InputSplit + @see RecordReader + @see FileInputFormat]]> + + + + + + + + + + + + + + + + + + + + + + + + + SplitLocationInfos describing how the split + data is stored at each location. A null value indicates that all the + locations have the data stored on disk. + @throws IOException]]> + + + + InputSplit represents the data to be processed by an + individual {@link Mapper}. + +

      Typically, it presents a byte-oriented view on the input and is the + responsibility of {@link RecordReader} of the job to process this and present + a record-oriented view. + + @see InputFormat + @see RecordReader]]> + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + Job makes a copy of the Configuration so + that any necessary internal modifications do not reflect on the incoming + parameter. + + A Cluster will be created from the conf parameter only when it's needed. + + @param conf the configuration + @return the {@link Job} , with no connection to a cluster yet. + @throws IOException]]> + + + + + + + + Job makes a copy of the Configuration so + that any necessary internal modifications do not reflect on the incoming + parameter. + + @param conf the configuration + @return the {@link Job} , with no connection to a cluster yet. + @throws IOException]]> + + + + + + + + Job makes a copy of the Configuration so + that any necessary internal modifications do not reflect on the incoming + parameter. + + @param status job status + @param conf job configuration + @return the {@link Job} , with no connection to a cluster yet. + @throws IOException]]> + + + + + + + Job makes a copy of the Configuration so + that any necessary internal modifications do not reflect on the incoming + parameter. + + @param ignored + @return the {@link Job} , with no connection to a cluster yet. + @throws IOException + @deprecated Use {@link #getInstance()}]]> + + + + + + + + Job makes a copy of the Configuration so + that any necessary internal modifications do not reflect on the incoming + parameter. + + @param ignored + @param conf job configuration + @return the {@link Job} , with no connection to a cluster yet. + @throws IOException + @deprecated Use {@link #getInstance(Configuration)}]]> + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + progress of the job's map-tasks, as a float between 0.0 + and 1.0. When all map tasks have completed, the function returns 1.0. + + @return the progress of the job's map-tasks. + @throws IOException]]> + + + + + + progress of the job's reduce-tasks, as a float between 0.0 + and 1.0. When all reduce tasks have completed, the function returns 1.0. + + @return the progress of the job's reduce-tasks. + @throws IOException]]> + + + + + + + progress of the job's cleanup-tasks, as a float between 0.0 + and 1.0. When all cleanup tasks have completed, the function returns 1.0. + + @return the progress of the job's cleanup-tasks. + @throws IOException]]> + + + + + + progress of the job's setup-tasks, as a float between 0.0 + and 1.0. When all setup tasks have completed, the function returns 1.0. + + @return the progress of the job's setup-tasks. + @throws IOException]]> + + + + + + true if the job is complete, else false. + @throws IOException]]> + + + + + + true if the job succeeded, else false. + @throws IOException]]> + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + InputFormat to use + @throws IllegalStateException if the job is submitted]]> + + + + + + + OutputFormat to use + @throws IllegalStateException if the job is submitted]]> + + + + + + + Mapper to use + @throws IllegalStateException if the job is submitted]]> + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + Reducer to use + @throws IllegalStateException if the job is submitted]]> + + + + + + + Partitioner to use + @throws IllegalStateException if the job is submitted]]> + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + true if speculative execution + should be turned on, else false.]]> + + + + + + true if speculative execution + should be turned on for map tasks, + else false.]]> + + + + + + true if speculative execution + should be turned on for reduce tasks, + else false.]]> + + + + + + true, job-setup and job-cleanup will be + considered from {@link OutputCommitter} + else ignored.]]> + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + JobTracker is lost]]> + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + Job. + @throws IOException if fail to close.]]> + + + + + + + + + + + + + + + + + + + + + + + + + + It allows the user to configure the + job, submit it, control its execution, and query the state. The set methods + only work until the job is submitted, afterwards they will throw an + IllegalStateException.

      + +

      + Normally the user creates the application, describes various facets of the + job via {@link Job} and then submits the job and monitor its progress.

      + +

      Here is an example on how to submit a job:

      +

      +     // Create a new Job
      +     Job job = Job.getInstance();
      +     job.setJarByClass(MyJob.class);
      +     
      +     // Specify various job-specific parameters     
      +     job.setJobName("myjob");
      +     
      +     job.setInputPath(new Path("in"));
      +     job.setOutputPath(new Path("out"));
      +     
      +     job.setMapperClass(MyJob.MyMapper.class);
      +     job.setReducerClass(MyJob.MyReducer.class);
      +
      +     // Submit the job, then poll for progress until the job is complete
      +     job.waitForCompletion(true);
      + 
      ]]> +
      +
      + + + + + + + + + + + + + + + + + + + + + + 1. + @return the number of reduce tasks for this job.]]> + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + mapred.map.max.attempts + property. If this property is not already set, the default is 4 attempts. + + @return the max number of attempts per map task.]]> + + + + + mapred.reduce.max.attempts + property. If this property is not already set, the default is 4 attempts. + + @return the max number of attempts per reduce task.]]> + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + An example JobID is : + job_200707121733_0003 , which represents the third job + running at the jobtracker started at 200707121733. +

      + Applications should never construct or parse JobID strings, but rather + use appropriate constructors or {@link #forName(String)} method. + + @see TaskID + @see TaskAttemptID]]> + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + the key input type to the Mapper + @param the value input type to the Mapper + @param the key output type from the Mapper + @param the value output type from the Mapper]]> + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + Maps are the individual tasks which transform input records into a + intermediate records. The transformed intermediate records need not be of + the same type as the input records. A given input pair may map to zero or + many output pairs.

      + +

      The Hadoop Map-Reduce framework spawns one map task for each + {@link InputSplit} generated by the {@link InputFormat} for the job. + Mapper implementations can access the {@link Configuration} for + the job via the {@link JobContext#getConfiguration()}. + +

      The framework first calls + {@link #setup(org.apache.hadoop.mapreduce.Mapper.Context)}, followed by + {@link #map(Object, Object, org.apache.hadoop.mapreduce.Mapper.Context)} + for each key/value pair in the InputSplit. Finally + {@link #cleanup(org.apache.hadoop.mapreduce.Mapper.Context)} is called.

      + +

      All intermediate values associated with a given output key are + subsequently grouped by the framework, and passed to a {@link Reducer} to + determine the final output. Users can control the sorting and grouping by + specifying two key {@link RawComparator} classes.

      + +

      The Mapper outputs are partitioned per + Reducer. Users can control which keys (and hence records) go to + which Reducer by implementing a custom {@link Partitioner}. + +

      Users can optionally specify a combiner, via + {@link Job#setCombinerClass(Class)}, to perform local aggregation of the + intermediate outputs, which helps to cut down the amount of data transferred + from the Mapper to the Reducer. + +

      Applications can specify if and how the intermediate + outputs are to be compressed and which {@link CompressionCodec}s are to be + used via the Configuration.

      + +

      If the job has zero + reduces then the output of the Mapper is directly written + to the {@link OutputFormat} without sorting by keys.

      + +

      Example:

      +

      + public class TokenCounterMapper 
      +     extends Mapper<Object, Text, Text, IntWritable>{
      +    
      +   private final static IntWritable one = new IntWritable(1);
      +   private Text word = new Text();
      +   
      +   public void map(Object key, Text value, Context context) throws IOException, InterruptedException {
      +     StringTokenizer itr = new StringTokenizer(value.toString());
      +     while (itr.hasMoreTokens()) {
      +       word.set(itr.nextToken());
      +       context.write(word, one);
      +     }
      +   }
      + }
      + 
      + +

      Applications may override the + {@link #run(org.apache.hadoop.mapreduce.Mapper.Context)} method to exert + greater control on map processing e.g. multi-threaded Mappers + etc.

      + + @see InputFormat + @see JobContext + @see Partitioner + @see Reducer]]> +
      +
      + + + + + + + + + + + + + + + + + + + + + + + + + + MarkableIterator is a wrapper iterator class that + implements the {@link MarkableIteratorInterface}.]]> + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + true if task output recovery is supported, + false otherwise + @see #recoverTask(TaskAttemptContext) + @deprecated Use {@link #isRecoverySupported(JobContext)} instead.]]> + + + + + + + true repeatable job commit is supported, + false otherwise + @throws IOException]]> + + + + + + + true if task output recovery is supported, + false otherwise + @throws IOException + @see #recoverTask(TaskAttemptContext)]]> + + + + + + + OutputCommitter. This is called from the application master + process, but it is called individually for each task. + + If an exception is thrown the task will be attempted again. + + This may be called multiple times for the same task. But from different + application attempts. + + @param taskContext Context of the task whose output is being recovered + @throws IOException]]> + + + + OutputCommitter describes the commit of task output for a + Map-Reduce job. + +

      The Map-Reduce framework relies on the OutputCommitter of + the job to:

      +

        +
      1. + Setup the job during initialization. For example, create the temporary + output directory for the job during the initialization of the job. +
      2. +
      3. + Cleanup the job after the job completion. For example, remove the + temporary output directory after the job completion. +
      4. +
      5. + Setup the task temporary output. +
      6. +
      7. + Check whether a task needs a commit. This is to avoid the commit + procedure if a task does not need commit. +
      8. +
      9. + Commit of the task output. +
      10. +
      11. + Discard the task commit. +
      12. +
      + The methods in this class can be called from several different processes and + from several different contexts. It is important to know which process and + which context each is called from. Each method should be marked accordingly + in its documentation. It is also important to note that not all methods are + guaranteed to be called once and only once. If a method is not guaranteed to + have this property the output committer needs to handle this appropriately. + Also note it will only be in rare situations where they may be called + multiple times for the same task. + + @see org.apache.hadoop.mapreduce.lib.output.FileOutputCommitter + @see JobContext + @see TaskAttemptContext]]> +
      +
      + + + + + + + + + + + + + + + + + + + This is to validate the output specification for the job when it is + a job is submitted. Typically checks that it does not already exist, + throwing an exception when it already exists, so that output is not + overwritten.

      + + Implementations which write to filesystems which support delegation + tokens usually collect the tokens for the destination path(s) + and attach them to the job context's JobConf. + @param context information about the job + @throws IOException when output should not be attempted]]> +
      +
      + + + + + + + + + + OutputFormat describes the output-specification for a + Map-Reduce job. + +

      The Map-Reduce framework relies on the OutputFormat of the + job to:

      +

        +
      1. + Validate the output-specification of the job. For e.g. check that the + output directory doesn't already exist. +
      2. + Provide the {@link RecordWriter} implementation to be used to write out + the output files of the job. Output files are stored in a + {@link FileSystem}. +
      3. +
      + + @see RecordWriter]]> +
      +
      + + + + + + + + + + + Typically a hash function on a all or a subset of the key.

      + + @param key the key to be partioned. + @param value the entry value. + @param numPartitions the total number of partitions. + @return the partition number for the key.]]> +
      +
      + + Partitioner controls the partitioning of the keys of the + intermediate map-outputs. The key (or a subset of the key) is used to derive + the partition, typically by a hash function. The total number of partitions + is the same as the number of reduce tasks for the job. Hence this controls + which of the m reduce tasks the intermediate key (and hence the + record) is sent for reduction.

      + +

      Note: A Partitioner is created only when there are multiple + reducers.

      + +

      Note: If you require your Partitioner class to obtain the Job's + configuration object, implement the {@link Configurable} interface.

      + + @see Reducer]]> +
      +
      + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + "N/A" + + @return Scheduling information associated to particular Job Queue]]> + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + @param ]]> + + + + + + + + + + + + + + + + + + + + + + RecordWriter to future operations. + + @param context the context of the task + @throws IOException]]> + + + + RecordWriter writes the output <key, value> pairs + to an output file. + +

      RecordWriter implementations write the job outputs to the + {@link FileSystem}. + + @see OutputFormat]]> + + + + + + + + + + + + + + + + + + + + + + the class of the input keys + @param the class of the input values + @param the class of the output keys + @param the class of the output values]]> + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + Reducer implementations + can access the {@link Configuration} for the job via the + {@link JobContext#getConfiguration()} method.

      + +

      Reducer has 3 primary phases:

      +
        +
      1. + + Shuffle + +

        The Reducer copies the sorted output from each + {@link Mapper} using HTTP across the network.

        +
      2. + +
      3. + Sort + +

        The framework merge sorts Reducer inputs by + keys + (since different Mappers may have output the same key).

        + +

        The shuffle and sort phases occur simultaneously i.e. while outputs are + being fetched they are merged.

        + + SecondarySort + +

        To achieve a secondary sort on the values returned by the value + iterator, the application should extend the key with the secondary + key and define a grouping comparator. The keys will be sorted using the + entire key, but will be grouped using the grouping comparator to decide + which keys and values are sent in the same call to reduce.The grouping + comparator is specified via + {@link Job#setGroupingComparatorClass(Class)}. The sort order is + controlled by + {@link Job#setSortComparatorClass(Class)}.

        + + + For example, say that you want to find duplicate web pages and tag them + all with the url of the "best" known example. You would set up the job + like: +
          +
        • Map Input Key: url
        • +
        • Map Input Value: document
        • +
        • Map Output Key: document checksum, url pagerank
        • +
        • Map Output Value: url
        • +
        • Partitioner: by checksum
        • +
        • OutputKeyComparator: by checksum and then decreasing pagerank
        • +
        • OutputValueGroupingComparator: by checksum
        • +
        +
      4. + +
      5. + Reduce + +

        In this phase the + {@link #reduce(Object, Iterable, org.apache.hadoop.mapreduce.Reducer.Context)} + method is called for each <key, (collection of values)> in + the sorted inputs.

        +

        The output of the reduce task is typically written to a + {@link RecordWriter} via + {@link Context#write(Object, Object)}.

        +
      6. +
      + +

      The output of the Reducer is not re-sorted.

      + +

      Example:

      +

      + public class IntSumReducer<Key> extends Reducer<Key,IntWritable,
      +                                                 Key,IntWritable> {
      +   private IntWritable result = new IntWritable();
      + 
      +   public void reduce(Key key, Iterable<IntWritable> values,
      +                      Context context) throws IOException, InterruptedException {
      +     int sum = 0;
      +     for (IntWritable val : values) {
      +       sum += val.get();
      +     }
      +     result.set(sum);
      +     context.write(key, result);
      +   }
      + }
      + 
      + + @see Mapper + @see Partitioner]]> +
      +
      + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + counterName. + @param counterName counter name + @return the Counter for the given counterName]]> + + + + + + + groupName and + counterName. + @param counterName counter name + @return the Counter for the given groupName and + counterName]]> + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + An example TaskAttemptID is : + attempt_200707121733_0003_m_000005_0 , which represents the + zeroth task attempt for the fifth map task in the third job + running at the jobtracker started at 200707121733. +

      + Applications should never construct or parse TaskAttemptID strings + , but rather use appropriate constructors or {@link #forName(String)} + method. + + @see JobID + @see TaskID]]> + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + An example TaskID is : + task_200707121733_0003_m_000005 , which represents the + fifth map task in the third job running at the jobtracker + started at 200707121733. +

      + Applications should never construct or parse TaskID strings + , but rather use appropriate constructors or {@link #forName(String)} + method. + + @see JobID + @see TaskAttemptID]]> + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + OutputCommitter for the task-attempt]]> + + + + the input key type for the task + @param the input value type for the task + @param the output key type for the task + @param the output value type for the task]]> + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + type of the other counter + @param type of the other counter group + @param counters the counters object to copy + @param groupFactory the factory for new groups]]> + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + type of counter inside the counters + @param type of group inside the counters]]> + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + type of the counter for the group]]> + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + The key and values are passed from one element of the chain to the next, by + value. For the added Mapper the configuration given for it, + mapperConf, have precedence over the job's Configuration. This + precedence is in effect when the task is running. +

      +

      + IMPORTANT: There is no need to specify the output key/value classes for the + ChainMapper, this is done by the addMapper for the last mapper in the chain +

      + + @param job + The job. + @param klass + the Mapper class to add. + @param inputKeyClass + mapper input key class. + @param inputValueClass + mapper input value class. + @param outputKeyClass + mapper output key class. + @param outputValueClass + mapper output value class. + @param mapperConf + a configuration for the Mapper class. It is recommended to use a + Configuration without default values using the + Configuration(boolean loadDefaults) constructor with + FALSE.]]> +
      + + + + + + + + + + + + The Mapper classes are invoked in a chained (or piped) fashion, the output of + the first becomes the input of the second, and so on until the last Mapper, + the output of the last Mapper will be written to the task's output. +

      +

      + The key functionality of this feature is that the Mappers in the chain do not + need to be aware that they are executed in a chain. This enables having + reusable specialized Mappers that can be combined to perform composite + operations within a single task. +

      +

      + Special care has to be taken when creating chains that the key/values output + by a Mapper are valid for the following Mapper in the chain. It is assumed + all Mappers and the Reduce in the chain use matching output and input key and + value classes as no conversion is done by the chaining code. +

      +

      + Using the ChainMapper and the ChainReducer classes is possible to compose + Map/Reduce jobs that look like [MAP+ / REDUCE MAP*]. And + immediate benefit of this pattern is a dramatic reduction in disk IO. +

      +

      + IMPORTANT: There is no need to specify the output key/value classes for the + ChainMapper, this is done by the addMapper for the last mapper in the chain. +

      + ChainMapper usage pattern: +

      + +

      + ...
      + Job = new Job(conf);
      +
      + Configuration mapAConf = new Configuration(false);
      + ...
      + ChainMapper.addMapper(job, AMap.class, LongWritable.class, Text.class,
      +   Text.class, Text.class, true, mapAConf);
      +
      + Configuration mapBConf = new Configuration(false);
      + ...
      + ChainMapper.addMapper(job, BMap.class, Text.class, Text.class,
      +   LongWritable.class, Text.class, false, mapBConf);
      +
      + ...
      +
      + job.waitForComplettion(true);
      + ...
      + 
      ]]> +
      +
      + + + + + + + + + + + + + + + + The key and values are passed from one element of the chain to the next, by + value. For the added Reducer the configuration given for it, + reducerConf, have precedence over the job's Configuration. + This precedence is in effect when the task is running. +

      +

      + IMPORTANT: There is no need to specify the output key/value classes for the + ChainReducer, this is done by the setReducer or the addMapper for the last + element in the chain. +

      + + @param job + the job + @param klass + the Reducer class to add. + @param inputKeyClass + reducer input key class. + @param inputValueClass + reducer input value class. + @param outputKeyClass + reducer output key class. + @param outputValueClass + reducer output value class. + @param reducerConf + a configuration for the Reducer class. It is recommended to use a + Configuration without default values using the + Configuration(boolean loadDefaults) constructor with + FALSE.]]> +
      +
      + + + + + + + + + + + + The key and values are passed from one element of the chain to the next, by + value For the added Mapper the configuration given for it, + mapperConf, have precedence over the job's Configuration. This + precedence is in effect when the task is running. +

      +

      + IMPORTANT: There is no need to specify the output key/value classes for the + ChainMapper, this is done by the addMapper for the last mapper in the + chain. +

      + + @param job + The job. + @param klass + the Mapper class to add. + @param inputKeyClass + mapper input key class. + @param inputValueClass + mapper input value class. + @param outputKeyClass + mapper output key class. + @param outputValueClass + mapper output value class. + @param mapperConf + a configuration for the Mapper class. It is recommended to use a + Configuration without default values using the + Configuration(boolean loadDefaults) constructor with + FALSE.]]> +
      +
      + + + + + + + + + + + For each record output by the Reducer, the Mapper classes are invoked in a + chained (or piped) fashion. The output of the reducer becomes the input of + the first mapper and output of first becomes the input of the second, and so + on until the last Mapper, the output of the last Mapper will be written to + the task's output. +

      +

      + The key functionality of this feature is that the Mappers in the chain do not + need to be aware that they are executed after the Reducer or in a chain. This + enables having reusable specialized Mappers that can be combined to perform + composite operations within a single task. +

      +

      + Special care has to be taken when creating chains that the key/values output + by a Mapper are valid for the following Mapper in the chain. It is assumed + all Mappers and the Reduce in the chain use matching output and input key and + value classes as no conversion is done by the chaining code. +

      +

      Using the ChainMapper and the ChainReducer classes is possible to + compose Map/Reduce jobs that look like [MAP+ / REDUCE MAP*]. And + immediate benefit of this pattern is a dramatic reduction in disk IO.

      +

      + IMPORTANT: There is no need to specify the output key/value classes for the + ChainReducer, this is done by the setReducer or the addMapper for the last + element in the chain. +

      + ChainReducer usage pattern: +

      + +

      + ...
      + Job = new Job(conf);
      + ....
      +
      + Configuration reduceConf = new Configuration(false);
      + ...
      + ChainReducer.setReducer(job, XReduce.class, LongWritable.class, Text.class,
      +   Text.class, Text.class, true, reduceConf);
      +
      + ChainReducer.addMapper(job, CMap.class, Text.class, Text.class,
      +   LongWritable.class, Text.class, false, null);
      +
      + ChainReducer.addMapper(job, DMap.class, LongWritable.class, Text.class,
      +   LongWritable.class, LongWritable.class, true, null);
      +
      + ...
      +
      + job.waitForCompletion(true);
      + ...
      + 
      ]]> +
      +
      + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + DBInputFormat emits LongWritables containing the record number as + key and DBWritables as value. + + The SQL query, and input class can be using one of the two + setInput methods.]]> + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + {@link DBOutputFormat} accepts <key,value> pairs, where + key has a type extending DBWritable. Returned {@link RecordWriter} + writes only the key to the database with a batch SQL query.]]> + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + DBWritable. DBWritable, is similar to {@link Writable} + except that the {@link #write(PreparedStatement)} method takes a + {@link PreparedStatement}, and {@link #readFields(ResultSet)} + takes a {@link ResultSet}. +

      + Implementations are responsible for writing the fields of the object + to PreparedStatement, and reading the fields of the object from the + ResultSet. + +

      Example:

      + If we have the following table in the database : +
      + CREATE TABLE MyTable (
      +   counter        INTEGER NOT NULL,
      +   timestamp      BIGINT  NOT NULL,
      + );
      + 
      + then we can read/write the tuples from/to the table with : +

      + public class MyWritable implements Writable, DBWritable {
      +   // Some data     
      +   private int counter;
      +   private long timestamp;
      +       
      +   //Writable#write() implementation
      +   public void write(DataOutput out) throws IOException {
      +     out.writeInt(counter);
      +     out.writeLong(timestamp);
      +   }
      +       
      +   //Writable#readFields() implementation
      +   public void readFields(DataInput in) throws IOException {
      +     counter = in.readInt();
      +     timestamp = in.readLong();
      +   }
      +       
      +   public void write(PreparedStatement statement) throws SQLException {
      +     statement.setInt(1, counter);
      +     statement.setLong(2, timestamp);
      +   }
      +       
      +   public void readFields(ResultSet resultSet) throws SQLException {
      +     counter = resultSet.getInt(1);
      +     timestamp = resultSet.getLong(2);
      +   } 
      + }
      + 
      ]]> +
      +
      + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + +
      + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + RecordReader's for + CombineFileSplit's. + + @see CombineFileSplit]]> + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + CombineFileRecordReader. + + Subclassing is needed to get a concrete record reader wrapper because of the + constructor requirement. + + @see CombineFileRecordReader + @see CombineFileInputFormat]]> + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + th Path]]> + + + + + + th Path]]> + + + + + + + + + + + th Path]]> + + + + + + + + + + + + + + + + + + + + + + + + + + CombineFileSplit can be used to implement {@link RecordReader}'s, + with reading one record per file. + + @see FileSplit + @see CombineFileInputFormat]]> + + + + + + + + + + + + + + CombineFileInputFormat-equivalent for + SequenceFileInputFormat. + + @see CombineFileInputFormat]]> + + + + + + + + + + + + + + CombineFileInputFormat-equivalent for + TextInputFormat. + + @see CombineFileInputFormat]]> + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + FileInputFormat always returns + true. Implementations that may deal with non-splittable files must + override this method. + + FileInputFormat implementations can override this and return + false to ensure that individual input files are never split-up + so that {@link Mapper}s process entire files. + + @param context the job context + @param filename the file name to check + @return is this file splitable?]]> + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + FileInputFormat is the base class for all file-based + InputFormats. This provides a generic implementation of + {@link #getSplits(JobContext)}. + + Implementations of FileInputFormat can also override the + {@link #isSplitable(JobContext, Path)} method to prevent input files + from being split-up in certain situations. Implementations that may + deal with non-splittable files must override this method, since + the default implementation assumes splitting is always possible.]]> + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + +
      or + conf.setInt(FixedLengthInputFormat.FIXED_RECORD_LENGTH, recordLength); +

      + @see FixedLengthRecordReader]]> +
      +
      + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + +
      + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + true if the Job was added.]]> + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + ([,]*) + func ::= tbl(,"") + class ::= @see java.lang.Class#forName(java.lang.String) + path ::= @see org.apache.hadoop.fs.Path#Path(java.lang.String) + } + Reads expression from the mapreduce.join.expr property and + user-supplied join types from mapreduce.join.define.<ident> + types. Paths supplied to tbl are given as input paths to the + InputFormat class listed. + @see #compose(java.lang.String, java.lang.Class, java.lang.String...)]]> + + + + + + + + + + + + + + + + + + + + + + + + + + + + + ,

      ) }]]> + + + + + + + + (tbl(,),tbl(,),...,tbl(,)) }]]> + + + + + + + + (tbl(,),tbl(,),...,tbl(,)) }]]> + + + + + + + + mapreduce.join.define.<ident> to a classname. + In the expression mapreduce.join.expr, the identifier will be + assumed to be a ComposableRecordReader. + mapreduce.join.keycomparator can be a classname used to compare + keys in the join. + @see #setFormat + @see JoinRecordReader + @see MultiFilterRecordReader]]> + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + ...... + }]]> + + + + + + + + + + + + + + + + + + + + + capacity children to position + id in the parent reader. + The id of a root CompositeRecordReader is -1 by convention, but relying + on this is not recommended.]]> + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + override(S1,S2,S3) will prefer values + from S3 over S2, and values from S2 over S1 for all keys + emitted from all sources.]]> + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + [<child1>,<child2>,...,<childn>]]]> + + + + + + + out. + TupleWritable format: + {@code + ...... + }]]> + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + the map's input key type + @param the map's input value type + @param the map's output key type + @param the map's output value type + @param job the job + @return the mapper class to run]]> + + + + + + + the map input key type + @param the map input value type + @param the map output key type + @param the map output value type + @param job the job to modify + @param cls the class to use as the mapper]]> + + + + + + + + + + + + + + + + + It can be used instead of the default implementation, + {@link org.apache.hadoop.mapred.MapRunner}, when the Map operation is not CPU + bound in order to improve throughput. +

      + Mapper implementations using this MapRunnable must be thread-safe. +

      + The Map-Reduce job has to be configured with the mapper to use via + {@link #setMapperClass(Job, Class)} and + the number of thread the thread-pool can use with the + {@link #getNumberOfThreads(JobContext)} method. The default + value is 10 threads. +

      ]]> + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + MapContext to be wrapped + @return a wrapped Mapper.Context for custom implementations]]> + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + +

    3. + In applications which take a classname of committer in + a configuration option, set it to the canonical name of this class + (see {@link #NAME}). When this class is instantiated, it will + use the factory mechanism to locate the configured committer for the + destination. +
    4. +
    5. + In code, explicitly create an instance of this committer through + its constructor, then invoke commit lifecycle operations on it. + The dynamically configured committer will be created in the constructor + and have the lifecycle operations relayed to it. +
    6. +
    ]]> + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + true if the job output should be compressed, + false otherwise]]> + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + Tasks' Side-Effect Files + +

    Some applications need to create/write-to side-files, which differ from + the actual job-outputs. + +

    In such cases there could be issues with 2 instances of the same TIP + (running simultaneously e.g. speculative tasks) trying to open/write-to the + same file (path) on HDFS. Hence the application-writer will have to pick + unique names per task-attempt (e.g. using the attemptid, say + attempt_200709221812_0001_m_000000_0), not just per TIP.

    + +

    To get around this the Map-Reduce framework helps the application-writer + out by maintaining a special + ${mapreduce.output.fileoutputformat.outputdir}/_temporary/_${taskid} + sub-directory for each task-attempt on HDFS where the output of the + task-attempt goes. On successful completion of the task-attempt the files + in the ${mapreduce.output.fileoutputformat.outputdir}/_temporary/_${taskid} (only) + are promoted to ${mapreduce.output.fileoutputformat.outputdir}. Of course, the + framework discards the sub-directory of unsuccessful task-attempts. This + is completely transparent to the application.

    + +

    The application-writer can take advantage of this by creating any + side-files required in a work directory during execution + of his task i.e. via + {@link #getWorkOutputPath(TaskInputOutputContext)}, and + the framework will move them out similarly - thus she doesn't have to pick + unique paths per task-attempt.

    + +

    The entire discussion holds true for maps of jobs with + reducer=NONE (i.e. 0 reduces) since output of the map, in that case, + goes directly to HDFS.

    + + @return the {@link Path} to the task's temporary output directory + for the map-reduce job.]]> +
    +
    + + + + + + + + The path can be used to create custom files from within the map and + reduce tasks. The path name will be unique for each task. The path parent + will be the job output directory.

    ls + +

    This method uses the {@link #getUniqueFile} method to make the file name + unique for the task.

    + + @param context the context for the task. + @param name the name for the file. + @param extension the extension for the file + @return a unique path accross all tasks of the job.]]> +
    +
    + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + +
    + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + Warning: when the baseOutputPath is a path that resolves + outside of the final job output directory, the directory is created + immediately and then persists through subsequent task retries, breaking + the concept of output committing.]]> + + + + + + + + + + Warning: when the baseOutputPath is a path that resolves + outside of the final job output directory, the directory is created + immediately and then persists through subsequent task retries, breaking + the concept of output committing.]]> + + + + + + + super.close() at the + end of their close()]]> + + + + + Case one: writing to additional outputs other than the job default output. + + Each additional output, or named output, may be configured with its own + OutputFormat, with its own key class and with its own value + class. +

    + +

    + Case two: to write data to different files provided by user +

    + +

    + MultipleOutputs supports counters, by default they are disabled. The + counters group is the {@link MultipleOutputs} class name. The names of the + counters are the same as the output name. These count the number records + written to each output name. +

    + + Usage pattern for job submission: +
    +
    + Job job = new Job();
    +
    + FileInputFormat.setInputPath(job, inDir);
    + FileOutputFormat.setOutputPath(job, outDir);
    +
    + job.setMapperClass(MOMap.class);
    + job.setReducerClass(MOReduce.class);
    + ...
    +
    + // Defines additional single text based output 'text' for the job
    + MultipleOutputs.addNamedOutput(job, "text", TextOutputFormat.class,
    + LongWritable.class, Text.class);
    +
    + // Defines additional sequence-file based output 'sequence' for the job
    + MultipleOutputs.addNamedOutput(job, "seq",
    +   SequenceFileOutputFormat.class,
    +   LongWritable.class, Text.class);
    + ...
    +
    + job.waitForCompletion(true);
    + ...
    + 
    +

    + Usage in Reducer: +

    + <K, V> String generateFileName(K k, V v) {
    +   return k.toString() + "_" + v.toString();
    + }
    + 
    + public class MOReduce extends
    +   Reducer<WritableComparable, Writable,WritableComparable, Writable> {
    + private MultipleOutputs mos;
    + public void setup(Context context) {
    + ...
    + mos = new MultipleOutputs(context);
    + }
    +
    + public void reduce(WritableComparable key, Iterator<Writable> values,
    + Context context)
    + throws IOException {
    + ...
    + mos.write("text", , key, new Text("Hello"));
    + mos.write("seq", LongWritable(1), new Text("Bye"), "seq_a");
    + mos.write("seq", LongWritable(2), key, new Text("Chau"), "seq_b");
    + mos.write(key, new Text("value"), generateFileName(key, new Text("value")));
    + ...
    + }
    +
    + public void cleanup(Context) throws IOException {
    + mos.close();
    + ...
    + }
    +
    + }
    + 
    + +

    + When used in conjuction with org.apache.hadoop.mapreduce.lib.output.LazyOutputFormat, + MultipleOutputs can mimic the behaviour of MultipleTextOutputFormat and MultipleSequenceFileOutputFormat + from the old Hadoop API - ie, output can be written from the Reducer to more than one location. +

    + +

    + Use MultipleOutputs.write(KEYOUT key, VALUEOUT value, String baseOutputPath) to write key and + value to a path specified by baseOutputPath, with no need to specify a named output. + Warning: when the baseOutputPath passed to MultipleOutputs.write + is a path that resolves outside of the final job output directory, the + directory is created immediately and then persists through subsequent + task retries, breaking the concept of output committing: +

    + +
    + private MultipleOutputs<Text, Text> out;
    + 
    + public void setup(Context context) {
    +   out = new MultipleOutputs<Text, Text>(context);
    +   ...
    + }
    + 
    + public void reduce(Text key, Iterable<Text> values, Context context) throws IOException, InterruptedException {
    + for (Text t : values) {
    +   out.write(key, t, generateFileName(<parameter list...>));
    +   }
    + }
    + 
    + protected void cleanup(Context context) throws IOException, InterruptedException {
    +   out.close();
    + }
    + 
    + +

    + Use your own code in generateFileName() to create a custom path to your results. + '/' characters in baseOutputPath will be translated into directory levels in your file system. + Also, append your custom-generated path with "part" or similar, otherwise your output will be -00000, -00001 etc. + No call to context.write() is necessary. See example generateFileName() code below. +

    + +
    + private String generateFileName(Text k) {
    +   // expect Text k in format "Surname|Forename"
    +   String[] kStr = k.toString().split("\\|");
    +   
    +   String sName = kStr[0];
    +   String fName = kStr[1];
    +
    +   // example for k = Smith|John
    +   // output written to /user/hadoop/path/to/output/Smith/John-r-00000 (etc)
    +   return sName + "/" + fName;
    + }
    + 
    + +

    + Using MultipleOutputs in this way will still create zero-sized default output, eg part-00000. + To prevent this use LazyOutputFormat.setOutputFormatClass(job, TextOutputFormat.class); + instead of job.setOutputFormatClass(TextOutputFormat.class); in your Hadoop job configuration. +

    ]]> +
    +
    + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + This allows the user to specify the key class to be different + from the actual class ({@link BytesWritable}) used for writing

    + + @param job the {@link Job} to modify + @param theClass the SequenceFile output key class.]]> +
    +
    + + + + + This allows the user to specify the value class to be different + from the actual class ({@link BytesWritable}) used for writing

    + + @param job the {@link Job} to modify + @param theClass the SequenceFile output key class.]]> +
    +
    + + + + + + + + + + + + + + + + + + + + + + + + + + + +
    + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + bytes[left:(right+1)] in Python syntax. + + @param conf configuration object + @param left left Python-style offset + @param right right Python-style offset]]> + + + + + + + bytes[offset:] in Python syntax. + + @param conf configuration object + @param offset left Python-style offset]]> + + + + + + + bytes[:(offset+1)] in Python syntax. + + @param conf configuration object + @param offset right Python-style offset]]> + + + + + + + + + + + + + + + + + + + + + Partition {@link BinaryComparable} keys using a configurable part of + the bytes array returned by {@link BinaryComparable#getBytes()}.

    + +

    The subarray to be used for the partitioning can be defined by means + of the following properties: +

      +
    • + mapreduce.partition.binarypartitioner.left.offset: + left offset in array (0 by default) +
    • +
    • + mapreduce.partition.binarypartitioner.right.offset: + right offset in array (-1 by default) +
    • +
    + Like in Python, both negative and positive offsets are allowed, but + the meaning is slightly different. In case of an array of length 5, + for instance, the possible offsets are: +
    
    +  +---+---+---+---+---+
    +  | B | B | B | B | B |
    +  +---+---+---+---+---+
    +    0   1   2   3   4
    +   -5  -4  -3  -2  -1
    + 
    + The first row of numbers gives the position of the offsets 0...5 in + the array; the second row gives the corresponding negative offsets. + Contrary to Python, the specified subarray has byte i + and j as first and last element, repectively, when + i and j are the left and right offset. + +

    For Hadoop programs written in Java, it is advisable to use one of + the following static convenience methods for setting the offsets: +

      +
    • {@link #setOffsets}
    • +
    • {@link #setLeftOffset}
    • +
    • {@link #setRightOffset}
    • +
    ]]> +
    +
    + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + total.order.partitioner.natural.order is not false, a trie + of the first total.order.partitioner.max.trie.depth(2) + 1 bytes + will be built. Otherwise, keys will be located using a binary search of + the partition keyset using the {@link org.apache.hadoop.io.RawComparator} + defined for this job. The input file must be sorted with the same + comparator and contain {@link Job#getNumReduceTasks()} - 1 keys.]]> + + + + + + + + + + + + + + R reduces, there are R-1 + keys in the SequenceFile.]]> + + + + + + + + + + + + + + + + + + + + + +
    + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + ReduceContext to be wrapped + @return a wrapped Reducer.Context for custom implementations]]> + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + diff --git a/hadoop-mapreduce-project/dev-support/jdiff/Apache_Hadoop_MapReduce_JobClient_3.3.3.xml b/hadoop-mapreduce-project/dev-support/jdiff/Apache_Hadoop_MapReduce_JobClient_3.3.3.xml new file mode 100644 index 0000000000000..3b5eb71c92539 --- /dev/null +++ b/hadoop-mapreduce-project/dev-support/jdiff/Apache_Hadoop_MapReduce_JobClient_3.3.3.xml @@ -0,0 +1,16 @@ + + + + + + + + + + + + diff --git a/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-app/src/main/java/org/apache/hadoop/mapred/TaskAttemptListenerImpl.java b/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-app/src/main/java/org/apache/hadoop/mapred/TaskAttemptListenerImpl.java index 7d151adea7d6c..5dffd735fdafd 100644 --- a/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-app/src/main/java/org/apache/hadoop/mapred/TaskAttemptListenerImpl.java +++ b/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-app/src/main/java/org/apache/hadoop/mapred/TaskAttemptListenerImpl.java @@ -28,6 +28,10 @@ import java.util.concurrent.ConcurrentMap; import java.util.concurrent.atomic.AtomicReference; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + +import org.apache.hadoop.classification.VisibleForTesting; import org.apache.hadoop.conf.Configuration; import org.apache.hadoop.fs.CommonConfigurationKeysPublic; import org.apache.hadoop.ipc.ProtocolSignature; @@ -50,8 +54,8 @@ import org.apache.hadoop.mapreduce.v2.app.job.event.TaskAttemptEvent; import org.apache.hadoop.mapreduce.v2.app.job.event.TaskAttemptEventType; import org.apache.hadoop.mapreduce.v2.app.job.event.TaskAttemptFailEvent; -import org.apache.hadoop.mapreduce.v2.app.job.event.TaskAttemptStatusUpdateEvent.TaskAttemptStatus; import org.apache.hadoop.mapreduce.v2.app.job.event.TaskAttemptStatusUpdateEvent; +import org.apache.hadoop.mapreduce.v2.app.job.event.TaskAttemptStatusUpdateEvent.TaskAttemptStatus; import org.apache.hadoop.mapreduce.v2.app.rm.RMHeartbeatHandler; import org.apache.hadoop.mapreduce.v2.app.rm.preemption.AMPreemptionPolicy; import org.apache.hadoop.mapreduce.v2.app.security.authorize.MRAMPolicyProvider; @@ -61,10 +65,6 @@ import org.apache.hadoop.util.StringInterner; import org.apache.hadoop.util.Time; import org.apache.hadoop.yarn.exceptions.YarnRuntimeException; -import org.slf4j.Logger; -import org.slf4j.LoggerFactory; - -import org.apache.hadoop.classification.VisibleForTesting; /** * This class is responsible for talking to the task umblical. @@ -409,6 +409,11 @@ public AMFeedback statusUpdate(TaskAttemptID taskAttemptID, if (LOG.isDebugEnabled()) { LOG.debug("Ping from " + taskAttemptID.toString()); } + // Consider ping from the tasks for liveliness check + if (getConfig().getBoolean(MRJobConfig.MR_TASK_ENABLE_PING_FOR_LIVELINESS_CHECK, + MRJobConfig.DEFAULT_MR_TASK_ENABLE_PING_FOR_LIVELINESS_CHECK)) { + taskHeartbeatHandler.progressing(yarnAttemptID); + } return feedback; } diff --git a/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-app/src/main/java/org/apache/hadoop/mapred/YarnChild.java b/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-app/src/main/java/org/apache/hadoop/mapred/YarnChild.java index a513c16d2308b..bbf527ebff53a 100644 --- a/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-app/src/main/java/org/apache/hadoop/mapred/YarnChild.java +++ b/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-app/src/main/java/org/apache/hadoop/mapred/YarnChild.java @@ -59,7 +59,7 @@ import org.apache.hadoop.yarn.api.ApplicationConstants.Environment; import org.apache.hadoop.yarn.api.records.ApplicationAttemptId; import org.apache.hadoop.yarn.api.records.ContainerId; -import org.apache.hadoop.yarn.util.ConverterUtils; + import org.slf4j.Logger; import org.slf4j.LoggerFactory; diff --git a/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-app/src/main/java/org/apache/hadoop/mapreduce/v2/app/launcher/ContainerLauncher.java b/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-app/src/main/java/org/apache/hadoop/mapreduce/v2/app/launcher/ContainerLauncher.java index 82360f08d4518..2465811c82a30 100644 --- a/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-app/src/main/java/org/apache/hadoop/mapreduce/v2/app/launcher/ContainerLauncher.java +++ b/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-app/src/main/java/org/apache/hadoop/mapreduce/v2/app/launcher/ContainerLauncher.java @@ -19,7 +19,6 @@ package org.apache.hadoop.mapreduce.v2.app.launcher; -import org.apache.hadoop.mapreduce.MRJobConfig; import org.apache.hadoop.yarn.event.EventHandler; public interface ContainerLauncher diff --git a/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-app/src/main/java/org/apache/hadoop/mapreduce/v2/app/rm/RMContainerRequestor.java b/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-app/src/main/java/org/apache/hadoop/mapreduce/v2/app/rm/RMContainerRequestor.java index 61cc2eb898eba..16ca585e9b460 100644 --- a/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-app/src/main/java/org/apache/hadoop/mapreduce/v2/app/rm/RMContainerRequestor.java +++ b/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-app/src/main/java/org/apache/hadoop/mapreduce/v2/app/rm/RMContainerRequestor.java @@ -213,12 +213,10 @@ protected AllocateResponse makeRemoteRequest() throws YarnException, allocateResponse.getCompletedContainersStatuses().size(); if (ask.size() > 0 || release.size() > 0) { - LOG.info("getResources() for " + applicationId + ":" + " ask=" - + ask.size() + " release= " + release.size() + " newContainers=" - + allocateResponse.getAllocatedContainers().size() - + " finishedContainers=" + numCompletedContainers - + " resourcelimit=" + availableResources + " knownNMs=" - + clusterNmCount); + LOG.info("applicationId={}: ask={} release={} newContainers={} finishedContainers={}" + + " resourceLimit={} knownNMs={}", applicationId, ask.size(), release.size(), + allocateResponse.getAllocatedContainers().size(), numCompletedContainers, + availableResources, clusterNmCount); } ask.clear(); diff --git a/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-app/src/main/java/org/apache/hadoop/mapreduce/v2/app/webapp/AttemptsPage.java b/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-app/src/main/java/org/apache/hadoop/mapreduce/v2/app/webapp/AttemptsPage.java index 3b45b168b7dc4..9a6a39555f983 100644 --- a/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-app/src/main/java/org/apache/hadoop/mapreduce/v2/app/webapp/AttemptsPage.java +++ b/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-app/src/main/java/org/apache/hadoop/mapreduce/v2/app/webapp/AttemptsPage.java @@ -28,7 +28,6 @@ import org.apache.hadoop.conf.Configuration; import org.apache.hadoop.mapreduce.v2.api.records.TaskAttemptId; -import org.apache.hadoop.mapreduce.v2.api.records.TaskAttemptState; import org.apache.hadoop.mapreduce.v2.api.records.TaskId; import org.apache.hadoop.mapreduce.v2.api.records.TaskType; import org.apache.hadoop.mapreduce.v2.app.job.Task; diff --git a/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-app/src/test/java/org/apache/hadoop/mapred/TestTaskAttemptListenerImpl.java b/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-app/src/test/java/org/apache/hadoop/mapred/TestTaskAttemptListenerImpl.java index f8b8c6ccdf1de..b5a7694e4cc6b 100644 --- a/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-app/src/test/java/org/apache/hadoop/mapred/TestTaskAttemptListenerImpl.java +++ b/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-app/src/test/java/org/apache/hadoop/mapred/TestTaskAttemptListenerImpl.java @@ -17,23 +17,30 @@ */ package org.apache.hadoop.mapred; -import java.util.function.Supplier; -import org.apache.hadoop.mapred.Counters.Counter; -import org.apache.hadoop.mapreduce.checkpoint.EnumCounter; - import java.io.IOException; import java.util.ArrayList; import java.util.Arrays; import java.util.List; import java.util.concurrent.ConcurrentMap; import java.util.concurrent.atomic.AtomicReference; +import java.util.function.Supplier; + +import org.junit.After; +import org.junit.Test; +import org.junit.runner.RunWith; +import org.mockito.ArgumentCaptor; +import org.mockito.Captor; +import org.mockito.Mock; +import org.mockito.junit.MockitoJUnitRunner; import org.apache.hadoop.conf.Configuration; import org.apache.hadoop.fs.Path; +import org.apache.hadoop.mapred.Counters.Counter; import org.apache.hadoop.mapreduce.MRJobConfig; import org.apache.hadoop.mapreduce.TaskType; import org.apache.hadoop.mapreduce.TypeConverter; import org.apache.hadoop.mapreduce.checkpoint.CheckpointID; +import org.apache.hadoop.mapreduce.checkpoint.EnumCounter; import org.apache.hadoop.mapreduce.checkpoint.FSCheckpointID; import org.apache.hadoop.mapreduce.checkpoint.TaskCheckpointID; import org.apache.hadoop.mapreduce.security.token.JobTokenSecretManager; @@ -48,9 +55,9 @@ import org.apache.hadoop.mapreduce.v2.app.job.Job; import org.apache.hadoop.mapreduce.v2.app.job.event.TaskAttemptStatusUpdateEvent; import org.apache.hadoop.mapreduce.v2.app.job.event.TaskAttemptStatusUpdateEvent.TaskAttemptStatus; +import org.apache.hadoop.mapreduce.v2.app.rm.RMHeartbeatHandler; import org.apache.hadoop.mapreduce.v2.app.rm.preemption.AMPreemptionPolicy; import org.apache.hadoop.mapreduce.v2.app.rm.preemption.CheckpointAMPreemptionPolicy; -import org.apache.hadoop.mapreduce.v2.app.rm.RMHeartbeatHandler; import org.apache.hadoop.mapreduce.v2.util.MRBuilderUtils; import org.apache.hadoop.test.GenericTestUtils; import org.apache.hadoop.yarn.event.Dispatcher; @@ -60,17 +67,22 @@ import org.apache.hadoop.yarn.factory.providers.RecordFactoryProvider; import org.apache.hadoop.yarn.util.ControlledClock; import org.apache.hadoop.yarn.util.SystemClock; -import org.junit.After; -import org.junit.Test; -import org.junit.runner.RunWith; -import org.mockito.ArgumentCaptor; -import org.mockito.Captor; -import org.mockito.Mock; -import org.mockito.junit.MockitoJUnitRunner; import static org.assertj.core.api.Assertions.assertThat; -import static org.junit.Assert.*; -import static org.mockito.Mockito.*; +import static org.junit.Assert.assertEquals; +import static org.junit.Assert.assertFalse; +import static org.junit.Assert.assertNotNull; +import static org.junit.Assert.assertNull; +import static org.junit.Assert.assertTrue; +import static org.junit.Assert.fail; +import static org.mockito.Mockito.any; +import static org.mockito.Mockito.doReturn; +import static org.mockito.Mockito.eq; +import static org.mockito.Mockito.mock; +import static org.mockito.Mockito.never; +import static org.mockito.Mockito.times; +import static org.mockito.Mockito.verify; +import static org.mockito.Mockito.when; /** * Tests the behavior of TaskAttemptListenerImpl. @@ -417,6 +429,23 @@ public void testStatusUpdateProgress() verify(hbHandler).progressing(eq(attemptId)); } + @Test + public void testPingUpdateProgress() throws IOException, InterruptedException { + configureMocks(); + Configuration conf = new Configuration(); + conf.setBoolean(MRJobConfig.MR_TASK_ENABLE_PING_FOR_LIVELINESS_CHECK, true); + listener.init(conf); + listener.start(); + listener.registerPendingTask(task, wid); + listener.registerLaunchedTask(attemptId, wid); + verify(hbHandler).register(attemptId); + + // make sure a ping does report progress + AMFeedback feedback = listener.statusUpdate(attemptID, null); + assertTrue(feedback.getTaskFound()); + verify(hbHandler, times(1)).progressing(eq(attemptId)); + } + @Test public void testSingleStatusUpdate() throws IOException, InterruptedException { diff --git a/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-app/src/test/java/org/apache/hadoop/mapreduce/v2/app/TestFail.java b/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-app/src/test/java/org/apache/hadoop/mapreduce/v2/app/TestFail.java index 44a4760eb6803..3b5cfe221eded 100644 --- a/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-app/src/test/java/org/apache/hadoop/mapreduce/v2/app/TestFail.java +++ b/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-app/src/test/java/org/apache/hadoop/mapreduce/v2/app/TestFail.java @@ -46,9 +46,7 @@ import org.apache.hadoop.mapreduce.v2.app.launcher.ContainerLauncherImpl; import org.apache.hadoop.mapreduce.v2.app.rm.preemption.AMPreemptionPolicy; import org.apache.hadoop.net.NetUtils; -import org.apache.hadoop.yarn.api.ContainerManagementProtocol; import org.apache.hadoop.yarn.api.records.ContainerId; -import org.apache.hadoop.yarn.api.records.Token; import org.apache.hadoop.yarn.client.api.impl.ContainerManagementProtocolProxy.ContainerManagementProtocolProxyData; import org.junit.Test; diff --git a/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-app/src/test/java/org/apache/hadoop/mapreduce/v2/app/TestMRAppMaster.java b/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-app/src/test/java/org/apache/hadoop/mapreduce/v2/app/TestMRAppMaster.java index 5bf51022327bc..06550378ba939 100644 --- a/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-app/src/test/java/org/apache/hadoop/mapreduce/v2/app/TestMRAppMaster.java +++ b/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-app/src/test/java/org/apache/hadoop/mapreduce/v2/app/TestMRAppMaster.java @@ -83,7 +83,7 @@ import org.apache.hadoop.yarn.api.records.ContainerId; import org.apache.hadoop.yarn.event.EventHandler; import org.apache.hadoop.yarn.security.AMRMTokenIdentifier; -import org.apache.hadoop.yarn.util.ConverterUtils; + import org.junit.AfterClass; import org.junit.Before; import org.junit.BeforeClass; diff --git a/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-common/src/main/java/org/apache/hadoop/mapred/LocalDistributedCacheManager.java b/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-common/src/main/java/org/apache/hadoop/mapred/LocalDistributedCacheManager.java index e5a50f013ff69..5469d4a9298fe 100644 --- a/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-common/src/main/java/org/apache/hadoop/mapred/LocalDistributedCacheManager.java +++ b/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-common/src/main/java/org/apache/hadoop/mapred/LocalDistributedCacheManager.java @@ -51,7 +51,6 @@ import org.apache.hadoop.util.concurrent.HadoopExecutors; import org.apache.hadoop.yarn.api.records.LocalResource; import org.apache.hadoop.yarn.api.records.LocalResourceType; -import org.apache.hadoop.yarn.util.ConverterUtils; import org.apache.hadoop.yarn.util.FSDownload; import org.apache.hadoop.thirdparty.com.google.common.collect.Maps; diff --git a/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-common/src/main/java/org/apache/hadoop/mapreduce/v2/jobhistory/JHAdminConfig.java b/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-common/src/main/java/org/apache/hadoop/mapreduce/v2/jobhistory/JHAdminConfig.java index 1c11e7728b24a..15e07ac6abbe7 100644 --- a/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-common/src/main/java/org/apache/hadoop/mapreduce/v2/jobhistory/JHAdminConfig.java +++ b/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-common/src/main/java/org/apache/hadoop/mapreduce/v2/jobhistory/JHAdminConfig.java @@ -20,7 +20,6 @@ import org.apache.hadoop.classification.InterfaceAudience; import org.apache.hadoop.classification.InterfaceStability; -import org.apache.hadoop.fs.CommonConfigurationKeysPublic; import org.apache.hadoop.http.HttpConfig; /** diff --git a/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-common/src/test/java/org/apache/hadoop/mapred/TestLocalDistributedCacheManager.java b/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-common/src/test/java/org/apache/hadoop/mapred/TestLocalDistributedCacheManager.java index 5d1c669e50083..50cc63094bd8c 100644 --- a/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-common/src/test/java/org/apache/hadoop/mapred/TestLocalDistributedCacheManager.java +++ b/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-common/src/test/java/org/apache/hadoop/mapred/TestLocalDistributedCacheManager.java @@ -21,7 +21,6 @@ import static org.junit.Assert.assertFalse; import static org.junit.Assert.assertTrue; import static org.mockito.ArgumentMatchers.any; -import static org.mockito.ArgumentMatchers.anyInt; import static org.mockito.Mockito.mock; import static org.mockito.Mockito.when; @@ -31,9 +30,11 @@ import java.io.IOException; import java.net.URI; import java.util.HashMap; +import java.util.List; import java.util.Map; import java.util.ArrayList; import java.util.concurrent.Callable; +import java.util.concurrent.CompletableFuture; import java.util.concurrent.CyclicBarrier; import java.util.concurrent.ExecutorService; import java.util.concurrent.Executors; @@ -44,22 +45,31 @@ import org.apache.hadoop.fs.FileStatus; import org.apache.hadoop.fs.FileSystem; import org.apache.hadoop.fs.FilterFileSystem; +import org.apache.hadoop.fs.FutureDataInputStreamBuilder; import org.apache.hadoop.fs.Path; import org.apache.hadoop.fs.PositionedReadable; import org.apache.hadoop.fs.Seekable; +import org.apache.hadoop.fs.impl.FutureDataInputStreamBuilderImpl; import org.apache.hadoop.fs.permission.FsPermission; import org.apache.hadoop.mapreduce.Job; import org.apache.hadoop.mapreduce.MRConfig; import org.apache.hadoop.mapreduce.MRJobConfig; +import org.apache.hadoop.util.functional.CallableRaisingIOE; import org.junit.After; import org.junit.Before; import org.junit.Test; import org.mockito.invocation.InvocationOnMock; import org.mockito.stubbing.Answer; +/** + * Test the LocalDistributedCacheManager using mocking. + * This suite is brittle to changes in the class under test. + */ @SuppressWarnings("deprecation") public class TestLocalDistributedCacheManager { + private static final byte[] TEST_DATA = "This is a test file\n".getBytes(); + private static FileSystem mockfs; public static class MockFileSystem extends FilterFileSystem { @@ -70,6 +80,14 @@ public MockFileSystem() { private File localDir; + /** + * Recursive delete of a path. + * For safety, paths of length under 5 are rejected. + * @param file path to delete. + * @throws IOException never, it is just "a dummy in the method signature" + * @throws IllegalArgumentException path too short + * @throws RuntimeException File.delete() failed. + */ private static void delete(File file) throws IOException { if (file.getAbsolutePath().length() < 5) { throw new IllegalArgumentException( @@ -109,9 +127,9 @@ public void cleanup() throws Exception { * Mock input stream based on a byte array so that it can be used by a * FSDataInputStream. */ - private static class MockInputStream extends ByteArrayInputStream + private static final class MockInputStream extends ByteArrayInputStream implements Seekable, PositionedReadable { - public MockInputStream(byte[] buf) { + private MockInputStream(byte[] buf) { super(buf); } @@ -134,47 +152,45 @@ public void testDownload() throws Exception { when(mockfs.getUri()).thenReturn(mockBase); Path working = new Path("mock://test-nn1/user/me/"); when(mockfs.getWorkingDirectory()).thenReturn(working); - when(mockfs.resolvePath(any(Path.class))).thenAnswer(new Answer() { - @Override - public Path answer(InvocationOnMock args) throws Throwable { - return (Path) args.getArguments()[0]; - } - }); + when(mockfs.resolvePath(any(Path.class))).thenAnswer( + (Answer) args -> (Path) args.getArguments()[0]); final URI file = new URI("mock://test-nn1/user/me/file.txt#link"); final Path filePath = new Path(file); File link = new File("link"); + // return a filestatus for the file "*/file.txt"; raise FNFE for anything else when(mockfs.getFileStatus(any(Path.class))).thenAnswer(new Answer() { @Override public FileStatus answer(InvocationOnMock args) throws Throwable { Path p = (Path)args.getArguments()[0]; if("file.txt".equals(p.getName())) { - return new FileStatus(201, false, 1, 500, 101, 101, - FsPermission.getDefault(), "me", "me", filePath); + return createMockTestFileStatus(filePath); } else { - throw new FileNotFoundException(p+" not supported by mocking"); + throw notMocked(p); } } }); when(mockfs.getConf()).thenReturn(conf); final FSDataInputStream in = - new FSDataInputStream(new MockInputStream("This is a test file\n".getBytes())); - when(mockfs.open(any(Path.class), anyInt())).thenAnswer(new Answer() { - @Override - public FSDataInputStream answer(InvocationOnMock args) throws Throwable { - Path src = (Path)args.getArguments()[0]; - if ("file.txt".equals(src.getName())) { - return in; - } else { - throw new FileNotFoundException(src+" not supported by mocking"); - } - } - }); + new FSDataInputStream(new MockInputStream(TEST_DATA)); + + // file.txt: return an openfile builder which will eventually return the data, + // anything else: FNFE + when(mockfs.openFile(any(Path.class))).thenAnswer( + (Answer) args -> { + Path src = (Path)args.getArguments()[0]; + if ("file.txt".equals(src.getName())) { + return new MockOpenFileBuilder(mockfs, src, + () -> CompletableFuture.completedFuture(in)); + } else { + throw notMocked(src); + } + }); Job.addCacheFile(file, conf); - Map policies = new HashMap(); + Map policies = new HashMap<>(); policies.put(file.toString(), true); Job.setFileSharedCacheUploadPolicies(conf, policies); conf.set(MRJobConfig.CACHE_FILE_TIMESTAMPS, "101"); @@ -191,6 +207,12 @@ public FSDataInputStream answer(InvocationOnMock args) throws Throwable { assertFalse(link.exists()); } + /** + * This test case sets the mock FS to raise FNFE + * on any getFileStatus/openFile calls. + * If the manager successfully starts up, it means that + * no files were probed for/opened. + */ @Test public void testEmptyDownload() throws Exception { JobID jobId = new JobID(); @@ -201,30 +223,21 @@ public void testEmptyDownload() throws Exception { when(mockfs.getUri()).thenReturn(mockBase); Path working = new Path("mock://test-nn1/user/me/"); when(mockfs.getWorkingDirectory()).thenReturn(working); - when(mockfs.resolvePath(any(Path.class))).thenAnswer(new Answer() { - @Override - public Path answer(InvocationOnMock args) throws Throwable { - return (Path) args.getArguments()[0]; - } - }); + when(mockfs.resolvePath(any(Path.class))).thenAnswer( + (Answer) args -> (Path) args.getArguments()[0]); - when(mockfs.getFileStatus(any(Path.class))).thenAnswer(new Answer() { - @Override - public FileStatus answer(InvocationOnMock args) throws Throwable { - Path p = (Path)args.getArguments()[0]; - throw new FileNotFoundException(p+" not supported by mocking"); - } - }); + when(mockfs.getFileStatus(any(Path.class))).thenAnswer( + (Answer) args -> { + Path p = (Path)args.getArguments()[0]; + throw notMocked(p); + }); when(mockfs.getConf()).thenReturn(conf); - when(mockfs.open(any(Path.class), anyInt())).thenAnswer(new Answer() { - @Override - public FSDataInputStream answer(InvocationOnMock args) throws Throwable { - Path src = (Path)args.getArguments()[0]; - throw new FileNotFoundException(src+" not supported by mocking"); - } - }); - + when(mockfs.openFile(any(Path.class))).thenAnswer( + (Answer) args -> { + Path src = (Path)args.getArguments()[0]; + throw notMocked(src); + }); conf.set(MRJobConfig.CACHE_FILES, ""); conf.set(MRConfig.LOCAL_DIR, localDir.getAbsolutePath()); LocalDistributedCacheManager manager = new LocalDistributedCacheManager(); @@ -236,6 +249,9 @@ public FSDataInputStream answer(InvocationOnMock args) throws Throwable { } + /** + * The same file can be added to the cache twice. + */ @Test public void testDuplicateDownload() throws Exception { JobID jobId = new JobID(); @@ -246,12 +262,8 @@ public void testDuplicateDownload() throws Exception { when(mockfs.getUri()).thenReturn(mockBase); Path working = new Path("mock://test-nn1/user/me/"); when(mockfs.getWorkingDirectory()).thenReturn(working); - when(mockfs.resolvePath(any(Path.class))).thenAnswer(new Answer() { - @Override - public Path answer(InvocationOnMock args) throws Throwable { - return (Path) args.getArguments()[0]; - } - }); + when(mockfs.resolvePath(any(Path.class))).thenAnswer( + (Answer) args -> (Path) args.getArguments()[0]); final URI file = new URI("mock://test-nn1/user/me/file.txt#link"); final Path filePath = new Path(file); @@ -262,32 +274,30 @@ public Path answer(InvocationOnMock args) throws Throwable { public FileStatus answer(InvocationOnMock args) throws Throwable { Path p = (Path)args.getArguments()[0]; if("file.txt".equals(p.getName())) { - return new FileStatus(201, false, 1, 500, 101, 101, - FsPermission.getDefault(), "me", "me", filePath); + return createMockTestFileStatus(filePath); } else { - throw new FileNotFoundException(p+" not supported by mocking"); + throw notMocked(p); } } }); when(mockfs.getConf()).thenReturn(conf); final FSDataInputStream in = - new FSDataInputStream(new MockInputStream("This is a test file\n".getBytes())); - when(mockfs.open(any(Path.class), anyInt())).thenAnswer(new Answer() { - @Override - public FSDataInputStream answer(InvocationOnMock args) throws Throwable { - Path src = (Path)args.getArguments()[0]; - if ("file.txt".equals(src.getName())) { - return in; - } else { - throw new FileNotFoundException(src+" not supported by mocking"); - } - } - }); + new FSDataInputStream(new MockInputStream(TEST_DATA)); + when(mockfs.openFile(any(Path.class))).thenAnswer( + (Answer) args -> { + Path src = (Path)args.getArguments()[0]; + if ("file.txt".equals(src.getName())) { + return new MockOpenFileBuilder(mockfs, src, + () -> CompletableFuture.completedFuture(in)); + } else { + throw notMocked(src); + } + }); Job.addCacheFile(file, conf); Job.addCacheFile(file, conf); - Map policies = new HashMap(); + Map policies = new HashMap<>(); policies.put(file.toString(), true); Job.setFileSharedCacheUploadPolicies(conf, policies); conf.set(MRJobConfig.CACHE_FILE_TIMESTAMPS, "101,101"); @@ -306,7 +316,7 @@ public FSDataInputStream answer(InvocationOnMock args) throws Throwable { /** * This test tries to replicate the issue with the previous version of - * {@ref LocalDistributedCacheManager} when the resulting timestamp is + * {@link LocalDistributedCacheManager} when the resulting timestamp is * identical as that in another process. Unfortunately, it is difficult * to mimic such behavior in a single process unit test. And mocking * the unique id (timestamp previously, UUID otherwise) won't prove the @@ -321,7 +331,7 @@ public void testMultipleCacheSetup() throws Exception { final int threadCount = 10; final CyclicBarrier barrier = new CyclicBarrier(threadCount); - ArrayList> setupCallable = new ArrayList<>(); + List> setupCallable = new ArrayList<>(); for (int i = 0; i < threadCount; ++i) { setupCallable.add(() -> { barrier.await(); @@ -340,4 +350,58 @@ public void testMultipleCacheSetup() throws Exception { manager.close(); } } + + /** + * Create test file status using test data as the length. + * @param filePath path to the file + * @return a file status. + */ + private FileStatus createMockTestFileStatus(final Path filePath) { + return new FileStatus(TEST_DATA.length, false, 1, 500, 101, 101, + FsPermission.getDefault(), "me", "me", filePath); + } + + /** + * Exception to throw on a not mocked path. + * @return a FileNotFoundException + */ + private FileNotFoundException notMocked(final Path p) { + return new FileNotFoundException(p + " not supported by mocking"); + } + + /** + * Openfile builder where the build operation is a l-expression + * supplied in the constructor. + */ + private static final class MockOpenFileBuilder extends + FutureDataInputStreamBuilderImpl { + + /** + * Operation to invoke to build the result. + */ + private final CallableRaisingIOE> + buildTheResult; + + /** + * Create the builder. the FS and path must be non-null. + * FileSystem.getConf() is the only method invoked of the FS by + * the superclass. + * @param fileSystem fs + * @param path path to open + * @param buildTheResult builder operation. + */ + private MockOpenFileBuilder(final FileSystem fileSystem, Path path, + final CallableRaisingIOE> buildTheResult) { + super(fileSystem, path); + this.buildTheResult = buildTheResult; + } + + @Override + public CompletableFuture build() + throws IllegalArgumentException, UnsupportedOperationException, + IOException { + return buildTheResult.apply(); + } + } + } diff --git a/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-common/src/test/java/org/apache/hadoop/mapred/TestLocalModeWithNewApis.java b/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-common/src/test/java/org/apache/hadoop/mapred/TestLocalModeWithNewApis.java index d8ca74aec2f0d..556f7fd716aa7 100644 --- a/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-common/src/test/java/org/apache/hadoop/mapred/TestLocalModeWithNewApis.java +++ b/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-common/src/test/java/org/apache/hadoop/mapred/TestLocalModeWithNewApis.java @@ -21,7 +21,6 @@ import java.io.BufferedReader; import java.io.DataOutputStream; -import java.io.File; import java.io.IOException; import java.io.InputStreamReader; import java.util.Random; diff --git a/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-common/src/test/java/org/apache/hadoop/mapreduce/v2/util/TestMRApps.java b/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-common/src/test/java/org/apache/hadoop/mapreduce/v2/util/TestMRApps.java index 62235b7267300..b4952ecc4ddcd 100644 --- a/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-common/src/test/java/org/apache/hadoop/mapreduce/v2/util/TestMRApps.java +++ b/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-common/src/test/java/org/apache/hadoop/mapreduce/v2/util/TestMRApps.java @@ -40,7 +40,6 @@ import org.apache.hadoop.fs.FileSystem; import org.apache.hadoop.fs.FilterFileSystem; import org.apache.hadoop.fs.Path; -import org.apache.hadoop.mapred.InvalidJobConfException; import org.apache.hadoop.mapreduce.Job; import org.apache.hadoop.mapreduce.JobID; import org.apache.hadoop.mapreduce.MRConfig; diff --git a/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-core/src/main/java/org/apache/hadoop/mapred/JobACLsManager.java b/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-core/src/main/java/org/apache/hadoop/mapred/JobACLsManager.java index 1761500ff4a8a..136ac650bff1b 100644 --- a/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-core/src/main/java/org/apache/hadoop/mapred/JobACLsManager.java +++ b/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-core/src/main/java/org/apache/hadoop/mapred/JobACLsManager.java @@ -24,7 +24,6 @@ import org.apache.hadoop.conf.Configuration; import org.apache.hadoop.mapreduce.JobACL; import org.apache.hadoop.mapreduce.MRConfig; -import org.apache.hadoop.security.AccessControlException; import org.apache.hadoop.security.UserGroupInformation; import org.apache.hadoop.security.authorize.AccessControlList; import org.slf4j.Logger; diff --git a/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-core/src/main/java/org/apache/hadoop/mapred/JobClient.java b/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-core/src/main/java/org/apache/hadoop/mapred/JobClient.java index 2f8dd63ccede3..f94752fc238e3 100644 --- a/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-core/src/main/java/org/apache/hadoop/mapred/JobClient.java +++ b/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-core/src/main/java/org/apache/hadoop/mapred/JobClient.java @@ -48,7 +48,6 @@ import org.apache.hadoop.security.UserGroupInformation; import org.apache.hadoop.security.token.SecretManager.InvalidToken; import org.apache.hadoop.security.token.Token; -import org.apache.hadoop.security.token.TokenRenewer; import org.apache.hadoop.util.Tool; import org.apache.hadoop.util.ToolRunner; diff --git a/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-core/src/main/java/org/apache/hadoop/mapred/TaskUmbilicalProtocol.java b/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-core/src/main/java/org/apache/hadoop/mapred/TaskUmbilicalProtocol.java index 041ab3908046a..fe8767529e47d 100644 --- a/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-core/src/main/java/org/apache/hadoop/mapred/TaskUmbilicalProtocol.java +++ b/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-core/src/main/java/org/apache/hadoop/mapred/TaskUmbilicalProtocol.java @@ -24,8 +24,6 @@ import org.apache.hadoop.classification.InterfaceStability; import org.apache.hadoop.ipc.VersionedProtocol; import org.apache.hadoop.mapred.JvmTask; -import org.apache.hadoop.mapreduce.checkpoint.CheckpointID; -import org.apache.hadoop.mapreduce.checkpoint.FSCheckpointID; import org.apache.hadoop.mapreduce.checkpoint.TaskCheckpointID; import org.apache.hadoop.mapreduce.security.token.JobTokenSelector; import org.apache.hadoop.security.token.TokenInfo; diff --git a/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-core/src/main/java/org/apache/hadoop/mapreduce/Job.java b/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-core/src/main/java/org/apache/hadoop/mapreduce/Job.java index 830a327172771..c73c5bdd839f2 100644 --- a/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-core/src/main/java/org/apache/hadoop/mapreduce/Job.java +++ b/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-core/src/main/java/org/apache/hadoop/mapreduce/Job.java @@ -22,7 +22,6 @@ import java.net.URI; import java.security.PrivilegedExceptionAction; import java.util.Collection; -import java.util.Iterator; import java.util.LinkedHashMap; import java.util.Map; diff --git a/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-core/src/main/java/org/apache/hadoop/mapreduce/MRJobConfig.java b/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-core/src/main/java/org/apache/hadoop/mapreduce/MRJobConfig.java index a90c58dd28b4c..15d57a6746b13 100644 --- a/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-core/src/main/java/org/apache/hadoop/mapreduce/MRJobConfig.java +++ b/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-core/src/main/java/org/apache/hadoop/mapreduce/MRJobConfig.java @@ -919,6 +919,13 @@ public interface MRJobConfig { MR_AM_PREFIX + "scheduler.heartbeat.interval-ms"; public static final int DEFAULT_MR_AM_TO_RM_HEARTBEAT_INTERVAL_MS = 1000; + /** Whether to consider ping from tasks in liveliness check. */ + String MR_TASK_ENABLE_PING_FOR_LIVELINESS_CHECK = + "mapreduce.task.ping-for-liveliness-check.enabled"; + boolean DEFAULT_MR_TASK_ENABLE_PING_FOR_LIVELINESS_CHECK + = false; + + /** * If contact with RM is lost, the AM will wait MR_AM_TO_RM_WAIT_INTERVAL_MS * milliseconds before aborting. During this interval, AM will still try diff --git a/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-core/src/main/java/org/apache/hadoop/mapreduce/Mapper.java b/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-core/src/main/java/org/apache/hadoop/mapreduce/Mapper.java index 6b4147b900bff..8ec6e8e20d4f4 100644 --- a/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-core/src/main/java/org/apache/hadoop/mapreduce/Mapper.java +++ b/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-core/src/main/java/org/apache/hadoop/mapreduce/Mapper.java @@ -25,7 +25,6 @@ import org.apache.hadoop.conf.Configuration; import org.apache.hadoop.io.RawComparator; import org.apache.hadoop.io.compress.CompressionCodec; -import org.apache.hadoop.mapreduce.task.MapContextImpl; /** * Maps input key/value pairs to a set of intermediate key/value pairs. diff --git a/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-core/src/main/java/org/apache/hadoop/mapreduce/jobhistory/AMStartedEvent.java b/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-core/src/main/java/org/apache/hadoop/mapreduce/jobhistory/AMStartedEvent.java index a1447c58b4631..a3fa57ee242b7 100644 --- a/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-core/src/main/java/org/apache/hadoop/mapreduce/jobhistory/AMStartedEvent.java +++ b/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-core/src/main/java/org/apache/hadoop/mapreduce/jobhistory/AMStartedEvent.java @@ -28,7 +28,6 @@ import org.apache.hadoop.yarn.api.records.ContainerId; import org.apache.hadoop.yarn.api.records.timelineservice.TimelineEvent; import org.apache.hadoop.yarn.api.records.timelineservice.TimelineMetric; -import org.apache.hadoop.yarn.util.ConverterUtils; /** * Event to record start of a task attempt diff --git a/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-core/src/main/java/org/apache/hadoop/mapreduce/jobhistory/TaskAttemptStartedEvent.java b/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-core/src/main/java/org/apache/hadoop/mapreduce/jobhistory/TaskAttemptStartedEvent.java index d09d5ca6e9b8a..275ff1d187122 100644 --- a/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-core/src/main/java/org/apache/hadoop/mapreduce/jobhistory/TaskAttemptStartedEvent.java +++ b/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-core/src/main/java/org/apache/hadoop/mapreduce/jobhistory/TaskAttemptStartedEvent.java @@ -30,7 +30,6 @@ import org.apache.hadoop.yarn.api.records.ContainerId; import org.apache.hadoop.yarn.api.records.timelineservice.TimelineEvent; import org.apache.hadoop.yarn.api.records.timelineservice.TimelineMetric; -import org.apache.hadoop.yarn.util.ConverterUtils; /** * Event to record start of a task attempt diff --git a/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-core/src/main/java/org/apache/hadoop/mapreduce/lib/db/DBRecordReader.java b/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-core/src/main/java/org/apache/hadoop/mapreduce/lib/db/DBRecordReader.java index f780715e7acec..90fc67d2bc6dc 100644 --- a/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-core/src/main/java/org/apache/hadoop/mapreduce/lib/db/DBRecordReader.java +++ b/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-core/src/main/java/org/apache/hadoop/mapreduce/lib/db/DBRecordReader.java @@ -18,30 +18,19 @@ package org.apache.hadoop.mapreduce.lib.db; -import java.io.DataInput; -import java.io.DataOutput; import java.io.IOException; import java.sql.Connection; -import java.sql.DatabaseMetaData; import java.sql.PreparedStatement; import java.sql.ResultSet; import java.sql.SQLException; -import java.sql.Statement; -import java.util.ArrayList; -import java.util.List; import org.apache.hadoop.io.LongWritable; -import org.apache.hadoop.io.Writable; -import org.apache.hadoop.mapreduce.InputFormat; import org.apache.hadoop.mapreduce.InputSplit; -import org.apache.hadoop.mapreduce.Job; -import org.apache.hadoop.mapreduce.JobContext; import org.apache.hadoop.mapreduce.RecordReader; import org.apache.hadoop.mapreduce.TaskAttemptContext; import org.apache.hadoop.util.ReflectionUtils; import org.apache.hadoop.classification.InterfaceAudience; import org.apache.hadoop.classification.InterfaceStability; -import org.apache.hadoop.conf.Configurable; import org.apache.hadoop.conf.Configuration; import org.slf4j.Logger; import org.slf4j.LoggerFactory; diff --git a/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-core/src/main/java/org/apache/hadoop/mapreduce/lib/db/DataDrivenDBInputFormat.java b/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-core/src/main/java/org/apache/hadoop/mapreduce/lib/db/DataDrivenDBInputFormat.java index d1c61c4dcb4fd..080c2751c3383 100644 --- a/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-core/src/main/java/org/apache/hadoop/mapreduce/lib/db/DataDrivenDBInputFormat.java +++ b/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-core/src/main/java/org/apache/hadoop/mapreduce/lib/db/DataDrivenDBInputFormat.java @@ -21,9 +21,6 @@ import java.io.DataInput; import java.io.DataOutput; import java.io.IOException; -import java.sql.Connection; -import java.sql.DatabaseMetaData; -import java.sql.PreparedStatement; import java.sql.ResultSet; import java.sql.SQLException; import java.sql.Statement; @@ -36,15 +33,11 @@ import org.apache.hadoop.io.LongWritable; import org.apache.hadoop.io.Text; -import org.apache.hadoop.io.Writable; -import org.apache.hadoop.mapreduce.InputFormat; import org.apache.hadoop.mapreduce.InputSplit; import org.apache.hadoop.mapreduce.Job; import org.apache.hadoop.mapreduce.JobContext; import org.apache.hadoop.mapreduce.MRJobConfig; import org.apache.hadoop.mapreduce.RecordReader; -import org.apache.hadoop.mapreduce.TaskAttemptContext; -import org.apache.hadoop.util.ReflectionUtils; import org.apache.hadoop.classification.InterfaceAudience; import org.apache.hadoop.classification.InterfaceStability; import org.apache.hadoop.conf.Configurable; diff --git a/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-core/src/main/java/org/apache/hadoop/mapreduce/lib/db/DataDrivenDBRecordReader.java b/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-core/src/main/java/org/apache/hadoop/mapreduce/lib/db/DataDrivenDBRecordReader.java index 447a8fd55b46a..0682954698011 100644 --- a/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-core/src/main/java/org/apache/hadoop/mapreduce/lib/db/DataDrivenDBRecordReader.java +++ b/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-core/src/main/java/org/apache/hadoop/mapreduce/lib/db/DataDrivenDBRecordReader.java @@ -18,30 +18,11 @@ package org.apache.hadoop.mapreduce.lib.db; -import java.io.DataInput; -import java.io.DataOutput; -import java.io.IOException; import java.sql.Connection; -import java.sql.DatabaseMetaData; -import java.sql.PreparedStatement; -import java.sql.ResultSet; import java.sql.SQLException; -import java.sql.Statement; -import java.util.ArrayList; -import java.util.List; - -import org.apache.hadoop.io.LongWritable; -import org.apache.hadoop.io.Writable; -import org.apache.hadoop.mapreduce.InputFormat; -import org.apache.hadoop.mapreduce.InputSplit; -import org.apache.hadoop.mapreduce.Job; -import org.apache.hadoop.mapreduce.JobContext; -import org.apache.hadoop.mapreduce.RecordReader; -import org.apache.hadoop.mapreduce.TaskAttemptContext; -import org.apache.hadoop.util.ReflectionUtils; + import org.apache.hadoop.classification.InterfaceAudience; import org.apache.hadoop.classification.InterfaceStability; -import org.apache.hadoop.conf.Configurable; import org.apache.hadoop.conf.Configuration; import org.slf4j.Logger; import org.slf4j.LoggerFactory; diff --git a/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-core/src/main/java/org/apache/hadoop/mapreduce/lib/db/DateSplitter.java b/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-core/src/main/java/org/apache/hadoop/mapreduce/lib/db/DateSplitter.java index b0866f56cf04c..ba85cb3c8d342 100644 --- a/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-core/src/main/java/org/apache/hadoop/mapreduce/lib/db/DateSplitter.java +++ b/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-core/src/main/java/org/apache/hadoop/mapreduce/lib/db/DateSplitter.java @@ -20,8 +20,6 @@ import java.sql.ResultSet; import java.sql.SQLException; -import java.sql.Time; -import java.sql.Timestamp; import java.sql.Types; import java.util.ArrayList; import java.util.Date; diff --git a/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-core/src/main/java/org/apache/hadoop/mapreduce/lib/db/MySQLDBRecordReader.java b/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-core/src/main/java/org/apache/hadoop/mapreduce/lib/db/MySQLDBRecordReader.java index e71e598e5073b..9e5537926af06 100644 --- a/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-core/src/main/java/org/apache/hadoop/mapreduce/lib/db/MySQLDBRecordReader.java +++ b/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-core/src/main/java/org/apache/hadoop/mapreduce/lib/db/MySQLDBRecordReader.java @@ -20,7 +20,6 @@ import java.sql.Connection; import java.sql.ResultSet; -import java.sql.PreparedStatement; import java.sql.SQLException; import org.apache.hadoop.classification.InterfaceAudience; diff --git a/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-core/src/main/java/org/apache/hadoop/mapreduce/lib/db/MySQLDataDrivenDBRecordReader.java b/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-core/src/main/java/org/apache/hadoop/mapreduce/lib/db/MySQLDataDrivenDBRecordReader.java index bd8dea6b34691..a55cdc95a733b 100644 --- a/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-core/src/main/java/org/apache/hadoop/mapreduce/lib/db/MySQLDataDrivenDBRecordReader.java +++ b/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-core/src/main/java/org/apache/hadoop/mapreduce/lib/db/MySQLDataDrivenDBRecordReader.java @@ -20,7 +20,6 @@ import java.sql.Connection; import java.sql.ResultSet; -import java.sql.PreparedStatement; import java.sql.SQLException; import org.apache.hadoop.classification.InterfaceAudience; diff --git a/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-core/src/main/java/org/apache/hadoop/mapreduce/lib/db/OracleDataDrivenDBInputFormat.java b/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-core/src/main/java/org/apache/hadoop/mapreduce/lib/db/OracleDataDrivenDBInputFormat.java index 981b040e10af4..f0f67b9ae682d 100644 --- a/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-core/src/main/java/org/apache/hadoop/mapreduce/lib/db/OracleDataDrivenDBInputFormat.java +++ b/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-core/src/main/java/org/apache/hadoop/mapreduce/lib/db/OracleDataDrivenDBInputFormat.java @@ -18,29 +18,12 @@ package org.apache.hadoop.mapreduce.lib.db; -import java.io.DataInput; -import java.io.DataOutput; import java.io.IOException; -import java.sql.Connection; -import java.sql.DatabaseMetaData; -import java.sql.PreparedStatement; -import java.sql.ResultSet; import java.sql.SQLException; -import java.sql.Statement; import java.sql.Types; -import java.util.ArrayList; -import java.util.List; import org.apache.hadoop.io.LongWritable; -import org.apache.hadoop.io.Text; -import org.apache.hadoop.io.Writable; -import org.apache.hadoop.mapreduce.InputFormat; -import org.apache.hadoop.mapreduce.InputSplit; -import org.apache.hadoop.mapreduce.Job; -import org.apache.hadoop.mapreduce.JobContext; import org.apache.hadoop.mapreduce.RecordReader; -import org.apache.hadoop.mapreduce.TaskAttemptContext; -import org.apache.hadoop.util.ReflectionUtils; import org.apache.hadoop.classification.InterfaceAudience; import org.apache.hadoop.classification.InterfaceStability; import org.apache.hadoop.conf.Configurable; diff --git a/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-core/src/main/java/org/apache/hadoop/mapreduce/lib/db/OracleDataDrivenDBRecordReader.java b/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-core/src/main/java/org/apache/hadoop/mapreduce/lib/db/OracleDataDrivenDBRecordReader.java index 25dcaa0aa5f71..4e5f86b4533f7 100644 --- a/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-core/src/main/java/org/apache/hadoop/mapreduce/lib/db/OracleDataDrivenDBRecordReader.java +++ b/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-core/src/main/java/org/apache/hadoop/mapreduce/lib/db/OracleDataDrivenDBRecordReader.java @@ -19,8 +19,6 @@ package org.apache.hadoop.mapreduce.lib.db; import java.sql.Connection; -import java.sql.ResultSet; -import java.sql.PreparedStatement; import java.sql.SQLException; import org.apache.hadoop.classification.InterfaceAudience; diff --git a/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-core/src/main/java/org/apache/hadoop/mapreduce/lib/db/TextSplitter.java b/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-core/src/main/java/org/apache/hadoop/mapreduce/lib/db/TextSplitter.java index a5c6f56072de1..1cc919999946a 100644 --- a/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-core/src/main/java/org/apache/hadoop/mapreduce/lib/db/TextSplitter.java +++ b/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-core/src/main/java/org/apache/hadoop/mapreduce/lib/db/TextSplitter.java @@ -22,7 +22,6 @@ import java.sql.SQLException; import java.math.BigDecimal; import java.util.ArrayList; -import java.util.Iterator; import java.util.List; import org.slf4j.Logger; diff --git a/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-core/src/main/java/org/apache/hadoop/mapreduce/lib/output/MultipleOutputs.java b/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-core/src/main/java/org/apache/hadoop/mapreduce/lib/output/MultipleOutputs.java index a3a0e76ab81d1..a4d8acbbbfef2 100644 --- a/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-core/src/main/java/org/apache/hadoop/mapreduce/lib/output/MultipleOutputs.java +++ b/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-core/src/main/java/org/apache/hadoop/mapreduce/lib/output/MultipleOutputs.java @@ -20,9 +20,7 @@ import org.apache.hadoop.classification.InterfaceAudience; import org.apache.hadoop.classification.InterfaceStability; import org.apache.hadoop.conf.Configuration; -import org.apache.hadoop.io.Text; import org.apache.hadoop.mapreduce.*; -import org.apache.hadoop.mapreduce.Reducer.Context; import org.apache.hadoop.mapreduce.lib.output.MultipleOutputs; import org.apache.hadoop.mapreduce.task.TaskAttemptContextImpl; import org.apache.hadoop.util.ReflectionUtils; diff --git a/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-core/src/main/java/org/apache/hadoop/mapreduce/lib/output/committer/manifest/files/AbstractManifestData.java b/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-core/src/main/java/org/apache/hadoop/mapreduce/lib/output/committer/manifest/files/AbstractManifestData.java index 7020d5ca2d337..bf67281785c07 100644 --- a/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-core/src/main/java/org/apache/hadoop/mapreduce/lib/output/committer/manifest/files/AbstractManifestData.java +++ b/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-core/src/main/java/org/apache/hadoop/mapreduce/lib/output/committer/manifest/files/AbstractManifestData.java @@ -70,7 +70,6 @@ public static Path unmarshallPath(String path) { throw new RuntimeException( "Failed to parse \"" + path + "\" : " + e, e); - } } diff --git a/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-core/src/main/java/org/apache/hadoop/mapreduce/lib/partition/TotalOrderPartitioner.java b/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-core/src/main/java/org/apache/hadoop/mapreduce/lib/partition/TotalOrderPartitioner.java index 68de7a9f5c25d..c19724e842d26 100644 --- a/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-core/src/main/java/org/apache/hadoop/mapreduce/lib/partition/TotalOrderPartitioner.java +++ b/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-core/src/main/java/org/apache/hadoop/mapreduce/lib/partition/TotalOrderPartitioner.java @@ -31,10 +31,8 @@ import org.apache.hadoop.fs.Path; import org.apache.hadoop.io.BinaryComparable; import org.apache.hadoop.io.IOUtils; -import org.apache.hadoop.io.NullWritable; import org.apache.hadoop.io.SequenceFile; import org.apache.hadoop.io.RawComparator; -import org.apache.hadoop.io.WritableComparable; import org.apache.hadoop.mapreduce.Job; import org.apache.hadoop.mapreduce.Partitioner; import org.apache.hadoop.util.ReflectionUtils; diff --git a/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-core/src/main/java/org/apache/hadoop/mapreduce/task/reduce/MergeManager.java b/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-core/src/main/java/org/apache/hadoop/mapreduce/task/reduce/MergeManager.java index 2ecc55ecbb1cc..0245f64267b30 100644 --- a/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-core/src/main/java/org/apache/hadoop/mapreduce/task/reduce/MergeManager.java +++ b/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-core/src/main/java/org/apache/hadoop/mapreduce/task/reduce/MergeManager.java @@ -20,19 +20,8 @@ import org.apache.hadoop.classification.InterfaceAudience; import org.apache.hadoop.classification.InterfaceStability; -import org.apache.hadoop.fs.FileSystem; -import org.apache.hadoop.fs.LocalDirAllocator; -import org.apache.hadoop.fs.Path; -import org.apache.hadoop.io.compress.CompressionCodec; -import org.apache.hadoop.mapred.Counters; -import org.apache.hadoop.mapred.JobConf; -import org.apache.hadoop.mapred.MapOutputFile; import org.apache.hadoop.mapred.RawKeyValueIterator; -import org.apache.hadoop.mapred.Reducer; -import org.apache.hadoop.mapred.Reporter; -import org.apache.hadoop.mapred.Task.CombineOutputCollector; import org.apache.hadoop.mapreduce.TaskAttemptID; -import org.apache.hadoop.util.Progress; import java.io.IOException; diff --git a/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-core/src/main/java/org/apache/hadoop/mapreduce/util/ProcessTree.java b/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-core/src/main/java/org/apache/hadoop/mapreduce/util/ProcessTree.java index 93cff0f95c534..585a21d568231 100644 --- a/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-core/src/main/java/org/apache/hadoop/mapreduce/util/ProcessTree.java +++ b/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-core/src/main/java/org/apache/hadoop/mapreduce/util/ProcessTree.java @@ -19,7 +19,6 @@ package org.apache.hadoop.mapreduce.util; import java.io.IOException; -import java.util.Arrays; import org.apache.hadoop.classification.InterfaceAudience; import org.apache.hadoop.classification.InterfaceStability; diff --git a/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-core/src/main/resources/mapred-default.xml b/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-core/src/main/resources/mapred-default.xml index d315a00ba4a6c..848d33d92453b 100644 --- a/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-core/src/main/resources/mapred-default.xml +++ b/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-core/src/main/resources/mapred-default.xml @@ -286,6 +286,13 @@ + + mapreduce.task.ping-for-liveliness-check.enabled + false + Whether to consider ping from tasks in liveliness check. + + + mapreduce.map.memory.mb -1 @@ -1715,7 +1722,7 @@ must specify the appropriate classpath for that archive, and the name of the archive must be present in the classpath. If mapreduce.app-submission.cross-platform is false, platform-specific - environment vairable expansion syntax would be used to construct the default + environment variable expansion syntax would be used to construct the default CLASSPATH entries. For Linux: $HADOOP_MAPRED_HOME/share/hadoop/mapreduce/*, diff --git a/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-core/src/test/java/org/apache/hadoop/mapreduce/lib/jobcontrol/TestJobControl.java b/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-core/src/test/java/org/apache/hadoop/mapreduce/lib/jobcontrol/TestJobControl.java index 41a4eaf64a759..48006f12282c3 100644 --- a/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-core/src/test/java/org/apache/hadoop/mapreduce/lib/jobcontrol/TestJobControl.java +++ b/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-core/src/test/java/org/apache/hadoop/mapreduce/lib/jobcontrol/TestJobControl.java @@ -20,7 +20,6 @@ import static org.junit.Assert.assertTrue; import java.io.IOException; -import java.util.List; import org.apache.hadoop.conf.Configuration; import org.junit.Test; diff --git a/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-core/src/test/java/org/apache/hadoop/mapreduce/task/reduce/TestEventFetcher.java b/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-core/src/test/java/org/apache/hadoop/mapreduce/task/reduce/TestEventFetcher.java index 1a0bfc751d380..9df55f126dbf8 100644 --- a/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-core/src/test/java/org/apache/hadoop/mapreduce/task/reduce/TestEventFetcher.java +++ b/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-core/src/test/java/org/apache/hadoop/mapreduce/task/reduce/TestEventFetcher.java @@ -19,7 +19,6 @@ import static org.mockito.Mockito.any; import static org.mockito.Mockito.anyInt; -import static org.mockito.Mockito.anyString; import static org.mockito.Mockito.eq; import static org.mockito.Mockito.inOrder; import static org.mockito.Mockito.mock; diff --git a/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-hs/src/main/java/org/apache/hadoop/mapreduce/v2/hs/HistoryClientService.java b/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-hs/src/main/java/org/apache/hadoop/mapreduce/v2/hs/HistoryClientService.java index 0136820a39296..6f2a528452c23 100644 --- a/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-hs/src/main/java/org/apache/hadoop/mapreduce/v2/hs/HistoryClientService.java +++ b/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-hs/src/main/java/org/apache/hadoop/mapreduce/v2/hs/HistoryClientService.java @@ -84,7 +84,6 @@ import org.apache.hadoop.yarn.client.ClientRMProxy; import org.apache.hadoop.yarn.factories.RecordFactory; import org.apache.hadoop.yarn.factory.providers.RecordFactoryProvider; -import org.apache.hadoop.yarn.ipc.RPCUtil; import org.apache.hadoop.yarn.ipc.YarnRPC; import org.apache.hadoop.yarn.util.Records; import org.apache.hadoop.yarn.webapp.WebApp; diff --git a/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-hs/src/main/java/org/apache/hadoop/mapreduce/v2/hs/webapp/HsNavBlock.java b/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-hs/src/main/java/org/apache/hadoop/mapreduce/v2/hs/webapp/HsNavBlock.java index fb077dd784991..8b0e741ae6ac9 100644 --- a/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-hs/src/main/java/org/apache/hadoop/mapreduce/v2/hs/webapp/HsNavBlock.java +++ b/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-hs/src/main/java/org/apache/hadoop/mapreduce/v2/hs/webapp/HsNavBlock.java @@ -21,7 +21,6 @@ import org.apache.hadoop.conf.Configuration; import org.apache.hadoop.mapreduce.v2.app.webapp.App; import org.apache.hadoop.mapreduce.v2.util.MRApps; -import org.apache.hadoop.yarn.conf.YarnConfiguration; import org.apache.hadoop.yarn.server.webapp.WebPageUtils; import org.apache.hadoop.yarn.webapp.hamlet2.Hamlet; import org.apache.hadoop.yarn.webapp.hamlet2.Hamlet.DIV; diff --git a/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-jobclient/src/test/java/org/apache/hadoop/io/TestSequenceFileMergeProgress.java b/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-jobclient/src/test/java/org/apache/hadoop/io/TestSequenceFileMergeProgress.java index 4262f1f23b5d8..4f1318c044103 100644 --- a/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-jobclient/src/test/java/org/apache/hadoop/io/TestSequenceFileMergeProgress.java +++ b/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-jobclient/src/test/java/org/apache/hadoop/io/TestSequenceFileMergeProgress.java @@ -19,16 +19,11 @@ package org.apache.hadoop.io; import java.io.IOException; -import java.util.ArrayList; -import java.util.List; -import java.util.Random; import org.apache.hadoop.fs.*; import org.apache.hadoop.io.*; import org.apache.hadoop.io.SequenceFile.CompressionType; import org.apache.hadoop.io.SequenceFile.Sorter.RawKeyValueIterator; -import org.apache.hadoop.io.SequenceFile.Sorter.SegmentDescriptor; -import org.apache.hadoop.io.compress.CompressionCodec; import org.apache.hadoop.io.compress.DefaultCodec; import org.apache.hadoop.mapred.*; diff --git a/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-jobclient/src/test/java/org/apache/hadoop/mapred/EmptyInputFormat.java b/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-jobclient/src/test/java/org/apache/hadoop/mapred/EmptyInputFormat.java index edb5476febae5..4eaa73566c247 100644 --- a/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-jobclient/src/test/java/org/apache/hadoop/mapred/EmptyInputFormat.java +++ b/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-jobclient/src/test/java/org/apache/hadoop/mapred/EmptyInputFormat.java @@ -19,8 +19,6 @@ import java.io.IOException; -import org.apache.hadoop.fs.FileSystem; - /** * InputFormat which simulates the absence of input data * by returning zero split. diff --git a/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-jobclient/src/test/java/org/apache/hadoop/mapred/TestLazyOutput.java b/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-jobclient/src/test/java/org/apache/hadoop/mapred/TestLazyOutput.java index 04a5127aaf202..ef559d95081db 100644 --- a/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-jobclient/src/test/java/org/apache/hadoop/mapred/TestLazyOutput.java +++ b/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-jobclient/src/test/java/org/apache/hadoop/mapred/TestLazyOutput.java @@ -32,8 +32,6 @@ import org.apache.hadoop.hdfs.MiniDFSCluster; import org.apache.hadoop.io.LongWritable; import org.apache.hadoop.io.Text; -import org.apache.hadoop.io.Writable; -import org.apache.hadoop.io.WritableComparable; import org.apache.hadoop.mapred.lib.LazyOutputFormat; import org.junit.Test; import static org.junit.Assert.assertTrue; diff --git a/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-jobclient/src/test/java/org/apache/hadoop/mapred/TestMiniMRClasspath.java b/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-jobclient/src/test/java/org/apache/hadoop/mapred/TestMiniMRClasspath.java index 8984e3740187e..60e32683c3a75 100644 --- a/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-jobclient/src/test/java/org/apache/hadoop/mapred/TestMiniMRClasspath.java +++ b/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-jobclient/src/test/java/org/apache/hadoop/mapred/TestMiniMRClasspath.java @@ -24,8 +24,6 @@ import java.io.InputStreamReader; import java.net.URI; -import junit.framework.TestCase; - import org.apache.hadoop.conf.Configuration; import org.apache.hadoop.fs.FileSystem; import org.apache.hadoop.fs.FileUtil; diff --git a/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-jobclient/src/test/java/org/apache/hadoop/mapred/ThreadedMapBenchmark.java b/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-jobclient/src/test/java/org/apache/hadoop/mapred/ThreadedMapBenchmark.java index 68fe1a0198dc6..5698df3128a04 100644 --- a/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-jobclient/src/test/java/org/apache/hadoop/mapred/ThreadedMapBenchmark.java +++ b/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-jobclient/src/test/java/org/apache/hadoop/mapred/ThreadedMapBenchmark.java @@ -26,7 +26,6 @@ import org.apache.hadoop.fs.FileSystem; import org.apache.hadoop.fs.Path; import org.apache.hadoop.io.BytesWritable; -import org.apache.hadoop.io.Text; import org.apache.hadoop.io.Writable; import org.apache.hadoop.io.WritableComparable; import org.apache.hadoop.mapred.SortValidator.RecordStatsChecker.NonSplitableSequenceFileInputFormat; diff --git a/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-jobclient/src/test/java/org/apache/hadoop/mapreduce/FailJob.java b/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-jobclient/src/test/java/org/apache/hadoop/mapreduce/FailJob.java index 6f83acf2f5441..105b00295e15f 100644 --- a/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-jobclient/src/test/java/org/apache/hadoop/mapreduce/FailJob.java +++ b/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-jobclient/src/test/java/org/apache/hadoop/mapreduce/FailJob.java @@ -23,10 +23,8 @@ import org.apache.hadoop.conf.Configuration; import org.apache.hadoop.conf.Configured; -import org.apache.hadoop.fs.FSDataOutputStream; import org.apache.hadoop.fs.FileSystem; import org.apache.hadoop.fs.Path; -import org.apache.hadoop.io.IntWritable; import org.apache.hadoop.io.LongWritable; import org.apache.hadoop.io.NullWritable; import org.apache.hadoop.io.Text; diff --git a/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-jobclient/src/test/java/org/apache/hadoop/mapreduce/TestMapReduceLazyOutput.java b/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-jobclient/src/test/java/org/apache/hadoop/mapreduce/TestMapReduceLazyOutput.java index 7c010382eab68..4d4f2276f4ea4 100644 --- a/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-jobclient/src/test/java/org/apache/hadoop/mapreduce/TestMapReduceLazyOutput.java +++ b/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-jobclient/src/test/java/org/apache/hadoop/mapreduce/TestMapReduceLazyOutput.java @@ -32,7 +32,6 @@ import org.apache.hadoop.hdfs.MiniDFSCluster; import org.apache.hadoop.io.LongWritable; import org.apache.hadoop.io.Text; -import org.apache.hadoop.mapred.JobConf; import org.apache.hadoop.mapred.MiniMRCluster; import org.apache.hadoop.mapred.Utils; import org.apache.hadoop.mapreduce.lib.input.FileInputFormat; diff --git a/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-jobclient/src/test/java/org/apache/hadoop/mapreduce/lib/input/TestMultipleInputs.java b/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-jobclient/src/test/java/org/apache/hadoop/mapreduce/lib/input/TestMultipleInputs.java index a6f8a725f6991..aca07468b6d91 100644 --- a/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-jobclient/src/test/java/org/apache/hadoop/mapreduce/lib/input/TestMultipleInputs.java +++ b/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-jobclient/src/test/java/org/apache/hadoop/mapreduce/lib/input/TestMultipleInputs.java @@ -39,7 +39,6 @@ import org.junit.Before; import org.junit.Test; import static org.junit.Assert.assertEquals; -import static org.junit.Assert.assertTrue; /** * @see TestDelegatingInputFormat diff --git a/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-jobclient/src/test/java/org/apache/hadoop/mapreduce/lib/join/TestJoinDatamerge.java b/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-jobclient/src/test/java/org/apache/hadoop/mapreduce/lib/join/TestJoinDatamerge.java index 1173ea4fa478f..11741d4450a74 100644 --- a/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-jobclient/src/test/java/org/apache/hadoop/mapreduce/lib/join/TestJoinDatamerge.java +++ b/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-jobclient/src/test/java/org/apache/hadoop/mapreduce/lib/join/TestJoinDatamerge.java @@ -32,9 +32,8 @@ import org.apache.hadoop.mapreduce.lib.input.SequenceFileInputFormat; import org.apache.hadoop.mapreduce.lib.output.FileOutputFormat; import org.apache.hadoop.mapreduce.lib.output.SequenceFileOutputFormat; -import org.junit.After; + import org.junit.AfterClass; -import org.junit.Before; import org.junit.BeforeClass; import org.junit.Test; diff --git a/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-jobclient/src/test/java/org/apache/hadoop/mapreduce/lib/partition/TestTotalOrderPartitioner.java b/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-jobclient/src/test/java/org/apache/hadoop/mapreduce/lib/partition/TestTotalOrderPartitioner.java index bdb4ff4794ee9..14dd12ac27713 100644 --- a/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-jobclient/src/test/java/org/apache/hadoop/mapreduce/lib/partition/TestTotalOrderPartitioner.java +++ b/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-jobclient/src/test/java/org/apache/hadoop/mapreduce/lib/partition/TestTotalOrderPartitioner.java @@ -36,7 +36,6 @@ import org.apache.hadoop.io.SequenceFile.CompressionType; import org.apache.hadoop.io.serializer.JavaSerialization; import org.apache.hadoop.io.serializer.JavaSerializationComparator; -import org.apache.hadoop.io.serializer.Serialization; import org.apache.hadoop.io.serializer.WritableSerialization; import org.apache.hadoop.mapreduce.MRJobConfig; import org.junit.Test; diff --git a/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-jobclient/src/test/java/org/apache/hadoop/mapreduce/security/TestJHSSecurity.java b/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-jobclient/src/test/java/org/apache/hadoop/mapreduce/security/TestJHSSecurity.java index 6115c590d5fd6..9e58d460d1783 100644 --- a/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-jobclient/src/test/java/org/apache/hadoop/mapreduce/security/TestJHSSecurity.java +++ b/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-jobclient/src/test/java/org/apache/hadoop/mapreduce/security/TestJHSSecurity.java @@ -18,7 +18,6 @@ package org.apache.hadoop.mapreduce.security; -import static org.junit.Assert.assertTrue; import static org.junit.Assert.fail; import java.io.IOException; @@ -26,6 +25,8 @@ import java.security.PrivilegedAction; import java.security.PrivilegedExceptionAction; +import org.apache.hadoop.security.token.SecretManager; +import org.apache.hadoop.test.LambdaTestUtils; import org.junit.Assert; import org.apache.hadoop.conf.Configuration; @@ -61,7 +62,7 @@ public class TestJHSSecurity { LoggerFactory.getLogger(TestJHSSecurity.class); @Test - public void testDelegationToken() throws IOException, InterruptedException { + public void testDelegationToken() throws Exception { org.apache.log4j.Logger rootLogger = LogManager.getRootLogger(); rootLogger.setLevel(Level.DEBUG); @@ -80,7 +81,7 @@ public void testDelegationToken() throws IOException, InterruptedException { final long renewInterval = 10000l; JobHistoryServer jobHistoryServer = null; - MRClientProtocol clientUsingDT = null; + MRClientProtocol clientUsingDT; long tokenFetchTime; try { jobHistoryServer = new JobHistoryServer() { @@ -155,14 +156,11 @@ protected JHSDelegationTokenSecretManager createJHSSecretManager( } Thread.sleep(50l); LOG.info("At time: " + System.currentTimeMillis() + ", token should be invalid"); - // Token should have expired. - try { - clientUsingDT.getJobReport(jobReportRequest); - fail("Should not have succeeded with an expired token"); - } catch (IOException e) { - assertTrue(e.getCause().getMessage().contains("is expired")); - } - + // Token should have expired. + final MRClientProtocol finalClientUsingDT = clientUsingDT; + LambdaTestUtils.intercept(SecretManager.InvalidToken.class, "has expired", + () -> finalClientUsingDT.getJobReport(jobReportRequest)); + // Test cancellation // Stop the existing proxy, start another. if (clientUsingDT != null) { diff --git a/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-jobclient/src/test/java/org/apache/hadoop/mapreduce/security/token/delegation/TestDelegationToken.java b/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-jobclient/src/test/java/org/apache/hadoop/mapreduce/security/token/delegation/TestDelegationToken.java index a031cd73ac0ed..06b0c20c3f39a 100644 --- a/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-jobclient/src/test/java/org/apache/hadoop/mapreduce/security/token/delegation/TestDelegationToken.java +++ b/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-jobclient/src/test/java/org/apache/hadoop/mapreduce/security/token/delegation/TestDelegationToken.java @@ -21,7 +21,6 @@ import org.apache.hadoop.io.DataInputBuffer; import org.apache.hadoop.io.Text; import org.apache.hadoop.mapred.JobClient; -import org.apache.hadoop.mapred.JobConf; import org.apache.hadoop.mapred.MiniMRCluster; import org.apache.hadoop.security.AccessControlException; import org.apache.hadoop.security.UserGroupInformation; diff --git a/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-jobclient/src/test/java/org/apache/hadoop/mapreduce/v2/TestNonExistentJob.java b/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-jobclient/src/test/java/org/apache/hadoop/mapreduce/v2/TestNonExistentJob.java index e90c509d7a8af..d1e81b2b11631 100644 --- a/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-jobclient/src/test/java/org/apache/hadoop/mapreduce/v2/TestNonExistentJob.java +++ b/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-jobclient/src/test/java/org/apache/hadoop/mapreduce/v2/TestNonExistentJob.java @@ -31,7 +31,6 @@ import org.junit.Before; import org.junit.Test; -import java.io.IOException; import java.net.InetAddress; import static org.junit.Assert.assertNull; diff --git a/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-jobclient/src/test/java/testjar/ClassWordCount.java b/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-jobclient/src/test/java/testjar/ClassWordCount.java index 5f3abd232a18e..17fe2df86f4f3 100644 --- a/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-jobclient/src/test/java/testjar/ClassWordCount.java +++ b/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-jobclient/src/test/java/testjar/ClassWordCount.java @@ -18,22 +18,11 @@ package testjar; -import java.io.*; -import java.util.*; - -import org.apache.hadoop.fs.Path; import org.apache.hadoop.io.IntWritable; import org.apache.hadoop.io.LongWritable; import org.apache.hadoop.io.Text; -import org.apache.hadoop.io.Writable; -import org.apache.hadoop.io.WritableComparable; -import org.apache.hadoop.mapred.JobClient; -import org.apache.hadoop.mapred.JobConf; import org.apache.hadoop.mapred.Mapper; -import org.apache.hadoop.mapred.OutputCollector; import org.apache.hadoop.mapred.Reducer; -import org.apache.hadoop.mapred.Reporter; -import org.apache.hadoop.mapred.MapReduceBase; import org.apache.hadoop.mapred.WordCount; /** diff --git a/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-nativetask/src/main/java/org/apache/hadoop/mapred/nativetask/NativeMapOutputCollectorDelegator.java b/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-nativetask/src/main/java/org/apache/hadoop/mapred/nativetask/NativeMapOutputCollectorDelegator.java index 4ec1f7a1c561c..2c8d13fff1922 100644 --- a/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-nativetask/src/main/java/org/apache/hadoop/mapred/nativetask/NativeMapOutputCollectorDelegator.java +++ b/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-nativetask/src/main/java/org/apache/hadoop/mapred/nativetask/NativeMapOutputCollectorDelegator.java @@ -32,7 +32,6 @@ import org.apache.hadoop.mapred.nativetask.serde.NativeSerialization; import org.apache.hadoop.mapreduce.MRConfig; import org.apache.hadoop.mapreduce.MRJobConfig; -import org.apache.hadoop.mapreduce.TaskCounter; import org.apache.hadoop.util.QuickSort; import org.slf4j.Logger; import org.slf4j.LoggerFactory; diff --git a/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-nativetask/src/test/java/org/apache/hadoop/mapred/nativetask/buffer/TestBufferPushPull.java b/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-nativetask/src/test/java/org/apache/hadoop/mapred/nativetask/buffer/TestBufferPushPull.java index 48c32c0239f66..1c77aa850fa6c 100644 --- a/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-nativetask/src/test/java/org/apache/hadoop/mapred/nativetask/buffer/TestBufferPushPull.java +++ b/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-nativetask/src/test/java/org/apache/hadoop/mapred/nativetask/buffer/TestBufferPushPull.java @@ -37,7 +37,6 @@ import org.apache.hadoop.mapred.nativetask.handlers.IDataLoader; import org.apache.hadoop.mapred.nativetask.testutil.TestInput; import org.apache.hadoop.mapred.nativetask.testutil.TestInput.KV; -import org.apache.hadoop.mapred.nativetask.util.ReadWriteBuffer; import org.apache.hadoop.util.Progress; import org.junit.Assert; diff --git a/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-nativetask/src/test/java/org/apache/hadoop/mapred/nativetask/kvtest/TestInputFile.java b/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-nativetask/src/test/java/org/apache/hadoop/mapred/nativetask/kvtest/TestInputFile.java index 4cf534b9c6f0f..aa581e5a69148 100644 --- a/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-nativetask/src/test/java/org/apache/hadoop/mapred/nativetask/kvtest/TestInputFile.java +++ b/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-nativetask/src/test/java/org/apache/hadoop/mapred/nativetask/kvtest/TestInputFile.java @@ -22,7 +22,6 @@ import java.util.Random; import org.apache.hadoop.conf.Configuration; -import org.apache.hadoop.fs.FileSystem; import org.apache.hadoop.fs.Path; import org.apache.hadoop.io.BooleanWritable; import org.apache.hadoop.io.ByteWritable; diff --git a/hadoop-mapreduce-project/hadoop-mapreduce-examples/src/main/java/org/apache/hadoop/examples/terasort/TeraChecksum.java b/hadoop-mapreduce-project/hadoop-mapreduce-examples/src/main/java/org/apache/hadoop/examples/terasort/TeraChecksum.java index 652f32c00b576..12d34ea7d6639 100644 --- a/hadoop-mapreduce-project/hadoop-mapreduce-examples/src/main/java/org/apache/hadoop/examples/terasort/TeraChecksum.java +++ b/hadoop-mapreduce-project/hadoop-mapreduce-examples/src/main/java/org/apache/hadoop/examples/terasort/TeraChecksum.java @@ -25,7 +25,6 @@ import org.apache.hadoop.fs.Path; import org.apache.hadoop.io.NullWritable; import org.apache.hadoop.io.Text; -import org.apache.hadoop.mapreduce.Cluster; import org.apache.hadoop.mapreduce.Job; import org.apache.hadoop.mapreduce.Mapper; import org.apache.hadoop.mapreduce.Reducer; diff --git a/hadoop-mapreduce-project/hadoop-mapreduce-examples/src/main/java/org/apache/hadoop/examples/terasort/TeraGen.java b/hadoop-mapreduce-project/hadoop-mapreduce-examples/src/main/java/org/apache/hadoop/examples/terasort/TeraGen.java index ef20c4c23bd58..161e9715c8b0f 100644 --- a/hadoop-mapreduce-project/hadoop-mapreduce-examples/src/main/java/org/apache/hadoop/examples/terasort/TeraGen.java +++ b/hadoop-mapreduce-project/hadoop-mapreduce-examples/src/main/java/org/apache/hadoop/examples/terasort/TeraGen.java @@ -33,7 +33,6 @@ import org.apache.hadoop.io.Text; import org.apache.hadoop.io.Writable; import org.apache.hadoop.io.WritableUtils; -import org.apache.hadoop.mapreduce.Cluster; import org.apache.hadoop.mapreduce.Counter; import org.apache.hadoop.mapreduce.InputFormat; import org.apache.hadoop.mapreduce.InputSplit; diff --git a/hadoop-mapreduce-project/hadoop-mapreduce-examples/src/main/java/org/apache/hadoop/examples/terasort/TeraValidate.java b/hadoop-mapreduce-project/hadoop-mapreduce-examples/src/main/java/org/apache/hadoop/examples/terasort/TeraValidate.java index 6cff25f5def7c..f722fc41341a0 100644 --- a/hadoop-mapreduce-project/hadoop-mapreduce-examples/src/main/java/org/apache/hadoop/examples/terasort/TeraValidate.java +++ b/hadoop-mapreduce-project/hadoop-mapreduce-examples/src/main/java/org/apache/hadoop/examples/terasort/TeraValidate.java @@ -26,7 +26,6 @@ import org.apache.hadoop.fs.Path; import org.apache.hadoop.io.BytesWritable; import org.apache.hadoop.io.Text; -import org.apache.hadoop.mapreduce.Cluster; import org.apache.hadoop.mapreduce.Job; import org.apache.hadoop.mapreduce.Mapper; import org.apache.hadoop.mapreduce.Reducer; diff --git a/hadoop-project-dist/pom.xml b/hadoop-project-dist/pom.xml index 84577a20a2b46..9af619aba3b2c 100644 --- a/hadoop-project-dist/pom.xml +++ b/hadoop-project-dist/pom.xml @@ -134,7 +134,7 @@ false - 3.3.2 + 3.3.3 -unstable diff --git a/hadoop-project/pom.xml b/hadoop-project/pom.xml index 76ba2eeaa021e..72504c1825e90 100644 --- a/hadoop-project/pom.xml +++ b/hadoop-project/pom.xml @@ -48,7 +48,7 @@ 1.0.9 - 2.12.1 + 2.12.2 2.8.1 @@ -111,8 +111,8 @@ 1.68 - 2.0.0-M21 - 1.0.0-M33 + 2.0.0.AM26 + 2.0.0 1.9.4 @@ -135,10 +135,13 @@ 4.0.3 6.2.1.jre7 2.7.5 + 4.9.3 + 1.4.10 + 1.4.10 2.0.6.1 5.2.0 2.2.21 - 2.8.9 + 2.9.0 3.2.4 3.10.6.Final 4.1.68.Final @@ -151,7 +154,7 @@ 3.5.1 1.10.0 - 1.5.0.Final + 1.7.0 1.8 @@ -172,7 +175,6 @@ ${maven-surefire-plugin.version} 3.1.0 - 3.1 2.5.1 2.6 3.2.1 @@ -217,19 +219,46 @@ v12.22.1 v1.22.5 1.10.11 + 1.20 - com.squareup.okhttp + com.squareup.okhttp3 okhttp - ${okhttp.version} + ${okhttp3.version} + + + org.jetbrains.kotlin + kotlin-stdlib + + + org.jetbrains.kotlin + kotlin-stdlib-common + + + + + org.jetbrains.kotlin + kotlin-stdlib + ${kotlin-stdlib.verion} + + + org.jetbrains + annotations + + + + + org.jetbrains.kotlin + kotlin-stdlib-common + ${kotlin-stdlib-common.version} com.squareup.okhttp3 mockwebserver - 3.7.0 + 4.9.3 test @@ -1561,6 +1590,16 @@ + + org.openjdk.jmh + jmh-core + ${jmh.version} + + + org.openjdk.jmh + jmh-generator-annprocess + ${jmh.version} + org.apache.curator curator-test @@ -1856,7 +1895,6 @@ org.apache.maven.plugins maven-compiler-plugin - ${maven-compiler-plugin.version} ${javac.version} ${javac.version} diff --git a/hadoop-tools/hadoop-aws/pom.xml b/hadoop-tools/hadoop-aws/pom.xml index 5583bb7ad05ec..6d9085b51a078 100644 --- a/hadoop-tools/hadoop-aws/pom.xml +++ b/hadoop-tools/hadoop-aws/pom.xml @@ -405,6 +405,39 @@ + + org.apache.maven.plugins + maven-enforcer-plugin + + + banned-illegal-imports + process-sources + + enforce + + + + + false + Restrict mapreduce imports to committer code + + org.apache.hadoop.fs.s3a.commit.AbstractS3ACommitter + org.apache.hadoop.fs.s3a.commit.AbstractS3ACommitterFactory + org.apache.hadoop.fs.s3a.commit.S3ACommitterFactory + org.apache.hadoop.fs.s3a.commit.impl.* + org.apache.hadoop.fs.s3a.commit.magic.* + org.apache.hadoop.fs.s3a.commit.staging.* + + + org.apache.hadoop.mapreduce.** + org.apache.hadoop.mapred.** + + + + + + + diff --git a/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/Constants.java b/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/Constants.java index e5369b848830a..764a6adaca27d 100644 --- a/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/Constants.java +++ b/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/Constants.java @@ -429,6 +429,57 @@ private Constants() { */ public static final String CONTENT_ENCODING = "fs.s3a.object.content.encoding"; + /** + * S3 storage class: standard, reduced_redundancy, intelligent_tiering etc. + * Value {@value }. + */ + public static final String STORAGE_CLASS = "fs.s3a.create.storage.class"; + + /** + * S3 Storage option: {@value}. + */ + public static final String STORAGE_CLASS_STANDARD = "standard"; + + /** + * S3 Storage option: {@value}. + */ + public static final String STORAGE_CLASS_REDUCED_REDUNDANCY = "reduced_redundancy"; + + /** + * S3 Storage option: {@value}. + */ + public static final String STORAGE_CLASS_GLACIER = "glacier"; + + /** + * S3 Storage option: {@value}. + */ + public static final String STORAGE_CLASS_STANDARD_INFREQUENT_ACCESS = "standard_ia"; + + /** + * S3 Storage option: {@value}. + */ + public static final String STORAGE_CLASS_ONEZONE_INFREQUENT_ACCESS = "onezone_ia"; + + /** + * S3 Storage option: {@value}. + */ + public static final String STORAGE_CLASS_INTELLIGENT_TIERING = "intelligent_tiering"; + + /** + * S3 Storage option: {@value}. + */ + public static final String STORAGE_CLASS_DEEP_ARCHIVE = "deep_archive"; + + /** + * S3 Storage option: {@value}. + */ + public static final String STORAGE_CLASS_OUTPOSTS = "outposts"; + + /** + * S3 Storage option: {@value}. + */ + public static final String STORAGE_CLASS_GLACIER_INSTANT_RETRIEVAL = "glacier_ir"; + // should we try to purge old multipart uploads when starting up public static final String PURGE_EXISTING_MULTIPART = "fs.s3a.multipart.purge"; @@ -1108,4 +1159,48 @@ private Constants() { * Require that all S3 access is made through Access Points. */ public static final String AWS_S3_ACCESSPOINT_REQUIRED = "fs.s3a.accesspoint.required"; + + /** + * Flag for create performance. + * This is *not* a configuration option; it is for use in the + * {code createFile()} builder. + * Value {@value}. + */ + public static final String FS_S3A_CREATE_PERFORMANCE = "fs.s3a.create.performance"; + + /** + * Prefix for adding a header to the object when created. + * The actual value must have a "." suffix and then the actual header. + * This is *not* a configuration option; it is only for use in the + * {code createFile()} builder. + * Value {@value}. + */ + public static final String FS_S3A_CREATE_HEADER = "fs.s3a.create.header"; + + /** + * What is the smallest reasonable seek in bytes such + * that we group ranges together during vectored read operation. + * Value : {@value}. + */ + public static final String AWS_S3_VECTOR_READS_MIN_SEEK_SIZE = + "fs.s3a.vectored.read.min.seek.size"; + + /** + * What is the largest merged read size in bytes such + * that we group ranges together during vectored read. + * Setting this value to 0 will disable merging of ranges. + * Value : {@value}. + */ + public static final String AWS_S3_VECTOR_READS_MAX_MERGED_READ_SIZE = + "fs.s3a.vectored.read.max.merged.size"; + + /** + * Default minimum seek in bytes during vectored reads : {@value}. + */ + public static final int DEFAULT_AWS_S3_VECTOR_READS_MIN_SEEK_SIZE = 4896; // 4K + + /** + * Default maximum read size in bytes during vectored reads : {@value}. + */ + public static final int DEFAULT_AWS_S3_VECTOR_READS_MAX_MERGED_READ_SIZE = 1253376; //1M } diff --git a/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/S3ABlockOutputStream.java b/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/S3ABlockOutputStream.java index 6e6871751d7e4..8b1865c77c9eb 100644 --- a/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/S3ABlockOutputStream.java +++ b/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/S3ABlockOutputStream.java @@ -40,6 +40,7 @@ import com.amazonaws.services.s3.model.PutObjectResult; import com.amazonaws.services.s3.model.UploadPartRequest; +import org.apache.hadoop.fs.s3a.impl.PutObjectOptions; import org.apache.hadoop.util.Preconditions; import org.apache.hadoop.thirdparty.com.google.common.util.concurrent.Futures; import org.apache.hadoop.thirdparty.com.google.common.util.concurrent.ListenableFuture; @@ -68,6 +69,7 @@ import static org.apache.hadoop.fs.s3a.S3AUtils.*; import static org.apache.hadoop.fs.s3a.Statistic.*; import static org.apache.hadoop.fs.s3a.statistics.impl.EmptyS3AStatisticsContext.EMPTY_BLOCK_OUTPUT_STREAM_STATISTICS; +import static org.apache.hadoop.fs.statistics.impl.IOStatisticsBinding.trackDuration; import static org.apache.hadoop.fs.statistics.impl.IOStatisticsBinding.trackDurationOfInvocation; import static org.apache.hadoop.io.IOUtils.cleanupWithLogger; @@ -103,6 +105,11 @@ class S3ABlockOutputStream extends OutputStream implements /** IO Statistics. */ private final IOStatistics iostatistics; + /** + * The options this instance was created with. + */ + private final BlockOutputStreamBuilder builder; + /** Total bytes for uploads submitted so far. */ private long bytesSubmitted; @@ -167,6 +174,7 @@ class S3ABlockOutputStream extends OutputStream implements S3ABlockOutputStream(BlockOutputStreamBuilder builder) throws IOException { builder.validate(); + this.builder = builder; this.key = builder.key; this.blockFactory = builder.blockFactory; this.blockSize = (int) builder.blockSize; @@ -332,6 +340,7 @@ public synchronized void write(byte[] source, int offset, int len) * initializing the upload, or if a previous operation * has failed. */ + @Retries.RetryTranslated private synchronized void uploadCurrentBlock(boolean isLast) throws IOException { Preconditions.checkState(hasActiveBlock(), "No active block"); @@ -353,6 +362,7 @@ private synchronized void uploadCurrentBlock(boolean isLast) * can take time and potentially fail. * @throws IOException failure to initialize the upload */ + @Retries.RetryTranslated private void initMultipartUpload() throws IOException { if (multiPartUpload == null) { LOG.debug("Initiating Multipart upload"); @@ -546,9 +556,15 @@ private int putObject() throws IOException { int size = block.dataSize(); final S3ADataBlocks.BlockUploadData uploadData = block.startUpload(); final PutObjectRequest putObjectRequest = uploadData.hasFile() ? - writeOperationHelper.createPutObjectRequest(key, uploadData.getFile()) - : writeOperationHelper.createPutObjectRequest(key, - uploadData.getUploadStream(), size, null); + writeOperationHelper.createPutObjectRequest( + key, + uploadData.getFile(), + builder.putOptions) + : writeOperationHelper.createPutObjectRequest( + key, + uploadData.getUploadStream(), + size, + builder.putOptions); BlockUploadProgress callback = new BlockUploadProgress( block, progressListener, now()); @@ -559,7 +575,7 @@ private int putObject() throws IOException { try { // the putObject call automatically closes the input // stream afterwards. - return writeOperationHelper.putObject(putObjectRequest); + return writeOperationHelper.putObject(putObjectRequest, builder.putOptions); } finally { cleanupWithLogger(LOG, uploadData, block); } @@ -702,8 +718,21 @@ private class MultiPartUpload { */ private IOException blockUploadFailure; + /** + * Constructor. + * Initiates the MPU request against S3. + * @param key upload destination + * @throws IOException failure + */ + + @Retries.RetryTranslated MultiPartUpload(String key) throws IOException { - this.uploadId = writeOperationHelper.initiateMultiPartUpload(key); + this.uploadId = trackDuration(statistics, + OBJECT_MULTIPART_UPLOAD_INITIATED.getSymbol(), + () -> writeOperationHelper.initiateMultiPartUpload( + key, + builder.putOptions)); + this.partETagsFutures = new ArrayList<>(2); LOG.debug("Initiated multi-part upload for {} with " + "id '{}'", writeOperationHelper, uploadId); @@ -887,7 +916,8 @@ private void complete(List partETags) uploadId, partETags, bytesSubmitted, - errorCount); + errorCount, + builder.putOptions); }); } finally { statistics.exceptionInMultipartComplete(errorCount.get()); @@ -1057,6 +1087,11 @@ public static final class BlockOutputStreamBuilder { /** is Client side Encryption enabled? */ private boolean isCSEEnabled; + /** + * Put object options. + */ + private PutObjectOptions putOptions; + private BlockOutputStreamBuilder() { } @@ -1070,6 +1105,7 @@ public void validate() { requireNonNull(statistics, "null statistics"); requireNonNull(writeOperations, "null writeOperationHelper"); requireNonNull(putTracker, "null putTracker"); + requireNonNull(putOptions, "null putOptions"); Preconditions.checkArgument(blockSize >= Constants.MULTIPART_MIN_SIZE, "Block size is too small: %s", blockSize); } @@ -1182,5 +1218,16 @@ public BlockOutputStreamBuilder withCSEEnabled(boolean value) { isCSEEnabled = value; return this; } + + /** + * Set builder value. + * @param value new value + * @return the builder + */ + public BlockOutputStreamBuilder withPutOptions( + final PutObjectOptions value) { + putOptions = value; + return this; + } } } diff --git a/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/S3AFileSystem.java b/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/S3AFileSystem.java index 15e240f901865..40671e0d334cc 100644 --- a/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/S3AFileSystem.java +++ b/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/S3AFileSystem.java @@ -22,6 +22,7 @@ import java.io.FileNotFoundException; import java.io.IOException; import java.io.InterruptedIOException; +import java.io.UncheckedIOException; import java.net.URI; import java.nio.file.AccessDeniedException; import java.text.DateFormat; @@ -32,6 +33,7 @@ import java.util.Date; import java.util.EnumSet; import java.util.List; +import java.util.Locale; import java.util.Map; import java.util.Optional; import java.util.Set; @@ -66,6 +68,7 @@ import com.amazonaws.services.s3.model.PutObjectRequest; import com.amazonaws.services.s3.model.PutObjectResult; import com.amazonaws.services.s3.model.S3Object; +import com.amazonaws.services.s3.model.StorageClass; import com.amazonaws.services.s3.model.UploadPartRequest; import com.amazonaws.services.s3.model.UploadPartResult; import com.amazonaws.services.s3.transfer.Copy; @@ -89,6 +92,7 @@ import org.apache.hadoop.fs.CreateFlag; import org.apache.hadoop.fs.FSDataInputStream; import org.apache.hadoop.fs.FSDataOutputStream; +import org.apache.hadoop.fs.FSDataOutputStreamBuilder; import org.apache.hadoop.fs.Globber; import org.apache.hadoop.fs.Options; import org.apache.hadoop.fs.impl.OpenFileParameters; @@ -102,6 +106,7 @@ import org.apache.hadoop.fs.s3a.impl.ContextAccessors; import org.apache.hadoop.fs.s3a.impl.CopyFromLocalOperation; import org.apache.hadoop.fs.s3a.impl.CopyOutcome; +import org.apache.hadoop.fs.s3a.impl.CreateFileBuilder; import org.apache.hadoop.fs.s3a.impl.DeleteOperation; import org.apache.hadoop.fs.s3a.impl.DirectoryPolicy; import org.apache.hadoop.fs.s3a.impl.DirectoryPolicyImpl; @@ -112,6 +117,7 @@ import org.apache.hadoop.fs.s3a.impl.MkdirOperation; import org.apache.hadoop.fs.s3a.impl.OpenFileSupport; import org.apache.hadoop.fs.s3a.impl.OperationCallbacks; +import org.apache.hadoop.fs.s3a.impl.PutObjectOptions; import org.apache.hadoop.fs.s3a.impl.RenameOperation; import org.apache.hadoop.fs.s3a.impl.RequestFactoryImpl; import org.apache.hadoop.fs.s3a.impl.S3AMultipartUploaderBuilder; @@ -205,7 +211,8 @@ import static org.apache.hadoop.fs.s3a.commit.CommitConstants.FS_S3A_COMMITTER_ABORT_PENDING_UPLOADS; import static org.apache.hadoop.fs.s3a.commit.CommitConstants.FS_S3A_COMMITTER_STAGING_ABORT_PENDING_UPLOADS; import static org.apache.hadoop.fs.s3a.impl.CallableSupplier.submit; -import static org.apache.hadoop.fs.s3a.impl.CallableSupplier.waitForCompletionIgnoringExceptions; +import static org.apache.hadoop.fs.s3a.impl.CreateFileBuilder.OPTIONS_CREATE_FILE_NO_OVERWRITE; +import static org.apache.hadoop.fs.s3a.impl.CreateFileBuilder.OPTIONS_CREATE_FILE_OVERWRITE; import static org.apache.hadoop.fs.s3a.impl.ErrorTranslation.isObjectNotFound; import static org.apache.hadoop.fs.s3a.impl.ErrorTranslation.isUnknownBucket; import static org.apache.hadoop.fs.s3a.impl.InternalConstants.AP_INACCESSIBLE; @@ -306,6 +313,10 @@ public class S3AFileSystem extends FileSystem implements StreamCapabilities, * {@code openFile()}. */ private S3AInputPolicy inputPolicy; + /** Vectored IO context. */ + private VectoredIOContext vectoredIOContext; + + private long readAhead; private ChangeDetectionPolicy changeDetectionPolicy; private final AtomicBoolean closed = new AtomicBoolean(false); private volatile boolean isClosed = false; @@ -577,6 +588,7 @@ public void initialize(URI name, Configuration originalConf) longBytesOption(conf, ASYNC_DRAIN_THRESHOLD, DEFAULT_ASYNC_DRAIN_THRESHOLD, 0), inputPolicy); + vectoredIOContext = populateVectoredIOContext(conf); } catch (AmazonClientException e) { // amazon client exception: stop all services then throw the translation cleanupWithLogger(LOG, span); @@ -590,6 +602,23 @@ public void initialize(URI name, Configuration originalConf) } } + /** + * Populates the configurations related to vectored IO operation + * in the context which has to passed down to input streams. + * @param conf configuration object. + * @return VectoredIOContext. + */ + private VectoredIOContext populateVectoredIOContext(Configuration conf) { + final int minSeekVectored = (int) longBytesOption(conf, AWS_S3_VECTOR_READS_MIN_SEEK_SIZE, + DEFAULT_AWS_S3_VECTOR_READS_MIN_SEEK_SIZE, 0); + final int maxReadSizeVectored = (int) longBytesOption(conf, AWS_S3_VECTOR_READS_MAX_MERGED_READ_SIZE, + DEFAULT_AWS_S3_VECTOR_READS_MAX_MERGED_READ_SIZE, 0); + return new VectoredIOContext() + .setMinSeekForVectoredReads(minSeekVectored) + .setMaxReadSizeForVectoredReads(maxReadSizeVectored) + .build(); + } + /** * Set the client side encryption gauge to 0 or 1, indicating if CSE is * enabled through the gauge or not. @@ -963,6 +992,18 @@ protected RequestFactory createRequestFactory() { // Any encoding type String contentEncoding = getConf().getTrimmed(CONTENT_ENCODING, null); + String storageClassConf = getConf() + .getTrimmed(STORAGE_CLASS, "") + .toUpperCase(Locale.US); + StorageClass storageClass; + try { + storageClass = StorageClass.fromValue(storageClassConf); + } catch (IllegalArgumentException e) { + LOG.warn("Unknown storage class property {}: {}; falling back to default storage class", + STORAGE_CLASS, storageClassConf); + storageClass = null; + } + return RequestFactoryImpl.builder() .withBucket(requireNonNull(bucket)) .withCannedACL(getCannedACL()) @@ -970,6 +1011,7 @@ protected RequestFactory createRequestFactory() { .withMultipartPartCountLimit(partCountLimit) .withRequestPreparer(getAuditManager()::requestCreated) .withContentEncoding(contentEncoding) + .withStorageClass(storageClass) .build(); } @@ -1443,11 +1485,12 @@ private FSDataInputStream executeOpen( fileInformation.applyOptions(readContext); LOG.debug("Opening '{}'", readContext); return new FSDataInputStream( - new S3AInputStream( - readContext.build(), - createObjectAttributes(path, fileStatus), - createInputStreamCallbacks(auditSpan), - inputStreamStats)); + new S3AInputStream( + readContext.build(), + createObjectAttributes(path, fileStatus), + createInputStreamCallbacks(auditSpan), + inputStreamStats, + unboundedThreadPool)); } /** @@ -1531,7 +1574,8 @@ protected S3AReadOpContext createReadContext( invoker, statistics, statisticsContext, - fileStatus) + fileStatus, + vectoredIOContext) .withAuditSpan(auditSpan); openFileHelper.applyDefaultOptions(roc); return roc.build(); @@ -1599,10 +1643,15 @@ public FSDataOutputStream create(Path f, FsPermission permission, boolean overwrite, int bufferSize, short replication, long blockSize, Progressable progress) throws IOException { final Path path = qualify(f); + // the span will be picked up inside the output stream return trackDurationAndSpan(INVOCATION_CREATE, path, () -> - innerCreateFile(path, permission, overwrite, bufferSize, replication, - blockSize, progress)); + innerCreateFile(path, + progress, + getActiveAuditSpan(), + overwrite + ? OPTIONS_CREATE_FILE_OVERWRITE + : OPTIONS_CREATE_FILE_NO_OVERWRITE)); } /** @@ -1610,58 +1659,68 @@ public FSDataOutputStream create(Path f, FsPermission permission, * reporting; in the active span. * Retry policy: retrying, translated on the getFileStatus() probe. * No data is uploaded to S3 in this call, so no retry issues related to that. + * The "performance" flag disables safety checks for the path being a file, + * parent directory existing, and doesn't attempt to delete + * dir markers, irrespective of FS settings. + * If true, this method call does no IO at all. * @param path the file name to open - * @param permission the permission to set. - * @param overwrite if a file with this name already exists, then if true, - * the file will be overwritten, and if false an error will be thrown. - * @param bufferSize the size of the buffer to be used. - * @param replication required block replication for the file. - * @param blockSize the requested block size. * @param progress the progress reporter. + * @param auditSpan audit span + * @param options options for the file * @throws IOException in the event of IO related errors. - * @see #setPermission(Path, FsPermission) */ @SuppressWarnings("IOResourceOpenedButNotSafelyClosed") @Retries.RetryTranslated - private FSDataOutputStream innerCreateFile(Path path, - FsPermission permission, - boolean overwrite, - int bufferSize, - short replication, - long blockSize, - Progressable progress) throws IOException { + private FSDataOutputStream innerCreateFile( + final Path path, + final Progressable progress, + final AuditSpan auditSpan, + final CreateFileBuilder.CreateFileOptions options) throws IOException { + auditSpan.activate(); String key = pathToKey(path); - FileStatus status = null; - try { - // get the status or throw an FNFE. - // when overwriting, there is no need to look for any existing file, - // and attempting to do so can poison the load balancers with 404 - // entries. - status = innerGetFileStatus(path, false, - overwrite - ? StatusProbeEnum.DIRECTORIES - : StatusProbeEnum.ALL); - - // if the thread reaches here, there is something at the path - if (status.isDirectory()) { - // path references a directory: automatic error - throw new FileAlreadyExistsException(path + " is a directory"); - } - if (!overwrite) { - // path references a file and overwrite is disabled - throw new FileAlreadyExistsException(path + " already exists"); + EnumSet flags = options.getFlags(); + boolean overwrite = flags.contains(CreateFlag.OVERWRITE); + boolean performance = options.isPerformance(); + boolean skipProbes = performance || isUnderMagicCommitPath(path); + if (skipProbes) { + LOG.debug("Skipping existence/overwrite checks"); + } else { + try { + // get the status or throw an FNFE. + // when overwriting, there is no need to look for any existing file, + // just a directory (for safety) + FileStatus status = innerGetFileStatus(path, false, + overwrite + ? StatusProbeEnum.DIRECTORIES + : StatusProbeEnum.ALL); + + // if the thread reaches here, there is something at the path + if (status.isDirectory()) { + // path references a directory: automatic error + throw new FileAlreadyExistsException(path + " is a directory"); + } + if (!overwrite) { + // path references a file and overwrite is disabled + throw new FileAlreadyExistsException(path + " already exists"); + } + LOG.debug("Overwriting file {}", path); + } catch (FileNotFoundException e) { + // this means there is nothing at the path; all good. } - LOG.debug("Overwriting file {}", path); - } catch (FileNotFoundException e) { - // this means the file is not found - } instrumentation.fileCreated(); - PutTracker putTracker = - committerIntegration.createTracker(path, key); - String destKey = putTracker.getDestKey(); final BlockOutputStreamStatistics outputStreamStatistics = statisticsContext.newOutputStreamStatistics(); + PutTracker putTracker = + committerIntegration.createTracker(path, key, outputStreamStatistics); + String destKey = putTracker.getDestKey(); + + // put options are derived from the path and the + // option builder. + boolean keep = performance || keepDirectoryMarkers(path); + final PutObjectOptions putOptions = + new PutObjectOptions(keep, null, options.getHeaders()); + final S3ABlockOutputStream.BlockOutputStreamBuilder builder = S3ABlockOutputStream.builder() .withKey(destKey) @@ -1671,7 +1730,7 @@ private FSDataOutputStream innerCreateFile(Path path, .withProgress(progress) .withPutTracker(putTracker) .withWriteOperations( - createWriteOperationHelper(getActiveAuditSpan())) + createWriteOperationHelper(auditSpan)) .withExecutorService( new SemaphoredDelegatingExecutor( boundedThreadPool, @@ -1682,12 +1741,12 @@ private FSDataOutputStream innerCreateFile(Path path, getConf().getBoolean( DOWNGRADE_SYNCABLE_EXCEPTIONS, DOWNGRADE_SYNCABLE_EXCEPTIONS_DEFAULT)) - .withCSEEnabled(isCSEEnabled); + .withCSEEnabled(isCSEEnabled) + .withPutOptions(putOptions); return new FSDataOutputStream( new S3ABlockOutputStream(builder), null); } - /** * Create a Write Operation Helper with the current active span. * All operations made through this helper will activate the @@ -1720,10 +1779,65 @@ public WriteOperationHelper createWriteOperationHelper(AuditSpan auditSpan) { auditSpan); } + /** + * Create instance of an FSDataOutputStreamBuilder for + * creating a file at the given path. + * @param path path to create + * @return a builder. + * @throws UncheckedIOException for problems creating the audit span + */ + @Override + @AuditEntryPoint + public FSDataOutputStreamBuilder createFile(final Path path) { + try { + final Path qualified = qualify(path); + final AuditSpan span = entryPoint(INVOCATION_CREATE_FILE, + pathToKey(qualified), + null); + return new CreateFileBuilder(this, + qualified, + new CreateFileBuilderCallbacksImpl(INVOCATION_CREATE_FILE, span)) + .create() + .overwrite(true); + } catch (IOException e) { + // catch any IOEs raised in span creation and convert to + // an UncheckedIOException + throw new UncheckedIOException(e); + } + } + + /** + * Callback for create file operations. + */ + private final class CreateFileBuilderCallbacksImpl implements + CreateFileBuilder.CreateFileBuilderCallbacks { + + private final Statistic statistic; + /** span for operations. */ + private final AuditSpan span; + + private CreateFileBuilderCallbacksImpl( + final Statistic statistic, + final AuditSpan span) { + this.statistic = statistic; + this.span = span; + } + + @Override + public FSDataOutputStream createFileFromBuilder( + final Path path, + final Progressable progress, + final CreateFileBuilder.CreateFileOptions options) throws IOException { + // the span will be picked up inside the output stream + return trackDuration(getDurationTrackerFactory(), statistic.getSymbol(), () -> + innerCreateFile(path, progress, span, options)); + } + } + /** * {@inheritDoc} - * @throws FileNotFoundException if the parent directory is not present -or - * is not a directory. + * The S3A implementations downgrades to the recursive creation, to avoid + * any race conditions with parent entries "disappearing". */ @Override @AuditEntryPoint @@ -1735,30 +1849,23 @@ public FSDataOutputStream createNonRecursive(Path p, long blockSize, Progressable progress) throws IOException { final Path path = makeQualified(p); - // this span is passed into the stream. - try (AuditSpan span = entryPoint(INVOCATION_CREATE_NON_RECURSIVE, path)) { - Path parent = path.getParent(); - // expect this to raise an exception if there is no parent dir - if (parent != null && !parent.isRoot()) { - S3AFileStatus status; - try { - // optimize for the directory existing: Call list first - status = innerGetFileStatus(parent, false, - StatusProbeEnum.DIRECTORIES); - } catch (FileNotFoundException e) { - // no dir, fall back to looking for a file - // (failure condition if true) - status = innerGetFileStatus(parent, false, - StatusProbeEnum.HEAD_ONLY); - } - if (!status.isDirectory()) { - throw new FileAlreadyExistsException("Not a directory: " + parent); - } - } - return innerCreateFile(path, permission, - flags.contains(CreateFlag.OVERWRITE), bufferSize, - replication, blockSize, progress); + + // span is created and passed in to the callbacks. + final AuditSpan span = entryPoint(INVOCATION_CREATE_NON_RECURSIVE, + pathToKey(path), + null); + // uses the CreateFileBuilder, filling it in with the relevant arguments. + final CreateFileBuilder builder = new CreateFileBuilder(this, + path, + new CreateFileBuilderCallbacksImpl(INVOCATION_CREATE_NON_RECURSIVE, span)) + .create() + .withFlags(flags) + .blockSize(blockSize) + .bufferSize(bufferSize); + if (progress != null) { + builder.progress(progress); } + return builder.build(); } /** @@ -2656,7 +2763,7 @@ private DeleteObjectsResult deleteObjects(DeleteObjectsRequest deleteRequest) */ public PutObjectRequest newPutObjectRequest(String key, ObjectMetadata metadata, File srcfile) { - return requestFactory.newPutObjectRequest(key, metadata, srcfile); + return requestFactory.newPutObjectRequest(key, metadata, null, srcfile); } /** @@ -2706,12 +2813,14 @@ public UploadInfo putObject(PutObjectRequest putObjectRequest) { * Auditing: must be inside an audit span. * Important: this call will close any input stream in the request. * @param putObjectRequest the request + * @param putOptions put object options * @return the upload initiated * @throws AmazonClientException on problems */ @VisibleForTesting - @Retries.OnceRaw("For PUT; post-PUT actions are RetryTranslated") - PutObjectResult putObjectDirect(PutObjectRequest putObjectRequest) + @Retries.OnceRaw("For PUT; post-PUT actions are RetryExceptionsSwallowed") + PutObjectResult putObjectDirect(PutObjectRequest putObjectRequest, + PutObjectOptions putOptions) throws AmazonClientException { long len = getPutRequestLength(putObjectRequest); LOG.debug("PUT {} bytes to {}", len, putObjectRequest.getKey()); @@ -2722,9 +2831,10 @@ PutObjectResult putObjectDirect(PutObjectRequest putObjectRequest) OBJECT_PUT_REQUESTS.getSymbol(), () -> s3.putObject(putObjectRequest)); incrementPutCompletedStatistics(true, len); - // update metadata + // apply any post-write actions. finishedWrite(putObjectRequest.getKey(), len, - result.getETag(), result.getVersionId()); + result.getETag(), result.getVersionId(), + putOptions); return result; } catch (SdkBaseException e) { incrementPutCompletedStatistics(false, len); @@ -2996,7 +3106,7 @@ private void createFakeDirectoryIfNecessary(Path f) // is mostly harmless to create a new one. if (!key.isEmpty() && !s3Exists(f, StatusProbeEnum.DIRECTORIES)) { LOG.debug("Creating new fake directory at {}", f); - createFakeDirectory(key); + createFakeDirectory(key, putOptionsForPath(f)); } } @@ -3011,7 +3121,7 @@ private void createFakeDirectoryIfNecessary(Path f) protected void maybeCreateFakeParentDirectory(Path path) throws IOException, AmazonClientException { Path parent = path.getParent(); - if (parent != null && !parent.isRoot()) { + if (parent != null && !parent.isRoot() && !isUnderMagicCommitPath(parent)) { createFakeDirectoryIfNecessary(parent); } } @@ -3182,6 +3292,11 @@ public UserGroupInformation getOwner() { * Make the given path and all non-existent parents into * directories. Has the semantics of Unix {@code 'mkdir -p'}. * Existence of the directory hierarchy is not an error. + * Parent elements are scanned to see if any are a file, + * except under __magic paths. + * There the FS assumes that the destination directory creation + * did that scan and that paths in job/task attempts are all + * "well formed" * @param p path to create * @param permission to apply to path * @return true if a directory was created or already existed @@ -3199,7 +3314,8 @@ public boolean mkdirs(Path p, FsPermission permission) throws IOException, new MkdirOperation( createStoreContext(), path, - createMkdirOperationCallbacks())); + createMkdirOperationCallbacks(), + isMagicCommitPath(path))); } /** @@ -3225,9 +3341,13 @@ public S3AFileStatus probePathStatus(final Path path, } @Override - public void createFakeDirectory(final String key) + public void createFakeDirectory(final Path dir, final boolean keepMarkers) throws IOException { - S3AFileSystem.this.createEmptyObject(key); + S3AFileSystem.this.createFakeDirectory( + pathToKey(dir), + keepMarkers + ? PutObjectOptions.keepingDirs() + : putOptionsForPath(dir)); } } @@ -3593,7 +3713,7 @@ public void copyLocalFileFromTo(File file, Path from, Path to) throws IOExceptio S3AFileSystem.this.invoker.retry( "putObject(" + "" + ")", to.toString(), true, - () -> executePut(putObjectRequest, progress)); + () -> executePut(putObjectRequest, progress, putOptionsForPath(to))); return null; }); @@ -3612,7 +3732,7 @@ public boolean createEmptyDir(Path path, StoreContext storeContext) new MkdirOperation( storeContext, path, - createMkdirOperationCallbacks())); + createMkdirOperationCallbacks(), false)); } } @@ -3622,14 +3742,18 @@ public boolean createEmptyDir(Path path, StoreContext storeContext) * aborted before an {@code InterruptedIOException} is thrown. * @param putObjectRequest request * @param progress optional progress callback + * @param putOptions put object options * @return the upload result * @throws InterruptedIOException if the blocking was interrupted. */ - @Retries.OnceRaw("For PUT; post-PUT actions are RetryTranslated") - UploadResult executePut(PutObjectRequest putObjectRequest, - Progressable progress) + @Retries.OnceRaw("For PUT; post-PUT actions are RetrySwallowed") + UploadResult executePut( + final PutObjectRequest putObjectRequest, + final Progressable progress, + final PutObjectOptions putOptions) throws InterruptedIOException { String key = putObjectRequest.getKey(); + long len = getPutRequestLength(putObjectRequest); UploadInfo info = putObject(putObjectRequest); Upload upload = info.getUpload(); ProgressableProgressListener listener = new ProgressableProgressListener( @@ -3637,9 +3761,10 @@ UploadResult executePut(PutObjectRequest putObjectRequest, upload.addProgressListener(listener); UploadResult result = waitForUploadCompletion(key, info); listener.uploadCompleted(); + // post-write actions - finishedWrite(key, info.getLength(), - result.getETag(), result.getVersionId()); + finishedWrite(key, len, + result.getETag(), result.getVersionId(), putOptions); return result; } @@ -3648,7 +3773,9 @@ UploadResult executePut(PutObjectRequest putObjectRequest, * If the waiting for completion is interrupted, the upload will be * aborted before an {@code InterruptedIOException} is thrown. * If the upload (or its result collection) failed, this is where - * the failure is raised as an AWS exception + * the failure is raised as an AWS exception. + * Calls {@link #incrementPutCompletedStatistics(boolean, long)} + * to update the statistics. * @param key destination key * @param uploadInfo upload to wait for * @return the upload result @@ -3970,63 +4097,64 @@ InitiateMultipartUploadResult initiateMultipartUpload( /** * Perform post-write actions. - *

    + *

    * This operation MUST be called after any PUT/multipart PUT completes * successfully. - *

    - * The actions include: - *
      - *
    1. - * Calling - * {@link #deleteUnnecessaryFakeDirectories(Path)} - * if directory markers are not being retained. - *
    2. - *
    3. - * Updating any metadata store with details on the newly created - * object. - *
    4. - *
    + *

    + * The actions include calling + * {@link #deleteUnnecessaryFakeDirectories(Path)} + * if directory markers are not being retained. * @param key key written to * @param length total length of file written * @param eTag eTag of the written object * @param versionId S3 object versionId of the written object + * @param putOptions put object options */ @InterfaceAudience.Private - @Retries.RetryTranslated("Except if failOnMetadataWriteError=false, in which" - + " case RetryExceptionsSwallowed") - void finishedWrite(String key, long length, String eTag, String versionId) { + @Retries.RetryExceptionsSwallowed + void finishedWrite( + String key, + long length, + String eTag, + String versionId, + PutObjectOptions putOptions) { LOG.debug("Finished write to {}, len {}. etag {}, version {}", key, length, eTag, versionId); - Path p = keyToQualifiedPath(key); Preconditions.checkArgument(length >= 0, "content length is negative"); - // kick off an async delete - CompletableFuture deletion; - if (!keepDirectoryMarkers(p)) { - deletion = submit( - unboundedThreadPool, getActiveAuditSpan(), - () -> { - deleteUnnecessaryFakeDirectories( - p.getParent() - ); - return null; - }); - } else { - deletion = null; + if (!putOptions.isKeepMarkers()) { + Path p = keyToQualifiedPath(key); + deleteUnnecessaryFakeDirectories(p.getParent()); } - - // catch up with any delete operation. - waitForCompletionIgnoringExceptions(deletion); } /** * Should we keep directory markers under the path being created * by mkdir/file creation/rename? + * This is done if marker retention is enabled for the path, + * or it is under a magic path where we are saving IOPs + * knowing that all committers are on the same code version and + * therefore marker aware. * @param path path to probe * @return true if the markers MAY be retained, * false if they MUST be deleted */ private boolean keepDirectoryMarkers(Path path) { - return directoryPolicy.keepDirectoryMarkers(path); + return directoryPolicy.keepDirectoryMarkers(path) + || isUnderMagicCommitPath(path); + } + + /** + * Should we keep directory markers under the path being created + * by mkdir/file creation/rename? + * See {@link #keepDirectoryMarkers(Path)} for the policy. + * + * @param path path to probe + * @return the options to use with the put request + */ + private PutObjectOptions putOptionsForPath(Path path) { + return keepDirectoryMarkers(path) + ? PutObjectOptions.keepingDirs() + : PutObjectOptions.deletingDirs(); } /** @@ -4063,27 +4191,32 @@ private void deleteUnnecessaryFakeDirectories(Path path) { * Create a fake directory, always ending in "/". * Retry policy: retrying; translated. * @param objectName name of directory object. + * @param putOptions put object options * @throws IOException IO failure */ @Retries.RetryTranslated - private void createFakeDirectory(final String objectName) + private void createFakeDirectory(final String objectName, + final PutObjectOptions putOptions) throws IOException { - createEmptyObject(objectName); + createEmptyObject(objectName, putOptions); } /** * Used to create an empty file that represents an empty directory. + * The policy for deleting parent dirs depends on the path, dir + * status and the putOptions value. * Retry policy: retrying; translated. * @param objectName object to create + * @param putOptions put object options * @throws IOException IO failure */ @Retries.RetryTranslated - private void createEmptyObject(final String objectName) + private void createEmptyObject(final String objectName, PutObjectOptions putOptions) throws IOException { invoker.retry("PUT 0-byte object ", objectName, true, () -> putObjectDirect(getRequestFactory() - .newDirectoryMarkerRequest(objectName))); + .newDirectoryMarkerRequest(objectName), putOptions)); incrementPutProgressStatistics(objectName, 0); instrumentation.directoryCreated(); } @@ -4192,14 +4325,26 @@ public boolean isMagicCommitEnabled() { /** * Predicate: is a path a magic commit path? - * True if magic commit is enabled and the path qualifies as special. + * True if magic commit is enabled and the path qualifies as special, + * and is not a a .pending or .pendingset file, * @param path path to examine - * @return true if the path is or is under a magic directory + * @return true if writing a file to the path triggers a "magic" write. */ public boolean isMagicCommitPath(Path path) { return committerIntegration.isMagicCommitPath(path); } + /** + * Predicate: is a path under a magic commit path? + * True if magic commit is enabled and the path is under __magic, + * irrespective of file type. + * @param path path to examine + * @return true if the path is in a magic dir and the FS has magic writes enabled. + */ + private boolean isUnderMagicCommitPath(Path path) { + return committerIntegration.isUnderMagicPath(path); + } + /** * Increments the statistic {@link Statistic#INVOCATION_GLOB_STATUS}. * Override superclass so as to disable symlink resolution as symlinks @@ -4751,9 +4896,19 @@ public boolean hasPathCapability(final Path path, final String capability) case STORE_CAPABILITY_DIRECTORY_MARKER_POLICY_KEEP: case STORE_CAPABILITY_DIRECTORY_MARKER_POLICY_DELETE: case STORE_CAPABILITY_DIRECTORY_MARKER_POLICY_AUTHORITATIVE: + return getDirectoryMarkerPolicy().hasPathCapability(path, cap); + + // keep for a magic path or if the policy retains it case STORE_CAPABILITY_DIRECTORY_MARKER_ACTION_KEEP: + return keepDirectoryMarkers(path); + // delete is the opposite of keep case STORE_CAPABILITY_DIRECTORY_MARKER_ACTION_DELETE: - return getDirectoryMarkerPolicy().hasPathCapability(path, cap); + return !keepDirectoryMarkers(path); + + // create file options + case FS_S3A_CREATE_PERFORMANCE: + case FS_S3A_CREATE_HEADER: + return true; default: return super.hasPathCapability(p, cap); @@ -4795,9 +4950,8 @@ public AWSCredentialProviderList shareCredentials(final String purpose) { /** * This is a proof of concept of a select API. * @param source path to source data - * @param expression select expression * @param options request configuration from the builder. - * @param providedStatus any passed in status + * @param fileInformation any passed in information. * @return the stream of the results * @throws IOException IO failure */ diff --git a/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/S3AInputStream.java b/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/S3AInputStream.java index 6beeb2891eea7..3069f17289119 100644 --- a/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/S3AInputStream.java +++ b/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/S3AInputStream.java @@ -19,38 +19,50 @@ package org.apache.hadoop.fs.s3a; import javax.annotation.Nullable; +import java.io.Closeable; +import java.io.EOFException; +import java.io.IOException; +import java.io.InterruptedIOException; +import java.net.SocketTimeoutException; +import java.nio.ByteBuffer; +import java.util.List; +import java.util.concurrent.CompletableFuture; +import java.util.concurrent.ThreadPoolExecutor; +import java.util.concurrent.atomic.AtomicBoolean; +import java.util.function.IntFunction; import com.amazonaws.services.s3.model.GetObjectRequest; import com.amazonaws.services.s3.model.S3Object; import com.amazonaws.services.s3.model.S3ObjectInputStream; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; -import org.apache.hadoop.classification.VisibleForTesting; -import org.apache.hadoop.util.Preconditions; import org.apache.hadoop.classification.InterfaceAudience; import org.apache.hadoop.classification.InterfaceStability; +import org.apache.hadoop.classification.VisibleForTesting; import org.apache.hadoop.fs.CanSetReadahead; import org.apache.hadoop.fs.CanUnbuffer; import org.apache.hadoop.fs.FSExceptionMessages; -import org.apache.hadoop.fs.s3a.statistics.S3AInputStreamStatistics; +import org.apache.hadoop.fs.FSInputStream; +import org.apache.hadoop.fs.FileRange; +import org.apache.hadoop.fs.PathIOException; +import org.apache.hadoop.fs.StreamCapabilities; +import org.apache.hadoop.fs.impl.CombinedFileRange; +import org.apache.hadoop.fs.VectoredReadUtils; import org.apache.hadoop.fs.s3a.impl.ChangeTracker; +import org.apache.hadoop.fs.s3a.statistics.S3AInputStreamStatistics; +import org.apache.hadoop.fs.statistics.DurationTracker; import org.apache.hadoop.fs.statistics.IOStatistics; import org.apache.hadoop.fs.statistics.IOStatisticsSource; -import org.apache.hadoop.fs.PathIOException; -import org.apache.hadoop.fs.StreamCapabilities; -import org.apache.hadoop.fs.FSInputStream; +import org.apache.hadoop.io.IOUtils; +import org.apache.hadoop.util.Preconditions; import org.apache.hadoop.util.functional.CallableRaisingIOE; -import org.slf4j.Logger; -import org.slf4j.LoggerFactory; - -import java.io.Closeable; -import java.io.EOFException; -import java.io.IOException; -import java.net.SocketTimeoutException; -import java.util.concurrent.CompletableFuture; - import static java.util.Objects.requireNonNull; import static org.apache.commons.lang3.StringUtils.isNotEmpty; +import static org.apache.hadoop.fs.VectoredReadUtils.isOrderedDisjoint; +import static org.apache.hadoop.fs.VectoredReadUtils.mergeSortedRanges; +import static org.apache.hadoop.fs.VectoredReadUtils.validateNonOverlappingAndReturnSortedRanges; import static org.apache.hadoop.fs.s3a.Invoker.onceTrackingDuration; import static org.apache.hadoop.fs.statistics.impl.IOStatisticsBinding.invokeTrackingDuration; import static org.apache.hadoop.util.StringUtils.toLowerCase; @@ -88,6 +100,20 @@ public class S3AInputStream extends FSInputStream implements CanSetReadahead, * size of a buffer to create when draining the stream. */ private static final int DRAIN_BUFFER_SIZE = 16384; + /** + * This is the maximum temporary buffer size we use while + * populating the data in direct byte buffers during a vectored IO + * operation. This is to ensure that when a big range of data is + * requested in direct byte buffer doesn't leads to OOM errors. + */ + private static final int TMP_BUFFER_MAX_SIZE = 64 * 1024; + + /** + * Atomic boolean variable to stop all ongoing vectored read operation + * for this input stream. This will be set to true when the stream is + * closed or unbuffer is called. + */ + private final AtomicBoolean stopVectoredIOOperations = new AtomicBoolean(false); /** * This is the public position; the one set in {@link #seek(long)} @@ -111,6 +137,11 @@ public class S3AInputStream extends FSInputStream implements CanSetReadahead, private S3ObjectInputStream wrappedStream; private final S3AReadOpContext context; private final InputStreamCallbacks client; + + /** + * Thread pool used for vectored IO operation. + */ + private final ThreadPoolExecutor unboundedThreadPool; private final String bucket; private final String key; private final String pathStr; @@ -122,6 +153,9 @@ public class S3AInputStream extends FSInputStream implements CanSetReadahead, private S3AInputPolicy inputPolicy; private long readahead = Constants.DEFAULT_READAHEAD_RANGE; + /** Vectored IO context. */ + private final VectoredIOContext vectoredIOContext; + /** * This is the actual position within the object, used by * lazy seek to decide whether to seek on the next read or not. @@ -160,12 +194,14 @@ public class S3AInputStream extends FSInputStream implements CanSetReadahead, * @param ctx operation context * @param s3Attributes object attributes * @param client S3 client to use - * @param streamStatistics statistics for this stream + * @param streamStatistics stream io stats. + * @param unboundedThreadPool thread pool to use. */ public S3AInputStream(S3AReadOpContext ctx, - S3ObjectAttributes s3Attributes, - InputStreamCallbacks client, - S3AInputStreamStatistics streamStatistics) { + S3ObjectAttributes s3Attributes, + InputStreamCallbacks client, + S3AInputStreamStatistics streamStatistics, + ThreadPoolExecutor unboundedThreadPool) { Preconditions.checkArgument(isNotEmpty(s3Attributes.getBucket()), "No Bucket"); Preconditions.checkArgument(isNotEmpty(s3Attributes.getKey()), "No Key"); @@ -187,6 +223,8 @@ public S3AInputStream(S3AReadOpContext ctx, setInputPolicy(ctx.getInputPolicy()); setReadahead(ctx.getReadahead()); this.asyncDrainThreshold = ctx.getAsyncDrainThreshold(); + this.unboundedThreadPool = unboundedThreadPool; + this.vectoredIOContext = context.getVectoredIOContext(); } /** @@ -559,6 +597,7 @@ public synchronized void close() throws IOException { if (!closed) { closed = true; try { + stopVectoredIOOperations.set(true); // close or abort the stream; blocking awaitFuture(closeStream("close() operation", false, true)); LOG.debug("Statistics of stream {}\n{}", key, streamStatistics); @@ -834,6 +873,7 @@ public String toString() { sb.append(" remainingInCurrentRequest=") .append(remainingInCurrentRequest()); sb.append(" ").append(changeTracker); + sb.append(" ").append(vectoredIOContext); sb.append('\n').append(s); sb.append('}'); return sb.toString(); @@ -880,6 +920,313 @@ public void readFully(long position, byte[] buffer, int offset, int length) } } + /** + * {@inheritDoc}. + */ + @Override + public int minSeekForVectorReads() { + return vectoredIOContext.getMinSeekForVectorReads(); + } + + /** + * {@inheritDoc}. + */ + @Override + public int maxReadSizeForVectorReads() { + return vectoredIOContext.getMaxReadSizeForVectorReads(); + } + + /** + * {@inheritDoc} + * Vectored read implementation for S3AInputStream. + * @param ranges the byte ranges to read. + * @param allocate the function to allocate ByteBuffer. + * @throws IOException IOE if any. + */ + @Override + public void readVectored(List ranges, + IntFunction allocate) throws IOException { + + LOG.debug("Starting vectored read on path {} for ranges {} ", pathStr, ranges); + checkNotClosed(); + if (stopVectoredIOOperations.getAndSet(false)) { + LOG.debug("Reinstating vectored read operation for path {} ", pathStr); + } + List sortedRanges = validateNonOverlappingAndReturnSortedRanges(ranges); + for (FileRange range : ranges) { + validateRangeRequest(range); + CompletableFuture result = new CompletableFuture<>(); + range.setData(result); + } + + if (isOrderedDisjoint(sortedRanges, 1, minSeekForVectorReads())) { + LOG.debug("Not merging the ranges as they are disjoint"); + for (FileRange range: sortedRanges) { + ByteBuffer buffer = allocate.apply(range.getLength()); + unboundedThreadPool.submit(() -> readSingleRange(range, buffer)); + } + } else { + LOG.debug("Trying to merge the ranges as they are not disjoint"); + List combinedFileRanges = mergeSortedRanges(sortedRanges, + 1, minSeekForVectorReads(), + maxReadSizeForVectorReads()); + LOG.debug("Number of original ranges size {} , Number of combined ranges {} ", + ranges.size(), combinedFileRanges.size()); + for (CombinedFileRange combinedFileRange: combinedFileRanges) { + unboundedThreadPool.submit( + () -> readCombinedRangeAndUpdateChildren(combinedFileRange, allocate)); + } + } + LOG.debug("Finished submitting vectored read to threadpool" + + " on path {} for ranges {} ", pathStr, ranges); + } + + /** + * Read the data from S3 for the bigger combined file range and update all the + * underlying ranges. + * @param combinedFileRange big combined file range. + * @param allocate method to create byte buffers to hold result data. + */ + private void readCombinedRangeAndUpdateChildren(CombinedFileRange combinedFileRange, + IntFunction allocate) { + LOG.debug("Start reading combined range {} from path {} ", combinedFileRange, pathStr); + // This reference is must be kept till all buffers are populated as this is a + // finalizable object which closes the internal stream when gc triggers. + S3Object objectRange = null; + S3ObjectInputStream objectContent = null; + try { + checkIfVectoredIOStopped(); + final String operationName = "readCombinedFileRange"; + objectRange = getS3Object(operationName, + combinedFileRange.getOffset(), + combinedFileRange.getLength()); + objectContent = objectRange.getObjectContent(); + if (objectContent == null) { + throw new PathIOException(uri, + "Null IO stream received during " + operationName); + } + populateChildBuffers(combinedFileRange, objectContent, allocate); + } catch (Exception ex) { + LOG.debug("Exception while reading a range {} from path {} ", combinedFileRange, pathStr, ex); + for(FileRange child : combinedFileRange.getUnderlying()) { + child.getData().completeExceptionally(ex); + } + } finally { + IOUtils.cleanupWithLogger(LOG, objectRange, objectContent); + } + LOG.debug("Finished reading range {} from path {} ", combinedFileRange, pathStr); + } + + /** + * Populate underlying buffers of the child ranges. + * @param combinedFileRange big combined file range. + * @param objectContent data from s3. + * @param allocate method to allocate child byte buffers. + * @throws IOException any IOE. + */ + private void populateChildBuffers(CombinedFileRange combinedFileRange, + S3ObjectInputStream objectContent, + IntFunction allocate) throws IOException { + // If the combined file range just contains a single child + // range, we only have to fill that one child buffer else + // we drain the intermediate data between consecutive ranges + // and fill the buffers one by one. + if (combinedFileRange.getUnderlying().size() == 1) { + FileRange child = combinedFileRange.getUnderlying().get(0); + ByteBuffer buffer = allocate.apply(child.getLength()); + populateBuffer(child.getLength(), buffer, objectContent); + child.getData().complete(buffer); + } else { + FileRange prev = null; + for (FileRange child : combinedFileRange.getUnderlying()) { + if (prev != null && prev.getOffset() + prev.getLength() < child.getOffset()) { + long drainQuantity = child.getOffset() - prev.getOffset() - prev.getLength(); + drainUnnecessaryData(objectContent, drainQuantity); + } + ByteBuffer buffer = allocate.apply(child.getLength()); + populateBuffer(child.getLength(), buffer, objectContent); + child.getData().complete(buffer); + prev = child; + } + } + } + + /** + * Drain unnecessary data in between ranges. + * @param objectContent s3 data stream. + * @param drainQuantity how many bytes to drain. + * @throws IOException any IOE. + */ + private void drainUnnecessaryData(S3ObjectInputStream objectContent, long drainQuantity) + throws IOException { + int drainBytes = 0; + int readCount; + while (drainBytes < drainQuantity) { + if (drainBytes + DRAIN_BUFFER_SIZE <= drainQuantity) { + byte[] drainBuffer = new byte[DRAIN_BUFFER_SIZE]; + readCount = objectContent.read(drainBuffer); + } else { + byte[] drainBuffer = new byte[(int) (drainQuantity - drainBytes)]; + readCount = objectContent.read(drainBuffer); + } + drainBytes += readCount; + } + LOG.debug("{} bytes drained from stream ", drainBytes); + } + + /** + * Validates range parameters. + * In case of S3 we already have contentLength from the first GET request + * during an open file operation so failing fast here. + * @param range requested range. + * @throws EOFException end of file exception. + */ + private void validateRangeRequest(FileRange range) throws EOFException { + VectoredReadUtils.validateRangeRequest(range); + if(range.getOffset() + range.getLength() > contentLength) { + LOG.warn("Requested range [{}, {}) is beyond EOF for path {}", + range.getOffset(), range.getLength(), pathStr); + throw new EOFException("Requested range [" + range.getOffset() +", " + + range.getLength() + ") is beyond EOF for path " + pathStr); + } + } + + /** + * Read data from S3 for this range and populate the buffer. + * @param range range of data to read. + * @param buffer buffer to fill. + */ + private void readSingleRange(FileRange range, ByteBuffer buffer) { + LOG.debug("Start reading range {} from path {} ", range, pathStr); + S3Object objectRange = null; + S3ObjectInputStream objectContent = null; + try { + checkIfVectoredIOStopped(); + long position = range.getOffset(); + int length = range.getLength(); + final String operationName = "readRange"; + objectRange = getS3Object(operationName, position, length); + objectContent = objectRange.getObjectContent(); + if (objectContent == null) { + throw new PathIOException(uri, + "Null IO stream received during " + operationName); + } + populateBuffer(length, buffer, objectContent); + range.getData().complete(buffer); + } catch (Exception ex) { + LOG.warn("Exception while reading a range {} from path {} ", range, pathStr, ex); + range.getData().completeExceptionally(ex); + } finally { + IOUtils.cleanupWithLogger(LOG, objectRange, objectContent); + } + LOG.debug("Finished reading range {} from path {} ", range, pathStr); + } + + /** + * Populates the buffer with data from objectContent + * till length. Handles both direct and heap byte buffers. + * @param length length of data to populate. + * @param buffer buffer to fill. + * @param objectContent result retrieved from S3 store. + * @throws IOException any IOE. + */ + private void populateBuffer(int length, + ByteBuffer buffer, + S3ObjectInputStream objectContent) throws IOException { + if (buffer.isDirect()) { + int readBytes = 0; + int offset = 0; + byte[] tmp = new byte[TMP_BUFFER_MAX_SIZE]; + while (readBytes < length) { + checkIfVectoredIOStopped(); + int currentLength = readBytes + TMP_BUFFER_MAX_SIZE < length ? + TMP_BUFFER_MAX_SIZE + : length - readBytes; + readByteArray(objectContent, tmp, 0, currentLength); + buffer.put(tmp, 0, currentLength); + offset = offset + currentLength; + readBytes = readBytes + currentLength; + } + buffer.flip(); + } else { + readByteArray(objectContent, buffer.array(), 0, length); + } + } + + /** + * Read data into destination buffer from s3 object content. + * @param objectContent result from S3. + * @param dest destination buffer. + * @param offset start offset of dest buffer. + * @param length number of bytes to fill in dest. + * @throws IOException any IOE. + */ + private void readByteArray(S3ObjectInputStream objectContent, + byte[] dest, + int offset, + int length) throws IOException { + int readBytes = 0; + while (readBytes < length) { + int readBytesCurr = objectContent.read(dest, + offset + readBytes, + length - readBytes); + readBytes +=readBytesCurr; + if (readBytesCurr < 0) { + throw new EOFException(FSExceptionMessages.EOF_IN_READ_FULLY); + } + } + } + + /** + * Read data from S3 using a http request with retries. + * This also handles if file has been changed while the + * http call is getting executed. If the file has been + * changed RemoteFileChangedException is thrown. + * @param operationName name of the operation for which get object on S3 is called. + * @param position position of the object to be read from S3. + * @param length length from position of the object to be read from S3. + * @return S3Object result s3 object. + * @throws IOException exception if any. + */ + private S3Object getS3Object(String operationName, long position, + int length) throws IOException { + final GetObjectRequest request = client.newGetRequest(key) + .withRange(position, position + length - 1); + changeTracker.maybeApplyConstraint(request); + DurationTracker tracker = streamStatistics.initiateGetRequest(); + S3Object objectRange; + Invoker invoker = context.getReadInvoker(); + try { + objectRange = invoker.retry(operationName, pathStr, true, + () -> { + checkIfVectoredIOStopped(); + return client.getObject(request); + }); + + } catch (IOException ex) { + tracker.failed(); + throw ex; + } finally { + tracker.close(); + } + changeTracker.processResponse(objectRange, operationName, + position); + return objectRange; + } + + /** + * Check if vectored io operation has been stooped. This happens + * when the stream is closed or unbuffer is called. + * @throws InterruptedIOException throw InterruptedIOException such + * that all running vectored io is + * terminated thus releasing resources. + */ + private void checkIfVectoredIOStopped() throws InterruptedIOException { + if (stopVectoredIOOperations.get()) { + throw new InterruptedIOException("Stream closed or unbuffer is called"); + } + } + /** * Access the input stream statistics. * This is for internal testing and may be removed without warning. @@ -965,10 +1312,15 @@ public static long validateReadahead(@Nullable Long readahead) { /** * Closes the underlying S3 stream, and merges the {@link #streamStatistics} * instance associated with the stream. + * Also sets the {@code stopVectoredIOOperations} flag to true such that + * active vectored read operations are terminated. However termination of + * old vectored reads are not guaranteed if a new vectored read operation + * is initiated after unbuffer is called. */ @Override public synchronized void unbuffer() { try { + stopVectoredIOOperations.set(true); closeStream("unbuffer()", false, false); } finally { streamStatistics.unbuffered(); @@ -981,6 +1333,7 @@ public boolean hasCapability(String capability) { case StreamCapabilities.IOSTATISTICS: case StreamCapabilities.READAHEAD: case StreamCapabilities.UNBUFFER: + case StreamCapabilities.VECTOREDIO: return true; default: return false; diff --git a/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/S3AInstrumentation.java b/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/S3AInstrumentation.java index eec636672010a..67734b7502976 100644 --- a/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/S3AInstrumentation.java +++ b/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/S3AInstrumentation.java @@ -1308,8 +1308,18 @@ private void mergeOutputStreamStatistics( incrementCounter(STREAM_WRITE_EXCEPTIONS, source.lookupCounterValue( StreamStatisticNames.STREAM_WRITE_EXCEPTIONS)); + // merge in all the IOStatistics - this.getIOStatistics().aggregate(source.getIOStatistics()); + final IOStatisticsStore sourceIOStatistics = source.getIOStatistics(); + this.getIOStatistics().aggregate(sourceIOStatistics); + + // propagate any extra values into the FS-level stats. + incrementMutableCounter(OBJECT_PUT_REQUESTS.getSymbol(), + sourceIOStatistics.counters().get(OBJECT_PUT_REQUESTS.getSymbol())); + incrementMutableCounter( + COMMITTER_MAGIC_MARKER_PUT.getSymbol(), + sourceIOStatistics.counters().get(COMMITTER_MAGIC_MARKER_PUT.getSymbol())); + } /** @@ -1366,9 +1376,12 @@ private OutputStreamStatistics( STREAM_WRITE_BLOCK_UPLOADS_BYTES_PENDING.getSymbol()) .withDurationTracking( ACTION_EXECUTOR_ACQUIRED, + COMMITTER_MAGIC_MARKER_PUT.getSymbol(), INVOCATION_ABORT.getSymbol(), + MULTIPART_UPLOAD_COMPLETED.getSymbol(), OBJECT_MULTIPART_UPLOAD_ABORTED.getSymbol(), - MULTIPART_UPLOAD_COMPLETED.getSymbol()) + OBJECT_MULTIPART_UPLOAD_INITIATED.getSymbol(), + OBJECT_PUT_REQUESTS.getSymbol()) .build(); setIOStatistics(st); // these are extracted to avoid lookups on heavily used counters. @@ -1630,6 +1643,7 @@ private CommitterStatisticsImpl() { COMMITTER_TASKS_SUCCEEDED.getSymbol()) .withDurationTracking( COMMITTER_COMMIT_JOB.getSymbol(), + COMMITTER_LOAD_SINGLE_PENDING_FILE.getSymbol(), COMMITTER_MATERIALIZE_FILE.getSymbol(), COMMITTER_STAGE_FILE_UPLOAD.getSymbol()) .build(); diff --git a/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/S3AReadOpContext.java b/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/S3AReadOpContext.java index f416cf9485d12..803b7757d252b 100644 --- a/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/S3AReadOpContext.java +++ b/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/S3AReadOpContext.java @@ -64,6 +64,12 @@ public class S3AReadOpContext extends S3AOpContext { */ private long asyncDrainThreshold; + /** + * Vectored IO context for vectored read api + * in {@code S3AInputStream#readVectored(List, IntFunction)}. + */ + private final VectoredIOContext vectoredIOContext; + /** * Instantiate. * @param path path of read @@ -71,17 +77,19 @@ public class S3AReadOpContext extends S3AOpContext { * @param stats Fileystem statistics (may be null) * @param instrumentation statistics context * @param dstFileStatus target file status + * @param vectoredIOContext context for vectored read operation. */ public S3AReadOpContext( final Path path, Invoker invoker, @Nullable FileSystem.Statistics stats, S3AStatisticsContext instrumentation, - FileStatus dstFileStatus) { - + FileStatus dstFileStatus, + VectoredIOContext vectoredIOContext) { super(invoker, stats, instrumentation, dstFileStatus); this.path = requireNonNull(path); + this.vectoredIOContext = requireNonNull(vectoredIOContext, "vectoredIOContext"); } /** @@ -199,6 +207,14 @@ public long getAsyncDrainThreshold() { return asyncDrainThreshold; } + /** + * Get Vectored IO context for this this read op. + * @return vectored IO context. + */ + public VectoredIOContext getVectoredIOContext() { + return vectoredIOContext; + } + @Override public String toString() { final StringBuilder sb = new StringBuilder( diff --git a/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/S3AUtils.java b/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/S3AUtils.java index d644d3f47667c..e7e741d42c521 100644 --- a/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/S3AUtils.java +++ b/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/S3AUtils.java @@ -88,6 +88,7 @@ import static org.apache.hadoop.fs.s3a.impl.InternalConstants.CSE_PADDING_LENGTH; import static org.apache.hadoop.fs.s3a.impl.MultiObjectDeleteSupport.translateDeleteException; import static org.apache.hadoop.io.IOUtils.cleanupWithLogger; +import static org.apache.hadoop.util.functional.RemoteIterators.filteringRemoteIterator; /** * Utility methods for S3A code. @@ -1467,17 +1468,22 @@ public static List flatmapLocatedFiles( /** * List located files and filter them as a classic listFiles(path, filter) * would do. + * This will be incremental, fetching pages async. + * While it is rare for job to have many thousands of files, jobs + * against versioned buckets may return earlier if there are many + * non-visible objects. * @param fileSystem filesystem * @param path path to list * @param recursive recursive listing? * @param filter filter for the filename - * @return the filtered list of entries + * @return interator over the entries. * @throws IOException IO failure. */ - public static List listAndFilter(FileSystem fileSystem, + public static RemoteIterator listAndFilter(FileSystem fileSystem, Path path, boolean recursive, PathFilter filter) throws IOException { - return flatmapLocatedFiles(fileSystem.listFiles(path, recursive), - status -> maybe(filter.accept(status.getPath()), status)); + return filteringRemoteIterator( + fileSystem.listFiles(path, recursive), + status -> filter.accept(status.getPath())); } /** diff --git a/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/Statistic.java b/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/Statistic.java index 86cb18076cc6c..dfe9fdf2d8d37 100644 --- a/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/Statistic.java +++ b/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/Statistic.java @@ -111,6 +111,10 @@ public enum Statistic { StoreStatisticNames.OP_CREATE, "Calls of create()", TYPE_DURATION), + INVOCATION_CREATE_FILE( + StoreStatisticNames.OP_CREATE_FILE, + "Calls of createFile()", + TYPE_DURATION), INVOCATION_CREATE_NON_RECURSIVE( StoreStatisticNames.OP_CREATE_NON_RECURSIVE, "Calls of createNonRecursive()", @@ -459,10 +463,19 @@ public enum Statistic { "committer_commits_reverted", "Count of commits reverted", TYPE_COUNTER), + COMMITTER_LOAD_SINGLE_PENDING_FILE( + "committer_load_single_pending_file", + "Duration to load a single pending file in task commit", + TYPE_DURATION), COMMITTER_MAGIC_FILES_CREATED( "committer_magic_files_created", "Count of files created under 'magic' paths", TYPE_COUNTER), + + COMMITTER_MAGIC_MARKER_PUT( + "committer_magic_marker_put", + "Duration Tracking of marker files created under 'magic' paths", + TYPE_DURATION), COMMITTER_MATERIALIZE_FILE( "committer_materialize_file", "Duration Tracking of time to materialize a file in job commit", diff --git a/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/VectoredIOContext.java b/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/VectoredIOContext.java new file mode 100644 index 0000000000000..31f0ae4cb5515 --- /dev/null +++ b/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/VectoredIOContext.java @@ -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 + * + * 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.fs.s3a; + +import java.util.List; +import java.util.function.IntFunction; + +/** + * Context related to vectored IO operation. + * See {@link S3AInputStream#readVectored(List, IntFunction)}. + */ +public class VectoredIOContext { + + /** + * What is the smallest reasonable seek that we should group + * ranges together during vectored read operation. + */ + private int minSeekForVectorReads; + + /** + * What is the largest size that we should group ranges + * together during vectored read operation. + * Setting this value 0 will disable merging of ranges. + */ + private int maxReadSizeForVectorReads; + + /** + * Default no arg constructor. + */ + public VectoredIOContext() { + } + + public VectoredIOContext setMinSeekForVectoredReads(int minSeek) { + this.minSeekForVectorReads = minSeek; + return this; + } + + public VectoredIOContext setMaxReadSizeForVectoredReads(int maxSize) { + this.maxReadSizeForVectorReads = maxSize; + return this; + } + + public VectoredIOContext build() { + return this; + } + + public int getMinSeekForVectorReads() { + return minSeekForVectorReads; + } + + public int getMaxReadSizeForVectorReads() { + return maxReadSizeForVectorReads; + } + + @Override + public String toString() { + return "VectoredIOContext{" + + "minSeekForVectorReads=" + minSeekForVectorReads + + ", maxReadSizeForVectorReads=" + maxReadSizeForVectorReads + + '}'; + } +} diff --git a/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/WriteOperationHelper.java b/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/WriteOperationHelper.java index ee91bacfb0740..ce50f0b85ed73 100644 --- a/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/WriteOperationHelper.java +++ b/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/WriteOperationHelper.java @@ -24,7 +24,6 @@ import java.io.IOException; import java.io.InputStream; import java.util.List; -import java.util.Map; import java.util.concurrent.atomic.AtomicInteger; import com.amazonaws.services.s3.model.AmazonS3Exception; @@ -40,8 +39,6 @@ import com.amazonaws.services.s3.model.SelectObjectContentResult; import com.amazonaws.services.s3.model.UploadPartRequest; import com.amazonaws.services.s3.model.UploadPartResult; -import com.amazonaws.services.s3.transfer.model.UploadResult; -import org.apache.hadoop.util.Preconditions; import org.slf4j.Logger; import org.slf4j.LoggerFactory; @@ -51,6 +48,7 @@ import org.apache.hadoop.fs.Path; import org.apache.hadoop.fs.PathIOException; import org.apache.hadoop.fs.s3a.api.RequestFactory; +import org.apache.hadoop.fs.s3a.impl.PutObjectOptions; import org.apache.hadoop.fs.s3a.impl.StoreContext; import org.apache.hadoop.fs.s3a.statistics.S3AStatisticsContext; import org.apache.hadoop.fs.s3a.select.SelectBinding; @@ -58,6 +56,7 @@ import org.apache.hadoop.fs.store.audit.AuditSpanSource; import org.apache.hadoop.util.DurationInfo; import org.apache.hadoop.util.functional.CallableRaisingIOE; +import org.apache.hadoop.util.Preconditions; import static org.apache.hadoop.util.Preconditions.checkNotNull; import static org.apache.hadoop.fs.s3a.Invoker.*; @@ -234,22 +233,20 @@ private void deactivateAuditSpan() { * @param destKey destination key * @param inputStream source data. * @param length size, if known. Use -1 for not known - * @param headers optional map of custom headers. + * @param options options for the request * @return the request */ @Retries.OnceRaw public PutObjectRequest createPutObjectRequest(String destKey, InputStream inputStream, long length, - final Map headers) { + final PutObjectOptions options) { activateAuditSpan(); ObjectMetadata objectMetadata = newObjectMetadata(length); - if (headers != null) { - objectMetadata.setUserMetadata(headers); - } return getRequestFactory().newPutObjectRequest( destKey, objectMetadata, + options, inputStream); } @@ -257,18 +254,26 @@ public PutObjectRequest createPutObjectRequest(String destKey, * Create a {@link PutObjectRequest} request to upload a file. * @param dest key to PUT to. * @param sourceFile source file + * @param options options for the request * @return the request */ @Retries.OnceRaw - public PutObjectRequest createPutObjectRequest(String dest, - File sourceFile) { + public PutObjectRequest createPutObjectRequest( + String dest, + File sourceFile, + final PutObjectOptions options) { Preconditions.checkState(sourceFile.length() < Integer.MAX_VALUE, "File length is too big for a single PUT upload"); activateAuditSpan(); - return getRequestFactory(). + final ObjectMetadata objectMetadata = + newObjectMetadata((int) sourceFile.length()); + + PutObjectRequest putObjectRequest = getRequestFactory(). newPutObjectRequest(dest, - newObjectMetadata((int) sourceFile.length()), + objectMetadata, + options, sourceFile); + return putObjectRequest; } /** @@ -298,21 +303,20 @@ public ObjectMetadata newObjectMetadata(long length) { } /** - * Start the multipart upload process. - * Retry policy: retrying, translated. - * @param destKey destination of upload - * @return the upload result containing the ID - * @throws IOException IO problem + * {@inheritDoc} */ @Retries.RetryTranslated - public String initiateMultiPartUpload(String destKey) throws IOException { + public String initiateMultiPartUpload( + final String destKey, + final PutObjectOptions options) + throws IOException { LOG.debug("Initiating Multipart upload to {}", destKey); try (AuditSpan span = activateAuditSpan()) { return retry("initiate MultiPartUpload", destKey, true, () -> { final InitiateMultipartUploadRequest initiateMPURequest = getRequestFactory().newMultipartUploadRequest( - destKey); + destKey, options); return owner.initiateMultipartUpload(initiateMPURequest) .getUploadId(); }); @@ -322,13 +326,14 @@ public String initiateMultiPartUpload(String destKey) throws IOException { /** * Finalize a multipart PUT operation. * This completes the upload, and, if that works, calls - * {@link S3AFileSystem#finishedWrite(String, long, String, String)} + * {@link S3AFileSystem#finishedWrite(String, long, String, String, org.apache.hadoop.fs.s3a.impl.PutObjectOptions)} * to update the filesystem. * Retry policy: retrying, translated. * @param destKey destination of the commit * @param uploadId multipart operation Id * @param partETags list of partial uploads * @param length length of the upload + * @param putOptions put object options * @param retrying retrying callback * @return the result of the operation. * @throws IOException on problems. @@ -339,6 +344,7 @@ private CompleteMultipartUploadResult finalizeMultipartUpload( String uploadId, List partETags, long length, + PutObjectOptions putOptions, Retried retrying) throws IOException { if (partETags.isEmpty()) { throw new PathIOException(destKey, @@ -357,7 +363,8 @@ private CompleteMultipartUploadResult finalizeMultipartUpload( request); }); owner.finishedWrite(destKey, length, uploadResult.getETag(), - uploadResult.getVersionId()); + uploadResult.getVersionId(), + putOptions); return uploadResult; } } @@ -373,6 +380,7 @@ private CompleteMultipartUploadResult finalizeMultipartUpload( * @param length length of the upload * @param errorCount a counter incremented by 1 on every error; for * use in statistics + * @param putOptions put object options * @return the result of the operation. * @throws IOException if problems arose which could not be retried, or * the retry count was exceeded @@ -383,7 +391,8 @@ public CompleteMultipartUploadResult completeMPUwithRetries( String uploadId, List partETags, long length, - AtomicInteger errorCount) + AtomicInteger errorCount, + PutObjectOptions putOptions) throws IOException { checkNotNull(uploadId); checkNotNull(partETags); @@ -393,8 +402,8 @@ public CompleteMultipartUploadResult completeMPUwithRetries( uploadId, partETags, length, - (text, e, r, i) -> errorCount.incrementAndGet() - ); + putOptions, + (text, e, r, i) -> errorCount.incrementAndGet()); } /** @@ -550,37 +559,43 @@ public String toString() { * Byte length is calculated from the file length, or, if there is no * file, from the content length of the header. * @param putObjectRequest the request + * @param putOptions put object options * @return the upload initiated * @throws IOException on problems */ @Retries.RetryTranslated - public PutObjectResult putObject(PutObjectRequest putObjectRequest) + public PutObjectResult putObject(PutObjectRequest putObjectRequest, + PutObjectOptions putOptions) throws IOException { return retry("Writing Object", putObjectRequest.getKey(), true, withinAuditSpan(getAuditSpan(), () -> - owner.putObjectDirect(putObjectRequest))); + owner.putObjectDirect(putObjectRequest, putOptions))); } /** - * PUT an object via the transfer manager. + * PUT an object. + * * @param putObjectRequest the request - * @return the result of the operation + * @param putOptions put object options + * * @throws IOException on problems */ @Retries.RetryTranslated - public UploadResult uploadObject(PutObjectRequest putObjectRequest) + public void uploadObject(PutObjectRequest putObjectRequest, + PutObjectOptions putOptions) throws IOException { - // no retry; rely on xfer manager logic - return retry("Writing Object", + + retry("Writing Object", putObjectRequest.getKey(), true, withinAuditSpan(getAuditSpan(), () -> - owner.executePut(putObjectRequest, null))); + owner.putObjectDirect(putObjectRequest, putOptions))); } /** * Revert a commit by deleting the file. - * Relies on retry code in filesystem + * Relies on retry code in filesystem. + * Does not attempt to recreate the parent directory * @throws IOException on problems * @param destKey destination key */ @@ -591,13 +606,14 @@ public void revertCommit(String destKey) throws IOException { Path destPath = owner.keyToQualifiedPath(destKey); owner.deleteObjectAtPath(destPath, destKey, true); - owner.maybeCreateFakeParentDirectory(destPath); })); } /** * This completes a multipart upload to the destination key via * {@code finalizeMultipartUpload()}. + * Markers are never deleted on commit; this avoids having to + * issue many duplicate deletions. * Retry policy: retrying, translated. * Retries increment the {@code errorCount} counter. * @param destKey destination @@ -623,8 +639,8 @@ public CompleteMultipartUploadResult commitUpload( uploadId, partETags, length, - Invoker.NO_OP - ); + PutObjectOptions.keepingDirs(), + Invoker.NO_OP); } /** diff --git a/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/WriteOperations.java b/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/WriteOperations.java index 7604bbe46b51a..321390446f705 100644 --- a/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/WriteOperations.java +++ b/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/WriteOperations.java @@ -25,7 +25,6 @@ import java.io.IOException; import java.io.InputStream; import java.util.List; -import java.util.Map; import java.util.concurrent.atomic.AtomicInteger; import com.amazonaws.services.s3.model.CompleteMultipartUploadResult; @@ -38,11 +37,11 @@ import com.amazonaws.services.s3.model.SelectObjectContentResult; import com.amazonaws.services.s3.model.UploadPartRequest; import com.amazonaws.services.s3.model.UploadPartResult; -import com.amazonaws.services.s3.transfer.model.UploadResult; import org.apache.hadoop.conf.Configuration; import org.apache.hadoop.fs.Path; import org.apache.hadoop.fs.PathIOException; +import org.apache.hadoop.fs.s3a.impl.PutObjectOptions; import org.apache.hadoop.fs.store.audit.AuditSpanSource; import org.apache.hadoop.util.functional.CallableRaisingIOE; @@ -79,22 +78,25 @@ T retry(String action, * @param destKey destination key * @param inputStream source data. * @param length size, if known. Use -1 for not known - * @param headers optional map of custom headers. + * @param options options for the request * @return the request */ PutObjectRequest createPutObjectRequest(String destKey, InputStream inputStream, long length, - @Nullable Map headers); + @Nullable PutObjectOptions options); /** * Create a {@link PutObjectRequest} request to upload a file. * @param dest key to PUT to. * @param sourceFile source file + * @param options options for the request * @return the request */ - PutObjectRequest createPutObjectRequest(String dest, - File sourceFile); + PutObjectRequest createPutObjectRequest( + String dest, + File sourceFile, + @Nullable PutObjectOptions options); /** * Callback on a successful write. @@ -121,11 +123,12 @@ PutObjectRequest createPutObjectRequest(String dest, * Start the multipart upload process. * Retry policy: retrying, translated. * @param destKey destination of upload + * @param options options for the put request * @return the upload result containing the ID * @throws IOException IO problem */ @Retries.RetryTranslated - String initiateMultiPartUpload(String destKey) throws IOException; + String initiateMultiPartUpload(String destKey, PutObjectOptions options) throws IOException; /** * This completes a multipart upload to the destination key via @@ -138,6 +141,7 @@ PutObjectRequest createPutObjectRequest(String dest, * @param length length of the upload * @param errorCount a counter incremented by 1 on every error; for * use in statistics + * @param putOptions put object options * @return the result of the operation. * @throws IOException if problems arose which could not be retried, or * the retry count was exceeded @@ -148,7 +152,8 @@ CompleteMultipartUploadResult completeMPUwithRetries( String uploadId, List partETags, long length, - AtomicInteger errorCount) + AtomicInteger errorCount, + PutObjectOptions putOptions) throws IOException; /** @@ -238,26 +243,32 @@ UploadPartRequest newUploadPartRequest( * Byte length is calculated from the file length, or, if there is no * file, from the content length of the header. * @param putObjectRequest the request + * @param putOptions put object options * @return the upload initiated * @throws IOException on problems */ @Retries.RetryTranslated - PutObjectResult putObject(PutObjectRequest putObjectRequest) + PutObjectResult putObject(PutObjectRequest putObjectRequest, + PutObjectOptions putOptions) throws IOException; /** * PUT an object via the transfer manager. + * * @param putObjectRequest the request - * @return the result of the operation + * @param putOptions put object options + * * @throws IOException on problems */ @Retries.RetryTranslated - UploadResult uploadObject(PutObjectRequest putObjectRequest) + void uploadObject(PutObjectRequest putObjectRequest, + PutObjectOptions putOptions) throws IOException; /** * Revert a commit by deleting the file. - * Relies on retry code in filesystem + * No attempt is made to probe for/recreate a parent dir marker + * Relies on retry code in filesystem. * @throws IOException on problems * @param destKey destination key */ diff --git a/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/api/RequestFactory.java b/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/api/RequestFactory.java index 97a15d95132f4..cae4d3ef034e8 100644 --- a/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/api/RequestFactory.java +++ b/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/api/RequestFactory.java @@ -44,11 +44,13 @@ import com.amazonaws.services.s3.model.SSEAwsKeyManagementParams; import com.amazonaws.services.s3.model.SSECustomerKey; import com.amazonaws.services.s3.model.SelectObjectContentRequest; +import com.amazonaws.services.s3.model.StorageClass; import com.amazonaws.services.s3.model.UploadPartRequest; import org.apache.hadoop.fs.PathIOException; import org.apache.hadoop.fs.s3a.S3AEncryptionMethods; import org.apache.hadoop.fs.s3a.auth.delegation.EncryptionSecrets; +import org.apache.hadoop.fs.s3a.impl.PutObjectOptions; /** * Factory for S3 objects. @@ -106,6 +108,12 @@ public interface RequestFactory { */ String getContentEncoding(); + /** + * Get the object storage class, return null if none. + * @return storage class + */ + StorageClass getStorageClass(); + /** * Create a new object metadata instance. * Any standard metadata headers are added here, for example: @@ -134,11 +142,12 @@ CopyObjectRequest newCopyObjectRequest(String srcKey, * Adds the ACL and metadata * @param key key of object * @param metadata metadata header + * @param options options for the request * @param srcfile source file * @return the request */ PutObjectRequest newPutObjectRequest(String key, - ObjectMetadata metadata, File srcfile); + ObjectMetadata metadata, PutObjectOptions options, File srcfile); /** * Create a {@link PutObjectRequest} request. @@ -146,11 +155,13 @@ PutObjectRequest newPutObjectRequest(String key, * operation. * @param key key of object * @param metadata metadata header + * @param options options for the request * @param inputStream source data. * @return the request */ PutObjectRequest newPutObjectRequest(String key, ObjectMetadata metadata, + PutObjectOptions options, InputStream inputStream); /** @@ -183,10 +194,12 @@ AbortMultipartUploadRequest newAbortMultipartUploadRequest( /** * Start a multipart upload. * @param destKey destination object key + * @param options options for the request * @return the request. */ InitiateMultipartUploadRequest newMultipartUploadRequest( - String destKey); + String destKey, + @Nullable PutObjectOptions options); /** * Complete a multipart upload. diff --git a/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/commit/AbstractS3ACommitter.java b/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/commit/AbstractS3ACommitter.java index f08d6448e4993..78b687cc6f19c 100644 --- a/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/commit/AbstractS3ACommitter.java +++ b/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/commit/AbstractS3ACommitter.java @@ -26,32 +26,30 @@ import java.util.Date; import java.util.List; import java.util.UUID; -import java.util.concurrent.ExecutorService; -import java.util.concurrent.Future; -import java.util.concurrent.TimeUnit; import com.amazonaws.services.s3.model.MultipartUpload; - -import org.apache.hadoop.classification.VisibleForTesting; -import org.apache.hadoop.util.Preconditions; -import org.apache.hadoop.thirdparty.com.google.common.util.concurrent.ThreadFactoryBuilder; import org.slf4j.Logger; import org.slf4j.LoggerFactory; import org.apache.commons.lang3.tuple.Pair; import org.apache.commons.lang3.StringUtils; +import org.apache.hadoop.classification.VisibleForTesting; import org.apache.hadoop.conf.Configuration; import org.apache.hadoop.fs.FileStatus; import org.apache.hadoop.fs.FileSystem; import org.apache.hadoop.fs.Path; -import org.apache.hadoop.fs.s3a.S3AFileSystem; +import org.apache.hadoop.fs.RemoteIterator; import org.apache.hadoop.fs.audit.AuditConstants; -import org.apache.hadoop.fs.audit.CommonAuditContext; import org.apache.hadoop.fs.store.audit.AuditSpan; import org.apache.hadoop.fs.store.audit.AuditSpanSource; +import org.apache.hadoop.fs.s3a.S3AFileSystem; import org.apache.hadoop.fs.s3a.commit.files.PendingSet; +import org.apache.hadoop.fs.s3a.commit.files.PersistentCommitData; import org.apache.hadoop.fs.s3a.commit.files.SinglePendingCommit; import org.apache.hadoop.fs.s3a.commit.files.SuccessData; +import org.apache.hadoop.fs.s3a.commit.impl.AuditContextUpdater; +import org.apache.hadoop.fs.s3a.commit.impl.CommitContext; +import org.apache.hadoop.fs.s3a.commit.impl.CommitOperations; import org.apache.hadoop.fs.s3a.statistics.CommitterStatistics; import org.apache.hadoop.fs.statistics.IOStatistics; import org.apache.hadoop.fs.statistics.IOStatisticsSnapshot; @@ -62,24 +60,33 @@ import org.apache.hadoop.mapreduce.TaskAttemptContext; import org.apache.hadoop.mapreduce.TaskAttemptID; import org.apache.hadoop.mapreduce.lib.output.PathOutputCommitter; +import org.apache.hadoop.mapreduce.lib.output.committer.manifest.impl.ManifestStoreOperations; +import org.apache.hadoop.mapreduce.lib.output.committer.manifest.impl.ManifestStoreOperationsThroughFileSystem; import org.apache.hadoop.net.NetUtils; import org.apache.hadoop.util.DurationInfo; -import org.apache.hadoop.util.concurrent.HadoopExecutors; import org.apache.hadoop.util.functional.InvocationRaisingIOE; +import org.apache.hadoop.util.functional.TaskPool; -import static org.apache.hadoop.fs.s3a.Constants.THREAD_POOL_SHUTDOWN_DELAY_SECONDS; +import static java.util.Objects.requireNonNull; +import static org.apache.hadoop.fs.s3a.Constants.DEFAULT_MAXIMUM_CONNECTIONS; +import static org.apache.hadoop.fs.s3a.Constants.DEFAULT_MAX_TOTAL_TASKS; +import static org.apache.hadoop.fs.s3a.Constants.MAXIMUM_CONNECTIONS; +import static org.apache.hadoop.fs.s3a.Constants.MAX_TOTAL_TASKS; import static org.apache.hadoop.fs.s3a.Invoker.ignoreIOExceptions; import static org.apache.hadoop.fs.s3a.S3AUtils.*; import static org.apache.hadoop.fs.s3a.Statistic.COMMITTER_COMMIT_JOB; import static org.apache.hadoop.fs.audit.CommonAuditContext.currentAuditContext; import static org.apache.hadoop.fs.s3a.commit.CommitConstants.*; import static org.apache.hadoop.fs.s3a.commit.CommitUtils.*; -import static org.apache.hadoop.fs.s3a.commit.CommitUtilsWithMR.*; import static org.apache.hadoop.fs.s3a.commit.InternalCommitterConstants.E_NO_SPARK_UUID; import static org.apache.hadoop.fs.s3a.commit.InternalCommitterConstants.FS_S3A_COMMITTER_UUID; import static org.apache.hadoop.fs.s3a.commit.InternalCommitterConstants.FS_S3A_COMMITTER_UUID_SOURCE; import static org.apache.hadoop.fs.s3a.commit.InternalCommitterConstants.SPARK_WRITE_UUID; +import static org.apache.hadoop.fs.s3a.commit.impl.CommitUtilsWithMR.*; import static org.apache.hadoop.fs.statistics.impl.IOStatisticsBinding.trackDurationOfInvocation; +import static org.apache.hadoop.mapreduce.lib.output.committer.manifest.files.DiagnosticKeys.STAGE; +import static org.apache.hadoop.mapreduce.lib.output.committer.manifest.impl.ManifestCommitterSupport.createJobSummaryFilename; +import static org.apache.hadoop.util.functional.RemoteIterators.toList; /** * Abstract base class for S3A committers; allows for any commonality @@ -136,8 +143,6 @@ public abstract class AbstractS3ACommitter extends PathOutputCommitter */ private final JobUUIDSource uuidSource; - private final CommonAuditContext commonAuditContext; - /** * Has this instance been used for job setup? * If so then it is safe for a locally generated @@ -145,11 +150,6 @@ public abstract class AbstractS3ACommitter extends PathOutputCommitter */ private boolean jobSetup; - /** - * Thread pool for task execution. - */ - private ExecutorService threadPool; - /** Underlying commit operations. */ private final CommitOperations commitOperations; @@ -203,9 +203,8 @@ protected AbstractS3ACommitter( Path outputPath, TaskAttemptContext context) throws IOException { super(outputPath, context); - Preconditions.checkArgument(outputPath != null, "null output path"); - Preconditions.checkArgument(context != null, "null job context"); - this.jobContext = context; + setOutputPath(outputPath); + this.jobContext = requireNonNull(context, "null job context"); this.role = "Task committer " + context.getTaskAttemptID(); setConf(context.getConfiguration()); Pair id = buildJobUUID( @@ -219,17 +218,20 @@ protected AbstractS3ACommitter( S3AFileSystem fs = getDestS3AFS(); // set this thread's context with the job ID. // audit spans created in this thread will pick - // up this value. - this.commonAuditContext = currentAuditContext(); - updateCommonContext(); + // up this value., including the commit operations instance + // soon to be created. + new AuditContextUpdater(jobContext) + .updateCurrentAuditContext(); + // the filesystem is the span source, always. - auditSpanSource = fs.getAuditSpanSource(); + this.auditSpanSource = fs.getAuditSpanSource(); this.createJobMarker = context.getConfiguration().getBoolean( CREATE_SUCCESSFUL_JOB_OUTPUT_DIR_MARKER, DEFAULT_CREATE_SUCCESSFUL_JOB_DIR_MARKER); // the statistics are shared between this committer and its operations. this.committerStatistics = fs.newCommitterStatistics(); - this.commitOperations = new CommitOperations(fs, committerStatistics); + this.commitOperations = new CommitOperations(fs, committerStatistics, + outputPath.toString()); } /** @@ -267,8 +269,7 @@ public final Path getOutputPath() { * @param outputPath new value */ protected final void setOutputPath(Path outputPath) { - Preconditions.checkNotNull(outputPath, "Null output path"); - this.outputPath = outputPath; + this.outputPath = requireNonNull(outputPath, "Null output path"); } /** @@ -338,6 +339,12 @@ public Path getJobAttemptPath(JobContext context) { return getJobAttemptPath(getAppAttemptId(context)); } + /** + * Compute the path under which all job attempts will be placed. + * @return the path to store job attempt data. + */ + protected abstract Path getJobPath(); + /** * Compute the path where the output of a given job attempt will be placed. * @param appAttemptId the ID of the application attempt for this job. @@ -440,6 +447,7 @@ protected FileSystem getDestinationFS(Path out, Configuration config) protected boolean requiresDelayedCommitOutputInFileSystem() { return false; } + /** * Task recovery considered Unsupported: Warn and fail. * @param taskContext Context of the task whose output is being recovered @@ -450,7 +458,7 @@ public void recoverTask(TaskAttemptContext taskContext) throws IOException { LOG.warn("Cannot recover task {}", taskContext.getTaskAttemptID()); throw new PathCommitException(outputPath, String.format("Unable to recover task %s", - taskContext.getTaskAttemptID())); + taskContext.getTaskAttemptID())); } /** @@ -459,11 +467,16 @@ public void recoverTask(TaskAttemptContext taskContext) throws IOException { * * While the classic committers create a 0-byte file, the S3A committers * PUT up a the contents of a {@link SuccessData} file. + * * @param context job context * @param pending the pending commits + * + * @return the success data, even if the marker wasn't created + * * @throws IOException IO failure */ - protected void maybeCreateSuccessMarkerFromCommits(JobContext context, + protected SuccessData maybeCreateSuccessMarkerFromCommits( + JobContext context, ActiveCommit pending) throws IOException { List filenames = new ArrayList<>(pending.size()); // The list of committed objects in pending is size limited in @@ -472,41 +485,86 @@ protected void maybeCreateSuccessMarkerFromCommits(JobContext context, // load in all the pending statistics IOStatisticsSnapshot snapshot = new IOStatisticsSnapshot( pending.getIOStatistics()); + // and the current statistics snapshot.aggregate(getIOStatistics()); - maybeCreateSuccessMarker(context, filenames, snapshot); + return maybeCreateSuccessMarker(context, filenames, snapshot); } /** * if the job requires a success marker on a successful job, - * create the file {@link CommitConstants#_SUCCESS}. + * create the {@code _SUCCESS} file. * * While the classic committers create a 0-byte file, the S3A committers * PUT up a the contents of a {@link SuccessData} file. + * The file is returned, even if no marker is created. + * This is so it can be saved to a report directory. * @param context job context * @param filenames list of filenames. * @param ioStatistics any IO Statistics to include * @throws IOException IO failure + * @return the success data. */ - protected void maybeCreateSuccessMarker(JobContext context, - List filenames, + protected SuccessData maybeCreateSuccessMarker( + final JobContext context, + final List filenames, final IOStatisticsSnapshot ioStatistics) throws IOException { + + SuccessData successData = + createSuccessData(context, filenames, ioStatistics, + getDestFS().getConf()); if (createJobMarker) { - // create a success data structure and then save it - SuccessData successData = new SuccessData(); - successData.setCommitter(getName()); - successData.setJobId(uuid); - successData.setJobIdSource(uuidSource.getText()); - successData.setDescription(getRole()); - successData.setHostname(NetUtils.getLocalHostname()); - Date now = new Date(); - successData.setTimestamp(now.getTime()); - successData.setDate(now.toString()); - successData.setFilenames(filenames); - successData.getIOStatistics().aggregate(ioStatistics); + // save it to the job dest dir commitOperations.createSuccessMarker(getOutputPath(), successData, true); } + return successData; + } + + /** + * Create the success data structure from a job context. + * @param context job context. + * @param filenames short list of filenames; nullable + * @param ioStatistics IOStatistics snapshot + * @param destConf config of the dest fs, can be null + * @return the structure + * + */ + private SuccessData createSuccessData(final JobContext context, + final List filenames, + final IOStatisticsSnapshot ioStatistics, + final Configuration destConf) { + // create a success data structure + SuccessData successData = new SuccessData(); + successData.setCommitter(getName()); + successData.setJobId(uuid); + successData.setJobIdSource(uuidSource.getText()); + successData.setDescription(getRole()); + successData.setHostname(NetUtils.getLocalHostname()); + Date now = new Date(); + successData.setTimestamp(now.getTime()); + successData.setDate(now.toString()); + if (filenames != null) { + successData.setFilenames(filenames); + } + successData.getIOStatistics().aggregate(ioStatistics); + // attach some config options as diagnostics to assist + // in debugging performance issues. + + // commit thread pool size + successData.addDiagnostic(FS_S3A_COMMITTER_THREADS, + Integer.toString(getJobCommitThreadCount(context))); + + // and filesystem http connection and thread pool sizes + if (destConf != null) { + successData.addDiagnostic(MAXIMUM_CONNECTIONS, + destConf.get(MAXIMUM_CONNECTIONS, + Integer.toString(DEFAULT_MAXIMUM_CONNECTIONS))); + successData.addDiagnostic(MAX_TOTAL_TASKS, + destConf.get(MAX_TOTAL_TASKS, + Integer.toString(DEFAULT_MAX_TOTAL_TASKS))); + } + return successData; } /** @@ -556,7 +614,11 @@ public void setupJob(JobContext context) throws IOException { @Override public void setupTask(TaskAttemptContext context) throws IOException { TaskAttemptID attemptID = context.getTaskAttemptID(); - updateCommonContext(); + + // update the context so that task IO in the same thread has + // the relevant values. + new AuditContextUpdater(context) + .updateCurrentAuditContext(); try (DurationInfo d = new DurationInfo(LOG, "Setup Task %s", attemptID)) { @@ -572,6 +634,9 @@ && getUUIDSource() == JobUUIDSource.GeneratedLocally) { } Path taskAttemptPath = getTaskAttemptPath(context); FileSystem fs = taskAttemptPath.getFileSystem(getConf()); + // delete that ta path if somehow it was there + fs.delete(taskAttemptPath, true); + // create an empty directory fs.mkdirs(taskAttemptPath); } } @@ -596,25 +661,23 @@ protected FileSystem getTaskAttemptFilesystem(TaskAttemptContext context) * On a failure or abort of a single file's commit, all its uploads are * aborted. * The revert operation lists the files already committed and deletes them. - * @param context job context + * @param commitContext commit context * @param pending pending uploads * @throws IOException on any failure */ protected void commitPendingUploads( - final JobContext context, + final CommitContext commitContext, final ActiveCommit pending) throws IOException { if (pending.isEmpty()) { LOG.warn("{}: No pending uploads to commit", getRole()); } try (DurationInfo ignored = new DurationInfo(LOG, - "committing the output of %s task(s)", pending.size()); - CommitOperations.CommitContext commitContext - = initiateCommitOperation()) { + "committing the output of %s task(s)", pending.size())) { - Tasks.foreach(pending.getSourceFiles()) + TaskPool.foreach(pending.getSourceFiles()) .stopOnFailure() .suppressExceptions(false) - .executeWith(buildSubmitter(context)) + .executeWith(commitContext.getOuterSubmitter()) .abortWith(status -> loadAndAbort(commitContext, pending, status, true, false)) .revertWith(status -> @@ -629,35 +692,39 @@ protected void commitPendingUploads( * This check avoids the situation where the inability to read * a file only surfaces partway through the job commit, so * results in the destination being tainted. - * @param context job context + * @param commitContext commit context * @param pending the pending operations * @throws IOException any failure */ protected void precommitCheckPendingFiles( - final JobContext context, + final CommitContext commitContext, final ActiveCommit pending) throws IOException { FileSystem sourceFS = pending.getSourceFS(); try (DurationInfo ignored = new DurationInfo(LOG, "Preflight Load of pending files")) { - Tasks.foreach(pending.getSourceFiles()) + TaskPool.foreach(pending.getSourceFiles()) .stopOnFailure() .suppressExceptions(false) - .executeWith(buildSubmitter(context)) - .run(status -> PendingSet.load(sourceFS, status)); + .executeWith(commitContext.getOuterSubmitter()) + .run(status -> PersistentCommitData.load(sourceFS, status, + commitContext.getPendingSetSerializer())); } } /** * Load a pendingset file and commit all of its contents. + * Invoked within a parallel run; the commitContext thread + * pool is already busy/possibly full, so do not + * execute work through the same submitter. * @param commitContext context to commit through * @param activeCommit commit state * @param status file to load * @throws IOException failure */ private void loadAndCommit( - final CommitOperations.CommitContext commitContext, + final CommitContext commitContext, final ActiveCommit activeCommit, final FileStatus status) throws IOException { @@ -665,18 +732,20 @@ private void loadAndCommit( try (DurationInfo ignored = new DurationInfo(LOG, "Loading and committing files in pendingset %s", path)) { - PendingSet pendingSet = PendingSet.load(activeCommit.getSourceFS(), - status); + PendingSet pendingSet = PersistentCommitData.load( + activeCommit.getSourceFS(), + status, + commitContext.getPendingSetSerializer()); String jobId = pendingSet.getJobId(); if (!StringUtils.isEmpty(jobId) && !getUUID().equals(jobId)) { throw new PathCommitException(path, String.format("Mismatch in Job ID (%s) and commit job ID (%s)", getUUID(), jobId)); } - Tasks.foreach(pendingSet.getCommits()) + TaskPool.foreach(pendingSet.getCommits()) .stopOnFailure() .suppressExceptions(false) - .executeWith(singleThreadSubmitter()) + .executeWith(commitContext.getInnerSubmitter()) .onFailure((commit, exception) -> commitContext.abortSingleCommit(commit)) .abortWith(commitContext::abortSingleCommit) @@ -692,22 +761,27 @@ private void loadAndCommit( /** * Load a pendingset file and revert all of its contents. + * Invoked within a parallel run; the commitContext thread + * pool is already busy/possibly full, so do not + * execute work through the same submitter. * @param commitContext context to commit through * @param activeCommit commit state * @param status status of file to load * @throws IOException failure */ private void loadAndRevert( - final CommitOperations.CommitContext commitContext, + final CommitContext commitContext, final ActiveCommit activeCommit, final FileStatus status) throws IOException { final Path path = status.getPath(); try (DurationInfo ignored = new DurationInfo(LOG, false, "Committing %s", path)) { - PendingSet pendingSet = PendingSet.load(activeCommit.getSourceFS(), - status); - Tasks.foreach(pendingSet.getCommits()) + PendingSet pendingSet = PersistentCommitData.load( + activeCommit.getSourceFS(), + status, + commitContext.getPendingSetSerializer()); + TaskPool.foreach(pendingSet.getCommits()) .suppressExceptions(true) .run(commitContext::revertCommit); } @@ -715,6 +789,9 @@ private void loadAndRevert( /** * Load a pendingset file and abort all of its contents. + * Invoked within a parallel run; the commitContext thread + * pool is already busy/possibly full, so do not + * execute work through the same submitter. * @param commitContext context to commit through * @param activeCommit commit state * @param status status of file to load @@ -722,7 +799,7 @@ private void loadAndRevert( * @throws IOException failure */ private void loadAndAbort( - final CommitOperations.CommitContext commitContext, + final CommitContext commitContext, final ActiveCommit activeCommit, final FileStatus status, final boolean suppressExceptions, @@ -731,11 +808,13 @@ private void loadAndAbort( final Path path = status.getPath(); try (DurationInfo ignored = new DurationInfo(LOG, false, "Aborting %s", path)) { - PendingSet pendingSet = PendingSet.load(activeCommit.getSourceFS(), - status); + PendingSet pendingSet = PersistentCommitData.load( + activeCommit.getSourceFS(), + status, + commitContext.getPendingSetSerializer()); FileSystem fs = getDestFS(); - Tasks.foreach(pendingSet.getCommits()) - .executeWith(singleThreadSubmitter()) + TaskPool.foreach(pendingSet.getCommits()) + .executeWith(commitContext.getInnerSubmitter()) .suppressExceptions(suppressExceptions) .run(commit -> { try { @@ -752,28 +831,51 @@ private void loadAndAbort( } /** - * Start the final commit/abort commit operations. + * Start the final job commit/abort commit operations. + * @param context job context + * @return a commit context through which the operations can be invoked. + * @throws IOException failure. + */ + protected CommitContext initiateJobOperation( + final JobContext context) + throws IOException { + + return getCommitOperations().createCommitContext( + context, + getOutputPath(), + getJobCommitThreadCount(context)); + } + /** + * Start a ask commit/abort commit operations. + * This may have a different thread count. + * @param context job or task context * @return a commit context through which the operations can be invoked. * @throws IOException failure. */ - protected CommitOperations.CommitContext initiateCommitOperation() + protected CommitContext initiateTaskOperation( + final JobContext context) throws IOException { - return getCommitOperations().initiateCommitOperation(getOutputPath()); + + return getCommitOperations().createCommitContext( + context, + getOutputPath(), + getTaskCommitThreadCount(context)); } /** * Internal Job commit operation: where the S3 requests are made * (potentially in parallel). - * @param context job context + * @param commitContext commit context * @param pending pending commits * @throws IOException any failure */ - protected void commitJobInternal(JobContext context, - ActiveCommit pending) + protected void commitJobInternal( + final CommitContext commitContext, + final ActiveCommit pending) throws IOException { trackDurationOfInvocation(committerStatistics, COMMITTER_COMMIT_JOB.getSymbol(), - () -> commitPendingUploads(context, pending)); + () -> commitPendingUploads(commitContext, pending)); } @Override @@ -782,7 +884,9 @@ public void abortJob(JobContext context, JobStatus.State state) LOG.info("{}: aborting job {} in state {}", getRole(), jobIdString(context), state); // final cleanup operations - abortJobInternal(context, false); + try (CommitContext commitContext = initiateJobOperation(context)){ + abortJobInternal(commitContext, false); + } } @@ -790,30 +894,33 @@ public void abortJob(JobContext context, JobStatus.State state) * The internal job abort operation; can be overridden in tests. * This must clean up operations; it is called when a commit fails, as * well as in an {@link #abortJob(JobContext, JobStatus.State)} call. - * The base implementation calls {@link #cleanup(JobContext, boolean)} + * The base implementation calls {@link #cleanup(CommitContext, boolean)} * so cleans up the filesystems and destroys the thread pool. * Subclasses must always invoke this superclass method after their * own operations. - * @param context job context + * Creates and closes its own commit context. + * + * @param commitContext commit context * @param suppressExceptions should exceptions be suppressed? * @throws IOException any IO problem raised when suppressExceptions is false. */ - protected void abortJobInternal(JobContext context, + protected void abortJobInternal(CommitContext commitContext, boolean suppressExceptions) throws IOException { - cleanup(context, suppressExceptions); + cleanup(commitContext, suppressExceptions); } /** * Abort all pending uploads to the destination directory during * job cleanup operations. * Note: this instantiates the thread pool if required -so - * {@link #destroyThreadPool()} must be called after this. * @param suppressExceptions should exceptions be suppressed + * @param commitContext commit context * @throws IOException IO problem */ protected void abortPendingUploadsInCleanup( - boolean suppressExceptions) throws IOException { + boolean suppressExceptions, + CommitContext commitContext) throws IOException { // return early if aborting is disabled. if (!shouldAbortUploadsInCleanup()) { LOG.debug("Not cleanup up pending uploads to {} as {} is false ", @@ -824,9 +931,7 @@ protected void abortPendingUploadsInCleanup( Path dest = getOutputPath(); try (DurationInfo ignored = new DurationInfo(LOG, "Aborting all pending commits under %s", - dest); - CommitOperations.CommitContext commitContext - = initiateCommitOperation()) { + dest)) { CommitOperations ops = getCommitOperations(); List pending; try { @@ -840,8 +945,8 @@ protected void abortPendingUploadsInCleanup( LOG.warn("{} pending uploads were found -aborting", pending.size()); LOG.warn("If other tasks/jobs are writing to {}," + "this action may cause them to fail", dest); - Tasks.foreach(pending) - .executeWith(buildSubmitter(getJobContext())) + TaskPool.foreach(pending) + .executeWith(commitContext.getOuterSubmitter()) .suppressExceptions(suppressExceptions) .run(u -> commitContext.abortMultipartCommit( u.getKey(), u.getUploadId())); @@ -863,12 +968,12 @@ private boolean shouldAbortUploadsInCleanup() { * they can be loaded. * The Magic committer does not, because of the overhead of reading files * from S3 makes it too expensive. - * @param context job context + * @param commitContext commit context * @param pending the pending operations * @throws IOException any failure */ @VisibleForTesting - public void preCommitJob(JobContext context, + public void preCommitJob(CommitContext commitContext, ActiveCommit pending) throws IOException { } @@ -890,23 +995,55 @@ public void preCommitJob(JobContext context, @Override public void commitJob(JobContext context) throws IOException { String id = jobIdString(context); + // the commit context is created outside a try-with-resources block + // so it can be used in exception handling. + CommitContext commitContext = null; + + SuccessData successData = null; + IOException failure = null; + String stage = "preparing"; try (DurationInfo d = new DurationInfo(LOG, "%s: commitJob(%s)", getRole(), id)) { + commitContext = initiateJobOperation(context); ActiveCommit pending - = listPendingUploadsToCommit(context); - preCommitJob(context, pending); - commitJobInternal(context, pending); + = listPendingUploadsToCommit(commitContext); + stage = "precommit"; + preCommitJob(commitContext, pending); + stage = "commit"; + commitJobInternal(commitContext, pending); + stage = "completed"; jobCompleted(true); - maybeCreateSuccessMarkerFromCommits(context, pending); - cleanup(context, false); + stage = "marker"; + successData = maybeCreateSuccessMarkerFromCommits(context, pending); + stage = "cleanup"; + cleanup(commitContext, false); } catch (IOException e) { + // failure. record it for the summary + failure = e; LOG.warn("Commit failure for job {}", id, e); jobCompleted(false); - abortJobInternal(context, true); + abortJobInternal(commitContext, true); throw e; } finally { - resetCommonContext(); + // save the report summary, even on failure + if (commitContext != null) { + if (successData == null) { + // if the commit did not get as far as creating success data, create one. + successData = createSuccessData(context, null, null, + getDestFS().getConf()); + } + // save quietly, so no exceptions are raised + maybeSaveSummary(stage, + commitContext, + successData, + failure, + true, + true); + // and close that commit context + commitContext.close(); + } } + } /** @@ -925,30 +1062,28 @@ protected void jobCompleted(boolean success) { /** * Get the list of pending uploads for this job attempt. - * @param context job context + * @param commitContext commit context * @return a list of pending uploads. * @throws IOException Any IO failure */ protected abstract ActiveCommit listPendingUploadsToCommit( - JobContext context) + CommitContext commitContext) throws IOException; /** * Cleanup the job context, including aborting anything pending * and destroying the thread pool. - * @param context job context + * @param commitContext commit context * @param suppressExceptions should exceptions be suppressed? * @throws IOException any failure if exceptions were not suppressed. */ - protected void cleanup(JobContext context, + protected void cleanup(CommitContext commitContext, boolean suppressExceptions) throws IOException { try (DurationInfo d = new DurationInfo(LOG, - "Cleanup job %s", jobIdString(context))) { - abortPendingUploadsInCleanup(suppressExceptions); + "Cleanup job %s", jobIdString(commitContext.getJobContext()))) { + abortPendingUploadsInCleanup(suppressExceptions, commitContext); } finally { - destroyThreadPool(); cleanupStagingDirs(); - resetCommonContext(); } } @@ -958,8 +1093,9 @@ public void cleanupJob(JobContext context) throws IOException { String r = getRole(); String id = jobIdString(context); LOG.warn("{}: using deprecated cleanupJob call for {}", r, id); - try (DurationInfo d = new DurationInfo(LOG, "%s: cleanup Job %s", r, id)) { - cleanup(context, true); + try (DurationInfo d = new DurationInfo(LOG, "%s: cleanup Job %s", r, id); + CommitContext commitContext = initiateJobOperation(context)) { + cleanup(commitContext, true); } } @@ -1016,137 +1152,26 @@ protected String getRole() { return role; } - /** - * Returns an {@link Tasks.Submitter} for parallel tasks. The number of - * threads in the thread-pool is set by fs.s3a.committer.threads. - * If num-threads is 0, this will return null; - * this is used in Tasks as a cue - * to switch to single-threaded execution. - * - * @param context the JobContext for this commit - * @return a submitter or null - */ - protected Tasks.Submitter buildSubmitter( - JobContext context) { - if (getThreadCount(context) > 0) { - return new PoolSubmitter(context); - } else { - return null; - } - } - - /** - * Returns an {@link ExecutorService} for parallel tasks. The number of - * threads in the thread-pool is set by fs.s3a.committer.threads. - * If num-threads is 0, this will raise an exception. - * - * @param context the JobContext for this commit - * @param numThreads threads - * @return an {@link ExecutorService} for the number of threads - */ - private synchronized ExecutorService buildThreadPool( - JobContext context, int numThreads) { - Preconditions.checkArgument(numThreads > 0, - "Cannot create a thread pool with no threads"); - if (threadPool == null) { - LOG.debug("{}: creating thread pool of size {}", getRole(), numThreads); - threadPool = HadoopExecutors.newFixedThreadPool(numThreads, - new ThreadFactoryBuilder() - .setDaemon(true) - .setNameFormat(THREAD_PREFIX + context.getJobID() + "-%d") - .build()); - } - return threadPool; - } - /** * Get the thread count for this job's commit operations. * @param context the JobContext for this commit * @return a possibly zero thread count. */ - private int getThreadCount(final JobContext context) { + private int getJobCommitThreadCount(final JobContext context) { return context.getConfiguration().getInt( FS_S3A_COMMITTER_THREADS, DEFAULT_COMMITTER_THREADS); } /** - * Submit a runnable. - * This will demand-create the thread pool if needed. - *

    - * This is synchronized to ensure the thread pool is always valid when - * work is synchronized. See HADOOP-16798. + * Get the thread count for this task's commit operations. * @param context the JobContext for this commit - * @param task task to execute - * @return the future of the submitted task. - */ - private synchronized Future submitRunnable( - final JobContext context, - final Runnable task) { - return buildThreadPool(context, getThreadCount(context)).submit(task); - } - - /** - * The real task submitter, which hands off the work to - * the current thread pool. - */ - private final class PoolSubmitter implements Tasks.Submitter { - - private final JobContext context; - - private final int numThreads; - - private PoolSubmitter(final JobContext context) { - this.numThreads = getThreadCount(context); - Preconditions.checkArgument(numThreads > 0, - "Cannot create a thread pool with no threads"); - this.context = context; - } - - @Override - public Future submit(final Runnable task) { - return submitRunnable(context, task); - } - - } - - /** - * Destroy any thread pools; wait for that to finish, - * but don't overreact if it doesn't finish in time. - */ - protected void destroyThreadPool() { - ExecutorService pool; - // reset the thread pool in a sync block, then shut it down - // afterwards. This allows for other threads to create a - // new thread pool on demand. - synchronized(this) { - pool = this.threadPool; - threadPool = null; - } - if (pool != null) { - LOG.debug("Destroying thread pool"); - HadoopExecutors.shutdown(pool, LOG, - THREAD_POOL_SHUTDOWN_DELAY_SECONDS, TimeUnit.SECONDS); - } - } - - /** - * Get the thread pool for executing the single file commit/revert - * within the commit of all uploads of a single task. - * This is currently null; it is here to allow the Tasks class to - * provide the logic for execute/revert. - * @return null. always. - */ - protected final synchronized Tasks.Submitter singleThreadSubmitter() { - return null; - } - - /** - * Does this committer have a thread pool? - * @return true if a thread pool exists. + * @return a possibly zero thread count. */ - public synchronized boolean hasThreadPool() { - return threadPool != null; + private int getTaskCommitThreadCount(final JobContext context) { + return context.getConfiguration().getInt( + FS_S3A_COMMITTER_THREADS, + DEFAULT_COMMITTER_THREADS); } /** @@ -1164,24 +1189,23 @@ protected void deleteTaskAttemptPathQuietly(TaskAttemptContext context) { * Abort all pending uploads in the list. * This operation is used by the magic committer as part of its * rollback after a failure during task commit. - * @param context job context + * @param commitContext commit context * @param pending pending uploads * @param suppressExceptions should exceptions be suppressed * @throws IOException any exception raised */ - protected void abortPendingUploads(JobContext context, - List pending, - boolean suppressExceptions) + protected void abortPendingUploads( + final CommitContext commitContext, + final List pending, + final boolean suppressExceptions) throws IOException { if (pending == null || pending.isEmpty()) { LOG.info("{}: no pending commits to abort", getRole()); } else { try (DurationInfo d = new DurationInfo(LOG, - "Aborting %s uploads", pending.size()); - CommitOperations.CommitContext commitContext - = initiateCommitOperation()) { - Tasks.foreach(pending) - .executeWith(buildSubmitter(context)) + "Aborting %s uploads", pending.size())) { + TaskPool.foreach(pending) + .executeWith(commitContext.getOuterSubmitter()) .suppressExceptions(suppressExceptions) .run(commitContext::abortSingleCommit); } @@ -1190,27 +1214,24 @@ protected void abortPendingUploads(JobContext context, /** * Abort all pending uploads in the list. - * @param context job context + * @param commitContext commit context * @param pending pending uploads * @param suppressExceptions should exceptions be suppressed? * @param deleteRemoteFiles should remote files be deleted? * @throws IOException any exception raised */ protected void abortPendingUploads( - final JobContext context, + final CommitContext commitContext, final ActiveCommit pending, final boolean suppressExceptions, final boolean deleteRemoteFiles) throws IOException { - if (pending.isEmpty()) { LOG.info("{}: no pending commits to abort", getRole()); } else { try (DurationInfo d = new DurationInfo(LOG, - "Aborting %s uploads", pending.size()); - CommitOperations.CommitContext commitContext - = initiateCommitOperation()) { - Tasks.foreach(pending.getSourceFiles()) - .executeWith(buildSubmitter(context)) + "Aborting %s uploads", pending.size())) { + TaskPool.foreach(pending.getSourceFiles()) + .executeWith(commitContext.getOuterSubmitter()) .suppressExceptions(suppressExceptions) .run(path -> loadAndAbort(commitContext, @@ -1392,13 +1413,7 @@ public String toString() { */ protected final void updateCommonContext() { currentAuditContext().put(AuditConstants.PARAM_JOB_ID, uuid); - } - /** - * Remove JobID from the current thread's context. - */ - protected final void resetCommonContext() { - currentAuditContext().remove(AuditConstants.PARAM_JOB_ID); } protected AuditSpanSource getAuditSpanSource() { @@ -1424,6 +1439,77 @@ protected AuditSpan startOperation(String name, return getAuditSpanSource().createSpan(name, path1, path2); } + + /** + * Save a summary to the report dir if the config option + * is set. + * The report will be updated with the current active stage, + * and if {@code thrown} is non-null, it will be added to the + * diagnostics (and the job tagged as a failure). + * Static for testability. + * @param activeStage active stage + * @param context commit context. + * @param report summary file. + * @param thrown any exception indicting failure. + * @param quiet should exceptions be swallowed. + * @param overwrite should the existing file be overwritten + * @return the path of a file, if successfully saved + * @throws IOException if a failure occured and quiet==false + */ + private static Path maybeSaveSummary( + String activeStage, + CommitContext context, + SuccessData report, + Throwable thrown, + boolean quiet, + boolean overwrite) throws IOException { + Configuration conf = context.getConf(); + String reportDir = conf.getTrimmed(OPT_SUMMARY_REPORT_DIR, ""); + if (reportDir.isEmpty()) { + LOG.debug("No summary directory set in " + OPT_SUMMARY_REPORT_DIR); + return null; + } + LOG.debug("Summary directory set to {}", reportDir); + + Path reportDirPath = new Path(reportDir); + Path path = new Path(reportDirPath, + createJobSummaryFilename(context.getJobId())); + + if (thrown != null) { + report.recordJobFailure(thrown); + } + report.putDiagnostic(STAGE, activeStage); + // the store operations here is explicitly created for the FS where + // the reports go, which may not be the target FS of the job. + + final FileSystem fs = path.getFileSystem(conf); + try (ManifestStoreOperations operations = new ManifestStoreOperationsThroughFileSystem( + fs)) { + if (!overwrite) { + // check for file existence so there is no need to worry about + // precisely what exception is raised when overwrite=false and dest file + // exists + try { + FileStatus st = operations.getFileStatus(path); + // get here and the file exists + LOG.debug("Report already exists: {}", st); + return null; + } catch (FileNotFoundException ignored) { + } + } + report.save(fs, path, SuccessData.serializer()); + LOG.info("Job summary saved to {}", path); + return path; + } catch (IOException e) { + LOG.debug("Failed to save summary to {}", path, e); + if (quiet) { + return null; + } else { + throw e; + } + } + } + /** * State of the active commit operation. * @@ -1445,7 +1531,7 @@ protected AuditSpan startOperation(String name, * * */ - public static class ActiveCommit { + public static final class ActiveCommit { private static final AbstractS3ACommitter.ActiveCommit EMPTY = new ActiveCommit(null, new ArrayList<>()); @@ -1486,6 +1572,7 @@ public static class ActiveCommit { * @param sourceFS filesystem containing the list of pending files * @param sourceFiles .pendingset files to load and commit. */ + @SuppressWarnings("unchecked") public ActiveCommit( final FileSystem sourceFS, final List sourceFiles) { @@ -1496,13 +1583,14 @@ public ActiveCommit( /** * Create an active commit of the given pending files. * @param pendingFS source filesystem. - * @param statuses list of file status or subclass to use. + * @param statuses iterator of file status or subclass to use. * @return the commit + * @throws IOException if the iterator raises one. */ - public static ActiveCommit fromStatusList( + public static ActiveCommit fromStatusIterator( final FileSystem pendingFS, - final List statuses) { - return new ActiveCommit(pendingFS, statuses); + final RemoteIterator statuses) throws IOException { + return new ActiveCommit(pendingFS, toList(statuses)); } /** diff --git a/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/commit/CommitConstants.java b/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/commit/CommitConstants.java index bbc59f168f60d..b85ce276504ba 100644 --- a/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/commit/CommitConstants.java +++ b/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/commit/CommitConstants.java @@ -22,7 +22,6 @@ import org.apache.hadoop.classification.InterfaceStability; import static org.apache.hadoop.fs.s3a.Constants.XA_HEADER_PREFIX; -import static org.apache.hadoop.mapreduce.lib.output.PathOutputCommitterFactory.COMMITTER_FACTORY_SCHEME_PATTERN; /** * Constants for working with committers. @@ -57,6 +56,12 @@ private CommitConstants() { */ public static final String PENDINGSET_SUFFIX = ".pendingset"; + + /** + * Prefix to use for config options: {@value}. + */ + public static final String OPT_PREFIX = "fs.s3a.committer."; + /** * Flag to indicate whether support for the Magic committer is enabled * in the filesystem. @@ -121,9 +126,8 @@ private CommitConstants() { /** * Temp data which is not auto-committed: {@value}. - * Uses a different name from normal just to make clear it is different. */ - public static final String TEMP_DATA = "__temp-data"; + public static final String TEMP_DATA = TEMPORARY; /** @@ -144,7 +148,7 @@ private CommitConstants() { * Key to set for the S3A schema to use the specific committer. */ public static final String S3A_COMMITTER_FACTORY_KEY = String.format( - COMMITTER_FACTORY_SCHEME_PATTERN, "s3a"); + "mapreduce.outputcommitter.factory.scheme.s3a"); /** * S3 Committer factory: {@value}. @@ -222,13 +226,19 @@ private CommitConstants() { /** * Number of threads in committers for parallel operations on files * (upload, commit, abort, delete...): {@value}. + * Two thread pools this size are created, one for the outer + * task-level parallelism, and one for parallel execution + * within tasks (POSTs to commit individual uploads) + * If the value is negative, it is inverted and then multiplied + * by the number of cores in the CPU. */ public static final String FS_S3A_COMMITTER_THREADS = "fs.s3a.committer.threads"; + /** * Default value for {@link #FS_S3A_COMMITTER_THREADS}: {@value}. */ - public static final int DEFAULT_COMMITTER_THREADS = 8; + public static final int DEFAULT_COMMITTER_THREADS = 32; /** * Path in the cluster filesystem for temporary data: {@value}. @@ -330,4 +340,18 @@ private CommitConstants() { public static final String XA_MAGIC_MARKER = XA_HEADER_PREFIX + X_HEADER_MAGIC_MARKER; + /** + * Task Attempt ID query header: {@value}. + */ + public static final String PARAM_TASK_ATTEMPT_ID = "ta"; + + /** + * Directory for saving job summary reports. + * These are the _SUCCESS files, but are saved even on + * job failures. + * Value: {@value}. + */ + public static final String OPT_SUMMARY_REPORT_DIR = + OPT_PREFIX + "summary.report.directory"; + } diff --git a/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/commit/CommitterStatisticNames.java b/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/commit/CommitterStatisticNames.java new file mode 100644 index 0000000000000..20af292b3eb66 --- /dev/null +++ b/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/commit/CommitterStatisticNames.java @@ -0,0 +1,211 @@ +/* + * 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.fs.s3a.commit; + +import org.apache.hadoop.classification.InterfaceAudience; +import org.apache.hadoop.classification.InterfaceStability; +import org.apache.hadoop.fs.statistics.StoreStatisticNames; + +import static org.apache.hadoop.fs.statistics.StoreStatisticNames.OP_RENAME; + +/** + * Statistic names for committers. + * Please keep in sync with org.apache.hadoop.fs.s3a.Statistic + * so that S3A and manifest committers are in sync. + */ +@InterfaceAudience.Public +@InterfaceStability.Unstable +public final class CommitterStatisticNames { + + + /** Amount of data committed: {@value}. */ + public static final String COMMITTER_BYTES_COMMITTED_COUNT = + "committer_bytes_committed"; + + /** Duration Tracking of time to commit an entire job: {@value}. */ + public static final String COMMITTER_COMMIT_JOB = + "committer_commit_job"; + + /** Number of files committed: {@value}. */ + public static final String COMMITTER_FILES_COMMITTED_COUNT = + "committer_files_committed"; + /** "Count of successful tasks:: {@value}. */ + public static final String COMMITTER_TASKS_COMPLETED_COUNT = + "committer_tasks_completed"; + + /** Count of failed tasks: {@value}. */ + public static final String COMMITTER_TASKS_FAILED_COUNT = + "committer_tasks_failed"; + + /** Count of commits aborted: {@value}. */ + public static final String COMMITTER_COMMITS_ABORTED_COUNT = + "committer_commits_aborted"; + + /** Count of commits reverted: {@value}. */ + public static final String COMMITTER_COMMITS_REVERTED_COUNT = + "committer_commits_reverted"; + + /** Count of commits failed: {@value}. */ + public static final String COMMITTER_COMMITS_FAILED = + "committer_commits" + StoreStatisticNames.SUFFIX_FAILURES; + + /** + * The number of files in a task. This will be a MeanStatistic. + */ + public static final String COMMITTER_FILE_COUNT_MEAN = + "committer_task_file_count"; + + /** + * File Size. + */ + public static final String COMMITTER_FILE_SIZE_MEAN = + "committer_task_file_size"; + + /** + * What is a task attempt's directory count. + */ + public static final String COMMITTER_TASK_DIRECTORY_COUNT_MEAN = + "committer_task_directory_count"; + + /** + * What is the depth of a task attempt's directory tree. + */ + public static final String COMMITTER_TASK_DIRECTORY_DEPTH_MEAN = + "committer_task_directory_depth"; + + /** + * The number of files in a task. This will be a MeanStatistic. + */ + public static final String COMMITTER_TASK_FILE_COUNT_MEAN = + "committer_task_file_count"; + + /** + * The number of files in a task. This will be a MeanStatistic. + */ + public static final String COMMITTER_TASK_FILE_SIZE_MEAN = + "committer_task_file_size"; + + /** Directory creation {@value}. */ + public static final String OP_CREATE_DIRECTORIES = "op_create_directories"; + + /** Creating a single directory {@value}. */ + public static final String OP_CREATE_ONE_DIRECTORY = + "op_create_one_directory"; + + /** Directory scan {@value}. */ + public static final String OP_DIRECTORY_SCAN = "op_directory_scan"; + + /** + * Overall job commit {@value}. + */ + public static final String OP_STAGE_JOB_COMMIT = COMMITTER_COMMIT_JOB; + + /** {@value}. */ + public static final String OP_LOAD_ALL_MANIFESTS = "op_load_all_manifests"; + + /** + * Load a task manifest: {@value}. + */ + public static final String OP_LOAD_MANIFEST = "op_load_manifest"; + + /** Rename a file: {@value}. */ + public static final String OP_RENAME_FILE = OP_RENAME; + + /** + * Save a task manifest: {@value}. + */ + public static final String OP_SAVE_TASK_MANIFEST = + "op_task_stage_save_task_manifest"; + + /** + * Task abort: {@value}. + */ + public static final String OP_STAGE_TASK_ABORT_TASK + = "op_task_stage_abort_task"; + + /** + * Job abort: {@value}. + */ + public static final String OP_STAGE_JOB_ABORT = "op_job_stage_abort"; + + /** + * Job cleanup: {@value}. + */ + public static final String OP_STAGE_JOB_CLEANUP = "op_job_stage_cleanup"; + + /** + * Prepare Directories Stage: {@value}. + */ + public static final String OP_STAGE_JOB_CREATE_TARGET_DIRS = + "op_job_stage_create_target_dirs"; + + /** + * Load Manifest Stage: {@value}. + */ + public static final String OP_STAGE_JOB_LOAD_MANIFESTS = + "op_job_stage_load_manifests"; + + /** + * Rename files stage duration: {@value}. + */ + public static final String OP_STAGE_JOB_RENAME_FILES = + "op_job_stage_rename_files"; + + + /** + * Job Setup Stage: {@value}. + */ + public static final String OP_STAGE_JOB_SETUP = "op_job_stage_setup"; + + /** + * Job saving _SUCCESS marker Stage: {@value}. + */ + public static final String OP_STAGE_JOB_SAVE_SUCCESS = + "op_job_stage_save_success_marker"; + + /** + * Output Validation (within job commit) Stage: {@value}. + */ + public static final String OP_STAGE_JOB_VALIDATE_OUTPUT = + + "op_job_stage_optional_validate_output"; + /** + * Task saving manifest file Stage: {@value}. + */ + public static final String OP_STAGE_TASK_SAVE_MANIFEST = + "op_task_stage_save_manifest"; + + /** + * Task Setup Stage: {@value}. + */ + public static final String OP_STAGE_TASK_SETUP = "op_task_stage_setup"; + + + /** + * Task Commit Stage: {@value}. + */ + public static final String OP_STAGE_TASK_COMMIT = "op_stage_task_commit"; + + /** Task Scan directory Stage: {@value}. */ + public static final String OP_STAGE_TASK_SCAN_DIRECTORY + = "op_stage_task_scan_directory"; + + private CommitterStatisticNames() { + } +} diff --git a/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/commit/InternalCommitterConstants.java b/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/commit/InternalCommitterConstants.java index fcafdd1ed1280..b2d4bfaeeabab 100644 --- a/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/commit/InternalCommitterConstants.java +++ b/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/commit/InternalCommitterConstants.java @@ -35,6 +35,12 @@ @InterfaceStability.Unstable public final class InternalCommitterConstants { + /** + * How long threads in the thread pool stay alive when + * idle. Value in seconds: {@value}. + */ + public static final long THREAD_KEEP_ALIVE_TIME = 60L; + private InternalCommitterConstants() { } diff --git a/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/commit/MagicCommitIntegration.java b/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/commit/MagicCommitIntegration.java index 41d36b2a8d7a0..e6524c91961dc 100644 --- a/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/commit/MagicCommitIntegration.java +++ b/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/commit/MagicCommitIntegration.java @@ -28,6 +28,7 @@ import org.apache.hadoop.fs.s3a.Statistic; import org.apache.hadoop.fs.s3a.commit.magic.MagicCommitTracker; import org.apache.hadoop.fs.s3a.impl.AbstractStoreOperation; +import org.apache.hadoop.fs.s3a.statistics.PutTrackerStatistics; import static org.apache.hadoop.fs.s3a.commit.MagicCommitPaths.*; @@ -38,8 +39,9 @@ * in this case: *
      *
    1. {@link #isMagicCommitPath(Path)} will always return false.
    2. - *
    3. {@link #createTracker(Path, String)} will always return an instance - * of {@link PutTracker}.
    4. + *
    5. {@link #isUnderMagicPath(Path)} will always return false.
    6. + *
    7. {@link #createTracker(Path, String, PutTrackerStatistics)} will always + * return an instance of {@link PutTracker}.
    8. *
    * *

    Important

    : must not directly or indirectly import a class which @@ -88,9 +90,11 @@ public String keyOfFinalDestination(List elements, String key) { * for the commit tracker. * @param path path of nominal write * @param key key of path of nominal write + * @param trackerStatistics tracker statistics * @return the tracker for this operation. */ - public PutTracker createTracker(Path path, String key) { + public PutTracker createTracker(Path path, String key, + PutTrackerStatistics trackerStatistics) { final List elements = splitPathToElements(path); PutTracker tracker; @@ -106,7 +110,8 @@ public PutTracker createTracker(Path path, String key) { key, destKey, pendingsetPath, - owner.getWriteOperationHelper()); + owner.getWriteOperationHelper(), + trackerStatistics); LOG.debug("Created {}", tracker); } else { LOG.warn("File being created has a \"magic\" path, but the filesystem" @@ -184,4 +189,13 @@ private boolean isCommitMetadataFile(List elements) { || last.endsWith(CommitConstants.PENDINGSET_SUFFIX); } + /** + * Is this path in/under a magic path...regardless of file type. + * This is used to optimize create() operations. + * @param path path to check + * @return true if the path is one a magic file write expects. + */ + public boolean isUnderMagicPath(Path path) { + return magicCommitEnabled && isMagicPath(splitPathToElements(path)); + } } diff --git a/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/commit/Tasks.java b/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/commit/Tasks.java deleted file mode 100644 index c318e86605e0c..0000000000000 --- a/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/commit/Tasks.java +++ /dev/null @@ -1,423 +0,0 @@ -/* - * 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.fs.s3a.commit; - -import java.util.ArrayList; -import java.util.Arrays; -import java.util.Collection; -import java.util.Iterator; -import java.util.List; -import java.util.Queue; -import java.util.concurrent.ConcurrentLinkedQueue; -import java.util.concurrent.Future; -import java.util.concurrent.atomic.AtomicBoolean; - -import org.slf4j.Logger; -import org.slf4j.LoggerFactory; - -/** - * Utility class for parallel execution, takes closures for the various - * actions. - * There is no retry logic: it is expected to be handled by the closures. - */ -public final class Tasks { - private static final Logger LOG = LoggerFactory.getLogger(Tasks.class); - - private Tasks() { - } - - /** - * Callback invoked to process an item. - * @param item type being processed - * @param exception class which may be raised - */ - @FunctionalInterface - public interface Task { - void run(I item) throws E; - } - - /** - * Callback invoked on a failure. - * @param item type being processed - * @param exception class which may be raised - */ - @FunctionalInterface - public interface FailureTask { - - /** - * process a failure. - * @param item item the task is processing - * @param exception the exception which was raised. - * @throws E Exception of type E - */ - void run(I item, Exception exception) throws E; - } - - /** - * Builder for task execution. - * @param item type - */ - public static class Builder { - private final Iterable items; - private Submitter service = null; - private FailureTask onFailure = null; - private boolean stopOnFailure = false; - private boolean suppressExceptions = false; - private Task revertTask = null; - private boolean stopRevertsOnFailure = false; - private Task abortTask = null; - private boolean stopAbortsOnFailure = false; - - /** - * Create the builder. - * @param items items to process - */ - Builder(Iterable items) { - this.items = items; - } - - /** - * Declare executor service: if null, the tasks are executed in a single - * thread. - * @param submitter service to schedule tasks with. - * @return this builder. - */ - public Builder executeWith(Submitter submitter) { - this.service = submitter; - return this; - } - - public Builder onFailure(FailureTask task) { - this.onFailure = task; - return this; - } - - public Builder stopOnFailure() { - this.stopOnFailure = true; - return this; - } - - public Builder suppressExceptions() { - return suppressExceptions(true); - } - - public Builder suppressExceptions(boolean suppress) { - this.suppressExceptions = suppress; - return this; - } - - public Builder revertWith(Task task) { - this.revertTask = task; - return this; - } - - public Builder stopRevertsOnFailure() { - this.stopRevertsOnFailure = true; - return this; - } - - public Builder abortWith(Task task) { - this.abortTask = task; - return this; - } - - public Builder stopAbortsOnFailure() { - this.stopAbortsOnFailure = true; - return this; - } - - public boolean run(Task task) throws E { - if (service != null) { - return runParallel(task); - } else { - return runSingleThreaded(task); - } - } - - private boolean runSingleThreaded(Task task) - throws E { - List succeeded = new ArrayList<>(); - List exceptions = new ArrayList<>(); - - Iterator iterator = items.iterator(); - boolean threw = true; - try { - while (iterator.hasNext()) { - I item = iterator.next(); - try { - task.run(item); - succeeded.add(item); - - } catch (Exception e) { - exceptions.add(e); - - if (onFailure != null) { - try { - onFailure.run(item, e); - } catch (Exception failException) { - LOG.error("Failed to clean up on failure", e); - // keep going - } - } - - if (stopOnFailure) { - break; - } - } - } - - threw = false; - - } finally { - // threw handles exceptions that were *not* caught by the catch block, - // and exceptions that were caught and possibly handled by onFailure - // are kept in exceptions. - if (threw || !exceptions.isEmpty()) { - if (revertTask != null) { - boolean failed = false; - for (I item : succeeded) { - try { - revertTask.run(item); - } catch (Exception e) { - LOG.error("Failed to revert task", e); - failed = true; - // keep going - } - if (stopRevertsOnFailure && failed) { - break; - } - } - } - - if (abortTask != null) { - boolean failed = false; - while (iterator.hasNext()) { - try { - abortTask.run(iterator.next()); - } catch (Exception e) { - failed = true; - LOG.error("Failed to abort task", e); - // keep going - } - if (stopAbortsOnFailure && failed) { - break; - } - } - } - } - } - - if (!suppressExceptions && !exceptions.isEmpty()) { - Tasks.throwOne(exceptions); - } - - return !threw && exceptions.isEmpty(); - } - - private boolean runParallel(final Task task) - throws E { - final Queue succeeded = new ConcurrentLinkedQueue<>(); - final Queue exceptions = new ConcurrentLinkedQueue<>(); - final AtomicBoolean taskFailed = new AtomicBoolean(false); - final AtomicBoolean abortFailed = new AtomicBoolean(false); - final AtomicBoolean revertFailed = new AtomicBoolean(false); - - List> futures = new ArrayList<>(); - - for (final I item : items) { - // submit a task for each item that will either run or abort the task - futures.add(service.submit(new Runnable() { - @Override - public void run() { - if (!(stopOnFailure && taskFailed.get())) { - // run the task - boolean threw = true; - try { - LOG.debug("Executing task"); - task.run(item); - succeeded.add(item); - LOG.debug("Task succeeded"); - - threw = false; - - } catch (Exception e) { - taskFailed.set(true); - exceptions.add(e); - LOG.info("Task failed", e); - - if (onFailure != null) { - try { - onFailure.run(item, e); - } catch (Exception failException) { - LOG.error("Failed to clean up on failure", e); - // swallow the exception - } - } - } finally { - if (threw) { - taskFailed.set(true); - } - } - - } else if (abortTask != null) { - // abort the task instead of running it - if (stopAbortsOnFailure && abortFailed.get()) { - return; - } - - boolean failed = true; - try { - LOG.info("Aborting task"); - abortTask.run(item); - failed = false; - } catch (Exception e) { - LOG.error("Failed to abort task", e); - // swallow the exception - } finally { - if (failed) { - abortFailed.set(true); - } - } - } - } - })); - } - - // let the above tasks complete (or abort) - waitFor(futures); - int futureCount = futures.size(); - futures.clear(); - - if (taskFailed.get() && revertTask != null) { - // at least one task failed, revert any that succeeded - LOG.info("Reverting all {} succeeded tasks from {} futures", - succeeded.size(), futureCount); - for (final I item : succeeded) { - futures.add(service.submit(() -> { - if (stopRevertsOnFailure && revertFailed.get()) { - return; - } - - boolean failed = true; - try { - revertTask.run(item); - failed = false; - } catch (Exception e) { - LOG.error("Failed to revert task", e); - // swallow the exception - } finally { - if (failed) { - revertFailed.set(true); - } - } - })); - } - - // let the revert tasks complete - waitFor(futures); - } - - if (!suppressExceptions && !exceptions.isEmpty()) { - Tasks.throwOne(exceptions); - } - - return !taskFailed.get(); - } - } - - /** - * Wait for all the futures to complete; there's a small sleep between - * each iteration; enough to yield the CPU. - * @param futures futures. - */ - private static void waitFor(Collection> futures) { - int size = futures.size(); - LOG.debug("Waiting for {} tasks to complete", size); - int oldNumFinished = 0; - while (true) { - int numFinished = (int) futures.stream().filter(Future::isDone).count(); - - if (oldNumFinished != numFinished) { - LOG.debug("Finished count -> {}/{}", numFinished, size); - oldNumFinished = numFinished; - } - - if (numFinished == size) { - // all of the futures are done, stop looping - break; - } else { - try { - Thread.sleep(10); - } catch (InterruptedException e) { - futures.forEach(future -> future.cancel(true)); - Thread.currentThread().interrupt(); - break; - } - } - } - } - - public static Builder foreach(Iterable items) { - return new Builder<>(items); - } - - public static Builder foreach(I[] items) { - return new Builder<>(Arrays.asList(items)); - } - - @SuppressWarnings("unchecked") - private static void throwOne( - Collection exceptions) - throws E { - Iterator iter = exceptions.iterator(); - Exception e = iter.next(); - Class exceptionClass = e.getClass(); - - while (iter.hasNext()) { - Exception other = iter.next(); - if (!exceptionClass.isInstance(other)) { - e.addSuppressed(other); - } - } - - Tasks.castAndThrow(e); - } - - @SuppressWarnings("unchecked") - private static void castAndThrow(Exception e) throws E { - if (e instanceof RuntimeException) { - throw (RuntimeException) e; - } - throw (E) e; - } - - /** - * Interface to whatever lets us submit tasks. - */ - public interface Submitter { - - /** - * Submit work. - * @param task task to execute - * @return the future of the submitted task. - */ - Future submit(Runnable task); - } - -} diff --git a/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/commit/files/PendingSet.java b/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/commit/files/PendingSet.java index 318896e236137..108e6d45bc621 100644 --- a/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/commit/files/PendingSet.java +++ b/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/commit/files/PendingSet.java @@ -38,6 +38,7 @@ import org.apache.hadoop.fs.FileSystem; import org.apache.hadoop.fs.Path; import org.apache.hadoop.fs.s3a.commit.ValidationFailure; +import org.apache.hadoop.fs.statistics.IOStatistics; import org.apache.hadoop.fs.statistics.IOStatisticsSnapshot; import org.apache.hadoop.fs.statistics.IOStatisticsSource; import org.apache.hadoop.util.JsonSerialization; @@ -63,8 +64,7 @@ @SuppressWarnings("unused") @InterfaceAudience.Private @InterfaceStability.Unstable -public class PendingSet extends PersistentCommitData - implements IOStatisticsSource { +public class PendingSet extends PersistentCommitData { private static final Logger LOG = LoggerFactory.getLogger(PendingSet.class); /** @@ -112,38 +112,13 @@ public PendingSet(int size) { } /** - * Get a JSON serializer for this class. + * Get a shared JSON serializer for this class. * @return a serializer. */ public static JsonSerialization serializer() { - return new JsonSerialization<>(PendingSet.class, false, true); + return new JsonSerialization<>(PendingSet.class, false, false); } - /** - * Load an instance from a file, then validate it. - * @param fs filesystem - * @param path path - * @return the loaded instance - * @throws IOException IO failure - * @throws ValidationFailure if the data is invalid - */ - public static PendingSet load(FileSystem fs, Path path) - throws IOException { - return load(fs, path, null); - } - - /** - * Load an instance from a file, then validate it. - * @param fs filesystem - * @param status status of file to load - * @return the loaded instance - * @throws IOException IO failure - * @throws ValidationFailure if the data is invalid - */ - public static PendingSet load(FileSystem fs, FileStatus status) - throws IOException { - return load(fs, status.getPath(), status); - } /** * Load an instance from a file, then validate it. @@ -211,8 +186,8 @@ public void validate() throws ValidationFailure { } @Override - public byte[] toBytes() throws IOException { - return serializer().toBytes(this); + public byte[] toBytes(JsonSerialization serializer) throws IOException { + return serializer.toBytes(this); } /** @@ -224,9 +199,10 @@ public int size() { } @Override - public void save(FileSystem fs, Path path, boolean overwrite) - throws IOException { - serializer().save(fs, path, this, overwrite); + public IOStatistics save(final FileSystem fs, + final Path path, + final JsonSerialization serializer) throws IOException { + return saveFile(fs, path, this, serializer, true); } /** @return the version marker. */ diff --git a/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/commit/files/PersistentCommitData.java b/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/commit/files/PersistentCommitData.java index dba44b9a011d9..be2de6cf89cfd 100644 --- a/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/commit/files/PersistentCommitData.java +++ b/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/commit/files/PersistentCommitData.java @@ -21,19 +21,33 @@ import java.io.IOException; import java.io.Serializable; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + import org.apache.hadoop.classification.InterfaceAudience; import org.apache.hadoop.classification.InterfaceStability; +import org.apache.hadoop.fs.FSDataOutputStream; +import org.apache.hadoop.fs.FSDataOutputStreamBuilder; +import org.apache.hadoop.fs.FileStatus; import org.apache.hadoop.fs.FileSystem; import org.apache.hadoop.fs.Path; import org.apache.hadoop.fs.s3a.commit.ValidationFailure; +import org.apache.hadoop.fs.statistics.IOStatistics; +import org.apache.hadoop.fs.statistics.IOStatisticsSource; +import org.apache.hadoop.util.JsonSerialization; + +import static org.apache.hadoop.fs.s3a.Constants.FS_S3A_CREATE_PERFORMANCE; /** * Class for single/multiple commit data structures. + * The mapreduce hierarchy {@code AbstractManifestData} is a fork + * of this; the Success data JSON format must stay compatible */ -@SuppressWarnings("serial") @InterfaceAudience.Private @InterfaceStability.Unstable -public abstract class PersistentCommitData implements Serializable { +public abstract class PersistentCommitData + implements Serializable, IOStatisticsSource { + private static final Logger LOG = LoggerFactory.getLogger(PersistentCommitData.class); /** * Supported version value: {@value}. @@ -52,18 +66,109 @@ public abstract class PersistentCommitData implements Serializable { * Serialize to JSON and then to a byte array, after performing a * preflight validation of the data to be saved. * @return the data in a persistable form. + * @param serializer serializer to use * @throws IOException serialization problem or validation failure. */ - public abstract byte[] toBytes() throws IOException; + public abstract byte[] toBytes(JsonSerialization serializer) throws IOException; /** * Save to a hadoop filesystem. + * The destination file is overwritten, and on s3a stores the + * performance flag is set to turn off all existence checks and + * parent dir cleanup. + * The assumption here is: the job knows what it is doing. + * * @param fs filesystem * @param path path - * @param overwrite should any existing file be overwritten + * @param serializer serializer to use + * @return IOStats from the output stream. + * * @throws IOException IO exception */ - public abstract void save(FileSystem fs, Path path, boolean overwrite) + public abstract IOStatistics save(FileSystem fs, Path path, JsonSerialization serializer) throws IOException; + /** + * Load an instance from a status, then validate it. + * This uses the openFile() API, which S3A supports for + * faster load and declaring sequential access, always + * @param type of persistent format + * @param fs filesystem + * @param status status of file to load + * @param serializer serializer to use + * @return the loaded instance + * @throws IOException IO failure + * @throws ValidationFailure if the data is invalid + */ + public static T load(FileSystem fs, + FileStatus status, + JsonSerialization serializer) + throws IOException { + Path path = status.getPath(); + LOG.debug("Reading commit data from file {}", path); + T result = serializer.load(fs, path, status); + result.validate(); + return result; + } + + /** + * Save to a file. + * This uses the createFile() API, which S3A supports for + * faster load and declaring sequential access, always + * + * @param type of persistent format + * @param fs filesystem + * @param path path to save to + * @param instance data to save + * @param serializer serializer to use + * @param performance skip all safety check on the write + * + * @return any IOStatistics from the output stream, or null + * + * @throws IOException IO failure + */ + public static IOStatistics saveFile( + final FileSystem fs, + final Path path, + final T instance, + final JsonSerialization serializer, + final boolean performance) + throws IOException { + + FSDataOutputStreamBuilder builder = fs.createFile(path) + .create() + .recursive() + .overwrite(true); + // switch to performance mode + builder.opt(FS_S3A_CREATE_PERFORMANCE, performance); + return saveToStream(path, instance, builder, serializer); + } + + /** + * Save to a file. + * This uses the createFile() API, which S3A supports for + * faster load and declaring sequential access, always + * @param type of persistent format + * @param path path to save to (used for logging) + * @param instance data to save + * @param builder builder already prepared for the write + * @param serializer serializer to use + * @return any IOStatistics from the output stream, or null + * @throws IOException IO failure + */ + public static IOStatistics saveToStream( + final Path path, + final T instance, + final FSDataOutputStreamBuilder builder, + final JsonSerialization serializer) throws IOException { + LOG.debug("saving commit data to file {}", path); + FSDataOutputStream dataOutputStream = builder.build(); + try { + dataOutputStream.write(serializer.toBytes(instance)); + } finally { + dataOutputStream.close(); + } + return dataOutputStream.getIOStatistics(); + } + } diff --git a/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/commit/files/SinglePendingCommit.java b/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/commit/files/SinglePendingCommit.java index b53ef75d823df..77c3fed11fb24 100644 --- a/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/commit/files/SinglePendingCommit.java +++ b/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/commit/files/SinglePendingCommit.java @@ -34,8 +34,6 @@ import com.amazonaws.services.s3.model.PartETag; import com.fasterxml.jackson.annotation.JsonProperty; -import org.apache.hadoop.util.Preconditions; - import org.apache.commons.lang3.StringUtils; import org.apache.hadoop.classification.InterfaceAudience; import org.apache.hadoop.classification.InterfaceStability; @@ -43,9 +41,11 @@ import org.apache.hadoop.fs.FileSystem; import org.apache.hadoop.fs.Path; import org.apache.hadoop.fs.s3a.commit.ValidationFailure; +import org.apache.hadoop.fs.statistics.IOStatistics; import org.apache.hadoop.fs.statistics.IOStatisticsSnapshot; import org.apache.hadoop.fs.statistics.IOStatisticsSource; import org.apache.hadoop.util.JsonSerialization; +import org.apache.hadoop.util.Preconditions; import static org.apache.hadoop.fs.s3a.commit.CommitUtils.validateCollectionClass; import static org.apache.hadoop.fs.s3a.commit.ValidationFailure.verify; @@ -69,8 +69,8 @@ @SuppressWarnings("unused") @InterfaceAudience.Private @InterfaceStability.Unstable -public class SinglePendingCommit extends PersistentCommitData - implements Iterable, IOStatisticsSource { +public class SinglePendingCommit extends PersistentCommitData + implements Iterable { /** * Serialization ID: {@value}. @@ -141,26 +141,32 @@ public SinglePendingCommit() { * @return a serializer. */ public static JsonSerialization serializer() { - return new JsonSerialization<>(SinglePendingCommit.class, false, true); + return new JsonSerialization<>(SinglePendingCommit.class, false, false); } /** * Load an instance from a file, then validate it. * @param fs filesystem * @param path path + * @param status nullable status of file to load + * @param serDeser serializer; if null use the shared static one. * @return the loaded instance * @throws IOException IO failure * @throws ValidationFailure if the data is invalid */ - public static SinglePendingCommit load(FileSystem fs, Path path) + public static SinglePendingCommit load(FileSystem fs, + Path path, + FileStatus status, + JsonSerialization serDeser) throws IOException { - return load(fs, path, null); + return load(fs, path, serDeser, null); } /** * Load an instance from a file, then validate it. * @param fs filesystem * @param path path + * @param serDeser deserializer * @param status status of file to load or null * @return the loaded instance * @throws IOException IO failure @@ -168,9 +174,12 @@ public static SinglePendingCommit load(FileSystem fs, Path path) */ public static SinglePendingCommit load(FileSystem fs, Path path, + JsonSerialization serDeser, @Nullable FileStatus status) throws IOException { - SinglePendingCommit instance = serializer().load(fs, path, status); + JsonSerialization jsonSerialization = + serDeser != null ? serDeser : serializer(); + SinglePendingCommit instance = jsonSerialization.load(fs, path, status); instance.filename = path.toString(); instance.validate(); return instance; @@ -264,15 +273,16 @@ public String toString() { } @Override - public byte[] toBytes() throws IOException { + public byte[] toBytes(JsonSerialization serializer) throws IOException { validate(); - return serializer().toBytes(this); + return serializer.toBytes(this); } @Override - public void save(FileSystem fs, Path path, boolean overwrite) - throws IOException { - serializer().save(fs, path, this, overwrite); + public IOStatistics save(final FileSystem fs, + final Path path, + final JsonSerialization serializer) throws IOException { + return saveFile(fs, path, this, serializer, true); } /** diff --git a/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/commit/files/SuccessData.java b/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/commit/files/SuccessData.java index 00f196a0b406e..0e0a03f587081 100644 --- a/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/commit/files/SuccessData.java +++ b/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/commit/files/SuccessData.java @@ -30,13 +30,14 @@ import org.slf4j.LoggerFactory; import org.apache.commons.lang3.StringUtils; +import org.apache.commons.lang3.exception.ExceptionUtils; import org.apache.hadoop.classification.InterfaceAudience; import org.apache.hadoop.classification.InterfaceStability; import org.apache.hadoop.fs.FileSystem; import org.apache.hadoop.fs.Path; import org.apache.hadoop.fs.s3a.commit.ValidationFailure; +import org.apache.hadoop.fs.statistics.IOStatistics; import org.apache.hadoop.fs.statistics.IOStatisticsSnapshot; -import org.apache.hadoop.fs.statistics.IOStatisticsSource; import org.apache.hadoop.util.JsonSerialization; /** @@ -51,10 +52,24 @@ *
  • Not loadable? Something else.
  • * * - * This is an unstable structure intended for diagnostics and testing. - * Applications reading this data should use/check the {@link #name} field - * to differentiate from any other JSON-based manifest and to identify - * changes in the output format. + * This should be considered public, and MUST stay compatible + * at the JSON format level with that of + * {@code org.apache.hadoop.mapreduce.lib.output.committer.manifest.files.ManifestSuccessData} + *

    + * The JSON format SHOULD be considered public and evolving + * with compatibility across versions. + *

    + * All the Java serialization data is different and may change + * across versions with no stability guarantees other than + * "manifest summaries MAY be serialized between processes with + * the exact same version of this binary on their classpaths." + * That is sufficient for testing in Spark. + *

    + * To aid with Java serialization, the maps and lists are + * exclusively those which serialize well. + * IOStatisticsSnapshot has a lot of complexity in marshalling + * there; this class doesn't worry about concurrent access + * so is simpler. * * Note: to deal with scale issues, the S3A committers do not include any * more than the number of objects listed in @@ -65,8 +80,7 @@ @SuppressWarnings("unused") @InterfaceAudience.Private @InterfaceStability.Unstable -public class SuccessData extends PersistentCommitData - implements IOStatisticsSource { +public class SuccessData extends PersistentCommitData { private static final Logger LOG = LoggerFactory.getLogger(SuccessData.class); @@ -80,7 +94,7 @@ public class SuccessData extends PersistentCommitData /** * Serialization ID: {@value}. */ - private static final long serialVersionUID = 507133045258460083L + VERSION; + private static final long serialVersionUID = 507133045258460084L + VERSION; /** * Name to include in persisted data, so as to differentiate from @@ -92,7 +106,14 @@ public class SuccessData extends PersistentCommitData /** * Name of file; includes version marker. */ - private String name; + private String name = NAME; + + /** + * Did this succeed? + * It is implicitly true in a _SUCCESS file, but if the file + * is also saved to a log dir, then it depends on the outcome + */ + private boolean success = true; /** Timestamp of creation. */ private long timestamp; @@ -142,7 +163,17 @@ public class SuccessData extends PersistentCommitData * IOStatistics. */ @JsonProperty("iostatistics") - private IOStatisticsSnapshot iostats = new IOStatisticsSnapshot(); + private IOStatisticsSnapshot iostatistics = new IOStatisticsSnapshot(); + + /** + * State (committed, aborted). + */ + private String state; + + /** + * Stage: last stage executed. + */ + private String stage; @Override public void validate() throws ValidationFailure { @@ -153,16 +184,17 @@ public void validate() throws ValidationFailure { } @Override - public byte[] toBytes() throws IOException { - return serializer().toBytes(this); + public byte[] toBytes(JsonSerialization serializer) throws IOException { + return serializer.toBytes(this); } @Override - public void save(FileSystem fs, Path path, boolean overwrite) - throws IOException { + public IOStatistics save(final FileSystem fs, + final Path path, + final JsonSerialization serializer) throws IOException { // always set the name field before being saved. name = NAME; - serializer().save(fs, path, this, overwrite); + return saveFile(fs, path, this, serializer, true); } @Override @@ -250,8 +282,8 @@ public static SuccessData load(FileSystem fs, Path path) * Get a JSON serializer for this class. * @return a serializer. */ - private static JsonSerialization serializer() { - return new JsonSerialization<>(SuccessData.class, false, true); + public static JsonSerialization serializer() { + return new JsonSerialization<>(SuccessData.class, false, false); } public String getName() { @@ -371,10 +403,59 @@ public void setJobIdSource(final String jobIdSource) { @Override public IOStatisticsSnapshot getIOStatistics() { - return iostats; + return iostatistics; } public void setIOStatistics(final IOStatisticsSnapshot ioStatistics) { - this.iostats = ioStatistics; + this.iostatistics = ioStatistics; + } + + /** + * Set the success flag. + * @param success did the job succeed? + */ + public void setSuccess(boolean success) { + this.success = success; + } + + /** + * Get the success flag. + * @return did the job succeed? + */ + public boolean getSuccess() { + return success; + } + + public String getState() { + return state; + } + + public void setState(String state) { + this.state = state; + } + + public String getStage() { + return stage; + } + + /** + * Add a diagnostics entry. + * @param key name + * @param value value + */ + public void putDiagnostic(String key, String value) { + diagnostics.put(key, value); + } + + /** + * Note a failure by setting success flag to false, + * then add the exception to the diagnostics. + * @param thrown throwable + */ + public void recordJobFailure(Throwable thrown) { + setSuccess(false); + String stacktrace = ExceptionUtils.getStackTrace(thrown); + diagnostics.put("exception", thrown.toString()); + diagnostics.put("stacktrace", stacktrace); } } diff --git a/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/commit/files/package-info.java b/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/commit/files/package-info.java index 0d574948e91a3..36eedba83b854 100644 --- a/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/commit/files/package-info.java +++ b/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/commit/files/package-info.java @@ -29,12 +29,11 @@ *

  • The summary information saved in the {@code _SUCCESS} file.
  • * * - * There are no guarantees of stability between versions; these are internal - * structures. * * The {@link org.apache.hadoop.fs.s3a.commit.files.SuccessData} file is - * the one visible to callers after a job completes; it is an unstable - * manifest intended for testing only. + * the one visible to callers after a job completes; it is compatible with + * the manifest committer format persisted in + * {@code org.apache.hadoop.mapreduce.lib.output.committer.manifest.files.ManifestSuccessData} * */ @InterfaceAudience.Private diff --git a/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/commit/impl/AuditContextUpdater.java b/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/commit/impl/AuditContextUpdater.java new file mode 100644 index 0000000000000..20024ba601d45 --- /dev/null +++ b/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/commit/impl/AuditContextUpdater.java @@ -0,0 +1,90 @@ +/* + * 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.fs.s3a.commit.impl; + +import org.apache.hadoop.fs.audit.AuditConstants; +import org.apache.hadoop.fs.audit.CommonAuditContext; +import org.apache.hadoop.fs.s3a.commit.CommitConstants; +import org.apache.hadoop.mapreduce.JobContext; +import org.apache.hadoop.mapreduce.TaskAttemptContext; +import org.apache.hadoop.mapreduce.TaskAttemptID; + +import static org.apache.hadoop.fs.audit.CommonAuditContext.currentAuditContext; + +/** + * Class to track/update context information to set + * in threads. + */ +public final class AuditContextUpdater { + + /** + * Job ID. + */ + private final String jobId; + + /** + * Task attempt ID for auditing. + */ + private final String taskAttemptId; + + /** + * Construct. Stores job information + * to attach to thread contexts. + * @param jobContext job/task context. + */ + public AuditContextUpdater(final JobContext jobContext) { + this.jobId = jobContext.getJobID().toString(); + + if (jobContext instanceof TaskAttemptContext) { + // it's a task, extract info for auditing + final TaskAttemptID tid = ((TaskAttemptContext) jobContext).getTaskAttemptID(); + this.taskAttemptId = tid.toString(); + } else { + this.taskAttemptId = null; + } + } + + public AuditContextUpdater(String jobId) { + this.jobId = jobId; + this.taskAttemptId = null; + } + + /** + * Add job/task info to current audit context. + */ + public void updateCurrentAuditContext() { + final CommonAuditContext auditCtx = currentAuditContext(); + auditCtx.put(AuditConstants.PARAM_JOB_ID, jobId); + if (taskAttemptId != null) { + auditCtx.put(AuditConstants.PARAM_TASK_ATTEMPT_ID, taskAttemptId); + } else { + currentAuditContext().remove(CommitConstants.PARAM_TASK_ATTEMPT_ID); + } + + } + + /** + * Remove job/task info from the current audit context. + */ + public void resetCurrentAuditContext() { + currentAuditContext().remove(AuditConstants.PARAM_JOB_ID); + currentAuditContext().remove(CommitConstants.PARAM_TASK_ATTEMPT_ID); + } + +} diff --git a/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/commit/impl/CommitContext.java b/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/commit/impl/CommitContext.java new file mode 100644 index 0000000000000..8bff165f2e8f5 --- /dev/null +++ b/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/commit/impl/CommitContext.java @@ -0,0 +1,401 @@ +/* + * 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.fs.s3a.commit.impl; + +import java.io.Closeable; +import java.io.FileNotFoundException; +import java.io.IOException; +import java.util.concurrent.ExecutorService; +import java.util.concurrent.Future; +import java.util.concurrent.LinkedBlockingQueue; +import java.util.concurrent.ThreadFactory; +import java.util.concurrent.ThreadPoolExecutor; +import java.util.concurrent.TimeUnit; + +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + +import org.apache.hadoop.conf.Configuration; +import org.apache.hadoop.fs.impl.WeakReferenceThreadMap; +import org.apache.hadoop.fs.s3a.commit.InternalCommitterConstants; +import org.apache.hadoop.fs.s3a.commit.files.PendingSet; +import org.apache.hadoop.fs.s3a.commit.files.SinglePendingCommit; +import org.apache.hadoop.io.IOUtils; +import org.apache.hadoop.mapreduce.JobContext; +import org.apache.hadoop.thirdparty.com.google.common.util.concurrent.ThreadFactoryBuilder; +import org.apache.hadoop.util.JsonSerialization; +import org.apache.hadoop.util.Preconditions; +import org.apache.hadoop.util.concurrent.HadoopExecutors; +import org.apache.hadoop.util.concurrent.HadoopThreadPoolExecutor; +import org.apache.hadoop.util.functional.TaskPool; + +import static org.apache.hadoop.fs.s3a.Constants.THREAD_POOL_SHUTDOWN_DELAY_SECONDS; +import static org.apache.hadoop.fs.s3a.commit.AbstractS3ACommitter.THREAD_PREFIX; +import static org.apache.hadoop.fs.s3a.commit.InternalCommitterConstants.THREAD_KEEP_ALIVE_TIME; + +/** + * Commit context. + * + * It is used to manage the final commit sequence where files become + * visible. + * + * Once the commit operation has completed, it must be closed. + * It MUST NOT be reused. + * + * Audit integration: job and task attributes are added to the thread local context + * on create, removed on close(). + * + * JSON Serializers are created on demand, on a per thread basis. + * A {@link WeakReferenceThreadMap} is used here; a GC may lose the + * references, but they will recreated as needed. + */ +public final class CommitContext implements Closeable { + + private static final Logger LOG = LoggerFactory.getLogger( + CommitContext.class); + + /** + * The actual commit operations. + */ + private final CommitOperations commitOperations; + + /** + * Job Context. + */ + private final JobContext jobContext; + + /** + * Serializer pool. + */ + + private final WeakReferenceThreadMap> + pendingSetSerializer = + new WeakReferenceThreadMap<>((k) -> PendingSet.serializer(), null); + + private final WeakReferenceThreadMap> + singleCommitSerializer = + new WeakReferenceThreadMap<>((k) -> SinglePendingCommit.serializer(), null); + + /** + * Submitter for per task operations, e.g loading manifests. + */ + private PoolSubmitter outerSubmitter; + + /** + * Submitter for operations within the tasks, + * such as POSTing the final commit operations. + */ + private PoolSubmitter innerSubmitter; + + /** + * Job Configuration. + */ + private final Configuration conf; + + /** + * Job ID. + */ + private final String jobId; + + /** + * Audit context; will be reset when this is closed. + */ + private final AuditContextUpdater auditContextUpdater; + + /** + * Number of committer threads. + */ + private final int committerThreads; + + /** + * Create. + * @param commitOperations commit callbacks + * @param jobContext job context + * @param committerThreads number of commit threads + */ + public CommitContext( + final CommitOperations commitOperations, + final JobContext jobContext, + final int committerThreads) { + this.commitOperations = commitOperations; + this.jobContext = jobContext; + this.conf = jobContext.getConfiguration(); + this.jobId = jobContext.getJobID().toString(); + this.auditContextUpdater = new AuditContextUpdater(jobContext); + this.auditContextUpdater.updateCurrentAuditContext(); + this.committerThreads = committerThreads; + + buildSubmitters(); + } + + /** + * Create for testing. + * This has no job context; instead the values + * are set explicitly. + * @param commitOperations commit callbacks + * @param conf job conf + * @param jobId ID + * @param committerThreads number of commit threads + */ + public CommitContext(final CommitOperations commitOperations, + final Configuration conf, + final String jobId, + final int committerThreads) { + this.commitOperations = commitOperations; + this.jobContext = null; + this.conf = conf; + this.jobId = jobId; + this.auditContextUpdater = new AuditContextUpdater(jobId); + this.auditContextUpdater.updateCurrentAuditContext(); + this.committerThreads = committerThreads; + buildSubmitters(); + } + + /** + * Build the submitters and thread pools if the number of committerThreads + * is greater than zero. + * This should only be called in constructors; it is synchronized to keep + * SpotBugs happy. + */ + private synchronized void buildSubmitters() { + if (committerThreads != 0) { + outerSubmitter = new PoolSubmitter(buildThreadPool(committerThreads)); + } + } + + /** + * Returns an {@link ExecutorService} for parallel tasks. The number of + * threads in the thread-pool is set by fs.s3a.committer.threads. + * If num-threads is 0, this will raise an exception. + * The threads have a lifespan set by + * {@link InternalCommitterConstants#THREAD_KEEP_ALIVE_TIME}. + * When the thread pool is full, the caller runs + * policy takes over. + * @param numThreads thread count, may be negative. + * @return an {@link ExecutorService} for the number of threads + */ + private ExecutorService buildThreadPool( + int numThreads) { + if (numThreads < 0) { + // a negative number means "multiple of available processors" + numThreads = numThreads * -Runtime.getRuntime().availableProcessors(); + } + Preconditions.checkArgument(numThreads > 0, + "Cannot create a thread pool with no threads"); + LOG.debug("creating thread pool of size {}", numThreads); + final ThreadFactory factory = new ThreadFactoryBuilder() + .setDaemon(true) + .setNameFormat(THREAD_PREFIX + jobId + "-%d") + .build(); + return new HadoopThreadPoolExecutor(0, numThreads, + THREAD_KEEP_ALIVE_TIME, + TimeUnit.SECONDS, + new LinkedBlockingQueue<>(), + factory, + new ThreadPoolExecutor.CallerRunsPolicy()); + } + + /** + * Commit the operation, throwing an exception on any failure. + * See {@code CommitOperations#commitOrFail(SinglePendingCommit)}. + * @param commit commit to execute + * @throws IOException on a failure + */ + public void commitOrFail(SinglePendingCommit commit) throws IOException { + commitOperations.commitOrFail(commit); + } + + /** + * Commit a single pending commit; exceptions are caught + * and converted to an outcome. + * See {@link CommitOperations#commit(SinglePendingCommit, String)}. + * @param commit entry to commit + * @param origin origin path/string for outcome text + * @return the outcome + */ + public CommitOperations.MaybeIOE commit(SinglePendingCommit commit, + String origin) { + return commitOperations.commit(commit, origin); + } + + /** + * See {@link CommitOperations#abortSingleCommit(SinglePendingCommit)}. + * @param commit pending commit to abort + * @throws FileNotFoundException if the abort ID is unknown + * @throws IOException on any failure + */ + public void abortSingleCommit(final SinglePendingCommit commit) + throws IOException { + commitOperations.abortSingleCommit(commit); + } + + /** + * See {@link CommitOperations#revertCommit(SinglePendingCommit)}. + * @param commit pending commit + * @throws IOException failure + */ + public void revertCommit(final SinglePendingCommit commit) + throws IOException { + commitOperations.revertCommit(commit); + } + + /** + * See {@link CommitOperations#abortMultipartCommit(String, String)}.. + * @param destKey destination key + * @param uploadId upload to cancel + * @throws FileNotFoundException if the abort ID is unknown + * @throws IOException on any failure + */ + public void abortMultipartCommit( + final String destKey, + final String uploadId) + throws IOException { + commitOperations.abortMultipartCommit(destKey, uploadId); + } + + @Override + public synchronized void close() throws IOException { + + destroyThreadPools(); + auditContextUpdater.resetCurrentAuditContext(); + } + + @Override + public String toString() { + return "CommitContext{}"; + } + + /** + * Job Context. + * @return job context. + */ + public JobContext getJobContext() { + return jobContext; + } + + /** + * Return a submitter. + * If created with 0 threads, this returns null so + * TaskPool knows to run it in the current thread. + * @return a submitter or null + */ + public synchronized TaskPool.Submitter getOuterSubmitter() { + return outerSubmitter; + } + + /** + * Return a submitter. As this pool is used less often, + * create it on demand. + * If created with 0 threads, this returns null so + * TaskPool knows to run it in the current thread. + * @return a submitter or null + */ + public synchronized TaskPool.Submitter getInnerSubmitter() { + if (innerSubmitter == null && committerThreads > 0) { + innerSubmitter = new PoolSubmitter(buildThreadPool(committerThreads)); + } + return innerSubmitter; + } + + /** + * Get a serializer for .pending files. + * @return a serializer. + */ + public JsonSerialization getSinglePendingFileSerializer() { + return singleCommitSerializer.getForCurrentThread(); + } + + /** + * Get a serializer for .pendingset files. + * @return a serializer. + */ + public JsonSerialization getPendingSetSerializer() { + return pendingSetSerializer.getForCurrentThread(); + } + + /** + * Destroy any thread pools; wait for that to finish, + * but don't overreact if it doesn't finish in time. + */ + private synchronized void destroyThreadPools() { + try { + IOUtils.cleanupWithLogger(LOG, outerSubmitter, innerSubmitter); + } finally { + outerSubmitter = null; + innerSubmitter = null; + } + } + + /** + * Job configuration. + * @return configuration (never null) + */ + public Configuration getConf() { + return conf; + } + + /** + * Get the job ID. + * @return job ID. + */ + public String getJobId() { + return jobId; + } + + /** + * Submitter for a given thread pool. + */ + private final class PoolSubmitter implements TaskPool.Submitter, Closeable { + + private ExecutorService executor; + + private PoolSubmitter(ExecutorService executor) { + this.executor = executor; + } + + @Override + public synchronized void close() throws IOException { + if (executor != null) { + HadoopExecutors.shutdown(executor, LOG, + THREAD_POOL_SHUTDOWN_DELAY_SECONDS, TimeUnit.SECONDS); + } + executor = null; + } + + /** + * Forward to the submitter, wrapping in task + * context setting, so as to ensure that all operations + * have job/task attributes. + * @param task task to execute + * @return the future. + */ + @Override + public Future submit(Runnable task) { + return executor.submit(() -> { + auditContextUpdater.updateCurrentAuditContext(); + try { + task.run(); + } finally { + auditContextUpdater.resetCurrentAuditContext(); + } + }); + } + + } +} diff --git a/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/commit/CommitOperations.java b/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/commit/impl/CommitOperations.java similarity index 82% rename from hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/commit/CommitOperations.java rename to hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/commit/impl/CommitOperations.java index 840cf8e0f23cc..0772e143f69d6 100644 --- a/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/commit/CommitOperations.java +++ b/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/commit/impl/CommitOperations.java @@ -16,24 +16,26 @@ * limitations under the License. */ -package org.apache.hadoop.fs.s3a.commit; +package org.apache.hadoop.fs.s3a.commit.impl; -import java.io.Closeable; import java.io.File; import java.io.FileNotFoundException; import java.io.IOException; import java.util.ArrayList; +import java.util.Collections; import java.util.List; import java.util.Map; import java.util.Optional; +import java.util.UUID; import java.util.stream.Collectors; import java.util.stream.IntStream; +import javax.annotation.Nullable; + import com.amazonaws.services.s3.model.MultipartUpload; import com.amazonaws.services.s3.model.PartETag; import com.amazonaws.services.s3.model.UploadPartRequest; import com.amazonaws.services.s3.model.UploadPartResult; -import org.apache.hadoop.util.Preconditions; import org.slf4j.Logger; import org.slf4j.LoggerFactory; @@ -47,25 +49,33 @@ import org.apache.hadoop.fs.s3a.S3AFileSystem; import org.apache.hadoop.fs.s3a.S3AUtils; import org.apache.hadoop.fs.s3a.WriteOperations; +import org.apache.hadoop.fs.s3a.commit.CommitConstants; +import org.apache.hadoop.fs.s3a.commit.PathCommitException; import org.apache.hadoop.fs.s3a.commit.files.PendingSet; import org.apache.hadoop.fs.s3a.commit.files.SinglePendingCommit; import org.apache.hadoop.fs.s3a.commit.files.SuccessData; import org.apache.hadoop.fs.s3a.impl.AbstractStoreOperation; import org.apache.hadoop.fs.s3a.impl.HeaderProcessing; import org.apache.hadoop.fs.s3a.impl.InternalConstants; +import org.apache.hadoop.fs.s3a.impl.PutObjectOptions; import org.apache.hadoop.fs.s3a.statistics.CommitterStatistics; import org.apache.hadoop.fs.statistics.DurationTracker; import org.apache.hadoop.fs.statistics.IOStatistics; import org.apache.hadoop.fs.statistics.IOStatisticsSource; +import org.apache.hadoop.mapreduce.JobContext; import org.apache.hadoop.util.DurationInfo; +import org.apache.hadoop.util.Preconditions; import org.apache.hadoop.util.Progressable; +import org.apache.hadoop.util.functional.TaskPool; import static java.util.Objects.requireNonNull; -import static org.apache.hadoop.fs.s3a.S3AUtils.*; +import static org.apache.hadoop.fs.s3a.S3AUtils.listAndFilter; import static org.apache.hadoop.fs.s3a.Statistic.COMMITTER_COMMIT_JOB; import static org.apache.hadoop.fs.s3a.Statistic.COMMITTER_MATERIALIZE_FILE; +import static org.apache.hadoop.fs.s3a.Statistic.COMMITTER_LOAD_SINGLE_PENDING_FILE; import static org.apache.hadoop.fs.s3a.Statistic.COMMITTER_STAGE_FILE_UPLOAD; -import static org.apache.hadoop.fs.s3a.commit.CommitConstants.*; +import static org.apache.hadoop.fs.s3a.commit.CommitConstants.XA_MAGIC_MARKER; +import static org.apache.hadoop.fs.s3a.commit.CommitConstants._SUCCESS; import static org.apache.hadoop.fs.statistics.impl.IOStatisticsBinding.trackDuration; import static org.apache.hadoop.util.functional.RemoteIterators.cleanupRemoteIterator; @@ -115,7 +125,7 @@ public class CommitOperations extends AbstractStoreOperation * @throws IOException failure to bind. */ public CommitOperations(S3AFileSystem fs) throws IOException { - this(requireNonNull(fs), fs.newCommitterStatistics()); + this(requireNonNull(fs), fs.newCommitterStatistics(), "/"); } /** @@ -123,10 +133,12 @@ public CommitOperations(S3AFileSystem fs) throws IOException { * the commit operations. * @param fs FS to bind to * @param committerStatistics committer statistics + * @param outputPath destination of work. * @throws IOException failure to bind. */ public CommitOperations(S3AFileSystem fs, - CommitterStatistics committerStatistics) throws IOException { + CommitterStatistics committerStatistics, + String outputPath) throws IOException { super(requireNonNull(fs).createStoreContext()); this.fs = fs; statistics = requireNonNull(committerStatistics); @@ -134,7 +146,7 @@ public CommitOperations(S3AFileSystem fs, writeOperations = fs.createWriteOperationHelper( fs.getAuditSpanSource().createSpan( COMMITTER_COMMIT_JOB.getSymbol(), - "/", null)); + outputPath, null)); } /** @@ -168,7 +180,7 @@ public IOStatistics getIOStatistics() { * @param commit commit to execute * @throws IOException on a failure */ - private void commitOrFail( + public void commitOrFail( final SinglePendingCommit commit) throws IOException { commit(commit, commit.getFilename()).maybeRethrow(); } @@ -180,7 +192,7 @@ private void commitOrFail( * @param origin origin path/string for outcome text * @return the outcome */ - private MaybeIOE commit( + public MaybeIOE commit( final SinglePendingCommit commit, final String origin) { LOG.debug("Committing single commit {}", commit); @@ -225,10 +237,9 @@ private long innerCommit( // finalize the commit writeOperations.commitUpload( commit.getDestinationKey(), - commit.getUploadId(), - toPartEtags(commit.getEtags()), - commit.getLength() - ); + commit.getUploadId(), + toPartEtags(commit.getEtags()), + commit.getLength()); return commit.getLength(); } @@ -236,43 +247,69 @@ private long innerCommit( * Locate all files with the pending suffix under a directory. * @param pendingDir directory * @param recursive recursive listing? - * @return the list of all located entries + * @return iterator of all located entries * @throws IOException if there is a problem listing the path. */ - public List locateAllSinglePendingCommits( + public RemoteIterator locateAllSinglePendingCommits( Path pendingDir, boolean recursive) throws IOException { return listAndFilter(fs, pendingDir, recursive, PENDING_FILTER); } /** - * Load all single pending commits in the directory. + * Load all single pending commits in the directory, using the + * outer submitter. * All load failures are logged and then added to list of files which would * not load. + * * @param pendingDir directory containing commits * @param recursive do a recursive scan? + * @param commitContext commit context + * * @return tuple of loaded entries and those pending files which would * not load/validate. + * * @throws IOException on a failure to list the files. */ public Pair>> - loadSinglePendingCommits(Path pendingDir, boolean recursive) + loadSinglePendingCommits(Path pendingDir, + boolean recursive, + CommitContext commitContext) throws IOException { - List statusList = locateAllSinglePendingCommits( - pendingDir, recursive); - PendingSet commits = new PendingSet( - statusList.size()); - List> failures = new ArrayList<>(1); - for (LocatedFileStatus status : statusList) { - try { - commits.add(SinglePendingCommit.load(fs, status.getPath(), status)); - } catch (IOException e) { - LOG.warn("Failed to load commit file {}", status.getPath(), e); - failures.add(Pair.of(status, e)); - } - } + PendingSet commits = new PendingSet(); + List pendingFiles = Collections.synchronizedList( + new ArrayList<>(1)); + List> failures = Collections.synchronizedList( + new ArrayList<>(1)); + + TaskPool.foreach(locateAllSinglePendingCommits(pendingDir, recursive)) + //. stopOnFailure() + .suppressExceptions(false) + .executeWith(commitContext.getOuterSubmitter()) + .run(status -> { + Path path = status.getPath(); + try { + // load the file + SinglePendingCommit singleCommit = trackDuration(statistics, + COMMITTER_LOAD_SINGLE_PENDING_FILE.getSymbol(), () -> + SinglePendingCommit.load(fs, + path, + status, + commitContext.getSinglePendingFileSerializer())); + // aggregate stats + commits.getIOStatistics() + .aggregate(singleCommit.getIOStatistics()); + // then clear so they aren't marshalled again. + singleCommit.getIOStatistics().clear(); + pendingFiles.add(singleCommit); + } catch (IOException e) { + LOG.warn("Failed to load commit file {}", path, e); + failures.add(Pair.of(status, e)); + } + }); + commits.setCommits(pendingFiles); return Pair.of(commits, failures); } @@ -296,7 +333,7 @@ public IOException makeIOE(String key, Exception ex) { * @throws FileNotFoundException if the abort ID is unknown * @throws IOException on any failure */ - private void abortSingleCommit(SinglePendingCommit commit) + public void abortSingleCommit(SinglePendingCommit commit) throws IOException { String destKey = commit.getDestinationKey(); String origin = commit.getFilename() != null @@ -315,7 +352,7 @@ private void abortSingleCommit(SinglePendingCommit commit) * @throws FileNotFoundException if the abort ID is unknown * @throws IOException on any failure */ - private void abortMultipartCommit(String destKey, String uploadId) + public void abortMultipartCommit(String destKey, String uploadId) throws IOException { try (DurationInfo d = new DurationInfo(LOG, "Aborting commit ID %s to path %s", uploadId, destKey)) { @@ -328,11 +365,13 @@ private void abortMultipartCommit(String destKey, String uploadId) /** * Enumerate all pending files in a dir/tree, abort. * @param pendingDir directory of pending operations + * @param commitContext commit context * @param recursive recurse? * @return the outcome of all the abort operations * @throws IOException if there is a problem listing the path. */ public MaybeIOE abortAllSinglePendingCommits(Path pendingDir, + CommitContext commitContext, boolean recursive) throws IOException { Preconditions.checkArgument(pendingDir != null, "null pendingDir"); @@ -350,12 +389,14 @@ public MaybeIOE abortAllSinglePendingCommits(Path pendingDir, LOG.debug("No files to abort under {}", pendingDir); } while (pendingFiles.hasNext()) { - LocatedFileStatus status = pendingFiles.next(); + final LocatedFileStatus status = pendingFiles.next(); Path pendingFile = status.getPath(); if (pendingFile.getName().endsWith(CommitConstants.PENDING_SUFFIX)) { try { - abortSingleCommit(SinglePendingCommit.load(fs, pendingFile, - status)); + abortSingleCommit(SinglePendingCommit.load(fs, + pendingFile, + status, + commitContext.getSinglePendingFileSerializer())); } catch (FileNotFoundException e) { LOG.debug("listed file already deleted: {}", pendingFile); } catch (IOException | IllegalArgumentException e) { @@ -437,7 +478,7 @@ public void createSuccessMarker(Path outputPath, successData); try (DurationInfo ignored = new DurationInfo(LOG, "Writing success file %s", markerPath)) { - successData.save(fs, markerPath, true); + successData.save(fs, markerPath, SuccessData.serializer()); } } @@ -466,7 +507,7 @@ public void revertCommit(SinglePendingCommit commit) throws IOException { * @return a pending upload entry * @throws IOException failure */ - public SinglePendingCommit uploadFileToPendingCommit(File localFile, + public SinglePendingCommit uploadFileToPendingCommit(File localFile, Path destPath, String partition, long uploadPartSize, @@ -494,7 +535,8 @@ public SinglePendingCommit uploadFileToPendingCommit(File localFile, destPath)) { statistics.commitCreated(); - uploadId = writeOperations.initiateMultiPartUpload(destKey); + uploadId = writeOperations.initiateMultiPartUpload(destKey, + PutObjectOptions.keepingDirs()); long length = localFile.length(); SinglePendingCommit commitData = new SinglePendingCommit(); @@ -592,13 +634,41 @@ public void jobCompleted(boolean success) { } /** - * Begin the final commit. + * Create a commit context for a job or task. + * + * @param context job context + * @param path path for all work. + * @param committerThreads thread pool size + * @return the commit context to pass in. + * @throws IOException failure. + */ + public CommitContext createCommitContext( + JobContext context, + Path path, + int committerThreads) throws IOException { + return new CommitContext(this, context, + committerThreads); + } + + /** + * Create a stub commit context for tests. + * There's no job context. * @param path path for all work. + * @param jobId job ID; if null a random UUID is generated. + * @param committerThreads number of committer threads. * @return the commit context to pass in. * @throws IOException failure. */ - public CommitContext initiateCommitOperation(Path path) throws IOException { - return new CommitContext(); + public CommitContext createCommitContextForTesting( + Path path, @Nullable String jobId, int committerThreads) throws IOException { + final String id = jobId != null + ? jobId + : UUID.randomUUID().toString(); + + return new CommitContext(this, + getStoreContext().getConfiguration(), + id, + committerThreads); } /** @@ -624,98 +694,6 @@ public static Optional extractMagicFileLength(FileSystem fs, Path path) return HeaderProcessing.extractXAttrLongValue(bytes); } - /** - * Commit context. - * - * It is used to manage the final commit sequence where files become - * visible. - * - * This can only be created through {@link #initiateCommitOperation(Path)}. - * - * Once the commit operation has completed, it must be closed. - * It must not be reused. - */ - public final class CommitContext implements Closeable { - - - /** - * Create. - */ - private CommitContext() { - } - - /** - * Commit the operation, throwing an exception on any failure. - * See {@link CommitOperations#commitOrFail(SinglePendingCommit)}. - * @param commit commit to execute - * @throws IOException on a failure - */ - public void commitOrFail(SinglePendingCommit commit) throws IOException { - CommitOperations.this.commitOrFail(commit); - } - - /** - * Commit a single pending commit; exceptions are caught - * and converted to an outcome. - * See {@link CommitOperations#commit(SinglePendingCommit, String)}. - * @param commit entry to commit - * @param origin origin path/string for outcome text - * @return the outcome - */ - public MaybeIOE commit(SinglePendingCommit commit, - String origin) { - return CommitOperations.this.commit(commit, origin); - } - - /** - * See {@link CommitOperations#abortSingleCommit(SinglePendingCommit)}. - * @param commit pending commit to abort - * @throws FileNotFoundException if the abort ID is unknown - * @throws IOException on any failure - */ - public void abortSingleCommit(final SinglePendingCommit commit) - throws IOException { - CommitOperations.this.abortSingleCommit(commit); - } - - /** - * See {@link CommitOperations#revertCommit(SinglePendingCommit)}. - * @param commit pending commit - * @throws IOException failure - */ - public void revertCommit(final SinglePendingCommit commit) - throws IOException { - CommitOperations.this.revertCommit(commit); - } - - /** - * See {@link CommitOperations#abortMultipartCommit(String, String)}.. - * @param destKey destination key - * @param uploadId upload to cancel - * @throws FileNotFoundException if the abort ID is unknown - * @throws IOException on any failure - */ - public void abortMultipartCommit( - final String destKey, - final String uploadId) - throws IOException { - CommitOperations.this.abortMultipartCommit(destKey, uploadId); - } - - @Override - public void close() throws IOException { - } - - @Override - public String toString() { - final StringBuilder sb = new StringBuilder( - "CommitContext{"); - sb.append('}'); - return sb.toString(); - } - - } - /** * A holder for a possible IOException; the call {@link #maybeRethrow()} * will throw any exception passed into the constructor, and be a no-op @@ -788,5 +766,4 @@ public static MaybeIOE of(IOException ex) { } } - } diff --git a/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/commit/CommitUtilsWithMR.java b/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/commit/impl/CommitUtilsWithMR.java similarity index 74% rename from hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/commit/CommitUtilsWithMR.java rename to hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/commit/impl/CommitUtilsWithMR.java index 9e5ee860e85ff..c38ab2e9ba1af 100644 --- a/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/commit/CommitUtilsWithMR.java +++ b/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/commit/impl/CommitUtilsWithMR.java @@ -16,7 +16,7 @@ * limitations under the License. */ -package org.apache.hadoop.fs.s3a.commit; +package org.apache.hadoop.fs.s3a.commit.impl; import org.apache.hadoop.classification.InterfaceAudience; import org.apache.hadoop.classification.InterfaceStability; @@ -68,31 +68,67 @@ public static int getAppAttemptId(JobContext context) { /** * Compute the "magic" path for a job attempt. * @param jobUUID unique Job ID. + * @param appAttemptId the ID of the application attempt for this job. * @param dest the final output directory * @return the path to store job attempt data. */ - public static Path getMagicJobAttemptPath(String jobUUID, Path dest) { - return new Path(getMagicJobAttemptsPath(dest), - formatAppAttemptDir(jobUUID)); + public static Path getMagicJobAttemptPath(String jobUUID, + int appAttemptId, + Path dest) { + return new Path( + getMagicJobAttemptsPath(dest), + formatAppAttemptDir(jobUUID, appAttemptId)); + } + + /** + * Compute the "magic" path for a job. + * @param jobUUID unique Job ID. + * @param dest the final output directory + * @return the path to store job attempt data. + */ + public static Path getMagicJobPath(String jobUUID, + Path dest) { + return new Path( + getMagicJobAttemptsPath(dest), + formatJobDir(jobUUID)); } /** - * Format the application attempt directory. + * Build the name of the job directory, without + * app attempt. + * This is the path to use for cleanup. * @param jobUUID unique Job ID. - * @return the directory name for the application attempt + * @return the directory name for the job */ - public static String formatAppAttemptDir(String jobUUID) { + public static String formatJobDir( + String jobUUID) { return String.format("job-%s", jobUUID); } + /** + * Build the name of the job attempt directory. + * @param jobUUID unique Job ID. + * @param appAttemptId the ID of the application attempt for this job. + * @return the directory tree for the application attempt + */ + public static String formatAppAttemptDir( + String jobUUID, + int appAttemptId) { + return formatJobDir(jobUUID) + String.format("/%02d", appAttemptId); + } + /** * Compute the path where the output of magic task attempts are stored. * @param jobUUID unique Job ID. * @param dest destination of work + * @param appAttemptId the ID of the application attempt for this job. * @return the path where the output of magic task attempts are stored. */ - public static Path getMagicTaskAttemptsPath(String jobUUID, Path dest) { - return new Path(getMagicJobAttemptPath(jobUUID, dest), "tasks"); + public static Path getMagicTaskAttemptsPath( + String jobUUID, + Path dest, + int appAttemptId) { + return new Path(getMagicJobAttemptPath(jobUUID, appAttemptId, dest), "tasks"); } /** @@ -115,6 +151,8 @@ public static Path getMagicTaskAttemptPath(TaskAttemptContext context, /** * Get the base Magic attempt path, without any annotations to mark relative * references. + * If there is an app attempt property in the context configuration, that + * is included. * @param context task context. * @param jobUUID unique Job ID. * @param dest The output path to commit work into @@ -123,8 +161,9 @@ public static Path getMagicTaskAttemptPath(TaskAttemptContext context, public static Path getBaseMagicTaskAttemptPath(TaskAttemptContext context, String jobUUID, Path dest) { - return new Path(getMagicTaskAttemptsPath(jobUUID, dest), - String.valueOf(context.getTaskAttemptID())); + return new Path( + getMagicTaskAttemptsPath(jobUUID, dest, getAppAttemptId(context)), + String.valueOf(context.getTaskAttemptID())); } /** @@ -132,12 +171,13 @@ public static Path getBaseMagicTaskAttemptPath(TaskAttemptContext context, * This data is not magic * @param jobUUID unique Job ID. * @param out output directory of job + * @param appAttemptId the ID of the application attempt for this job. * @return the path to store temporary job attempt data. */ public static Path getTempJobAttemptPath(String jobUUID, - Path out) { + Path out, final int appAttemptId) { return new Path(new Path(out, TEMP_DATA), - formatAppAttemptDir(jobUUID)); + formatAppAttemptDir(jobUUID, appAttemptId)); } /** @@ -150,7 +190,7 @@ public static Path getTempJobAttemptPath(String jobUUID, public static Path getTempTaskAttemptPath(TaskAttemptContext context, final String jobUUID, Path out) { return new Path( - getTempJobAttemptPath(jobUUID, out), + getTempJobAttemptPath(jobUUID, out, getAppAttemptId(context)), String.valueOf(context.getTaskAttemptID())); } diff --git a/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/commit/impl/package-info.java b/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/commit/impl/package-info.java new file mode 100644 index 0000000000000..b4977f34d57f4 --- /dev/null +++ b/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/commit/impl/package-info.java @@ -0,0 +1,31 @@ +/* + * 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. + */ + +/** + * Internal classes which make use of mapreduce code. + * These MUST NOT be referred to in production code except + * in org.apache.hadoop.fs.s3a.commit classes which are only + * used within job/task committers. + */ + +@InterfaceAudience.Private +@InterfaceStability.Unstable +package org.apache.hadoop.fs.s3a.commit.impl; + +import org.apache.hadoop.classification.InterfaceAudience; +import org.apache.hadoop.classification.InterfaceStability; \ No newline at end of file diff --git a/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/commit/magic/MagicCommitTracker.java b/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/commit/magic/MagicCommitTracker.java index 7ea4e88308db4..c85571a1949a1 100644 --- a/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/commit/magic/MagicCommitTracker.java +++ b/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/commit/magic/MagicCommitTracker.java @@ -26,20 +26,26 @@ import com.amazonaws.services.s3.model.PartETag; import com.amazonaws.services.s3.model.PutObjectRequest; -import org.apache.hadoop.util.Preconditions; import org.slf4j.Logger; import org.slf4j.LoggerFactory; import org.apache.commons.lang3.StringUtils; import org.apache.hadoop.classification.InterfaceAudience; import org.apache.hadoop.fs.Path; +import org.apache.hadoop.fs.s3a.Retries; import org.apache.hadoop.fs.s3a.WriteOperationHelper; import org.apache.hadoop.fs.s3a.commit.PutTracker; import org.apache.hadoop.fs.s3a.commit.files.SinglePendingCommit; +import org.apache.hadoop.fs.s3a.impl.PutObjectOptions; +import org.apache.hadoop.fs.s3a.statistics.PutTrackerStatistics; import org.apache.hadoop.fs.statistics.IOStatistics; import org.apache.hadoop.fs.statistics.IOStatisticsSnapshot; +import org.apache.hadoop.util.Preconditions; +import static java.util.Objects.requireNonNull; +import static org.apache.hadoop.fs.s3a.Statistic.COMMITTER_MAGIC_MARKER_PUT; import static org.apache.hadoop.fs.s3a.commit.CommitConstants.X_HEADER_MAGIC_MARKER; +import static org.apache.hadoop.fs.statistics.impl.IOStatisticsBinding.trackDurationOfInvocation; /** * Put tracker for Magic commits. @@ -57,6 +63,7 @@ public class MagicCommitTracker extends PutTracker { private final WriteOperationHelper writer; private final String bucket; private static final byte[] EMPTY = new byte[0]; + private final PutTrackerStatistics trackerStatistics; /** * Magic commit tracker. @@ -66,19 +73,22 @@ public class MagicCommitTracker extends PutTracker { * @param destKey key for the destination * @param pendingsetKey key of the pendingset file * @param writer writer instance to use for operations; includes audit span + * @param trackerStatistics tracker statistics */ public MagicCommitTracker(Path path, String bucket, String originalDestKey, String destKey, String pendingsetKey, - WriteOperationHelper writer) { + WriteOperationHelper writer, + PutTrackerStatistics trackerStatistics) { super(destKey); this.bucket = bucket; this.path = path; this.originalDestKey = originalDestKey; this.pendingPartKey = pendingsetKey; this.writer = writer; + this.trackerStatistics = requireNonNull(trackerStatistics); LOG.info("File {} is written as magic file to path {}", path, destKey); } @@ -126,6 +136,19 @@ public boolean aboutToComplete(String uploadId, Preconditions.checkArgument(!parts.isEmpty(), "No uploaded parts to save"); + // put a 0-byte file with the name of the original under-magic path + // Add the final file length as a header + // this is done before the task commit, so its duration can be + // included in the statistics + Map headers = new HashMap<>(); + headers.put(X_HEADER_MAGIC_MARKER, Long.toString(bytesWritten)); + PutObjectRequest originalDestPut = writer.createPutObjectRequest( + originalDestKey, + new ByteArrayInputStream(EMPTY), + 0, + new PutObjectOptions(true, null, headers)); + upload(originalDestPut); + // build the commit summary SinglePendingCommit commitData = new SinglePendingCommit(); commitData.touch(System.currentTimeMillis()); @@ -138,7 +161,8 @@ public boolean aboutToComplete(String uploadId, commitData.bindCommitData(parts); commitData.setIOStatistics( new IOStatisticsSnapshot(iostatistics)); - byte[] bytes = commitData.toBytes(); + + byte[] bytes = commitData.toBytes(SinglePendingCommit.serializer()); LOG.info("Uncommitted data pending to file {};" + " commit metadata for {} parts in {}. size: {} byte(s)", path.toUri(), parts.size(), pendingPartKey, bytesWritten); @@ -148,19 +172,20 @@ public boolean aboutToComplete(String uploadId, pendingPartKey, new ByteArrayInputStream(bytes), bytes.length, null); - writer.uploadObject(put); - - // Add the final file length as a header - Map headers = new HashMap<>(); - headers.put(X_HEADER_MAGIC_MARKER, Long.toString(bytesWritten)); - // now put a 0-byte file with the name of the original under-magic path - PutObjectRequest originalDestPut = writer.createPutObjectRequest( - originalDestKey, - new ByteArrayInputStream(EMPTY), - 0, - headers); - writer.uploadObject(originalDestPut); + upload(put); return false; + + } + /** + * PUT an object. + * @param request the request + * @throws IOException on problems + */ + @Retries.RetryTranslated + private void upload(PutObjectRequest request) throws IOException { + trackDurationOfInvocation(trackerStatistics, + COMMITTER_MAGIC_MARKER_PUT.getSymbol(), () -> + writer.uploadObject(request, PutObjectOptions.keepingDirs())); } @Override diff --git a/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/commit/magic/MagicS3GuardCommitter.java b/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/commit/magic/MagicS3GuardCommitter.java index c1ecd7d6b9b53..007e9b3709623 100644 --- a/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/commit/magic/MagicS3GuardCommitter.java +++ b/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/commit/magic/MagicS3GuardCommitter.java @@ -32,11 +32,12 @@ import org.apache.hadoop.fs.Path; import org.apache.hadoop.fs.s3a.Invoker; import org.apache.hadoop.fs.s3a.commit.AbstractS3ACommitter; -import org.apache.hadoop.fs.s3a.commit.CommitOperations; +import org.apache.hadoop.fs.s3a.commit.impl.CommitContext; +import org.apache.hadoop.fs.s3a.commit.impl.CommitOperations; import org.apache.hadoop.fs.s3a.commit.CommitConstants; -import org.apache.hadoop.fs.s3a.commit.CommitUtilsWithMR; import org.apache.hadoop.fs.s3a.commit.files.PendingSet; import org.apache.hadoop.fs.s3a.commit.files.SinglePendingCommit; +import org.apache.hadoop.fs.s3a.commit.impl.CommitUtilsWithMR; import org.apache.hadoop.fs.statistics.IOStatisticsLogging; import org.apache.hadoop.mapreduce.JobContext; import org.apache.hadoop.mapreduce.TaskAttemptContext; @@ -45,9 +46,10 @@ import static org.apache.hadoop.fs.s3a.S3AUtils.*; import static org.apache.hadoop.fs.s3a.commit.CommitConstants.TASK_ATTEMPT_ID; +import static org.apache.hadoop.fs.s3a.commit.CommitConstants.TEMP_DATA; import static org.apache.hadoop.fs.s3a.commit.CommitUtils.*; import static org.apache.hadoop.fs.s3a.commit.MagicCommitPaths.*; -import static org.apache.hadoop.fs.s3a.commit.CommitUtilsWithMR.*; +import static org.apache.hadoop.fs.s3a.commit.impl.CommitUtilsWithMR.*; import static org.apache.hadoop.fs.statistics.IOStatisticsLogging.demandStringifyIOStatistics; /** @@ -100,25 +102,28 @@ public void setupJob(JobContext context) throws IOException { try (DurationInfo d = new DurationInfo(LOG, "Setup Job %s", jobIdString(context))) { super.setupJob(context); - Path jobAttemptPath = getJobAttemptPath(context); - getDestinationFS(jobAttemptPath, - context.getConfiguration()).mkdirs(jobAttemptPath); + Path jobPath = getJobPath(); + final FileSystem destFS = getDestinationFS(jobPath, + context.getConfiguration()); + destFS.delete(jobPath, true); + destFS.mkdirs(jobPath); } } /** * Get the list of pending uploads for this job attempt, by listing * all .pendingset files in the job attempt directory. - * @param context job context + * @param commitContext job context * @return a list of pending commits. * @throws IOException Any IO failure */ protected ActiveCommit listPendingUploadsToCommit( - JobContext context) + CommitContext commitContext) throws IOException { FileSystem fs = getDestFS(); - return ActiveCommit.fromStatusList(fs, - listAndFilter(fs, getJobAttemptPath(context), false, + return ActiveCommit.fromStatusIterator(fs, + listAndFilter(fs, getJobAttemptPath(commitContext.getJobContext()), + false, CommitOperations.PENDINGSET_FILTER)); } @@ -126,11 +131,16 @@ protected ActiveCommit listPendingUploadsToCommit( * Delete the magic directory. */ public void cleanupStagingDirs() { - Path path = magicSubdir(getOutputPath()); + final Path out = getOutputPath(); + Path path = magicSubdir(out); try(DurationInfo ignored = new DurationInfo(LOG, true, "Deleting magic directory %s", path)) { Invoker.ignoreIOExceptions(LOG, "cleanup magic directory", path.toString(), () -> deleteWithWarning(getDestFS(), path, true)); + // and the job temp directory with manifests + Invoker.ignoreIOExceptions(LOG, "cleanup job directory", path.toString(), + () -> deleteWithWarning(getDestFS(), + new Path(out, TEMP_DATA), true)); } } @@ -146,13 +156,8 @@ public void cleanupStagingDirs() { @Override public boolean needsTaskCommit(TaskAttemptContext context) throws IOException { - Path taskAttemptPath = getTaskAttemptPath(context); - try (DurationInfo d = new DurationInfo(LOG, - "needsTaskCommit task %s", context.getTaskAttemptID())) { - return taskAttemptPath.getFileSystem( - context.getConfiguration()) - .exists(taskAttemptPath); - } + // return true as a dir was created here in setup; + return true; } @Override @@ -167,9 +172,9 @@ public void commitTask(TaskAttemptContext context) throws IOException { throw e; } finally { // delete the task attempt so there's no possibility of a second attempt + // incurs a LIST, a bulk DELETE and maybe a parent dir creation, however + // as it happens during task commit, it should be off the critical path. deleteTaskAttemptPathQuietly(context); - destroyThreadPool(); - resetCommonContext(); } getCommitOperations().taskCompleted(true); LOG.debug("aggregate statistics\n{}", @@ -191,43 +196,48 @@ private PendingSet innerCommitTask( Path taskAttemptPath = getTaskAttemptPath(context); // load in all pending commits. CommitOperations actions = getCommitOperations(); - Pair>> - loaded = actions.loadSinglePendingCommits( - taskAttemptPath, true); - PendingSet pendingSet = loaded.getKey(); - List> failures = loaded.getValue(); - if (!failures.isEmpty()) { - // At least one file failed to load - // revert all which did; report failure with first exception - LOG.error("At least one commit file could not be read: failing"); - abortPendingUploads(context, pendingSet.getCommits(), true); - throw failures.get(0).getValue(); - } - // patch in IDs - String jobId = getUUID(); - String taskId = String.valueOf(context.getTaskAttemptID()); - for (SinglePendingCommit commit : pendingSet.getCommits()) { - commit.setJobId(jobId); - commit.setTaskId(taskId); - } - pendingSet.putExtraData(TASK_ATTEMPT_ID, taskId); - pendingSet.setJobId(jobId); - Path jobAttemptPath = getJobAttemptPath(context); - TaskAttemptID taskAttemptID = context.getTaskAttemptID(); - Path taskOutcomePath = new Path(jobAttemptPath, - taskAttemptID.getTaskID().toString() + - CommitConstants.PENDINGSET_SUFFIX); - LOG.info("Saving work of {} to {}", taskAttemptID, taskOutcomePath); - LOG.debug("task statistics\n{}", - IOStatisticsLogging.demandStringifyIOStatisticsSource(pendingSet)); - try { - // We will overwrite if there exists a pendingSet file already - pendingSet.save(getDestFS(), taskOutcomePath, true); - } catch (IOException e) { - LOG.warn("Failed to save task commit data to {} ", - taskOutcomePath, e); - abortPendingUploads(context, pendingSet.getCommits(), true); - throw e; + PendingSet pendingSet; + try (CommitContext commitContext = initiateTaskOperation(context)) { + Pair>> + loaded = actions.loadSinglePendingCommits( + taskAttemptPath, true, commitContext); + pendingSet = loaded.getKey(); + List> failures = loaded.getValue(); + if (!failures.isEmpty()) { + // At least one file failed to load + // revert all which did; report failure with first exception + LOG.error("At least one commit file could not be read: failing"); + abortPendingUploads(commitContext, pendingSet.getCommits(), true); + throw failures.get(0).getValue(); + } + // patch in IDs + String jobId = getUUID(); + String taskId = String.valueOf(context.getTaskAttemptID()); + for (SinglePendingCommit commit : pendingSet.getCommits()) { + commit.setJobId(jobId); + commit.setTaskId(taskId); + } + pendingSet.putExtraData(TASK_ATTEMPT_ID, taskId); + pendingSet.setJobId(jobId); + Path jobAttemptPath = getJobAttemptPath(context); + TaskAttemptID taskAttemptID = context.getTaskAttemptID(); + Path taskOutcomePath = new Path(jobAttemptPath, + taskAttemptID.getTaskID().toString() + + CommitConstants.PENDINGSET_SUFFIX); + LOG.info("Saving work of {} to {}", taskAttemptID, taskOutcomePath); + LOG.debug("task statistics\n{}", + IOStatisticsLogging.demandStringifyIOStatisticsSource(pendingSet)); + try { + // We will overwrite if there exists a pendingSet file already + pendingSet.save(getDestFS(), + taskOutcomePath, + commitContext.getPendingSetSerializer()); + } catch (IOException e) { + LOG.warn("Failed to save task commit data to {} ", + taskOutcomePath, e); + abortPendingUploads(commitContext, pendingSet.getCommits(), true); + throw e; + } } return pendingSet; } @@ -246,25 +256,35 @@ private PendingSet innerCommitTask( public void abortTask(TaskAttemptContext context) throws IOException { Path attemptPath = getTaskAttemptPath(context); try (DurationInfo d = new DurationInfo(LOG, - "Abort task %s", context.getTaskAttemptID())) { - getCommitOperations().abortAllSinglePendingCommits(attemptPath, true); + "Abort task %s", context.getTaskAttemptID()); + CommitContext commitContext = initiateTaskOperation(context)) { + getCommitOperations().abortAllSinglePendingCommits(attemptPath, + commitContext, + true); } finally { deleteQuietly( attemptPath.getFileSystem(context.getConfiguration()), attemptPath, true); - destroyThreadPool(); - resetCommonContext(); } } + /** + * Compute the path under which all job attempts will be placed. + * @return the path to store job attempt data. + */ + @Override + protected Path getJobPath() { + return getMagicJobPath(getUUID(), getOutputPath()); + } + /** * Compute the path where the output of a given job attempt will be placed. * For the magic committer, the path includes the job UUID. * @param appAttemptId the ID of the application attempt for this job. * @return the path to store job attempt data. */ - protected Path getJobAttemptPath(int appAttemptId) { - return getMagicJobAttemptPath(getUUID(), getOutputPath()); + protected final Path getJobAttemptPath(int appAttemptId) { + return getMagicJobAttemptPath(getUUID(), appAttemptId, getOutputPath()); } /** @@ -274,12 +294,12 @@ protected Path getJobAttemptPath(int appAttemptId) { * @param context the context of the task attempt. * @return the path where a task attempt should be stored. */ - public Path getTaskAttemptPath(TaskAttemptContext context) { + public final Path getTaskAttemptPath(TaskAttemptContext context) { return getMagicTaskAttemptPath(context, getUUID(), getOutputPath()); } @Override - protected Path getBaseTaskAttemptPath(TaskAttemptContext context) { + protected final Path getBaseTaskAttemptPath(TaskAttemptContext context) { return getBaseMagicTaskAttemptPath(context, getUUID(), getOutputPath()); } diff --git a/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/commit/staging/DirectoryStagingCommitter.java b/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/commit/staging/DirectoryStagingCommitter.java index 1a5a63c940f47..99683db984983 100644 --- a/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/commit/staging/DirectoryStagingCommitter.java +++ b/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/commit/staging/DirectoryStagingCommitter.java @@ -30,6 +30,7 @@ import org.apache.hadoop.fs.Path; import org.apache.hadoop.fs.PathExistsException; import org.apache.hadoop.fs.s3a.commit.InternalCommitterConstants; +import org.apache.hadoop.fs.s3a.commit.impl.CommitContext; import org.apache.hadoop.mapreduce.JobContext; import org.apache.hadoop.mapreduce.TaskAttemptContext; @@ -98,17 +99,18 @@ public void setupJob(JobContext context) throws IOException { * Pre-commit actions for a job. * Here: look at the conflict resolution mode and choose * an action based on the current policy. - * @param context job context + * @param commitContext commit context * @param pending pending commits * @throws IOException any failure */ @Override public void preCommitJob( - final JobContext context, + final CommitContext commitContext, final ActiveCommit pending) throws IOException { + final JobContext context = commitContext.getJobContext(); // see if the files can be loaded. - super.preCommitJob(context, pending); + super.preCommitJob(commitContext, pending); Path outputPath = getOutputPath(); FileSystem fs = getDestFS(); Configuration fsConf = fs.getConf(); diff --git a/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/commit/staging/PartitionedStagingCommitter.java b/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/commit/staging/PartitionedStagingCommitter.java index 214c7abdc732a..5d1a20e4240c2 100644 --- a/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/commit/staging/PartitionedStagingCommitter.java +++ b/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/commit/staging/PartitionedStagingCommitter.java @@ -32,12 +32,13 @@ import org.apache.hadoop.fs.FileSystem; import org.apache.hadoop.fs.Path; import org.apache.hadoop.fs.s3a.commit.PathCommitException; -import org.apache.hadoop.fs.s3a.commit.Tasks; import org.apache.hadoop.fs.s3a.commit.files.PendingSet; +import org.apache.hadoop.fs.s3a.commit.files.PersistentCommitData; import org.apache.hadoop.fs.s3a.commit.files.SinglePendingCommit; -import org.apache.hadoop.mapreduce.JobContext; +import org.apache.hadoop.fs.s3a.commit.impl.CommitContext; import org.apache.hadoop.mapreduce.TaskAttemptContext; import org.apache.hadoop.util.DurationInfo; +import org.apache.hadoop.util.functional.TaskPool; import static org.apache.hadoop.fs.s3a.commit.CommitConstants.COMMITTER_NAME_PARTITIONED; @@ -89,7 +90,8 @@ public String toString() { @Override protected int commitTaskInternal(TaskAttemptContext context, - List taskOutput) throws IOException { + List taskOutput, + CommitContext commitContext) throws IOException { Path attemptPath = getTaskAttemptPath(context); Set partitions = Paths.getPartitions(attemptPath, taskOutput); @@ -109,7 +111,7 @@ protected int commitTaskInternal(TaskAttemptContext context, } } } - return super.commitTaskInternal(context, taskOutput); + return super.commitTaskInternal(context, taskOutput, commitContext); } /** @@ -121,13 +123,13 @@ protected int commitTaskInternal(TaskAttemptContext context, *
  • APPEND: allowed.; no need to check.
  • *
  • REPLACE deletes all existing partitions.
  • * - * @param context job context + * @param commitContext commit context * @param pending the pending operations * @throws IOException any failure */ @Override public void preCommitJob( - final JobContext context, + final CommitContext commitContext, final ActiveCommit pending) throws IOException { FileSystem fs = getDestFS(); @@ -135,7 +137,7 @@ public void preCommitJob( // enforce conflict resolution Configuration fsConf = fs.getConf(); boolean shouldPrecheckPendingFiles = true; - switch (getConflictResolutionMode(context, fsConf)) { + switch (getConflictResolutionMode(commitContext.getJobContext(), fsConf)) { case FAIL: // FAIL checking is done on the task side, so this does nothing break; @@ -144,17 +146,17 @@ public void preCommitJob( break; case REPLACE: // identify and replace the destination partitions - replacePartitions(context, pending); + replacePartitions(commitContext, pending); // and so there is no need to do another check. shouldPrecheckPendingFiles = false; break; default: throw new PathCommitException("", getRole() + ": unknown conflict resolution mode: " - + getConflictResolutionMode(context, fsConf)); + + getConflictResolutionMode(commitContext.getJobContext(), fsConf)); } if (shouldPrecheckPendingFiles) { - precommitCheckPendingFiles(context, pending); + precommitCheckPendingFiles(commitContext, pending); } } @@ -176,17 +178,16 @@ public void preCommitJob( * } * * - * @param context job context + * @param commitContext commit context * @param pending the pending operations * @throws IOException any failure */ private void replacePartitions( - final JobContext context, + final CommitContext commitContext, final ActiveCommit pending) throws IOException { Map partitions = new ConcurrentHashMap<>(); FileSystem sourceFS = pending.getSourceFS(); - Tasks.Submitter submitter = buildSubmitter(context); try (DurationInfo ignored = new DurationInfo(LOG, "Replacing partitions")) { @@ -194,13 +195,15 @@ private void replacePartitions( // for a marginal optimisation, the previous parent is tracked, so // if a task writes many files to the same dir, the synchronized map // is updated only once. - Tasks.foreach(pending.getSourceFiles()) + TaskPool.foreach(pending.getSourceFiles()) .stopOnFailure() .suppressExceptions(false) - .executeWith(submitter) + .executeWith(commitContext.getOuterSubmitter()) .run(status -> { - PendingSet pendingSet = PendingSet.load(sourceFS, - status); + PendingSet pendingSet = PersistentCommitData.load( + sourceFS, + status, + commitContext.getPendingSetSerializer()); Path lastParent = null; for (SinglePendingCommit commit : pendingSet.getCommits()) { Path parent = commit.destinationPath().getParent(); @@ -213,10 +216,10 @@ private void replacePartitions( } // now do the deletes FileSystem fs = getDestFS(); - Tasks.foreach(partitions.keySet()) + TaskPool.foreach(partitions.keySet()) .stopOnFailure() .suppressExceptions(false) - .executeWith(submitter) + .executeWith(commitContext.getOuterSubmitter()) .run(partitionPath -> { LOG.debug("{}: removing partition path to be replaced: " + getRole(), partitionPath); diff --git a/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/commit/staging/StagingCommitter.java b/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/commit/staging/StagingCommitter.java index 121ea7f851c02..36eae012dea7d 100644 --- a/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/commit/staging/StagingCommitter.java +++ b/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/commit/staging/StagingCommitter.java @@ -39,24 +39,25 @@ import org.apache.hadoop.fs.s3a.S3AFileSystem; import org.apache.hadoop.fs.s3a.commit.AbstractS3ACommitter; import org.apache.hadoop.fs.s3a.commit.CommitConstants; -import org.apache.hadoop.fs.s3a.commit.CommitOperations; import org.apache.hadoop.fs.s3a.commit.InternalCommitterConstants; -import org.apache.hadoop.fs.s3a.commit.Tasks; import org.apache.hadoop.fs.s3a.commit.files.PendingSet; import org.apache.hadoop.fs.s3a.commit.files.SinglePendingCommit; +import org.apache.hadoop.fs.s3a.commit.impl.CommitContext; import org.apache.hadoop.mapreduce.JobContext; import org.apache.hadoop.mapreduce.TaskAttemptContext; import org.apache.hadoop.mapreduce.lib.output.FileOutputCommitter; import org.apache.hadoop.util.DurationInfo; +import org.apache.hadoop.util.functional.TaskPool; -import static org.apache.hadoop.util.Preconditions.*; +import static java.util.Objects.requireNonNull; import static org.apache.hadoop.fs.s3a.Constants.*; import static org.apache.hadoop.fs.s3a.S3AUtils.*; import static org.apache.hadoop.fs.s3a.Invoker.*; import static org.apache.hadoop.fs.s3a.commit.CommitConstants.*; import static org.apache.hadoop.fs.s3a.commit.CommitUtils.*; -import static org.apache.hadoop.fs.s3a.commit.CommitUtilsWithMR.*; +import static org.apache.hadoop.fs.s3a.commit.impl.CommitUtilsWithMR.*; import static org.apache.hadoop.util.functional.RemoteIterators.cleanupRemoteIterator; +import static org.apache.hadoop.util.functional.RemoteIterators.toList; /** * Committer based on the contributed work of the @@ -111,7 +112,7 @@ public class StagingCommitter extends AbstractS3ACommitter { public StagingCommitter(Path outputPath, TaskAttemptContext context) throws IOException { super(outputPath, context); - this.constructorOutputPath = checkNotNull(getOutputPath(), "output path"); + this.constructorOutputPath = requireNonNull(getOutputPath(), "output path"); Configuration conf = getConf(); this.uploadPartSize = conf.getLongBytes( MULTIPART_SIZE, DEFAULT_MULTIPART_SIZE); @@ -121,8 +122,8 @@ public StagingCommitter(Path outputPath, setWorkPath(buildWorkPath(context, getUUID())); this.wrappedCommitter = createWrappedCommitter(context, conf); setOutputPath(constructorOutputPath); - Path finalOutputPath = getOutputPath(); - checkNotNull(finalOutputPath, "Output path cannot be null"); + Path finalOutputPath = requireNonNull(getOutputPath(), + "Output path cannot be null"); S3AFileSystem fs = getS3AFileSystem(finalOutputPath, context.getConfiguration(), false); s3KeyPrefix = fs.pathToKey(finalOutputPath); @@ -243,10 +244,18 @@ private static Path getJobAttemptPath(int appAttemptId, Path out) { @Override protected Path getJobAttemptPath(int appAttemptId) { - return new Path(getPendingJobAttemptsPath(commitsDirectory), + return new Path(getJobPath(), String.valueOf(appAttemptId)); } + /** + * Compute the path under which all job attempts will be placed. + * @return the path to store job attempt data. + */ + protected Path getJobPath() { + return getPendingJobAttemptsPath(commitsDirectory); + } + /** * Compute the path where the output of pending task attempts are stored. * @param context the context of the job with pending tasks. @@ -275,7 +284,7 @@ public static Path getTaskAttemptPath(TaskAttemptContext context, Path out) { * @return the location of pending job attempts. */ private static Path getPendingJobAttemptsPath(Path out) { - checkNotNull(out, "Null 'out' path"); + requireNonNull(out, "Null 'out' path"); return new Path(out, TEMPORARY); } @@ -296,12 +305,12 @@ public Path getCommittedTaskPath(TaskAttemptContext context) { * @param context task context */ private static void validateContext(TaskAttemptContext context) { - checkNotNull(context, "null context"); - checkNotNull(context.getTaskAttemptID(), + requireNonNull(context, "null context"); + requireNonNull(context.getTaskAttemptID(), "null task attempt ID"); - checkNotNull(context.getTaskAttemptID().getTaskID(), + requireNonNull(context.getTaskAttemptID().getTaskID(), "null task ID"); - checkNotNull(context.getTaskAttemptID().getJobID(), + requireNonNull(context.getTaskAttemptID().getJobID(), "null job ID"); } @@ -342,14 +351,13 @@ protected List getTaskOutput(TaskAttemptContext context) throws IOException { // get files on the local FS in the attempt path - Path attemptPath = getTaskAttemptPath(context); - checkNotNull(attemptPath, - "No attemptPath path in {}", this); + Path attemptPath = requireNonNull(getTaskAttemptPath(context), + "No attemptPath path"); LOG.debug("Scanning {} for files to commit", attemptPath); - return listAndFilter(getTaskAttemptFilesystem(context), - attemptPath, true, HIDDEN_FILE_FILTER); + return toList(listAndFilter(getTaskAttemptFilesystem(context), + attemptPath, true, HIDDEN_FILE_FILTER)); } /** @@ -425,46 +433,46 @@ public void setupJob(JobContext context) throws IOException { /** * Get the list of pending uploads for this job attempt. - * @param context job context + * @param commitContext job context * @return a list of pending uploads. * @throws IOException Any IO failure */ @Override protected ActiveCommit listPendingUploadsToCommit( - JobContext context) + CommitContext commitContext) throws IOException { - return listPendingUploads(context, false); + return listPendingUploads(commitContext, false); } /** * Get the list of pending uploads for this job attempt, swallowing * exceptions. - * @param context job context + * @param commitContext commit context * @return a list of pending uploads. If an exception was swallowed, * then this may not match the actual set of pending operations * @throws IOException shouldn't be raised, but retained for the compiler */ protected ActiveCommit listPendingUploadsToAbort( - JobContext context) throws IOException { - return listPendingUploads(context, true); + CommitContext commitContext) throws IOException { + return listPendingUploads(commitContext, true); } /** * Get the list of pending uploads for this job attempt. - * @param context job context + * @param commitContext commit context * @param suppressExceptions should exceptions be swallowed? * @return a list of pending uploads. If exceptions are being swallowed, * then this may not match the actual set of pending operations * @throws IOException Any IO failure which wasn't swallowed. */ protected ActiveCommit listPendingUploads( - JobContext context, boolean suppressExceptions) throws IOException { + CommitContext commitContext, boolean suppressExceptions) throws IOException { try (DurationInfo ignored = new DurationInfo(LOG, "Listing pending uploads")) { - Path wrappedJobAttemptPath = getJobAttemptPath(context); + Path wrappedJobAttemptPath = getJobAttemptPath(commitContext.getJobContext()); final FileSystem attemptFS = wrappedJobAttemptPath.getFileSystem( - context.getConfiguration()); - return ActiveCommit.fromStatusList(attemptFS, + commitContext.getConf()); + return ActiveCommit.fromStatusIterator(attemptFS, listAndFilter(attemptFS, wrappedJobAttemptPath, false, HIDDEN_FILE_FILTER)); @@ -491,27 +499,39 @@ public void cleanupStagingDirs() { } } + /** + * Staging committer cleanup includes calling wrapped committer's + * cleanup method, and removing all destination paths in the final + * filesystem. + * @param commitContext commit context + * @param suppressExceptions should exceptions be suppressed? + * @throws IOException IO failures if exceptions are not suppressed. + */ @Override @SuppressWarnings("deprecation") - protected void cleanup(JobContext context, + protected void cleanup(CommitContext commitContext, boolean suppressExceptions) throws IOException { maybeIgnore(suppressExceptions, "Cleanup wrapped committer", - () -> wrappedCommitter.cleanupJob(context)); + () -> wrappedCommitter.cleanupJob( + commitContext.getJobContext())); maybeIgnore(suppressExceptions, "Delete destination paths", - () -> deleteDestinationPaths(context)); - super.cleanup(context, suppressExceptions); + () -> deleteDestinationPaths( + commitContext.getJobContext())); + super.cleanup(commitContext, suppressExceptions); } @Override - protected void abortJobInternal(JobContext context, + protected void abortJobInternal(CommitContext commitContext, boolean suppressExceptions) throws IOException { String r = getRole(); + JobContext context = commitContext.getJobContext(); boolean failed = false; try (DurationInfo d = new DurationInfo(LOG, "%s: aborting job in state %s ", r, jobIdString(context))) { - ActiveCommit pending = listPendingUploadsToAbort(context); - abortPendingUploads(context, pending, suppressExceptions, true); + ActiveCommit pending = listPendingUploadsToAbort(commitContext); + abortPendingUploads(commitContext, + pending, suppressExceptions, true); } catch (FileNotFoundException e) { // nothing to list LOG.debug("No job directory to read uploads from"); @@ -519,7 +539,7 @@ protected void abortJobInternal(JobContext context, failed = true; maybeIgnore(suppressExceptions, "aborting job", e); } finally { - super.abortJobInternal(context, failed || suppressExceptions); + super.abortJobInternal(commitContext, failed || suppressExceptions); } } @@ -590,17 +610,16 @@ public boolean needsTaskCommit(TaskAttemptContext context) @Override public void commitTask(TaskAttemptContext context) throws IOException { try (DurationInfo d = new DurationInfo(LOG, - "%s: commit task %s", getRole(), context.getTaskAttemptID())) { - int count = commitTaskInternal(context, getTaskOutput(context)); + "%s: commit task %s", getRole(), context.getTaskAttemptID()); + CommitContext commitContext + = initiateTaskOperation(context)) { + int count = commitTaskInternal(context, getTaskOutput(context), commitContext); LOG.info("{}: upload file count: {}", getRole(), count); } catch (IOException e) { LOG.error("{}: commit of task {} failed", getRole(), context.getTaskAttemptID(), e); getCommitOperations().taskCompleted(false); throw e; - } finally { - destroyThreadPool(); - resetCommonContext(); } getCommitOperations().taskCompleted(true); } @@ -610,11 +629,13 @@ public void commitTask(TaskAttemptContext context) throws IOException { * writing a pending entry for them. * @param context task context * @param taskOutput list of files from the output + * @param commitContext commit context * @return number of uploads committed. * @throws IOException IO Failures. */ protected int commitTaskInternal(final TaskAttemptContext context, - List taskOutput) + List taskOutput, + CommitContext commitContext) throws IOException { LOG.debug("{}: commitTaskInternal", getRole()); Configuration conf = context.getConfiguration(); @@ -649,10 +670,10 @@ protected int commitTaskInternal(final TaskAttemptContext context, pendingCommits.putExtraData(TASK_ATTEMPT_ID, context.getTaskAttemptID().toString()); try { - Tasks.foreach(taskOutput) + TaskPool.foreach(taskOutput) .stopOnFailure() .suppressExceptions(false) - .executeWith(buildSubmitter(context)) + .executeWith(commitContext.getOuterSubmitter()) .run(stat -> { Path path = stat.getPath(); File localFile = new File(path.toUri().getPath()); @@ -676,13 +697,14 @@ protected int commitTaskInternal(final TaskAttemptContext context, } // save the data - // although overwrite=false, there's still a risk of > 1 entry being - // committed if the FS doesn't have create-no-overwrite consistency. + // overwrite any existing file, so whichever task attempt + // committed last wins. LOG.debug("Saving {} pending commit(s)) to file {}", pendingCommits.size(), commitsAttemptPath); - pendingCommits.save(commitsFS, commitsAttemptPath, false); + pendingCommits.save(commitsFS, commitsAttemptPath, + commitContext.getPendingSetSerializer()); threw = false; } finally { @@ -690,12 +712,11 @@ protected int commitTaskInternal(final TaskAttemptContext context, LOG.error( "{}: Exception during commit process, aborting {} commit(s)", getRole(), commits.size()); - try(CommitOperations.CommitContext commitContext - = initiateCommitOperation(); - DurationInfo ignored = new DurationInfo(LOG, + try(DurationInfo ignored = new DurationInfo(LOG, "Aborting %s uploads", commits.size())) { - Tasks.foreach(commits) + TaskPool.foreach(commits) .suppressExceptions() + .executeWith(commitContext.getOuterSubmitter()) .run(commitContext::abortSingleCommit); } deleteTaskAttemptPathQuietly(context); @@ -738,9 +759,6 @@ public void abortTask(TaskAttemptContext context) throws IOException { LOG.error("{}: exception when aborting task {}", getRole(), context.getTaskAttemptID(), e); throw e; - } finally { - destroyThreadPool(); - resetCommonContext(); } } @@ -859,16 +877,16 @@ public static String getConfictModeOption(JobContext context, * Pre-commit actions for a job. * Loads all the pending files to verify they can be loaded * and parsed. - * @param context job context + * @param commitContext commit context * @param pending pending commits * @throws IOException any failure */ @Override public void preCommitJob( - final JobContext context, + CommitContext commitContext, final ActiveCommit pending) throws IOException { // see if the files can be loaded. - precommitCheckPendingFiles(context, pending); + precommitCheckPendingFiles(commitContext, pending); } } diff --git a/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/impl/CreateFileBuilder.java b/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/impl/CreateFileBuilder.java new file mode 100644 index 0000000000000..0392afac59d91 --- /dev/null +++ b/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/impl/CreateFileBuilder.java @@ -0,0 +1,263 @@ +/* + * 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.fs.s3a.impl; + +import java.io.IOException; +import java.util.EnumSet; +import java.util.HashMap; +import java.util.HashSet; +import java.util.Map; +import java.util.Set; +import javax.annotation.Nonnull; + +import org.apache.hadoop.conf.Configuration; +import org.apache.hadoop.fs.CreateFlag; +import org.apache.hadoop.fs.FSDataOutputStream; +import org.apache.hadoop.fs.FSDataOutputStreamBuilder; +import org.apache.hadoop.fs.FileSystem; +import org.apache.hadoop.fs.Path; +import org.apache.hadoop.fs.PathIOException; +import org.apache.hadoop.fs.s3a.Constants; +import org.apache.hadoop.util.Progressable; + +import static org.apache.hadoop.fs.s3a.Constants.FS_S3A_CREATE_HEADER; +import static org.apache.hadoop.fs.s3a.impl.InternalConstants.CREATE_FILE_KEYS; + +/** + * Builder used in create file; takes a callback to the operation + * to create the file. + * Is non-recursive unless explicitly changed. + */ +public class CreateFileBuilder extends + FSDataOutputStreamBuilder { + + /** + * Flag set to create with overwrite. + */ + public static final EnumSet CREATE_OVERWRITE_FLAGS = + EnumSet.of(CreateFlag.CREATE, CreateFlag.OVERWRITE); + + /** + * Flag set to create without overwrite. + */ + public static final EnumSet CREATE_NO_OVERWRITE_FLAGS = + EnumSet.of(CreateFlag.CREATE); + + /** + * Classic create file option set: overwriting. + */ + public static final CreateFileOptions OPTIONS_CREATE_FILE_OVERWRITE = + new CreateFileOptions(CREATE_OVERWRITE_FLAGS, true, false, null); + + /** + * Classic create file option set: no overwrite. + */ + public static final CreateFileOptions OPTIONS_CREATE_FILE_NO_OVERWRITE = + new CreateFileOptions(CREATE_NO_OVERWRITE_FLAGS, true, false, null); + + /** + * Callback interface. + */ + private final CreateFileBuilderCallbacks callbacks; + + /** + * Constructor. + * @param fileSystem fs; used by superclass. + * @param path qualified path to create + * @param callbacks callbacks. + */ + public CreateFileBuilder( + @Nonnull final FileSystem fileSystem, + @Nonnull final Path path, + @Nonnull final CreateFileBuilderCallbacks callbacks) { + + super(fileSystem, path); + this.callbacks = callbacks; + } + + @Override + public CreateFileBuilder getThisBuilder() { + return this; + } + + @Override + public FSDataOutputStream build() throws IOException { + Path path = getPath(); + + final Configuration options = getOptions(); + final Map headers = new HashMap<>(); + final Set mandatoryKeys = getMandatoryKeys(); + final Set keysToValidate = new HashSet<>(); + + // pick up all headers from the mandatory list and strip them before + // validating the keys + String headerPrefix = FS_S3A_CREATE_HEADER + "."; + final int prefixLen = headerPrefix.length(); + mandatoryKeys.stream().forEach(key -> { + if (key.startsWith(headerPrefix) && key.length() > prefixLen) { + headers.put(key.substring(prefixLen), options.get(key)); + } else { + keysToValidate.add(key); + } + }); + + rejectUnknownMandatoryKeys(keysToValidate, CREATE_FILE_KEYS, "for " + path); + + // and add any optional headers + getOptionalKeys().stream() + .filter(key -> key.startsWith(headerPrefix) && key.length() > prefixLen) + .forEach(key -> headers.put(key.substring(prefixLen), options.get(key))); + + + EnumSet flags = getFlags(); + if (flags.contains(CreateFlag.APPEND)) { + throw new UnsupportedOperationException("Append is not supported"); + } + if (!flags.contains(CreateFlag.CREATE) && + !flags.contains(CreateFlag.OVERWRITE)) { + throw new PathIOException(path.toString(), + "Must specify either create or overwrite"); + } + + final boolean performance = + options.getBoolean(Constants.FS_S3A_CREATE_PERFORMANCE, false); + return callbacks.createFileFromBuilder( + path, + getProgress(), + new CreateFileOptions(flags, isRecursive(), performance, headers)); + + } + + /** + * Pass flags down. + * @param flags input flags. + * @return this builder. + */ + public CreateFileBuilder withFlags(EnumSet flags) { + if (flags.contains(CreateFlag.CREATE)) { + create(); + } + if (flags.contains(CreateFlag.APPEND)) { + append(); + } + overwrite(flags.contains(CreateFlag.OVERWRITE)); + return this; + } + + /** + * make the flag getter public. + * @return creation flags. + */ + public EnumSet getFlags() { + return super.getFlags(); + } + + /** + * Callbacks for creating the file. + */ + public interface CreateFileBuilderCallbacks { + + /** + * Create a file from the builder. + * @param path path to file + * @param progress progress callback + * @param options options for the file + * @return the stream + * @throws IOException any IO problem + */ + FSDataOutputStream createFileFromBuilder( + Path path, + Progressable progress, + CreateFileOptions options) throws IOException; + } + + /** + * Create file options as built from the builder set or the classic + * entry point. + */ + public static final class CreateFileOptions { + + /** + * creation flags. + * create parent dirs? + * progress callback. + * performance flag. + */ + private final EnumSet flags; + + /** + * create parent dirs? + */ + private final boolean recursive; + + /** + * performance flag. + */ + private final boolean performance; + + /** + * Headers; may be null. + */ + private final Map headers; + + /** + * @param flags creation flags + * @param recursive create parent dirs? + * @param performance performance flag + * @param headers nullable header map. + */ + public CreateFileOptions( + final EnumSet flags, + final boolean recursive, + final boolean performance, + final Map headers) { + this.flags = flags; + this.recursive = recursive; + this.performance = performance; + this.headers = headers; + } + + @Override + public String toString() { + return "CreateFileOptions{" + + "flags=" + flags + + ", recursive=" + recursive + + ", performance=" + performance + + ", headers=" + headers + + '}'; + } + + public EnumSet getFlags() { + return flags; + } + + public boolean isRecursive() { + return recursive; + } + + public boolean isPerformance() { + return performance; + } + + public Map getHeaders() { + return headers; + } + } + +} diff --git a/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/impl/GetContentSummaryOperation.java b/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/impl/GetContentSummaryOperation.java index 248bffb9401fb..257cef8192b2c 100644 --- a/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/impl/GetContentSummaryOperation.java +++ b/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/impl/GetContentSummaryOperation.java @@ -220,8 +220,7 @@ public interface GetContentSummaryCallbacks { /*** * List all entries under a path. - * - * @param path + * @param path path. * @param recursive if the subdirectories need to be traversed recursively * @return an iterator over the listing. * @throws IOException failure diff --git a/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/impl/InternalConstants.java b/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/impl/InternalConstants.java index 49b9feeb6f1f2..6e4946dfb53ac 100644 --- a/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/impl/InternalConstants.java +++ b/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/impl/InternalConstants.java @@ -18,7 +18,9 @@ package org.apache.hadoop.fs.s3a.impl; +import java.util.Arrays; import java.util.Collections; +import java.util.HashSet; import java.util.Set; import java.util.stream.Collectors; import java.util.stream.Stream; @@ -160,4 +162,12 @@ private InternalConstants() { * will go through the AccessPoint. */ public static final String ARN_BUCKET_OPTION = "fs.s3a.bucket.%s.accesspoint.arn"; + + /** + * The known keys used in a createFile call. + */ + public static final Set CREATE_FILE_KEYS = + Collections.unmodifiableSet( + new HashSet<>(Arrays.asList(Constants.FS_S3A_CREATE_PERFORMANCE))); + } diff --git a/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/impl/MkdirOperation.java b/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/impl/MkdirOperation.java index 2b9a0e89b1da4..98a91b1881ba1 100644 --- a/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/impl/MkdirOperation.java +++ b/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/impl/MkdirOperation.java @@ -41,6 +41,18 @@ * It performs the directory listing probe ahead of the simple object HEAD * call for this reason -the object is the failure mode which SHOULD NOT * be encountered on normal execution. + * + * Magic paths are handled specially + *
      + *
    • The only path check is for a directory already existing there.
    • + *
    • No ancestors are checked
    • + *
    • Parent markers are never deleted, irrespective of FS settings
    • + *
    + * As a result, irrespective of depth, the operations performed are only + *
      + *
    1. One LIST
    2. + *
    3. If needed, one PUT
    4. + *
    */ public class MkdirOperation extends ExecutingStoreOperation { @@ -51,13 +63,21 @@ public class MkdirOperation extends ExecutingStoreOperation { private final MkdirCallbacks callbacks; + /** + * Should checks for ancestors existing be skipped? + * This flag is set when working with magic directories. + */ + private final boolean isMagicPath; + public MkdirOperation( final StoreContext storeContext, final Path dir, - final MkdirCallbacks callbacks) { + final MkdirCallbacks callbacks, + final boolean isMagicPath) { super(storeContext); this.dir = dir; this.callbacks = callbacks; + this.isMagicPath = isMagicPath; } /** @@ -77,6 +97,14 @@ public Boolean execute() throws IOException { return true; } + // get the file status of the path. + // this is done even for a magic path, to avoid always issuing PUT + // requests. Doing that without a check wouild seem to be an + // optimization, but it is not because + // 1. PUT is slower than HEAD + // 2. Write capacity is less than read capacity on a shard + // 3. It adds needless entries in versioned buckets, slowing + // down subsequent operations. FileStatus fileStatus = getPathStatusExpectingDir(dir); if (fileStatus != null) { if (fileStatus.isDirectory()) { @@ -85,7 +113,17 @@ public Boolean execute() throws IOException { throw new FileAlreadyExistsException("Path is a file: " + dir); } } - // dir, walk up tree + // file status was null + + // is the path magic? + // If so, we declare success without looking any further + if (isMagicPath) { + // Create the marker file immediately, + // and don't delete markers + callbacks.createFakeDirectory(dir, true); + return true; + } + // Walk path to root, ensuring closest ancestor is a directory, not file Path fPart = dir.getParent(); try { @@ -110,14 +148,15 @@ public Boolean execute() throws IOException { LOG.info("mkdirs({}}: Access denied when looking" + " for parent directory {}; skipping checks", dir, fPart); - LOG.debug("{}", e.toString(), e); + LOG.debug("{}", e, e); } // if we get here there is no directory at the destination. // so create one. - String key = getStoreContext().pathToKey(dir); - // Create the marker file, maybe delete the parent entries - callbacks.createFakeDirectory(key); + + // Create the marker file, delete the parent entries + // if the filesystem isn't configured to retain them + callbacks.createFakeDirectory(dir, false); return true; } @@ -140,15 +179,21 @@ private S3AFileStatus probePathStatusOrNull(final Path path, /** * Get the status of a path -optimized for paths * where there is a directory marker or child entries. + * + * Under a magic path, there's no check for a file, + * just the listing. + * * @param path path to probe. + * * @return the status + * * @throws IOException failure */ private S3AFileStatus getPathStatusExpectingDir(final Path path) throws IOException { S3AFileStatus status = probePathStatusOrNull(path, StatusProbeEnum.DIRECTORIES); - if (status == null) { + if (status == null && !isMagicPath) { status = probePathStatusOrNull(path, StatusProbeEnum.FILE); } @@ -174,10 +219,15 @@ S3AFileStatus probePathStatus(Path path, /** * Create a fake directory, always ending in "/". * Retry policy: retrying; translated. - * @param key name of directory object. + * the keepMarkers flag controls whether or not markers + * are automatically kept (this is set when creating + * directories under a magic path, always) + * @param dir dir to create + * @param keepMarkers always keep markers + * * @throws IOException IO failure */ @Retries.RetryTranslated - void createFakeDirectory(String key) throws IOException; + void createFakeDirectory(Path dir, boolean keepMarkers) throws IOException; } } diff --git a/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/impl/PutObjectOptions.java b/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/impl/PutObjectOptions.java new file mode 100644 index 0000000000000..e14285a1ca8b1 --- /dev/null +++ b/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/impl/PutObjectOptions.java @@ -0,0 +1,104 @@ +/* + * 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.fs.s3a.impl; + +import java.util.Map; +import javax.annotation.Nullable; + +/** + * Extensible structure for options when putting/writing objects. + */ +public final class PutObjectOptions { + + /** + * Can the PUT operation skip marker deletion? + */ + private final boolean keepMarkers; + + /** + * Storage class, if not null. + */ + private final String storageClass; + + /** + * Headers; may be null. + */ + private final Map headers; + + /** + * Constructor. + * @param keepMarkers Can the PUT operation skip marker deletion? + * @param storageClass Storage class, if not null. + * @param headers Headers; may be null. + */ + public PutObjectOptions( + final boolean keepMarkers, + @Nullable final String storageClass, + @Nullable final Map headers) { + this.keepMarkers = keepMarkers; + this.storageClass = storageClass; + this.headers = headers; + } + + /** + * Get the marker retention flag. + * @return true if markers are to be retained. + */ + public boolean isKeepMarkers() { + return keepMarkers; + } + + /** + * Headers for the put/post request. + * @return headers or null. + */ + public Map getHeaders() { + return headers; + } + + @Override + public String toString() { + return "PutObjectOptions{" + + "keepMarkers=" + keepMarkers + + ", storageClass='" + storageClass + '\'' + + '}'; + } + + private static final PutObjectOptions KEEP_DIRS = new PutObjectOptions(true, + null, null); + private static final PutObjectOptions DELETE_DIRS = new PutObjectOptions(false, + null, null); + + /** + * Get the options to keep directories. + * @return an instance which keeps dirs + */ + public static PutObjectOptions keepingDirs() { + return KEEP_DIRS; + } + + /** + * Get the options to delete directory markers. + * @return an instance which deletes dirs + */ + public static PutObjectOptions deletingDirs() { + return DELETE_DIRS; + } + +} \ No newline at end of file diff --git a/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/impl/RequestFactoryImpl.java b/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/impl/RequestFactoryImpl.java index a73e7199380fc..1e6629f9c7343 100644 --- a/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/impl/RequestFactoryImpl.java +++ b/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/impl/RequestFactoryImpl.java @@ -23,7 +23,9 @@ import java.io.InputStream; import java.util.ArrayList; import java.util.List; +import java.util.Map; import java.util.Optional; +import javax.annotation.Nullable; import com.amazonaws.AmazonWebServiceRequest; import com.amazonaws.services.s3.model.AbortMultipartUploadRequest; @@ -46,6 +48,7 @@ import com.amazonaws.services.s3.model.SSEAwsKeyManagementParams; import com.amazonaws.services.s3.model.SSECustomerKey; import com.amazonaws.services.s3.model.SelectObjectContentRequest; +import com.amazonaws.services.s3.model.StorageClass; import com.amazonaws.services.s3.model.UploadPartRequest; import org.apache.hadoop.util.Preconditions; import org.slf4j.Logger; @@ -116,6 +119,11 @@ public class RequestFactoryImpl implements RequestFactory { */ private final String contentEncoding; + /** + * Storage class. + */ + private final StorageClass storageClass; + /** * Constructor. * @param builder builder with all the configuration. @@ -128,6 +136,7 @@ protected RequestFactoryImpl( this.multipartPartCountLimit = builder.multipartPartCountLimit; this.requestPreparer = builder.requestPreparer; this.contentEncoding = builder.contentEncoding; + this.storageClass = builder.storageClass; } /** @@ -200,6 +209,15 @@ public String getContentEncoding() { return contentEncoding; } + /** + * Get the object storage class, return null if none. + * @return storage class + */ + @Override + public StorageClass getStorageClass() { + return storageClass; + } + /** * Sets server side encryption parameters to the part upload * request when encryption is enabled. @@ -343,20 +361,27 @@ protected void copyEncryptionParameters( } /** * Create a putObject request. - * Adds the ACL and metadata + * Adds the ACL, storage class and metadata * @param key key of object * @param metadata metadata header + * @param options options for the request, including headers * @param srcfile source file * @return the request */ @Override public PutObjectRequest newPutObjectRequest(String key, - ObjectMetadata metadata, File srcfile) { + ObjectMetadata metadata, + final PutObjectOptions options, + File srcfile) { Preconditions.checkNotNull(srcfile); PutObjectRequest putObjectRequest = new PutObjectRequest(getBucket(), key, srcfile); + maybeSetMetadata(options, metadata); setOptionalPutRequestParameters(putObjectRequest); putObjectRequest.setCannedAcl(cannedACL); + if (storageClass != null) { + putObjectRequest.setStorageClass(storageClass); + } putObjectRequest.setMetadata(metadata); return prepareRequest(putObjectRequest); } @@ -367,15 +392,18 @@ public PutObjectRequest newPutObjectRequest(String key, * operation. * @param key key of object * @param metadata metadata header + * @param options options for the request * @param inputStream source data. * @return the request */ @Override public PutObjectRequest newPutObjectRequest(String key, ObjectMetadata metadata, + @Nullable final PutObjectOptions options, InputStream inputStream) { Preconditions.checkNotNull(inputStream); Preconditions.checkArgument(isNotEmpty(key), "Null/empty key"); + maybeSetMetadata(options, metadata); PutObjectRequest putObjectRequest = new PutObjectRequest(getBucket(), key, inputStream, metadata); setOptionalPutRequestParameters(putObjectRequest); @@ -399,7 +427,7 @@ public int read() throws IOException { final ObjectMetadata md = createObjectMetadata(0L, true); md.setContentType(HeaderProcessing.CONTENT_TYPE_X_DIRECTORY); PutObjectRequest putObjectRequest = - newPutObjectRequest(key, md, im); + newPutObjectRequest(key, md, null, im); return putObjectRequest; } @@ -425,12 +453,18 @@ public AbortMultipartUploadRequest newAbortMultipartUploadRequest( @Override public InitiateMultipartUploadRequest newMultipartUploadRequest( - String destKey) { + final String destKey, + @Nullable final PutObjectOptions options) { + final ObjectMetadata objectMetadata = newObjectMetadata(-1); + maybeSetMetadata(options, objectMetadata); final InitiateMultipartUploadRequest initiateMPURequest = new InitiateMultipartUploadRequest(getBucket(), destKey, - newObjectMetadata(-1)); + objectMetadata); initiateMPURequest.setCannedACL(getCannedACL()); + if (getStorageClass() != null) { + initiateMPURequest.withStorageClass(getStorageClass()); + } setOptionalMultipartUploadRequestParameters(initiateMPURequest); return prepareRequest(initiateMPURequest); } @@ -579,6 +613,23 @@ public void setEncryptionSecrets(final EncryptionSecrets secrets) { encryptionSecrets = secrets; } + /** + * Set the metadata from the options if the options are not + * null and the metadata contains headers. + * @param options options for the request + * @param objectMetadata metadata to patch + */ + private void maybeSetMetadata( + @Nullable PutObjectOptions options, + final ObjectMetadata objectMetadata) { + if (options != null) { + Map headers = options.getHeaders(); + if (headers != null) { + objectMetadata.setUserMetadata(headers); + } + } + } + /** * Create a builder. * @return new builder. @@ -610,6 +661,11 @@ public static final class RequestFactoryBuilder { /** Content Encoding. */ private String contentEncoding; + /** + * Storage class. + */ + private StorageClass storageClass; + /** * Multipart limit. */ @@ -641,6 +697,16 @@ public RequestFactoryBuilder withContentEncoding(final String value) { return this; } + /** + * Storage class. + * @param value new value + * @return the builder + */ + public RequestFactoryBuilder withStorageClass(final StorageClass value) { + storageClass = value; + return this; + } + /** * Target bucket. * @param value new value diff --git a/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/impl/S3AMultipartUploader.java b/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/impl/S3AMultipartUploader.java index db6beaff5b222..3a6a04e5e711c 100644 --- a/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/impl/S3AMultipartUploader.java +++ b/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/impl/S3AMultipartUploader.java @@ -123,7 +123,8 @@ public CompletableFuture startUpload( String key = context.pathToKey(dest); return context.submit(new CompletableFuture<>(), () -> { - String uploadId = writeOperations.initiateMultiPartUpload(key); + String uploadId = writeOperations.initiateMultiPartUpload(key, + PutObjectOptions.keepingDirs()); statistics.uploadStarted(); return BBUploadHandle.from(ByteBuffer.wrap( uploadId.getBytes(Charsets.UTF_8))); diff --git a/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/statistics/BlockOutputStreamStatistics.java b/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/statistics/BlockOutputStreamStatistics.java index 772b965d4f4a3..bd1466b2a432f 100644 --- a/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/statistics/BlockOutputStreamStatistics.java +++ b/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/statistics/BlockOutputStreamStatistics.java @@ -25,7 +25,8 @@ * Block output stream statistics. */ public interface BlockOutputStreamStatistics extends Closeable, - S3AStatisticInterface { + S3AStatisticInterface, + PutTrackerStatistics { /** * Block is queued for upload. diff --git a/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/statistics/CommitterStatistics.java b/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/statistics/CommitterStatistics.java index fd232a058d0b8..53c25b0c65aea 100644 --- a/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/statistics/CommitterStatistics.java +++ b/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/statistics/CommitterStatistics.java @@ -18,6 +18,8 @@ package org.apache.hadoop.fs.s3a.statistics; +import org.apache.hadoop.fs.statistics.impl.IOStatisticsStore; + /** * Statistics for S3A committers. */ @@ -63,4 +65,10 @@ public interface CommitterStatistics * @param success success flag */ void jobCompleted(boolean success); + + /** + * Return the writeable IOStatisticsStore. + * @return the statistics + */ + IOStatisticsStore getIOStatistics(); } diff --git a/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/statistics/PutTrackerStatistics.java b/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/statistics/PutTrackerStatistics.java new file mode 100644 index 0000000000000..b422e0c9d53aa --- /dev/null +++ b/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/statistics/PutTrackerStatistics.java @@ -0,0 +1,29 @@ +/* + * 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.fs.s3a.statistics; + +/** + * Interface for PUT tracking. + * It is subclassed by {@link BlockOutputStreamStatistics}, + * so that operations performed by the PutTracker update + * the stream statistics. + * Having a separate interface helps isolate operations. + */ +public interface PutTrackerStatistics extends S3AStatisticInterface { +} diff --git a/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/statistics/impl/EmptyS3AStatisticsContext.java b/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/statistics/impl/EmptyS3AStatisticsContext.java index f618270798e08..5c0995e41b3dd 100644 --- a/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/statistics/impl/EmptyS3AStatisticsContext.java +++ b/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/statistics/impl/EmptyS3AStatisticsContext.java @@ -33,6 +33,7 @@ import org.apache.hadoop.fs.s3a.statistics.StatisticsFromAwsSdk; import org.apache.hadoop.fs.statistics.IOStatistics; import org.apache.hadoop.fs.statistics.DurationTracker; +import org.apache.hadoop.fs.statistics.impl.IOStatisticsStore; import static org.apache.hadoop.fs.statistics.impl.IOStatisticsBinding.emptyStatistics; import static org.apache.hadoop.fs.statistics.IOStatisticsSupport.stubDurationTracker; @@ -137,6 +138,7 @@ private static class EmptyS3AStatisticImpl implements public DurationTracker trackDuration(String key, long count) { return stubDurationTracker(); } + } /** @@ -381,6 +383,11 @@ public void taskCompleted(final boolean success) { @Override public void jobCompleted(final boolean success) { } + + @Override + public IOStatisticsStore getIOStatistics() { + return null; + } } private static final class EmptyBlockOutputStreamStatistics diff --git a/hadoop-tools/hadoop-aws/src/site/markdown/tools/hadoop-aws/auditing.md b/hadoop-tools/hadoop-aws/src/site/markdown/tools/hadoop-aws/auditing.md index 7c004627357b4..8ccc36cf83bb1 100644 --- a/hadoop-tools/hadoop-aws/src/site/markdown/tools/hadoop-aws/auditing.md +++ b/hadoop-tools/hadoop-aws/src/site/markdown/tools/hadoop-aws/auditing.md @@ -226,18 +226,25 @@ If any of the field values were `null`, the field is omitted. | `cm` | Command | `S3GuardTool$BucketInfo` | | `fs` | FileSystem ID | `af5943a9-b6f6-4eec-9c58-008982fc492a` | | `id` | Span ID | `3c0d9b7e-2a63-43d9-a220-3c574d768ef3-3` | -| `ji` | Job ID | `(Generated by query engine)` | +| `ji` | Job ID (S3A committer)| `(Generated by query engine)` | | `op` | Filesystem API call | `op_rename` | | `p1` | Path 1 of operation | `s3a://alice-london/path1` | | `p2` | Path 2 of operation | `s3a://alice-london/path2` | | `pr` | Principal | `alice` | | `ps` | Unique process UUID | `235865a0-d399-4696-9978-64568db1b51c` | +| `ta` | Task Attempt ID (S3A committer) | | | `t0` | Thread 0: thread span was created in | `100` | | `t1` | Thread 1: thread this operation was executed in | `200` | | `ts` | Timestamp (UTC epoch millis) | `1617116985923` | +_Notes_ -Thread IDs are from the current thread in the JVM. +* Thread IDs are from the current thread in the JVM, so can be compared to those in````````` + Log4J logs. They are never unique. +* Task Attempt/Job IDs are only ever set during operations involving the S3A committers, specifically + all operations excecuted by the committer. + Operations executed in the same thread as the committer's instantiation _may_ also report the + IDs, even if they are unrelated to the actual task. Consider them "best effort". ```java Long.toString(Thread.currentThread().getId()) diff --git a/hadoop-tools/hadoop-aws/src/site/markdown/tools/hadoop-aws/committers.md b/hadoop-tools/hadoop-aws/src/site/markdown/tools/hadoop-aws/committers.md index b19f30f1a3412..cfeff28d54e87 100644 --- a/hadoop-tools/hadoop-aws/src/site/markdown/tools/hadoop-aws/committers.md +++ b/hadoop-tools/hadoop-aws/src/site/markdown/tools/hadoop-aws/committers.md @@ -549,7 +549,7 @@ Conflict management is left to the execution engine itself. | `fs.s3a.buffer.dir` | Local filesystem directory for data being written and/or staged. | `${env.LOCAL_DIRS:-${hadoop.tmp.dir}}/s3a` | | `fs.s3a.committer.magic.enabled` | Enable "magic committer" support in the filesystem. | `true` | | `fs.s3a.committer.abort.pending.uploads` | list and abort all pending uploads under the destination path when the job is committed or aborted. | `true` | -| `fs.s3a.committer.threads` | Number of threads in committers for parallel operations on files. | 8 | +| `fs.s3a.committer.threads` | Number of threads in committers for parallel operations on files.| -4 | | `fs.s3a.committer.generate.uuid` | Generate a Job UUID if none is passed down from Spark | `false` | | `fs.s3a.committer.require.uuid` |Require the Job UUID to be passed down from Spark | `false` | @@ -587,10 +587,15 @@ Conflict management is left to the execution engine itself. fs.s3a.committer.threads - 8 + -4 Number of threads in committers for parallel operations on files - (upload, commit, abort, delete...) + (upload, commit, abort, delete...). + Two thread pools this size are created, one for the outer + task-level parallelism, and one for parallel execution + within tasks (POSTs to commit individual uploads) + If the value is negative, it is inverted and then multiplied + by the number of cores in the CPU. diff --git a/hadoop-tools/hadoop-aws/src/site/markdown/tools/hadoop-aws/index.md b/hadoop-tools/hadoop-aws/src/site/markdown/tools/hadoop-aws/index.md index f4a2f036ce3fd..7c0a49f8fbeda 100644 --- a/hadoop-tools/hadoop-aws/src/site/markdown/tools/hadoop-aws/index.md +++ b/hadoop-tools/hadoop-aws/src/site/markdown/tools/hadoop-aws/index.md @@ -1079,6 +1079,17 @@ options are covered in [Testing](./testing.md).
    + + fs.s3a.create.storage.class + + + Storage class: standard, reduced_redundancy, intelligent_tiering, etc. + Specify the storage class for S3A PUT object requests. + If not set the storage class will be null + and mapped to default standard class on S3. + + + ``` ## Retry and Recovery @@ -1595,7 +1606,7 @@ Accessing data through an access point, is done by using its ARN, as opposed to You can set the Access Point ARN property using the following per bucket configuration property: ```xml - fs.s3a.sample-bucket.accesspoint.arn + fs.s3a.bucket.sample-bucket.accesspoint.arn {ACCESSPOINT_ARN_HERE} Configure S3a traffic to use this AccessPoint @@ -1605,21 +1616,11 @@ This configures access to the `sample-bucket` bucket for S3A, to go through the new Access Point ARN. So, for example `s3a://sample-bucket/key` will now use your configured ARN when getting data from S3 instead of your bucket. -You can also use an Access Point name as a path URI such as `s3a://finance-team-access/key`, by -configuring the `.accesspoint.arn` property as a per-bucket override: -```xml - - fs.s3a.finance-team-access.accesspoint.arn - {ACCESSPOINT_ARN_HERE} - Configure S3a traffic to use this AccessPoint - -``` - The `fs.s3a.accesspoint.required` property can also require all access to S3 to go through Access Points. This has the advantage of increasing security inside a VPN / VPC as you only allow access to known sources of data defined through Access Points. In case there is a need to access a bucket directly (without Access Points) then you can use per bucket overrides to disable this setting on a -bucket by bucket basis i.e. `fs.s3a.{YOUR-BUCKET}.accesspoint.required`. +bucket by bucket basis i.e. `fs.s3a.bucket.{YOUR-BUCKET}.accesspoint.required`. ```xml @@ -1629,7 +1630,7 @@ bucket by bucket basis i.e. `fs.s3a.{YOUR-BUCKET}.accesspoint.required`. - fs.s3a.example-bucket.accesspoint.required + fs.s3a.bucket.example-bucket.accesspoint.required false ``` @@ -1660,6 +1661,26 @@ To enable this feature within S3A, configure the `fs.s3a.requester.pays.enabled` ``` +## Storage Classes + +Amazon S3 offers a range of [Storage Classes](https://aws.amazon.com/s3/storage-classes/) +that you can choose from based on behavior of your applications. By using the right +storage class, you can reduce the cost of your bucket. + +S3A uses Standard storage class for PUT object requests by default, which is suitable for +general use cases. To use a specific storage class, set the value in `fs.s3a.create.storage.class` property to +the storage class you want. + +```xml + + fs.s3a.create.storage.class + intelligent_tiering + +``` + +Please note that S3A does not support reading from archive storage classes at the moment. +`AccessDeniedException` with InvalidObjectState will be thrown if you're trying to do so. + ## How S3A writes data to S3 The original S3A client implemented file writes by diff --git a/hadoop-tools/hadoop-aws/src/site/markdown/tools/hadoop-aws/performance.md b/hadoop-tools/hadoop-aws/src/site/markdown/tools/hadoop-aws/performance.md index f398c4cbcbe37..06eb137cd9bd9 100644 --- a/hadoop-tools/hadoop-aws/src/site/markdown/tools/hadoop-aws/performance.md +++ b/hadoop-tools/hadoop-aws/src/site/markdown/tools/hadoop-aws/performance.md @@ -55,6 +55,36 @@ it isn't, and some attempts to preserve the metaphor are "aggressively suboptima To make most efficient use of S3, care is needed. +## Improving read performance using Vectored IO +The S3A FileSystem supports implementation of vectored read api using which +a client can provide a list of file ranges to read returning a future read +object associated with each range. For full api specification please see +[FSDataInputStream](../../hadoop-common-project/hadoop-common/filesystem/fsdatainputstream.html). + +The following properties can be configured to optimise vectored reads based +on the client requirements. + +```xml + + fs.s3a.vectored.read.min.seek.size + 4K + + What is the smallest reasonable seek in bytes such + that we group ranges together during vectored + read operation. + + + +fs.s3a.vectored.read.max.merged.size +1M + + What is the largest merged read size in bytes such + that we group ranges together during vectored read. + Setting this value to 0 will disable merging of ranges. + + +``` + ## Improving data input performance through fadvise The S3A Filesystem client supports the notion of input policies, similar diff --git a/hadoop-tools/hadoop-aws/src/site/markdown/tools/hadoop-aws/testing.md b/hadoop-tools/hadoop-aws/src/site/markdown/tools/hadoop-aws/testing.md index e1761538f21bc..44525c8ccd943 100644 --- a/hadoop-tools/hadoop-aws/src/site/markdown/tools/hadoop-aws/testing.md +++ b/hadoop-tools/hadoop-aws/src/site/markdown/tools/hadoop-aws/testing.md @@ -572,6 +572,18 @@ can be turned off. Encryption is only used for those specific test suites with `Encryption` in their classname. +### Disabling the storage class tests + +When running storage class tests against third party object store that doesn't support +S3 storage class, these tests might fail. They can be disabled. + +```xml + + test.fs.s3a.create.storage.class.enabled + false + +``` + ### Configuring the CSV file read tests** To test on alternate infrastructures supporting diff --git a/hadoop-tools/hadoop-aws/src/site/markdown/tools/hadoop-aws/troubleshooting_s3a.md b/hadoop-tools/hadoop-aws/src/site/markdown/tools/hadoop-aws/troubleshooting_s3a.md index 96e6e287deaaf..903310a94bfe8 100644 --- a/hadoop-tools/hadoop-aws/src/site/markdown/tools/hadoop-aws/troubleshooting_s3a.md +++ b/hadoop-tools/hadoop-aws/src/site/markdown/tools/hadoop-aws/troubleshooting_s3a.md @@ -561,24 +561,41 @@ Error Code: 403 Forbidden; Request ID: myshortreqid; S3 Extended Request ID: myl To enable requester pays, set `fs.s3a.requester.pays.enabled` property to `true`. +### `AccessDeniedException` "InvalidObjectState" when trying to read files + +``` +java.nio.file.AccessDeniedException: file1: copyFile(file1, file2) on file1: com.amazonaws.services.s3.model.AmazonS3Exception: Operation is not valid for the source object's storage class (Service: Amazon S3; Status Code: 403; Error Code: InvalidObjectState; Request ID: SK9EMPC1YRX75VZR; S3 Extended Request ID: /nhUfdwJ+y5DLz6B4YR2FdA0FnQWwhDAkSCakn42zs2JssK3qWTrfwdNDiy6bOyXHOvJY0VAlHw=; Proxy: null), S3 Extended Request ID: /nhUfdwJ+y5DLz6B4YR2FdA0FnQWwhDAkSCakn42zs2JssK3qWTrfwdNDiy6bOyXHOvJY0VAlHw=:InvalidObjectState + +Caused by: com.amazonaws.services.s3.model.AmazonS3Exception: Operation is not valid for the source object's storage class (Service: Amazon S3; Status Code: 403; Error Code: InvalidObjectState; Request ID: SK9EMPC1YRX75VZR; S3 Extended Request ID: /nhUfdwJ+y5DLz6B4YR2FdA0FnQWwhDAkSCakn42zs2JssK3qWTrfwdNDiy6bOyXHOvJY0VAlHw=; Proxy: null), S3 Extended Request ID: /nhUfdwJ+y5DLz6B4YR2FdA0FnQWwhDAkSCakn42zs2JssK3qWTrfwdNDiy6bOyXHOvJY0VAlHw= +``` + +This happens when you're trying to read or copy files that have archive storage class such as +Glacier. + +If you want to access the file with S3A after writes, do not set `fs.s3a.create.storage.class` to `glacier` or `deep_archive`. + ### "Unable to find a region via the region provider chain." when using session credentials. -Region must be provided when requesting session credentials, or an exception will be thrown with the message: +Region must be provided when requesting session credentials, or an exception will be thrown with the +message: + ``` com.amazonaws.SdkClientException: Unable to find a region via the region provider chain. Must provide an explicit region in the builder or setup environment to supply a region. ``` -In this case you have to set the `fs.s3a.assumed.role.sts.endpoint` property to a valid -S3 sts endpoint and region like the following: + +In this case you have to set the `fs.s3a.assumed.role.sts.endpoint` property to a valid S3 sts +endpoint and region like the following: ```xml + fs.s3a.assumed.role.sts.endpoint ${sts.endpoint} - fs.s3a.assumed.role.sts.endpoint.region - ${sts.region} +fs.s3a.assumed.role.sts.endpoint.region +${sts.region} ``` diff --git a/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/contract/s3a/ITestS3AContractVectoredRead.java b/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/contract/s3a/ITestS3AContractVectoredRead.java new file mode 100644 index 0000000000000..18a727dcdceed --- /dev/null +++ b/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/contract/s3a/ITestS3AContractVectoredRead.java @@ -0,0 +1,159 @@ +/* + * 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.fs.contract.s3a; + +import java.io.EOFException; +import java.io.InterruptedIOException; +import java.util.ArrayList; +import java.util.List; + +import org.junit.Test; + +import org.apache.hadoop.conf.Configuration; +import org.apache.hadoop.fs.FSDataInputStream; +import org.apache.hadoop.fs.FileRange; +import org.apache.hadoop.fs.FileSystem; +import org.apache.hadoop.fs.contract.AbstractContractVectoredReadTest; +import org.apache.hadoop.fs.contract.AbstractFSContract; +import org.apache.hadoop.fs.s3a.Constants; +import org.apache.hadoop.fs.s3a.S3AFileSystem; +import org.apache.hadoop.fs.s3a.S3ATestUtils; +import org.apache.hadoop.test.LambdaTestUtils; + +import static org.apache.hadoop.fs.contract.ContractTestUtils.validateVectoredReadResult; +import static org.apache.hadoop.test.MoreAsserts.assertEqual; + +public class ITestS3AContractVectoredRead extends AbstractContractVectoredReadTest { + + public ITestS3AContractVectoredRead(String bufferType) { + super(bufferType); + } + + @Override + protected AbstractFSContract createContract(Configuration conf) { + return new S3AContract(conf); + } + + /** + * Overriding in S3 vectored read api fails fast in case of EOF + * requested range. + */ + @Override + public void testEOFRanges() throws Exception { + FileSystem fs = getFileSystem(); + List fileRanges = new ArrayList<>(); + fileRanges.add(FileRange.createFileRange(DATASET_LEN, 100)); + verifyExceptionalVectoredRead(fs, fileRanges, EOFException.class); + } + + @Test + public void testMinSeekAndMaxSizeConfigsPropagation() throws Exception { + Configuration conf = getFileSystem().getConf(); + S3ATestUtils.removeBaseAndBucketOverrides(conf, + Constants.AWS_S3_VECTOR_READS_MAX_MERGED_READ_SIZE, + Constants.AWS_S3_VECTOR_READS_MIN_SEEK_SIZE); + S3ATestUtils.disableFilesystemCaching(conf); + final int configuredMinSeek = 2 * 1024; + final int configuredMaxSize = 10 * 1024 * 1024; + conf.set(Constants.AWS_S3_VECTOR_READS_MIN_SEEK_SIZE, "2K"); + conf.set(Constants.AWS_S3_VECTOR_READS_MAX_MERGED_READ_SIZE, "10M"); + try (S3AFileSystem fs = S3ATestUtils.createTestFileSystem(conf)) { + try (FSDataInputStream fis = fs.open(path(VECTORED_READ_FILE_NAME))) { + int newMinSeek = fis.minSeekForVectorReads(); + int newMaxSize = fis.maxReadSizeForVectorReads(); + assertEqual(newMinSeek, configuredMinSeek, + "configured s3a min seek for vectored reads"); + assertEqual(newMaxSize, configuredMaxSize, + "configured s3a max size for vectored reads"); + } + } + } + + @Test + public void testMinSeekAndMaxSizeDefaultValues() throws Exception { + Configuration conf = getFileSystem().getConf(); + S3ATestUtils.removeBaseAndBucketOverrides(conf, + Constants.AWS_S3_VECTOR_READS_MIN_SEEK_SIZE, + Constants.AWS_S3_VECTOR_READS_MAX_MERGED_READ_SIZE); + try (S3AFileSystem fs = S3ATestUtils.createTestFileSystem(conf)) { + try (FSDataInputStream fis = fs.open(path(VECTORED_READ_FILE_NAME))) { + int minSeek = fis.minSeekForVectorReads(); + int maxSize = fis.maxReadSizeForVectorReads(); + assertEqual(minSeek, Constants.DEFAULT_AWS_S3_VECTOR_READS_MIN_SEEK_SIZE, + "default s3a min seek for vectored reads"); + assertEqual(maxSize, Constants.DEFAULT_AWS_S3_VECTOR_READS_MAX_MERGED_READ_SIZE, + "default s3a max read size for vectored reads"); + } + } + } + + @Test + public void testStopVectoredIoOperationsCloseStream() throws Exception { + FileSystem fs = getFileSystem(); + List fileRanges = createSampleNonOverlappingRanges(); + try (FSDataInputStream in = fs.open(path(VECTORED_READ_FILE_NAME))){ + in.readVectored(fileRanges, getAllocate()); + in.close(); + LambdaTestUtils.intercept(InterruptedIOException.class, + () -> validateVectoredReadResult(fileRanges, DATASET)); + } + // reopening the stream should succeed. + try (FSDataInputStream in = fs.open(path(VECTORED_READ_FILE_NAME))){ + in.readVectored(fileRanges, getAllocate()); + validateVectoredReadResult(fileRanges, DATASET); + } + } + + @Test + public void testStopVectoredIoOperationsUnbuffer() throws Exception { + FileSystem fs = getFileSystem(); + List fileRanges = createSampleNonOverlappingRanges(); + try (FSDataInputStream in = fs.open(path(VECTORED_READ_FILE_NAME))){ + in.readVectored(fileRanges, getAllocate()); + in.unbuffer(); + LambdaTestUtils.intercept(InterruptedIOException.class, + () -> validateVectoredReadResult(fileRanges, DATASET)); + // re-initiating the vectored reads after unbuffer should succeed. + in.readVectored(fileRanges, getAllocate()); + validateVectoredReadResult(fileRanges, DATASET); + } + + } + + /** + * S3 vectored IO doesn't support overlapping ranges. + */ + @Override + public void testOverlappingRanges() throws Exception { + FileSystem fs = getFileSystem(); + List fileRanges = getSampleOverlappingRanges(); + verifyExceptionalVectoredRead(fs, fileRanges, UnsupportedOperationException.class); + } + + /** + * S3 vectored IO doesn't support overlapping ranges. + */ + @Override + public void testSameRanges() throws Exception { + // Same ranges are special case of overlapping only. + FileSystem fs = getFileSystem(); + List fileRanges = getSampleSameRanges(); + verifyExceptionalVectoredRead(fs, fileRanges, UnsupportedOperationException.class); + } +} diff --git a/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/ITestS3AFileOperationCost.java b/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/ITestS3AFileOperationCost.java index 27c70b2b2148d..dae6312d48098 100644 --- a/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/ITestS3AFileOperationCost.java +++ b/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/ITestS3AFileOperationCost.java @@ -18,8 +18,6 @@ package org.apache.hadoop.fs.s3a; - -import org.apache.hadoop.fs.FileAlreadyExistsException; import org.apache.hadoop.fs.FileSystem; import org.apache.hadoop.fs.Path; import org.apache.hadoop.fs.s3a.impl.StatusProbeEnum; @@ -369,67 +367,6 @@ public void testNeedEmptyDirectoryProbeRequiresList() throws Throwable { fs.s3GetFileStatus(new Path("/something"), "/something", StatusProbeEnum.HEAD_ONLY, true)); } - @Test - public void testCreateCost() throws Throwable { - describe("Test file creation cost"); - Path testFile = methodPath(); - // when overwrite is false, the path is checked for existence. - create(testFile, false, - CREATE_FILE_NO_OVERWRITE); - // but when true: only the directory checks take place. - create(testFile, true, CREATE_FILE_OVERWRITE); - } - - @Test - public void testCreateCostFileExists() throws Throwable { - describe("Test cost of create file failing with existing file"); - Path testFile = file(methodPath()); - - // now there is a file there, an attempt with overwrite == false will - // fail on the first HEAD. - interceptOperation(FileAlreadyExistsException.class, "", - FILE_STATUS_FILE_PROBE, - () -> file(testFile, false)); - } - - @Test - public void testCreateCostDirExists() throws Throwable { - describe("Test cost of create file failing with existing dir"); - Path testFile = dir(methodPath()); - - // now there is a file there, an attempt with overwrite == false will - // fail on the first HEAD. - interceptOperation(FileAlreadyExistsException.class, "", - GET_FILE_STATUS_ON_DIR_MARKER, - () -> file(testFile, false)); - } - - /** - * Use the builder API. - * This always looks for a parent unless the caller says otherwise. - */ - @Test - public void testCreateBuilder() throws Throwable { - describe("Test builder file creation cost"); - Path testFile = methodPath(); - dir(testFile.getParent()); - - // builder defaults to looking for parent existence (non-recursive) - buildFile(testFile, false, false, - GET_FILE_STATUS_FNFE // destination file - .plus(FILE_STATUS_DIR_PROBE)); // parent dir - // recursive = false and overwrite=true: - // only make sure the dest path isn't a directory. - buildFile(testFile, true, true, - FILE_STATUS_DIR_PROBE); - - // now there is a file there, an attempt with overwrite == false will - // fail on the first HEAD. - interceptOperation(FileAlreadyExistsException.class, "", - GET_FILE_STATUS_ON_FILE, - () -> buildFile(testFile, false, true, - GET_FILE_STATUS_ON_FILE)); - } @Test public void testCostOfGlobStatus() throws Throwable { diff --git a/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/ITestS3AMiscOperations.java b/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/ITestS3AMiscOperations.java index b4d2527a46af8..2d29282ad0195 100644 --- a/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/ITestS3AMiscOperations.java +++ b/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/ITestS3AMiscOperations.java @@ -36,8 +36,8 @@ import org.apache.hadoop.conf.Configuration; import org.apache.hadoop.fs.CommonPathCapabilities; import org.apache.hadoop.fs.FSDataOutputStream; -import org.apache.hadoop.fs.FileAlreadyExistsException; import org.apache.hadoop.fs.Path; +import org.apache.hadoop.fs.s3a.impl.PutObjectOptions; import org.apache.hadoop.fs.store.audit.AuditSpan; import org.apache.hadoop.fs.store.EtagChecksum; import org.apache.hadoop.test.LambdaTestUtils; @@ -102,18 +102,6 @@ public void testCreateNonRecursiveSuccess() throws IOException { assertIsFile(shouldWork); } - @Test(expected = FileNotFoundException.class) - public void testCreateNonRecursiveNoParent() throws IOException { - createNonRecursive(path("/recursive/node")); - } - - @Test(expected = FileAlreadyExistsException.class) - public void testCreateNonRecursiveParentIsFile() throws IOException { - Path parent = path("/file.txt"); - touch(getFileSystem(), parent); - createNonRecursive(new Path(parent, "fail")); - } - @Test public void testPutObjectDirect() throws Throwable { final S3AFileSystem fs = getFileSystem(); @@ -126,7 +114,7 @@ public void testPutObjectDirect() throws Throwable { new ByteArrayInputStream("PUT".getBytes()), metadata); LambdaTestUtils.intercept(IllegalStateException.class, - () -> fs.putObjectDirect(put)); + () -> fs.putObjectDirect(put, PutObjectOptions.keepingDirs())); assertPathDoesNotExist("put object was created", path); } } diff --git a/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/ITestS3AStorageClass.java b/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/ITestS3AStorageClass.java new file mode 100644 index 0000000000000..e141ef5aa32f3 --- /dev/null +++ b/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/ITestS3AStorageClass.java @@ -0,0 +1,217 @@ +/* + * 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.fs.s3a; + +import java.nio.file.AccessDeniedException; +import java.util.Map; + +import org.assertj.core.api.Assertions; +import org.junit.Test; + +import org.apache.hadoop.conf.Configuration; +import org.apache.hadoop.fs.FileSystem; +import org.apache.hadoop.fs.Path; +import org.apache.hadoop.fs.contract.ContractTestUtils; +import org.apache.hadoop.fs.contract.s3a.S3AContract; + +import static org.apache.hadoop.fs.s3a.Constants.STORAGE_CLASS; +import static org.apache.hadoop.fs.s3a.Constants.STORAGE_CLASS_GLACIER; +import static org.apache.hadoop.fs.s3a.Constants.STORAGE_CLASS_REDUCED_REDUNDANCY; +import static org.apache.hadoop.fs.s3a.S3ATestUtils.disableFilesystemCaching; +import static org.apache.hadoop.fs.s3a.S3ATestUtils.removeBaseAndBucketOverrides; +import static org.apache.hadoop.fs.s3a.S3ATestUtils.skipIfStorageClassTestsDisabled; +import static org.apache.hadoop.fs.s3a.impl.HeaderProcessing.XA_STORAGE_CLASS; +import static org.apache.hadoop.fs.s3a.impl.HeaderProcessing.decodeBytes; +import static org.apache.hadoop.test.LambdaTestUtils.intercept; + +/** + * Tests of storage class. + */ +public class ITestS3AStorageClass extends AbstractS3ATestBase { + + @Override + protected Configuration createConfiguration() { + Configuration conf = super.createConfiguration(); + disableFilesystemCaching(conf); + removeBaseAndBucketOverrides(conf, STORAGE_CLASS); + + return conf; + } + + @Override + public void setup() throws Exception { + super.setup(); + skipIfStorageClassTestsDisabled(getConfiguration()); + } + + /* + * This test ensures the default storage class configuration (no config or null) + * works well with create and copy operations + */ + @Test + public void testCreateAndCopyObjectWithStorageClassDefault() throws Throwable { + Configuration conf = this.createConfiguration(); + S3AContract contract = (S3AContract) createContract(conf); + contract.init(); + + FileSystem fs = contract.getTestFileSystem(); + Path dir = methodPath(); + fs.mkdirs(dir); + assertObjectHasNoStorageClass(dir); + Path path = new Path(dir, "file1"); + ContractTestUtils.touch(fs, path); + assertObjectHasNoStorageClass(path); + Path path2 = new Path(dir, "file1"); + fs.rename(path, path2); + assertObjectHasNoStorageClass(path2); + } + + /* + * Verify object can be created and copied correctly + * with specified storage class + */ + @Test + public void testCreateAndCopyObjectWithStorageClassReducedRedundancy() throws Throwable { + Configuration conf = this.createConfiguration(); + conf.set(STORAGE_CLASS, STORAGE_CLASS_REDUCED_REDUNDANCY); + S3AContract contract = (S3AContract) createContract(conf); + contract.init(); + + FileSystem fs = contract.getTestFileSystem(); + Path dir = methodPath(); + fs.mkdirs(dir); + // even with storage class specified + // directories do not have storage class + assertObjectHasNoStorageClass(dir); + Path path = new Path(dir, "file1"); + ContractTestUtils.touch(fs, path); + assertObjectHasStorageClass(path, STORAGE_CLASS_REDUCED_REDUNDANCY); + Path path2 = new Path(dir, "file1"); + fs.rename(path, path2); + assertObjectHasStorageClass(path2, STORAGE_CLASS_REDUCED_REDUNDANCY); + } + + /* + * Archive storage classes have different behavior + * from general storage classes + */ + @Test + public void testCreateAndCopyObjectWithStorageClassGlacier() throws Throwable { + Configuration conf = this.createConfiguration(); + conf.set(STORAGE_CLASS, STORAGE_CLASS_GLACIER); + S3AContract contract = (S3AContract) createContract(conf); + contract.init(); + + FileSystem fs = contract.getTestFileSystem(); + Path dir = methodPath(); + fs.mkdirs(dir); + // even with storage class specified + // directories do not have storage class + assertObjectHasNoStorageClass(dir); + Path path = new Path(dir, "file1"); + ContractTestUtils.touch(fs, path); + assertObjectHasStorageClass(path, STORAGE_CLASS_GLACIER); + Path path2 = new Path(dir, "file2"); + + // this is the current behavior + // object with archive storage class can't be read directly + // when trying to read it, AccessDeniedException will be thrown + // with message InvalidObjectState + intercept(AccessDeniedException.class, "InvalidObjectState", () -> fs.rename(path, path2)); + } + + /* + * Verify object can be created and copied correctly + * with completely invalid storage class + */ + @Test + public void testCreateAndCopyObjectWithStorageClassInvalid() throws Throwable { + Configuration conf = this.createConfiguration(); + conf.set(STORAGE_CLASS, "testing"); + S3AContract contract = (S3AContract) createContract(conf); + contract.init(); + + FileSystem fs = contract.getTestFileSystem(); + Path dir = methodPath(); + fs.mkdirs(dir); + // even with storage class specified + // directories do not have storage class + assertObjectHasNoStorageClass(dir); + Path path = new Path(dir, "file1"); + ContractTestUtils.touch(fs, path); + assertObjectHasNoStorageClass(path); + Path path2 = new Path(dir, "file1"); + fs.rename(path, path2); + assertObjectHasNoStorageClass(path2); + } + + /* + * Verify object can be created and copied correctly + * with empty string configuration + */ + @Test + public void testCreateAndCopyObjectWithStorageClassEmpty() throws Throwable { + Configuration conf = this.createConfiguration(); + conf.set(STORAGE_CLASS, ""); + S3AContract contract = (S3AContract) createContract(conf); + contract.init(); + + FileSystem fs = contract.getTestFileSystem(); + Path dir = methodPath(); + fs.mkdirs(dir); + // even with storage class specified + // directories do not have storage class + assertObjectHasNoStorageClass(dir); + Path path = new Path(dir, "file1"); + ContractTestUtils.touch(fs, path); + assertObjectHasNoStorageClass(path); + Path path2 = new Path(dir, "file1"); + fs.rename(path, path2); + assertObjectHasNoStorageClass(path2); + } + + /** + * Assert that a given object has no storage class specified. + * + * @param path path + */ + protected void assertObjectHasNoStorageClass(Path path) throws Throwable { + S3AFileSystem fs = getFileSystem(); + Map xAttrs = fs.getXAttrs(path); + String storageClass = decodeBytes(xAttrs.get(XA_STORAGE_CLASS)); + + Assertions.assertThat(storageClass).describedAs("Storage class of object %s", path).isNull(); + } + + /** + * Assert that a given object has the given storage class specified. + * + * @param path path + * @param expectedStorageClass expected storage class for the object + */ + protected void assertObjectHasStorageClass(Path path, String expectedStorageClass) + throws Throwable { + S3AFileSystem fs = getFileSystem(); + Map xAttrs = fs.getXAttrs(path); + String actualStorageClass = decodeBytes(xAttrs.get(XA_STORAGE_CLASS)); + + Assertions.assertThat(actualStorageClass).describedAs("Storage class of object %s", path) + .isEqualToIgnoringCase(expectedStorageClass); + } +} diff --git a/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/MockS3AFileSystem.java b/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/MockS3AFileSystem.java index b597460bfd75a..2eb35daab4c2b 100644 --- a/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/MockS3AFileSystem.java +++ b/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/MockS3AFileSystem.java @@ -40,7 +40,11 @@ import org.apache.hadoop.fs.s3a.audit.AuditTestSupport; import org.apache.hadoop.fs.s3a.auth.delegation.EncryptionSecrets; import org.apache.hadoop.fs.s3a.commit.staging.StagingTestBase; +import org.apache.hadoop.fs.s3a.impl.PutObjectOptions; import org.apache.hadoop.fs.s3a.impl.RequestFactoryImpl; +import org.apache.hadoop.fs.s3a.impl.StoreContext; +import org.apache.hadoop.fs.s3a.impl.StoreContextBuilder; +import org.apache.hadoop.fs.s3a.impl.StubContextAccessor; import org.apache.hadoop.fs.s3a.statistics.CommitterStatistics; import org.apache.hadoop.fs.s3a.statistics.impl.EmptyS3AStatisticsContext; import org.apache.hadoop.fs.statistics.DurationTrackerFactory; @@ -211,7 +215,11 @@ public boolean exists(Path f) throws IOException { } @Override - void finishedWrite(String key, long length, String eTag, String versionId) { + void finishedWrite(String key, + long length, + String eTag, + String versionId, + final PutObjectOptions putOptions) { } @@ -377,11 +385,29 @@ public CommitterStatistics newCommitterStatistics() { @Override public void operationRetried(Exception ex) { - /** no-op */ + /* no-op */ } @Override protected DurationTrackerFactory getDurationTrackerFactory() { return stubDurationTrackerFactory(); } + + /** + * Build an immutable store context. + * If called while the FS is being initialized, + * some of the context will be incomplete. + * new store context instances should be created as appropriate. + * @return the store context of this FS. + */ + public StoreContext createStoreContext() { + return new StoreContextBuilder().setFsURI(getUri()) + .setBucket(getBucket()) + .setConfiguration(getConf()) + .setUsername(getUsername()) + .setAuditor(getAuditor()) + .setContextAccessors(new StubContextAccessor(getBucket())) + .build(); + } + } diff --git a/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/MultipartTestUtils.java b/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/MultipartTestUtils.java index 04c2b2a09bda2..9ea33cf69c92b 100644 --- a/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/MultipartTestUtils.java +++ b/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/MultipartTestUtils.java @@ -22,6 +22,7 @@ import com.amazonaws.services.s3.model.PartETag; import com.amazonaws.services.s3.model.UploadPartRequest; import org.apache.hadoop.fs.Path; +import org.apache.hadoop.fs.s3a.impl.PutObjectOptions; import org.apache.hadoop.fs.store.audit.AuditSpan; import org.apache.hadoop.io.IOUtils; @@ -78,7 +79,7 @@ public static IdKey createPartUpload(S3AFileSystem fs, String key, int len, WriteOperationHelper writeHelper = fs.getWriteOperationHelper(); byte[] data = dataset(len, 'a', 'z'); InputStream in = new ByteArrayInputStream(data); - String uploadId = writeHelper.initiateMultiPartUpload(key); + String uploadId = writeHelper.initiateMultiPartUpload(key, PutObjectOptions.keepingDirs()); UploadPartRequest req = writeHelper.newUploadPartRequest(key, uploadId, partNo, len, in, null, 0L); PartETag partEtag = writeHelper.uploadPart(req).getPartETag(); diff --git a/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/S3ATestConstants.java b/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/S3ATestConstants.java index 6a74338cdeb62..a6269c437665a 100644 --- a/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/S3ATestConstants.java +++ b/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/S3ATestConstants.java @@ -53,6 +53,11 @@ public interface S3ATestConstants { */ String KEY_ENCRYPTION_TESTS = TEST_FS_S3A + "encryption.enabled"; + /** + * A property set to true if storage class tests are enabled: {@value }. + */ + String KEY_STORAGE_CLASS_TESTS_ENABLED = TEST_FS_S3A + "create.storage.class.enabled"; + /** * Tell tests that they are being executed in parallel: {@value}. */ @@ -240,4 +245,10 @@ public interface S3ATestConstants { * used. */ int KMS_KEY_GENERATION_REQUEST_PARAMS_BYTES_WRITTEN = 94; + + /** + * Build directory property. + * Value: {@value}. + */ + String PROJECT_BUILD_DIRECTORY_PROPERTY = "project.build.directory"; } diff --git a/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/S3ATestUtils.java b/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/S3ATestUtils.java index e20de5edf06ff..48cb52c5ac29c 100644 --- a/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/S3ATestUtils.java +++ b/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/S3ATestUtils.java @@ -35,8 +35,6 @@ import org.apache.hadoop.fs.s3a.auth.MarshalledCredentialBinding; import org.apache.hadoop.fs.s3a.auth.MarshalledCredentials; import org.apache.hadoop.fs.s3a.auth.delegation.EncryptionSecrets; -import org.apache.hadoop.fs.s3a.commit.CommitConstants; - import org.apache.hadoop.fs.s3a.impl.ChangeDetectionPolicy; import org.apache.hadoop.fs.s3a.impl.ContextAccessors; import org.apache.hadoop.fs.s3a.impl.StatusProbeEnum; @@ -465,6 +463,17 @@ public static void skipIfEncryptionTestsDisabled( } } + /** + * Skip a test if storage class tests are disabled. + * @param configuration configuration to probe + */ + public static void skipIfStorageClassTestsDisabled( + Configuration configuration) { + if (!configuration.getBoolean(KEY_STORAGE_CLASS_TESTS_ENABLED, true)) { + skip("Skipping storage class tests"); + } + } + /** * Create a test path, using the value of * {@link S3ATestConstants#TEST_UNIQUE_FORK_ID} if it is set. @@ -566,6 +575,19 @@ public static Configuration prepareTestConfiguration(final Configuration conf) { return conf; } + /** + * build dir. + * @return the directory for the project's build, as set by maven, + * falling back to pwd + "target" if running from an IDE; + */ + public static File getProjectBuildDir() { + String propval = System.getProperty(PROJECT_BUILD_DIRECTORY_PROPERTY); + if (StringUtils.isEmpty(propval)) { + propval = "target"; + } + return new File(propval).getAbsoluteFile(); + } + /** * Clear any Hadoop credential provider path. * This is needed if people's test setups switch to credential providers, @@ -1290,18 +1312,6 @@ public static long lsR(FileSystem fileSystem, Path path, boolean recursive) public static final DateFormat LISTING_FORMAT = new SimpleDateFormat( "yyyy-MM-dd HH:mm:ss"); - /** - * Skip a test if the FS isn't marked as supporting magic commits. - * @param fs filesystem - */ - public static void assumeMagicCommitEnabled(S3AFileSystem fs) - throws IOException { - assume("Magic commit option disabled on " + fs, - fs.hasPathCapability( - fs.getWorkingDirectory(), - CommitConstants.STORE_CAPABILITY_MAGIC_COMMITTER)); - } - /** * Probe for the configuration containing a specific credential provider. * If the list is empty, there will be no match, even if the named provider diff --git a/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/TestS3ABlockOutputStream.java b/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/TestS3ABlockOutputStream.java index 21f268dfb2ed5..08f4f8bc9df5c 100644 --- a/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/TestS3ABlockOutputStream.java +++ b/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/TestS3ABlockOutputStream.java @@ -22,6 +22,7 @@ import org.apache.hadoop.fs.PathIOException; import org.apache.hadoop.fs.s3a.audit.AuditTestSupport; import org.apache.hadoop.fs.s3a.commit.PutTracker; +import org.apache.hadoop.fs.s3a.impl.PutObjectOptions; import org.apache.hadoop.fs.s3a.statistics.impl.EmptyS3AStatisticsContext; import org.apache.hadoop.util.Progressable; import org.junit.Before; @@ -65,7 +66,8 @@ private S3ABlockOutputStream.BlockOutputStreamBuilder mockS3ABuilder() { .withKey("") .withProgress(progressable) .withPutTracker(putTracker) - .withWriteOperations(oHelper); + .withWriteOperations(oHelper) + .withPutOptions(PutObjectOptions.keepingDirs()); return builder; } diff --git a/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/TestS3AInputStreamRetry.java b/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/TestS3AInputStreamRetry.java index 62f5bff35c4fa..c62bf5daca3a4 100644 --- a/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/TestS3AInputStreamRetry.java +++ b/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/TestS3AInputStreamRetry.java @@ -111,7 +111,8 @@ private S3AInputStream getMockedS3AInputStream() { s3AReadOpContext, s3ObjectAttributes, getMockedInputStreamCallback(), - s3AReadOpContext.getS3AStatisticsContext().newInputStreamStatistics()); + s3AReadOpContext.getS3AStatisticsContext().newInputStreamStatistics(), + null); } /** diff --git a/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/audit/ITestAuditAccessChecks.java b/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/audit/ITestAuditAccessChecks.java index a5a1b454d03ad..4b0b65fd783ae 100644 --- a/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/audit/ITestAuditAccessChecks.java +++ b/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/audit/ITestAuditAccessChecks.java @@ -75,7 +75,9 @@ public Configuration createConfiguration() { @Override public void setup() throws Exception { super.setup(); - auditor = (AccessCheckingAuditor) getFileSystem().getAuditor(); + final S3AFileSystem fs = getFileSystem(); + auditor = (AccessCheckingAuditor) fs.getAuditor(); + setSpanSource(fs); } @Test diff --git a/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/auth/ITestAssumeRole.java b/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/auth/ITestAssumeRole.java index 86bfa2bb07d7d..51eac7e8cc349 100644 --- a/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/auth/ITestAssumeRole.java +++ b/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/auth/ITestAssumeRole.java @@ -47,9 +47,10 @@ import org.apache.hadoop.fs.s3a.S3ATestConstants; import org.apache.hadoop.fs.s3a.SimpleAWSCredentialsProvider; import org.apache.hadoop.fs.s3a.commit.CommitConstants; -import org.apache.hadoop.fs.s3a.commit.CommitOperations; import org.apache.hadoop.fs.s3a.commit.files.PendingSet; import org.apache.hadoop.fs.s3a.commit.files.SinglePendingCommit; +import org.apache.hadoop.fs.s3a.commit.impl.CommitContext; +import org.apache.hadoop.fs.s3a.commit.impl.CommitOperations; import org.apache.hadoop.fs.s3a.s3guard.S3GuardTool; import org.apache.hadoop.fs.s3a.statistics.CommitterStatistics; @@ -72,7 +73,7 @@ * Tests use of assumed roles. * Only run if an assumed role is provided. */ -@SuppressWarnings({"IOResourceOpenedButNotSafelyClosed", "ThrowableNotThrown"}) +@SuppressWarnings("ThrowableNotThrown") public class ITestAssumeRole extends AbstractS3ATestBase { private static final Logger LOG = @@ -563,9 +564,9 @@ public void testRestrictedCommitActions() throws Throwable { roleFS = (S3AFileSystem) writeableDir.getFileSystem(conf); CommitterStatistics committerStatistics = fs.newCommitterStatistics(); CommitOperations fullOperations = new CommitOperations(fs, - committerStatistics); + committerStatistics, "/"); CommitOperations operations = new CommitOperations(roleFS, - committerStatistics); + committerStatistics, "/"); File localSrc = File.createTempFile("source", ""); writeCSVData(localSrc); @@ -595,37 +596,37 @@ public void testRestrictedCommitActions() throws Throwable { SinglePendingCommit pending = fullOperations.uploadFileToPendingCommit(src, dest, "", uploadPartSize, progress); - pending.save(fs, new Path(readOnlyDir, - name + CommitConstants.PENDING_SUFFIX), true); + pending.save(fs, + new Path(readOnlyDir, name + CommitConstants.PENDING_SUFFIX), + SinglePendingCommit.serializer()); assertTrue(src.delete()); })); progress.assertCount("progress counter is not expected", range); - try { + try(CommitContext commitContext = + operations.createCommitContextForTesting(uploadDest, + null, 0)) { // we expect to be able to list all the files here Pair>> pendingCommits = operations.loadSinglePendingCommits(readOnlyDir, - true); + true, commitContext); // all those commits must fail List commits = pendingCommits.getLeft().getCommits(); assertEquals(range, commits.size()); - try(CommitOperations.CommitContext commitContext - = operations.initiateCommitOperation(uploadDest)) { - commits.parallelStream().forEach( - (c) -> { - CommitOperations.MaybeIOE maybeIOE = - commitContext.commit(c, "origin"); - Path path = c.destinationPath(); - assertCommitAccessDenied(path, maybeIOE); - }); - } + commits.parallelStream().forEach( + (c) -> { + CommitOperations.MaybeIOE maybeIOE = + commitContext.commit(c, "origin"); + Path path = c.destinationPath(); + assertCommitAccessDenied(path, maybeIOE); + }); // fail of all list and abort of .pending files. LOG.info("abortAllSinglePendingCommits({})", readOnlyDir); assertCommitAccessDenied(readOnlyDir, - operations.abortAllSinglePendingCommits(readOnlyDir, true)); + operations.abortAllSinglePendingCommits(readOnlyDir, commitContext, true)); // try writing a magic file Path magicDestPath = new Path(readOnlyDir, diff --git a/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/commit/AbstractCommitITest.java b/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/commit/AbstractCommitITest.java index 17ce7e4ddcacd..9d9000cafb936 100644 --- a/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/commit/AbstractCommitITest.java +++ b/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/commit/AbstractCommitITest.java @@ -18,9 +18,12 @@ package org.apache.hadoop.fs.s3a.commit; +import java.io.File; import java.io.IOException; +import java.time.LocalDateTime; +import java.time.format.DateTimeFormatter; +import java.time.format.DateTimeFormatterBuilder; import java.util.List; -import java.util.stream.Collectors; import org.assertj.core.api.Assertions; import org.slf4j.Logger; @@ -34,17 +37,20 @@ import org.apache.hadoop.fs.contract.ContractTestUtils; import org.apache.hadoop.fs.s3a.AbstractS3ATestBase; import org.apache.hadoop.fs.s3a.S3AFileSystem; -import org.apache.hadoop.fs.s3a.WriteOperationHelper; -import org.apache.hadoop.fs.store.audit.AuditSpan; import org.apache.hadoop.fs.s3a.commit.files.SuccessData; import org.apache.hadoop.mapreduce.JobContext; import org.apache.hadoop.mapreduce.RecordWriter; import org.apache.hadoop.mapreduce.TaskAttemptContext; import org.apache.hadoop.mapreduce.TypeConverter; +import org.apache.hadoop.mapreduce.lib.output.committer.manifest.files.ManifestPrinter; +import org.apache.hadoop.mapreduce.lib.output.committer.manifest.files.ManifestSuccessData; import org.apache.hadoop.mapreduce.task.TaskAttemptContextImpl; import org.apache.hadoop.mapreduce.v2.api.records.JobId; import org.apache.hadoop.mapreduce.v2.util.MRBuilderUtils; +import static java.time.temporal.ChronoField.DAY_OF_MONTH; +import static java.time.temporal.ChronoField.MONTH_OF_YEAR; +import static java.time.temporal.ChronoField.YEAR; import static org.apache.hadoop.fs.s3a.Constants.*; import static org.apache.hadoop.fs.s3a.MultipartTestUtils.listMultipartUploads; import static org.apache.hadoop.fs.s3a.S3ATestUtils.*; @@ -60,6 +66,17 @@ public abstract class AbstractCommitITest extends AbstractS3ATestBase { private static final Logger LOG = LoggerFactory.getLogger(AbstractCommitITest.class); + /** + * Helper class for commit operations and assertions. + */ + private CommitterTestHelper testHelper; + + /** + * Directory for job summary reports. + * This should be set up in test suites testing against real object stores. + */ + private File reportDir; + /** * Creates a configuration for commit operations: commit is enabled in the FS * and output is multipart to on-heap arrays. @@ -81,6 +98,8 @@ protected Configuration createConfiguration() { conf.setLong(MIN_MULTIPART_THRESHOLD, MULTIPART_MIN_SIZE); conf.setInt(MULTIPART_SIZE, MULTIPART_MIN_SIZE); conf.set(FAST_UPLOAD_BUFFER, FAST_UPLOAD_BUFFER_ARRAY); + // and bind the report dir + conf.set(OPT_SUMMARY_REPORT_DIR, reportDir.toURI().toString()); return conf; } @@ -92,6 +111,36 @@ public Logger log() { return LOG; } + /** + * Get directory for reports; valid after + * setup. + * @return where success/failure reports go. + */ + protected File getReportDir() { + return reportDir; + } + + @Override + public void setup() throws Exception { + // set the manifest committer to a localfs path for reports across + // all threads. + // do this before superclass setup so reportDir is non-null there + // and can be used in creating the configuration. + reportDir = new File(getProjectBuildDir(), "reports"); + reportDir.mkdirs(); + + super.setup(); + testHelper = new CommitterTestHelper(getFileSystem()); + } + + /** + * Get helper class. + * @return helper; only valid after setup. + */ + public CommitterTestHelper getTestHelper() { + return testHelper; + } + /*** * Bind to the named committer. * @@ -117,12 +166,14 @@ public void rmdir(Path dir, Configuration conf) throws IOException { if (dir != null) { describe("deleting %s", dir); FileSystem fs = dir.getFileSystem(conf); + fs.delete(dir, true); + } } /** - * Create a random Job ID using the fork ID as part of the number. + * Create a random Job ID using the fork ID and the current time. * @return fork ID string in a format parseable by Jobs * @throws Exception failure */ @@ -132,7 +183,14 @@ public static String randomJobId() throws Exception { String trailingDigits = testUniqueForkId.substring(l - 4, l); try { int digitValue = Integer.valueOf(trailingDigits); - return String.format("20070712%04d_%04d", + DateTimeFormatter formatter = new DateTimeFormatterBuilder() + .parseCaseInsensitive() + .appendValue(YEAR, 4) + .appendValue(MONTH_OF_YEAR, 2) + .appendValue(DAY_OF_MONTH, 2) + .toFormatter(); + return String.format("%s%04d_%04d", + LocalDateTime.now().format(formatter), (long)(Math.random() * 1000), digitValue); } catch (NumberFormatException e) { @@ -146,22 +204,9 @@ public static String randomJobId() throws Exception { * @return a count of aborts * @throws IOException trouble. */ - protected int abortMultipartUploadsUnderPath(Path path) throws IOException { - S3AFileSystem fs = getFileSystem(); - if (fs != null && path != null) { - String key = fs.pathToKey(path); - int count = 0; - try (AuditSpan span = span()) { - WriteOperationHelper writeOps = fs.getWriteOperationHelper(); - count = writeOps.abortMultipartUploadsUnderPath(key); - if (count > 0) { - log().info("Multipart uploads deleted: {}", count); - } - } - return count; - } else { - return 0; - } + protected void abortMultipartUploadsUnderPath(Path path) throws IOException { + getTestHelper() + .abortMultipartUploadsUnderPath(path); } /** @@ -183,10 +228,9 @@ protected void assertMultipartUploadsPending(Path path) throws IOException { protected void assertNoMultipartUploadsPending(Path path) throws IOException { List uploads = listMultipartUploads(getFileSystem(), pathToPrefix(path)); - if (!uploads.isEmpty()) { - String result = uploads.stream().collect(Collectors.joining("\n")); - fail("Multipart uploads in progress under " + path + " \n" + result); - } + Assertions.assertThat(uploads) + .describedAs("Multipart uploads in progress under " + path) + .isEmpty(); } /** @@ -341,6 +385,13 @@ public static SuccessData validateSuccessFile(final Path outputPath, .describedAs("JobID in " + commitDetails) .isEqualTo(jobId); } + // also load as a manifest success data file + // to verify consistency and that the CLI tool works. + Path success = new Path(outputPath, _SUCCESS); + final ManifestPrinter showManifest = new ManifestPrinter(); + ManifestSuccessData manifestSuccessData = + showManifest.loadAndPrintManifest(fs, success); + return successData; } diff --git a/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/commit/AbstractITCommitProtocol.java b/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/commit/AbstractITCommitProtocol.java index 4cb90a7373c59..b193cca03db00 100644 --- a/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/commit/AbstractITCommitProtocol.java +++ b/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/commit/AbstractITCommitProtocol.java @@ -71,6 +71,8 @@ import org.apache.hadoop.util.concurrent.HadoopExecutors; import static org.apache.hadoop.fs.contract.ContractTestUtils.*; +import static org.apache.hadoop.fs.s3a.Constants.STORAGE_CLASS; +import static org.apache.hadoop.fs.s3a.Constants.STORAGE_CLASS_INTELLIGENT_TIERING; import static org.apache.hadoop.fs.s3a.S3AUtils.*; import static org.apache.hadoop.fs.s3a.S3ATestUtils.*; import static org.apache.hadoop.fs.s3a.commit.AbstractS3ACommitter.E_SELF_GENERATED_JOB_UUID; @@ -124,7 +126,7 @@ public abstract class AbstractITCommitProtocol extends AbstractCommitITest { private static final Text VAL_2 = new Text("val2"); /** A job to abort in test case teardown. */ - private List abortInTeardown = new ArrayList<>(1); + private final List abortInTeardown = new ArrayList<>(1); private final StandardCommitterFactory standardCommitterFactory = new StandardCommitterFactory(); @@ -560,7 +562,7 @@ protected void commit(AbstractS3ACommitter committer, describe("\ncommitting job"); committer.commitJob(jContext); describe("commit complete\n"); - verifyCommitterHasNoThreads(committer); + } } @@ -615,6 +617,9 @@ public void testRecoveryAndCleanup() throws Exception { assertNotNull("null outputPath in committer " + committer, committer.getOutputPath()); + // note the task attempt path. + Path job1TaskAttempt0Path = committer.getTaskAttemptPath(tContext); + // Commit the task. This will promote data and metadata to where // job commits will pick it up on commit or abort. commitTask(committer, tContext); @@ -634,6 +639,15 @@ public void testRecoveryAndCleanup() throws Exception { intercept(PathCommitException.class, "recover", () -> committer2.recoverTask(tContext2)); + // the new task attempt path is different from the first, because the + // job attempt counter is used in the path + final Path job2TaskAttempt0Path = committer2.getTaskAttemptPath(tContext2); + LOG.info("Job attempt 1 task attempt path {}; attempt 2 path {}", + job1TaskAttempt0Path, job2TaskAttempt0Path); + assertNotEquals("Task attempt paths must differ", + job1TaskAttempt0Path, + job2TaskAttempt0Path); + // at this point, task attempt 0 has failed to recover // it should be abortable though. This will be a no-op as it already // committed @@ -643,7 +657,7 @@ public void testRecoveryAndCleanup() throws Exception { committer2.abortJob(jContext2, JobStatus.State.KILLED); // now, state of system may still have pending data assertNoMultipartUploadsPending(outDir); - verifyCommitterHasNoThreads(committer2); + } protected void assertTaskAttemptPathDoesNotExist( @@ -695,6 +709,22 @@ private void validateContent(Path dir, expectedOutput.toString(), output); } + /** + * Verify storage class of output file matches the expected storage class. + * @param dir output directory. + * @param expectedStorageClass expected storage class value. + * @throws Exception failure. + */ + private void validateStorageClass(Path dir, String expectedStorageClass) throws Exception { + Path expectedFile = getPart0000(dir); + S3AFileSystem fs = getFileSystem(); + String actualStorageClass = fs.getObjectMetadata(expectedFile).getStorageClass(); + + Assertions.assertThat(actualStorageClass) + .describedAs("Storage class of object %s", expectedFile) + .isEqualToIgnoringCase(expectedStorageClass); + } + /** * Identify any path under the directory which begins with the * {@code "part-m-00000"} sequence. @@ -729,7 +759,9 @@ private void validateMapFileOutputContent( assertPathExists("Map output", expectedMapDir); assertIsDirectory(expectedMapDir); FileStatus[] files = fs.listStatus(expectedMapDir); - assertTrue("No files found in " + expectedMapDir, files.length > 0); + Assertions.assertThat(files) + .describedAs("Files found in " + expectedMapDir) + .hasSizeGreaterThan(0); assertPathExists("index file in " + expectedMapDir, new Path(expectedMapDir, MapFile.INDEX_FILE_NAME)); assertPathExists("data file in " + expectedMapDir, @@ -777,9 +809,9 @@ public void testCommitLifecycle() throws Exception { try { applyLocatedFiles(getFileSystem().listFiles(outDir, false), - (status) -> - assertFalse("task committed file to dest :" + status, - status.getPath().toString().contains("part"))); + (status) -> Assertions.assertThat(status.getPath().toString()) + .describedAs("task committed file to dest :" + status) + .doesNotContain("part")); } catch (FileNotFoundException ignored) { log().info("Outdir {} is not created by task commit phase ", outDir); @@ -796,6 +828,41 @@ public void testCommitLifecycle() throws Exception { assertNoMultipartUploadsPending(outDir); } + @Test + public void testCommitWithStorageClassConfig() throws Exception { + describe("Commit with specific storage class configuration;" + + " expect the final file has correct storage class."); + + Configuration conf = getConfiguration(); + skipIfStorageClassTestsDisabled(conf); + conf.set(STORAGE_CLASS, STORAGE_CLASS_INTELLIGENT_TIERING); + + JobData jobData = startJob(false); + JobContext jContext = jobData.jContext; + TaskAttemptContext tContext = jobData.tContext; + AbstractS3ACommitter committer = jobData.committer; + validateTaskAttemptWorkingDirectory(committer, tContext); + + // write output + writeTextOutput(tContext); + + // commit task + dumpMultipartUploads(); + commitTask(committer, tContext); + + // commit job + assertMultipartUploadsPending(outDir); + commitJob(committer, jContext); + + // validate output + validateContent(outDir, shouldExpectSuccessMarker(), + committer.getUUID()); + assertNoMultipartUploadsPending(outDir); + + // validate storage class + validateStorageClass(outDir, STORAGE_CLASS_INTELLIGENT_TIERING); + } + @Test public void testCommitterWithDuplicatedCommit() throws Exception { describe("Call a task then job commit twice;" + @@ -1018,27 +1085,34 @@ public void testMapFileOutputCommitter() throws Exception { // hidden filenames (_ or . prefixes) describe("listing"); FileStatus[] filtered = fs.listStatus(outDir, HIDDEN_FILE_FILTER); - assertEquals("listed children under " + ls, - 1, filtered.length); + Assertions.assertThat(filtered) + .describedAs("listed children under " + ls) + .hasSize(1); FileStatus fileStatus = filtered[0]; - assertTrue("Not the part file: " + fileStatus, - fileStatus.getPath().getName().startsWith(PART_00000)); + Assertions.assertThat(fileStatus.getPath().getName()) + .describedAs("Not a part file: " + fileStatus) + .startsWith(PART_00000); describe("getReaders()"); - assertEquals("Number of MapFile.Reader entries with shared FS " - + outDir + " : " + ls, - 1, getReaders(fs, outDir, conf).length); + Assertions.assertThat(getReaders(fs, outDir, conf)) + .describedAs("Number of MapFile.Reader entries with shared FS %s: %s", + outDir, ls) + .hasSize(1); describe("getReaders(new FS)"); FileSystem fs2 = FileSystem.get(outDir.toUri(), conf); - assertEquals("Number of MapFile.Reader entries with shared FS2 " - + outDir + " : " + ls, - 1, getReaders(fs2, outDir, conf).length); + Assertions.assertThat(getReaders(fs2, outDir, conf)) + .describedAs("Number of MapFile.Reader entries with shared FS2 %s: %s", + outDir, ls) + .hasSize(1); describe("MapFileOutputFormat.getReaders"); - assertEquals("Number of MapFile.Reader entries with new FS in " - + outDir + " : " + ls, - 1, MapFileOutputFormat.getReaders(outDir, conf).length); + + Assertions.assertThat(MapFileOutputFormat.getReaders(outDir, conf)) + .describedAs("Number of MapFile.Reader entries with new FS in %s: %s", + outDir, ls) + .hasSize(1); + } /** Open the output generated by this format. */ @@ -1112,7 +1186,7 @@ public void testAbortTaskThenJob() throws Exception { committer.abortJob(jobData.jContext, JobStatus.State.FAILED); assertJobAbortCleanedUp(jobData); - verifyCommitterHasNoThreads(committer); + } /** @@ -1166,7 +1240,7 @@ public void testFailAbort() throws Exception { // try again; expect abort to be idempotent. committer.abortJob(jContext, JobStatus.State.FAILED); assertNoMultipartUploadsPending(outDir); - verifyCommitterHasNoThreads(committer); + } public void assertPart0000DoesNotExist(Path dir) throws Exception { @@ -1380,7 +1454,7 @@ public void testAMWorkflow() throws Throwable { AbstractS3ACommitter committer2 = (AbstractS3ACommitter) outputFormat.getOutputCommitter(newAttempt); committer2.abortTask(tContext); - verifyCommitterHasNoThreads(committer2); + assertNoMultipartUploadsPending(getOutDir()); } @@ -1724,11 +1798,10 @@ public void testS3ACommitterFactoryBinding() throws Throwable { conf.setInt(MRJobConfig.APPLICATION_ATTEMPT_ID, 1); TaskAttemptContext tContext = new TaskAttemptContextImpl(conf, taskAttempt0); - String name = getCommitterName(); S3ACommitterFactory factory = new S3ACommitterFactory(); - assertEquals("Wrong committer from factory", - createCommitter(outDir, tContext).getClass(), - factory.createOutputCommitter(outDir, tContext).getClass()); + Assertions.assertThat(factory.createOutputCommitter(outDir, tContext).getClass()) + .describedAs("Committer from factory with name %s", getCommitterName()) + .isEqualTo(createCommitter(outDir, tContext).getClass()); } /** @@ -1777,7 +1850,7 @@ protected void validateTaskAttemptWorkingDirectory( protected void commitTask(final AbstractS3ACommitter committer, final TaskAttemptContext tContext) throws IOException { committer.commitTask(tContext); - verifyCommitterHasNoThreads(committer); + } /** @@ -1789,15 +1862,7 @@ protected void commitTask(final AbstractS3ACommitter committer, protected void commitJob(final AbstractS3ACommitter committer, final JobContext jContext) throws IOException { committer.commitJob(jContext); - verifyCommitterHasNoThreads(committer); - } - /** - * Verify that the committer does not have a thread pool. - * @param committer committer to validate. - */ - protected void verifyCommitterHasNoThreads(AbstractS3ACommitter committer) { - assertFalse("Committer has an active thread pool", - committer.hasThreadPool()); } + } diff --git a/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/commit/CommitterTestHelper.java b/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/commit/CommitterTestHelper.java new file mode 100644 index 0000000000000..cd703f96da8dd --- /dev/null +++ b/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/commit/CommitterTestHelper.java @@ -0,0 +1,148 @@ +/* + * 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.fs.s3a.commit; + +import java.io.IOException; +import java.util.List; + +import org.assertj.core.api.Assertions; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + +import org.apache.hadoop.fs.FSDataOutputStream; +import org.apache.hadoop.fs.FileStatus; +import org.apache.hadoop.fs.Path; +import org.apache.hadoop.fs.s3a.MultipartTestUtils; +import org.apache.hadoop.fs.s3a.S3AFileSystem; + +import static java.util.Objects.requireNonNull; +import static org.apache.hadoop.fs.contract.ContractTestUtils.verifyPathExists; +import static org.apache.hadoop.fs.s3a.commit.CommitConstants.BASE; +import static org.apache.hadoop.fs.s3a.commit.CommitConstants.MAGIC; +import static org.apache.hadoop.fs.s3a.commit.CommitConstants.STREAM_CAPABILITY_MAGIC_OUTPUT; +import static org.apache.hadoop.fs.s3a.commit.CommitConstants.XA_MAGIC_MARKER; +import static org.apache.hadoop.fs.s3a.commit.impl.CommitOperations.extractMagicFileLength; + +/** + * Helper for committer tests: extra assertions and the like. + */ +public class CommitterTestHelper { + + private static final Logger LOG = + LoggerFactory.getLogger(CommitterTestHelper.class); + + /** + * Filesystem under test. + */ + private final S3AFileSystem fileSystem; + + /** + * Constructor. + * @param fileSystem filesystem to work with. + */ + public CommitterTestHelper(S3AFileSystem fileSystem) { + this.fileSystem = requireNonNull(fileSystem); + } + + /** + * Get the filesystem. + * @return the filesystem. + */ + public S3AFileSystem getFileSystem() { + return fileSystem; + } + + /** + * Assert a path refers to a marker file of an expected length; + * the length is extracted from the custom header. + * @param path magic file. + * @param dataSize expected data size + * @throws IOException IO failure + */ + public void assertIsMarkerFile(Path path, long dataSize) throws IOException { + final S3AFileSystem fs = getFileSystem(); + FileStatus status = verifyPathExists(fs, + "uploaded file commit", path); + Assertions.assertThat(status.getLen()) + .describedAs("Marker File file %s: %s", path, status) + .isEqualTo(0); + Assertions.assertThat(extractMagicFileLength(fs, path)) + .describedAs("XAttribute " + XA_MAGIC_MARKER + " of " + path) + .isNotEmpty() + .hasValue(dataSize); + } + + /** + * Assert a file does not have the magic marker header. + * @param path magic file. + * @throws IOException IO failure + */ + public void assertFileLacksMarkerHeader(Path path) throws IOException { + Assertions.assertThat(extractMagicFileLength(getFileSystem(), + path)) + .describedAs("XAttribute " + XA_MAGIC_MARKER + " of " + path) + .isEmpty(); + } + + /** + * Create a new path which has the same filename as the dest file, but + * is in a magic directory under the destination dir. + * @param destFile final destination file + * @return magic path + */ + public static Path makeMagic(Path destFile) { + return new Path(destFile.getParent(), + MAGIC + '/' + BASE + "/" + destFile.getName()); + } + + /** + * Assert that an output stream is magic. + * @param stream stream to probe. + */ + public static void assertIsMagicStream(final FSDataOutputStream stream) { + Assertions.assertThat(stream.hasCapability(STREAM_CAPABILITY_MAGIC_OUTPUT)) + .describedAs("Stream capability %s in stream %s", + STREAM_CAPABILITY_MAGIC_OUTPUT, stream) + .isTrue(); + } + + /** + * Abort all multipart uploads under a path. + * @param path path for uploads to abort; may be null + * @return a count of aborts + * @throws IOException trouble. + */ + public void abortMultipartUploadsUnderPath(Path path) { + + MultipartTestUtils.clearAnyUploads(getFileSystem(), path); + } + + /** + * Get a list of all pending uploads under a prefix, one which can be printed. + * @param prefix prefix to look under + * @return possibly empty list + * @throws IOException IO failure. + */ + public List listMultipartUploads( + String prefix) throws IOException { + + return MultipartTestUtils.listMultipartUploads(getFileSystem(), prefix); + } + +} diff --git a/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/commit/ITestCommitOperationCost.java b/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/commit/ITestCommitOperationCost.java new file mode 100644 index 0000000000000..d0c86b738ca10 --- /dev/null +++ b/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/commit/ITestCommitOperationCost.java @@ -0,0 +1,325 @@ +/* + * 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.fs.s3a.commit; + +import java.io.IOException; +import java.nio.charset.StandardCharsets; +import java.util.ArrayList; +import java.util.List; + +import org.assertj.core.api.Assertions; +import org.junit.Test; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + +import org.apache.hadoop.fs.FSDataOutputStream; +import org.apache.hadoop.fs.FileStatus; +import org.apache.hadoop.fs.LocatedFileStatus; +import org.apache.hadoop.fs.Path; +import org.apache.hadoop.fs.permission.FsPermission; +import org.apache.hadoop.fs.s3a.S3AFileSystem; +import org.apache.hadoop.fs.s3a.commit.files.PersistentCommitData; +import org.apache.hadoop.fs.s3a.commit.files.SinglePendingCommit; +import org.apache.hadoop.fs.s3a.commit.impl.CommitOperations; +import org.apache.hadoop.fs.s3a.performance.AbstractS3ACostTest; +import org.apache.hadoop.fs.statistics.IOStatisticsLogging; + +import static org.apache.hadoop.fs.s3a.Statistic.ACTION_HTTP_GET_REQUEST; +import static org.apache.hadoop.fs.s3a.Statistic.COMMITTER_MAGIC_FILES_CREATED; +import static org.apache.hadoop.fs.s3a.Statistic.COMMITTER_MAGIC_MARKER_PUT; +import static org.apache.hadoop.fs.s3a.Statistic.DIRECTORIES_CREATED; +import static org.apache.hadoop.fs.s3a.Statistic.FAKE_DIRECTORIES_DELETED; +import static org.apache.hadoop.fs.s3a.Statistic.OBJECT_BULK_DELETE_REQUEST; +import static org.apache.hadoop.fs.s3a.Statistic.OBJECT_DELETE_REQUEST; +import static org.apache.hadoop.fs.s3a.Statistic.OBJECT_LIST_REQUEST; +import static org.apache.hadoop.fs.s3a.Statistic.OBJECT_METADATA_REQUESTS; +import static org.apache.hadoop.fs.s3a.Statistic.OBJECT_MULTIPART_UPLOAD_INITIATED; +import static org.apache.hadoop.fs.s3a.Statistic.OBJECT_PUT_REQUESTS; +import static org.apache.hadoop.fs.s3a.commit.CommitConstants.MAGIC; +import static org.apache.hadoop.fs.s3a.commit.CommitterTestHelper.assertIsMagicStream; +import static org.apache.hadoop.fs.s3a.commit.CommitterTestHelper.makeMagic; +import static org.apache.hadoop.fs.s3a.performance.OperationCost.LIST_FILES_LIST_OP; +import static org.apache.hadoop.fs.s3a.performance.OperationCost.LIST_OPERATION; +import static org.apache.hadoop.fs.s3a.performance.OperationCost.NO_HEAD_OR_LIST; +import static org.apache.hadoop.fs.statistics.IOStatisticsLogging.ioStatisticsToPrettyString; +import static org.apache.hadoop.util.functional.RemoteIterators.toList; + +/** + * Assert cost of commit operations; + *
      + *
    1. Even on marker deleting filesystems, + * operations under magic dirs do not trigger marker deletion.
    2. + *
    3. Loading pending files from FileStatus entries skips HEAD checks.
    4. + *
    5. Mkdir under magic dirs doesn't check ancestor or dest type
    6. + *
    + */ +public class ITestCommitOperationCost extends AbstractS3ACostTest { + + private static final Logger LOG = + LoggerFactory.getLogger(ITestCommitOperationCost.class); + + /** + * Helper for the tests. + */ + private CommitterTestHelper testHelper; + + /** + * Create with markers kept, always. + */ + public ITestCommitOperationCost() { + super(false); + } + + @Override + public void setup() throws Exception { + super.setup(); + testHelper = new CommitterTestHelper(getFileSystem()); + } + + @Override + public void teardown() throws Exception { + try { + if (testHelper != null) { + testHelper.abortMultipartUploadsUnderPath(methodPath()); + } + } finally { + super.teardown(); + } + } + + /** + * Get a method-relative path. + * @param filename filename + * @return new path + * @throws IOException failure to create/parse the path. + */ + private Path methodSubPath(String filename) throws IOException { + return new Path(methodPath(), filename); + } + + /** + * Return the FS IOStats, prettified. + * @return string for assertions. + */ + protected String fileSystemIOStats() { + return ioStatisticsToPrettyString(getFileSystem().getIOStatistics()); + } + + @Test + public void testMagicMkdir() throws Throwable { + describe("Mkdirs __magic always skips dir marker deletion"); + S3AFileSystem fs = getFileSystem(); + Path baseDir = methodPath(); + // create dest dir marker, always + fs.mkdirs(baseDir); + Path magicDir = new Path(baseDir, MAGIC); + verifyMetrics(() -> { + fs.mkdirs(magicDir); + return fileSystemIOStats(); + }, + with(OBJECT_BULK_DELETE_REQUEST, 0), + with(OBJECT_DELETE_REQUEST, 0), + with(DIRECTORIES_CREATED, 1)); + verifyMetrics(() -> { + fs.delete(magicDir, true); + return fileSystemIOStats(); + }, + with(OBJECT_BULK_DELETE_REQUEST, 0), + with(OBJECT_DELETE_REQUEST, 1), + with(DIRECTORIES_CREATED, 0)); + assertPathExists("parent", baseDir); + } + + /** + * When a magic subdir is deleted, parent dirs are not recreated. + */ + @Test + public void testMagicMkdirs() throws Throwable { + describe("Mkdirs __magic/subdir always skips dir marker deletion"); + S3AFileSystem fs = getFileSystem(); + Path baseDir = methodPath(); + Path magicDir = new Path(baseDir, MAGIC); + fs.delete(baseDir, true); + + Path magicSubdir = new Path(magicDir, "subdir"); + verifyMetrics(() -> { + fs.mkdirs(magicSubdir, FsPermission.getDirDefault()); + return "after mkdirs " + fileSystemIOStats(); + }, + always(LIST_OPERATION), + with(OBJECT_BULK_DELETE_REQUEST, 0), + with(OBJECT_DELETE_REQUEST, 0), + with(DIRECTORIES_CREATED, 1)); + assertPathExists("magicSubdir", magicSubdir); + + verifyMetrics(() -> { + fs.delete(magicSubdir, true); + return "after delete " + fileSystemIOStats(); + }, + with(OBJECT_BULK_DELETE_REQUEST, 0), + with(OBJECT_DELETE_REQUEST, 1), + with(OBJECT_LIST_REQUEST, 1), + with(OBJECT_METADATA_REQUESTS, 1), + with(DIRECTORIES_CREATED, 0)); + // no marker dir creation + assertPathDoesNotExist("magicDir", magicDir); + assertPathDoesNotExist("baseDir", baseDir); + } + + /** + * Active stream; a field is used so closures can write to + * it. + */ + private FSDataOutputStream stream; + + /** + * Abort any active stream. + * @throws IOException failure + */ + private void abortActiveStream() throws IOException { + if (stream != null) { + stream.abort(); + stream.close(); + } + } + + @Test + public void testCostOfCreatingMagicFile() throws Throwable { + describe("Files created under magic paths skip existence checks and marker deletes"); + S3AFileSystem fs = getFileSystem(); + Path destFile = methodSubPath("file.txt"); + fs.delete(destFile.getParent(), true); + Path magicDest = makeMagic(destFile); + + // when the file is created, there is no check for overwrites + // or the dest being a directory, even if overwrite=false + try { + verifyMetrics(() -> { + stream = fs.create(magicDest, false); + return stream.toString(); + }, + always(NO_HEAD_OR_LIST), + with(COMMITTER_MAGIC_FILES_CREATED, 1), + with(COMMITTER_MAGIC_MARKER_PUT, 0), + with(OBJECT_MULTIPART_UPLOAD_INITIATED, 1)); + assertIsMagicStream(stream); + + stream.write("hello".getBytes(StandardCharsets.UTF_8)); + + // when closing, there will be no directories deleted + // we do expect two PUT requests, because the marker and manifests + // are both written + LOG.info("closing magic stream to {}", magicDest); + verifyMetrics(() -> { + stream.close(); + return stream.toString(); + }, + always(NO_HEAD_OR_LIST), + with(OBJECT_PUT_REQUESTS, 2), + with(COMMITTER_MAGIC_MARKER_PUT, 2), + with(OBJECT_BULK_DELETE_REQUEST, 0), + with(OBJECT_DELETE_REQUEST, 0)); + + } catch (Exception e) { + abortActiveStream(); + throw e; + } + // list the manifests + final CommitOperations commitOperations = new CommitOperations(fs); + List pending = verifyMetrics(() -> + toList(commitOperations. + locateAllSinglePendingCommits(magicDest.getParent(), false)), + always(LIST_FILES_LIST_OP)); + Assertions.assertThat(pending) + .describedAs("pending commits") + .hasSize(1); + + // load the only pending commit + SinglePendingCommit singleCommit = verifyMetrics(() -> + PersistentCommitData.load(fs, + pending.get(0), + SinglePendingCommit.serializer()), + always(NO_HEAD_OR_LIST), + with(ACTION_HTTP_GET_REQUEST, 1)); + + // commit it through the commit operations. + verifyMetrics(() -> { + commitOperations.commitOrFail(singleCommit); + return ioStatisticsToPrettyString( + commitOperations.getIOStatistics()); + }, + always(NO_HEAD_OR_LIST), // no probes for the dest path + with(FAKE_DIRECTORIES_DELETED, 0), // no fake dirs + with(OBJECT_DELETE_REQUEST, 0)); // no deletes + + LOG.info("Final Statistics {}", + IOStatisticsLogging.ioStatisticsToPrettyString(stream.getIOStatistics())); + } + + /** + * saving pending files MUST NOT trigger HEAD/LIST calls + * when created under a magic path; when opening + * with an S3AFileStatus the HEAD will be skipped too. + */ + @Test + public void testCostOfSavingLoadingPendingFile() throws Throwable { + describe("Verify costs of saving .pending file under a magic path"); + + S3AFileSystem fs = getFileSystem(); + Path partDir = methodSubPath("file.pending"); + Path destFile = new Path(partDir, "file.pending"); + Path magicDest = makeMagic(destFile); + // create a pending file with minimal values needed + // for validation to work + final SinglePendingCommit commit = new SinglePendingCommit(); + commit.touch(System.currentTimeMillis()); + commit.setUri(destFile.toUri().toString()); + commit.setBucket(fs.getBucket()); + commit.setLength(0); + commit.setDestinationKey(fs.pathToKey(destFile)); + commit.setUploadId("uploadId"); + commit.setEtags(new ArrayList<>()); + // fail fast if the commit data is incomplete + commit.validate(); + + // save the file: no checks will be made + verifyMetrics(() -> { + commit.save(fs, magicDest, + SinglePendingCommit.serializer()); + return commit.toString(); + }, + with(COMMITTER_MAGIC_FILES_CREATED, 0), + always(NO_HEAD_OR_LIST), + with(OBJECT_BULK_DELETE_REQUEST, 0), + with(OBJECT_DELETE_REQUEST, 0) + ); + + LOG.info("File written; Validating"); + testHelper.assertFileLacksMarkerHeader(magicDest); + FileStatus status = fs.getFileStatus(magicDest); + + LOG.info("Reading file {}", status); + // opening a file with a status passed in will skip the HEAD + verifyMetrics(() -> + PersistentCommitData.load(fs, status, SinglePendingCommit.serializer()), + always(NO_HEAD_OR_LIST), + with(ACTION_HTTP_GET_REQUEST, 1)); + } + +} diff --git a/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/commit/ITestCommitOperations.java b/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/commit/ITestCommitOperations.java index 3f0e2e7a1348b..a76a65be8bb3e 100644 --- a/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/commit/ITestCommitOperations.java +++ b/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/commit/ITestCommitOperations.java @@ -23,8 +23,8 @@ import java.io.IOException; import java.util.ArrayList; import java.util.List; +import java.util.UUID; -import com.amazonaws.services.s3.model.PartETag; import org.assertj.core.api.Assertions; import org.junit.Test; import org.slf4j.Logger; @@ -33,12 +33,15 @@ import org.apache.commons.io.FileUtils; import org.apache.hadoop.conf.Configuration; import org.apache.hadoop.fs.FSDataOutputStream; +import org.apache.hadoop.fs.FSDataOutputStreamBuilder; import org.apache.hadoop.fs.FileStatus; import org.apache.hadoop.fs.FileSystem; import org.apache.hadoop.fs.Path; import org.apache.hadoop.fs.s3a.S3AFileSystem; import org.apache.hadoop.fs.s3a.auth.ProgressCounter; import org.apache.hadoop.fs.s3a.commit.files.SinglePendingCommit; +import org.apache.hadoop.fs.s3a.commit.impl.CommitContext; +import org.apache.hadoop.fs.s3a.commit.impl.CommitOperations; import org.apache.hadoop.fs.s3a.commit.magic.MagicCommitTracker; import org.apache.hadoop.fs.s3a.commit.magic.MagicS3GuardCommitter; import org.apache.hadoop.fs.s3a.commit.magic.MagicS3GuardCommitterFactory; @@ -52,11 +55,13 @@ import static org.apache.hadoop.fs.contract.ContractTestUtils.*; import static org.apache.hadoop.fs.s3a.S3ATestUtils.*; import static org.apache.hadoop.fs.s3a.commit.CommitConstants.*; -import static org.apache.hadoop.fs.s3a.commit.CommitOperations.extractMagicFileLength; import static org.apache.hadoop.fs.s3a.commit.CommitUtils.*; +import static org.apache.hadoop.fs.s3a.commit.CommitterTestHelper.assertIsMagicStream; import static org.apache.hadoop.fs.s3a.commit.MagicCommitPaths.*; import static org.apache.hadoop.fs.s3a.Constants.*; +import static org.apache.hadoop.fs.s3a.statistics.impl.EmptyS3AStatisticsContext.EMPTY_BLOCK_OUTPUT_STREAM_STATISTICS; import static org.apache.hadoop.mapreduce.lib.output.PathOutputCommitterFactory.*; +import static org.apache.hadoop.test.LambdaTestUtils.intercept; /** * Test the low-level binding of the S3A FS to the magic commit mechanism, @@ -69,6 +74,8 @@ public class ITestCommitOperations extends AbstractCommitITest { private static final byte[] DATASET = dataset(1000, 'a', 32); private static final String S3A_FACTORY_KEY = String.format( COMMITTER_FACTORY_SCHEME_PATTERN, "s3a"); + private static final String JOB_ID = UUID.randomUUID().toString(); + private ProgressCounter progress; @Override @@ -94,11 +101,13 @@ public void testCreateTrackerNormalPath() throws Throwable { MagicCommitIntegration integration = new MagicCommitIntegration(fs, true); String filename = "notdelayed.txt"; - Path destFile = methodPath(filename); + Path destFile = methodSubPath(filename); String origKey = fs.pathToKey(destFile); - PutTracker tracker = integration.createTracker(destFile, origKey); - assertFalse("wrong type: " + tracker + " for " + destFile, - tracker instanceof MagicCommitTracker); + PutTracker tracker = integration.createTracker(destFile, origKey, + EMPTY_BLOCK_OUTPUT_STREAM_STATISTICS); + Assertions.assertThat(tracker) + .describedAs("Tracker for %s", destFile) + .isNotInstanceOf(MagicCommitTracker.class); } /** @@ -111,36 +120,45 @@ public void testCreateTrackerMagicPath() throws Throwable { MagicCommitIntegration integration = new MagicCommitIntegration(fs, true); String filename = "delayed.txt"; - Path destFile = methodPath(filename); + Path destFile = methodSubPath(filename); String origKey = fs.pathToKey(destFile); Path pendingPath = makeMagic(destFile); verifyIsMagicCommitPath(fs, pendingPath); String pendingPathKey = fs.pathToKey(pendingPath); - assertTrue("wrong path of " + pendingPathKey, - pendingPathKey.endsWith(filename)); + Assertions.assertThat(pendingPathKey) + .describedAs("pending path") + .endsWith(filename); final List elements = splitPathToElements(pendingPath); - assertEquals("splitPathToElements()", filename, lastElement(elements)); + Assertions.assertThat(lastElement(elements)) + .describedAs("splitPathToElements(%s)", pendingPath) + .isEqualTo(filename); List finalDestination = finalDestination(elements); - assertEquals("finalDestination()", - filename, - lastElement(finalDestination)); - final String destKey = elementsToKey(finalDestination); - assertEquals("destination key", origKey, destKey); + Assertions.assertThat(lastElement(finalDestination)) + .describedAs("finalDestination(%s)", pendingPath) + .isEqualTo(filename); + Assertions.assertThat(elementsToKey(finalDestination)) + .describedAs("destination key") + .isEqualTo(origKey); PutTracker tracker = integration.createTracker(pendingPath, - pendingPathKey); - assertTrue("wrong type: " + tracker + " for " + pendingPathKey, - tracker instanceof MagicCommitTracker); - assertEquals("tracker destination key", origKey, tracker.getDestKey()); - - Path pendingSuffixedPath = new Path(pendingPath, - "part-0000" + PENDING_SUFFIX); - assertFalse("still a delayed complete path " + pendingSuffixedPath, - fs.isMagicCommitPath(pendingSuffixedPath)); - Path pendingSet = new Path(pendingPath, - "part-0000" + PENDINGSET_SUFFIX); - assertFalse("still a delayed complete path " + pendingSet, - fs.isMagicCommitPath(pendingSet)); + pendingPathKey, EMPTY_BLOCK_OUTPUT_STREAM_STATISTICS); + Assertions.assertThat(tracker) + .describedAs("Tracker for %s", pendingPathKey) + .isInstanceOf(MagicCommitTracker.class); + Assertions.assertThat(tracker.getDestKey()) + .describedAs("tracker destination key") + .isEqualTo(origKey); + + assertNotDelayedWrite(new Path(pendingPath, + "part-0000" + PENDING_SUFFIX)); + assertNotDelayedWrite(new Path(pendingPath, + "part-0000" + PENDINGSET_SUFFIX)); + } + + private void assertNotDelayedWrite(Path pendingSuffixedPath) { + Assertions.assertThat(getFileSystem().isMagicCommitPath(pendingSuffixedPath)) + .describedAs("Expected %s to not be magic/delayed write", pendingSuffixedPath) + .isFalse(); } @Test @@ -148,7 +166,7 @@ public void testCreateAbortEmptyFile() throws Throwable { describe("create then abort an empty file; throttled"); S3AFileSystem fs = getFileSystem(); String filename = "empty-abort.txt"; - Path destFile = methodPath(filename); + Path destFile = methodSubPath(filename); Path pendingFilePath = makeMagic(destFile); touch(fs, pendingFilePath); @@ -160,13 +178,22 @@ public void testCreateAbortEmptyFile() throws Throwable { // abort,; rethrow on failure LOG.info("Abort call"); - actions.abortAllSinglePendingCommits(pendingDataPath.getParent(), true) - .maybeRethrow(); + Path parent = pendingDataPath.getParent(); + try (CommitContext commitContext = + actions.createCommitContextForTesting(parent, JOB_ID, 0)) { + actions.abortAllSinglePendingCommits(parent, commitContext, true) + .maybeRethrow(); + } assertPathDoesNotExist("pending file not deleted", pendingDataPath); assertPathDoesNotExist("dest file was created", destFile); } + /** + * Create a new commit operations instance for the test FS. + * @return commit operations. + * @throws IOException IO failure. + */ private CommitOperations newCommitOperations() throws IOException { return new CommitOperations(getFileSystem()); @@ -198,10 +225,16 @@ public void testCommitSmallFile() throws Throwable { @Test public void testAbortNonexistentDir() throws Throwable { describe("Attempt to abort a directory that does not exist"); - Path destFile = methodPath("testAbortNonexistentPath"); - newCommitOperations() - .abortAllSinglePendingCommits(destFile, true) - .maybeRethrow(); + Path destFile = methodSubPath("testAbortNonexistentPath"); + final CommitOperations operations = newCommitOperations(); + try (CommitContext commitContext + = operations.createCommitContextForTesting(destFile, JOB_ID, 0)) { + final CommitOperations.MaybeIOE outcome = operations + .abortAllSinglePendingCommits(destFile, commitContext, true); + outcome.maybeRethrow(); + Assertions.assertThat(outcome) + .isEqualTo(CommitOperations.MaybeIOE.NONE); + } } @Test @@ -244,7 +277,7 @@ public void testBaseRelativePath() throws Throwable { describe("Test creating file with a __base marker and verify that it ends" + " up in where expected"); S3AFileSystem fs = getFileSystem(); - Path destDir = methodPath("testBaseRelativePath"); + Path destDir = methodSubPath("testBaseRelativePath"); fs.delete(destDir, true); Path pendingBaseDir = new Path(destDir, MAGIC + "/child/" + BASE); String child = "subdir/child.txt"; @@ -270,14 +303,17 @@ public void testMarkerFileRename() fs.delete(destDir, true); Path magicDest = makeMagic(destFile); Path magicDir = magicDest.getParent(); - fs.mkdirs(magicDir); + fs.mkdirs(magicDest); // use the builder API to verify it works exactly the // same. - try (FSDataOutputStream stream = fs.createFile(magicDest) - .overwrite(true) - .recursive() - .build()) { + FSDataOutputStreamBuilder builder = fs.createFile(magicDest) + .overwrite(true); + builder.recursive(); + // this has a broken return type; not sure why + builder.must(FS_S3A_CREATE_PERFORMANCE, true); + + try (FSDataOutputStream stream = builder.build()) { assertIsMagicStream(stream); stream.write(DATASET); } @@ -286,9 +322,7 @@ public void testMarkerFileRename() fs.rename(magicDest, magic2); // the renamed file has no header - Assertions.assertThat(extractMagicFileLength(fs, magic2)) - .describedAs("XAttribute " + XA_MAGIC_MARKER + " of " + magic2) - .isEmpty(); + getTestHelper().assertFileLacksMarkerHeader(magic2); // abort the upload, which is driven by the .pending files // there must be 1 deleted file; during test debugging with aborted // runs there may be more. @@ -298,17 +332,6 @@ public void testMarkerFileRename() .isGreaterThanOrEqualTo(1); } - /** - * Assert that an output stream is magic. - * @param stream stream to probe. - */ - protected void assertIsMagicStream(final FSDataOutputStream stream) { - Assertions.assertThat(stream.hasCapability(STREAM_CAPABILITY_MAGIC_OUTPUT)) - .describedAs("Stream capability %s in stream %s", - STREAM_CAPABILITY_MAGIC_OUTPUT, stream) - .isTrue(); - } - /** * Create a file through the magic commit mechanism. * @param filename file to create (with __magic path.) @@ -318,39 +341,27 @@ protected void assertIsMagicStream(final FSDataOutputStream stream) { private void createCommitAndVerify(String filename, byte[] data) throws Exception { S3AFileSystem fs = getFileSystem(); - Path destFile = methodPath(filename); + Path destFile = methodSubPath(filename); fs.delete(destFile.getParent(), true); Path magicDest = makeMagic(destFile); assertPathDoesNotExist("Magic file should not exist", magicDest); long dataSize = data != null ? data.length : 0; - try(FSDataOutputStream stream = fs.create(magicDest, true)) { + try (FSDataOutputStream stream = fs.create(magicDest, true)) { assertIsMagicStream(stream); if (dataSize > 0) { stream.write(data); } - stream.close(); } - FileStatus status = fs.getFileStatus(magicDest); - assertEquals("Magic marker file is not zero bytes: " + status, - 0, 0); - Assertions.assertThat(extractMagicFileLength(fs, - magicDest)) - .describedAs("XAttribute " + XA_MAGIC_MARKER + " of " + magicDest) - .isNotEmpty() - .hasValue(dataSize); + getTestHelper().assertIsMarkerFile(magicDest, dataSize); commit(filename, destFile); verifyFileContents(fs, destFile, data); // the destination file doesn't have the attribute - Assertions.assertThat(extractMagicFileLength(fs, - destFile)) - .describedAs("XAttribute " + XA_MAGIC_MARKER + " of " + destFile) - .isEmpty(); + getTestHelper().assertFileLacksMarkerHeader(destFile); } /** * Commit the file, with before and after checks on the dest and magic * values. - * Failures can be set; they'll be reset after the commit. * @param filename filename of file * @param destFile destination path of file * @throws Exception any failure of the operation @@ -371,20 +382,21 @@ private void commit(String filename, Path destFile) throws IOException { + final CommitOperations actions = newCommitOperations(); validateIntermediateAndFinalPaths(magicFile, destFile); SinglePendingCommit commit = SinglePendingCommit.load(getFileSystem(), - validatePendingCommitData(filename, magicFile)); - - commitOrFail(destFile, commit, newCommitOperations()); + validatePendingCommitData(filename, magicFile), + null, + SinglePendingCommit.serializer()); - verifyCommitExists(commit); + commitOrFail(destFile, commit, actions); } private void commitOrFail(final Path destFile, final SinglePendingCommit commit, final CommitOperations actions) throws IOException { - try (CommitOperations.CommitContext commitContext - = actions.initiateCommitOperation(destFile)) { + try (CommitContext commitContext + = actions.createCommitContextForTesting(destFile, JOB_ID, 0)) { commitContext.commitOrFail(commit); } } @@ -401,26 +413,6 @@ private void validateIntermediateAndFinalPaths(Path magicFilePath, assertPathDoesNotExist("dest file was created", destFile); } - /** - * Verify that the path at the end of a commit exists. - * This does not validate the size. - * @param commit commit to verify - * @throws FileNotFoundException dest doesn't exist - * @throws ValidationFailure commit arg is invalid - * @throws IOException invalid commit, IO failure - */ - private void verifyCommitExists(SinglePendingCommit commit) - throws FileNotFoundException, ValidationFailure, IOException { - commit.validate(); - // this will force an existence check - Path path = getFileSystem().keyToQualifiedPath(commit.getDestinationKey()); - FileStatus status = getFileSystem().getFileStatus(path); - LOG.debug("Destination entry: {}", status); - if (!status.isFile()) { - throw new PathCommitException(path, "Not a file: " + status); - } - } - /** * Validate that a pending commit data file exists, load it and validate * its contents. @@ -443,14 +435,19 @@ private Path validatePendingCommitData(String filename, SinglePendingCommit persisted = SinglePendingCommit.serializer() .load(fs, pendingDataPath); persisted.validate(); - assertTrue("created timestamp wrong in " + persisted, - persisted.getCreated() > 0); - assertTrue("saved timestamp wrong in " + persisted, - persisted.getSaved() > 0); + Assertions.assertThat(persisted.getCreated()) + .describedAs("Created timestamp in %s", persisted) + .isGreaterThan(0); + Assertions.assertThat(persisted.getSaved()) + .describedAs("saved timestamp in %s", persisted) + .isGreaterThan(0); List etags = persisted.getEtags(); - assertEquals("etag list " + persisted, 1, etags.size()); - List partList = CommitOperations.toPartEtags(etags); - assertEquals("part list " + persisted, 1, partList.size()); + Assertions.assertThat(etags) + .describedAs("Etag list") + .hasSize(1); + Assertions.assertThat(CommitOperations.toPartEtags(etags)) + .describedAs("Etags to parts") + .hasSize(1); return pendingDataPath; } @@ -460,24 +457,16 @@ private Path validatePendingCommitData(String filename, * @return new path * @throws IOException failure to create/parse the path. */ - private Path methodPath(String filename) throws IOException { + private Path methodSubPath(String filename) throws IOException { return new Path(methodPath(), filename); } - /** - * Get a unique path for a method. - * @return a path - * @throws IOException - */ - protected Path methodPath() throws IOException { - return path(getMethodName()); - } - @Test public void testUploadEmptyFile() throws Throwable { + describe("Upload a zero byte file to a magic path"); File tempFile = File.createTempFile("commit", ".txt"); CommitOperations actions = newCommitOperations(); - Path dest = methodPath("testUploadEmptyFile"); + Path dest = methodSubPath("testUploadEmptyFile"); S3AFileSystem fs = getFileSystem(); fs.delete(dest, false); @@ -492,11 +481,14 @@ public void testUploadEmptyFile() throws Throwable { commitOrFail(dest, pendingCommit, actions); - FileStatus status = verifyPathExists(fs, - "uploaded file commit", dest); progress.assertCount("Progress counter should be 1.", 1); - assertEquals("File length in " + status, 0, status.getLen()); + FileStatus status = verifyPathExists(fs, + "uploaded file commit", dest); + Assertions.assertThat(status.getLen()) + .describedAs("Committed File file %s: %s", dest, status) + .isEqualTo(0); + getTestHelper().assertFileLacksMarkerHeader(dest); } @Test @@ -505,7 +497,7 @@ public void testUploadSmallFile() throws Throwable { String text = "hello, world"; FileUtils.write(tempFile, text, "UTF-8"); CommitOperations actions = newCommitOperations(); - Path dest = methodPath("testUploadSmallFile"); + Path dest = methodSubPath("testUploadSmallFile"); S3AFileSystem fs = getFileSystem(); fs.delete(dest, true); @@ -523,51 +515,57 @@ public void testUploadSmallFile() throws Throwable { commitOrFail(dest, pendingCommit, actions); String s = readUTF8(fs, dest, -1); - assertEquals(text, s); + Assertions.assertThat(s) + .describedAs("contents of committed file %s", dest) + .isEqualTo(text); progress.assertCount("Progress counter should be 1.", 1); } - @Test(expected = FileNotFoundException.class) + @Test public void testUploadMissingFile() throws Throwable { File tempFile = File.createTempFile("commit", ".txt"); tempFile.delete(); CommitOperations actions = newCommitOperations(); - Path dest = methodPath("testUploadMissingile"); - - actions.uploadFileToPendingCommit(tempFile, dest, null, - DEFAULT_MULTIPART_SIZE, progress); - progress.assertCount("Progress counter should be 1.", - 1); + Path dest = methodSubPath("testUploadMissingFile"); + intercept(FileNotFoundException.class, () -> + actions.uploadFileToPendingCommit(tempFile, dest, null, + DEFAULT_MULTIPART_SIZE, progress)); + progress.assertCount("Progress counter should be 0.", + 0); } @Test public void testRevertCommit() throws Throwable { - Path destFile = methodPath("part-0000"); + describe("Revert a commit; the destination file will be deleted"); + Path destFile = methodSubPath("part-0000"); S3AFileSystem fs = getFileSystem(); touch(fs, destFile); - CommitOperations actions = newCommitOperations(); SinglePendingCommit commit = new SinglePendingCommit(); + CommitOperations actions = newCommitOperations(); commit.setDestinationKey(fs.pathToKey(destFile)); - - actions.revertCommit(commit); - - assertPathExists("parent of reverted commit", destFile.getParent()); + newCommitOperations().revertCommit(commit); + assertPathDoesNotExist("should have been reverted", destFile); } @Test public void testRevertMissingCommit() throws Throwable { - Path destFile = methodPath("part-0000"); + Path destFile = methodSubPath("part-0000"); S3AFileSystem fs = getFileSystem(); fs.delete(destFile, false); - CommitOperations actions = newCommitOperations(); SinglePendingCommit commit = new SinglePendingCommit(); commit.setDestinationKey(fs.pathToKey(destFile)); + newCommitOperations().revertCommit(commit); + assertPathDoesNotExist("should have been reverted", destFile); + } - actions.revertCommit(commit); - - assertPathExists("parent of reverted (nonexistent) commit", - destFile.getParent()); + @Test + public void testFailuresInAbortListing() throws Throwable { + Path path = path("testFailuresInAbort"); + getFileSystem().mkdirs(path); + LOG.info("Aborting"); + newCommitOperations().abortPendingUploadsUnderPath(path); + LOG.info("Abort completed"); } /** @@ -578,16 +576,16 @@ public void testRevertMissingCommit() throws Throwable { @Test public void testWriteNormalStream() throws Throwable { S3AFileSystem fs = getFileSystem(); - assumeMagicCommitEnabled(fs); Path destFile = path("normal"); try (FSDataOutputStream out = fs.create(destFile, true)) { out.writeChars("data"); assertFalse("stream has magic output: " + out, out.hasCapability(STREAM_CAPABILITY_MAGIC_OUTPUT)); - out.close(); } FileStatus status = fs.getFileStatus(destFile); - assertTrue("Empty marker file: " + status, status.getLen() > 0); + Assertions.assertThat(status.getLen()) + .describedAs("Normal file %s: %s", destFile, status) + .isGreaterThan(0); } /** @@ -598,7 +596,7 @@ public void testBulkCommitFiles() throws Throwable { describe("verify bulk commit"); File localFile = File.createTempFile("commit", ".txt"); CommitOperations actions = newCommitOperations(); - Path destDir = methodPath("out"); + Path destDir = methodSubPath("out"); S3AFileSystem fs = getFileSystem(); fs.delete(destDir, false); @@ -612,7 +610,7 @@ public void testBulkCommitFiles() throws Throwable { destFile3); List commits = new ArrayList<>(3); - for (Path destination : destinations) { + for (Path destination: destinations) { SinglePendingCommit commit1 = actions.uploadFileToPendingCommit(localFile, destination, null, @@ -624,8 +622,8 @@ public void testBulkCommitFiles() throws Throwable { assertPathDoesNotExist("destination dir", destDir); assertPathDoesNotExist("subdirectory", subdir); LOG.info("Initiating commit operations"); - try (CommitOperations.CommitContext commitContext - = actions.initiateCommitOperation(destDir)) { + try (CommitContext commitContext + = actions.createCommitContextForTesting(destDir, JOB_ID, 0)) { LOG.info("Commit #1"); commitContext.commitOrFail(commits.get(0)); final String firstCommitContextString = commitContext.toString(); diff --git a/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/commit/ITestS3ACommitterFactory.java b/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/commit/ITestS3ACommitterFactory.java index a8547d672894a..2ad2568d5cc20 100644 --- a/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/commit/ITestS3ACommitterFactory.java +++ b/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/commit/ITestS3ACommitterFactory.java @@ -157,7 +157,7 @@ public void testBindingsInFSConfig() throws Throwable { } /** - * Create an invalid committer via the FS binding, + * Create an invalid committer via the FS binding. */ public void testInvalidFileBinding() throws Throwable { taskConfRef.unset(FS_S3A_COMMITTER_NAME); diff --git a/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/commit/TestTasks.java b/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/commit/TestTasks.java deleted file mode 100644 index 7ff5c3d280938..0000000000000 --- a/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/commit/TestTasks.java +++ /dev/null @@ -1,569 +0,0 @@ -/* - * 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.fs.s3a.commit; - -import java.io.IOException; -import java.util.Arrays; -import java.util.Collection; -import java.util.List; -import java.util.Optional; -import java.util.concurrent.ExecutorService; -import java.util.concurrent.Executors; -import java.util.concurrent.Future; -import java.util.concurrent.atomic.AtomicInteger; -import java.util.function.Function; -import java.util.stream.Collectors; -import java.util.stream.IntStream; - -import org.apache.hadoop.thirdparty.com.google.common.util.concurrent.ThreadFactoryBuilder; -import org.junit.After; -import org.junit.Before; -import org.junit.Test; -import org.junit.runner.RunWith; -import org.junit.runners.Parameterized; -import org.slf4j.Logger; -import org.slf4j.LoggerFactory; - -import org.apache.hadoop.test.HadoopTestBase; - -import static org.apache.hadoop.test.LambdaTestUtils.intercept; - -/** - * Test Tasks class. - */ -@RunWith(Parameterized.class) -public class TestTasks extends HadoopTestBase { - private static final Logger LOG = LoggerFactory.getLogger(TestTasks.class); - public static final int ITEM_COUNT = 16; - private static final int FAILPOINT = 8; - - private final int numThreads; - /** - * Thread pool for task execution. - */ - private ExecutorService threadPool; - - /** - * Task submitter bonded to the thread pool, or - * null for the 0-thread case. - */ - Tasks.Submitter submitter; - private final CounterTask failingTask - = new CounterTask("failing committer", FAILPOINT, Item::commit); - - private final FailureCounter failures - = new FailureCounter("failures", 0, null); - private final CounterTask reverter - = new CounterTask("reverter", 0, Item::revert); - private final CounterTask aborter - = new CounterTask("aborter", 0, Item::abort); - - /** - * Test array for parameterized test runs: how many threads and - * to use. Threading makes some of the assertions brittle; there are - * more checks on single thread than parallel ops. - * @return a list of parameter tuples. - */ - @Parameterized.Parameters(name = "threads={0}") - public static Collection params() { - return Arrays.asList(new Object[][]{ - {0}, - {1}, - {3}, - {8}, - {16}, - }); - } - - - private List items; - - - /** - * Construct the parameterized test. - * @param numThreads number of threads - */ - public TestTasks(int numThreads) { - this.numThreads = numThreads; - } - - /** - * In a parallel test run there is more than one thread doing the execution. - * @return true if the threadpool size is >1 - */ - public boolean isParallel() { - return numThreads > 1; - } - - @Before - public void setup() { - items = IntStream.rangeClosed(1, ITEM_COUNT) - .mapToObj(i -> new Item(i, - String.format("With %d threads", numThreads))) - .collect(Collectors.toList()); - - if (numThreads > 0) { - threadPool = Executors.newFixedThreadPool(numThreads, - new ThreadFactoryBuilder() - .setDaemon(true) - .setNameFormat(getMethodName() + "-pool-%d") - .build()); - submitter = new PoolSubmitter(); - } else { - submitter = null; - } - - } - - @After - public void teardown() { - if (threadPool != null) { - threadPool.shutdown(); - threadPool = null; - } - } - - private class PoolSubmitter implements Tasks.Submitter { - - @Override - public Future submit(final Runnable task) { - return threadPool.submit(task); - } - - } - - /** - * create the builder. - * @return pre-inited builder - */ - private Tasks.Builder builder() { - return Tasks.foreach(items).executeWith(submitter); - } - - private void assertRun(Tasks.Builder builder, - CounterTask task) throws IOException { - boolean b = builder.run(task); - assertTrue("Run of " + task + " failed", b); - } - - private void assertFailed(Tasks.Builder builder, - CounterTask task) throws IOException { - boolean b = builder.run(task); - assertFalse("Run of " + task + " unexpectedly succeeded", b); - } - - private String itemsToString() { - return "[" + items.stream().map(Item::toString) - .collect(Collectors.joining("\n")) +"]"; - } - - @Test - public void testSimpleInvocation() throws Throwable { - CounterTask t = new CounterTask("simple", 0, Item::commit); - assertRun(builder(), t); - t.assertInvoked("", ITEM_COUNT); - } - - @Test - public void testFailNoStoppingSuppressed() throws Throwable { - assertFailed(builder().suppressExceptions(), failingTask); - failingTask.assertInvoked("Continued through operations", ITEM_COUNT); - items.forEach(Item::assertCommittedOrFailed); - } - - @Test - public void testFailFastSuppressed() throws Throwable { - assertFailed(builder() - .suppressExceptions() - .stopOnFailure(), - failingTask); - if (isParallel()) { - failingTask.assertInvokedAtLeast("stop fast", FAILPOINT); - } else { - failingTask.assertInvoked("stop fast", FAILPOINT); - } - } - - @Test - public void testFailedCallAbortSuppressed() throws Throwable { - assertFailed(builder() - .stopOnFailure() - .suppressExceptions() - .abortWith(aborter), - failingTask); - failingTask.assertInvokedAtLeast("success", FAILPOINT); - if (!isParallel()) { - aborter.assertInvokedAtLeast("abort", 1); - // all uncommitted items were aborted - items.stream().filter(i -> !i.committed) - .map(Item::assertAborted); - items.stream().filter(i -> i.committed) - .forEach(i -> assertFalse(i.toString(), i.aborted)); - } - } - - @Test - public void testFailedCalledWhenNotStoppingSuppressed() throws Throwable { - assertFailed(builder() - .suppressExceptions() - .onFailure(failures), - failingTask); - failingTask.assertInvokedAtLeast("success", FAILPOINT); - // only one failure was triggered - failures.assertInvoked("failure event", 1); - } - - @Test - public void testFailFastCallRevertSuppressed() throws Throwable { - assertFailed(builder() - .stopOnFailure() - .revertWith(reverter) - .abortWith(aborter) - .suppressExceptions() - .onFailure(failures), - failingTask); - failingTask.assertInvokedAtLeast("success", FAILPOINT); - if (!isParallel()) { - aborter.assertInvokedAtLeast("abort", 1); - // all uncommitted items were aborted - items.stream().filter(i -> !i.committed) - .filter(i -> !i.failed) - .forEach(Item::assertAborted); - } - // all committed were reverted - items.stream().filter(i -> i.committed && !i.failed) - .forEach(Item::assertReverted); - // all reverted items are committed - items.stream().filter(i -> i.reverted) - .forEach(Item::assertCommitted); - - // only one failure was triggered - failures.assertInvoked("failure event", 1); - } - - @Test - public void testFailSlowCallRevertSuppressed() throws Throwable { - assertFailed(builder() - .suppressExceptions() - .revertWith(reverter) - .onFailure(failures), - failingTask); - failingTask.assertInvokedAtLeast("success", FAILPOINT); - // all committed were reverted - // identify which task failed from the set - int failing = failures.getItem().id; - items.stream() - .filter(i -> i.id != failing) - .filter(i -> i.committed) - .forEach(Item::assertReverted); - // all reverted items are committed - items.stream().filter(i -> i.reverted) - .forEach(Item::assertCommitted); - - // only one failure was triggered - failures.assertInvoked("failure event", 1); - } - - @Test - public void testFailFastExceptions() throws Throwable { - intercept(IOException.class, - () -> builder() - .stopOnFailure() - .run(failingTask)); - if (isParallel()) { - failingTask.assertInvokedAtLeast("stop fast", FAILPOINT); - } else { - failingTask.assertInvoked("stop fast", FAILPOINT); - } - } - - @Test - public void testFailSlowExceptions() throws Throwable { - intercept(IOException.class, - () -> builder() - .run(failingTask)); - failingTask.assertInvoked("continued through operations", ITEM_COUNT); - items.forEach(Item::assertCommittedOrFailed); - } - - @Test - public void testFailFastExceptionsWithAbortFailure() throws Throwable { - CounterTask failFirst = new CounterTask("task", 1, Item::commit); - CounterTask a = new CounterTask("aborter", 1, Item::abort); - intercept(IOException.class, - () -> builder() - .stopOnFailure() - .abortWith(a) - .run(failFirst)); - if (!isParallel()) { - // expect the other tasks to be aborted - a.assertInvokedAtLeast("abort", ITEM_COUNT - 1); - } - } - - @Test - public void testFailFastExceptionsWithAbortFailureStopped() throws Throwable { - CounterTask failFirst = new CounterTask("task", 1, Item::commit); - CounterTask a = new CounterTask("aborter", 1, Item::abort); - intercept(IOException.class, - () -> builder() - .stopOnFailure() - .stopAbortsOnFailure() - .abortWith(a) - .run(failFirst)); - if (!isParallel()) { - // expect the other tasks to be aborted - a.assertInvoked("abort", 1); - } - } - - /** - * Fail the last one committed, all the rest will be reverted. - * The actual ID of the last task has to be picke dup from the - * failure callback, as in the pool it may be one of any. - */ - @Test - public void testRevertAllSuppressed() throws Throwable { - CounterTask failLast = new CounterTask("task", ITEM_COUNT, Item::commit); - - assertFailed(builder() - .suppressExceptions() - .stopOnFailure() - .revertWith(reverter) - .abortWith(aborter) - .onFailure(failures), - failLast); - failLast.assertInvoked("success", ITEM_COUNT); - int abCount = aborter.getCount(); - int revCount = reverter.getCount(); - assertEquals(ITEM_COUNT, 1 + abCount + revCount); - // identify which task failed from the set - int failing = failures.getItem().id; - // all committed were reverted - items.stream() - .filter(i -> i.id != failing) - .filter(i -> i.committed) - .forEach(Item::assertReverted); - items.stream() - .filter(i -> i.id != failing) - .filter(i -> !i.committed) - .forEach(Item::assertAborted); - // all reverted items are committed - items.stream().filter(i -> i.reverted) - .forEach(Item::assertCommitted); - - // only one failure was triggered - failures.assertInvoked("failure event", 1); - } - - - /** - * The Item which tasks process. - */ - private final class Item { - private final int id; - private final String text; - - private volatile boolean committed, aborted, reverted, failed; - - private Item(int item, String text) { - this.id = item; - this.text = text; - } - - boolean commit() { - committed = true; - return true; - } - - boolean abort() { - aborted = true; - return true; - } - - boolean revert() { - reverted = true; - return true; - } - - boolean fail() { - failed = true; - return true; - } - - public Item assertCommitted() { - assertTrue(toString() + " was not committed in\n" - + itemsToString(), - committed); - return this; - } - - public Item assertCommittedOrFailed() { - assertTrue(toString() + " was not committed nor failed in\n" - + itemsToString(), - committed || failed); - return this; - } - - public Item assertAborted() { - assertTrue(toString() + " was not aborted in\n" - + itemsToString(), - aborted); - return this; - } - - public Item assertReverted() { - assertTrue(toString() + " was not reverted in\n" - + itemsToString(), - reverted); - return this; - } - - @Override - public String toString() { - final StringBuilder sb = new StringBuilder("Item{"); - sb.append(String.format("[%02d]", id)); - sb.append(", committed=").append(committed); - sb.append(", aborted=").append(aborted); - sb.append(", reverted=").append(reverted); - sb.append(", failed=").append(failed); - sb.append(", text=").append(text); - sb.append('}'); - return sb.toString(); - } - } - - /** - * Class which can count invocations and, if limit > 0, will raise - * an exception on the specific invocation of {@link #note(Object)} - * whose count == limit. - */ - private class BaseCounter { - private final AtomicInteger counter = new AtomicInteger(0); - private final int limit; - private final String name; - private Item item; - private final Optional> action; - - /** - * Base counter, tracks items. - * @param name name for string/exception/logs. - * @param limit limit at which an exception is raised, 0 == never - * @param action optional action to invoke after the increment, - * before limit check - */ - BaseCounter(String name, - int limit, - Function action) { - this.name = name; - this.limit = limit; - this.action = Optional.ofNullable(action); - } - - /** - * Apply the action to an item; log at info afterwards with both the - * before and after string values of the item. - * @param i item to process. - * @throws IOException failure in the action - */ - void process(Item i) throws IOException { - this.item = i; - int count = counter.incrementAndGet(); - if (limit == count) { - i.fail(); - LOG.info("{}: Failed {}", this, i); - throw new IOException(String.format("%s: Limit %d reached for %s", - this, limit, i)); - } - String before = i.toString(); - action.map(a -> a.apply(i)); - LOG.info("{}: {} -> {}", this, before, i); - } - - int getCount() { - return counter.get(); - } - - Item getItem() { - return item; - } - - void assertInvoked(String text, int expected) { - assertEquals(toString() + ": " + text, expected, getCount()); - } - - void assertInvokedAtLeast(String text, int expected) { - int actual = getCount(); - assertTrue(toString() + ": " + text - + "-expected " + expected - + " invocations, but got " + actual - + " in " + itemsToString(), - expected <= actual); - } - - @Override - public String toString() { - final StringBuilder sb = new StringBuilder( - "BaseCounter{"); - sb.append("name='").append(name).append('\''); - sb.append(", count=").append(counter.get()); - sb.append(", limit=").append(limit); - sb.append(", item=").append(item); - sb.append('}'); - return sb.toString(); - } - } - - private final class CounterTask - extends BaseCounter implements Tasks.Task { - - private CounterTask(String name, int limit, - Function action) { - super(name, limit, action); - } - - @Override - public void run(Item item) throws IOException { - process(item); - } - - } - - private final class FailureCounter - extends BaseCounter implements Tasks.FailureTask { - private Exception exception; - - private FailureCounter(String name, int limit, - Function action) { - super(name, limit, action); - } - - @Override - public void run(Item item, Exception ex) throws IOException { - process(item); - this.exception = ex; - } - - private Exception getException() { - return exception; - } - - } - -} diff --git a/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/commit/magic/ITestMagicCommitProtocol.java b/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/commit/magic/ITestMagicCommitProtocol.java index 5265163d83fc6..32cab03770c2d 100644 --- a/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/commit/magic/ITestMagicCommitProtocol.java +++ b/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/commit/magic/ITestMagicCommitProtocol.java @@ -25,8 +25,6 @@ import org.assertj.core.api.Assertions; import org.junit.Test; -import org.apache.hadoop.conf.Configuration; -import org.apache.hadoop.fs.FileStatus; import org.apache.hadoop.fs.LocatedFileStatus; import org.apache.hadoop.fs.Path; import org.apache.hadoop.fs.contract.ContractTestUtils; @@ -34,7 +32,6 @@ import org.apache.hadoop.fs.s3a.commit.AbstractITCommitProtocol; import org.apache.hadoop.fs.s3a.commit.AbstractS3ACommitter; import org.apache.hadoop.fs.s3a.commit.CommitConstants; -import org.apache.hadoop.fs.s3a.commit.CommitOperations; import org.apache.hadoop.fs.s3a.commit.CommitUtils; import org.apache.hadoop.fs.s3a.commit.CommitterFaultInjection; import org.apache.hadoop.fs.s3a.commit.CommitterFaultInjectionImpl; @@ -45,7 +42,7 @@ import static org.apache.hadoop.fs.s3a.S3AUtils.listAndFilter; import static org.apache.hadoop.fs.s3a.commit.CommitConstants.*; -import static org.hamcrest.CoreMatchers.containsString; +import static org.apache.hadoop.util.functional.RemoteIterators.toList; /** * Test the magic committer's commit protocol. @@ -99,8 +96,9 @@ public MagicS3GuardCommitter createFailingCommitter( protected void validateTaskAttemptPathDuringWrite(Path p, final long expectedLength) throws IOException { String pathStr = p.toString(); - assertTrue("not magic " + pathStr, - pathStr.contains(MAGIC)); + Assertions.assertThat(pathStr) + .describedAs("Magic path") + .contains(MAGIC); assertPathDoesNotExist("task attempt visible", p); } @@ -116,9 +114,9 @@ protected void validateTaskAttemptPathAfterWrite(Path marker, // if you list the parent dir and find the marker, it // is really 0 bytes long String name = marker.getName(); - List filtered = listAndFilter(fs, + List filtered = toList(listAndFilter(fs, marker.getParent(), false, - (path) -> path.getName().equals(name)); + (path) -> path.getName().equals(name))); Assertions.assertThat(filtered) .hasSize(1); Assertions.assertThat(filtered.get(0)) @@ -126,14 +124,7 @@ protected void validateTaskAttemptPathAfterWrite(Path marker, "Listing should return 0 byte length"); // marker file is empty - FileStatus st = fs.getFileStatus(marker); - assertEquals("file length in " + st, 0, st.getLen()); - // xattr header - Assertions.assertThat(CommitOperations.extractMagicFileLength(fs, - marker)) - .describedAs("XAttribute " + XA_MAGIC_MARKER) - .isNotEmpty() - .hasValue(expectedLength); + getTestHelper().assertIsMarkerFile(marker, expectedLength); } /** @@ -151,8 +142,8 @@ protected void validateTaskAttemptWorkingDirectory( assertEquals("Wrong schema for working dir " + wd + " with committer " + committer, "s3a", wd.getScheme()); - assertThat(wd.getPath(), - containsString('/' + CommitConstants.MAGIC + '/')); + Assertions.assertThat(wd.getPath()) + .contains('/' + CommitConstants.MAGIC + '/'); } /** diff --git a/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/commit/magic/ITestS3AHugeMagicCommits.java b/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/commit/magic/ITestS3AHugeMagicCommits.java index 3c15454e7edfb..aeea195aacf89 100644 --- a/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/commit/magic/ITestS3AHugeMagicCommits.java +++ b/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/commit/magic/ITestS3AHugeMagicCommits.java @@ -20,8 +20,8 @@ import java.io.IOException; import java.util.List; -import java.util.stream.Collectors; +import org.assertj.core.api.Assertions; import org.slf4j.Logger; import org.slf4j.LoggerFactory; @@ -33,10 +33,11 @@ import org.apache.hadoop.fs.s3a.Constants; import org.apache.hadoop.fs.s3a.S3AFileSystem; import org.apache.hadoop.fs.s3a.commit.CommitConstants; -import org.apache.hadoop.fs.s3a.commit.CommitOperations; import org.apache.hadoop.fs.s3a.commit.CommitUtils; import org.apache.hadoop.fs.s3a.commit.files.PendingSet; import org.apache.hadoop.fs.s3a.commit.files.SinglePendingCommit; +import org.apache.hadoop.fs.s3a.commit.impl.CommitContext; +import org.apache.hadoop.fs.s3a.commit.impl.CommitOperations; import org.apache.hadoop.fs.s3a.scale.AbstractSTestS3AHugeFiles; import static org.apache.hadoop.fs.s3a.MultipartTestUtils.listMultipartUploads; @@ -54,6 +55,7 @@ public class ITestS3AHugeMagicCommits extends AbstractSTestS3AHugeFiles { private static final Logger LOG = LoggerFactory.getLogger( ITestS3AHugeMagicCommits.class); + private static final int COMMITTER_THREADS = 64; private Path magicDir; private Path jobDir; @@ -123,16 +125,16 @@ public void test_030_postCreationAssertions() throws Throwable { Path destDir = getHugefile().getParent(); assertPathExists("Magic dir", new Path(destDir, CommitConstants.MAGIC)); String destDirKey = fs.pathToKey(destDir); - List uploads = listMultipartUploads(fs, destDirKey); - assertEquals("Pending uploads: " - + uploads.stream() - .collect(Collectors.joining("\n")), 1, uploads.size()); + Assertions.assertThat(listMultipartUploads(fs, destDirKey)) + .describedAs("Pending uploads") + .hasSize(1); assertNotNull("jobDir", jobDir); - Pair>> - results = operations.loadSinglePendingCommits(jobDir, false); - try(CommitOperations.CommitContext commitContext - = operations.initiateCommitOperation(jobDir)) { + try(CommitContext commitContext + = operations.createCommitContextForTesting(jobDir, null, COMMITTER_THREADS)) { + Pair>> + results = operations.loadSinglePendingCommits(jobDir, false, commitContext + ); for (SinglePendingCommit singlePendingCommit : results.getKey().getCommits()) { commitContext.commitOrFail(singlePendingCommit); @@ -140,10 +142,9 @@ public void test_030_postCreationAssertions() throws Throwable { } timer.end("time to commit %s", pendingDataFile); // upload is no longer pending - uploads = listMultipartUploads(fs, destDirKey); - assertEquals("Pending uploads" - + uploads.stream().collect(Collectors.joining("\n")), - 0, operations.listPendingUploadsUnderPath(destDir).size()); + Assertions.assertThat(operations.listPendingUploadsUnderPath(destDir)) + .describedAs("Pending uploads undedr path") + .isEmpty(); // at this point, the huge file exists, so the normal assertions // on that file must be valid. Verify. super.test_030_postCreationAssertions(); diff --git a/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/commit/staging/MockedStagingCommitter.java b/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/commit/staging/MockedStagingCommitter.java index d3da8185c8d65..beccf4f328c05 100644 --- a/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/commit/staging/MockedStagingCommitter.java +++ b/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/commit/staging/MockedStagingCommitter.java @@ -26,6 +26,7 @@ import org.apache.hadoop.fs.FileSystem; import org.apache.hadoop.fs.Path; import org.apache.hadoop.fs.s3a.MockS3AFileSystem; +import org.apache.hadoop.fs.s3a.commit.files.SuccessData; import org.apache.hadoop.fs.s3a.commit.staging.StagingTestBase.ClientErrors; import org.apache.hadoop.fs.s3a.commit.staging.StagingTestBase.ClientResults; import org.apache.hadoop.fs.statistics.IOStatisticsSnapshot; @@ -76,11 +77,12 @@ public void commitJob(JobContext context) throws IOException { } @Override - protected void maybeCreateSuccessMarker(JobContext context, + protected SuccessData maybeCreateSuccessMarker(JobContext context, List filenames, final IOStatisticsSnapshot ioStatistics) throws IOException { //skipped + return null; } public ClientResults getResults() throws IOException { diff --git a/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/commit/staging/StagingTestBase.java b/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/commit/staging/StagingTestBase.java index 6e13fd0227a3b..6f2953762439a 100644 --- a/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/commit/staging/StagingTestBase.java +++ b/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/commit/staging/StagingTestBase.java @@ -105,10 +105,15 @@ public class StagingTestBase { /** The raw bucket URI Path before any canonicalization. */ public static final URI RAW_BUCKET_URI = RAW_BUCKET_PATH.toUri(); - public static Path outputPath = + + @SuppressWarnings("StaticNonFinalField") + private static Path outputPath = new Path("s3a://" + BUCKET + "/" + OUTPUT_PREFIX); - public static URI outputPathUri = outputPath.toUri(); - public static Path root; + + @SuppressWarnings("StaticNonFinalField") + private static URI outputPathUri = getOutputPath().toUri(); + @SuppressWarnings("StaticNonFinalField") + private static Path root; protected StagingTestBase() { } @@ -131,8 +136,8 @@ protected static S3AFileSystem createAndBindMockFSInstance(Configuration conf, URI uri = RAW_BUCKET_URI; wrapperFS.initialize(uri, conf); root = wrapperFS.makeQualified(new Path("/")); - outputPath = new Path(root, OUTPUT_PREFIX); - outputPathUri = outputPath.toUri(); + outputPath = new Path(getRoot(), OUTPUT_PREFIX); + outputPathUri = getOutputPath().toUri(); FileSystemTestHelper.addFileSystemForTesting(uri, conf, wrapperFS); return mockFs; } @@ -154,7 +159,7 @@ private static S3AFileSystem mockS3AFileSystemRobustly() { */ public static MockS3AFileSystem lookupWrapperFS(Configuration conf) throws IOException { - return (MockS3AFileSystem) FileSystem.get(outputPathUri, conf); + return (MockS3AFileSystem) FileSystem.get(getOutputPathUri(), conf); } public static void verifyCompletion(FileSystem mockS3) throws IOException { @@ -169,13 +174,13 @@ public static void verifyDeleted(FileSystem mockS3, Path path) public static void verifyDeleted(FileSystem mockS3, String child) throws IOException { - verifyDeleted(mockS3, new Path(outputPath, child)); + verifyDeleted(mockS3, new Path(getOutputPath(), child)); } public static void verifyCleanupTempFiles(FileSystem mockS3) throws IOException { verifyDeleted(mockS3, - new Path(outputPath, CommitConstants.TEMPORARY)); + new Path(getOutputPath(), CommitConstants.TEMPORARY)); } protected static void assertConflictResolution( @@ -189,7 +194,7 @@ protected static void assertConflictResolution( public static void pathsExist(FileSystem mockS3, String... children) throws IOException { for (String child : children) { - pathExists(mockS3, new Path(outputPath, child)); + pathExists(mockS3, new Path(getOutputPath(), child)); } } @@ -231,7 +236,7 @@ public static void mkdirsHasOutcome(FileSystem mockS3, public static void canDelete(FileSystem mockS3, String... children) throws IOException { for (String child : children) { - canDelete(mockS3, new Path(outputPath, child)); + canDelete(mockS3, new Path(getOutputPath(), child)); } } @@ -243,7 +248,7 @@ public static void canDelete(FileSystem mockS3, Path f) throws IOException { public static void verifyExistenceChecked(FileSystem mockS3, String child) throws IOException { - verifyExistenceChecked(mockS3, new Path(outputPath, child)); + verifyExistenceChecked(mockS3, new Path(getOutputPath(), child)); } public static void verifyExistenceChecked(FileSystem mockS3, Path path) @@ -262,6 +267,18 @@ public static void verifyMkdirsInvoked(FileSystem mockS3, Path path) verify(mockS3).mkdirs(path); } + protected static URI getOutputPathUri() { + return outputPathUri; + } + + static Path getRoot() { + return root; + } + + static Path getOutputPath() { + return outputPath; + } + /** * Provides setup/teardown of a MiniDFSCluster for tests that need one. */ diff --git a/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/commit/staging/TestDirectoryCommitterScale.java b/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/commit/staging/TestDirectoryCommitterScale.java index cb7202b10d133..4d24c07dacfe2 100644 --- a/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/commit/staging/TestDirectoryCommitterScale.java +++ b/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/commit/staging/TestDirectoryCommitterScale.java @@ -28,6 +28,7 @@ import java.util.stream.IntStream; import com.amazonaws.services.s3.model.PartETag; + import org.apache.hadoop.thirdparty.com.google.common.collect.Maps; import org.assertj.core.api.Assertions; import org.junit.AfterClass; @@ -48,18 +49,21 @@ import org.apache.hadoop.fs.s3a.commit.CommitConstants; import org.apache.hadoop.fs.s3a.commit.files.PendingSet; import org.apache.hadoop.fs.s3a.commit.files.SinglePendingCommit; +import org.apache.hadoop.fs.s3a.commit.impl.CommitContext; +import org.apache.hadoop.fs.s3a.commit.impl.CommitOperations; import org.apache.hadoop.mapred.JobConf; import org.apache.hadoop.mapreduce.JobContext; import org.apache.hadoop.mapreduce.JobStatus; import org.apache.hadoop.mapreduce.TaskAttemptContext; import org.apache.hadoop.util.DurationInfo; +import org.apache.hadoop.util.JsonSerialization; import static org.apache.hadoop.fs.s3a.commit.CommitConstants.CONFLICT_MODE_APPEND; import static org.apache.hadoop.fs.s3a.commit.CommitConstants.FS_S3A_COMMITTER_STAGING_CONFLICT_MODE; import static org.apache.hadoop.fs.s3a.commit.CommitConstants.PENDINGSET_SUFFIX; import static org.apache.hadoop.fs.s3a.commit.staging.StagingTestBase.BUCKET; -import static org.apache.hadoop.fs.s3a.commit.staging.StagingTestBase.outputPath; -import static org.apache.hadoop.fs.s3a.commit.staging.StagingTestBase.outputPathUri; +import static org.apache.hadoop.fs.s3a.commit.staging.StagingTestBase.getOutputPath; +import static org.apache.hadoop.fs.s3a.commit.staging.StagingTestBase.getOutputPathUri; import static org.apache.hadoop.fs.s3a.commit.staging.StagingTestBase.pathIsDirectory; /** @@ -83,6 +87,7 @@ public class TestDirectoryCommitterScale public static final int TOTAL_COMMIT_COUNT = FILES_PER_TASK * TASKS; public static final int BLOCKS_PER_TASK = 1000; + private static final int COMMITTER_THREAD_COUNT = 100; private static File stagingDir; @@ -95,13 +100,13 @@ public class TestDirectoryCommitterScale @Override DirectoryCommitterForTesting newJobCommitter() throws Exception { - return new DirectoryCommitterForTesting(outputPath, + return new DirectoryCommitterForTesting(getOutputPath(), createTaskAttemptForJob()); } @BeforeClass public static void setupStaging() throws Exception { - stagingDir = File.createTempFile("staging", ""); + stagingDir = File.createTempFile("staging", null); stagingDir.delete(); stagingDir.mkdir(); stagingPath = new Path(stagingDir.toURI()); @@ -125,7 +130,7 @@ protected JobConf createJobConf() { JobConf conf = super.createJobConf(); conf.setInt( CommitConstants.FS_S3A_COMMITTER_THREADS, - 100); + COMMITTER_THREAD_COUNT); return conf; } @@ -149,7 +154,8 @@ public void test_010_createTaskFiles() throws Exception { */ private void createTasks() throws IOException { // create a stub multipart commit containing multiple files. - + JsonSerialization serializer = + SinglePendingCommit.serializer(); // step1: a list of tags. // this is the md5sum of hadoop 3.2.1.tar String tag = "9062dcf18ffaee254821303bbd11c72b"; @@ -164,12 +170,14 @@ private void createTasks() throws IOException { // these get overwritten base.setDestinationKey("/base"); base.setUploadId("uploadId"); - base.setUri(outputPathUri.toString()); + base.setUri(getOutputPathUri().toString()); + byte[] bytes = base.toBytes(serializer); SinglePendingCommit[] singles = new SinglePendingCommit[FILES_PER_TASK]; - byte[] bytes = base.toBytes(); + for (int i = 0; i < FILES_PER_TASK; i++) { - singles[i] = SinglePendingCommit.serializer().fromBytes(bytes); + + singles[i] = serializer.fromBytes(bytes); } // now create the files, using this as the template @@ -182,7 +190,7 @@ private void createTasks() throws IOException { String uploadId = String.format("%05d-task-%04d-file-%02d", uploadCount, task, i); // longer paths to take up more space. - Path p = new Path(outputPath, + Path p = new Path(getOutputPath(), "datasets/examples/testdirectoryscale/" + "year=2019/month=09/day=26/hour=20/second=53" + uploadId); @@ -199,7 +207,7 @@ private void createTasks() throws IOException { } Path path = new Path(stagingPath, String.format("task-%04d." + PENDINGSET_SUFFIX, task)); - pending.save(localFS, path, true); + pending.save(localFS, path, PendingSet.serializer()); } } @@ -211,12 +219,14 @@ public void test_020_loadFilesToAttempt() throws Exception { Configuration jobConf = getJobConf(); jobConf.set( FS_S3A_COMMITTER_STAGING_CONFLICT_MODE, CONFLICT_MODE_APPEND); - FileSystem mockS3 = getMockS3A(); - pathIsDirectory(mockS3, outputPath); - try (DurationInfo ignored = - new DurationInfo(LOG, "listing pending uploads")) { + S3AFileSystem mockS3 = getMockS3A(); + pathIsDirectory(mockS3, getOutputPath()); + final CommitOperations operations = new CommitOperations(getWrapperFS()); + try (CommitContext commitContext + = operations.createCommitContextForTesting(getOutputPath(), + null, COMMITTER_THREAD_COUNT)) { AbstractS3ACommitter.ActiveCommit activeCommit - = committer.listPendingUploadsToCommit(getJob()); + = committer.listPendingUploadsToCommit(commitContext); Assertions.assertThat(activeCommit.getSourceFiles()) .describedAs("Source files of %s", activeCommit) .hasSize(TASKS); @@ -232,7 +242,7 @@ public void test_030_commitFiles() throws Exception { jobConf.set( FS_S3A_COMMITTER_STAGING_CONFLICT_MODE, CONFLICT_MODE_APPEND); S3AFileSystem mockS3 = getMockS3A(); - pathIsDirectory(mockS3, outputPath); + pathIsDirectory(mockS3, getOutputPath()); try (DurationInfo ignored = new DurationInfo(LOG, "Committing Job")) { @@ -261,7 +271,7 @@ public void test_040_abortFiles() throws Exception { jobConf.set( FS_S3A_COMMITTER_STAGING_CONFLICT_MODE, CONFLICT_MODE_APPEND); FileSystem mockS3 = getMockS3A(); - pathIsDirectory(mockS3, outputPath); + pathIsDirectory(mockS3, getOutputPath()); committer.abortJob(getJob(), JobStatus.State.FAILED); } @@ -304,11 +314,11 @@ public Path getJobAttemptPath(JobContext context) { } @Override - protected void commitJobInternal(final JobContext context, + protected void commitJobInternal(final CommitContext commitContext, final ActiveCommit pending) throws IOException { activeCommit = pending; - super.commitJobInternal(context, pending); + super.commitJobInternal(commitContext, pending); } } } diff --git a/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/commit/staging/TestStagingCommitter.java b/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/commit/staging/TestStagingCommitter.java index 94c0b29a3cdcc..11edf0d216376 100644 --- a/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/commit/staging/TestStagingCommitter.java +++ b/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/commit/staging/TestStagingCommitter.java @@ -37,7 +37,6 @@ import org.apache.hadoop.util.Sets; import org.assertj.core.api.Assertions; -import org.hamcrest.core.StringStartsWith; import org.junit.After; import org.junit.Before; import org.junit.Test; @@ -58,6 +57,7 @@ import org.apache.hadoop.fs.s3a.commit.AbstractS3ACommitter; import org.apache.hadoop.fs.s3a.commit.PathCommitException; import org.apache.hadoop.fs.s3a.commit.files.PendingSet; +import org.apache.hadoop.fs.s3a.commit.files.PersistentCommitData; import org.apache.hadoop.fs.s3a.commit.files.SinglePendingCommit; import org.apache.hadoop.mapred.JobConf; import org.apache.hadoop.mapreduce.JobContext; @@ -168,7 +168,7 @@ public void setupCommitter() throws Exception { this.tac = new TaskAttemptContextImpl( new Configuration(job.getConfiguration()), AID); - this.jobCommitter = new MockedStagingCommitter(outputPath, tac); + this.jobCommitter = new MockedStagingCommitter(getOutputPath(), tac); jobCommitter.setupJob(job); // get the task's configuration copy so modifications take effect @@ -183,7 +183,7 @@ public void setupCommitter() throws Exception { this.conf.set(BUFFER_DIR, String.format("%s/local-0/, %s/local-1 ", tmp, tmp)); - this.committer = new MockedStagingCommitter(outputPath, tac); + this.committer = new MockedStagingCommitter(getOutputPath(), tac); Paths.resetTempFolderCache(); } @@ -335,10 +335,11 @@ public void testAttemptPathConstructionWithSchema() throws Exception { config.set(BUFFER_DIR, "file:/tmp/mr-local-0,file:/tmp/mr-local-1"); - assertThat("Path should be the same with file scheme", + Assertions.assertThat( getLocalTaskAttemptTempDir(config, - jobUUID, tac.getTaskAttemptID()).toString(), - StringStartsWith.startsWith(commonPath)); + jobUUID, tac.getTaskAttemptID()).toString()) + .describedAs("Path should be the same with file scheme") + .startsWith(commonPath); } @Test @@ -379,7 +380,7 @@ public void testSingleTaskCommit() throws Exception { assertEquals("Should name the commits file with the task ID: " + results, "task_job_0001_r_000002", stats[0].getPath().getName()); - PendingSet pending = PendingSet.load(dfs, stats[0]); + PendingSet pending = PersistentCommitData.load(dfs, stats[0], PendingSet.serializer()); assertEquals("Should have one pending commit", 1, pending.size()); SinglePendingCommit commit = pending.getCommits().get(0); assertEquals("Should write to the correct bucket:" + results, @@ -419,7 +420,7 @@ public void testSingleTaskEmptyFileCommit() throws Exception { assertEquals("Should name the commits file with the task ID", "task_job_0001_r_000002", stats[0].getPath().getName()); - PendingSet pending = PendingSet.load(dfs, stats[0]); + PendingSet pending = PersistentCommitData.load(dfs, stats[0], PendingSet.serializer()); assertEquals("Should have one pending commit", 1, pending.size()); } @@ -442,7 +443,7 @@ public void testSingleTaskMultiFileCommit() throws Exception { "task_job_0001_r_000002", stats[0].getPath().getName()); List pending = - PendingSet.load(dfs, stats[0]).getCommits(); + PersistentCommitData.load(dfs, stats[0], PendingSet.serializer()).getCommits(); assertEquals("Should have correct number of pending commits", files.size(), pending.size()); @@ -717,7 +718,7 @@ private Set runTasks(JobContext jobContext, TaskAttemptContext attempt = new TaskAttemptContextImpl( new Configuration(jobContext.getConfiguration()), attemptID); MockedStagingCommitter taskCommitter = new MockedStagingCommitter( - outputPath, attempt); + getOutputPath(), attempt); commitTask(taskCommitter, attempt, numFiles); } diff --git a/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/commit/staging/TestStagingDirectoryOutputCommitter.java b/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/commit/staging/TestStagingDirectoryOutputCommitter.java index 98075b827a7c2..b92605ca25357 100644 --- a/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/commit/staging/TestStagingDirectoryOutputCommitter.java +++ b/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/commit/staging/TestStagingDirectoryOutputCommitter.java @@ -29,8 +29,11 @@ import org.apache.hadoop.conf.Configuration; import org.apache.hadoop.fs.FileSystem; import org.apache.hadoop.fs.PathExistsException; +import org.apache.hadoop.fs.s3a.S3AFileSystem; import org.apache.hadoop.fs.s3a.commit.AbstractS3ACommitter; import org.apache.hadoop.fs.s3a.commit.InternalCommitterConstants; +import org.apache.hadoop.fs.s3a.commit.impl.CommitContext; +import org.apache.hadoop.fs.s3a.commit.impl.CommitOperations; import static org.apache.hadoop.fs.s3a.commit.CommitConstants.*; import static org.apache.hadoop.fs.s3a.commit.staging.StagingTestBase.*; @@ -46,7 +49,7 @@ public class TestStagingDirectoryOutputCommitter @Override DirectoryStagingCommitter newJobCommitter() throws Exception { - return new DirectoryStagingCommitter(outputPath, + return new DirectoryStagingCommitter(getOutputPath(), createTaskAttemptForJob()); } @@ -63,7 +66,7 @@ public void testBadConflictMode() throws Throwable { public void testDefaultConflictResolution() throws Exception { getJob().getConfiguration().unset( FS_S3A_COMMITTER_STAGING_CONFLICT_MODE); - pathIsDirectory(getMockS3A(), outputPath); + pathIsDirectory(getMockS3A(), getOutputPath()); verifyJobSetupAndCommit(); } @@ -75,7 +78,8 @@ public void testFailConflictResolution() throws Exception { } protected void verifyFailureConflictOutcome() throws Exception { - pathIsDirectory(getMockS3A(), outputPath); + final S3AFileSystem mockFS = getMockS3A(); + pathIsDirectory(mockFS, getOutputPath()); final DirectoryStagingCommitter committer = newJobCommitter(); // this should fail @@ -86,20 +90,23 @@ protected void verifyFailureConflictOutcome() throws Exception { // but there are no checks in job commit (HADOOP-15469) // this is done by calling the preCommit method directly, - committer.preCommitJob(getJob(), AbstractS3ACommitter.ActiveCommit.empty()); - reset(getMockS3A()); - pathDoesNotExist(getMockS3A(), outputPath); + final CommitContext commitContext = new CommitOperations(getWrapperFS()). + createCommitContext(getJob(), getOutputPath(), 0); + committer.preCommitJob(commitContext, AbstractS3ACommitter.ActiveCommit.empty()); + + reset(mockFS); + pathDoesNotExist(mockFS, getOutputPath()); committer.setupJob(getJob()); - verifyExistenceChecked(getMockS3A(), outputPath); - verifyMkdirsInvoked(getMockS3A(), outputPath); - verifyNoMoreInteractions(getMockS3A()); + verifyExistenceChecked(mockFS, getOutputPath()); + verifyMkdirsInvoked(mockFS, getOutputPath()); + verifyNoMoreInteractions(mockFS); - reset(getMockS3A()); - pathDoesNotExist(getMockS3A(), outputPath); + reset(mockFS); + pathDoesNotExist(mockFS, getOutputPath()); committer.commitJob(getJob()); - verifyCompletion(getMockS3A()); + verifyCompletion(mockFS); } @Test @@ -108,7 +115,7 @@ public void testAppendConflictResolution() throws Exception { getJob().getConfiguration().set( FS_S3A_COMMITTER_STAGING_CONFLICT_MODE, CONFLICT_MODE_APPEND); FileSystem mockS3 = getMockS3A(); - pathIsDirectory(mockS3, outputPath); + pathIsDirectory(mockS3, getOutputPath()); verifyJobSetupAndCommit(); } @@ -120,7 +127,7 @@ protected void verifyJobSetupAndCommit() FileSystem mockS3 = getMockS3A(); Mockito.reset(mockS3); - pathExists(mockS3, outputPath); + pathExists(mockS3, getOutputPath()); committer.commitJob(getJob()); verifyCompletion(mockS3); @@ -130,7 +137,7 @@ protected void verifyJobSetupAndCommit() public void testReplaceConflictResolution() throws Exception { FileSystem mockS3 = getMockS3A(); - pathIsDirectory(mockS3, outputPath); + pathIsDirectory(mockS3, getOutputPath()); getJob().getConfiguration().set( FS_S3A_COMMITTER_STAGING_CONFLICT_MODE, CONFLICT_MODE_REPLACE); @@ -140,17 +147,17 @@ public void testReplaceConflictResolution() throws Exception { committer.setupJob(getJob()); Mockito.reset(mockS3); - pathExists(mockS3, outputPath); - canDelete(mockS3, outputPath); + pathExists(mockS3, getOutputPath()); + canDelete(mockS3, getOutputPath()); committer.commitJob(getJob()); - verifyDeleted(mockS3, outputPath); + verifyDeleted(mockS3, getOutputPath()); verifyCompletion(mockS3); } @Test public void testReplaceConflictFailsIfDestIsFile() throws Exception { - pathIsFile(getMockS3A(), outputPath); + pathIsFile(getMockS3A(), getOutputPath()); getJob().getConfiguration().set( FS_S3A_COMMITTER_STAGING_CONFLICT_MODE, CONFLICT_MODE_REPLACE); @@ -166,7 +173,7 @@ public void testReplaceConflictFailsIfDestIsFile() throws Exception { @Test public void testAppendConflictFailsIfDestIsFile() throws Exception { - pathIsFile(getMockS3A(), outputPath); + pathIsFile(getMockS3A(), getOutputPath()); getJob().getConfiguration().set( FS_S3A_COMMITTER_STAGING_CONFLICT_MODE, CONFLICT_MODE_APPEND); diff --git a/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/commit/staging/TestStagingPartitionedFileListing.java b/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/commit/staging/TestStagingPartitionedFileListing.java index 76a0de225371e..64a9be0888ffa 100644 --- a/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/commit/staging/TestStagingPartitionedFileListing.java +++ b/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/commit/staging/TestStagingPartitionedFileListing.java @@ -51,13 +51,13 @@ public class TestStagingPartitionedFileListing @Override PartitionedStagingCommitter newJobCommitter() throws IOException { - return new PartitionedStagingCommitter(outputPath, + return new PartitionedStagingCommitter(getOutputPath(), createTaskAttemptForJob()); } @Override PartitionedStagingCommitter newTaskCommitter() throws IOException { - return new PartitionedStagingCommitter(outputPath, getTAC()); + return new PartitionedStagingCommitter(getOutputPath(), getTAC()); } private FileSystem attemptFS; diff --git a/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/commit/staging/TestStagingPartitionedJobCommit.java b/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/commit/staging/TestStagingPartitionedJobCommit.java index 86b677c70a305..28161979f0b79 100644 --- a/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/commit/staging/TestStagingPartitionedJobCommit.java +++ b/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/commit/staging/TestStagingPartitionedJobCommit.java @@ -34,7 +34,7 @@ import org.apache.hadoop.fs.s3a.commit.PathCommitException; import org.apache.hadoop.fs.s3a.commit.files.PendingSet; import org.apache.hadoop.fs.s3a.commit.files.SinglePendingCommit; -import org.apache.hadoop.mapreduce.JobContext; +import org.apache.hadoop.fs.s3a.commit.impl.CommitContext; import org.apache.hadoop.mapreduce.TaskAttemptContext; import static org.apache.hadoop.test.LambdaTestUtils.intercept; @@ -67,7 +67,7 @@ private final class PartitionedStagingCommitterForTesting private PartitionedStagingCommitterForTesting(TaskAttemptContext context) throws IOException { - super(StagingTestBase.outputPath, context); + super(StagingTestBase.getOutputPath(), context); } /** @@ -75,14 +75,15 @@ private PartitionedStagingCommitterForTesting(TaskAttemptContext context) * This is quite complex as the mock pending uploads need to be saved * to a filesystem for the next stage of the commit process. * To simulate multiple commit, more than one .pendingset file is created, - * @param context job context + * @param commitContext job context * @return an active commit containing a list of paths to valid pending set * file. * @throws IOException IO failure */ + @SuppressWarnings("CollectionDeclaredAsConcreteClass") @Override protected ActiveCommit listPendingUploadsToCommit( - JobContext context) throws IOException { + CommitContext commitContext) throws IOException { LocalFileSystem localFS = FileSystem.getLocal(getConf()); ActiveCommit activeCommit = new ActiveCommit(localFS, @@ -109,17 +110,17 @@ protected ActiveCommit listPendingUploadsToCommit( File file = File.createTempFile("staging", ".pendingset"); file.deleteOnExit(); Path path = new Path(file.toURI()); - pendingSet.save(localFS, path, true); + pendingSet.save(localFS, path, PendingSet.serializer()); activeCommit.add(localFS.getFileStatus(path)); } return activeCommit; } @Override - protected void abortJobInternal(JobContext context, + protected void abortJobInternal(CommitContext commitContext, boolean suppressExceptions) throws IOException { this.aborted = true; - super.abortJobInternal(context, suppressExceptions); + super.abortJobInternal(commitContext, suppressExceptions); } } @@ -242,7 +243,7 @@ public void testReplaceWithDeleteFailure() throws Exception { pathsExist(mockS3, "dateint=20161116/hour=14"); when(mockS3 .delete( - new Path(outputPath, "dateint=20161116/hour=14"), + new Path(getOutputPath(), "dateint=20161116/hour=14"), true)) .thenThrow(new PathCommitException("fake", "Fake IOException for delete")); diff --git a/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/commit/staging/TestStagingPartitionedTaskCommit.java b/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/commit/staging/TestStagingPartitionedTaskCommit.java index fb252102491d6..4e82b94314d34 100644 --- a/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/commit/staging/TestStagingPartitionedTaskCommit.java +++ b/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/commit/staging/TestStagingPartitionedTaskCommit.java @@ -47,13 +47,13 @@ public class TestStagingPartitionedTaskCommit @Override PartitionedStagingCommitter newJobCommitter() throws IOException { - return new PartitionedStagingCommitter(outputPath, + return new PartitionedStagingCommitter(getOutputPath(), createTaskAttemptForJob()); } @Override PartitionedStagingCommitter newTaskCommitter() throws Exception { - return new PartitionedStagingCommitter(outputPath, getTAC()); + return new PartitionedStagingCommitter(getOutputPath(), getTAC()); } // The set of files used by this test @@ -104,7 +104,7 @@ public void testFail() throws Exception { // test failure when one partition already exists reset(mockS3); - Path existsPath = new Path(outputPath, relativeFiles.get(1)).getParent(); + Path existsPath = new Path(getOutputPath(), relativeFiles.get(1)).getParent(); pathExists(mockS3, existsPath); intercept(PathExistsException.class, "", @@ -133,7 +133,7 @@ public void testAppend() throws Exception { // test success when one partition already exists reset(mockS3); - pathExists(mockS3, new Path(outputPath, relativeFiles.get(2)).getParent()); + pathExists(mockS3, new Path(getOutputPath(), relativeFiles.get(2)).getParent()); committer.commitTask(getTAC()); verifyFilesCreated(committer); @@ -178,7 +178,7 @@ public void testReplace() throws Exception { // test success when one partition already exists reset(mockS3); - pathExists(mockS3, new Path(outputPath, relativeFiles.get(3)).getParent()); + pathExists(mockS3, new Path(getOutputPath(), relativeFiles.get(3)).getParent()); committer.commitTask(getTAC()); verifyFilesCreated(committer); diff --git a/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/commit/staging/integration/ITestStagingCommitProtocol.java b/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/commit/staging/integration/ITestStagingCommitProtocol.java index 3a820bcc11e21..bb3031b32c1a8 100644 --- a/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/commit/staging/integration/ITestStagingCommitProtocol.java +++ b/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/commit/staging/integration/ITestStagingCommitProtocol.java @@ -52,7 +52,7 @@ protected String suitename() { @Override protected Configuration createConfiguration() { Configuration conf = super.createConfiguration(); - conf.setInt(FS_S3A_COMMITTER_THREADS, 1); + conf.setInt(FS_S3A_COMMITTER_THREADS, 4); // disable unique filenames so that the protocol tests of FileOutputFormat // and this test generate consistent names. diff --git a/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/commit/terasort/ITestTerasortOnS3A.java b/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/commit/terasort/ITestTerasortOnS3A.java index 991969b0f05ce..d28ee5172b632 100644 --- a/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/commit/terasort/ITestTerasortOnS3A.java +++ b/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/commit/terasort/ITestTerasortOnS3A.java @@ -330,7 +330,7 @@ public void test_140_teracomplete() throws Throwable { stage.accept("teravalidate"); stage.accept("overall"); String text = results.toString(); - File resultsFile = File.createTempFile("results", ".csv"); + File resultsFile = new File(getReportDir(), committerName + ".csv"); FileUtils.write(resultsFile, text, StandardCharsets.UTF_8); LOG.info("Results are in {}\n{}", resultsFile, text); } diff --git a/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/impl/StubContextAccessor.java b/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/impl/StubContextAccessor.java new file mode 100644 index 0000000000000..41180667e1000 --- /dev/null +++ b/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/impl/StubContextAccessor.java @@ -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 + * + * 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.fs.s3a.impl; + +import java.io.File; +import java.io.IOException; + +import org.apache.hadoop.fs.Path; +import org.apache.hadoop.fs.s3a.MockS3AFileSystem; +import org.apache.hadoop.fs.s3a.api.RequestFactory; +import org.apache.hadoop.fs.s3a.audit.AuditTestSupport; +import org.apache.hadoop.fs.store.audit.AuditSpan; + +/** + * A Stub context acccessor for test. + */ +public final class StubContextAccessor + implements ContextAccessors { + + private final String bucket; + + /** + * Construct. + * @param bucket bucket to use when qualifying keys.]= + */ + public StubContextAccessor(String bucket) { + this.bucket = bucket; + } + + @Override + public Path keyToPath(final String key) { + return new Path("s3a://" + bucket + "/" + key); + } + + @Override + public String pathToKey(final Path path) { + return null; + } + + @Override + public File createTempFile(final String prefix, final long size) + throws IOException { + throw new UnsupportedOperationException("unsppported"); + } + + @Override + public String getBucketLocation() throws IOException { + return null; + } + + @Override + public Path makeQualified(final Path path) { + return path; + } + + @Override + public AuditSpan getActiveAuditSpan() { + return AuditTestSupport.NOOP_SPAN; + } + + @Override + public RequestFactory getRequestFactory() { + return MockS3AFileSystem.REQUEST_FACTORY; + } + +} diff --git a/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/impl/TestCreateFileBuilder.java b/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/impl/TestCreateFileBuilder.java new file mode 100644 index 0000000000000..65d7aa6192dd8 --- /dev/null +++ b/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/impl/TestCreateFileBuilder.java @@ -0,0 +1,173 @@ +/* + * 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.fs.s3a.impl; + +import java.io.IOException; +import java.io.OutputStream; +import java.util.Map; + +import org.assertj.core.api.Assertions; +import org.junit.Test; + +import org.apache.hadoop.conf.Configuration; +import org.apache.hadoop.fs.CreateFlag; +import org.apache.hadoop.fs.FSDataOutputStream; +import org.apache.hadoop.fs.FSDataOutputStreamBuilder; +import org.apache.hadoop.fs.FileSystem; +import org.apache.hadoop.fs.Path; +import org.apache.hadoop.test.HadoopTestBase; +import org.apache.hadoop.util.Progressable; + +import static org.apache.hadoop.fs.s3a.Constants.FS_S3A_CREATE_HEADER; +import static org.apache.hadoop.fs.s3a.Constants.FS_S3A_CREATE_PERFORMANCE; +import static org.apache.hadoop.test.LambdaTestUtils.intercept; + +/** + * Unit test of {@link CreateFileBuilder}. + */ +public class TestCreateFileBuilder extends HadoopTestBase { + + private static final BuilderCallbacks CALLBACKS = new BuilderCallbacks(); + + private CreateFileBuilder mkBuilder() throws IOException { + return new CreateFileBuilder( + FileSystem.getLocal(new Configuration()), + new Path("/"), + CALLBACKS); + } + + private BuilderOutputStream unwrap(FSDataOutputStream out) { + OutputStream s = out.getWrappedStream(); + Assertions.assertThat(s) + .isInstanceOf(BuilderOutputStream.class); + return (BuilderOutputStream) s; + } + + private BuilderOutputStream build(FSDataOutputStreamBuilder builder) + throws IOException { + return unwrap(builder.build()); + } + + @Test + public void testSimpleBuild() throws Throwable { + Assertions.assertThat(build(mkBuilder().create())) + .matches(p -> !p.isOverwrite()) + .matches(p -> !p.isPerformance()); + } + + @Test + public void testAppendForbidden() throws Throwable { + intercept(UnsupportedOperationException.class, () -> + build(mkBuilder().append())); + } + + @Test + public void testPerformanceSupport() throws Throwable { + CreateFileBuilder builder = mkBuilder().create(); + builder.must(FS_S3A_CREATE_PERFORMANCE, true); + Assertions.assertThat(build(builder)) + .matches(p -> p.isPerformance()); + } + + @Test + public void testHeaderOptions() throws Throwable { + final CreateFileBuilder builder = mkBuilder().create() + .must(FS_S3A_CREATE_HEADER + ".retention", "permanent") + .opt(FS_S3A_CREATE_HEADER + ".owner", "engineering"); + final Map headers = build(builder).getHeaders(); + Assertions.assertThat(headers) + .containsEntry("retention", "permanent") + .containsEntry("owner", "engineering"); + } + + @Test + public void testIncompleteHeader() throws Throwable { + final CreateFileBuilder builder = mkBuilder().create() + .must(FS_S3A_CREATE_HEADER, "permanent"); + intercept(IllegalArgumentException.class, () -> + build(builder)); + } + + private static final class BuilderCallbacks implements + CreateFileBuilder.CreateFileBuilderCallbacks { + + @Override + public FSDataOutputStream createFileFromBuilder(final Path path, + final Progressable progress, + final CreateFileBuilder.CreateFileOptions options) throws IOException { + return new FSDataOutputStream( + new BuilderOutputStream( + progress, + options), + null); + } + } + + /** + * Stream which will be wrapped and which returns the flags used + * creating the object. + */ + private static final class BuilderOutputStream extends OutputStream { + + private final Progressable progress; + + + private final CreateFileBuilder.CreateFileOptions options; + + private BuilderOutputStream(final Progressable progress, + final CreateFileBuilder.CreateFileOptions options) { + this.progress = progress; + this.options = options; + } + + private boolean isOverwrite() { + return options.getFlags().contains(CreateFlag.OVERWRITE); + } + + private Progressable getProgress() { + return progress; + } + + private boolean isPerformance() { + return options.isPerformance(); + } + + private CreateFileBuilder.CreateFileOptions getOptions() { + return options; + } + + private Map getHeaders() { + return options.getHeaders(); + } + + @Override + public void write(final int b) throws IOException { + + } + + @Override + public String toString() { + return "BuilderOutputStream{" + + "progress=" + progress + + ", options=" + options + + "} " + super.toString(); + } + } + +} diff --git a/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/impl/TestRequestFactory.java b/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/impl/TestRequestFactory.java index 9bc3aef83aacb..5c243bb820f02 100644 --- a/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/impl/TestRequestFactory.java +++ b/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/impl/TestRequestFactory.java @@ -90,7 +90,7 @@ public void testRequestFactoryWithCannedACL() throws Throwable { ObjectMetadata md = factory.newObjectMetadata(128); Assertions.assertThat( factory.newPutObjectRequest(path, md, - new ByteArrayInputStream(new byte[0])) + null, new ByteArrayInputStream(new byte[0])) .getCannedAcl()) .describedAs("ACL of PUT") .isEqualTo(acl); @@ -98,7 +98,8 @@ public void testRequestFactoryWithCannedACL() throws Throwable { .getCannedAccessControlList()) .describedAs("ACL of COPY") .isEqualTo(acl); - Assertions.assertThat(factory.newMultipartUploadRequest(path) + Assertions.assertThat(factory.newMultipartUploadRequest(path, + null) .getCannedACL()) .describedAs("ACL of MPU") .isEqualTo(acl); @@ -172,12 +173,12 @@ private void createFactoryObjects(RequestFactory factory) { a(factory.newListObjectsV1Request(path, "/", 1)); a(factory.newListNextBatchOfObjectsRequest(new ObjectListing())); a(factory.newListObjectsV2Request(path, "/", 1)); - a(factory.newMultipartUploadRequest(path)); + a(factory.newMultipartUploadRequest(path, null)); File srcfile = new File("/tmp/a"); a(factory.newPutObjectRequest(path, - factory.newObjectMetadata(-1), srcfile)); + factory.newObjectMetadata(-1), null, srcfile)); ByteArrayInputStream stream = new ByteArrayInputStream(new byte[0]); - a(factory.newPutObjectRequest(path, md, stream)); + a(factory.newPutObjectRequest(path, md, null, stream)); a(factory.newSelectRequest(path)); } diff --git a/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/performance/AbstractS3ACostTest.java b/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/performance/AbstractS3ACostTest.java index 3511020aa6cef..8bbf52b578e1a 100644 --- a/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/performance/AbstractS3ACostTest.java +++ b/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/performance/AbstractS3ACostTest.java @@ -132,16 +132,7 @@ public void setup() throws Exception { .isEqualTo(isKeepingMarkers() ? DirectoryPolicy.MarkerPolicy.Keep : DirectoryPolicy.MarkerPolicy.Delete); - // All counter statistics of the filesystem are added as metrics. - // Durations too, as they have counters of success and failure. - OperationCostValidator.Builder builder = OperationCostValidator.builder( - getFileSystem()); - EnumSet.allOf(Statistic.class).stream() - .filter(s -> - s.getType() == StatisticTypeEnum.TYPE_COUNTER - || s.getType() == StatisticTypeEnum.TYPE_DURATION) - .forEach(s -> builder.withMetric(s)); - costValidator = builder.build(); + setupCostValidator(); // determine bulk delete settings final Configuration fsConf = getFileSystem().getConf(); @@ -154,6 +145,19 @@ public void setup() throws Exception { setSpanSource(fs); } + protected void setupCostValidator() { + // All counter statistics of the filesystem are added as metrics. + // Durations too, as they have counters of success and failure. + OperationCostValidator.Builder builder = OperationCostValidator.builder( + getFileSystem()); + EnumSet.allOf(Statistic.class).stream() + .filter(s -> + s.getType() == StatisticTypeEnum.TYPE_COUNTER + || s.getType() == StatisticTypeEnum.TYPE_DURATION) + .forEach(s -> builder.withMetric(s)); + costValidator = builder.build(); + } + public boolean isDeleting() { return isDeleting; } diff --git a/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/performance/ITestCreateFileCost.java b/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/performance/ITestCreateFileCost.java new file mode 100644 index 0000000000000..39530d97bf794 --- /dev/null +++ b/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/performance/ITestCreateFileCost.java @@ -0,0 +1,248 @@ +/* + * 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.fs.s3a.performance; + +import java.io.IOException; + +import org.assertj.core.api.Assertions; +import org.junit.Test; + +import org.apache.hadoop.fs.FSDataOutputStream; +import org.apache.hadoop.fs.FSDataOutputStreamBuilder; +import org.apache.hadoop.fs.FileAlreadyExistsException; +import org.apache.hadoop.fs.Path; +import org.apache.hadoop.fs.contract.ContractTestUtils; +import org.apache.hadoop.fs.s3a.S3AFileSystem; + + +import static java.util.Objects.requireNonNull; +import static org.apache.hadoop.fs.contract.ContractTestUtils.toChar; +import static org.apache.hadoop.fs.s3a.Constants.FS_S3A_CREATE_HEADER; +import static org.apache.hadoop.fs.s3a.Constants.FS_S3A_CREATE_PERFORMANCE; +import static org.apache.hadoop.fs.s3a.Constants.XA_HEADER_PREFIX; +import static org.apache.hadoop.fs.s3a.Statistic.OBJECT_BULK_DELETE_REQUEST; +import static org.apache.hadoop.fs.s3a.Statistic.OBJECT_DELETE_REQUEST; +import static org.apache.hadoop.fs.s3a.performance.OperationCost.CREATE_FILE_NO_OVERWRITE; +import static org.apache.hadoop.fs.s3a.performance.OperationCost.CREATE_FILE_OVERWRITE; +import static org.apache.hadoop.fs.s3a.performance.OperationCost.FILE_STATUS_DIR_PROBE; +import static org.apache.hadoop.fs.s3a.performance.OperationCost.FILE_STATUS_FILE_PROBE; +import static org.apache.hadoop.fs.s3a.performance.OperationCost.GET_FILE_STATUS_ON_DIR_MARKER; +import static org.apache.hadoop.fs.s3a.performance.OperationCost.GET_FILE_STATUS_ON_FILE; +import static org.apache.hadoop.fs.s3a.performance.OperationCost.HEAD_OPERATION; +import static org.apache.hadoop.fs.s3a.performance.OperationCost.NO_HEAD_OR_LIST; + +/** + * Assert cost of createFile operations, especially + * with the FS_S3A_CREATE_PERFORMANCE option. + */ +@SuppressWarnings("resource") +public class ITestCreateFileCost extends AbstractS3ACostTest { + + /** + * Create with markers kept, always. + */ + public ITestCreateFileCost() { + super(false); + } + + @Test + public void testCreateNoOverwrite() throws Throwable { + describe("Test file creation without overwrite"); + Path testFile = methodPath(); + // when overwrite is false, the path is checked for existence. + create(testFile, false, + CREATE_FILE_NO_OVERWRITE); + } + + @Test + public void testCreateOverwrite() throws Throwable { + describe("Test file creation with overwrite"); + Path testFile = methodPath(); + // when overwrite is true: only the directory checks take place. + create(testFile, true, CREATE_FILE_OVERWRITE); + } + + @Test + public void testCreateNoOverwriteFileExists() throws Throwable { + describe("Test cost of create file failing with existing file"); + Path testFile = file(methodPath()); + + // now there is a file there, an attempt with overwrite == false will + // fail on the first HEAD. + interceptOperation(FileAlreadyExistsException.class, "", + FILE_STATUS_FILE_PROBE, + () -> file(testFile, false)); + } + + @Test + public void testCreateFileOverDir() throws Throwable { + describe("Test cost of create file failing with existing dir"); + Path testFile = dir(methodPath()); + + // now there is a file there, an attempt with overwrite == false will + // fail on the first HEAD. + interceptOperation(FileAlreadyExistsException.class, "", + GET_FILE_STATUS_ON_DIR_MARKER, + () -> file(testFile, false)); + } + + /** + * Use the builder API. + * on s3a this skips parent checks, always. + */ + @Test + public void testCreateBuilderSequence() throws Throwable { + describe("Test builder file creation cost"); + Path testFile = methodPath(); + dir(testFile.getParent()); + + // s3a fs skips the recursive checks to avoid race + // conditions with other processes/threads deleting + // files and so briefly the path not being present + // only make sure the dest path isn't a directory. + buildFile(testFile, true, false, + FILE_STATUS_DIR_PROBE); + + // now there is a file there, an attempt with overwrite == false will + // fail on the first HEAD. + interceptOperation(FileAlreadyExistsException.class, "", + GET_FILE_STATUS_ON_FILE, + () -> buildFile(testFile, false, true, + GET_FILE_STATUS_ON_FILE)); + } + + @Test + public void testCreateFilePerformanceFlag() throws Throwable { + describe("createFile with performance flag skips safety checks"); + S3AFileSystem fs = getFileSystem(); + + Path path = methodPath(); + FSDataOutputStreamBuilder builder = fs.createFile(path) + .overwrite(false) + .recursive(); + + // this has a broken return type; something to do with the return value of + // the createFile() call. only fixable via risky changes to the FileSystem class + builder.must(FS_S3A_CREATE_PERFORMANCE, true); + + verifyMetrics(() -> build(builder), + always(NO_HEAD_OR_LIST), + with(OBJECT_BULK_DELETE_REQUEST, 0), + with(OBJECT_DELETE_REQUEST, 0)); + } + + @Test + public void testCreateFileRecursive() throws Throwable { + describe("createFile without performance flag performs overwrite safety checks"); + S3AFileSystem fs = getFileSystem(); + + final Path path = methodPath(); + FSDataOutputStreamBuilder builder = fs.createFile(path) + .recursive() + .overwrite(false); + + // include a custom header to probe for after + final String custom = "custom"; + builder.must(FS_S3A_CREATE_HEADER + ".h1", custom); + + verifyMetrics(() -> build(builder), + always(CREATE_FILE_NO_OVERWRITE)); + + // the header is there and the probe should be a single HEAD call. + String header = verifyMetrics(() -> + toChar(requireNonNull( + fs.getXAttr(path, XA_HEADER_PREFIX + "h1"), + "no header")), + always(HEAD_OPERATION)); + Assertions.assertThat(header) + .isEqualTo(custom); + } + + @Test + public void testCreateFileNonRecursive() throws Throwable { + describe("nonrecursive createFile does not check parents"); + S3AFileSystem fs = getFileSystem(); + + verifyMetrics(() -> + build(fs.createFile(methodPath()).overwrite(true)), + always(CREATE_FILE_OVERWRITE)); + } + + + @Test + public void testCreateNonRecursive() throws Throwable { + describe("nonrecursive createFile does not check parents"); + S3AFileSystem fs = getFileSystem(); + + verifyMetrics(() -> { + fs.createNonRecursive(methodPath(), + true, 1000, (short)1, 0L, null) + .close(); + return ""; + }, + always(CREATE_FILE_OVERWRITE)); + } + + private FSDataOutputStream build(final FSDataOutputStreamBuilder builder) + throws IOException { + FSDataOutputStream out = builder.build(); + out.close(); + return out; + } + + /** + * Shows how the performance option allows the FS to become ill-formed. + */ + @Test + public void testPerformanceFlagPermitsInvalidStores() throws Throwable { + describe("createFile with performance flag over a directory"); + S3AFileSystem fs = getFileSystem(); + + Path path = methodPath(); + Path child = new Path(path, "child"); + ContractTestUtils.touch(fs, child); + try { + FSDataOutputStreamBuilder builder = fs.createFile(path) + .overwrite(false); + // this has a broken return type; a java typesystem quirk. + builder.must(FS_S3A_CREATE_PERFORMANCE, true); + + verifyMetrics(() -> build(builder), + always(NO_HEAD_OR_LIST), + with(OBJECT_BULK_DELETE_REQUEST, 0), + with(OBJECT_DELETE_REQUEST, 0)); + // the file is there + assertIsFile(path); + // the child is there + assertIsFile(child); + + // delete the path + fs.delete(path, true); + // the child is still there + assertIsFile(child); + // and the directory exists again + assertIsDirectory(path); + } finally { + // always delete the child, so if the test suite fails, the + // store is at least well-formed. + fs.delete(child, true); + } + } + +} diff --git a/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/performance/ITestS3ADeleteCost.java b/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/performance/ITestS3ADeleteCost.java index 01cadc7c86e3e..be4de7942f7b9 100644 --- a/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/performance/ITestS3ADeleteCost.java +++ b/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/performance/ITestS3ADeleteCost.java @@ -266,9 +266,10 @@ public void testDirMarkersFileCreation() throws Throwable { final int directories = directoriesInPath(srcDir); verifyMetrics(() -> { - file(new Path(srcDir, "source.txt")); + final Path srcPath = new Path(srcDir, "source.txt"); + file(srcPath); LOG.info("Metrics: {}\n{}", getMetricSummary(), getFileSystem()); - return "after touch(fs, srcFilePath) " + getMetricSummary(); + return "after touch(fs, " + srcPath + ")" + getMetricSummary(); }, with(DIRECTORIES_CREATED, 0), with(DIRECTORIES_DELETED, 0), @@ -276,10 +277,10 @@ public void testDirMarkersFileCreation() throws Throwable { withWhenKeeping(getDeleteMarkerStatistic(), 0), withWhenKeeping(FAKE_DIRECTORIES_DELETED, 0), // delete all possible fake dirs above the file - withWhenDeleting(getDeleteMarkerStatistic(), - isBulkDelete() ? 1: directories), withWhenDeleting(FAKE_DIRECTORIES_DELETED, - directories)); + directories), + withWhenDeleting(getDeleteMarkerStatistic(), + isBulkDelete() ? 1: directories)); } } diff --git a/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/performance/OperationCost.java b/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/performance/OperationCost.java index 03bc10f86cd25..08a19600d5830 100644 --- a/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/performance/OperationCost.java +++ b/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/performance/OperationCost.java @@ -57,7 +57,8 @@ public final class OperationCost { public static final int DELETE_MARKER_REQUEST = DELETE_OBJECT_REQUEST; /** - * No IO takes place. + * No Head or List IO takes place; other operations + * may still take place. */ public static final OperationCost NO_IO = new OperationCost(0, 0); @@ -87,7 +88,7 @@ public final class OperationCost { /** * Cost of getFileStatus on root directory. */ - public static final OperationCost ROOT_FILE_STATUS_PROBE = NO_IO; + public static final OperationCost ROOT_FILE_STATUS_PROBE = NO_HEAD_OR_LIST; /** * Cost of {@link org.apache.hadoop.fs.s3a.impl.StatusProbeEnum#ALL}. diff --git a/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/performance/OperationCostValidator.java b/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/performance/OperationCostValidator.java index 72e51ee3b9958..3a8a1f6ad7b6f 100644 --- a/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/performance/OperationCostValidator.java +++ b/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/performance/OperationCostValidator.java @@ -20,6 +20,7 @@ import java.util.ArrayList; import java.util.Arrays; +import java.util.EnumSet; import java.util.List; import java.util.Map; import java.util.TreeMap; @@ -34,6 +35,7 @@ import org.apache.hadoop.fs.s3a.S3AInstrumentation; import org.apache.hadoop.fs.s3a.S3ATestUtils; import org.apache.hadoop.fs.s3a.Statistic; +import org.apache.hadoop.fs.s3a.statistics.StatisticTypeEnum; import org.apache.hadoop.fs.statistics.impl.IOStatisticsStore; import org.apache.hadoop.metrics2.lib.MutableCounter; import org.apache.hadoop.metrics2.lib.MutableMetric; @@ -274,7 +276,7 @@ public Builder withMetric(Statistic statistic) { /** * Add a varargs list of metrics. - * @param stat statistics to monitor. + * @param stats statistics to monitor. * @return this. */ public Builder withMetrics(Statistic...stats) { @@ -282,6 +284,20 @@ public Builder withMetrics(Statistic...stats) { return this; } + /** + * Add all counters and duration types to the + * metrics which can be asserted over. + * @return this. + */ + public Builder withAllCounters() { + EnumSet.allOf(Statistic.class).stream() + .filter(s -> + s.getType() == StatisticTypeEnum.TYPE_COUNTER + || s.getType() == StatisticTypeEnum.TYPE_DURATION) + .forEach(metrics::add); + return this; + } + /** * Instantiate. * @return the validator. diff --git a/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/scale/AbstractSTestS3AHugeFiles.java b/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/scale/AbstractSTestS3AHugeFiles.java index 15700ce953589..f8d47011de3f0 100644 --- a/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/scale/AbstractSTestS3AHugeFiles.java +++ b/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/scale/AbstractSTestS3AHugeFiles.java @@ -19,8 +19,13 @@ package org.apache.hadoop.fs.s3a.scale; import java.io.IOException; +import java.nio.ByteBuffer; +import java.util.ArrayList; +import java.util.List; +import java.util.concurrent.CompletableFuture; import java.util.concurrent.atomic.AtomicInteger; import java.util.concurrent.atomic.AtomicLong; +import java.util.function.IntFunction; import com.amazonaws.event.ProgressEvent; import com.amazonaws.event.ProgressEventType; @@ -35,7 +40,9 @@ import org.apache.hadoop.conf.Configuration; import org.apache.hadoop.fs.FSDataInputStream; import org.apache.hadoop.fs.FSDataOutputStream; +import org.apache.hadoop.fs.FileRange; import org.apache.hadoop.fs.FileStatus; +import org.apache.hadoop.fs.FileSystem; import org.apache.hadoop.fs.Path; import org.apache.hadoop.fs.contract.ContractTestUtils; import org.apache.hadoop.fs.s3a.Constants; @@ -47,6 +54,7 @@ import org.apache.hadoop.util.Progressable; import static org.apache.hadoop.fs.contract.ContractTestUtils.*; +import static org.apache.hadoop.fs.contract.ContractTestUtils.validateVectoredReadResult; import static org.apache.hadoop.fs.s3a.Constants.*; import static org.apache.hadoop.fs.s3a.S3ATestUtils.*; import static org.apache.hadoop.fs.s3a.Statistic.STREAM_WRITE_BLOCK_UPLOADS_BYTES_PENDING; @@ -446,6 +454,30 @@ public void test_040_PositionedReadHugeFile() throws Throwable { toHuman(timer.nanosPerOperation(ops))); } + @Test + public void test_045_vectoredIOHugeFile() throws Throwable { + assumeHugeFileExists(); + List rangeList = new ArrayList<>(); + rangeList.add(FileRange.createFileRange(5856368, 116770)); + rangeList.add(FileRange.createFileRange(3520861, 116770)); + rangeList.add(FileRange.createFileRange(8191913, 116770)); + rangeList.add(FileRange.createFileRange(1520861, 116770)); + rangeList.add(FileRange.createFileRange(2520861, 116770)); + rangeList.add(FileRange.createFileRange(9191913, 116770)); + rangeList.add(FileRange.createFileRange(2820861, 156770)); + IntFunction allocate = ByteBuffer::allocate; + FileSystem fs = getFileSystem(); + CompletableFuture builder = + fs.openFile(hugefile).build(); + try (FSDataInputStream in = builder.get()) { + in.readVectored(rangeList, allocate); + byte[] readFullRes = new byte[(int)filesize]; + in.readFully(0, readFullRes); + // Comparing vectored read results with read fully. + validateVectoredReadResult(rangeList, readFullRes); + } + } + /** * Read in the entire file using read() calls. * @throws Throwable failure diff --git a/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/scale/ITestS3ADirectoryPerformance.java b/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/scale/ITestS3ADirectoryPerformance.java index 946e59e9e3c01..91ea0c8e62fb3 100644 --- a/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/scale/ITestS3ADirectoryPerformance.java +++ b/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/scale/ITestS3ADirectoryPerformance.java @@ -31,6 +31,7 @@ import org.apache.hadoop.fs.s3a.Statistic; import org.apache.hadoop.fs.s3a.WriteOperationHelper; import org.apache.hadoop.fs.s3a.api.RequestFactory; +import org.apache.hadoop.fs.s3a.impl.PutObjectOptions; import org.apache.hadoop.fs.statistics.IOStatistics; import org.apache.hadoop.fs.store.audit.AuditSpan; import org.apache.hadoop.util.functional.RemoteIterators; @@ -257,9 +258,9 @@ public void testMultiPagesListingPerformanceAndCorrectness() ObjectMetadata om = fs.newObjectMetadata(0L); PutObjectRequest put = requestFactory .newPutObjectRequest(fs.pathToKey(file), om, - new FailingInputStream()); + null, new FailingInputStream()); futures.add(submit(executorService, () -> - writeOperationHelper.putObject(put))); + writeOperationHelper.putObject(put, PutObjectOptions.keepingDirs()))); } LOG.info("Waiting for PUTs to complete"); waitForCompletion(futures); diff --git a/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/scale/ITestS3AHugeFilesStorageClass.java b/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/scale/ITestS3AHugeFilesStorageClass.java new file mode 100644 index 0000000000000..99407467df56d --- /dev/null +++ b/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/scale/ITestS3AHugeFilesStorageClass.java @@ -0,0 +1,135 @@ +/* + * 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.fs.s3a.scale; + +import java.io.IOException; + +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + +import org.apache.hadoop.conf.Configuration; +import org.apache.hadoop.fs.FileStatus; +import org.apache.hadoop.fs.Path; +import org.apache.hadoop.fs.contract.ContractTestUtils; +import org.apache.hadoop.fs.s3a.Constants; +import org.apache.hadoop.fs.s3a.S3AFileSystem; + +import static org.apache.hadoop.fs.contract.ContractTestUtils.bandwidth; +import static org.apache.hadoop.fs.contract.ContractTestUtils.toHuman; +import static org.apache.hadoop.fs.s3a.Constants.STORAGE_CLASS; +import static org.apache.hadoop.fs.s3a.Constants.STORAGE_CLASS_REDUCED_REDUNDANCY; +import static org.apache.hadoop.fs.s3a.S3ATestUtils.disableFilesystemCaching; +import static org.apache.hadoop.fs.s3a.S3ATestUtils.removeBaseAndBucketOverrides; +import static org.apache.hadoop.fs.s3a.S3ATestUtils.skipIfStorageClassTestsDisabled; + +/** + * Class to verify that {@link Constants#STORAGE_CLASS} is set correctly + * for creating and renaming huge files with multipart upload requests. + */ +public class ITestS3AHugeFilesStorageClass extends AbstractSTestS3AHugeFiles { + + private static final Logger LOG = LoggerFactory.getLogger(ITestS3AHugeFilesStorageClass.class); + + @Override + public void setup() throws Exception { + super.setup(); + skipIfStorageClassTestsDisabled(getConfiguration()); + } + + @Override + protected Configuration createScaleConfiguration() { + Configuration conf = super.createScaleConfiguration(); + disableFilesystemCaching(conf); + removeBaseAndBucketOverrides(conf, STORAGE_CLASS); + + conf.set(STORAGE_CLASS, STORAGE_CLASS_REDUCED_REDUNDANCY); + return conf; + } + + @Override + protected String getBlockOutputBufferName() { + return Constants.FAST_UPLOAD_BUFFER_ARRAY; + } + + @Override + public void test_010_CreateHugeFile() throws IOException { + super.test_010_CreateHugeFile(); + assertStorageClass(getPathOfFileToCreate()); + } + + @Override + public void test_030_postCreationAssertions() throws Throwable { + super.test_030_postCreationAssertions(); + assertStorageClass(getPathOfFileToCreate()); + } + + @Override + public void test_040_PositionedReadHugeFile() throws Throwable { + skipQuietly("PositionedReadHugeFile"); + } + + @Override + public void test_050_readHugeFile() throws Throwable { + skipQuietly("readHugeFile"); + } + + @Override + public void test_090_verifyRenameSourceEncryption() throws IOException { + skipQuietly("verifyRenameSourceEncryption"); + } + + @Override + public void test_100_renameHugeFile() throws Throwable { + Path hugefile = getHugefile(); + Path hugefileRenamed = getHugefileRenamed(); + assumeHugeFileExists(); + describe("renaming %s to %s", hugefile, hugefileRenamed); + S3AFileSystem fs = getFileSystem(); + FileStatus status = fs.getFileStatus(hugefile); + long size = status.getLen(); + fs.delete(hugefileRenamed, false); + ContractTestUtils.NanoTimer timer = new ContractTestUtils.NanoTimer(); + fs.rename(hugefile, hugefileRenamed); + long mb = Math.max(size / _1MB, 1); + timer.end("time to rename file of %d MB", mb); + LOG.info("Time per MB to rename = {} nS", toHuman(timer.nanosPerOperation(mb))); + bandwidth(timer, size); + FileStatus destFileStatus = fs.getFileStatus(hugefileRenamed); + assertEquals(size, destFileStatus.getLen()); + assertStorageClass(hugefileRenamed); + } + + @Override + public void test_110_verifyRenameDestEncryption() throws IOException { + skipQuietly("verifyRenameDestEncryption"); + } + + private void skipQuietly(String text) { + describe("Skipping: %s", text); + } + + protected void assertStorageClass(Path hugeFile) throws IOException { + S3AFileSystem fs = getFileSystem(); + String actual = fs.getObjectMetadata(hugeFile).getStorageClass(); + + assertTrue( + "Storage class of object is " + actual + ", expected " + STORAGE_CLASS_REDUCED_REDUNDANCY, + STORAGE_CLASS_REDUCED_REDUNDANCY.equalsIgnoreCase(actual)); + } +} diff --git a/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/scale/ITestS3AMultipartUploadSizeLimits.java b/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/scale/ITestS3AMultipartUploadSizeLimits.java index 231cfd884e0c8..b83d12b4c1a66 100644 --- a/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/scale/ITestS3AMultipartUploadSizeLimits.java +++ b/hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/scale/ITestS3AMultipartUploadSizeLimits.java @@ -35,7 +35,7 @@ import org.apache.hadoop.fs.s3a.S3AInstrumentation; import org.apache.hadoop.fs.s3a.Statistic; import org.apache.hadoop.fs.s3a.auth.ProgressCounter; -import org.apache.hadoop.fs.s3a.commit.CommitOperations; +import org.apache.hadoop.fs.s3a.commit.impl.CommitOperations; import static org.apache.hadoop.fs.StreamCapabilities.ABORTABLE_STREAM; import static org.apache.hadoop.fs.contract.ContractTestUtils.createFile; diff --git a/hadoop-tools/hadoop-aws/src/test/resources/log4j.properties b/hadoop-tools/hadoop-aws/src/test/resources/log4j.properties index fc287e9845c76..c831999008fec 100644 --- a/hadoop-tools/hadoop-aws/src/test/resources/log4j.properties +++ b/hadoop-tools/hadoop-aws/src/test/resources/log4j.properties @@ -52,7 +52,7 @@ log4j.logger.org.apache.hadoop.ipc.Server=WARN # for debugging low level S3a operations, uncomment these lines # Log all S3A classes -#log4j.logger.org.apache.hadoop.fs.s3a=DEBUG +log4j.logger.org.apache.hadoop.fs.s3a=DEBUG #log4j.logger.org.apache.hadoop.fs.s3a.S3AUtils=INFO #log4j.logger.org.apache.hadoop.fs.s3a.Listing=INFO diff --git a/hadoop-tools/hadoop-azure-datalake/pom.xml b/hadoop-tools/hadoop-azure-datalake/pom.xml index 446e2957b60ed..cec050d2c1be7 100644 --- a/hadoop-tools/hadoop-azure-datalake/pom.xml +++ b/hadoop-tools/hadoop-azure-datalake/pom.xml @@ -29,7 +29,6 @@ jar - ${okhttp.version} 0.9.1 UTF-8 true @@ -118,12 +117,6 @@ hadoop-common provided - - com.squareup.okhttp - okhttp - ${okhttp.version} - test - junit junit @@ -141,12 +134,6 @@ test test-jar - - com.squareup.okhttp - mockwebserver - ${okhttp.version} - test - org.apache.hadoop diff --git a/hadoop-tools/hadoop-azure/.gitignore b/hadoop-tools/hadoop-azure/.gitignore index 0e17efaa1eb24..8db1334021c38 100644 --- a/hadoop-tools/hadoop-azure/.gitignore +++ b/hadoop-tools/hadoop-azure/.gitignore @@ -1,5 +1,6 @@ .checkstyle bin/ src/test/resources/combinationConfigFiles +src/test/resources/accountSettings src/test/resources/abfs-combination-test-configs.xml dev-support/testlogs diff --git a/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/AbfsCountersImpl.java b/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/AbfsCountersImpl.java index c478256e706bc..67ee8e90efb3d 100644 --- a/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/AbfsCountersImpl.java +++ b/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/AbfsCountersImpl.java @@ -87,7 +87,11 @@ public class AbfsCountersImpl implements AbfsCounters { BYTES_RECEIVED, READ_THROTTLES, WRITE_THROTTLES, - SERVER_UNAVAILABLE + SERVER_UNAVAILABLE, + RENAME_RECOVERY, + METADATA_INCOMPLETE_RENAME_FAILURES, + RENAME_PATH_ATTEMPTS + }; private static final AbfsStatistic[] DURATION_TRACKER_LIST = { diff --git a/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/AbfsStatistic.java b/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/AbfsStatistic.java index bb65b0c902120..3a77e82ffb4fb 100644 --- a/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/AbfsStatistic.java +++ b/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/AbfsStatistic.java @@ -100,7 +100,16 @@ public enum AbfsStatistic { AbfsHttpConstants.HTTP_METHOD_PATCH), HTTP_POST_REQUEST(StoreStatisticNames.ACTION_HTTP_POST_REQUEST, "Time taken to complete a POST request", - AbfsHttpConstants.HTTP_METHOD_POST); + AbfsHttpConstants.HTTP_METHOD_POST), + + // Rename recovery + RENAME_RECOVERY("rename_recovery", + "Number of times Rename recoveries happened"), + METADATA_INCOMPLETE_RENAME_FAILURES("metadata_incomplete_rename_failures", + "Number of times rename operation failed due to metadata being " + + "incomplete"), + RENAME_PATH_ATTEMPTS("rename_path_attempts", + "Number of times we attempt to rename a path internally"); private String statName; private String statDescription; diff --git a/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/AzureBlobFileSystem.java b/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/AzureBlobFileSystem.java index 46141e7c4a838..d0bdd9818db24 100644 --- a/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/AzureBlobFileSystem.java +++ b/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/AzureBlobFileSystem.java @@ -1576,7 +1576,7 @@ public boolean failed() { } @VisibleForTesting - AzureBlobFileSystemStore getAbfsStore() { + public AzureBlobFileSystemStore getAbfsStore() { return abfsStore; } diff --git a/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/AzureBlobFileSystemStore.java b/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/AzureBlobFileSystemStore.java index b9cbeb756c19d..fa9e6eef3526d 100644 --- a/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/AzureBlobFileSystemStore.java +++ b/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/AzureBlobFileSystemStore.java @@ -65,7 +65,6 @@ import org.slf4j.Logger; import org.slf4j.LoggerFactory; -import org.apache.commons.lang3.tuple.Pair; import org.apache.hadoop.classification.InterfaceAudience; import org.apache.hadoop.classification.InterfaceStability; import org.apache.hadoop.conf.Configuration; @@ -100,6 +99,7 @@ import org.apache.hadoop.fs.azurebfs.services.AbfsClient; import org.apache.hadoop.fs.azurebfs.services.AbfsClientContext; import org.apache.hadoop.fs.azurebfs.services.AbfsClientContextBuilder; +import org.apache.hadoop.fs.azurebfs.services.AbfsClientRenameResult; import org.apache.hadoop.fs.azurebfs.services.AbfsCounters; import org.apache.hadoop.fs.azurebfs.services.AbfsHttpOperation; import org.apache.hadoop.fs.azurebfs.services.AbfsInputStream; @@ -135,6 +135,8 @@ import org.apache.hadoop.util.concurrent.HadoopExecutors; import org.apache.http.client.utils.URIBuilder; +import static org.apache.hadoop.fs.azurebfs.AbfsStatistic.METADATA_INCOMPLETE_RENAME_FAILURES; +import static org.apache.hadoop.fs.azurebfs.AbfsStatistic.RENAME_RECOVERY; import static org.apache.hadoop.fs.azurebfs.constants.AbfsHttpConstants.CHAR_EQUALS; import static org.apache.hadoop.fs.azurebfs.constants.AbfsHttpConstants.CHAR_FORWARD_SLASH; import static org.apache.hadoop.fs.azurebfs.constants.AbfsHttpConstants.CHAR_HYPHEN; @@ -950,18 +952,19 @@ public boolean rename(final Path source, do { try (AbfsPerfInfo perfInfo = startTracking("rename", "renamePath")) { - final Pair pair = + final AbfsClientRenameResult abfsClientRenameResult = client.renamePath(sourceRelativePath, destinationRelativePath, - continuation, tracingContext, sourceEtag); + continuation, tracingContext, sourceEtag, false); - AbfsRestOperation op = pair.getLeft(); + AbfsRestOperation op = abfsClientRenameResult.getOp(); perfInfo.registerResult(op.getResult()); continuation = op.getResult().getResponseHeader(HttpHeaderConfigurations.X_MS_CONTINUATION); perfInfo.registerSuccess(true); countAggregate++; shouldContinue = continuation != null && !continuation.isEmpty(); // update the recovery flag. - recovered |= pair.getRight(); + recovered |= abfsClientRenameResult.isRenameRecovered(); + populateRenameRecoveryStatistics(abfsClientRenameResult); if (!shouldContinue) { perfInfo.registerAggregates(startAggregate, countAggregate); } @@ -1984,7 +1987,7 @@ public AzureBlobFileSystemStoreBuilder build() { } @VisibleForTesting - AbfsClient getClient() { + public AbfsClient getClient() { return this.client; } @@ -2052,4 +2055,19 @@ public static String extractEtagHeader(AbfsHttpOperation result) { } return etag; } + + /** + * Increment rename recovery based counters in IOStatistics. + * + * @param abfsClientRenameResult Result of an ABFS rename operation. + */ + private void populateRenameRecoveryStatistics( + AbfsClientRenameResult abfsClientRenameResult) { + if (abfsClientRenameResult.isRenameRecovered()) { + abfsCounters.incrementCounter(RENAME_RECOVERY, 1); + } + if (abfsClientRenameResult.isIncompleteMetadataState()) { + abfsCounters.incrementCounter(METADATA_INCOMPLETE_RENAME_FAILURES, 1); + } + } } diff --git a/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/services/AbfsClient.java b/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/services/AbfsClient.java index abf9a73dd177f..c9d6a613327d0 100644 --- a/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/services/AbfsClient.java +++ b/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/services/AbfsClient.java @@ -36,6 +36,8 @@ import java.util.concurrent.TimeUnit; import org.apache.hadoop.classification.VisibleForTesting; +import org.apache.hadoop.fs.store.LogExactlyOnce; +import org.apache.hadoop.util.Preconditions; import org.apache.hadoop.fs.Path; import org.apache.hadoop.fs.azurebfs.AzureBlobFileSystemStore.Permissions; import org.apache.hadoop.fs.azurebfs.extensions.EncryptionContextProvider; @@ -54,7 +56,6 @@ import org.slf4j.Logger; import org.slf4j.LoggerFactory; -import org.apache.commons.lang3.tuple.Pair; import org.apache.hadoop.fs.azurebfs.constants.AbfsHttpConstants; import org.apache.hadoop.fs.azurebfs.constants.HttpHeaderConfigurations; import org.apache.hadoop.fs.azurebfs.constants.HttpQueryParams; @@ -74,6 +75,7 @@ import javax.security.auth.DestroyFailedException; import static org.apache.commons.lang3.StringUtils.isNotEmpty; +import static org.apache.hadoop.fs.azurebfs.AbfsStatistic.RENAME_PATH_ATTEMPTS; import static org.apache.hadoop.fs.azurebfs.AzureBlobFileSystemStore.extractEtagHeader; import static org.apache.hadoop.fs.azurebfs.constants.AbfsHttpConstants.*; import static org.apache.hadoop.fs.azurebfs.constants.FileSystemConfigurations.DEFAULT_DELETE_CONSIDERED_IDEMPOTENT; @@ -81,6 +83,7 @@ import static org.apache.hadoop.fs.azurebfs.constants.FileSystemUriSchemes.HTTPS_SCHEME; import static org.apache.hadoop.fs.azurebfs.constants.HttpHeaderConfigurations.*; import static org.apache.hadoop.fs.azurebfs.constants.HttpQueryParams.*; +import static org.apache.hadoop.fs.azurebfs.contracts.services.AzureServiceErrorCode.RENAME_DESTINATION_PARENT_PATH_NOT_FOUND; /** * AbfsClient. @@ -109,6 +112,12 @@ public class AbfsClient implements Closeable { private final ListeningScheduledExecutorService executorService; + + /** logging the rename failure if metadata is in an incomplete state. */ + private static final LogExactlyOnce ABFS_METADATA_INCOMPLETE_RENAME_FAILURE = + new LogExactlyOnce(LOG); + + private AbfsClient(final URL baseUrl, final SharedKeyCredentials sharedKeyCredentials, final AbfsConfiguration abfsConfiguration, @@ -550,15 +559,19 @@ public AbfsRestOperation breakLease(final String path, * @param continuation continuation. * @param tracingContext trace context * @param sourceEtag etag of source file. may be null or empty - * @return pair of (the rename operation, flag indicating recovery took place) + * @param isMetadataIncompleteState was there a rename failure due to + * incomplete metadata state? + * @return AbfsClientRenameResult result of rename operation indicating the + * AbfsRest operation, rename recovery and incomplete metadata state failure. * @throws AzureBlobFileSystemException failure, excluding any recovery from overload failures. */ - public Pair renamePath( + public AbfsClientRenameResult renamePath( final String source, final String destination, final String continuation, final TracingContext tracingContext, - final String sourceEtag) + final String sourceEtag, + boolean isMetadataIncompleteState) throws IOException { final List requestHeaders = createDefaultHeaders(); @@ -585,13 +598,45 @@ public Pair renamePath( url, requestHeaders); try { + incrementAbfsRenamePath(); op.execute(tracingContext); - return Pair.of(op, false); + // AbfsClientResult contains the AbfsOperation, If recovery happened or + // not, and the incompleteMetaDataState is true or false. + // If we successfully rename a path and isMetadataIncompleteState was + // true, then rename was recovered, else it didn't, this is why + // isMetadataIncompleteState is used for renameRecovery(as the 2nd param). + return new AbfsClientRenameResult(op, isMetadataIncompleteState, isMetadataIncompleteState); } catch (AzureBlobFileSystemException e) { // If we have no HTTP response, throw the original exception. if (!op.hasResult()) { throw e; } + + // ref: HADOOP-18242. Rename failure occurring due to a rare case of + // tracking metadata being in incomplete state. + if (op.getResult().getStorageErrorCode() + .equals(RENAME_DESTINATION_PARENT_PATH_NOT_FOUND.getErrorCode()) + && !isMetadataIncompleteState) { + //Logging + ABFS_METADATA_INCOMPLETE_RENAME_FAILURE + .info("Rename Failure attempting to resolve tracking metadata state and retrying."); + + // Doing a HEAD call resolves the incomplete metadata state and + // then we can retry the rename operation. + AbfsRestOperation sourceStatusOp = getPathStatus(source, false, + tracingContext); + isMetadataIncompleteState = true; + // Extract the sourceEtag, using the status Op, and set it + // for future rename recovery. + AbfsHttpOperation sourceStatusResult = sourceStatusOp.getResult(); + String sourceEtagAfterFailure = extractEtagHeader(sourceStatusResult); + renamePath(source, destination, continuation, tracingContext, + sourceEtagAfterFailure, isMetadataIncompleteState); + } + // if we get out of the condition without a successful rename, then + // it isn't metadata incomplete state issue. + isMetadataIncompleteState = false; + boolean etagCheckSucceeded = renameIdempotencyCheckOp( source, sourceEtag, op, destination, tracingContext); @@ -600,10 +645,14 @@ public Pair renamePath( // throw back the exception throw e; } - return Pair.of(op, true); + return new AbfsClientRenameResult(op, true, isMetadataIncompleteState); } } + private void incrementAbfsRenamePath() { + abfsCounters.incrementCounter(RENAME_PATH_ATTEMPTS, 1); + } + /** * Check if the rename request failure is post a retry and if earlier rename * request might have succeeded at back-end. diff --git a/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/services/AbfsClientRenameResult.java b/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/services/AbfsClientRenameResult.java new file mode 100644 index 0000000000000..86e3473a9fe5d --- /dev/null +++ b/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/services/AbfsClientRenameResult.java @@ -0,0 +1,61 @@ +/** + * 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.fs.azurebfs.services; + +/** + * A class to store the Result of an AbfsClient rename operation, signifying the + * AbfsRestOperation result and the rename recovery. + */ +public class AbfsClientRenameResult { + + /** Abfs Rest Operation. */ + private final AbfsRestOperation op; + /** Flag indicating recovery took place. */ + private final boolean renameRecovered; + /** Abfs storage tracking metadata is in an incomplete state. */ + private final boolean isIncompleteMetadataState; + + /** + * Constructing an ABFS rename operation result. + * @param op The AbfsRestOperation. + * @param renameRecovered Did rename recovery took place? + * @param isIncompleteMetadataState Did the rename failed due to incomplete + * metadata state and had to be retried? + */ + public AbfsClientRenameResult( + AbfsRestOperation op, + boolean renameRecovered, + boolean isIncompleteMetadataState) { + this.op = op; + this.renameRecovered = renameRecovered; + this.isIncompleteMetadataState = isIncompleteMetadataState; + } + + public AbfsRestOperation getOp() { + return op; + } + + public boolean isRenameRecovered() { + return renameRecovered; + } + + public boolean isIncompleteMetadataState() { + return isIncompleteMetadataState; + } +} diff --git a/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/ITestAbfsCustomEncryption.java b/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/ITestAbfsCustomEncryption.java index 61f1292396e9b..9fc71ec5fb349 100644 --- a/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/ITestAbfsCustomEncryption.java +++ b/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/ITestAbfsCustomEncryption.java @@ -241,7 +241,7 @@ private AbfsRestOperation callOperation(AzureBlobFileSystem fs, case LISTSTATUS: return client.listPath(path, false, 5, null, getTestTracingContext(fs, true)); case RENAME: return client.renamePath(path, new Path(path + "_2").toString(), - null, getTestTracingContext(fs, true), null).getLeft(); + null, getTestTracingContext(fs, true), null, false).getOp(); case DELETE: return client.deletePath(path, false, null, getTestTracingContext(fs, false)); case GET_ATTR: return client.getPathStatus(path, true, diff --git a/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/ITestAzureBlobFileSystemDelegationSAS.java b/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/ITestAzureBlobFileSystemDelegationSAS.java index 965e02a0a3ebe..edc3930607cd1 100644 --- a/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/ITestAzureBlobFileSystemDelegationSAS.java +++ b/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/ITestAzureBlobFileSystemDelegationSAS.java @@ -401,8 +401,8 @@ public void testSignatureMask() throws Exception { fs.create(new Path(src)).close(); AbfsRestOperation abfsHttpRestOperation = fs.getAbfsClient() .renamePath(src, "/testABC" + "/abc.txt", null, - getTestTracingContext(fs, false), null) - .getLeft(); + getTestTracingContext(fs, false), null, false) + .getOp(); AbfsHttpOperation result = abfsHttpRestOperation.getResult(); String url = result.getMaskedUrl(); String encodedUrl = result.getMaskedEncodedUrl(); @@ -419,7 +419,7 @@ public void testSignatureMaskOnExceptionMessage() throws Exception { intercept(IOException.class, "sig=XXXX", () -> getFileSystem().getAbfsClient() .renamePath("testABC/test.xt", "testABC/abc.txt", null, - getTestTracingContext(getFileSystem(), false), null)); + getTestTracingContext(getFileSystem(), false), null, false)); } @Test diff --git a/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/ITestAzureBlobFileSystemRename.java b/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/ITestAzureBlobFileSystemRename.java index 716c101493b3f..ea07650e90110 100644 --- a/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/ITestAzureBlobFileSystemRename.java +++ b/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/ITestAzureBlobFileSystemRename.java @@ -30,12 +30,17 @@ import org.apache.hadoop.fs.FileStatus; import org.apache.hadoop.fs.Path; +import org.apache.hadoop.fs.statistics.IOStatisticAssertions; +import org.apache.hadoop.fs.statistics.IOStatistics; +import static org.apache.hadoop.fs.azurebfs.AbfsStatistic.RENAME_PATH_ATTEMPTS; import static org.apache.hadoop.fs.contract.ContractTestUtils.assertIsFile; import static org.apache.hadoop.fs.contract.ContractTestUtils.assertMkdirs; import static org.apache.hadoop.fs.contract.ContractTestUtils.assertPathDoesNotExist; import static org.apache.hadoop.fs.contract.ContractTestUtils.assertPathExists; import static org.apache.hadoop.fs.contract.ContractTestUtils.assertRenameOutcome; +import static org.apache.hadoop.fs.contract.ContractTestUtils.dataset; +import static org.apache.hadoop.fs.contract.ContractTestUtils.writeDataset; /** * Test rename operation. @@ -167,4 +172,30 @@ public void testPosixRenameDirectory() throws Exception { new Path(testDir2 + "/test1/test2/test3")); } + @Test + public void testRenameWithNoDestinationParentDir() throws Exception { + describe("Verifying the expected behaviour of ABFS rename when " + + "destination parent Dir doesn't exist."); + + final AzureBlobFileSystem fs = getFileSystem(); + Path sourcePath = path(getMethodName()); + Path destPath = new Path("falseParent", "someChildFile"); + + byte[] data = dataset(1024, 'a', 'z'); + writeDataset(fs, sourcePath, data, data.length, 1024, true); + + // Verify that renaming on a destination with no parent dir wasn't + // successful. + assertFalse("Rename result expected to be false with no Parent dir", + fs.rename(sourcePath, destPath)); + + // Verify that metadata was in an incomplete state after the rename + // failure, and we retired the rename once more. + IOStatistics ioStatistics = fs.getIOStatistics(); + IOStatisticAssertions.assertThatStatisticCounter(ioStatistics, + RENAME_PATH_ATTEMPTS.getStatName()) + .describedAs("There should be 2 rename attempts if metadata " + + "incomplete state failure is hit") + .isEqualTo(2); + } } diff --git a/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/services/TestAbfsRenameRetryRecovery.java b/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/services/TestAbfsRenameRetryRecovery.java new file mode 100644 index 0000000000000..65ea79b36bd0e --- /dev/null +++ b/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/services/TestAbfsRenameRetryRecovery.java @@ -0,0 +1,139 @@ +/** + * 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.fs.azurebfs.services; + +import org.assertj.core.api.Assertions; +import org.junit.Test; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + +import org.apache.hadoop.fs.azurebfs.AbstractAbfsIntegrationTest; +import org.apache.hadoop.fs.azurebfs.AzureBlobFileSystem; +import org.apache.hadoop.fs.azurebfs.contracts.exceptions.AbfsRestOperationException; +import org.apache.hadoop.fs.azurebfs.contracts.exceptions.AzureBlobFileSystemException; + +import static org.apache.hadoop.fs.azurebfs.constants.AbfsHttpConstants.HTTP_METHOD_PUT; +import static org.apache.hadoop.fs.azurebfs.contracts.services.AzureServiceErrorCode.RENAME_DESTINATION_PARENT_PATH_NOT_FOUND; +import static org.apache.hadoop.test.LambdaTestUtils.intercept; +import static org.mockito.Mockito.doReturn; +import static org.mockito.Mockito.mock; +import static org.mockito.Mockito.times; +import static org.mockito.Mockito.verify; +import static org.mockito.Mockito.when; + +/** + * Testing Abfs Rename recovery using Mockito. + */ +public class TestAbfsRenameRetryRecovery extends AbstractAbfsIntegrationTest { + + private static final Logger LOG = + LoggerFactory.getLogger(TestAbfsRenameRetryRecovery.class); + + public TestAbfsRenameRetryRecovery() throws Exception { + } + + /** + * Mock the AbfsClient to run a metadata incomplete scenario with recovery + * rename. + */ + @Test + public void testRenameFailuresDueToIncompleteMetadata() throws Exception { + String sourcePath = getMethodName() + "Source"; + String destNoParentPath = "/NoParent/Dest"; + AzureBlobFileSystem fs = getFileSystem(); + + AbfsClient mockClient = TestAbfsClient.getMockAbfsClient( + fs.getAbfsStore().getClient(), + fs.getAbfsStore().getAbfsConfiguration()); + + AbfsCounters abfsCounters = mock(AbfsCounters.class); + when(mockClient.getAbfsCounters()).thenReturn(abfsCounters); + // SuccessFul Result. + AbfsRestOperation successOp = + new AbfsRestOperation(AbfsRestOperationType.RenamePath, mockClient, + HTTP_METHOD_PUT, null, null); + AbfsClientRenameResult successResult = mock(AbfsClientRenameResult.class); + doReturn(successOp).when(successResult).getOp(); + when(successResult.isIncompleteMetadataState()).thenReturn(false); + + // Failed Result. + AbfsRestOperation failedOp = new AbfsRestOperation(AbfsRestOperationType.RenamePath, mockClient, + HTTP_METHOD_PUT, null, null); + AbfsClientRenameResult recoveredMetaDataIncompleteResult = + mock(AbfsClientRenameResult.class); + + doReturn(failedOp).when(recoveredMetaDataIncompleteResult).getOp(); + when(recoveredMetaDataIncompleteResult.isIncompleteMetadataState()).thenReturn(true); + + // No destination Parent dir exception. + AzureBlobFileSystemException destParentNotFound + = getMockAbfsRestOperationException( + RENAME_DESTINATION_PARENT_PATH_NOT_FOUND.getStatusCode(), + RENAME_DESTINATION_PARENT_PATH_NOT_FOUND.getErrorCode()); + + // We need to throw an exception once a rename is triggered with + // destination having no parent, but after a retry it needs to succeed. + when(mockClient.renamePath(sourcePath, destNoParentPath, null, null, + null, false)) + .thenThrow(destParentNotFound) + .thenReturn(recoveredMetaDataIncompleteResult); + + // Dest parent not found exc. to be raised. + intercept(AzureBlobFileSystemException.class, + () -> mockClient.renamePath(sourcePath, + destNoParentPath, null, null, + null, false)); + + AbfsClientRenameResult resultOfSecondRenameCall = + mockClient.renamePath(sourcePath, + destNoParentPath, null, null, + null, false); + + // the second rename call should be the recoveredResult due to + // metaDataIncomplete + Assertions.assertThat(resultOfSecondRenameCall) + .describedAs("This result should be recovered result due to MetaData " + + "being in incomplete state") + .isSameAs(recoveredMetaDataIncompleteResult); + // Verify Incomplete metadata state happened for our second rename call. + assertTrue("Metadata incomplete state should be true if a rename is " + + "retried after no Parent directory is found", + resultOfSecondRenameCall.isIncompleteMetadataState()); + + + // Verify renamePath occurred two times implying a retry was attempted. + verify(mockClient, times(2)) + .renamePath(sourcePath, destNoParentPath, null, null, null, false); + + } + + /** + * Method to create an AbfsRestOperationException. + * @param statusCode status code to be used. + * @param errorCode error code to be used. + * @return the exception. + */ + private AbfsRestOperationException getMockAbfsRestOperationException( + int statusCode, String errorCode) { + return new AbfsRestOperationException(statusCode, errorCode, + "No Parent found for the Destination file", + new Exception()); + } + +} diff --git a/hadoop-tools/hadoop-benchmark/pom.xml b/hadoop-tools/hadoop-benchmark/pom.xml new file mode 100644 index 0000000000000..3d742fab5c877 --- /dev/null +++ b/hadoop-tools/hadoop-benchmark/pom.xml @@ -0,0 +1,94 @@ + + + + 4.0.0 + + org.apache.hadoop + hadoop-project + 3.4.0-SNAPSHOT + ../../hadoop-project/pom.xml + + hadoop-benchmark + 3.4.0-SNAPSHOT + jar + + Apache Hadoop Common Benchmark + Apache Hadoop Common Benchmark + + + + org.apache.hadoop + hadoop-common + + + org.openjdk.jmh + jmh-core + + + org.openjdk.jmh + jmh-generator-annprocess + + + + + + + maven-assembly-plugin + + + + org.apache.hadoop.benchmark.VectoredReadBenchmark + + + + src/main/assembly/uber.xml + + + + + make-assembly + package + + single + + + + + + org.codehaus.mojo + findbugs-maven-plugin + + ${basedir}/src/main/findbugs/exclude.xml + + + + com.github.spotbugs + spotbugs-maven-plugin + + ${basedir}/src/main/findbugs/exclude.xml + + + + org.apache.maven.plugins + maven-compiler-plugin + + + + diff --git a/hadoop-tools/hadoop-benchmark/src/main/assembly/uber.xml b/hadoop-tools/hadoop-benchmark/src/main/assembly/uber.xml new file mode 100644 index 0000000000000..014eab951b3cf --- /dev/null +++ b/hadoop-tools/hadoop-benchmark/src/main/assembly/uber.xml @@ -0,0 +1,33 @@ + + + uber + + jar + + false + + + / + true + true + runtime + + + + + metaInf-services + + + diff --git a/hadoop-tools/hadoop-benchmark/src/main/findbugs/exclude.xml b/hadoop-tools/hadoop-benchmark/src/main/findbugs/exclude.xml new file mode 100644 index 0000000000000..05f2a067cf01e --- /dev/null +++ b/hadoop-tools/hadoop-benchmark/src/main/findbugs/exclude.xml @@ -0,0 +1,22 @@ + + + + + + + + + + diff --git a/hadoop-tools/hadoop-benchmark/src/main/java/org/apache/hadoop/benchmark/VectoredReadBenchmark.java b/hadoop-tools/hadoop-benchmark/src/main/java/org/apache/hadoop/benchmark/VectoredReadBenchmark.java new file mode 100644 index 0000000000000..631842f78e20d --- /dev/null +++ b/hadoop-tools/hadoop-benchmark/src/main/java/org/apache/hadoop/benchmark/VectoredReadBenchmark.java @@ -0,0 +1,245 @@ +/* + * 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.benchmark; + +import org.openjdk.jmh.annotations.Benchmark; +import org.openjdk.jmh.annotations.BenchmarkMode; +import org.openjdk.jmh.annotations.Level; +import org.openjdk.jmh.annotations.Mode; +import org.openjdk.jmh.annotations.OutputTimeUnit; +import org.openjdk.jmh.annotations.Param; +import org.openjdk.jmh.annotations.Scope; +import org.openjdk.jmh.annotations.Setup; +import org.openjdk.jmh.annotations.State; +import org.openjdk.jmh.infra.Blackhole; +import org.openjdk.jmh.runner.Runner; +import org.openjdk.jmh.runner.options.OptionsBuilder; + +import java.io.EOFException; +import java.io.IOException; +import java.nio.ByteBuffer; +import java.nio.channels.AsynchronousFileChannel; +import java.nio.channels.CompletionHandler; +import java.nio.file.FileSystems; +import java.nio.file.StandardOpenOption; +import java.util.ArrayList; +import java.util.List; +import java.util.concurrent.TimeUnit; +import java.util.function.IntFunction; + +import org.apache.hadoop.fs.FSDataInputStream; +import org.apache.hadoop.fs.LocalFileSystem; +import org.apache.hadoop.conf.Configuration; +import org.apache.hadoop.fs.FileRange; +import org.apache.hadoop.fs.impl.FileRangeImpl; +import org.apache.hadoop.fs.FileSystem; +import org.apache.hadoop.fs.Path; + +@BenchmarkMode(Mode.AverageTime) +@OutputTimeUnit(TimeUnit.MICROSECONDS) +public class VectoredReadBenchmark { + + static final Path DATA_PATH = getTestDataPath(); + static final String DATA_PATH_PROPERTY = "bench.data"; + static final int READ_SIZE = 64 * 1024; + static final long SEEK_SIZE = 1024L * 1024; + + + static Path getTestDataPath() { + String value = System.getProperty(DATA_PATH_PROPERTY); + return new Path(value == null ? "/tmp/taxi.orc" : value); + } + + @State(Scope.Thread) + public static class FileSystemChoice { + + @Param({"local", "raw"}) + private String fileSystemKind; + + private Configuration conf; + private FileSystem fs; + + @Setup(Level.Trial) + public void setup() { + conf = new Configuration(); + try { + LocalFileSystem local = FileSystem.getLocal(conf); + fs = "raw".equals(fileSystemKind) ? local.getRaw() : local; + } catch (IOException e) { + throw new IllegalArgumentException("Can't get filesystem", e); + } + } + } + + @State(Scope.Thread) + public static class BufferChoice { + @Param({"direct", "array"}) + private String bufferKind; + + private IntFunction allocate; + @Setup(Level.Trial) + public void setup() { + allocate = "array".equals(bufferKind) + ? ByteBuffer::allocate : ByteBuffer::allocateDirect; + } + } + + @Benchmark + public void asyncRead(FileSystemChoice fsChoice, + BufferChoice bufferChoice, + Blackhole blackhole) throws Exception { + FSDataInputStream stream = fsChoice.fs.open(DATA_PATH); + List ranges = new ArrayList<>(); + for(int m=0; m < 100; ++m) { + FileRange range = FileRange.createFileRange(m * SEEK_SIZE, READ_SIZE); + ranges.add(range); + } + stream.readVectored(ranges, bufferChoice.allocate); + for(FileRange range: ranges) { + blackhole.consume(range.getData().get()); + } + stream.close(); + } + + static class Joiner implements CompletionHandler { + private int remaining; + private final ByteBuffer[] result; + private Throwable exception = null; + + Joiner(int total) { + remaining = total; + result = new ByteBuffer[total]; + } + + synchronized void finish() { + remaining -= 1; + if (remaining == 0) { + notify(); + } + } + + synchronized ByteBuffer[] join() throws InterruptedException, IOException { + while (remaining > 0 && exception == null) { + wait(); + } + if (exception != null) { + throw new IOException("problem reading", exception); + } + return result; + } + + + @Override + public synchronized void completed(ByteBuffer buffer, FileRange attachment) { + result[--remaining] = buffer; + if (remaining == 0) { + notify(); + } + } + + @Override + public synchronized void failed(Throwable exc, FileRange attachment) { + this.exception = exc; + notify(); + } + } + + static class FileRangeCallback extends FileRangeImpl implements + CompletionHandler { + private final AsynchronousFileChannel channel; + private final ByteBuffer buffer; + private int completed = 0; + private final Joiner joiner; + + FileRangeCallback(AsynchronousFileChannel channel, long offset, + int length, Joiner joiner, ByteBuffer buffer) { + super(offset, length); + this.channel = channel; + this.joiner = joiner; + this.buffer = buffer; + } + + @Override + public void completed(Integer result, FileRangeCallback attachment) { + final int bytes = result; + if (bytes == -1) { + failed(new EOFException("Read past end of file"), this); + } + completed += bytes; + if (completed < this.getLength()) { + channel.read(buffer, this.getOffset() + completed, this, this); + } else { + buffer.flip(); + joiner.finish(); + } + } + + @Override + public void failed(Throwable exc, FileRangeCallback attachment) { + joiner.failed(exc, this); + } + } + + @Benchmark + public void asyncFileChanArray(BufferChoice bufferChoice, + Blackhole blackhole) throws Exception { + java.nio.file.Path path = FileSystems.getDefault().getPath(DATA_PATH.toString()); + AsynchronousFileChannel channel = AsynchronousFileChannel.open(path, StandardOpenOption.READ); + List ranges = new ArrayList<>(); + Joiner joiner = new Joiner(100); + for(int m=0; m < 100; ++m) { + ByteBuffer buffer = bufferChoice.allocate.apply(READ_SIZE); + FileRangeCallback range = new FileRangeCallback(channel, m * SEEK_SIZE, + READ_SIZE, joiner, buffer); + ranges.add(range); + channel.read(buffer, range.getOffset(), range, range); + } + joiner.join(); + channel.close(); + blackhole.consume(ranges); + } + + @Benchmark + public void syncRead(FileSystemChoice fsChoice, + Blackhole blackhole) throws Exception { + FSDataInputStream stream = fsChoice.fs.open(DATA_PATH); + List result = new ArrayList<>(); + for(int m=0; m < 100; ++m) { + byte[] buffer = new byte[READ_SIZE]; + stream.readFully(m * SEEK_SIZE, buffer); + result.add(buffer); + } + blackhole.consume(result); + stream.close(); + } + + /** + * Run the benchmarks. + * @param args the pathname of a 100MB data file + * @throws Exception any ex. + */ + public static void main(String[] args) throws Exception { + OptionsBuilder opts = new OptionsBuilder(); + opts.include("VectoredReadBenchmark"); + opts.jvmArgs("-server", "-Xms256m", "-Xmx2g", + "-D" + DATA_PATH_PROPERTY + "=" + args[0]); + opts.forks(1); + new Runner(opts.build()).run(); + } +} diff --git a/hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/hdfs/protocolPB/package-info.java b/hadoop-tools/hadoop-benchmark/src/main/java/org/apache/hadoop/benchmark/package-info.java similarity index 89% rename from hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/hdfs/protocolPB/package-info.java rename to hadoop-tools/hadoop-benchmark/src/main/java/org/apache/hadoop/benchmark/package-info.java index 6233024467df7..95d6977e3aba7 100644 --- a/hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/hdfs/protocolPB/package-info.java +++ b/hadoop-tools/hadoop-benchmark/src/main/java/org/apache/hadoop/benchmark/package-info.java @@ -1,4 +1,4 @@ -/** +/* * 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 @@ -15,4 +15,8 @@ * See the License for the specific language governing permissions and * limitations under the License. */ -package org.apache.hadoop.hdfs.protocolPB; \ No newline at end of file + +/** + * Benchmark for Vectored Read IO operations. + */ +package org.apache.hadoop.benchmark; diff --git a/hadoop-tools/hadoop-distcp/src/main/java/org/apache/hadoop/tools/DistCpOptions.java b/hadoop-tools/hadoop-distcp/src/main/java/org/apache/hadoop/tools/DistCpOptions.java index e5685c1492507..2745d828c361b 100644 --- a/hadoop-tools/hadoop-distcp/src/main/java/org/apache/hadoop/tools/DistCpOptions.java +++ b/hadoop-tools/hadoop-distcp/src/main/java/org/apache/hadoop/tools/DistCpOptions.java @@ -684,11 +684,28 @@ public Builder withAppend(boolean newAppend) { return this; } + /** + * whether builder with crc. + * @param newSkipCRC whether to skip crc check + * @return Builder object whether to skip crc check + * @deprecated Use {@link #withSkipCRC(boolean)} instead. + */ + @Deprecated public Builder withCRC(boolean newSkipCRC) { this.skipCRC = newSkipCRC; return this; } + /** + * whether builder with crc. + * @param newSkipCRC whether to skip crc check + * @return Builder object whether to skip crc check + */ + public Builder withSkipCRC(boolean newSkipCRC) { + this.skipCRC = newSkipCRC; + return this; + } + public Builder withBlocking(boolean newBlocking) { this.blocking = newBlocking; return this; diff --git a/hadoop-tools/hadoop-distcp/src/main/java/org/apache/hadoop/tools/OptionsParser.java b/hadoop-tools/hadoop-distcp/src/main/java/org/apache/hadoop/tools/OptionsParser.java index f2875c5f33a4f..49ee09f400400 100644 --- a/hadoop-tools/hadoop-distcp/src/main/java/org/apache/hadoop/tools/OptionsParser.java +++ b/hadoop-tools/hadoop-distcp/src/main/java/org/apache/hadoop/tools/OptionsParser.java @@ -108,7 +108,7 @@ public static DistCpOptions parse(String[] args) command.hasOption(DistCpOptionSwitch.OVERWRITE.getSwitch())) .withAppend( command.hasOption(DistCpOptionSwitch.APPEND.getSwitch())) - .withCRC( + .withSkipCRC( command.hasOption(DistCpOptionSwitch.SKIP_CRC.getSwitch())) .withBlocking( !command.hasOption(DistCpOptionSwitch.BLOCKING.getSwitch())) diff --git a/hadoop-tools/hadoop-distcp/src/test/java/org/apache/hadoop/tools/TestDistCpOptions.java b/hadoop-tools/hadoop-distcp/src/test/java/org/apache/hadoop/tools/TestDistCpOptions.java index ac2100ec98ad0..d126bfdc4f975 100644 --- a/hadoop-tools/hadoop-distcp/src/test/java/org/apache/hadoop/tools/TestDistCpOptions.java +++ b/hadoop-tools/hadoop-distcp/src/test/java/org/apache/hadoop/tools/TestDistCpOptions.java @@ -128,7 +128,7 @@ public void testSetSkipCRC() { new Path("hdfs://localhost:8020/target/")); Assert.assertFalse(builder.build().shouldSkipCRC()); - final DistCpOptions options = builder.withSyncFolder(true).withCRC(true) + final DistCpOptions options = builder.withSyncFolder(true).withSkipCRC(true) .build(); Assert.assertTrue(options.shouldSyncFolder()); Assert.assertTrue(options.shouldSkipCRC()); @@ -391,7 +391,7 @@ public void testAppendOption() { new Path("hdfs://localhost:8020/target/")) .withSyncFolder(true) .withAppend(true) - .withCRC(true) + .withSkipCRC(true) .build(); fail("Append should fail if skipCrc option is specified"); } catch (IllegalArgumentException e) { diff --git a/hadoop-tools/hadoop-distcp/src/test/java/org/apache/hadoop/tools/contract/AbstractContractDistCpTest.java b/hadoop-tools/hadoop-distcp/src/test/java/org/apache/hadoop/tools/contract/AbstractContractDistCpTest.java index 04aeea665c244..8545df30bac8e 100644 --- a/hadoop-tools/hadoop-distcp/src/test/java/org/apache/hadoop/tools/contract/AbstractContractDistCpTest.java +++ b/hadoop-tools/hadoop-distcp/src/test/java/org/apache/hadoop/tools/contract/AbstractContractDistCpTest.java @@ -349,7 +349,7 @@ private Job distCpUpdate(final Path srcDir, final Path destDir) Collections.singletonList(srcDir), destDir) .withDeleteMissing(true) .withSyncFolder(true) - .withCRC(true) + .withSkipCRC(true) .withDirectWrite(shouldUseDirectWrite()) .withOverwrite(false))); } diff --git a/hadoop-tools/hadoop-distcp/src/test/java/org/apache/hadoop/tools/mapred/TestCopyCommitter.java b/hadoop-tools/hadoop-distcp/src/test/java/org/apache/hadoop/tools/mapred/TestCopyCommitter.java index 62940f64b3488..599f3ec2db61e 100644 --- a/hadoop-tools/hadoop-distcp/src/test/java/org/apache/hadoop/tools/mapred/TestCopyCommitter.java +++ b/hadoop-tools/hadoop-distcp/src/test/java/org/apache/hadoop/tools/mapred/TestCopyCommitter.java @@ -536,7 +536,7 @@ private void testCommitWithChecksumMismatch(boolean skipCrc) Collections.singletonList(new Path(sourceBase)), new Path("/out")) .withBlocksPerChunk(blocksPerChunk) - .withCRC(skipCrc) + .withSkipCRC(skipCrc) .build(); options.appendToConf(conf); conf.setBoolean( diff --git a/hadoop-tools/pom.xml b/hadoop-tools/pom.xml index f026bc261e00b..4e934cd101f85 100644 --- a/hadoop-tools/pom.xml +++ b/hadoop-tools/pom.xml @@ -51,6 +51,7 @@ hadoop-azure-datalake hadoop-aliyun hadoop-fs2img + hadoop-benchmark diff --git a/hadoop-yarn-project/hadoop-yarn/dev-support/jdiff/Apache_Hadoop_YARN_API_2.10.2.xml b/hadoop-yarn-project/hadoop-yarn/dev-support/jdiff/Apache_Hadoop_YARN_API_2.10.2.xml new file mode 100644 index 0000000000000..150443f14a097 --- /dev/null +++ b/hadoop-yarn-project/hadoop-yarn/dev-support/jdiff/Apache_Hadoop_YARN_API_2.10.2.xml @@ -0,0 +1,22541 @@ + + + + + + + + + + + + + + + + + + + + The interface used by clients to obtain a new {@link ApplicationId} for + submitting new applications.

    + +

    The ResourceManager responds with a new, monotonically + increasing, {@link ApplicationId} which is used by the client to submit + a new application.

    + +

    The ResourceManager also responds with details such + as maximum resource capabilities in the cluster as specified in + {@link GetNewApplicationResponse}.

    + + @param request request to get a new ApplicationId + @return response containing the new ApplicationId to be used + to submit an application + @throws YarnException + @throws IOException + @see #submitApplication(SubmitApplicationRequest)]]> + + + + + + + + The interface used by clients to submit a new application to the + ResourceManager.

    + +

    The client is required to provide details such as queue, + {@link Resource} required to run the ApplicationMaster, + the equivalent of {@link ContainerLaunchContext} for launching + the ApplicationMaster etc. via the + {@link SubmitApplicationRequest}.

    + +

    Currently the ResourceManager sends an immediate (empty) + {@link SubmitApplicationResponse} on accepting the submission and throws + an exception if it rejects the submission. However, this call needs to be + followed by {@link #getApplicationReport(GetApplicationReportRequest)} + to make sure that the application gets properly submitted - obtaining a + {@link SubmitApplicationResponse} from ResourceManager doesn't guarantee + that RM 'remembers' this application beyond failover or restart. If RM + failover or RM restart happens before ResourceManager saves the + application's state successfully, the subsequent + {@link #getApplicationReport(GetApplicationReportRequest)} will throw + a {@link ApplicationNotFoundException}. The Clients need to re-submit + the application with the same {@link ApplicationSubmissionContext} when + it encounters the {@link ApplicationNotFoundException} on the + {@link #getApplicationReport(GetApplicationReportRequest)} call.

    + +

    During the submission process, it checks whether the application + already exists. If the application exists, it will simply return + SubmitApplicationResponse

    + +

    In secure mode,the ResourceManager verifies access to + queues etc. before accepting the application submission.

    + + @param request request to submit a new application + @return (empty) response on accepting the submission + @throws YarnException + @throws IOException + @see #getNewApplication(GetNewApplicationRequest)]]> +
    +
    + + + + + + The interface used by clients to request the + ResourceManager to fail an application attempt.

    + +

    The client, via {@link FailApplicationAttemptRequest} provides the + {@link ApplicationAttemptId} of the attempt to be failed.

    + +

    In secure mode,the ResourceManager verifies access to the + application, queue etc. before failing the attempt.

    + +

    Currently, the ResourceManager returns an empty response + on success and throws an exception on rejecting the request.

    + + @param request request to fail an attempt + @return ResourceManager returns an empty response + on success and throws an exception on rejecting the request + @throws YarnException + @throws IOException + @see #getQueueUserAcls(GetQueueUserAclsInfoRequest)]]> +
    +
    + + + + + + The interface used by clients to request the + ResourceManager to abort submitted application.

    + +

    The client, via {@link KillApplicationRequest} provides the + {@link ApplicationId} of the application to be aborted.

    + +

    In secure mode,the ResourceManager verifies access to the + application, queue etc. before terminating the application.

    + +

    Currently, the ResourceManager returns an empty response + on success and throws an exception on rejecting the request.

    + + @param request request to abort a submitted application + @return ResourceManager returns an empty response + on success and throws an exception on rejecting the request + @throws YarnException + @throws IOException + @see #getQueueUserAcls(GetQueueUserAclsInfoRequest)]]> +
    +
    + + + + + + The interface used by clients to get metrics about the cluster from + the ResourceManager.

    + +

    The ResourceManager responds with a + {@link GetClusterMetricsResponse} which includes the + {@link YarnClusterMetrics} with details such as number of current + nodes in the cluster.

    + + @param request request for cluster metrics + @return cluster metrics + @throws YarnException + @throws IOException]]> +
    +
    + + + + + + The interface used by clients to get a report of all nodes + in the cluster from the ResourceManager.

    + +

    The ResourceManager responds with a + {@link GetClusterNodesResponse} which includes the + {@link NodeReport} for all the nodes in the cluster.

    + + @param request request for report on all nodes + @return report on all nodes + @throws YarnException + @throws IOException]]> +
    +
    + + + + + + The interface used by clients to get information about queues + from the ResourceManager.

    + +

    The client, via {@link GetQueueInfoRequest}, can ask for details such + as used/total resources, child queues, running applications etc.

    + +

    In secure mode,the ResourceManager verifies access before + providing the information.

    + + @param request request to get queue information + @return queue information + @throws YarnException + @throws IOException]]> +
    +
    + + + + + + The interface used by clients to get information about queue + acls for current user from the ResourceManager. +

    + +

    The ResourceManager responds with queue acls for all + existing queues.

    + + @param request request to get queue acls for current user + @return queue acls for current user + @throws YarnException + @throws IOException]]> +
    +
    + + + + + + + + + + + + + + The interface used by clients to obtain a new {@link ReservationId} for + submitting new reservations.

    + +

    The ResourceManager responds with a new, unique, + {@link ReservationId} which is used by the client to submit + a new reservation.

    + + @param request to get a new ReservationId + @return response containing the new ReservationId to be used + to submit a new reservation + @throws YarnException if the reservation system is not enabled. + @throws IOException on IO failures. + @see #submitReservation(ReservationSubmissionRequest)]]> +
    +
    + + + + + + + The interface used by clients to submit a new reservation to the + {@code ResourceManager}. +

    + +

    + The client packages all details of its request in a + {@link ReservationSubmissionRequest} object. This contains information + about the amount of capacity, temporal constraints, and concurrency needs. + Furthermore, the reservation might be composed of multiple stages, with + ordering dependencies among them. +

    + +

    + In order to respond, a new admission control component in the + {@code ResourceManager} performs an analysis of the resources that have + been committed over the period of time the user is requesting, verify that + the user requests can be fulfilled, and that it respect a sharing policy + (e.g., {@code CapacityOverTimePolicy}). Once it has positively determined + that the ReservationSubmissionRequest is satisfiable the + {@code ResourceManager} answers with a + {@link ReservationSubmissionResponse} that include a non-null + {@link ReservationId}. Upon failure to find a valid allocation the response + is an exception with the reason. + + On application submission the client can use this {@link ReservationId} to + obtain access to the reserved resources. +

    + +

    + The system guarantees that during the time-range specified by the user, the + reservationID will be corresponding to a valid reservation. The amount of + capacity dedicated to such queue can vary overtime, depending of the + allocation that has been determined. But it is guaranteed to satisfy all + the constraint expressed by the user in the + {@link ReservationSubmissionRequest}. +

    + + @param request the request to submit a new Reservation + @return response the {@link ReservationId} on accepting the submission + @throws YarnException if the request is invalid or reservation cannot be + created successfully + @throws IOException]]> +
    +
    + + + + + + + The interface used by clients to update an existing Reservation. This is + referred to as a re-negotiation process, in which a user that has + previously submitted a Reservation. +

    + +

    + The allocation is attempted by virtually substituting all previous + allocations related to this Reservation with new ones, that satisfy the new + {@link ReservationUpdateRequest}. Upon success the previous allocation is + substituted by the new one, and on failure (i.e., if the system cannot find + a valid allocation for the updated request), the previous allocation + remains valid. + + The {@link ReservationId} is not changed, and applications currently + running within this reservation will automatically receive the resources + based on the new allocation. +

    + + @param request to update an existing Reservation (the ReservationRequest + should refer to an existing valid {@link ReservationId}) + @return response empty on successfully updating the existing reservation + @throws YarnException if the request is invalid or reservation cannot be + updated successfully + @throws IOException]]> +
    +
    + + + + + + + The interface used by clients to remove an existing Reservation. + + Upon deletion of a reservation applications running with this reservation, + are automatically downgraded to normal jobs running without any dedicated + reservation. +

    + + @param request to remove an existing Reservation (the ReservationRequest + should refer to an existing valid {@link ReservationId}) + @return response empty on successfully deleting the existing reservation + @throws YarnException if the request is invalid or reservation cannot be + deleted successfully + @throws IOException]]> +
    +
    + + + + + + + The interface used by clients to get the list of reservations in a plan. + The reservationId will be used to search for reservations to list if it is + provided. Otherwise, it will select active reservations within the + startTime and endTime (inclusive). +

    + + @param request to list reservations in a plan. Contains fields to select + String queue, ReservationId reservationId, long startTime, + long endTime, and a bool includeReservationAllocations. + + queue: Required. Cannot be null or empty. Refers to the + reservable queue in the scheduler that was selected when + creating a reservation submission + {@link ReservationSubmissionRequest}. + + reservationId: Optional. If provided, other fields will + be ignored. + + startTime: Optional. If provided, only reservations that + end after the startTime will be selected. This defaults + to 0 if an invalid number is used. + + endTime: Optional. If provided, only reservations that + start on or before endTime will be selected. This defaults + to Long.MAX_VALUE if an invalid number is used. + + includeReservationAllocations: Optional. Flag that + determines whether the entire reservation allocations are + to be returned. Reservation allocations are subject to + change in the event of re-planning as described by + {@code ReservationDefinition}. + + @return response that contains information about reservations that are + being searched for. + @throws YarnException if the request is invalid + @throws IOException on IO failures]]> +
    +
    + + + + + + + The interface used by client to get node to labels mappings in existing cluster +

    + + @param request + @return node to labels mappings + @throws YarnException + @throws IOException]]> +
    +
    + + + + + + + The interface used by client to get labels to nodes mappings + in existing cluster +

    + + @param request + @return labels to nodes mappings + @throws YarnException + @throws IOException]]> +
    +
    + + + + + + + The interface used by client to get node labels in the cluster +

    + + @param request to get node labels collection of this cluster + @return node labels collection of this cluster + @throws YarnException + @throws IOException]]> +
    +
    + + + + + + + The interface used by client to set priority of an application. +

    + @param request to set priority of an application + @return an empty response + @throws YarnException + @throws IOException]]> +
    +
    + + + + + + The interface used by clients to request the + ResourceManager to signal a container. For example, + the client can send command OUTPUT_THREAD_DUMP to dump threads of the + container.

    + +

    The client, via {@link SignalContainerRequest} provides the + id of the container and the signal command.

    + +

    In secure mode,the ResourceManager verifies access to the + application before signaling the container. + The user needs to have MODIFY_APP permission.

    + +

    Currently, the ResourceManager returns an empty response + on success and throws an exception on rejecting the request.

    + + @param request request to signal a container + @return ResourceManager returns an empty response + on success and throws an exception on rejecting the request + @throws YarnException + @throws IOException]]> +
    +
    + + + + + + + The interface used by client to set ApplicationTimeouts of an application. + The UpdateApplicationTimeoutsRequest should have timeout value with + absolute time with ISO8601 format yyyy-MM-dd'T'HH:mm:ss.SSSZ. +

    + Note: If application timeout value is less than or equal to current + time then update application throws YarnException. + @param request to set ApplicationTimeouts of an application + @return a response with updated timeouts. + @throws YarnException if update request has empty values or application is + in completing states. + @throws IOException on IO failures]]> +
    +
    + + + + + + + The interface to get the details for a specific resource profile. +

    + @param request request to get the details of a resource profile + @return Response containing the details for a particular resource profile + @throws YarnException if any error happens inside YARN + @throws IOException in case of other errors]]> +
    +
    + + The protocol between clients and the ResourceManager + to submit/abort jobs and to get information on applications, cluster metrics, + nodes, queues and ACLs.

    ]]> +
    + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + The protocol between clients and the ApplicationHistoryServer to + get the information of completed applications etc. +

    ]]> +
    +
    + + + + + + + + + + The interface used by a new ApplicationMaster to register with + the ResourceManager. +

    + +

    + The ApplicationMaster needs to provide details such as RPC + Port, HTTP tracking url etc. as specified in + {@link RegisterApplicationMasterRequest}. +

    + +

    + The ResourceManager responds with critical details such as + maximum resource capabilities in the cluster as specified in + {@link RegisterApplicationMasterResponse}. +

    + +

    + Re-register is only allowed for Unmanaged Application Master + (UAM) HA, with + {@link org.apache.hadoop.yarn.api.records.ApplicationSubmissionContext#getKeepContainersAcrossApplicationAttempts()} + set to true. +

    + + @param request registration request + @return registration respose + @throws YarnException + @throws IOException + @throws InvalidApplicationMasterRequestException The exception is thrown + when an ApplicationMaster tries to register more then once. + @see RegisterApplicationMasterRequest + @see RegisterApplicationMasterResponse]]> +
    +
    + + + + + + The interface used by an ApplicationMaster to notify the + ResourceManager about its completion (success or failed).

    + +

    The ApplicationMaster has to provide details such as + final state, diagnostics (in case of failures) etc. as specified in + {@link FinishApplicationMasterRequest}.

    + +

    The ResourceManager responds with + {@link FinishApplicationMasterResponse}.

    + + @param request completion request + @return completion response + @throws YarnException + @throws IOException + @see FinishApplicationMasterRequest + @see FinishApplicationMasterResponse]]> +
    +
    + + + + + + + The main interface between an ApplicationMaster and the + ResourceManager. +

    + +

    + The ApplicationMaster uses this interface to provide a list of + {@link ResourceRequest} and returns unused {@link Container} allocated to + it via {@link AllocateRequest}. Optionally, the + ApplicationMaster can also blacklist resources which + it doesn't want to use. +

    + +

    + This also doubles up as a heartbeat to let the + ResourceManager know that the ApplicationMaster + is alive. Thus, applications should periodically make this call to be kept + alive. The frequency depends on + {@link YarnConfiguration#RM_AM_EXPIRY_INTERVAL_MS} which defaults to + {@link YarnConfiguration#DEFAULT_RM_AM_EXPIRY_INTERVAL_MS}. +

    + +

    + The ResourceManager responds with list of allocated + {@link Container}, status of completed containers and headroom information + for the application. +

    + +

    + The ApplicationMaster can use the available headroom + (resources) to decide how to utilized allocated resources and make informed + decisions about future resource requests. +

    + + @param request + allocation request + @return allocation response + @throws YarnException + @throws IOException + @throws InvalidApplicationMasterRequestException + This exception is thrown when an ApplicationMaster calls allocate + without registering first. + @throws InvalidResourceBlacklistRequestException + This exception is thrown when an application provides an invalid + specification for blacklist of resources. + @throws InvalidResourceRequestException + This exception is thrown when a {@link ResourceRequest} is out of + the range of the configured lower and upper limits on the + resources. + @see AllocateRequest + @see AllocateResponse]]> +
    +
    + + The protocol between a live instance of ApplicationMaster + and the ResourceManager.

    + +

    This is used by the ApplicationMaster to register/unregister + and to request and obtain resources in the cluster from the + ResourceManager.

    ]]> +
    +
    + + + + + + + + + + The interface used by clients to claim a resource with the + SharedCacheManager. The client uses a checksum to identify the + resource and an {@link ApplicationId} to identify which application will be + using the resource. +

    + +

    + The SharedCacheManager responds with whether or not the + resource exists in the cache. If the resource exists, a Path + to the resource in the shared cache is returned. If the resource does not + exist, the response is empty. +

    + + @param request request to claim a resource in the shared cache + @return response indicating if the resource is already in the cache + @throws YarnException + @throws IOException]]> +
    +
    + + + + + + + The interface used by clients to release a resource with the + SharedCacheManager. This method is called once an application + is no longer using a claimed resource in the shared cache. The client uses + a checksum to identify the resource and an {@link ApplicationId} to + identify which application is releasing the resource. +

    + +

    + Note: This method is an optimization and the client is not required to call + it for correctness. +

    + +

    + Currently the SharedCacheManager sends an empty response. +

    + + @param request request to release a resource in the shared cache + @return (empty) response on releasing the resource + @throws YarnException + @throws IOException]]> +
    +
    + + + The protocol between clients and the SharedCacheManager to claim + and release resources in the shared cache. +

    ]]> +
    +
    + + + + + + + + + + The ApplicationMaster provides a list of + {@link StartContainerRequest}s to a NodeManager to + start {@link Container}s allocated to it using this interface. +

    + +

    + The ApplicationMaster has to provide details such as allocated + resource capability, security tokens (if enabled), command to be executed + to start the container, environment for the process, necessary + binaries/jar/shared-objects etc. via the {@link ContainerLaunchContext} in + the {@link StartContainerRequest}. +

    + +

    + The NodeManager sends a response via + {@link StartContainersResponse} which includes a list of + {@link Container}s of successfully launched {@link Container}s, a + containerId-to-exception map for each failed {@link StartContainerRequest} in + which the exception indicates errors from per container and a + allServicesMetaData map between the names of auxiliary services and their + corresponding meta-data. Note: None-container-specific exceptions will + still be thrown by the API method itself. +

    +

    + The ApplicationMaster can use + {@link #getContainerStatuses(GetContainerStatusesRequest)} to get updated + statuses of the to-be-launched or launched containers. +

    + + @param request + request to start a list of containers + @return response including conatinerIds of all successfully launched + containers, a containerId-to-exception map for failed requests and + a allServicesMetaData map. + @throws YarnException + @throws IOException]]> +
    +
    + + + + + + + The ApplicationMaster requests a NodeManager to + stop a list of {@link Container}s allocated to it using this + interface. +

    + +

    + The ApplicationMaster sends a {@link StopContainersRequest} + which includes the {@link ContainerId}s of the containers to be stopped. +

    + +

    + The NodeManager sends a response via + {@link StopContainersResponse} which includes a list of {@link ContainerId} + s of successfully stopped containers, a containerId-to-exception map for + each failed request in which the exception indicates errors from per + container. Note: None-container-specific exceptions will still be thrown by + the API method itself. ApplicationMaster can use + {@link #getContainerStatuses(GetContainerStatusesRequest)} to get updated + statuses of the containers. +

    + + @param request + request to stop a list of containers + @return response which includes a list of containerIds of successfully + stopped containers, a containerId-to-exception map for failed + requests. + @throws YarnException + @throws IOException]]> +
    +
    + + + + + + + The API used by the ApplicationMaster to request for current + statuses of Containers from the NodeManager. +

    + +

    + The ApplicationMaster sends a + {@link GetContainerStatusesRequest} which includes the {@link ContainerId}s + of all containers whose statuses are needed. +

    + +

    + The NodeManager responds with + {@link GetContainerStatusesResponse} which includes a list of + {@link ContainerStatus} of the successfully queried containers and a + containerId-to-exception map for each failed request in which the exception + indicates errors from per container. Note: None-container-specific + exceptions will still be thrown by the API method itself. +

    + + @param request + request to get ContainerStatuses of containers with + the specified ContainerIds + @return response containing the list of ContainerStatus of the + successfully queried containers and a containerId-to-exception map + for failed requests. + + @throws YarnException + @throws IOException]]> +
    +
    + + + + + + + The API used by the ApplicationMaster to request for + resource increase of running containers on the NodeManager. +

    + + @param request + request to increase resource of a list of containers + @return response which includes a list of containerIds of containers + whose resource has been successfully increased and a + containerId-to-exception map for failed requests. + + @throws YarnException + @throws IOException]]> +
    +
    + + + + + + + The API used by the ApplicationMaster to request for + resource update of running containers on the NodeManager. +

    + + @param request + request to update resource of a list of containers + @return response which includes a list of containerIds of containers + whose resource has been successfully updated and a + containerId-to-exception map for failed requests. + + @throws YarnException Exception specific to YARN + @throws IOException IOException thrown from NodeManager]]> +
    +
    + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + The protocol between an ApplicationMaster and a + NodeManager to start/stop and increase resource of containers + and to get status of running containers.

    + +

    If security is enabled the NodeManager verifies that the + ApplicationMaster has truly been allocated the container + by the ResourceManager and also verifies all interactions such + as stopping the container or obtaining status information for the container. +

    ]]> +
    +
    + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + response id used to track duplicate responses. + @return response id]]> + + + + + + response id used to track duplicate responses. + @param id response id]]> + + + + + current progress of application. + @return current progress of application]]> + + + + + + current progress of application + @param progress current progress of application]]> + + + + + ResourceRequest to update the + ResourceManager about the application's resource requirements. + @return the list of ResourceRequest + @see ResourceRequest]]> + + + + + + ResourceRequest to update the + ResourceManager about the application's resource requirements. + @param resourceRequests list of ResourceRequest to update the + ResourceManager about the application's + resource requirements + @see ResourceRequest]]> + + + + + ContainerId of containers being + released by the ApplicationMaster. + @return list of ContainerId of containers being + released by the ApplicationMaster]]> + + + + + + ContainerId of containers being + released by the ApplicationMaster + @param releaseContainers list of ContainerId of + containers being released by the + ApplicationMaster]]> + + + + + ResourceBlacklistRequest being sent by the + ApplicationMaster. + @return the ResourceBlacklistRequest being sent by the + ApplicationMaster + @see ResourceBlacklistRequest]]> + + + + + + ResourceBlacklistRequest to inform the + ResourceManager about the blacklist additions and removals + per the ApplicationMaster. + + @param resourceBlacklistRequest the ResourceBlacklistRequest + to inform the ResourceManager about + the blacklist additions and removals + per the ApplicationMaster + @see ResourceBlacklistRequest]]> + + + + + + + + + + + + + + + + ApplicationMaster. + @return list of {@link UpdateContainerRequest} + being sent by the + ApplicationMaster.]]> + + + + + + ResourceManager about the containers that need to be + updated. + @param updateRequests list of UpdateContainerRequest for + containers to be updated]]> + + + + + + + + + + + + + + + + + The core request sent by the ApplicationMaster to the + ResourceManager to obtain resources in the cluster.

    + +

    The request includes: +

      +
    • A response id to track duplicate responses.
    • +
    • Progress information.
    • +
    • + A list of {@link ResourceRequest} to inform the + ResourceManager about the application's + resource requirements. +
    • +
    • + A list of unused {@link Container} which are being returned. +
    • +
    • + A list of {@link UpdateContainerRequest} to inform + the ResourceManager about the change in + requirements of running containers. +
    • +
    + + @see ApplicationMasterProtocol#allocate(AllocateRequest)]]> +
    +
    + + + + + + + responseId of the request. + @see AllocateRequest#setResponseId(int) + @param responseId responseId of the request + @return {@link AllocateRequestBuilder}]]> + + + + + + progress of the request. + @see AllocateRequest#setProgress(float) + @param progress progress of the request + @return {@link AllocateRequestBuilder}]]> + + + + + + askList of the request. + @see AllocateRequest#setAskList(List) + @param askList askList of the request + @return {@link AllocateRequestBuilder}]]> + + + + + + releaseList of the request. + @see AllocateRequest#setReleaseList(List) + @param releaseList releaseList of the request + @return {@link AllocateRequestBuilder}]]> + + + + + + resourceBlacklistRequest of the request. + @see AllocateRequest#setResourceBlacklistRequest( + ResourceBlacklistRequest) + @param resourceBlacklistRequest + resourceBlacklistRequest of the request + @return {@link AllocateRequestBuilder}]]> + + + + + + updateRequests of the request. + @see AllocateRequest#setUpdateRequests(List) + @param updateRequests updateRequests of the request + @return {@link AllocateRequestBuilder}]]> + + + + + + trackingUrl of the request. + @see AllocateRequest#setTrackingUrl(String) + @param trackingUrl new tracking url + @return {@link AllocateRequestBuilder}]]> + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + ResourceManager needs the + ApplicationMaster to take some action then it will send an + AMCommand to the ApplicationMaster. See AMCommand + for details on commands and actions for them. + @return AMCommand if the ApplicationMaster should + take action, null otherwise + @see AMCommand]]> + + + + + last response id. + @return last response id]]> + + + + + newly allocated Container by the + ResourceManager. + @return list of newly allocated Container]]> + + + + + available headroom for resources in the cluster for the + application. + @return limit of available headroom for resources in the cluster for the + application]]> + + + + + completed containers' statuses. + @return the list of completed containers' statuses]]> + + + + + updated NodeReports. Updates could + be changes in health, availability etc of the nodes. + @return The delta of updated nodes since the last response]]> + + + + + + + + + + + The message is a snapshot of the resources the RM wants back from the AM. + While demand persists, the RM will repeat its request; applications should + not interpret each message as a request for additional + resources on top of previous messages. Resources requested consistently + over some duration may be forcibly killed by the RM. + + @return A specification of the resources to reclaim from this AM.]]> + + + + + + 1) AM is receiving first container on underlying NodeManager.
    + OR
    + 2) NMToken master key rolled over in ResourceManager and AM is getting new + container on the same underlying NodeManager. +

    + AM will receive one NMToken per NM irrespective of the number of containers + issued on same NM. AM is expected to store these tokens until issued a + new token for the same NM. + @return list of NMTokens required for communicating with NM]]> + + + + + ResourceManager. + @return list of newly increased containers]]> + + + + + + + + + + + + + + + + + + + + + + + + + + UpdateContainerError for + containers updates requests that were in error]]> + + + + + + + + + + + + + + ResourceManager the + ApplicationMaster during resource negotiation. +

    + The response, includes: +

      +
    • Response ID to track duplicate responses.
    • +
    • + An AMCommand sent by ResourceManager to let the + {@code ApplicationMaster} take some actions (resync, shutdown etc.). +
    • +
    • A list of newly allocated {@link Container}.
    • +
    • A list of completed {@link Container}s' statuses.
    • +
    • + The available headroom for resources in the cluster for the + application. +
    • +
    • A list of nodes whose status has been updated.
    • +
    • The number of available nodes in a cluster.
    • +
    • A description of resources requested back by the cluster
    • +
    • AMRMToken, if AMRMToken has been rolled over
    • +
    • + A list of {@link Container} representing the containers + whose resource has been increased. +
    • +
    • + A list of {@link Container} representing the containers + whose resource has been decreased. +
    • +
    + + @see ApplicationMasterProtocol#allocate(AllocateRequest)]]> +
    +
    + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + Note: {@link NMToken} will be used for authenticating communication with + {@code NodeManager}. + @return the list of container tokens to be used for authorization during + container resource update. + @see NMToken]]> + + + + + + AllocateResponse.getUpdatedContainers. + The token contains the container id and resource capability required for + container resource update. + @param containersToUpdate the list of container tokens to be used + for container resource increase.]]> + + + + The request sent by Application Master to the + Node Manager to change the resource quota of a container.

    + + @see ContainerManagementProtocol#updateContainer(ContainerUpdateRequest)]]> +
    +
    + + + + + + + + + + + + + + + + + + + + + + The response sent by the NodeManager to the + ApplicationMaster when asked to update container resource. +

    + + @see ContainerManagementProtocol#updateContainer(ContainerUpdateRequest)]]> +
    +
    + + + + + + + + + + + ApplicationAttemptId of the attempt to be failed. + @return ApplicationAttemptId of the attempt.]]> + + + + + + + The request sent by the client to the ResourceManager + to fail an application attempt.

    + +

    The request includes the {@link ApplicationAttemptId} of the attempt to + be failed.

    + + @see ApplicationClientProtocol#failApplicationAttempt(FailApplicationAttemptRequest)]]> +
    +
    + + + + + + + The response sent by the ResourceManager to the client + failing an application attempt.

    + +

    Currently it's empty.

    + + @see ApplicationClientProtocol#failApplicationAttempt(FailApplicationAttemptRequest)]]> +
    +
    + + + + + + + + + + + + + final state of the ApplicationMaster. + @return final state of the ApplicationMaster]]> + + + + + + final state of the ApplicationMaster + @param finalState final state of the ApplicationMaster]]> + + + + + diagnostic information on application failure. + @return diagnostic information on application failure]]> + + + + + + diagnostic information on application failure. + @param diagnostics diagnostic information on application failure]]> + + + + + tracking URL for the ApplicationMaster. + This url if contains scheme then that will be used by resource manager + web application proxy otherwise it will default to http. + @return tracking URLfor the ApplicationMaster]]> + + + + + + final tracking URLfor the ApplicationMaster. + This is the web-URL to which ResourceManager or web-application proxy will + redirect client/users once the application is finished and the + ApplicationMaster is gone. +

    + If the passed url has a scheme then that will be used by the + ResourceManager and web-application proxy, otherwise the scheme will + default to http. +

    +

    + Empty, null, "N/A" strings are all valid besides a real URL. In case an url + isn't explicitly passed, it defaults to "N/A" on the ResourceManager. +

    + + @param url + tracking URLfor the ApplicationMaster]]> + + + + + The final request includes details such: +

      +
    • Final state of the {@code ApplicationMaster}
    • +
    • + Diagnostic information in case of failure of the + {@code ApplicationMaster} +
    • +
    • Tracking URL
    • +
    + + @see ApplicationMasterProtocol#finishApplicationMaster(FinishApplicationMasterRequest)]]> +
    +
    + + + + + + + + + + + + ResourceManager to a + ApplicationMaster on it's completion. +

    + The response, includes: +

      +
    • A flag which indicates that the application has successfully unregistered + with the RM and the application can safely stop.
    • +
    +

    + Note: The flag indicates whether the application has successfully + unregistered and is safe to stop. The application may stop after the flag is + true. If the application stops before the flag is true then the RM may retry + the application. + + @see ApplicationMasterProtocol#finishApplicationMaster(FinishApplicationMasterRequest)]]> + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + ApplicationAttemptId of an application attempt. + + @return ApplicationAttemptId of an application attempt]]> + + + + + + ApplicationAttemptId of an application attempt + + @param applicationAttemptId + ApplicationAttemptId of an application attempt]]> + + + + + The request sent by a client to the ResourceManager to get an + {@link ApplicationAttemptReport} for an application attempt. +

    + +

    + The request should include the {@link ApplicationAttemptId} of the + application attempt. +

    + + @see ApplicationAttemptReport + @see ApplicationHistoryProtocol#getApplicationAttemptReport(GetApplicationAttemptReportRequest)]]> +
    +
    + + + + + + + + + + + ApplicationAttemptReport for the application attempt. + + @return ApplicationAttemptReport for the application attempt]]> + + + + + + ApplicationAttemptReport for the application attempt. + + @param applicationAttemptReport + ApplicationAttemptReport for the application attempt]]> + + + + + The response sent by the ResourceManager to a client requesting + an application attempt report. +

    + +

    + The response includes an {@link ApplicationAttemptReport} which has the + details about the particular application attempt +

    + + @see ApplicationAttemptReport + @see ApplicationHistoryProtocol#getApplicationAttemptReport(GetApplicationAttemptReportRequest)]]> +
    +
    + + + + + + + + + + + ApplicationId of an application + + @return ApplicationId of an application]]> + + + + + + ApplicationId of an application + + @param applicationId + ApplicationId of an application]]> + + + + + The request from clients to get a list of application attempt reports of an + application from the ResourceManager. +

    + + @see ApplicationHistoryProtocol#getApplicationAttempts(GetApplicationAttemptsRequest)]]> +
    +
    + + + + + + + + + + + ApplicationReport of an application. + + @return a list of ApplicationReport of an application]]> + + + + + + ApplicationReport of an application. + + @param applicationAttempts + a list of ApplicationReport of an application]]> + + + + + The response sent by the ResourceManager to a client requesting + a list of {@link ApplicationAttemptReport} for application attempts. +

    + +

    + The ApplicationAttemptReport for each application includes the + details of an application attempt. +

    + + @see ApplicationAttemptReport + @see ApplicationHistoryProtocol#getApplicationAttempts(GetApplicationAttemptsRequest)]]> +
    +
    + + + + + + + + + + + ApplicationId of the application. + @return ApplicationId of the application]]> + + + + + + ApplicationId of the application + @param applicationId ApplicationId of the application]]> + + + + The request sent by a client to the ResourceManager to + get an {@link ApplicationReport} for an application.

    + +

    The request should include the {@link ApplicationId} of the + application.

    + + @see ApplicationClientProtocol#getApplicationReport(GetApplicationReportRequest) + @see ApplicationReport]]> +
    +
    + + + + + + + + ApplicationReport for the application. + @return ApplicationReport for the application]]> + + + + The response sent by the ResourceManager to a client + requesting an application report.

    + +

    The response includes an {@link ApplicationReport} which has details such + as user, queue, name, host on which the ApplicationMaster is + running, RPC port, tracking URL, diagnostics, start time etc.

    + + @see ApplicationClientProtocol#getApplicationReport(GetApplicationReportRequest)]]> +
    +
    + + + + + + + + + + + + + + + + + + + + The request from clients to get a report of Applications matching the + giving application types in the cluster from the + ResourceManager. +

    + + @see ApplicationClientProtocol#getApplications(GetApplicationsRequest) + +

    Setting any of the parameters to null, would just disable that + filter

    + + @param scope {@link ApplicationsRequestScope} to filter by + @param users list of users to filter by + @param queues list of scheduler queues to filter by + @param applicationTypes types of applications + @param applicationTags application tags to filter by + @param applicationStates application states to filter by + @param startRange range of application start times to filter by + @param finishRange range of application finish times to filter by + @param limit number of applications to limit to + @return {@link GetApplicationsRequest} to be used with + {@link ApplicationClientProtocol#getApplications(GetApplicationsRequest)}]]> +
    +
    + + + + + The request from clients to get a report of Applications matching the + giving application types in the cluster from the + ResourceManager. +

    + + @param scope {@link ApplicationsRequestScope} to filter by + @see ApplicationClientProtocol#getApplications(GetApplicationsRequest) + @return a report of Applications in {@link GetApplicationsRequest}]]> +
    +
    + + + + + The request from clients to get a report of Applications matching the + giving application types in the cluster from the + ResourceManager. +

    + + + @see ApplicationClientProtocol#getApplications(GetApplicationsRequest) + @return a report of Applications in {@link GetApplicationsRequest}]]> +
    +
    + + + + + The request from clients to get a report of Applications matching the + giving application states in the cluster from the + ResourceManager. +

    + + + @see ApplicationClientProtocol#getApplications(GetApplicationsRequest) + @return a report of Applications in {@link GetApplicationsRequest}]]> +
    +
    + + + + + + The request from clients to get a report of Applications matching the + giving and application types and application types in the cluster from the + ResourceManager. +

    + + + @see ApplicationClientProtocol#getApplications(GetApplicationsRequest) + @return a report of Applications in GetApplicationsRequest]]> +
    +
    + + + + + + + + + + + + The request from clients to get a report of Applications + in the cluster from the ResourceManager.

    + + @see ApplicationClientProtocol#getApplications(GetApplicationsRequest)]]> +
    +
    + + + + + + + + ApplicationReport for applications. + @return ApplicationReport for applications]]> + + + + The response sent by the ResourceManager to a client + requesting an {@link ApplicationReport} for applications.

    + +

    The ApplicationReport for each application includes details + such as user, queue, name, host on which the ApplicationMaster + is running, RPC port, tracking URL, diagnostics, start time etc.

    + + @see ApplicationReport + @see ApplicationClientProtocol#getApplications(GetApplicationsRequest)]]> +
    +
    + + + + + + + + + The request sent by clients to get cluster metrics from the + ResourceManager.

    + +

    Currently, this is empty.

    + + @see ApplicationClientProtocol#getClusterMetrics(GetClusterMetricsRequest)]]> +
    +
    + + + + + + + + YarnClusterMetrics for the cluster. + @return YarnClusterMetrics for the cluster]]> + + + + ResourceManager to a client + requesting cluster metrics. + + @see YarnClusterMetrics + @see ApplicationClientProtocol#getClusterMetrics(GetClusterMetricsRequest)]]> + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + The request from clients to get a report of all nodes + in the cluster from the ResourceManager.

    + + The request will ask for all nodes in the given {@link NodeState}s. + + @see ApplicationClientProtocol#getClusterNodes(GetClusterNodesRequest)]]> +
    +
    + + + + + + + + NodeReport for all nodes in the cluster. + @return NodeReport for all nodes in the cluster]]> + + + + The response sent by the ResourceManager to a client + requesting a {@link NodeReport} for all nodes.

    + +

    The NodeReport contains per-node information such as + available resources, number of containers, tracking url, rack name, health + status etc. + + @see NodeReport + @see ApplicationClientProtocol#getClusterNodes(GetClusterNodesRequest)]]> + + + + + + + + + + + + + ContainerId of the Container. + + @return ContainerId of the Container]]> + + + + + + ContainerId of the container + + @param containerId + ContainerId of the container]]> + + + + + The request sent by a client to the ResourceManager to get an + {@link ContainerReport} for a container. +

    ]]> +
    +
    + + + + + + + + + + + ContainerReport for the container. + + @return ContainerReport for the container]]> + + + + + + + + The response sent by the ResourceManager to a client requesting + a container report. +

    + +

    + The response includes a {@link ContainerReport} which has details of a + container. +

    ]]> +
    +
    + + + + + + + + + + + ApplicationAttemptId of an application attempt. + + @return ApplicationAttemptId of an application attempt]]> + + + + + + ApplicationAttemptId of an application attempt + + @param applicationAttemptId + ApplicationAttemptId of an application attempt]]> + + + + + The request from clients to get a list of container reports, which belong to + an application attempt from the ResourceManager. +

    + + @see ApplicationHistoryProtocol#getContainers(GetContainersRequest)]]> +
    +
    + + + + + + + + + + + ContainerReport for all the containers of an + application attempt. + + @return a list of ContainerReport for all the containers of an + application attempt]]> + + + + + + ContainerReport for all the containers of an + application attempt. + + @param containers + a list of ContainerReport for all the containers of + an application attempt]]> + + + + + The response sent by the ResourceManager to a client requesting + a list of {@link ContainerReport} for containers. +

    + +

    + The ContainerReport for each container includes the container + details. +

    + + @see ContainerReport + @see ApplicationHistoryProtocol#getContainers(GetContainersRequest)]]> +
    +
    + + + + + + + + + + + ContainerIds of containers for which to obtain + the ContainerStatus. + + @return the list of ContainerIds of containers for which to + obtain the ContainerStatus.]]> + + + + + + ContainerIds of containers for which to obtain + the ContainerStatus + + @param containerIds + a list of ContainerIds of containers for which to + obtain the ContainerStatus]]> + + + + ApplicationMaster to the + NodeManager to get {@link ContainerStatus} of requested + containers. + + @see ContainerManagementProtocol#getContainerStatuses(GetContainerStatusesRequest)]]> + + + + + + + + + + ContainerStatuses of the requested containers. + + @return ContainerStatuses of the requested containers.]]> + + + + + + + + + NodeManager to the + ApplicationMaster when asked to obtain the + ContainerStatus of requested containers. + + @see ContainerManagementProtocol#getContainerStatuses(GetContainerStatusesRequest)]]> + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + The request sent by clients to get a new {@link ApplicationId} for + submitting an application.

    + +

    Currently, this is empty.

    + + @see ApplicationClientProtocol#getNewApplication(GetNewApplicationRequest)]]> +
    +
    + + + + + + + + new ApplicationId allocated by the + ResourceManager. + @return new ApplicationId allocated by the + ResourceManager]]> + + + + + ResourceManager in the cluster. + @return maximum capability of allocated resources in the cluster]]> + + + + The response sent by the ResourceManager to the client for + a request to get a new {@link ApplicationId} for submitting applications.

    + +

    Clients can submit an application with the returned + {@link ApplicationId}.

    + + @see ApplicationClientProtocol#getNewApplication(GetNewApplicationRequest)]]> +
    +
    + + + + + + + + + The request sent by clients to get a new {@code ReservationId} for + submitting an reservation.

    + + {@code ApplicationClientProtocol#getNewReservation(GetNewReservationRequest)}]]> +
    +
    + + + + + + + + + + + + The response sent by the ResourceManager to the client for + a request to get a new {@link ReservationId} for submitting reservations.

    + +

    Clients can submit an reservation with the returned + {@link ReservationId}.

    + + {@code ApplicationClientProtocol#getNewReservation(GetNewReservationRequest)}]]> +
    +
    + + + + + + + + + + + + + + queue name for which to get queue information. + @return queue name for which to get queue information]]> + + + + + + queue name for which to get queue information + @param queueName queue name for which to get queue information]]> + + + + + active applications required? + @return true if applications' information is to be included, + else false]]> + + + + + + active applications? + @param includeApplications fetch information about active + applications?]]> + + + + + child queues required? + @return true if information about child queues is required, + else false]]> + + + + + + child queues? + @param includeChildQueues fetch information about child queues?]]> + + + + + child queue hierarchy required? + @return true if information about entire hierarchy is + required, false otherwise]]> + + + + + + child queue hierarchy? + @param recursive fetch information on the entire child queue + hierarchy?]]> + + + + The request sent by clients to get queue information + from the ResourceManager.

    + + @see ApplicationClientProtocol#getQueueInfo(GetQueueInfoRequest)]]> +
    +
    + + + + + + + + QueueInfo for the specified queue. + @return QueueInfo for the specified queue]]> + + + + + The response includes a {@link QueueInfo} which has details such as + queue name, used/total capacities, running applications, child queues etc. + + @see QueueInfo + @see ApplicationClientProtocol#getQueueInfo(GetQueueInfoRequest)]]> + + + + + + + + + + + The request sent by clients to the ResourceManager to + get queue acls for the current user.

    + +

    Currently, this is empty.

    + + @see ApplicationClientProtocol#getQueueUserAcls(GetQueueUserAclsInfoRequest)]]> +
    +
    + + + + + + + + QueueUserACLInfo per queue for the user. + @return QueueUserACLInfo per queue for the user]]> + + + + The response sent by the ResourceManager to clients + seeking queue acls for the user.

    + +

    The response contains a list of {@link QueueUserACLInfo} which + provides information about {@link QueueACL} per queue.

    + + @see QueueACL + @see QueueUserACLInfo + @see ApplicationClientProtocol#getQueueUserAcls(GetQueueUserAclsInfoRequest)]]> +
    +
    + + + + + + + + + + + + Note: {@link NMToken} will be used for authenticating communication with + {@code NodeManager}. + @return the list of container tokens to be used for authorization during + container resource increase. + @see NMToken]]> + + + + + + AllocateResponse.getIncreasedContainers. + The token contains the container id and resource capability required for + container resource increase. + @param containersToIncrease the list of container tokens to be used + for container resource increase.]]> + + + + The request sent by Application Master to the + Node Manager to change the resource quota of a container.

    + + @see ContainerManagementProtocol#increaseContainersResource(IncreaseContainersResourceRequest)]]> +
    +
    + + + + + + + + + + + + + + + + + + The response sent by the NodeManager to the + ApplicationMaster when asked to increase container resource. +

    + + @see ContainerManagementProtocol#increaseContainersResource(IncreaseContainersResourceRequest)]]> +
    +
    + + + + + + + + + + + ApplicationId of the application to be aborted. + @return ApplicationId of the application to be aborted]]> + + + + + + + + diagnostics to which the application is being killed. + @return diagnostics to which the application is being killed]]> + + + + + + diagnostics to which the application is being killed. + @param diagnostics diagnostics to which the application is being + killed]]> + + + + The request sent by the client to the ResourceManager + to abort a submitted application.

    + +

    The request includes the {@link ApplicationId} of the application to be + aborted.

    + + @see ApplicationClientProtocol#forceKillApplication(KillApplicationRequest)]]> +
    +
    + + + + + + + + + + + + ResourceManager to the client aborting + a submitted application. +

    + The response, includes: +

      +
    • + A flag which indicates that the process of killing the application is + completed or not. +
    • +
    + Note: user is recommended to wait until this flag becomes true, otherwise if + the ResourceManager crashes before the process of killing the + application is completed, the ResourceManager may retry this + application on recovery. + + @see ApplicationClientProtocol#forceKillApplication(KillApplicationRequest)]]> +
    +
    + + + + + + + + + + + + ApplicationId of the application to be moved. + @return ApplicationId of the application to be moved]]> + + + + + + ApplicationId of the application to be moved. + @param appId ApplicationId of the application to be moved]]> + + + + + + + + + + + + + + + The request sent by the client to the ResourceManager + to move a submitted application to a different queue.

    + +

    The request includes the {@link ApplicationId} of the application to be + moved and the queue to place it in.

    + + @see ApplicationClientProtocol#moveApplicationAcrossQueues(MoveApplicationAcrossQueuesRequest)]]> +
    +
    + + + + + + + + The response sent by the ResourceManager to the client moving + a submitted application to a different queue. +

    +

    + A response without exception means that the move has completed successfully. +

    + + @see ApplicationClientProtocol#moveApplicationAcrossQueues(MoveApplicationAcrossQueuesRequest)]]> +
    +
    + + + + + + + + + + + RegisterApplicationMasterRequest. + If port, trackingUrl is not used, use the following default value: +
      +
    • port: -1
    • +
    • trackingUrl: null
    • +
    + The port is allowed to be any integer larger than or equal to -1. + @return the new instance of RegisterApplicationMasterRequest]]> +
    +
    + + + host on which the ApplicationMaster is + running. + @return host on which the ApplicationMaster is running]]> + + + + + + host on which the ApplicationMaster is + running. + @param host host on which the ApplicationMaster + is running]]> + + + + + RPC port on which the {@code ApplicationMaster} is + responding. + @return the RPC port on which the {@code ApplicationMaster} + is responding]]> + + + + + + RPC port on which the {@code ApplicationMaster} is + responding. + @param port RPC port on which the {@code ApplicationMaster} + is responding]]> + + + + + tracking URL for the ApplicationMaster. + This url if contains scheme then that will be used by resource manager + web application proxy otherwise it will default to http. + @return tracking URL for the ApplicationMaster]]> + + + + + + tracking URLfor the ApplicationMaster while + it is running. This is the web-URL to which ResourceManager or + web-application proxy will redirect client/users while the application and + the ApplicationMaster are still running. +

    + If the passed url has a scheme then that will be used by the + ResourceManager and web-application proxy, otherwise the scheme will + default to http. +

    +

    + Empty, null, "N/A" strings are all valid besides a real URL. In case an url + isn't explicitly passed, it defaults to "N/A" on the ResourceManager. +

    + + @param trackingUrl + tracking URLfor the ApplicationMaster]]> + + + + + The registration includes details such as: +

      +
    • Hostname on which the AM is running.
    • +
    • RPC Port
    • +
    • Tracking URL
    • +
    + + @see ApplicationMasterProtocol#registerApplicationMaster(RegisterApplicationMasterRequest)]]> +
    +
    + + + + + + + + ResourceManager in the cluster. + @return maximum capability of allocated resources in the cluster]]> + + + + + ApplicationACLs for the application. + @return all the ApplicationACLs]]> + + + + + Get ClientToAMToken master key.

    +

    The ClientToAMToken master key is sent to ApplicationMaster + by ResourceManager via {@link RegisterApplicationMasterResponse} + , used to verify corresponding ClientToAMToken.

    + @return ClientToAMToken master key]]> +
    +
    + + + + + + + + + Get the queue that the application was placed in.

    + @return the queue that the application was placed in.]]> + + + + + + Set the queue that the application was placed in.

    ]]> + + + + + + Get the list of running containers as viewed by + ResourceManager from previous application attempts. +

    + + @return the list of running containers as viewed by + ResourceManager from previous application attempts + @see RegisterApplicationMasterResponse#getNMTokensFromPreviousAttempts()]]> +
    +
    + + + + + + + + + + + + + + + + + + The response contains critical details such as: +
      +
    • Maximum capability for allocated resources in the cluster.
    • +
    • {@code ApplicationACL}s for the application.
    • +
    • ClientToAMToken master key.
    • +
    + + @see ApplicationMasterProtocol#registerApplicationMaster(RegisterApplicationMasterRequest)]]> +
    +
    + + + + + + + + + + + + + + + + ContainerId of the container to re-initialize. + + @return ContainerId of the container to re-initialize.]]> + + + + + ContainerLaunchContext to re-initialize the container + with. + + @return ContainerLaunchContext of to re-initialize the + container with.]]> + + + + + + + + + + + + + + + + + + + + + + + + + + + + ApplicationId of the resource to be released. + + @return ApplicationId]]> + + + + + + ApplicationId of the resource to be released. + + @param id ApplicationId]]> + + + + + key of the resource to be released. + + @return key]]> + + + + + + key of the resource to be released. + + @param key unique identifier for the resource]]> + + + + The request from clients to release a resource in the shared cache.

    ]]> +
    +
    + + + + + + + + The response to clients from the SharedCacheManager when + releasing a resource in the shared cache. +

    + +

    + Currently, this is empty. +

    ]]> +
    +
    + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + The response sent by the ResourceManager to a client on + reservation submission.

    + +

    Currently, this is empty.

    + + {@code ApplicationClientProtocol#submitReservation( + ReservationSubmissionRequest)}]]> +
    +
    + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + ContainerId of the container to localize resources. + + @return ContainerId of the container to localize resources.]]> + + + + + LocalResource required by the container. + + @return all LocalResource required by the container]]> + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + ContainerId of the container to signal. + @return ContainerId of the container to signal.]]> + + + + + + ContainerId of the container to signal.]]> + + + + + SignalContainerCommand of the signal request. + @return SignalContainerCommand of the signal request.]]> + + + + + + SignalContainerCommand of the signal request.]]> + + + + The request sent by the client to the ResourceManager + or by the ApplicationMaster to the NodeManager + to signal a container. + @see SignalContainerCommand

    ]]> +
    +
    + + + + + + + The response sent by the ResourceManager to the client + signalling a container.

    + +

    Currently it's empty.

    + + @see ApplicationClientProtocol#signalToContainer(SignalContainerRequest)]]> +
    +
    + + + + + + + + + + + + ContainerLaunchContext for the container to be started + by the NodeManager. + + @return ContainerLaunchContext for the container to be started + by the NodeManager]]> + + + + + + ContainerLaunchContext for the container to be started + by the NodeManager + @param context ContainerLaunchContext for the container to be + started by the NodeManager]]> + + + + + + Note: {@link NMToken} will be used for authenticating communication with + {@code NodeManager}. + @return the container token to be used for authorization during starting + container. + @see NMToken + @see ContainerManagementProtocol#startContainers(StartContainersRequest)]]> + + + + + + + The request sent by the ApplicationMaster to the + NodeManager to start a container.

    + +

    The ApplicationMaster has to provide details such as + allocated resource capability, security tokens (if enabled), command + to be executed to start the container, environment for the process, + necessary binaries/jar/shared-objects etc. via the + {@link ContainerLaunchContext}.

    + + @see ContainerManagementProtocol#startContainers(StartContainersRequest)]]> +
    +
    + + + + + + + + + + + + + + + + + + + + + + The request which contains a list of {@link StartContainerRequest} sent by + the ApplicationMaster to the NodeManager to + start containers. +

    + +

    + In each {@link StartContainerRequest}, the ApplicationMaster has + to provide details such as allocated resource capability, security tokens (if + enabled), command to be executed to start the container, environment for the + process, necessary binaries/jar/shared-objects etc. via the + {@link ContainerLaunchContext}. +

    + + @see ContainerManagementProtocol#startContainers(StartContainersRequest)]]> +
    +
    + + + + + + + + ContainerId s of the containers that are + started successfully. + + @return the list of ContainerId s of the containers that are + started successfully. + @see ContainerManagementProtocol#startContainers(StartContainersRequest)]]> + + + + + + + + + + + Get the meta-data from all auxiliary services running on the + NodeManager. +

    +

    + The meta-data is returned as a Map between the auxiliary service names and + their corresponding per service meta-data as an opaque blob + ByteBuffer +

    + +

    + To be able to interpret the per-service meta-data, you should consult the + documentation for the Auxiliary-service configured on the NodeManager +

    + + @return a Map between the names of auxiliary services and their + corresponding meta-data]]> +
    +
    + + + The response sent by the NodeManager to the + ApplicationMaster when asked to start an allocated + container. +

    + + @see ContainerManagementProtocol#startContainers(StartContainersRequest)]]> +
    +
    + + + + + + + + + + + ContainerIds of the containers to be stopped. + @return ContainerIds of containers to be stopped]]> + + + + + + ContainerIds of the containers to be stopped. + @param containerIds ContainerIds of the containers to be stopped]]> + + + + The request sent by the ApplicationMaster to the + NodeManager to stop containers.

    + + @see ContainerManagementProtocol#stopContainers(StopContainersRequest)]]> +
    +
    + + + + + + + + + + + + + + + + + + The response sent by the NodeManager to the + ApplicationMaster when asked to stop allocated + containers. +

    + + @see ContainerManagementProtocol#stopContainers(StopContainersRequest)]]> +
    +
    + + + + + + + + + + + ApplicationSubmissionContext for the application. + @return ApplicationSubmissionContext for the application]]> + + + + + + ApplicationSubmissionContext for the application. + @param context ApplicationSubmissionContext for the + application]]> + + + + The request sent by a client to submit an application to the + ResourceManager.

    + +

    The request, via {@link ApplicationSubmissionContext}, contains + details such as queue, {@link Resource} required to run the + ApplicationMaster, the equivalent of + {@link ContainerLaunchContext} for launching the + ApplicationMaster etc. + + @see ApplicationClientProtocol#submitApplication(SubmitApplicationRequest)]]> + + + + + + + + + The response sent by the ResourceManager to a client on + application submission.

    + +

    Currently, this is empty.

    + + @see ApplicationClientProtocol#submitApplication(SubmitApplicationRequest)]]> +
    +
    + + + + + + + + + + + + ApplicationId of the application. + + @return ApplicationId of the application]]> + + + + + + ApplicationId of the application. + + @param applicationId ApplicationId of the application]]> + + + + + Priority of the application to be set. + + @return Priority of the application to be set.]]> + + + + + + Priority of the application. + + @param priority Priority of the application]]> + + + + + The request sent by the client to the ResourceManager to set or + update the application priority. +

    +

    + The request includes the {@link ApplicationId} of the application and + {@link Priority} to be set for an application +

    + + @see ApplicationClientProtocol#updateApplicationPriority(UpdateApplicationPriorityRequest)]]> +
    +
    + + + + + + + + + + + Priority of the application to be set. + @return Updated Priority of the application.]]> + + + + + + Priority of the application. + + @param priority Priority of the application]]> + + + + + The response sent by the ResourceManager to the client on update + the application priority. +

    +

    + A response without exception means that the move has completed successfully. +

    + + @see ApplicationClientProtocol#updateApplicationPriority(UpdateApplicationPriorityRequest)]]> +
    +
    + + + + + + + + + + + + ApplicationId of the application. + @return ApplicationId of the application]]> + + + + + + ApplicationId of the application. + @param applicationId ApplicationId of the application]]> + + + + + ApplicationTimeouts of the application. Timeout value is + in ISO8601 standard with format yyyy-MM-dd'T'HH:mm:ss.SSSZ. + @return all ApplicationTimeouts of the application.]]> + + + + + + ApplicationTimeouts for the application. Timeout value + is absolute. Timeout value should meet ISO8601 format. Support ISO8601 + format is yyyy-MM-dd'T'HH:mm:ss.SSSZ. All pre-existing Map entries + are cleared before adding the new Map. + @param applicationTimeouts ApplicationTimeoutss for the + application]]> + + + + + The request sent by the client to the ResourceManager to set or + update the application timeout. +

    +

    + The request includes the {@link ApplicationId} of the application and timeout + to be set for an application +

    ]]> +
    +
    + + + + + + + + + + ApplicationTimeouts of the application. Timeout value is + in ISO8601 standard with format yyyy-MM-dd'T'HH:mm:ss.SSSZ. + @return all ApplicationTimeouts of the application.]]> + + + + + + ApplicationTimeouts for the application. Timeout value + is absolute. Timeout value should meet ISO8601 format. Support ISO8601 + format is yyyy-MM-dd'T'HH:mm:ss.SSSZ. All pre-existing Map entries + are cleared before adding the new Map. + @param applicationTimeouts ApplicationTimeoutss for the + application]]> + + + + + The response sent by the ResourceManager to the client on update + application timeout. +

    +

    + A response without exception means that the update has completed + successfully. +

    ]]> +
    +
    + + + + + + + + ApplicationId of the resource to be used. + + @return ApplicationId]]> + + + + + + ApplicationId of the resource to be used. + + @param id ApplicationId]]> + + + + + key of the resource to be used. + + @return key]]> + + + + + + key of the resource to be used. + + @param key unique identifier for the resource]]> + + + + + The request from clients to the SharedCacheManager that claims a + resource in the shared cache. +

    ]]> +
    +
    + + + + + + + + Path corresponding to the requested resource in the + shared cache. + + @return String A Path if the resource exists in the shared + cache, null otherwise]]> + + + + + + Path corresponding to a resource in the shared cache. + + @param p A Path corresponding to a resource in the shared + cache]]> + + + + + The response from the SharedCacheManager to the client that indicates whether + a requested resource exists in the cache. +

    ]]> +
    +
    + +
    + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + ApplicationId of the ApplicationAttempId. + @return ApplicationId of the ApplicationAttempId]]> + + + + + attempt id of the Application. + @return attempt id of the Application]]> + + + + + + + + + + + + + + + + + + + + + ApplicationAttemptId denotes the particular attempt + of an ApplicationMaster for a given {@link ApplicationId}.

    + +

    Multiple attempts might be needed to run an application to completion due + to temporal failures of the ApplicationMaster such as hardware + failures, connectivity issues etc. on the node on which it was scheduled.

    ]]> +
    +
    + + + + + + + + + + + + + + + + + + YarnApplicationAttemptState of the application attempt. + + @return YarnApplicationAttemptState of the application attempt]]> + + + + + RPC port of this attempt ApplicationMaster. + + @return RPC port of this attempt ApplicationMaster]]> + + + + + host on which this attempt of + ApplicationMaster is running. + + @return host on which this attempt of + ApplicationMaster is running]]> + + + + + diagnositic information of the application attempt in case + of errors. + + @return diagnositic information of the application attempt in case + of errors]]> + + + + + tracking url for the application attempt. + + @return tracking url for the application attempt]]> + + + + + original tracking url for the application attempt. + + @return original tracking url for the application attempt]]> + + + + + ApplicationAttemptId of this attempt of the + application + + @return ApplicationAttemptId of the attempt]]> + + + + + ContainerId of AMContainer for this attempt + + @return ContainerId of the attempt]]> + + + + + + + finish time of the application. + + @return finish time of the application]]> + + + + + It includes details such as: +
      +
    • {@link ApplicationAttemptId} of the application.
    • +
    • Host on which the ApplicationMaster of this attempt is + running.
    • +
    • RPC port of the ApplicationMaster of this attempt.
    • +
    • Tracking URL.
    • +
    • Diagnostic information in case of errors.
    • +
    • {@link YarnApplicationAttemptState} of the application attempt.
    • +
    • {@link ContainerId} of the master Container.
    • +
    ]]> +
    +
    + + + + + + + + + + + + + ApplicationId + which is unique for all applications started by a particular instance + of the ResourceManager. + @return short integer identifier of the ApplicationId]]> + + + + + start time of the ResourceManager which is + used to generate globally unique ApplicationId. + @return start time of the ResourceManager]]> + + + + + + + + + + + + + + + + + + + + + ApplicationId represents the globally unique + identifier for an application.

    + +

    The globally unique nature of the identifier is achieved by using the + cluster timestamp i.e. start-time of the + ResourceManager along with a monotonically increasing counter + for the application.

    ]]> +
    +
    + + + + + + + + ApplicationId of the application. + @return ApplicationId of the application]]> + + + + + ApplicationAttemptId of the current + attempt of the application + @return ApplicationAttemptId of the attempt]]> + + + + + user who submitted the application. + @return user who submitted the application]]> + + + + + queue to which the application was submitted. + @return queue to which the application was submitted]]> + + + + + name of the application. + @return name of the application]]> + + + + + host on which the ApplicationMaster + is running. + @return host on which the ApplicationMaster + is running]]> + + + + + RPC port of the ApplicationMaster. + @return RPC port of the ApplicationMaster]]> + + + + + client token for communicating with the + ApplicationMaster. +

    + ClientToAMToken is the security token used by the AMs to verify + authenticity of any client. +

    + +

    + The ResourceManager, provides a secure token (via + {@link ApplicationReport#getClientToAMToken()}) which is verified by the + ApplicationMaster when the client directly talks to an AM. +

    + @return client token for communicating with the + ApplicationMaster]]> +
    +
    + + + YarnApplicationState of the application. + @return YarnApplicationState of the application]]> + + + + + diagnositic information of the application in case of + errors. + @return diagnositic information of the application in case + of errors]]> + + + + + tracking url for the application. + @return tracking url for the application]]> + + + + + start time of the application. + @return start time of the application]]> + + + + + + + + + finish time of the application. + @return finish time of the application]]> + + + + + final finish status of the application. + @return final finish status of the application]]> + + + + + + + + + + + + + + + + + + + + + + + + + + The AMRM token is required for AM to RM scheduling operations. For + managed Application Masters Yarn takes care of injecting it. For unmanaged + Applications Masters, the token must be obtained via this method and set + in the {@link org.apache.hadoop.security.UserGroupInformation} of the + current user. +

    + The AMRM token will be returned only if all the following conditions are + met: +

      +
    • the requester is the owner of the ApplicationMaster
    • +
    • the application master is an unmanaged ApplicationMaster
    • +
    • the application master is in ACCEPTED state
    • +
    + Else this method returns NULL. + + @return the AM to RM token if available.]]> +
    +
    + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + It includes details such as: +
      +
    • {@link ApplicationId} of the application.
    • +
    • Applications user.
    • +
    • Application queue.
    • +
    • Application name.
    • +
    • Host on which the ApplicationMaster is running.
    • +
    • RPC port of the ApplicationMaster.
    • +
    • Tracking URL.
    • +
    • {@link YarnApplicationState} of the application.
    • +
    • Diagnostic information in case of errors.
    • +
    • Start time of the application.
    • +
    • Client {@link Token} of the application (if security is enabled).
    • +
    + + @see ApplicationClientProtocol#getApplicationReport(org.apache.hadoop.yarn.api.protocolrecords.GetApplicationReportRequest)]]> +
    +
    + + + + + + + + + + + + + Resource. -1 for invalid/inaccessible reports. + @return the used Resource]]> + + + + + Resource. -1 for invalid/inaccessible reports. + @return the reserved Resource]]> + + + + + Resource. -1 for invalid/inaccessible reports. + @return the needed Resource]]> + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + ApplicationId of the submitted application. + @return ApplicationId of the submitted application]]> + + + + + + ApplicationId of the submitted application. + @param applicationId ApplicationId of the submitted + application]]> + + + + + name. + @return application name]]> + + + + + + name. + @param applicationName application name]]> + + + + + queue to which the application is being submitted. + @return queue to which the application is being submitted]]> + + + + + + queue to which the application is being submitted + @param queue queue to which the application is being submitted]]> + + + + + Priority of the application. + @return Priority of the application]]> + + + + + ContainerLaunchContext to describe the + Container with which the ApplicationMaster is + launched. + @return ContainerLaunchContext for the + ApplicationMaster container]]> + + + + + + ContainerLaunchContext to describe the + Container with which the ApplicationMaster is + launched. + @param amContainer ContainerLaunchContext for the + ApplicationMaster container]]> + + + + + YarnApplicationState. + Such apps will not be retried by the RM on app attempt failure. + The default value is false. + @return true if the AM is not managed by the RM]]> + + + + + + + + + + + + + + + + + + + + + + ApplicationMaster for this + application. Please note this will be DEPRECATED, use getResource + in getAMContainerResourceRequest instead. + + @return the resource required by the ApplicationMaster for + this application.]]> + + + + + + ApplicationMaster for this + application. + + @param resource the resource required by the ApplicationMaster + for this application.]]> + + + + + + + + + + + + + + + + + + + + + + + For managed AM, if the flag is true, running containers will not be killed + when application attempt fails and these containers will be retrieved by + the new application attempt on registration via + {@link ApplicationMasterProtocol#registerApplicationMaster(RegisterApplicationMasterRequest)}. +

    +

    + For unmanaged AM, if the flag is true, RM allows re-register and returns + the running containers in the same attempt back to the UAM for HA. +

    + + @param keepContainers the flag which indicates whether to keep containers + across application attempts.]]> +
    +
    + + + + + + + + + + + + + + + + + + + + + + + + + getResource and getPriority of + ApplicationSubmissionContext. + + Number of containers and Priority will be ignored. + + @return ResourceRequest of the AM container + @deprecated See {@link #getAMContainerResourceRequests()}]]> + + + + + + + + + + + getAMContainerResourceRequest and its behavior. + + Number of containers and Priority will be ignored. + + @return List of ResourceRequests of the AM container]]> + + + + + + + + + + + + + + + + + + + + + + LogAggregationContext of the application + + @return LogAggregationContext of the application]]> + + + + + + LogAggregationContext for the application + + @param logAggregationContext + for the application]]> + + + + + + + + + + + + + + + + ApplicationTimeouts of the application. Timeout value is + in seconds. + @return all ApplicationTimeouts of the application.]]> + + + + + + ApplicationTimeouts for the application in seconds. + All pre-existing Map entries are cleared before adding the new Map. +

    + Note: If application timeout value is less than or equal to zero + then application submission will throw an exception. +

    + @param applicationTimeouts ApplicationTimeoutss for the + application]]> +
    +
    + + + It includes details such as: +
      +
    • {@link ApplicationId} of the application.
    • +
    • Application user.
    • +
    • Application name.
    • +
    • {@link Priority} of the application.
    • +
    • + {@link ContainerLaunchContext} of the container in which the + ApplicationMaster is executed. +
    • +
    • + maxAppAttempts. The maximum number of application attempts. + It should be no larger than the global number of max attempts in the + Yarn configuration. +
    • +
    • + attemptFailuresValidityInterval. The default value is -1. + when attemptFailuresValidityInterval in milliseconds is set to + {@literal >} 0, the failure number will no take failures which happen + out of the validityInterval into failure count. If failure count + reaches to maxAppAttempts, the application will be failed. +
    • +
    • Optional, application-specific {@link LogAggregationContext}
    • +
    + + @see ContainerLaunchContext + @see ApplicationClientProtocol#submitApplication(org.apache.hadoop.yarn.api.protocolrecords.SubmitApplicationRequest)]]> +
    +
    + + + + + + + + + + + + + + + + + + + + + + + + expiryTime for given timeout type. + @return expiryTime in ISO8601 standard with format + yyyy-MM-dd'T'HH:mm:ss.SSSZ.]]> + + + + + + expiryTime for given timeout type. + @param expiryTime in ISO8601 standard with format + yyyy-MM-dd'T'HH:mm:ss.SSSZ.]]> + + + + + Remaining Time of an application for given timeout type. + @return Remaining Time in seconds.]]> + + + + + + Remaining Time of an application for given timeout type. + @param remainingTime in seconds.]]> + + + + +
  • {@link ApplicationTimeoutType} of the timeout type.
  • +
  • Expiry time in ISO8601 standard with format + yyyy-MM-dd'T'HH:mm:ss.SSSZ or "UNLIMITED".
  • +
  • Remaining time in seconds.
  • + + The possible values for {ExpiryTime, RemainingTimeInSeconds} are +
      +
    • {UNLIMITED,-1} : Timeout is not configured for given timeout type + (LIFETIME).
    • +
    • {ISO8601 date string, 0} : Timeout is configured and application has + completed.
    • +
    • {ISO8601 date string, greater than zero} : Timeout is configured and + application is RUNNING. Application will be timed out after configured + value.
    • +
    ]]> +
    +
    + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + Resource allocated to the container. + @return Resource allocated to the container]]> + + + + + Priority at which the Container was + allocated. + @return Priority at which the Container was + allocated]]> + + + + + ContainerToken for the container. +

    ContainerToken is the security token used by the framework + to verify authenticity of any Container.

    + +

    The ResourceManager, on container allocation provides a + secure token which is verified by the NodeManager on + container launch.

    + +

    Applications do not need to care about ContainerToken, they + are transparently handled by the framework - the allocated + Container includes the ContainerToken.

    + + @see ApplicationMasterProtocol#allocate(org.apache.hadoop.yarn.api.protocolrecords.AllocateRequest) + @see ContainerManagementProtocol#startContainers(org.apache.hadoop.yarn.api.protocolrecords.StartContainersRequest) + + @return ContainerToken for the container]]> +
    +
    + + + ID corresponding to the original {@code + ResourceRequest{@link #getAllocationRequestId()}}s which is satisfied by + this allocated {@code Container}. +

    + The scheduler may return multiple {@code AllocateResponse}s corresponding + to the same ID as and when scheduler allocates {@code Container}s. + Applications can continue to completely ignore the returned ID in + the response and use the allocation for any of their outstanding requests. +

    + + @return the ID corresponding to the original allocation request + which is satisfied by this allocation.]]> + + + + + The {@code ResourceManager} is the sole authority to allocate any + {@code Container} to applications. The allocated {@code Container} + is always on a single node and has a unique {@link ContainerId}. It has + a specific amount of {@link Resource} allocated. +

    + It includes details such as: +

      +
    • {@link ContainerId} for the container, which is globally unique.
    • +
    • + {@link NodeId} of the node on which it is allocated. +
    • +
    • HTTP uri of the node.
    • +
    • {@link Resource} allocated to the container.
    • +
    • {@link Priority} at which the container was allocated.
    • +
    • + Container {@link Token} of the container, used to securely verify + authenticity of the allocation. +
    • +
    + + Typically, an {@code ApplicationMaster} receives the {@code Container} + from the {@code ResourceManager} during resource-negotiation and then + talks to the {@code NodeManager} to start/stop containers. + + @see ApplicationMasterProtocol#allocate(org.apache.hadoop.yarn.api.protocolrecords.AllocateRequest) + @see ContainerManagementProtocol#startContainers(org.apache.hadoop.yarn.api.protocolrecords.StartContainersRequest) + @see ContainerManagementProtocol#stopContainers(org.apache.hadoop.yarn.api.protocolrecords.StopContainersRequest)]]> +
    +
    + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + ApplicationAttemptId of the application to which the + Container was assigned. +

    + Note: If containers are kept alive across application attempts via + {@link ApplicationSubmissionContext#setKeepContainersAcrossApplicationAttempts(boolean)} + the ContainerId does not necessarily contain the current + running application attempt's ApplicationAttemptId This + container can be allocated by previously exited application attempt and + managed by the current running attempt thus have the previous application + attempt's ApplicationAttemptId. +

    + + @return ApplicationAttemptId of the application to which the + Container was assigned]]> +
    +
    + + + ContainerId, + which doesn't include epoch. Note that this method will be marked as + deprecated, so please use getContainerId instead. + @return lower 32 bits of identifier of the ContainerId]]> + + + + + ContainerId. Upper 24 bits are + reserved as epoch of cluster, and lower 40 bits are reserved as + sequential number of containers. + @return identifier of the ContainerId]]> + + + + + + + + + + + + + + + + + + + + + + + + ContainerId represents a globally unique identifier + for a {@link Container} in the cluster.

    ]]> +
    +
    + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + LocalResource required by the container. + @return all LocalResource required by the container]]> + + + + + + LocalResource required by the container. All pre-existing + Map entries are cleared before adding the new Map + @param localResources LocalResource required by the container]]> + + + + + + Get application-specific binary service data. This is a map keyed + by the name of each {@link AuxiliaryService} that is configured on a + NodeManager and value correspond to the application specific data targeted + for the keyed {@link AuxiliaryService}. +

    + +

    + This will be used to initialize this application on the specific + {@link AuxiliaryService} running on the NodeManager by calling + {@link AuxiliaryService#initializeApplication(ApplicationInitializationContext)} +

    + + @return application-specific binary service data]]> +
    +
    + + + + + Set application-specific binary service data. This is a map keyed + by the name of each {@link AuxiliaryService} that is configured on a + NodeManager and value correspond to the application specific data targeted + for the keyed {@link AuxiliaryService}. All pre-existing Map entries are + preserved. +

    + + @param serviceData + application-specific binary service data]]> +
    +
    + + + environment variables for the container. + @return environment variables for the container]]> + + + + + + environment variables for the container. All pre-existing Map + entries are cleared before adding the new Map + @param environment environment variables for the container]]> + + + + + commands for launching the container. + @return the list of commands for launching the container]]> + + + + + + commands for launching the container. All + pre-existing List entries are cleared before adding the new List + @param commands the list of commands for launching the container]]> + + + + + ApplicationACLs for the application. + @return all the ApplicationACLs]]> + + + + + + ApplicationACLs for the application. All pre-existing + Map entries are cleared before adding the new Map + @param acls ApplicationACLs for the application]]> + + + + + ContainerRetryContext to relaunch container. + @return ContainerRetryContext to relaunch container.]]> + + + + + + ContainerRetryContext to relaunch container. + @param containerRetryContext ContainerRetryContext to + relaunch container.]]> + + + + + It includes details such as: +
      +
    • {@link ContainerId} of the container.
    • +
    • {@link Resource} allocated to the container.
    • +
    • User to whom the container is allocated.
    • +
    • Security tokens (if security is enabled).
    • +
    • + {@link LocalResource} necessary for running the container such + as binaries, jar, shared-objects, side-files etc. +
    • +
    • Optional, application-specific binary service data.
    • +
    • Environment variables for the launched process.
    • +
    • Command to launch the container.
    • +
    • Retry strategy when container exits with failure.
    • +
    + + @see ContainerManagementProtocol#startContainers(org.apache.hadoop.yarn.api.protocolrecords.StartContainersRequest)]]> +
    +
    + + + + + + + + ContainerId of the container. + + @return ContainerId of the container.]]> + + + + + + + + Resource of the container. + + @return allocated Resource of the container.]]> + + + + + + + + NodeId where container is running. + + @return allocated NodeId where container is running.]]> + + + + + + + + Priority of the container. + + @return allocated Priority of the container.]]> + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + ContainerState of the container. + + @return final ContainerState of the container.]]> + + + + + + + + exit status of the container. + + @return final exit status of the container.]]> + + + + + + + + + + + + + + + + + + It includes details such as: +
      +
    • {@link ContainerId} of the container.
    • +
    • Allocated Resources to the container.
    • +
    • Assigned Node id.
    • +
    • Assigned Priority.
    • +
    • Creation Time.
    • +
    • Finish Time.
    • +
    • Container Exit Status.
    • +
    • {@link ContainerState} of the container.
    • +
    • Diagnostic information in case of errors.
    • +
    • Log URL.
    • +
    • nodeHttpAddress
    • +
    ]]> +
    +
    + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + It provides details such as: +
      +
    • + {@link ContainerRetryPolicy} : + - NEVER_RETRY(DEFAULT value): no matter what error code is when container + fails to run, just do not retry. + - RETRY_ON_ALL_ERRORS: no matter what error code is, when container fails + to run, just retry. + - RETRY_ON_SPECIFIC_ERROR_CODES: when container fails to run, do retry if + the error code is one of errorCodes, otherwise do not retry. + + Note: if error code is 137(SIGKILL) or 143(SIGTERM), it will not retry + because it is usually killed on purpose. +
    • +
    • + maxRetries specifies how many times to retry if need to retry. + If the value is -1, it means retry forever. +
    • +
    • retryInterval specifies delaying some time before relaunch + container, the unit is millisecond.
    • +
    ]]> +
    +
    + + + + + + + + + + Retry policy for relaunching a Container.

    ]]> +
    +
    + + + + + + + + + + State of a Container.

    ]]> +
    +
    + + + + + + + + ContainerId of the container. + @return ContainerId of the container]]> + + + + + ExecutionType of the container. + @return ExecutionType of the container]]> + + + + + ContainerState of the container. + @return ContainerState of the container]]> + + + + + Get the exit status for the container.

    + +

    Note: This is valid only for completed containers i.e. containers + with state {@link ContainerState#COMPLETE}. + Otherwise, it returns an ContainerExitStatus.INVALID. +

    + +

    Containers killed by the framework, either due to being released by + the application or being 'lost' due to node failures etc. have a special + exit code of ContainerExitStatus.ABORTED.

    + +

    When threshold number of the nodemanager-local-directories or + threshold number of the nodemanager-log-directories become bad, then + container is not launched and is exited with ContainersExitStatus.DISKS_FAILED. +

    + + @return exit status for the container]]> +
    +
    + + + diagnostic messages for failed containers. + @return diagnostic messages for failed containers]]> + + + + + Resource allocated to the container. + @return Resource allocated to the container]]> + + + + + + + + + + + + + + + It provides details such as: +
      +
    • {@code ContainerId} of the container.
    • +
    • {@code ExecutionType} of the container.
    • +
    • {@code ContainerState} of the container.
    • +
    • Exit status of a completed container.
    • +
    • Diagnostic message for a failed container.
    • +
    • {@link Resource} allocated to the container.
    • +
    ]]> +
    +
    + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + The execution types are the following: +
      +
    • {@link #GUARANTEED} - this container is guaranteed to start its + execution, once the corresponding start container request is received by + an NM. +
    • {@link #OPPORTUNISTIC} - the execution of this container may not start + immediately at the NM that receives the corresponding start container + request (depending on the NM's available resources). Moreover, it may be + preempted if it blocks a GUARANTEED container from being executed. +
    ]]> +
    +
    + + + + + + + + + + + + + + + + + + + ExecutionType of the requested container. + + @param execType + ExecutionType of the requested container]]> + + + + + ExecutionType. + + @return ExecutionType.]]> + + + + + + + + + + + ResourceRequest. + Defaults to false. + @return whether ExecutionType request should be strictly honored]]> + + + + + + + + + ExecutionType as well as flag that explicitly asks the + configuredScheduler to return Containers of exactly the Execution Type + requested.]]> + + + + + + + + + + + + Application.]]> + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + location of the resource to be localized. + @return location of the resource to be localized]]> + + + + + + location of the resource to be localized. + @param resource location of the resource to be localized]]> + + + + + size of the resource to be localized. + @return size of the resource to be localized]]> + + + + + + size of the resource to be localized. + @param size size of the resource to be localized]]> + + + + + timestamp of the resource to be localized, used + for verification. + @return timestamp of the resource to be localized]]> + + + + + + timestamp of the resource to be localized, used + for verification. + @param timestamp timestamp of the resource to be localized]]> + + + + + LocalResourceType of the resource to be localized. + @return LocalResourceType of the resource to be localized]]> + + + + + + LocalResourceType of the resource to be localized. + @param type LocalResourceType of the resource to be localized]]> + + + + + LocalResourceVisibility of the resource to be + localized. + @return LocalResourceVisibility of the resource to be + localized]]> + + + + + + LocalResourceVisibility of the resource to be + localized. + @param visibility LocalResourceVisibility of the resource to be + localized]]> + + + + + pattern that should be used to extract entries from the + archive (only used when type is PATTERN). + @return pattern that should be used to extract entries from the + archive.]]> + + + + + + pattern that should be used to extract entries from the + archive (only used when type is PATTERN). + @param pattern pattern that should be used to extract entries + from the archive.]]> + + + + + + + + + + + shouldBeUploadedToSharedCache + of this request]]> + + + + LocalResource represents a local resource required to + run a container.

    + +

    The NodeManager is responsible for localizing the resource + prior to launching the container.

    + +

    Applications can specify {@link LocalResourceType} and + {@link LocalResourceVisibility}.

    + + @see LocalResourceType + @see LocalResourceVisibility + @see ContainerLaunchContext + @see ApplicationSubmissionContext + @see ContainerManagementProtocol#startContainers(org.apache.hadoop.yarn.api.protocolrecords.StartContainersRequest)]]> +
    +
    + + + + + + + + + + type + of a resource localized by the {@code NodeManager}. +

    + The type can be one of: +

      +
    • + {@link #FILE} - Regular file i.e. uninterpreted bytes. +
    • +
    • + {@link #ARCHIVE} - Archive, which is automatically unarchived by the + NodeManager. +
    • +
    • + {@link #PATTERN} - A hybrid between {@link #ARCHIVE} and {@link #FILE}. +
    • +
    + + @see LocalResource + @see ContainerLaunchContext + @see ApplicationSubmissionContext + @see ContainerManagementProtocol#startContainers(org.apache.hadoop.yarn.api.protocolrecords.StartContainersRequest)]]> +
    +
    + + + + + + + + + + visibility + of a resource localized by the {@code NodeManager}. +

    + The visibility can be one of: +

      +
    • {@link #PUBLIC} - Shared by all users on the node.
    • +
    • + {@link #PRIVATE} - Shared among all applications of the + same user on the node. +
    • +
    • + {@link #APPLICATION} - Shared only among containers of the + same application on the node. +
    • +
    + + @see LocalResource + @see ContainerLaunchContext + @see ApplicationSubmissionContext + @see ContainerManagementProtocol#startContainers(org.apache.hadoop.yarn.api.protocolrecords.StartContainersRequest)]]> +
    +
    + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + It includes details such as: +
      +
    • + includePattern. It uses Java Regex to filter the log files + which match the defined include pattern and those log files + will be uploaded when the application finishes. +
    • +
    • + excludePattern. It uses Java Regex to filter the log files + which match the defined exclude pattern and those log files + will not be uploaded when application finishes. If the log file + name matches both the include and the exclude pattern, this file + will be excluded eventually. +
    • +
    • + rolledLogsIncludePattern. It uses Java Regex to filter the log files + which match the defined include pattern and those log files + will be aggregated in a rolling fashion. +
    • +
    • + rolledLogsExcludePattern. It uses Java Regex to filter the log files + which match the defined exclude pattern and those log files + will not be aggregated in a rolling fashion. If the log file + name matches both the include and the exclude pattern, this file + will be excluded eventually. +
    • +
    • + policyClassName. The policy class name that implements + ContainerLogAggregationPolicy. At runtime, nodemanager will the policy + if a given container's log should be aggregated based on the + ContainerType and other runtime state such as exit code by calling + ContainerLogAggregationPolicy#shouldDoLogAggregation. + This is useful when the app only wants to aggregate logs of a subset of + containers. Here are the available policies. Please make sure to specify + the canonical name by prefixing org.apache.hadoop.yarn.server. + nodemanager.containermanager.logaggregation. + to the class simple name below. + NoneContainerLogAggregationPolicy: skip aggregation for all containers. + AllContainerLogAggregationPolicy: aggregate all containers. + AMOrFailedContainerLogAggregationPolicy: aggregate application master + or failed containers. + FailedOrKilledContainerLogAggregationPolicy: aggregate failed or killed + containers + FailedContainerLogAggregationPolicy: aggregate failed containers + AMOnlyLogAggregationPolicy: aggregate application master containers + SampleContainerLogAggregationPolicy: sample logs of successful worker + containers, in addition to application master and failed/killed + containers. + If it isn't specified, it will use the cluster-wide default policy + defined by configuration yarn.nodemanager.log-aggregation.policy.class. + The default value of yarn.nodemanager.log-aggregation.policy.class is + AllContainerLogAggregationPolicy. +
    • +
    • + policyParameters. The parameters passed to the policy class via + ContainerLogAggregationPolicy#parseParameters during the policy object + initialization. This is optional. Some policy class might use parameters + to adjust its settings. It is up to policy class to define the scheme of + parameters. + For example, SampleContainerLogAggregationPolicy supports the format of + "SR:0.5,MIN:50", which means sample rate of 50% beyond the first 50 + successful worker containers. +
    • +
    + + @see ApplicationSubmissionContext]]> +
    +
    + + + + + + + + NodeManager for which the NMToken + is used to authenticate. + @return the {@link NodeId} of the NodeManager for which the + NMToken is used to authenticate.]]> + + + + + + + + NodeManager + @return the {@link Token} used for authenticating with NodeManager]]> + + + + + + + + + + + + The NMToken is used for authenticating communication with + NodeManager

    +

    It is issued by ResourceMananger when ApplicationMaster + negotiates resource with ResourceManager and + validated on NodeManager side.

    + @see AllocateResponse#getNMTokens()]]> +
    +
    + + + + + + + + + + + + + hostname of the node. + @return hostname of the node]]> + + + + + port for communicating with the node. + @return port for communicating with the node]]> + + + + + + + + + + + + + + + + + + + NodeId is the unique identifier for a node.

    + +

    It includes the hostname and port to uniquely + identify the node. Thus, it is unique across restarts of any + NodeManager.

    ]]> +
    +
    + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + NodeId of the node. + @return NodeId of the node]]> + + + + + NodeState of the node. + @return NodeState of the node]]> + + + + + http address of the node. + @return http address of the node]]> + + + + + rack name for the node. + @return rack name for the node]]> + + + + + used Resource on the node. + @return used Resource on the node]]> + + + + + total Resource on the node. + @return total Resource on the node]]> + + + + + diagnostic health report of the node. + @return diagnostic health report of the node]]> + + + + + last timestamp at which the health report was received. + @return last timestamp at which the health report was received]]> + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + It includes details such as: +
      +
    • {@link NodeId} of the node.
    • +
    • HTTP Tracking URL of the node.
    • +
    • Rack name for the node.
    • +
    • Used {@link Resource} on the node.
    • +
    • Total available {@link Resource} of the node.
    • +
    • Number of running containers on the node.
    • +
    + + @see ApplicationClientProtocol#getClusterNodes(org.apache.hadoop.yarn.api.protocolrecords.GetClusterNodesRequest)]]> +
    +
    + + + + + + + + + + + + + + + + State of a Node.

    ]]> +
    +
    + + + + + + + + + + + + ResourceManager. + @see PreemptionContract + @see StrictPreemptionContract]]> + + + + + + + + + + ApplicationMaster about resources requested back by the + ResourceManager. + @see AllocateRequest#setAskList(List)]]> + + + + + ApplicationMaster that may be reclaimed by the + ResourceManager. If the AM prefers a different set of + containers, then it may checkpoint or kill containers matching the + description in {@link #getResourceRequest}. + @return Set of containers at risk if the contract is not met.]]> + + + + ResourceManager. + The ApplicationMaster (AM) can satisfy this request according + to its own priorities to prevent containers from being forcibly killed by + the platform. + @see PreemptionMessage]]> + + + + + + + + + + ResourceManager]]> + + + + + + + + + + The AM should decode both parts of the message. The {@link + StrictPreemptionContract} specifies particular allocations that the RM + requires back. The AM can checkpoint containers' state, adjust its execution + plan to move the computation, or take no action and hope that conditions that + caused the RM to ask for the container will change. +

    + In contrast, the {@link PreemptionContract} also includes a description of + resources with a set of containers. If the AM releases containers matching + that profile, then the containers enumerated in {@link + PreemptionContract#getContainers()} may not be killed. +

    + Each preemption message reflects the RM's current understanding of the + cluster state, so a request to return N containers may not + reflect containers the AM is releasing, recently exited containers the RM has + yet to learn about, or new containers allocated before the message was + generated. Conversely, an RM may request a different profile of containers in + subsequent requests. +

    + The policy enforced by the RM is part of the scheduler. Generally, only + containers that have been requested consistently should be killed, but the + details are not specified.]]> + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + The ACL is one of: +

      +
    • + {@link #SUBMIT_APPLICATIONS} - ACL to submit applications to the queue. +
    • +
    • {@link #ADMINISTER_QUEUE} - ACL to administer the queue.
    • +
    + + @see QueueInfo + @see ApplicationClientProtocol#getQueueUserAcls(org.apache.hadoop.yarn.api.protocolrecords.GetQueueUserAclsInfoRequest)]]> +
    +
    + + + + + + + + name of the queue. + @return name of the queue]]> + + + + + configured capacity of the queue. + @return configured capacity of the queue]]> + + + + + maximum capacity of the queue. + @return maximum capacity of the queue]]> + + + + + current capacity of the queue. + @return current capacity of the queue]]> + + + + + child queues of the queue. + @return child queues of the queue]]> + + + + + running applications of the queue. + @return running applications of the queue]]> + + + + + QueueState of the queue. + @return QueueState of the queue]]> + + + + + accessible node labels of the queue. + @return accessible node labels of the queue]]> + + + + + default node label expression of the queue, this takes + affect only when the ApplicationSubmissionContext and + ResourceRequest don't specify their + NodeLabelExpression. + + @return default node label expression of the queue]]> + + + + + + + + queue stats for the queue + + @return queue stats of the queue]]> + + + + + + + + + + + preemption status of the queue. + @return if property is not in proto, return null; + otherwise, return preemption status of the queue]]> + + + + + + + + + + + + + + + It includes information such as: +
      +
    • Queue name.
    • +
    • Capacity of the queue.
    • +
    • Maximum capacity of the queue.
    • +
    • Current capacity of the queue.
    • +
    • Child queues.
    • +
    • Running applications.
    • +
    • {@link QueueState} of the queue.
    • +
    • {@link QueueConfigurations} of the queue.
    • +
    + + @see QueueState + @see QueueConfigurations + @see ApplicationClientProtocol#getQueueInfo(org.apache.hadoop.yarn.api.protocolrecords.GetQueueInfoRequest)]]> +
    +
    + + + + + + + + + + + A queue is in one of: +
      +
    • {@link #RUNNING} - normal state.
    • +
    • {@link #STOPPED} - not accepting new application submissions.
    • +
    • + {@link #DRAINING} - not accepting new application submissions + and waiting for applications finish. +
    • +
    + + @see QueueInfo + @see ApplicationClientProtocol#getQueueInfo(org.apache.hadoop.yarn.api.protocolrecords.GetQueueInfoRequest)]]> +
    +
    + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + queue name of the queue. + @return queue name of the queue]]> + + + + + QueueACL for the given user. + @return list of QueueACL for the given user]]> + + + + QueueUserACLInfo provides information {@link QueueACL} for + the given user.

    + + @see QueueACL + @see ApplicationClientProtocol#getQueueUserAcls(org.apache.hadoop.yarn.api.protocolrecords.GetQueueUserAclsInfoRequest)]]> +
    +
    + + + + + + + + + + + The ACL is one of: +
      +
    • + {@link #ADMINISTER_RESERVATIONS} - ACL to create, list, update and + delete reservations. +
    • +
    • {@link #LIST_RESERVATIONS} - ACL to list reservations.
    • +
    • {@link #SUBMIT_RESERVATIONS} - ACL to create reservations.
    • +
    + Users can always list, update and delete their own reservations.]]> +
    +
    + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + It includes: +
      +
    • Duration of the reservation.
    • +
    • Acceptance time of the duration.
    • +
    • + List of {@link ResourceAllocationRequest}, which includes the time + interval, and capability of the allocation. + {@code ResourceAllocationRequest} represents an allocation + made for a reservation for the current state of the queue. This can be + changed for reasons such as re-planning, but will always be subject to + the constraints of the user contract as described by + {@link ReservationDefinition} +
    • +
    • {@link ReservationId} of the reservation.
    • +
    • {@link ReservationDefinition} used to make the reservation.
    • +
    + + @see ResourceAllocationRequest + @see ReservationId + @see ReservationDefinition]]> +
    +
    + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + start time of the {@code ResourceManager} which is used to + generate globally unique {@link ReservationId}. + + @return start time of the {@code ResourceManager}]]> + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + {@link ReservationId} represents the globally unique identifier for + a reservation. +

    + +

    + The globally unique nature of the identifier is achieved by using the + cluster timestamp i.e. start-time of the {@code ResourceManager} + along with a monotonically increasing counter for the reservation. +

    ]]> +
    +
    + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + It includes: +
      +
    • {@link Resource} required for each request.
    • +
    • + Number of containers, of above specifications, which are required by the + application. +
    • +
    • Concurrency that indicates the gang size of the request.
    • +
    ]]> +
    +
    + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + memory of the resource. Note - while memory has + never had a unit specified, all YARN configurations have specified memory + in MB. The assumption has been that the daemons and applications are always + using the same units. With the introduction of the ResourceInformation + class we have support for units - so this function will continue to return + memory but in the units of MB + + @return memory(in MB) of the resource]]> + + + + + memory of the resource. Note - while memory has + never had a unit specified, all YARN configurations have specified memory + in MB. The assumption has been that the daemons and applications are always + using the same units. With the introduction of the ResourceInformation + class we have support for units - so this function will continue to return + memory but in the units of MB + + @return memory of the resource]]> + + + + + + memory of the resource. Note - while memory has + never had a unit specified, all YARN configurations have specified memory + in MB. The assumption has been that the daemons and applications are always + using the same units. With the introduction of the ResourceInformation + class we have support for units - so this function will continue to set + memory but the assumption is that the value passed is in units of MB. + + @param memory memory(in MB) of the resource]]> + + + + + + memory of the resource. + @param memory memory of the resource]]> + + + + + number of virtual cpu cores of the resource. + + Virtual cores are a unit for expressing CPU parallelism. A node's capacity + should be configured with virtual cores equal to its number of physical + cores. A container should be requested with the number of cores it can + saturate, i.e. the average number of threads it expects to have runnable + at a time. + + @return num of virtual cpu cores of the resource]]> + + + + + + number of virtual cpu cores of the resource. + + Virtual cores are a unit for expressing CPU parallelism. A node's capacity + should be configured with virtual cores equal to its number of physical + cores. A container should be requested with the number of cores it can + saturate, i.e. the average number of threads it expects to have runnable + at a time. + + @param vCores number of virtual cpu cores of the resource]]> + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + Resource models a set of computer resources in the + cluster.

    + +

    Currently it models both memory and CPU.

    + +

    The unit for memory is megabytes. CPU is modeled with virtual cores + (vcores), a unit for expressing parallelism. A node's capacity should + be configured with virtual cores equal to its number of physical cores. A + container should be requested with the number of cores it can saturate, i.e. + the average number of threads it expects to have runnable at a time.

    + +

    Virtual cores take integer values and thus currently CPU-scheduling is + very coarse. A complementary axis for CPU requests that represents + processing power will likely be added in the future to enable finer-grained + resource configuration.

    + +

    Typically, applications request Resource of suitable + capability to run their component tasks.

    + + @see ResourceRequest + @see ApplicationMasterProtocol#allocate(org.apache.hadoop.yarn.api.protocolrecords.AllocateRequest)]]> +
    +
    + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + It includes: +
      +
    • StartTime of the allocation.
    • +
    • EndTime of the allocation.
    • +
    • {@link Resource} reserved for the allocation.
    • +
    + + @see Resource]]> +
    +
    + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + blacklist of resources + for the application. + + @see ResourceRequest + @see ApplicationMasterProtocol#allocate(org.apache.hadoop.yarn.api.protocolrecords.AllocateRequest)]]> + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + host/rack string represents an arbitrary + host name. + + @param hostName host/rack on which the allocation is desired + @return whether the given host/rack string represents an arbitrary + host name]]> + + + + + Priority of the request. + @return Priority of the request]]> + + + + + + Priority of the request + @param priority Priority of the request]]> + + + + + host/rack) on which the allocation + is desired. + + A special value of * signifies that any resource + (host/rack) is acceptable. + + @return resource (e.g. host/rack) on which the allocation + is desired]]> + + + + + + host/rack) on which the allocation + is desired. + + A special value of * signifies that any resource name + (e.g. host/rack) is acceptable. + + @param resourceName (e.g. host/rack) on which the + allocation is desired]]> + + + + + + + + + + + + + + + + ResourceRequest. Defaults to true. + + @return whether locality relaxation is enabled with this + ResourceRequest.]]> + + + + + + ExecutionTypeRequest of the requested container. + + @param execSpec + ExecutionTypeRequest of the requested container]]> + + + + + ResourceRequest. Defaults to true. + + @return whether locality relaxation is enabled with this + ResourceRequest.]]> + + + + + + For a request at a network hierarchy level, set whether locality can be relaxed + to that level and beyond.

    + +

    If the flag is off on a rack-level ResourceRequest, + containers at that request's priority will not be assigned to nodes on that + request's rack unless requests specifically for those nodes have also been + submitted.

    + +

    If the flag is off on an {@link ResourceRequest#ANY}-level + ResourceRequest, containers at that request's priority will + only be assigned on racks for which specific requests have also been + submitted.

    + +

    For example, to request a container strictly on a specific node, the + corresponding rack-level and any-level requests should have locality + relaxation set to false. Similarly, to request a container strictly on a + specific rack, the corresponding any-level request should have locality + relaxation set to false.

    + + @param relaxLocality whether locality relaxation is enabled with this + ResourceRequest.]]> + + + + + + + + + + + + + + + + ID corresponding to this allocation request. This + ID is an identifier for different {@code ResourceRequest}s from the same + application. The allocated {@code Container}(s) received as part of the + {@code AllocateResponse} response will have the ID corresponding to the + original {@code ResourceRequest} for which the RM made the allocation. +

    + The scheduler may return multiple {@code AllocateResponse}s corresponding + to the same ID as and when scheduler allocates {@code Container}(s). + Applications can continue to completely ignore the returned ID in + the response and use the allocation for any of their outstanding requests. +

    + If one wishes to replace an entire {@code ResourceRequest} corresponding to + a specific ID, they can simply cancel the corresponding {@code + ResourceRequest} and submit a new one afresh. + + @return the ID corresponding to this allocation request.]]> + + + + + + ID corresponding to this allocation request. This + ID is an identifier for different {@code ResourceRequest}s from the same + application. The allocated {@code Container}(s) received as part of the + {@code AllocateResponse} response will have the ID corresponding to the + original {@code ResourceRequest} for which the RM made the allocation. +

    + The scheduler may return multiple {@code AllocateResponse}s corresponding + to the same ID as and when scheduler allocates {@code Container}(s). + Applications can continue to completely ignore the returned ID in + the response and use the allocation for any of their outstanding requests. +

    + If one wishes to replace an entire {@code ResourceRequest} corresponding to + a specific ID, they can simply cancel the corresponding {@code + ResourceRequest} and submit a new one afresh. +

    + If the ID is not set, scheduler will continue to work as previously and all + allocated {@code Container}(s) will have the default ID, -1. + + @param allocationRequestID the ID corresponding to this allocation + request.]]> + + + + + + Resource capability of the request. + @param capability Resource capability of the request]]> + + + + + Resource capability of the request. + @return Resource capability of the request]]> + + + + + + + + + + + + + + + + + + It includes: +

      +
    • {@link Priority} of the request.
    • +
    • + The name of the host or rack on which the allocation is + desired. A special value of * signifies that + any host/rack is acceptable to the application. +
    • +
    • {@link Resource} required for each request.
    • +
    • + Number of containers, of above specifications, which are required + by the application. +
    • +
    • + A boolean relaxLocality flag, defaulting to {@code true}, + which tells the {@code ResourceManager} if the application wants + locality to be loose (i.e. allows fall-through to rack or any) + or strict (i.e. specify hard constraint on resource allocation). +
    • +
    + + @see Resource + @see ApplicationMasterProtocol#allocate(org.apache.hadoop.yarn.api.protocolrecords.AllocateRequest)]]> +
    +
    + + + + + + + priority of the request. + @see ResourceRequest#setPriority(Priority) + @param priority priority of the request + @return {@link ResourceRequestBuilder}]]> + + + + + + resourceName of the request. + @see ResourceRequest#setResourceName(String) + @param resourceName resourceName of the request + @return {@link ResourceRequestBuilder}]]> + + + + + + capability of the request. + @see ResourceRequest#setCapability(Resource) + @param capability capability of the request + @return {@link ResourceRequestBuilder}]]> + + + + + + numContainers of the request. + @see ResourceRequest#setNumContainers(int) + @param numContainers numContainers of the request + @return {@link ResourceRequestBuilder}]]> + + + + + + relaxLocality of the request. + @see ResourceRequest#setRelaxLocality(boolean) + @param relaxLocality relaxLocality of the request + @return {@link ResourceRequestBuilder}]]> + + + + + + nodeLabelExpression of the request. + @see ResourceRequest#setNodeLabelExpression(String) + @param nodeLabelExpression + nodeLabelExpression of the request + @return {@link ResourceRequestBuilder}]]> + + + + + + executionTypeRequest of the request. + @see ResourceRequest#setExecutionTypeRequest( + ExecutionTypeRequest) + @param executionTypeRequest + executionTypeRequest of the request + @return {@link ResourceRequestBuilder}]]> + + + + + + executionTypeRequest of the request with 'ensure + execution type' flag set to true. + @see ResourceRequest#setExecutionTypeRequest( + ExecutionTypeRequest) + @param executionType executionType of the request. + @return {@link ResourceRequestBuilder}]]> + + + + + + allocationRequestId of the request. + @see ResourceRequest#setAllocationRequestId(long) + @param allocationRequestId + allocationRequestId of the request + @return {@link ResourceRequestBuilder}]]> + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + virtual memory. + + @return virtual memory in MB]]> + + + + + + virtual memory. + + @param vmem virtual memory in MB]]> + + + + + physical memory. + + @return physical memory in MB]]> + + + + + + physical memory. + + @param pmem physical memory in MB]]> + + + + + CPU utilization. + + @return CPU utilization normalized to 1 CPU]]> + + + + + + CPU utilization. + + @param cpu CPU utilization normalized to 1 CPU]]> + + + + + + + + + + + + + + + + + + + + + + + + + + + + ResourceUtilization models the utilization of a set of computer + resources in the cluster. +

    ]]> +
    +
    + + + + + + + + + + + + + + + + + + + + ApplicationMaster that may be reclaimed by the + ResourceManager. + @return the set of {@link ContainerId} to be preempted.]]> + + + + ApplicationMaster (AM) + may attempt to checkpoint work or adjust its execution plan to accommodate + it. In contrast to {@link PreemptionContract}, the AM has no flexibility in + selecting which resources to return to the cluster. + @see PreemptionMessage]]> + + + + + + + + + + + + + + + + + + + + + + + + + + + + + Token is the security entity used by the framework + to verify authenticity of any resource.

    ]]> +
    +
    + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + ContainerId of the container. + @return ContainerId of the container]]> + + + + + + + + + + + ContainerUpdateType of the container. + @return ContainerUpdateType of the container.]]> + + + + + + ContainerUpdateType of the container. + @param updateType of the Container]]> + + + + + ContainerId of the container. + @return ContainerId of the container]]> + + + + + + ContainerId of the container. + @param containerId ContainerId of the container]]> + + + + + ExecutionType of the container. + @return ExecutionType of the container]]> + + + + + + ExecutionType of the container. + @param executionType ExecutionType of the container]]> + + + + + + Resource capability of the request. + @param capability Resource capability of the request]]> + + + + + Resource capability of the request. + @return Resource capability of the request]]> + + + + + + + + + + + + It includes: +
      +
    • version for the container.
    • +
    • {@link ContainerId} for the container.
    • +
    • + {@link Resource} capability of the container after the update request + is completed. +
    • +
    • + {@link ExecutionType} of the container after the update request is + completed. +
    • +
    + + Update rules: +
      +
    • + Currently only ONE aspect of the container can be updated per request + (user can either update Capability OR ExecutionType in one request.. + not both). +
    • +
    • + There must be only 1 update request per container in an allocate call. +
    • +
    • + If a new update request is sent for a container (in a subsequent allocate + call) before the first one is satisfied by the Scheduler, it will + overwrite the previous request. +
    • +
    + @see ApplicationMasterProtocol#allocate(org.apache.hadoop.yarn.api.protocolrecords.AllocateRequest)]]> +
    +
    + + + + + + + + + + + + + + + ContainerUpdateType. + @return ContainerUpdateType]]> + + + + + + ContainerUpdateType. + @param updateType ContainerUpdateType]]> + + + + + Container. + @return Container]]> + + + + + + Container. + @param container Container]]> + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + URL represents a serializable {@link java.net.URL}.

    ]]> +
    +
    + + + + + + + + + + RMAppAttempt.]]> + + + + + + + + + + + + ApplicationMaster.]]> + + + + + + + + + + NodeManagers in the cluster. + @return number of NodeManagers in the cluster]]> + + + + + DecommissionedNodeManagers in the cluster. + + @return number of DecommissionedNodeManagers in the cluster]]> + + + + + ActiveNodeManagers in the cluster. + + @return number of ActiveNodeManagers in the cluster]]> + + + + + LostNodeManagers in the cluster. + + @return number of LostNodeManagers in the cluster]]> + + + + + UnhealthyNodeManagers in the cluster. + + @return number of UnhealthyNodeManagers in the cluster]]> + + + + + RebootedNodeManagers in the cluster. + + @return number of RebootedNodeManagers in the cluster]]> + + + + YarnClusterMetrics represents cluster metrics.

    + +

    Currently only number of NodeManagers is provided.

    ]]> +
    +
    + +
    + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + This class contains the information about a timeline domain, which is used + to a user to host a number of timeline entities, isolating them from others'. + The user can also define the reader and writer users/groups for the the + domain, which is used to control the access to its entities. +

    + +

    + The reader and writer users/groups pattern that the user can supply is the + same as what AccessControlList takes. +

    ]]> +
    +
    + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + The class that contains the the meta information of some conceptual entity + and its related events. The entity can be an application, an application + attempt, a container or whatever the user-defined object. +

    + +

    + Primary filters will be used to index the entities in + TimelineStore, such that users should carefully choose the + information they want to store as the primary filters. The remaining can be + stored as other information. +

    ]]> +
    +
    + + + + + + + + + + + + + ApplicationId of the + TimelineEntityGroupId. + + @return ApplicationId of the + TimelineEntityGroupId]]> + + + + + + + + timelineEntityGroupId. + + @return timelineEntityGroupId]]> + + + + + + + + + + + + + + + + + + + TimelineEntityGroupId is an abstract way for + timeline service users to represent #a group of related timeline data. + For example, all entities that represents one data flow DAG execution + can be grouped into one timeline entity group.

    ]]> +
    +
    + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + +
    + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + The constuctor is used to construct a proxy {@link TimelineEntity} or its + subclass object from the real entity object that carries information. +

    + +

    + It is usually used in the case where we want to recover class polymorphism + after deserializing the entity from its JSON form. +

    + @param entity the real entity that carries information]]> +
    +
    + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + Note: Entities will be stored in the order of idPrefix specified. + If users decide to set idPrefix for an entity, they MUST provide + the same prefix for every update of this entity. +

    + Example:
    + TimelineEntity entity = new TimelineEntity();
    + entity.setIdPrefix(value);
    + 
    + Users can use {@link TimelineServiceHelper#invertLong(long)} to invert + the prefix if necessary. + + @param entityIdPrefix prefix for an entity.]]> +
    +
    + + + + + + + +
    + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + +
    + + + + + + + + + + + + + + + + name property as a + InetSocketAddress. On an HA cluster, + this fetches the address corresponding to the RM identified by + {@link #RM_HA_ID}. + @param name property name. + @param defaultAddress the default value + @param defaultPort the default port + @return InetSocketAddress]]> + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + OPPORTUNISTIC containers on the NM.]]> + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + +
  • default
  • +
  • docker
  • + ]]> +
    +
    + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + Default platform-specific CLASSPATH for YARN applications. A + comma-separated list of CLASSPATH entries constructed based on the client + OS environment expansion syntax. +

    +

    + Note: Use {@link #DEFAULT_YARN_CROSS_PLATFORM_APPLICATION_CLASSPATH} for + cross-platform practice i.e. submit an application from a Windows client to + a Linux/Unix server or vice versa. +

    ]]> +
    +
    + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + +
    + +
    + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + The information is passed along to applications via + {@link StartContainersResponse#getAllServicesMetaData()} that is returned by + {@link ContainerManagementProtocol#startContainers(StartContainersRequest)} +

    + + @return meta-data for this service that should be made available to + applications.]]> +
    +
    + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + +
    + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + The method used by the NodeManager log aggregation service + to initial the policy object with parameters specified by the application + or the cluster-wide setting. +

    + + @param parameters parameters with scheme defined by the policy class.]]> +
    +
    + + + + + The method used by the NodeManager log aggregation service + to ask the policy object if a given container's logs should be aggregated. +

    + + @param logContext ContainerLogContext + @return Whether or not the container's logs should be aggregated.]]> +
    +
    + + + +
    + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + The method used by administrators to ask SCM to run cleaner task right away +

    + + @param request request SharedCacheManager to run a cleaner task + @return SharedCacheManager returns an empty response + on success and throws an exception on rejecting the request + @throws YarnException + @throws IOException]]> +
    +
    + + + The protocol between administrators and the SharedCacheManager +

    ]]> +
    +
    + +
    + + + + + + diff --git a/hadoop-yarn-project/hadoop-yarn/dev-support/jdiff/Apache_Hadoop_YARN_API_3.3.3.xml b/hadoop-yarn-project/hadoop-yarn/dev-support/jdiff/Apache_Hadoop_YARN_API_3.3.3.xml new file mode 100644 index 0000000000000..a994be99e7be4 --- /dev/null +++ b/hadoop-yarn-project/hadoop-yarn/dev-support/jdiff/Apache_Hadoop_YARN_API_3.3.3.xml @@ -0,0 +1,26407 @@ + + + + + + + + + + + + + + + + + + + + The interface used by clients to obtain a new {@link ApplicationId} for + submitting new applications.

    + +

    The ResourceManager responds with a new, monotonically + increasing, {@link ApplicationId} which is used by the client to submit + a new application.

    + +

    The ResourceManager also responds with details such + as maximum resource capabilities in the cluster as specified in + {@link GetNewApplicationResponse}.

    + + @param request request to get a new ApplicationId + @return response containing the new ApplicationId to be used + to submit an application + @throws YarnException + @throws IOException + @see #submitApplication(SubmitApplicationRequest)]]> +
    +
    + + + + + + The interface used by clients to submit a new application to the + ResourceManager.

    + +

    The client is required to provide details such as queue, + {@link Resource} required to run the ApplicationMaster, + the equivalent of {@link ContainerLaunchContext} for launching + the ApplicationMaster etc. via the + {@link SubmitApplicationRequest}.

    + +

    Currently the ResourceManager sends an immediate (empty) + {@link SubmitApplicationResponse} on accepting the submission and throws + an exception if it rejects the submission. However, this call needs to be + followed by {@link #getApplicationReport(GetApplicationReportRequest)} + to make sure that the application gets properly submitted - obtaining a + {@link SubmitApplicationResponse} from ResourceManager doesn't guarantee + that RM 'remembers' this application beyond failover or restart. If RM + failover or RM restart happens before ResourceManager saves the + application's state successfully, the subsequent + {@link #getApplicationReport(GetApplicationReportRequest)} will throw + a {@link ApplicationNotFoundException}. The Clients need to re-submit + the application with the same {@link ApplicationSubmissionContext} when + it encounters the {@link ApplicationNotFoundException} on the + {@link #getApplicationReport(GetApplicationReportRequest)} call.

    + +

    During the submission process, it checks whether the application + already exists. If the application exists, it will simply return + SubmitApplicationResponse

    + +

    In secure mode,the ResourceManager verifies access to + queues etc. before accepting the application submission.

    + + @param request request to submit a new application + @return (empty) response on accepting the submission + @throws YarnException + @throws IOException + @see #getNewApplication(GetNewApplicationRequest)]]> +
    +
    + + + + + + The interface used by clients to request the + ResourceManager to fail an application attempt.

    + +

    The client, via {@link FailApplicationAttemptRequest} provides the + {@link ApplicationAttemptId} of the attempt to be failed.

    + +

    In secure mode,the ResourceManager verifies access to the + application, queue etc. before failing the attempt.

    + +

    Currently, the ResourceManager returns an empty response + on success and throws an exception on rejecting the request.

    + + @param request request to fail an attempt + @return ResourceManager returns an empty response + on success and throws an exception on rejecting the request + @throws YarnException + @throws IOException + @see #getQueueUserAcls(GetQueueUserAclsInfoRequest)]]> +
    +
    + + + + + + The interface used by clients to request the + ResourceManager to abort submitted application.

    + +

    The client, via {@link KillApplicationRequest} provides the + {@link ApplicationId} of the application to be aborted.

    + +

    In secure mode,the ResourceManager verifies access to the + application, queue etc. before terminating the application.

    + +

    Currently, the ResourceManager returns an empty response + on success and throws an exception on rejecting the request.

    + + @param request request to abort a submitted application + @return ResourceManager returns an empty response + on success and throws an exception on rejecting the request + @throws YarnException + @throws IOException + @see #getQueueUserAcls(GetQueueUserAclsInfoRequest)]]> +
    +
    + + + + + + The interface used by clients to get metrics about the cluster from + the ResourceManager.

    + +

    The ResourceManager responds with a + {@link GetClusterMetricsResponse} which includes the + {@link YarnClusterMetrics} with details such as number of current + nodes in the cluster.

    + + @param request request for cluster metrics + @return cluster metrics + @throws YarnException + @throws IOException]]> +
    +
    + + + + + + The interface used by clients to get a report of all nodes + in the cluster from the ResourceManager.

    + +

    The ResourceManager responds with a + {@link GetClusterNodesResponse} which includes the + {@link NodeReport} for all the nodes in the cluster.

    + + @param request request for report on all nodes + @return report on all nodes + @throws YarnException + @throws IOException]]> +
    +
    + + + + + + The interface used by clients to get information about queues + from the ResourceManager.

    + +

    The client, via {@link GetQueueInfoRequest}, can ask for details such + as used/total resources, child queues, running applications etc.

    + +

    In secure mode,the ResourceManager verifies access before + providing the information.

    + + @param request request to get queue information + @return queue information + @throws YarnException + @throws IOException]]> +
    +
    + + + + + + The interface used by clients to get information about queue + acls for current user from the ResourceManager. +

    + +

    The ResourceManager responds with queue acls for all + existing queues.

    + + @param request request to get queue acls for current user + @return queue acls for current user + @throws YarnException + @throws IOException]]> +
    +
    + + + + + + + + + + + + + + The interface used by clients to obtain a new {@link ReservationId} for + submitting new reservations.

    + +

    The ResourceManager responds with a new, unique, + {@link ReservationId} which is used by the client to submit + a new reservation.

    + + @param request to get a new ReservationId + @return response containing the new ReservationId to be used + to submit a new reservation + @throws YarnException if the reservation system is not enabled. + @throws IOException on IO failures. + @see #submitReservation(ReservationSubmissionRequest)]]> +
    +
    + + + + + + + The interface used by clients to submit a new reservation to the + {@code ResourceManager}. +

    + +

    + The client packages all details of its request in a + {@link ReservationSubmissionRequest} object. This contains information + about the amount of capacity, temporal constraints, and concurrency needs. + Furthermore, the reservation might be composed of multiple stages, with + ordering dependencies among them. +

    + +

    + In order to respond, a new admission control component in the + {@code ResourceManager} performs an analysis of the resources that have + been committed over the period of time the user is requesting, verify that + the user requests can be fulfilled, and that it respect a sharing policy + (e.g., {@code CapacityOverTimePolicy}). Once it has positively determined + that the ReservationSubmissionRequest is satisfiable the + {@code ResourceManager} answers with a + {@link ReservationSubmissionResponse} that include a non-null + {@link ReservationId}. Upon failure to find a valid allocation the response + is an exception with the reason. + + On application submission the client can use this {@link ReservationId} to + obtain access to the reserved resources. +

    + +

    + The system guarantees that during the time-range specified by the user, the + reservationID will be corresponding to a valid reservation. The amount of + capacity dedicated to such queue can vary overtime, depending of the + allocation that has been determined. But it is guaranteed to satisfy all + the constraint expressed by the user in the + {@link ReservationSubmissionRequest}. +

    + + @param request the request to submit a new Reservation + @return response the {@link ReservationId} on accepting the submission + @throws YarnException if the request is invalid or reservation cannot be + created successfully + @throws IOException]]> +
    +
    + + + + + + + The interface used by clients to update an existing Reservation. This is + referred to as a re-negotiation process, in which a user that has + previously submitted a Reservation. +

    + +

    + The allocation is attempted by virtually substituting all previous + allocations related to this Reservation with new ones, that satisfy the new + {@link ReservationUpdateRequest}. Upon success the previous allocation is + substituted by the new one, and on failure (i.e., if the system cannot find + a valid allocation for the updated request), the previous allocation + remains valid. + + The {@link ReservationId} is not changed, and applications currently + running within this reservation will automatically receive the resources + based on the new allocation. +

    + + @param request to update an existing Reservation (the ReservationRequest + should refer to an existing valid {@link ReservationId}) + @return response empty on successfully updating the existing reservation + @throws YarnException if the request is invalid or reservation cannot be + updated successfully + @throws IOException]]> +
    +
    + + + + + + + The interface used by clients to remove an existing Reservation. + + Upon deletion of a reservation applications running with this reservation, + are automatically downgraded to normal jobs running without any dedicated + reservation. +

    + + @param request to remove an existing Reservation (the ReservationRequest + should refer to an existing valid {@link ReservationId}) + @return response empty on successfully deleting the existing reservation + @throws YarnException if the request is invalid or reservation cannot be + deleted successfully + @throws IOException]]> +
    +
    + + + + + + + The interface used by clients to get the list of reservations in a plan. + The reservationId will be used to search for reservations to list if it is + provided. Otherwise, it will select active reservations within the + startTime and endTime (inclusive). +

    + + @param request to list reservations in a plan. Contains fields to select + String queue, ReservationId reservationId, long startTime, + long endTime, and a bool includeReservationAllocations. + + queue: Required. Cannot be null or empty. Refers to the + reservable queue in the scheduler that was selected when + creating a reservation submission + {@link ReservationSubmissionRequest}. + + reservationId: Optional. If provided, other fields will + be ignored. + + startTime: Optional. If provided, only reservations that + end after the startTime will be selected. This defaults + to 0 if an invalid number is used. + + endTime: Optional. If provided, only reservations that + start on or before endTime will be selected. This defaults + to Long.MAX_VALUE if an invalid number is used. + + includeReservationAllocations: Optional. Flag that + determines whether the entire reservation allocations are + to be returned. Reservation allocations are subject to + change in the event of re-planning as described by + {@code ReservationDefinition}. + + @return response that contains information about reservations that are + being searched for. + @throws YarnException if the request is invalid + @throws IOException on IO failures]]> +
    +
    + + + + + + + The interface used by client to get node to labels mappings in existing cluster +

    + + @param request + @return node to labels mappings + @throws YarnException + @throws IOException]]> +
    +
    + + + + + + + The interface used by client to get labels to nodes mappings + in existing cluster +

    + + @param request + @return labels to nodes mappings + @throws YarnException + @throws IOException]]> +
    +
    + + + + + + + The interface used by client to get node labels in the cluster +

    + + @param request to get node labels collection of this cluster + @return node labels collection of this cluster + @throws YarnException + @throws IOException]]> +
    +
    + + + + + + + The interface used by client to set priority of an application. +

    + @param request to set priority of an application + @return an empty response + @throws YarnException + @throws IOException]]> +
    +
    + + + + + + The interface used by clients to request the + ResourceManager to signal a container. For example, + the client can send command OUTPUT_THREAD_DUMP to dump threads of the + container.

    + +

    The client, via {@link SignalContainerRequest} provides the + id of the container and the signal command.

    + +

    In secure mode,the ResourceManager verifies access to the + application before signaling the container. + The user needs to have MODIFY_APP permission.

    + +

    Currently, the ResourceManager returns an empty response + on success and throws an exception on rejecting the request.

    + + @param request request to signal a container + @return ResourceManager returns an empty response + on success and throws an exception on rejecting the request + @throws YarnException + @throws IOException]]> +
    +
    + + + + + + + The interface used by client to set ApplicationTimeouts of an application. + The UpdateApplicationTimeoutsRequest should have timeout value with + absolute time with ISO8601 format yyyy-MM-dd'T'HH:mm:ss.SSSZ. +

    + Note: If application timeout value is less than or equal to current + time then update application throws YarnException. + @param request to set ApplicationTimeouts of an application + @return a response with updated timeouts. + @throws YarnException if update request has empty values or application is + in completing states. + @throws IOException on IO failures]]> +
    +
    + + + + + + + The interface used by clients to get all the resource profiles that are + available on the ResourceManager. +

    + @param request request to get all the resource profiles + @return Response containing a map of the profile name to Resource + capabilities + @throws YARNFeatureNotEnabledException if resource-profile is disabled + @throws YarnException if any error happens inside YARN + @throws IOException in case of other errors]]> +
    +
    + + + + + + + The interface to get the details for a specific resource profile. +

    + @param request request to get the details of a resource profile + @return Response containing the details for a particular resource profile + @throws YARNFeatureNotEnabledException if resource-profile is disabled + @throws YarnException if any error happens inside YARN + @throws IOException in case of other errors]]> +
    +
    + + + + + + + The interface to get the details for a specific resource profile. +

    + @param request request to get the details of a resource profile + @return Response containing the details for a particular resource profile + @throws YarnException if any error happens inside YARN + @throws IOException in case of other errors]]> +
    +
    + + + + + + + The interface used by client to get attributes to nodes mappings + available in ResourceManager. +

    + + @param request request to get details of attributes to nodes mapping. + @return Response containing the details of attributes to nodes mappings. + @throws YarnException if any error happens inside YARN + @throws IOException incase of other errors]]> +
    +
    + + + + + + + The interface used by client to get node attributes available in + ResourceManager. +

    + + @param request request to get node attributes collection of this cluster. + @return Response containing node attributes collection. + @throws YarnException if any error happens inside YARN. + @throws IOException incase of other errors.]]> +
    +
    + + + + + + + The interface used by client to get node to attributes mappings. + in existing cluster. +

    + + @param request request to get nodes to attributes mapping. + @return nodes to attributes mappings. + @throws YarnException if any error happens inside YARN. + @throws IOException]]> +
    +
    + + The protocol between clients and the ResourceManager + to submit/abort jobs and to get information on applications, cluster metrics, + nodes, queues and ACLs.

    ]]> +
    +
    + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + The protocol between clients and the ApplicationHistoryServer to + get the information of completed applications etc. +

    ]]> +
    +
    + + + + + + + + + + The interface used by a new ApplicationMaster to register with + the ResourceManager. +

    + +

    + The ApplicationMaster needs to provide details such as RPC + Port, HTTP tracking url etc. as specified in + {@link RegisterApplicationMasterRequest}. +

    + +

    + The ResourceManager responds with critical details such as + maximum resource capabilities in the cluster as specified in + {@link RegisterApplicationMasterResponse}. +

    + +

    + Re-register is only allowed for Unmanaged Application Master + (UAM) HA, with + {@link org.apache.hadoop.yarn.api.records.ApplicationSubmissionContext#getKeepContainersAcrossApplicationAttempts()} + set to true. +

    + + @param request registration request + @return registration respose + @throws YarnException + @throws IOException + @throws InvalidApplicationMasterRequestException The exception is thrown + when an ApplicationMaster tries to register more then once. + @see RegisterApplicationMasterRequest + @see RegisterApplicationMasterResponse]]> +
    +
    + + + + + + The interface used by an ApplicationMaster to notify the + ResourceManager about its completion (success or failed).

    + +

    The ApplicationMaster has to provide details such as + final state, diagnostics (in case of failures) etc. as specified in + {@link FinishApplicationMasterRequest}.

    + +

    The ResourceManager responds with + {@link FinishApplicationMasterResponse}.

    + + @param request completion request + @return completion response + @throws YarnException + @throws IOException + @see FinishApplicationMasterRequest + @see FinishApplicationMasterResponse]]> +
    +
    + + + + + + + The main interface between an ApplicationMaster and the + ResourceManager. +

    + +

    + The ApplicationMaster uses this interface to provide a list of + {@link ResourceRequest} and returns unused {@link Container} allocated to + it via {@link AllocateRequest}. Optionally, the + ApplicationMaster can also blacklist resources which + it doesn't want to use. +

    + +

    + This also doubles up as a heartbeat to let the + ResourceManager know that the ApplicationMaster + is alive. Thus, applications should periodically make this call to be kept + alive. The frequency depends on + {@link YarnConfiguration#RM_AM_EXPIRY_INTERVAL_MS} which defaults to + {@link YarnConfiguration#DEFAULT_RM_AM_EXPIRY_INTERVAL_MS}. +

    + +

    + The ResourceManager responds with list of allocated + {@link Container}, status of completed containers and headroom information + for the application. +

    + +

    + The ApplicationMaster can use the available headroom + (resources) to decide how to utilized allocated resources and make informed + decisions about future resource requests. +

    + + @param request + allocation request + @return allocation response + @throws YarnException + @throws IOException + @throws InvalidApplicationMasterRequestException + This exception is thrown when an ApplicationMaster calls allocate + without registering first. + @throws InvalidResourceBlacklistRequestException + This exception is thrown when an application provides an invalid + specification for blacklist of resources. + @throws InvalidResourceRequestException + This exception is thrown when a {@link ResourceRequest} is out of + the range of the configured lower and upper limits on the + resources. + @see AllocateRequest + @see AllocateResponse]]> +
    +
    + + The protocol between a live instance of ApplicationMaster + and the ResourceManager.

    + +

    This is used by the ApplicationMaster to register/unregister + and to request and obtain resources in the cluster from the + ResourceManager.

    ]]> +
    +
    + + + + + + + + + + The interface used by clients to claim a resource with the + SharedCacheManager. The client uses a checksum to identify the + resource and an {@link ApplicationId} to identify which application will be + using the resource. +

    + +

    + The SharedCacheManager responds with whether or not the + resource exists in the cache. If the resource exists, a Path + to the resource in the shared cache is returned. If the resource does not + exist, the response is empty. +

    + + @param request request to claim a resource in the shared cache + @return response indicating if the resource is already in the cache + @throws YarnException + @throws IOException]]> +
    +
    + + + + + + + The interface used by clients to release a resource with the + SharedCacheManager. This method is called once an application + is no longer using a claimed resource in the shared cache. The client uses + a checksum to identify the resource and an {@link ApplicationId} to + identify which application is releasing the resource. +

    + +

    + Note: This method is an optimization and the client is not required to call + it for correctness. +

    + +

    + Currently the SharedCacheManager sends an empty response. +

    + + @param request request to release a resource in the shared cache + @return (empty) response on releasing the resource + @throws YarnException + @throws IOException]]> +
    +
    + + + The protocol between clients and the SharedCacheManager to claim + and release resources in the shared cache. +

    ]]> +
    +
    + + + + + + + + + + The ApplicationMaster provides a list of + {@link StartContainerRequest}s to a NodeManager to + start {@link Container}s allocated to it using this interface. +

    + +

    + The ApplicationMaster has to provide details such as allocated + resource capability, security tokens (if enabled), command to be executed + to start the container, environment for the process, necessary + binaries/jar/shared-objects etc. via the {@link ContainerLaunchContext} in + the {@link StartContainerRequest}. +

    + +

    + The NodeManager sends a response via + {@link StartContainersResponse} which includes a list of + {@link Container}s of successfully launched {@link Container}s, a + containerId-to-exception map for each failed {@link StartContainerRequest} in + which the exception indicates errors from per container and a + allServicesMetaData map between the names of auxiliary services and their + corresponding meta-data. Note: None-container-specific exceptions will + still be thrown by the API method itself. +

    +

    + The ApplicationMaster can use + {@link #getContainerStatuses(GetContainerStatusesRequest)} to get updated + statuses of the to-be-launched or launched containers. +

    + + @param request + request to start a list of containers + @return response including conatinerIds of all successfully launched + containers, a containerId-to-exception map for failed requests and + a allServicesMetaData map. + @throws YarnException + @throws IOException]]> +
    +
    + + + + + + + The ApplicationMaster requests a NodeManager to + stop a list of {@link Container}s allocated to it using this + interface. +

    + +

    + The ApplicationMaster sends a {@link StopContainersRequest} + which includes the {@link ContainerId}s of the containers to be stopped. +

    + +

    + The NodeManager sends a response via + {@link StopContainersResponse} which includes a list of {@link ContainerId} + s of successfully stopped containers, a containerId-to-exception map for + each failed request in which the exception indicates errors from per + container. Note: None-container-specific exceptions will still be thrown by + the API method itself. ApplicationMaster can use + {@link #getContainerStatuses(GetContainerStatusesRequest)} to get updated + statuses of the containers. +

    + + @param request + request to stop a list of containers + @return response which includes a list of containerIds of successfully + stopped containers, a containerId-to-exception map for failed + requests. + @throws YarnException + @throws IOException]]> +
    +
    + + + + + + + The API used by the ApplicationMaster to request for current + statuses of Containers from the NodeManager. +

    + +

    + The ApplicationMaster sends a + {@link GetContainerStatusesRequest} which includes the {@link ContainerId}s + of all containers whose statuses are needed. +

    + +

    + The NodeManager responds with + {@link GetContainerStatusesResponse} which includes a list of + {@link ContainerStatus} of the successfully queried containers and a + containerId-to-exception map for each failed request in which the exception + indicates errors from per container. Note: None-container-specific + exceptions will still be thrown by the API method itself. +

    + + @param request + request to get ContainerStatuses of containers with + the specified ContainerIds + @return response containing the list of ContainerStatus of the + successfully queried containers and a containerId-to-exception map + for failed requests. + + @throws YarnException + @throws IOException]]> +
    +
    + + + + + + + The API used by the ApplicationMaster to request for + resource increase of running containers on the NodeManager. +

    + + @param request + request to increase resource of a list of containers + @return response which includes a list of containerIds of containers + whose resource has been successfully increased and a + containerId-to-exception map for failed requests. + + @throws YarnException + @throws IOException]]> +
    +
    + + + + + + + The API used by the ApplicationMaster to request for + resource update of running containers on the NodeManager. +

    + + @param request + request to update resource of a list of containers + @return response which includes a list of containerIds of containers + whose resource has been successfully updated and a + containerId-to-exception map for failed requests. + + @throws YarnException Exception specific to YARN + @throws IOException IOException thrown from NodeManager]]> +
    +
    + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + The protocol between an ApplicationMaster and a + NodeManager to start/stop and increase resource of containers + and to get status of running containers.

    + +

    If security is enabled the NodeManager verifies that the + ApplicationMaster has truly been allocated the container + by the ResourceManager and also verifies all interactions such + as stopping the container or obtaining status information for the container. +

    ]]> +
    +
    + +
    + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + response id used to track duplicate responses. + @return response id]]> + + + + + + response id used to track duplicate responses. + @param id response id]]> + + + + + current progress of application. + @return current progress of application]]> + + + + + + current progress of application + @param progress current progress of application]]> + + + + + ResourceRequest to update the + ResourceManager about the application's resource requirements. + @return the list of ResourceRequest + @see ResourceRequest]]> + + + + + + ResourceRequest to update the + ResourceManager about the application's resource requirements. + @param resourceRequests list of ResourceRequest to update the + ResourceManager about the application's + resource requirements + @see ResourceRequest]]> + + + + + ContainerId of containers being + released by the ApplicationMaster. + @return list of ContainerId of containers being + released by the ApplicationMaster]]> + + + + + + ContainerId of containers being + released by the ApplicationMaster + @param releaseContainers list of ContainerId of + containers being released by the + ApplicationMaster]]> + + + + + ResourceBlacklistRequest being sent by the + ApplicationMaster. + @return the ResourceBlacklistRequest being sent by the + ApplicationMaster + @see ResourceBlacklistRequest]]> + + + + + + ResourceBlacklistRequest to inform the + ResourceManager about the blacklist additions and removals + per the ApplicationMaster. + + @param resourceBlacklistRequest the ResourceBlacklistRequest + to inform the ResourceManager about + the blacklist additions and removals + per the ApplicationMaster + @see ResourceBlacklistRequest]]> + + + + + ApplicationMaster. + @return list of {@link UpdateContainerRequest} + being sent by the + ApplicationMaster.]]> + + + + + + ResourceManager about the containers that need to be + updated. + @param updateRequests list of UpdateContainerRequest for + containers to be updated]]> + + + + + ApplicationMaster. + @return list of {@link SchedulingRequest} being sent by the + ApplicationMaster.]]> + + + + + + ResourceManager about the application's resource requirements + (potentially including allocation tags and placement constraints). + @param schedulingRequests list of {@link SchedulingRequest} to update + the ResourceManager about the application's resource + requirements.]]> + + + + + + + + + + + + + + + + + The core request sent by the ApplicationMaster to the + ResourceManager to obtain resources in the cluster.

    + +

    The request includes: +

      +
    • A response id to track duplicate responses.
    • +
    • Progress information.
    • +
    • + A list of {@link ResourceRequest} to inform the + ResourceManager about the application's + resource requirements. +
    • +
    • + A list of unused {@link Container} which are being returned. +
    • +
    • + A list of {@link UpdateContainerRequest} to inform + the ResourceManager about the change in + requirements of running containers. +
    • +
    + + @see ApplicationMasterProtocol#allocate(AllocateRequest)]]> +
    +
    + + + + + + + responseId of the request. + @see AllocateRequest#setResponseId(int) + @param responseId responseId of the request + @return {@link AllocateRequestBuilder}]]> + + + + + + progress of the request. + @see AllocateRequest#setProgress(float) + @param progress progress of the request + @return {@link AllocateRequestBuilder}]]> + + + + + + askList of the request. + @see AllocateRequest#setAskList(List) + @param askList askList of the request + @return {@link AllocateRequestBuilder}]]> + + + + + + releaseList of the request. + @see AllocateRequest#setReleaseList(List) + @param releaseList releaseList of the request + @return {@link AllocateRequestBuilder}]]> + + + + + + resourceBlacklistRequest of the request. + @see AllocateRequest#setResourceBlacklistRequest( + ResourceBlacklistRequest) + @param resourceBlacklistRequest + resourceBlacklistRequest of the request + @return {@link AllocateRequestBuilder}]]> + + + + + + updateRequests of the request. + @see AllocateRequest#setUpdateRequests(List) + @param updateRequests updateRequests of the request + @return {@link AllocateRequestBuilder}]]> + + + + + + schedulingRequests of the request. + @see AllocateRequest#setSchedulingRequests(List) + @param schedulingRequests SchedulingRequest of the request + @return {@link AllocateRequestBuilder}]]> + + + + + + trackingUrl of the request. + @see AllocateRequest#setTrackingUrl(String) + @param trackingUrl new tracking url + @return {@link AllocateRequestBuilder}]]> + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + ResourceManager needs the + ApplicationMaster to take some action then it will send an + AMCommand to the ApplicationMaster. See AMCommand + for details on commands and actions for them. + @return AMCommand if the ApplicationMaster should + take action, null otherwise + @see AMCommand]]> + + + + + last response id. + @return last response id]]> + + + + + newly allocated Container by the + ResourceManager. + @return list of newly allocated Container]]> + + + + + available headroom for resources in the cluster for the + application. + @return limit of available headroom for resources in the cluster for the + application]]> + + + + + completed containers' statuses. + @return the list of completed containers' statuses]]> + + + + + updated NodeReports. Updates could + be changes in health, availability etc of the nodes. + @return The delta of updated nodes since the last response]]> + + + + + + + + + + + The message is a snapshot of the resources the RM wants back from the AM. + While demand persists, the RM will repeat its request; applications should + not interpret each message as a request for additional + resources on top of previous messages. Resources requested consistently + over some duration may be forcibly killed by the RM. + + @return A specification of the resources to reclaim from this AM.]]> + + + + + + 1) AM is receiving first container on underlying NodeManager.
    + OR
    + 2) NMToken master key rolled over in ResourceManager and AM is getting new + container on the same underlying NodeManager. +

    + AM will receive one NMToken per NM irrespective of the number of containers + issued on same NM. AM is expected to store these tokens until issued a + new token for the same NM. + @return list of NMTokens required for communicating with NM]]> + + + + + ResourceManager. + @return list of newly increased containers]]> + + + + + + + + + + + + + + + + + + + + + + + + + + UpdateContainerError for + containers updates requests that were in error]]> + + + + + ResourceManager from previous application attempts which + have not been reported to the Application Master yet. +
    + These containers were recovered by the RM after the application master + had already registered. This may happen after RM restart when some NMs get + delayed in connecting to the RM and reporting the active containers. + Since they were not reported in the registration + response, they are reported in the response to the AM heartbeat. + + @return the list of running containers as viewed by + ResourceManager from previous application attempts.]]> +
    +
    + + + + + + + ResourceManager the + ApplicationMaster during resource negotiation. +

    + The response, includes: +

      +
    • Response ID to track duplicate responses.
    • +
    • + An AMCommand sent by ResourceManager to let the + {@code ApplicationMaster} take some actions (resync, shutdown etc.). +
    • +
    • A list of newly allocated {@link Container}.
    • +
    • A list of completed {@link Container}s' statuses.
    • +
    • + The available headroom for resources in the cluster for the + application. +
    • +
    • A list of nodes whose status has been updated.
    • +
    • The number of available nodes in a cluster.
    • +
    • A description of resources requested back by the cluster
    • +
    • AMRMToken, if AMRMToken has been rolled over
    • +
    • + A list of {@link Container} representing the containers + whose resource has been increased. +
    • +
    • + A list of {@link Container} representing the containers + whose resource has been decreased. +
    • +
    + + @see ApplicationMasterProtocol#allocate(AllocateRequest)]]> +
    +
    + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + Note: {@link NMToken} will be used for authenticating communication with + {@code NodeManager}. + @return the list of container tokens to be used for authorization during + container resource update. + @see NMToken]]> + + + + + + AllocateResponse.getUpdatedContainers. + The token contains the container id and resource capability required for + container resource update. + @param containersToUpdate the list of container tokens to be used + for container resource increase.]]> + + + + The request sent by Application Master to the + Node Manager to change the resource quota of a container.

    + + @see ContainerManagementProtocol#updateContainer(ContainerUpdateRequest)]]> +
    +
    + + + + + + + + + + + + + + + + + + + + + + The response sent by the NodeManager to the + ApplicationMaster when asked to update container resource. +

    + + @see ContainerManagementProtocol#updateContainer(ContainerUpdateRequest)]]> +
    +
    + + + + + + + + + + + ApplicationAttemptId of the attempt to be failed. + @return ApplicationAttemptId of the attempt.]]> + + + + + + + The request sent by the client to the ResourceManager + to fail an application attempt.

    + +

    The request includes the {@link ApplicationAttemptId} of the attempt to + be failed.

    + + @see ApplicationClientProtocol#failApplicationAttempt(FailApplicationAttemptRequest)]]> +
    +
    + + + + + + + The response sent by the ResourceManager to the client + failing an application attempt.

    + +

    Currently it's empty.

    + + @see ApplicationClientProtocol#failApplicationAttempt(FailApplicationAttemptRequest)]]> +
    +
    + + + + + + + + + + + + + final state of the ApplicationMaster. + @return final state of the ApplicationMaster]]> + + + + + + final state of the ApplicationMaster + @param finalState final state of the ApplicationMaster]]> + + + + + diagnostic information on application failure. + @return diagnostic information on application failure]]> + + + + + + diagnostic information on application failure. + @param diagnostics diagnostic information on application failure]]> + + + + + tracking URL for the ApplicationMaster. + This url if contains scheme then that will be used by resource manager + web application proxy otherwise it will default to http. + @return tracking URLfor the ApplicationMaster]]> + + + + + + final tracking URLfor the ApplicationMaster. + This is the web-URL to which ResourceManager or web-application proxy will + redirect client/users once the application is finished and the + ApplicationMaster is gone. +

    + If the passed url has a scheme then that will be used by the + ResourceManager and web-application proxy, otherwise the scheme will + default to http. +

    +

    + Empty, null, "N/A" strings are all valid besides a real URL. In case an url + isn't explicitly passed, it defaults to "N/A" on the ResourceManager. +

    + + @param url + tracking URLfor the ApplicationMaster]]> + + + + + The final request includes details such: +

      +
    • Final state of the {@code ApplicationMaster}
    • +
    • + Diagnostic information in case of failure of the + {@code ApplicationMaster} +
    • +
    • Tracking URL
    • +
    + + @see ApplicationMasterProtocol#finishApplicationMaster(FinishApplicationMasterRequest)]]> +
    +
    + + + + + + + + + + + + ResourceManager to a + ApplicationMaster on it's completion. +

    + The response, includes: +

      +
    • A flag which indicates that the application has successfully unregistered + with the RM and the application can safely stop.
    • +
    +

    + Note: The flag indicates whether the application has successfully + unregistered and is safe to stop. The application may stop after the flag is + true. If the application stops before the flag is true then the RM may retry + the application. + + @see ApplicationMasterProtocol#finishApplicationMaster(FinishApplicationMasterRequest)]]> + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + ApplicationAttemptId of an application attempt. + + @return ApplicationAttemptId of an application attempt]]> + + + + + + ApplicationAttemptId of an application attempt + + @param applicationAttemptId + ApplicationAttemptId of an application attempt]]> + + + + + The request sent by a client to the ResourceManager to get an + {@link ApplicationAttemptReport} for an application attempt. +

    + +

    + The request should include the {@link ApplicationAttemptId} of the + application attempt. +

    + + @see ApplicationAttemptReport + @see ApplicationHistoryProtocol#getApplicationAttemptReport(GetApplicationAttemptReportRequest)]]> +
    +
    + + + + + + + + + + + ApplicationAttemptReport for the application attempt. + + @return ApplicationAttemptReport for the application attempt]]> + + + + + + ApplicationAttemptReport for the application attempt. + + @param applicationAttemptReport + ApplicationAttemptReport for the application attempt]]> + + + + + The response sent by the ResourceManager to a client requesting + an application attempt report. +

    + +

    + The response includes an {@link ApplicationAttemptReport} which has the + details about the particular application attempt +

    + + @see ApplicationAttemptReport + @see ApplicationHistoryProtocol#getApplicationAttemptReport(GetApplicationAttemptReportRequest)]]> +
    +
    + + + + + + + + + + + ApplicationId of an application + + @return ApplicationId of an application]]> + + + + + + ApplicationId of an application + + @param applicationId + ApplicationId of an application]]> + + + + + The request from clients to get a list of application attempt reports of an + application from the ResourceManager. +

    + + @see ApplicationHistoryProtocol#getApplicationAttempts(GetApplicationAttemptsRequest)]]> +
    +
    + + + + + + + + + + + ApplicationReport of an application. + + @return a list of ApplicationReport of an application]]> + + + + + + ApplicationReport of an application. + + @param applicationAttempts + a list of ApplicationReport of an application]]> + + + + + The response sent by the ResourceManager to a client requesting + a list of {@link ApplicationAttemptReport} for application attempts. +

    + +

    + The ApplicationAttemptReport for each application includes the + details of an application attempt. +

    + + @see ApplicationAttemptReport + @see ApplicationHistoryProtocol#getApplicationAttempts(GetApplicationAttemptsRequest)]]> +
    +
    + + + + + + + + + + + ApplicationId of the application. + @return ApplicationId of the application]]> + + + + + + ApplicationId of the application + @param applicationId ApplicationId of the application]]> + + + + The request sent by a client to the ResourceManager to + get an {@link ApplicationReport} for an application.

    + +

    The request should include the {@link ApplicationId} of the + application.

    + + @see ApplicationClientProtocol#getApplicationReport(GetApplicationReportRequest) + @see ApplicationReport]]> +
    +
    + + + + + + + + ApplicationReport for the application. + @return ApplicationReport for the application]]> + + + + The response sent by the ResourceManager to a client + requesting an application report.

    + +

    The response includes an {@link ApplicationReport} which has details such + as user, queue, name, host on which the ApplicationMaster is + running, RPC port, tracking URL, diagnostics, start time etc.

    + + @see ApplicationClientProtocol#getApplicationReport(GetApplicationReportRequest)]]> +
    +
    + + + + + + + + + + + + + + + + + + + + The request from clients to get a report of Applications matching the + giving application types in the cluster from the + ResourceManager. +

    + + @see ApplicationClientProtocol#getApplications(GetApplicationsRequest) + +

    Setting any of the parameters to null, would just disable that + filter

    + + @param scope {@link ApplicationsRequestScope} to filter by + @param users list of users to filter by + @param queues list of scheduler queues to filter by + @param applicationTypes types of applications + @param applicationTags application tags to filter by + @param applicationStates application states to filter by + @param startRange range of application start times to filter by + @param finishRange range of application finish times to filter by + @param limit number of applications to limit to + @return {@link GetApplicationsRequest} to be used with + {@link ApplicationClientProtocol#getApplications(GetApplicationsRequest)}]]> +
    +
    + + + + + The request from clients to get a report of Applications matching the + giving application types in the cluster from the + ResourceManager. +

    + + @param scope {@link ApplicationsRequestScope} to filter by + @see ApplicationClientProtocol#getApplications(GetApplicationsRequest) + @return a report of Applications in {@link GetApplicationsRequest}]]> +
    +
    + + + + + The request from clients to get a report of Applications matching the + giving application types in the cluster from the + ResourceManager. +

    + + + @see ApplicationClientProtocol#getApplications(GetApplicationsRequest) + @return a report of Applications in {@link GetApplicationsRequest}]]> +
    +
    + + + + + The request from clients to get a report of Applications matching the + giving application states in the cluster from the + ResourceManager. +

    + + + @see ApplicationClientProtocol#getApplications(GetApplicationsRequest) + @return a report of Applications in {@link GetApplicationsRequest}]]> +
    +
    + + + + + + The request from clients to get a report of Applications matching the + giving and application types and application types in the cluster from the + ResourceManager. +

    + + + @see ApplicationClientProtocol#getApplications(GetApplicationsRequest) + @return a report of Applications in GetApplicationsRequest]]> +
    +
    + + + + + + + + + + + + The request from clients to get a report of Applications + in the cluster from the ResourceManager.

    + + @see ApplicationClientProtocol#getApplications(GetApplicationsRequest)]]> +
    +
    + + + + + + + + ApplicationReport for applications. + @return ApplicationReport for applications]]> + + + + The response sent by the ResourceManager to a client + requesting an {@link ApplicationReport} for applications.

    + +

    The ApplicationReport for each application includes details + such as user, queue, name, host on which the ApplicationMaster + is running, RPC port, tracking URL, diagnostics, start time etc.

    + + @see ApplicationReport + @see ApplicationClientProtocol#getApplications(GetApplicationsRequest)]]> +
    +
    + + + + + + + + + + + + + + + + + + + + + + + + The request from clients to get node to attribute value mapping for all or + given set of Node AttributeKey's in the cluster from the + ResourceManager. +

    + + @see ApplicationClientProtocol#getAttributesToNodes + (GetAttributesToNodesRequest)]]> +
    +
    + + + + + + + + + + + + + + + + + + + The response sent by the ResourceManager to a client requesting + node to attribute value mapping for all or given set of Node AttributeKey's. +

    + + @see ApplicationClientProtocol#getAttributesToNodes + (GetAttributesToNodesRequest)]]> +
    +
    + + + + + + + + + The request sent by clients to get cluster metrics from the + ResourceManager.

    + +

    Currently, this is empty.

    + + @see ApplicationClientProtocol#getClusterMetrics(GetClusterMetricsRequest)]]> +
    +
    + + + + + + + + YarnClusterMetrics for the cluster. + @return YarnClusterMetrics for the cluster]]> + + + + ResourceManager to a client + requesting cluster metrics. + + @see YarnClusterMetrics + @see ApplicationClientProtocol#getClusterMetrics(GetClusterMetricsRequest)]]> + + + + + + + + + + + + + + + The request from clients to get node attributes in the cluster from the + ResourceManager. +

    + + @see ApplicationClientProtocol#getClusterNodeAttributes + (GetClusterNodeAttributesRequest)]]> +
    +
    + + + + + + + + + + + + + + + + + + + + + + + + + The response sent by the ResourceManager to a client requesting + a node attributes in cluster. +

    + + @see ApplicationClientProtocol#getClusterNodeAttributes + (GetClusterNodeAttributesRequest)]]> +
    +
    + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + The request from clients to get a report of all nodes + in the cluster from the ResourceManager.

    + + The request will ask for all nodes in the given {@link NodeState}s. + + @see ApplicationClientProtocol#getClusterNodes(GetClusterNodesRequest)]]> +
    +
    + + + + + + + + NodeReport for all nodes in the cluster. + @return NodeReport for all nodes in the cluster]]> + + + + The response sent by the ResourceManager to a client + requesting a {@link NodeReport} for all nodes.

    + +

    The NodeReport contains per-node information such as + available resources, number of containers, tracking url, rack name, health + status etc. + + @see NodeReport + @see ApplicationClientProtocol#getClusterNodes(GetClusterNodesRequest)]]> + + + + + + + + + + + + + ContainerId of the Container. + + @return ContainerId of the Container]]> + + + + + + ContainerId of the container + + @param containerId + ContainerId of the container]]> + + + + + The request sent by a client to the ResourceManager to get an + {@link ContainerReport} for a container. +

    ]]> +
    +
    + + + + + + + + + + + ContainerReport for the container. + + @return ContainerReport for the container]]> + + + + + + + + The response sent by the ResourceManager to a client requesting + a container report. +

    + +

    + The response includes a {@link ContainerReport} which has details of a + container. +

    ]]> +
    +
    + + + + + + + + + + + ApplicationAttemptId of an application attempt. + + @return ApplicationAttemptId of an application attempt]]> + + + + + + ApplicationAttemptId of an application attempt + + @param applicationAttemptId + ApplicationAttemptId of an application attempt]]> + + + + + The request from clients to get a list of container reports, which belong to + an application attempt from the ResourceManager. +

    + + @see ApplicationHistoryProtocol#getContainers(GetContainersRequest)]]> +
    +
    + + + + + + + + + + + ContainerReport for all the containers of an + application attempt. + + @return a list of ContainerReport for all the containers of an + application attempt]]> + + + + + + ContainerReport for all the containers of an + application attempt. + + @param containers + a list of ContainerReport for all the containers of + an application attempt]]> + + + + + The response sent by the ResourceManager to a client requesting + a list of {@link ContainerReport} for containers. +

    + +

    + The ContainerReport for each container includes the container + details. +

    + + @see ContainerReport + @see ApplicationHistoryProtocol#getContainers(GetContainersRequest)]]> +
    +
    + + + + + + + + + + + ContainerIds of containers for which to obtain + the ContainerStatus. + + @return the list of ContainerIds of containers for which to + obtain the ContainerStatus.]]> + + + + + + ContainerIds of containers for which to obtain + the ContainerStatus + + @param containerIds + a list of ContainerIds of containers for which to + obtain the ContainerStatus]]> + + + + ApplicationMaster to the + NodeManager to get {@link ContainerStatus} of requested + containers. + + @see ContainerManagementProtocol#getContainerStatuses(GetContainerStatusesRequest)]]> + + + + + + + + + + ContainerStatuses of the requested containers. + + @return ContainerStatuses of the requested containers.]]> + + + + + + + + + NodeManager to the + ApplicationMaster when asked to obtain the + ContainerStatus of requested containers. + + @see ContainerManagementProtocol#getContainerStatuses(GetContainerStatusesRequest)]]> + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + The request sent by clients to get a new {@link ApplicationId} for + submitting an application.

    + +

    Currently, this is empty.

    + + @see ApplicationClientProtocol#getNewApplication(GetNewApplicationRequest)]]> +
    +
    + + + + + + + + new ApplicationId allocated by the + ResourceManager. + @return new ApplicationId allocated by the + ResourceManager]]> + + + + + ResourceManager in the cluster. + @return maximum capability of allocated resources in the cluster]]> + + + + The response sent by the ResourceManager to the client for + a request to get a new {@link ApplicationId} for submitting applications.

    + +

    Clients can submit an application with the returned + {@link ApplicationId}.

    + + @see ApplicationClientProtocol#getNewApplication(GetNewApplicationRequest)]]> +
    +
    + + + + + + + + + The request sent by clients to get a new {@code ReservationId} for + submitting an reservation.

    + + {@code ApplicationClientProtocol#getNewReservation(GetNewReservationRequest)}]]> +
    +
    + + + + + + + + + + + + The response sent by the ResourceManager to the client for + a request to get a new {@link ReservationId} for submitting reservations.

    + +

    Clients can submit an reservation with the returned + {@link ReservationId}.

    + + {@code ApplicationClientProtocol#getNewReservation(GetNewReservationRequest)}]]> +
    +
    + + + + + + + + + + + + + + + + + + + + + + The request from clients to get nodes to attributes mapping + in the cluster from the ResourceManager. +

    + + @see ApplicationClientProtocol#getNodesToAttributes + (GetNodesToAttributesRequest)]]> +
    +
    + + + + + + + + + + + + + + + + + + + The response sent by the ResourceManager to a client requesting + nodes to attributes mapping. +

    + + @see ApplicationClientProtocol#getNodesToAttributes + (GetNodesToAttributesRequest)]]> +
    +
    + + + + + + + + + + + + + + queue name for which to get queue information. + @return queue name for which to get queue information]]> + + + + + + queue name for which to get queue information + @param queueName queue name for which to get queue information]]> + + + + + active applications required? + @return true if applications' information is to be included, + else false]]> + + + + + + active applications? + @param includeApplications fetch information about active + applications?]]> + + + + + child queues required? + @return true if information about child queues is required, + else false]]> + + + + + + child queues? + @param includeChildQueues fetch information about child queues?]]> + + + + + child queue hierarchy required? + @return true if information about entire hierarchy is + required, false otherwise]]> + + + + + + child queue hierarchy? + @param recursive fetch information on the entire child queue + hierarchy?]]> + + + + The request sent by clients to get queue information + from the ResourceManager.

    + + @see ApplicationClientProtocol#getQueueInfo(GetQueueInfoRequest)]]> +
    +
    + + + + + + + + QueueInfo for the specified queue. + @return QueueInfo for the specified queue]]> + + + + + The response includes a {@link QueueInfo} which has details such as + queue name, used/total capacities, running applications, child queues etc. + + @see QueueInfo + @see ApplicationClientProtocol#getQueueInfo(GetQueueInfoRequest)]]> + + + + + + + + + + + The request sent by clients to the ResourceManager to + get queue acls for the current user.

    + +

    Currently, this is empty.

    + + @see ApplicationClientProtocol#getQueueUserAcls(GetQueueUserAclsInfoRequest)]]> +
    +
    + + + + + + + + QueueUserACLInfo per queue for the user. + @return QueueUserACLInfo per queue for the user]]> + + + + The response sent by the ResourceManager to clients + seeking queue acls for the user.

    + +

    The response contains a list of {@link QueueUserACLInfo} which + provides information about {@link QueueACL} per queue.

    + + @see QueueACL + @see QueueUserACLInfo + @see ApplicationClientProtocol#getQueueUserAcls(GetQueueUserAclsInfoRequest)]]> +
    +
    + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + Note: {@link NMToken} will be used for authenticating communication with + {@code NodeManager}. + @return the list of container tokens to be used for authorization during + container resource increase. + @see NMToken]]> + + + + + + AllocateResponse.getIncreasedContainers. + The token contains the container id and resource capability required for + container resource increase. + @param containersToIncrease the list of container tokens to be used + for container resource increase.]]> + + + + The request sent by Application Master to the + Node Manager to change the resource quota of a container.

    + + @see ContainerManagementProtocol#increaseContainersResource(IncreaseContainersResourceRequest)]]> +
    +
    + + + + + + + + + + + + + + + + + + The response sent by the NodeManager to the + ApplicationMaster when asked to increase container resource. +

    + + @see ContainerManagementProtocol#increaseContainersResource(IncreaseContainersResourceRequest)]]> +
    +
    + + + + + + + + + + + ApplicationId of the application to be aborted. + @return ApplicationId of the application to be aborted]]> + + + + + + + + diagnostics to which the application is being killed. + @return diagnostics to which the application is being killed]]> + + + + + + diagnostics to which the application is being killed. + @param diagnostics diagnostics to which the application is being + killed]]> + + + + The request sent by the client to the ResourceManager + to abort a submitted application.

    + +

    The request includes the {@link ApplicationId} of the application to be + aborted.

    + + @see ApplicationClientProtocol#forceKillApplication(KillApplicationRequest)]]> +
    +
    + + + + + + + + + + + + ResourceManager to the client aborting + a submitted application. +

    + The response, includes: +

      +
    • + A flag which indicates that the process of killing the application is + completed or not. +
    • +
    + Note: user is recommended to wait until this flag becomes true, otherwise if + the ResourceManager crashes before the process of killing the + application is completed, the ResourceManager may retry this + application on recovery. + + @see ApplicationClientProtocol#forceKillApplication(KillApplicationRequest)]]> +
    +
    + + + + + + + + + + + + ApplicationId of the application to be moved. + @return ApplicationId of the application to be moved]]> + + + + + + ApplicationId of the application to be moved. + @param appId ApplicationId of the application to be moved]]> + + + + + + + + + + + + + + + The request sent by the client to the ResourceManager + to move a submitted application to a different queue.

    + +

    The request includes the {@link ApplicationId} of the application to be + moved and the queue to place it in.

    + + @see ApplicationClientProtocol#moveApplicationAcrossQueues(MoveApplicationAcrossQueuesRequest)]]> +
    +
    + + + + + + + + The response sent by the ResourceManager to the client moving + a submitted application to a different queue. +

    +

    + A response without exception means that the move has completed successfully. +

    + + @see ApplicationClientProtocol#moveApplicationAcrossQueues(MoveApplicationAcrossQueuesRequest)]]> +
    +
    + + + + + + + + + + + RegisterApplicationMasterRequest. + If port, trackingUrl is not used, use the following default value: +
      +
    • port: -1
    • +
    • trackingUrl: null
    • +
    + The port is allowed to be any integer larger than or equal to -1. + @return the new instance of RegisterApplicationMasterRequest]]> +
    +
    + + + host on which the ApplicationMaster is + running. + @return host on which the ApplicationMaster is running]]> + + + + + + host on which the ApplicationMaster is + running. + @param host host on which the ApplicationMaster + is running]]> + + + + + RPC port on which the {@code ApplicationMaster} is + responding. + @return the RPC port on which the {@code ApplicationMaster} + is responding]]> + + + + + + RPC port on which the {@code ApplicationMaster} is + responding. + @param port RPC port on which the {@code ApplicationMaster} + is responding]]> + + + + + tracking URL for the ApplicationMaster. + This url if contains scheme then that will be used by resource manager + web application proxy otherwise it will default to http. + @return tracking URL for the ApplicationMaster]]> + + + + + + tracking URLfor the ApplicationMaster while + it is running. This is the web-URL to which ResourceManager or + web-application proxy will redirect client/users while the application and + the ApplicationMaster are still running. +

    + If the passed url has a scheme then that will be used by the + ResourceManager and web-application proxy, otherwise the scheme will + default to http. +

    +

    + Empty, null, "N/A" strings are all valid besides a real URL. In case an url + isn't explicitly passed, it defaults to "N/A" on the ResourceManager. +

    + + @param trackingUrl + tracking URLfor the ApplicationMaster]]> + + + + + PlacementConstraint associated with the tags, i.e., each + {@link org.apache.hadoop.yarn.api.records.SchedulingRequest} that has those + tags will be placed taking into account the corresponding constraint. + + @return A map of Placement Constraints.]]> + + + + + + PlacementConstraint associated with the tags. + For example: + Map < + <hb_regionserver> -> node_anti_affinity, + <hb_regionserver, hb_master> -> rack_affinity, + ... + > + @param placementConstraints Placement Constraint Mapping.]]> + + + + + The registration includes details such as: +

      +
    • Hostname on which the AM is running.
    • +
    • RPC Port
    • +
    • Tracking URL
    • +
    + + @see ApplicationMasterProtocol#registerApplicationMaster(RegisterApplicationMasterRequest)]]> +
    +
    + + + + + + + + ResourceManager in the cluster. + @return maximum capability of allocated resources in the cluster]]> + + + + + ApplicationACLs for the application. + @return all the ApplicationACLs]]> + + + + + Get ClientToAMToken master key.

    +

    The ClientToAMToken master key is sent to ApplicationMaster + by ResourceManager via {@link RegisterApplicationMasterResponse} + , used to verify corresponding ClientToAMToken.

    + @return ClientToAMToken master key]]> +
    +
    + + + + + + + + + Get the queue that the application was placed in.

    + @return the queue that the application was placed in.]]> + + + + + + Set the queue that the application was placed in.

    ]]> + + + + + + Get the list of running containers as viewed by + ResourceManager from previous application attempts. +

    + + @return the list of running containers as viewed by + ResourceManager from previous application attempts + @see RegisterApplicationMasterResponse#getNMTokensFromPreviousAttempts()]]> +
    +
    + + + + + + + + + + + + + + + + + + + + + + + The response contains critical details such as: +
      +
    • Maximum capability for allocated resources in the cluster.
    • +
    • {@code ApplicationACL}s for the application.
    • +
    • ClientToAMToken master key.
    • +
    + + @see ApplicationMasterProtocol#registerApplicationMaster(RegisterApplicationMasterRequest)]]> +
    +
    + + + + + + + + + + + + + + + + ContainerId of the container to re-initialize. + + @return ContainerId of the container to re-initialize.]]> + + + + + ContainerLaunchContext to re-initialize the container + with. + + @return ContainerLaunchContext of to re-initialize the + container with.]]> + + + + + + + + + + + + + + + + + + + + + + + + + + + + ApplicationId of the resource to be released. + + @return ApplicationId]]> + + + + + + ApplicationId of the resource to be released. + + @param id ApplicationId]]> + + + + + key of the resource to be released. + + @return key]]> + + + + + + key of the resource to be released. + + @param key unique identifier for the resource]]> + + + + The request from clients to release a resource in the shared cache.

    ]]> +
    +
    + + + + + + + + The response to clients from the SharedCacheManager when + releasing a resource in the shared cache. +

    + +

    + Currently, this is empty. +

    ]]> +
    +
    + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + The response sent by the ResourceManager to a client on + reservation submission.

    + +

    Currently, this is empty.

    + + {@code ApplicationClientProtocol#submitReservation( + ReservationSubmissionRequest)}]]> +
    +
    + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + ContainerId of the container to localize resources. + + @return ContainerId of the container to localize resources.]]> + + + + + LocalResource required by the container. + + @return all LocalResource required by the container]]> + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + ContainerId of the container to signal. + @return ContainerId of the container to signal.]]> + + + + + + ContainerId of the container to signal.]]> + + + + + SignalContainerCommand of the signal request. + @return SignalContainerCommand of the signal request.]]> + + + + + + SignalContainerCommand of the signal request.]]> + + + + The request sent by the client to the ResourceManager + or by the ApplicationMaster to the NodeManager + to signal a container. + @see SignalContainerCommand

    ]]> +
    +
    + + + + + + + The response sent by the ResourceManager to the client + signalling a container.

    + +

    Currently it's empty.

    + + @see ApplicationClientProtocol#signalToContainer(SignalContainerRequest)]]> +
    +
    + + + + + + + + + + + + ContainerLaunchContext for the container to be started + by the NodeManager. + + @return ContainerLaunchContext for the container to be started + by the NodeManager]]> + + + + + + ContainerLaunchContext for the container to be started + by the NodeManager + @param context ContainerLaunchContext for the container to be + started by the NodeManager]]> + + + + + + Note: {@link NMToken} will be used for authenticating communication with + {@code NodeManager}. + @return the container token to be used for authorization during starting + container. + @see NMToken + @see ContainerManagementProtocol#startContainers(StartContainersRequest)]]> + + + + + + + The request sent by the ApplicationMaster to the + NodeManager to start a container.

    + +

    The ApplicationMaster has to provide details such as + allocated resource capability, security tokens (if enabled), command + to be executed to start the container, environment for the process, + necessary binaries/jar/shared-objects etc. via the + {@link ContainerLaunchContext}.

    + + @see ContainerManagementProtocol#startContainers(StartContainersRequest)]]> +
    +
    + + + + + + + + + + + + + + + + + + + + + + The request which contains a list of {@link StartContainerRequest} sent by + the ApplicationMaster to the NodeManager to + start containers. +

    + +

    + In each {@link StartContainerRequest}, the ApplicationMaster has + to provide details such as allocated resource capability, security tokens (if + enabled), command to be executed to start the container, environment for the + process, necessary binaries/jar/shared-objects etc. via the + {@link ContainerLaunchContext}. +

    + + @see ContainerManagementProtocol#startContainers(StartContainersRequest)]]> +
    +
    + + + + + + + + ContainerId s of the containers that are + started successfully. + + @return the list of ContainerId s of the containers that are + started successfully. + @see ContainerManagementProtocol#startContainers(StartContainersRequest)]]> + + + + + + + + + + + Get the meta-data from all auxiliary services running on the + NodeManager. +

    +

    + The meta-data is returned as a Map between the auxiliary service names and + their corresponding per service meta-data as an opaque blob + ByteBuffer +

    + +

    + To be able to interpret the per-service meta-data, you should consult the + documentation for the Auxiliary-service configured on the NodeManager +

    + + @return a Map between the names of auxiliary services and their + corresponding meta-data]]> +
    +
    + + + The response sent by the NodeManager to the + ApplicationMaster when asked to start an allocated + container. +

    + + @see ContainerManagementProtocol#startContainers(StartContainersRequest)]]> +
    +
    + + + + + + + + + + + ContainerIds of the containers to be stopped. + @return ContainerIds of containers to be stopped]]> + + + + + + ContainerIds of the containers to be stopped. + @param containerIds ContainerIds of the containers to be stopped]]> + + + + The request sent by the ApplicationMaster to the + NodeManager to stop containers.

    + + @see ContainerManagementProtocol#stopContainers(StopContainersRequest)]]> +
    +
    + + + + + + + + + + + + + + + + + + The response sent by the NodeManager to the + ApplicationMaster when asked to stop allocated + containers. +

    + + @see ContainerManagementProtocol#stopContainers(StopContainersRequest)]]> +
    +
    + + + + + + + + + + + ApplicationSubmissionContext for the application. + @return ApplicationSubmissionContext for the application]]> + + + + + + ApplicationSubmissionContext for the application. + @param context ApplicationSubmissionContext for the + application]]> + + + + The request sent by a client to submit an application to the + ResourceManager.

    + +

    The request, via {@link ApplicationSubmissionContext}, contains + details such as queue, {@link Resource} required to run the + ApplicationMaster, the equivalent of + {@link ContainerLaunchContext} for launching the + ApplicationMaster etc. + + @see ApplicationClientProtocol#submitApplication(SubmitApplicationRequest)]]> + + + + + + + + + The response sent by the ResourceManager to a client on + application submission.

    + +

    Currently, this is empty.

    + + @see ApplicationClientProtocol#submitApplication(SubmitApplicationRequest)]]> +
    +
    + + + + + + + + + + + + ApplicationId of the application. + + @return ApplicationId of the application]]> + + + + + + ApplicationId of the application. + + @param applicationId ApplicationId of the application]]> + + + + + Priority of the application to be set. + + @return Priority of the application to be set.]]> + + + + + + Priority of the application. + + @param priority Priority of the application]]> + + + + + The request sent by the client to the ResourceManager to set or + update the application priority. +

    +

    + The request includes the {@link ApplicationId} of the application and + {@link Priority} to be set for an application +

    + + @see ApplicationClientProtocol#updateApplicationPriority(UpdateApplicationPriorityRequest)]]> +
    +
    + + + + + + + + + + + Priority of the application to be set. + @return Updated Priority of the application.]]> + + + + + + Priority of the application. + + @param priority Priority of the application]]> + + + + + The response sent by the ResourceManager to the client on update + the application priority. +

    +

    + A response without exception means that the move has completed successfully. +

    + + @see ApplicationClientProtocol#updateApplicationPriority(UpdateApplicationPriorityRequest)]]> +
    +
    + + + + + + + + + + + + ApplicationId of the application. + @return ApplicationId of the application]]> + + + + + + ApplicationId of the application. + @param applicationId ApplicationId of the application]]> + + + + + ApplicationTimeouts of the application. Timeout value is + in ISO8601 standard with format yyyy-MM-dd'T'HH:mm:ss.SSSZ. + @return all ApplicationTimeouts of the application.]]> + + + + + + ApplicationTimeouts for the application. Timeout value + is absolute. Timeout value should meet ISO8601 format. Support ISO8601 + format is yyyy-MM-dd'T'HH:mm:ss.SSSZ. All pre-existing Map entries + are cleared before adding the new Map. + @param applicationTimeouts ApplicationTimeoutss for the + application]]> + + + + + The request sent by the client to the ResourceManager to set or + update the application timeout. +

    +

    + The request includes the {@link ApplicationId} of the application and timeout + to be set for an application +

    ]]> +
    +
    + + + + + + + + + + ApplicationTimeouts of the application. Timeout value is + in ISO8601 standard with format yyyy-MM-dd'T'HH:mm:ss.SSSZ. + @return all ApplicationTimeouts of the application.]]> + + + + + + ApplicationTimeouts for the application. Timeout value + is absolute. Timeout value should meet ISO8601 format. Support ISO8601 + format is yyyy-MM-dd'T'HH:mm:ss.SSSZ. All pre-existing Map entries + are cleared before adding the new Map. + @param applicationTimeouts ApplicationTimeoutss for the + application]]> + + + + + The response sent by the ResourceManager to the client on update + application timeout. +

    +

    + A response without exception means that the update has completed + successfully. +

    ]]> +
    +
    + + + + + + + + ApplicationId of the resource to be used. + + @return ApplicationId]]> + + + + + + ApplicationId of the resource to be used. + + @param id ApplicationId]]> + + + + + key of the resource to be used. + + @return key]]> + + + + + + key of the resource to be used. + + @param key unique identifier for the resource]]> + + + + + The request from clients to the SharedCacheManager that claims a + resource in the shared cache. +

    ]]> +
    +
    + + + + + + + + Path corresponding to the requested resource in the + shared cache. + + @return String A Path if the resource exists in the shared + cache, null otherwise]]> + + + + + + Path corresponding to a resource in the shared cache. + + @param p A Path corresponding to a resource in the shared + cache]]> + + + + + The response from the SharedCacheManager to the client that indicates whether + a requested resource exists in the cache. +

    ]]> +
    +
    + +
    + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + ApplicationId of the ApplicationAttempId. + @return ApplicationId of the ApplicationAttempId]]> + + + + + attempt id of the Application. + @return attempt id of the Application]]> + + + + + + + + + + + + + + + + + + + + + ApplicationAttemptId denotes the particular attempt + of an ApplicationMaster for a given {@link ApplicationId}.

    + +

    Multiple attempts might be needed to run an application to completion due + to temporal failures of the ApplicationMaster such as hardware + failures, connectivity issues etc. on the node on which it was scheduled.

    ]]> +
    +
    + + + + + + + + + + + + + + + + + + YarnApplicationAttemptState of the application attempt. + + @return YarnApplicationAttemptState of the application attempt]]> + + + + + RPC port of this attempt ApplicationMaster. + + @return RPC port of this attempt ApplicationMaster]]> + + + + + host on which this attempt of + ApplicationMaster is running. + + @return host on which this attempt of + ApplicationMaster is running]]> + + + + + diagnositic information of the application attempt in case + of errors. + + @return diagnositic information of the application attempt in case + of errors]]> + + + + + tracking url for the application attempt. + + @return tracking url for the application attempt]]> + + + + + original tracking url for the application attempt. + + @return original tracking url for the application attempt]]> + + + + + ApplicationAttemptId of this attempt of the + application + + @return ApplicationAttemptId of the attempt]]> + + + + + ContainerId of AMContainer for this attempt + + @return ContainerId of the attempt]]> + + + + + + + finish time of the application. + + @return finish time of the application]]> + + + + + It includes details such as: +
      +
    • {@link ApplicationAttemptId} of the application.
    • +
    • Host on which the ApplicationMaster of this attempt is + running.
    • +
    • RPC port of the ApplicationMaster of this attempt.
    • +
    • Tracking URL.
    • +
    • Diagnostic information in case of errors.
    • +
    • {@link YarnApplicationAttemptState} of the application attempt.
    • +
    • {@link ContainerId} of the master Container.
    • +
    ]]> +
    +
    + + + + + + + + + + + + + ApplicationId + which is unique for all applications started by a particular instance + of the ResourceManager. + @return short integer identifier of the ApplicationId]]> + + + + + start time of the ResourceManager which is + used to generate globally unique ApplicationId. + @return start time of the ResourceManager]]> + + + + + + + + + + + + + + + + + + + + + ApplicationId represents the globally unique + identifier for an application.

    + +

    The globally unique nature of the identifier is achieved by using the + cluster timestamp i.e. start-time of the + ResourceManager along with a monotonically increasing counter + for the application.

    ]]> +
    +
    + + + + + + + + ApplicationId of the application. + @return ApplicationId of the application]]> + + + + + ApplicationAttemptId of the current + attempt of the application + @return ApplicationAttemptId of the attempt]]> + + + + + user who submitted the application. + @return user who submitted the application]]> + + + + + queue to which the application was submitted. + @return queue to which the application was submitted]]> + + + + + name of the application. + @return name of the application]]> + + + + + host on which the ApplicationMaster + is running. + @return host on which the ApplicationMaster + is running]]> + + + + + RPC port of the ApplicationMaster. + @return RPC port of the ApplicationMaster]]> + + + + + client token for communicating with the + ApplicationMaster. +

    + ClientToAMToken is the security token used by the AMs to verify + authenticity of any client. +

    + +

    + The ResourceManager, provides a secure token (via + {@link ApplicationReport#getClientToAMToken()}) which is verified by the + ApplicationMaster when the client directly talks to an AM. +

    + @return client token for communicating with the + ApplicationMaster]]> +
    +
    + + + YarnApplicationState of the application. + @return YarnApplicationState of the application]]> + + + + + diagnositic information of the application in case of + errors. + @return diagnositic information of the application in case + of errors]]> + + + + + tracking url for the application. + @return tracking url for the application]]> + + + + + start time of the application. + @return start time of the application]]> + + + + + + + + + finish time of the application. + @return finish time of the application]]> + + + + + final finish status of the application. + @return final finish status of the application]]> + + + + + + + + + + + + + + + + + + + + + + + + + + The AMRM token is required for AM to RM scheduling operations. For + managed Application Masters YARN takes care of injecting it. For unmanaged + Applications Masters, the token must be obtained via this method and set + in the {@link org.apache.hadoop.security.UserGroupInformation} of the + current user. +

    + The AMRM token will be returned only if all the following conditions are + met: +

      +
    • the requester is the owner of the ApplicationMaster
    • +
    • the application master is an unmanaged ApplicationMaster
    • +
    • the application master is in ACCEPTED state
    • +
    + Else this method returns NULL. + + @return the AM to RM token if available.]]> +
    +
    + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + It includes details such as: +
      +
    • {@link ApplicationId} of the application.
    • +
    • Applications user.
    • +
    • Application queue.
    • +
    • Application name.
    • +
    • Host on which the ApplicationMaster is running.
    • +
    • RPC port of the ApplicationMaster.
    • +
    • Tracking URL.
    • +
    • {@link YarnApplicationState} of the application.
    • +
    • Diagnostic information in case of errors.
    • +
    • Start time of the application.
    • +
    • Client {@link Token} of the application (if security is enabled).
    • +
    + + @see ApplicationClientProtocol#getApplicationReport(org.apache.hadoop.yarn.api.protocolrecords.GetApplicationReportRequest)]]> +
    +
    + + + + + + + + + + + + + Resource. -1 for invalid/inaccessible reports. + @return the used Resource]]> + + + + + Resource. -1 for invalid/inaccessible reports. + @return the reserved Resource]]> + + + + + Resource. -1 for invalid/inaccessible reports. + @return the needed Resource]]> + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + ApplicationId of the submitted application. + @return ApplicationId of the submitted application]]> + + + + + + ApplicationId of the submitted application. + @param applicationId ApplicationId of the submitted + application]]> + + + + + name. + @return application name]]> + + + + + + name. + @param applicationName application name]]> + + + + + queue to which the application is being submitted. + @return queue to which the application is being submitted]]> + + + + + + queue to which the application is being submitted + @param queue queue to which the application is being submitted]]> + + + + + Priority of the application. + @return Priority of the application]]> + + + + + ContainerLaunchContext to describe the + Container with which the ApplicationMaster is + launched. + @return ContainerLaunchContext for the + ApplicationMaster container]]> + + + + + + ContainerLaunchContext to describe the + Container with which the ApplicationMaster is + launched. + @param amContainer ContainerLaunchContext for the + ApplicationMaster container]]> + + + + + YarnApplicationState. + Such apps will not be retried by the RM on app attempt failure. + The default value is false. + @return true if the AM is not managed by the RM]]> + + + + + + + + + + + + + + + + + + + + + + ApplicationMaster for this + application. Please note this will be DEPRECATED, use getResource + in getAMContainerResourceRequest instead. + + @return the resource required by the ApplicationMaster for + this application.]]> + + + + + + ApplicationMaster for this + application. + + @param resource the resource required by the ApplicationMaster + for this application.]]> + + + + + + + + + + + + + + + + + + + + + + + For managed AM, if the flag is true, running containers will not be killed + when application attempt fails and these containers will be retrieved by + the new application attempt on registration via + {@link ApplicationMasterProtocol#registerApplicationMaster(RegisterApplicationMasterRequest)}. +

    +

    + For unmanaged AM, if the flag is true, RM allows re-register and returns + the running containers in the same attempt back to the UAM for HA. +

    + + @param keepContainers the flag which indicates whether to keep containers + across application attempts.]]> +
    +
    + + + + + + + + + + + + + + + + + + + + + + + + + getResource and getPriority of + ApplicationSubmissionContext. + + Number of containers and Priority will be ignored. + + @return ResourceRequest of the AM container + @deprecated See {@link #getAMContainerResourceRequests()}]]> + + + + + + + + + + + getAMContainerResourceRequest and its behavior. + + Number of containers and Priority will be ignored. + + @return List of ResourceRequests of the AM container]]> + + + + + + + + + + + + + + + + + + + + + + LogAggregationContext of the application + + @return LogAggregationContext of the application]]> + + + + + + LogAggregationContext for the application + + @param logAggregationContext + for the application]]> + + + + + + + + + + + + + + + + ApplicationTimeouts of the application. Timeout value is + in seconds. + @return all ApplicationTimeouts of the application.]]> + + + + + + ApplicationTimeouts for the application in seconds. + All pre-existing Map entries are cleared before adding the new Map. +

    + Note: If application timeout value is less than or equal to zero + then application submission will throw an exception. +

    + @param applicationTimeouts ApplicationTimeoutss for the + application]]> +
    +
    + + + + + + + + + + + + + + It includes details such as: +
      +
    • {@link ApplicationId} of the application.
    • +
    • Application user.
    • +
    • Application name.
    • +
    • {@link Priority} of the application.
    • +
    • + {@link ContainerLaunchContext} of the container in which the + ApplicationMaster is executed. +
    • +
    • + maxAppAttempts. The maximum number of application attempts. + It should be no larger than the global number of max attempts in the + YARN configuration. +
    • +
    • + attemptFailuresValidityInterval. The default value is -1. + when attemptFailuresValidityInterval in milliseconds is set to + {@literal >} 0, the failure number will no take failures which happen + out of the validityInterval into failure count. If failure count + reaches to maxAppAttempts, the application will be failed. +
    • +
    • Optional, application-specific {@link LogAggregationContext}
    • +
    + + @see ContainerLaunchContext + @see ApplicationClientProtocol#submitApplication(org.apache.hadoop.yarn.api.protocolrecords.SubmitApplicationRequest)]]> +
    +
    + + + + + + + + + + + + + + + + + + + + + + + + expiryTime for given timeout type. + @return expiryTime in ISO8601 standard with format + yyyy-MM-dd'T'HH:mm:ss.SSSZ.]]> + + + + + + expiryTime for given timeout type. + @param expiryTime in ISO8601 standard with format + yyyy-MM-dd'T'HH:mm:ss.SSSZ.]]> + + + + + Remaining Time of an application for given timeout type. + @return Remaining Time in seconds.]]> + + + + + + Remaining Time of an application for given timeout type. + @param remainingTime in seconds.]]> + + + + +
  • {@link ApplicationTimeoutType} of the timeout type.
  • +
  • Expiry time in ISO8601 standard with format + yyyy-MM-dd'T'HH:mm:ss.SSSZ or "UNLIMITED".
  • +
  • Remaining time in seconds.
  • + + The possible values for {ExpiryTime, RemainingTimeInSeconds} are +
      +
    • {UNLIMITED,-1} : Timeout is not configured for given timeout type + (LIFETIME).
    • +
    • {ISO8601 date string, 0} : Timeout is configured and application has + completed.
    • +
    • {ISO8601 date string, greater than zero} : Timeout is configured and + application is RUNNING. Application will be timed out after configured + value.
    • +
    ]]> +
    +
    + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + Resource allocated to the container. + @return Resource allocated to the container]]> + + + + + Priority at which the Container was + allocated. + @return Priority at which the Container was + allocated]]> + + + + + ContainerToken for the container. +

    ContainerToken is the security token used by the framework + to verify authenticity of any Container.

    + +

    The ResourceManager, on container allocation provides a + secure token which is verified by the NodeManager on + container launch.

    + +

    Applications do not need to care about ContainerToken, they + are transparently handled by the framework - the allocated + Container includes the ContainerToken.

    + + @see ApplicationMasterProtocol#allocate(org.apache.hadoop.yarn.api.protocolrecords.AllocateRequest) + @see ContainerManagementProtocol#startContainers(org.apache.hadoop.yarn.api.protocolrecords.StartContainersRequest) + + @return ContainerToken for the container]]> +
    +
    + + + ID corresponding to the original {@code + ResourceRequest{@link #getAllocationRequestId()}}s which is satisfied by + this allocated {@code Container}. +

    + The scheduler may return multiple {@code AllocateResponse}s corresponding + to the same ID as and when scheduler allocates {@code Container}s. + Applications can continue to completely ignore the returned ID in + the response and use the allocation for any of their outstanding requests. +

    + + @return the ID corresponding to the original allocation request + which is satisfied by this allocation.]]> + + + + + The {@code ResourceManager} is the sole authority to allocate any + {@code Container} to applications. The allocated {@code Container} + is always on a single node and has a unique {@link ContainerId}. It has + a specific amount of {@link Resource} allocated. +

    + It includes details such as: +

      +
    • {@link ContainerId} for the container, which is globally unique.
    • +
    • + {@link NodeId} of the node on which it is allocated. +
    • +
    • HTTP uri of the node.
    • +
    • {@link Resource} allocated to the container.
    • +
    • {@link Priority} at which the container was allocated.
    • +
    • + Container {@link Token} of the container, used to securely verify + authenticity of the allocation. +
    • +
    + + Typically, an {@code ApplicationMaster} receives the {@code Container} + from the {@code ResourceManager} during resource-negotiation and then + talks to the {@code NodeManager} to start/stop containers. + + @see ApplicationMasterProtocol#allocate(org.apache.hadoop.yarn.api.protocolrecords.AllocateRequest) + @see ContainerManagementProtocol#startContainers(org.apache.hadoop.yarn.api.protocolrecords.StartContainersRequest) + @see ContainerManagementProtocol#stopContainers(org.apache.hadoop.yarn.api.protocolrecords.StopContainersRequest)]]> +
    +
    + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + ApplicationAttemptId of the application to which the + Container was assigned. +

    + Note: If containers are kept alive across application attempts via + {@link ApplicationSubmissionContext#setKeepContainersAcrossApplicationAttempts(boolean)} + the ContainerId does not necessarily contain the current + running application attempt's ApplicationAttemptId This + container can be allocated by previously exited application attempt and + managed by the current running attempt thus have the previous application + attempt's ApplicationAttemptId. +

    + + @return ApplicationAttemptId of the application to which the + Container was assigned]]> +
    +
    + + + ContainerId, + which doesn't include epoch. Note that this method will be marked as + deprecated, so please use getContainerId instead. + @return lower 32 bits of identifier of the ContainerId]]> + + + + + ContainerId. Upper 24 bits are + reserved as epoch of cluster, and lower 40 bits are reserved as + sequential number of containers. + @return identifier of the ContainerId]]> + + + + + + + + + + + + + + + + + + + + + + + + ContainerId represents a globally unique identifier + for a {@link Container} in the cluster.

    ]]> +
    +
    + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + LocalResource required by the container. + @return all LocalResource required by the container]]> + + + + + + LocalResource required by the container. All pre-existing + Map entries are cleared before adding the new Map + @param localResources LocalResource required by the container]]> + + + + + + Get application-specific binary service data. This is a map keyed + by the name of each {@link AuxiliaryService} that is configured on a + NodeManager and value correspond to the application specific data targeted + for the keyed {@link AuxiliaryService}. +

    + +

    + This will be used to initialize this application on the specific + {@link AuxiliaryService} running on the NodeManager by calling + {@link AuxiliaryService#initializeApplication(ApplicationInitializationContext)} +

    + + @return application-specific binary service data]]> +
    +
    + + + + + Set application-specific binary service data. This is a map keyed + by the name of each {@link AuxiliaryService} that is configured on a + NodeManager and value correspond to the application specific data targeted + for the keyed {@link AuxiliaryService}. All pre-existing Map entries are + preserved. +

    + + @param serviceData + application-specific binary service data]]> +
    +
    + + + environment variables for the container. + @return environment variables for the container]]> + + + + + + environment variables for the container. All pre-existing Map + entries are cleared before adding the new Map + @param environment environment variables for the container]]> + + + + + commands for launching the container. + @return the list of commands for launching the container]]> + + + + + + commands for launching the container. All + pre-existing List entries are cleared before adding the new List + @param commands the list of commands for launching the container]]> + + + + + ApplicationACLs for the application. + @return all the ApplicationACLs]]> + + + + + + ApplicationACLs for the application. All pre-existing + Map entries are cleared before adding the new Map + @param acls ApplicationACLs for the application]]> + + + + + ContainerRetryContext to relaunch container. + @return ContainerRetryContext to relaunch container.]]> + + + + + + ContainerRetryContext to relaunch container. + @param containerRetryContext ContainerRetryContext to + relaunch container.]]> + + + + + It includes details such as: +
      +
    • {@link ContainerId} of the container.
    • +
    • {@link Resource} allocated to the container.
    • +
    • User to whom the container is allocated.
    • +
    • Security tokens (if security is enabled).
    • +
    • + {@link LocalResource} necessary for running the container such + as binaries, jar, shared-objects, side-files etc. +
    • +
    • Optional, application-specific binary service data.
    • +
    • Environment variables for the launched process.
    • +
    • Command to launch the container.
    • +
    • Retry strategy when container exits with failure.
    • +
    + + @see ContainerManagementProtocol#startContainers(org.apache.hadoop.yarn.api.protocolrecords.StartContainersRequest)]]> +
    +
    + + + + + + + + ContainerId of the container. + + @return ContainerId of the container.]]> + + + + + + + + Resource of the container. + + @return allocated Resource of the container.]]> + + + + + + + + NodeId where container is running. + + @return allocated NodeId where container is running.]]> + + + + + + + + Priority of the container. + + @return allocated Priority of the container.]]> + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + ContainerState of the container. + + @return final ContainerState of the container.]]> + + + + + + + + exit status of the container. + + @return final exit status of the container.]]> + + + + + + + + + + + + + + + + + + + + + + + It includes details such as: +
      +
    • {@link ContainerId} of the container.
    • +
    • Allocated Resources to the container.
    • +
    • Assigned Node id.
    • +
    • Assigned Priority.
    • +
    • Creation Time.
    • +
    • Finish Time.
    • +
    • Container Exit Status.
    • +
    • {@link ContainerState} of the container.
    • +
    • Diagnostic information in case of errors.
    • +
    • Log URL.
    • +
    • nodeHttpAddress
    • +
    ]]> +
    +
    + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + It provides details such as: +
      +
    • + {@link ContainerRetryPolicy} : + - NEVER_RETRY(DEFAULT value): no matter what error code is when container + fails to run, just do not retry. + - RETRY_ON_ALL_ERRORS: no matter what error code is, when container fails + to run, just retry. + - RETRY_ON_SPECIFIC_ERROR_CODES: when container fails to run, do retry if + the error code is one of errorCodes, otherwise do not retry. + + Note: if error code is 137(SIGKILL) or 143(SIGTERM), it will not retry + because it is usually killed on purpose. +
    • +
    • + maxRetries specifies how many times to retry if need to retry. + If the value is -1, it means retry forever. +
    • +
    • retryInterval specifies delaying some time before relaunch + container, the unit is millisecond.
    • +
    • + failuresValidityInterval: default value is -1. + When failuresValidityInterval in milliseconds is set to {@literal >} 0, + the failure number will not take failures which happen out of the + failuresValidityInterval into failure count. If failure count + reaches to maxRetries, the container will be failed. +
    • +
    ]]> +
    +
    + + + + + + + + + + Retry policy for relaunching a Container.

    ]]> +
    +
    + + + + + + + + + + State of a Container.

    ]]> +
    +
    + + + + + + + + ContainerId of the container. + @return ContainerId of the container]]> + + + + + ExecutionType of the container. + @return ExecutionType of the container]]> + + + + + ContainerState of the container. + @return ContainerState of the container]]> + + + + + Get the exit status for the container.

    + +

    Note: This is valid only for completed containers i.e. containers + with state {@link ContainerState#COMPLETE}. + Otherwise, it returns an ContainerExitStatus.INVALID. +

    + +

    Containers killed by the framework, either due to being released by + the application or being 'lost' due to node failures etc. have a special + exit code of ContainerExitStatus.ABORTED.

    + +

    When threshold number of the nodemanager-local-directories or + threshold number of the nodemanager-log-directories become bad, then + container is not launched and is exited with ContainersExitStatus.DISKS_FAILED. +

    + + @return exit status for the container]]> +
    +
    + + + diagnostic messages for failed containers. + @return diagnostic messages for failed containers]]> + + + + + Resource allocated to the container. + @return Resource allocated to the container]]> + + + + + + + + + + + + + + + + + + + + It provides details such as: +
      +
    • {@code ContainerId} of the container.
    • +
    • {@code ExecutionType} of the container.
    • +
    • {@code ContainerState} of the container.
    • +
    • Exit status of a completed container.
    • +
    • Diagnostic message for a failed container.
    • +
    • {@link Resource} allocated to the container.
    • +
    ]]> +
    +
    + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + The execution types are the following: +
      +
    • {@link #GUARANTEED} - this container is guaranteed to start its + execution, once the corresponding start container request is received by + an NM. +
    • {@link #OPPORTUNISTIC} - the execution of this container may not start + immediately at the NM that receives the corresponding start container + request (depending on the NM's available resources). Moreover, it may be + preempted if it blocks a GUARANTEED container from being executed. +
    ]]> +
    +
    + + + + + + + + + + + + + + + + + + + ExecutionType of the requested container. + + @param execType + ExecutionType of the requested container]]> + + + + + ExecutionType. + + @return ExecutionType.]]> + + + + + + + + + + + ResourceRequest. + Defaults to false. + @return whether ExecutionType request should be strictly honored]]> + + + + + + + + + ExecutionType as well as flag that explicitly asks the + configuredScheduler to return Containers of exactly the Execution Type + requested.]]> + + + + + + + + + + + + Application.]]> + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + +
  • resource key
  • +
  • {@link LocalizationState} of the resource
  • + ]]> +
    +
    + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + location of the resource to be localized. + @return location of the resource to be localized]]> + + + + + + location of the resource to be localized. + @param resource location of the resource to be localized]]> + + + + + size of the resource to be localized. + @return size of the resource to be localized]]> + + + + + + size of the resource to be localized. + @param size size of the resource to be localized]]> + + + + + timestamp of the resource to be localized, used + for verification. + @return timestamp of the resource to be localized]]> + + + + + + timestamp of the resource to be localized, used + for verification. + @param timestamp timestamp of the resource to be localized]]> + + + + + LocalResourceType of the resource to be localized. + @return LocalResourceType of the resource to be localized]]> + + + + + + LocalResourceType of the resource to be localized. + @param type LocalResourceType of the resource to be localized]]> + + + + + LocalResourceVisibility of the resource to be + localized. + @return LocalResourceVisibility of the resource to be + localized]]> + + + + + + LocalResourceVisibility of the resource to be + localized. + @param visibility LocalResourceVisibility of the resource to be + localized]]> + + + + + pattern that should be used to extract entries from the + archive (only used when type is PATTERN). + @return pattern that should be used to extract entries from the + archive.]]> + + + + + + pattern that should be used to extract entries from the + archive (only used when type is PATTERN). + @param pattern pattern that should be used to extract entries + from the archive.]]> + + + + + + + + + + + shouldBeUploadedToSharedCache + of this request]]> + + + + LocalResource represents a local resource required to + run a container.

    + +

    The NodeManager is responsible for localizing the resource + prior to launching the container.

    + +

    Applications can specify {@link LocalResourceType} and + {@link LocalResourceVisibility}.

    + + @see LocalResourceType + @see LocalResourceVisibility + @see ContainerLaunchContext + @see ApplicationSubmissionContext + @see ContainerManagementProtocol#startContainers(org.apache.hadoop.yarn.api.protocolrecords.StartContainersRequest)]]> +
    +
    + + + + + + + + + + type + of a resource localized by the {@code NodeManager}. +

    + The type can be one of: +

      +
    • + {@link #FILE} - Regular file i.e. uninterpreted bytes. +
    • +
    • + {@link #ARCHIVE} - Archive, which is automatically unarchived by the + NodeManager. +
    • +
    • + {@link #PATTERN} - A hybrid between {@link #ARCHIVE} and {@link #FILE}. +
    • +
    + + @see LocalResource + @see ContainerLaunchContext + @see ApplicationSubmissionContext + @see ContainerManagementProtocol#startContainers(org.apache.hadoop.yarn.api.protocolrecords.StartContainersRequest)]]> +
    +
    + + + + + + + + + + visibility + of a resource localized by the {@code NodeManager}. +

    + The visibility can be one of: +

      +
    • {@link #PUBLIC} - Shared by all users on the node.
    • +
    • + {@link #PRIVATE} - Shared among all applications of the + same user on the node. +
    • +
    • + {@link #APPLICATION} - Shared only among containers of the + same application on the node. +
    • +
    + + @see LocalResource + @see ContainerLaunchContext + @see ApplicationSubmissionContext + @see ContainerManagementProtocol#startContainers(org.apache.hadoop.yarn.api.protocolrecords.StartContainersRequest)]]> +
    +
    + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + It includes details such as: +
      +
    • + includePattern. It uses Java Regex to filter the log files + which match the defined include pattern and those log files + will be uploaded when the application finishes. +
    • +
    • + excludePattern. It uses Java Regex to filter the log files + which match the defined exclude pattern and those log files + will not be uploaded when application finishes. If the log file + name matches both the include and the exclude pattern, this file + will be excluded eventually. +
    • +
    • + rolledLogsIncludePattern. It uses Java Regex to filter the log files + which match the defined include pattern and those log files + will be aggregated in a rolling fashion. +
    • +
    • + rolledLogsExcludePattern. It uses Java Regex to filter the log files + which match the defined exclude pattern and those log files + will not be aggregated in a rolling fashion. If the log file + name matches both the include and the exclude pattern, this file + will be excluded eventually. +
    • +
    • + policyClassName. The policy class name that implements + ContainerLogAggregationPolicy. At runtime, nodemanager will the policy + if a given container's log should be aggregated based on the + ContainerType and other runtime state such as exit code by calling + ContainerLogAggregationPolicy#shouldDoLogAggregation. + This is useful when the app only wants to aggregate logs of a subset of + containers. Here are the available policies. Please make sure to specify + the canonical name by prefixing org.apache.hadoop.yarn.server. + nodemanager.containermanager.logaggregation. + to the class simple name below. + NoneContainerLogAggregationPolicy: skip aggregation for all containers. + AllContainerLogAggregationPolicy: aggregate all containers. + AMOrFailedContainerLogAggregationPolicy: aggregate application master + or failed containers. + FailedOrKilledContainerLogAggregationPolicy: aggregate failed or killed + containers + FailedContainerLogAggregationPolicy: aggregate failed containers + AMOnlyLogAggregationPolicy: aggregate application master containers + SampleContainerLogAggregationPolicy: sample logs of successful worker + containers, in addition to application master and failed/killed + containers. + LimitSizeContainerLogAggregationPolicy: skip aggregation for killed + containers whose log size exceeds the limit of container log size. + If it isn't specified, it will use the cluster-wide default policy + defined by configuration yarn.nodemanager.log-aggregation.policy.class. + The default value of yarn.nodemanager.log-aggregation.policy.class is + AllContainerLogAggregationPolicy. +
    • +
    • + policyParameters. The parameters passed to the policy class via + ContainerLogAggregationPolicy#parseParameters during the policy object + initialization. This is optional. Some policy class might use parameters + to adjust its settings. It is up to policy class to define the scheme of + parameters. + For example, SampleContainerLogAggregationPolicy supports the format of + "SR:0.5,MIN:50", which means sample rate of 50% beyond the first 50 + successful worker containers. +
    • +
    + + @see ApplicationSubmissionContext]]> +
    +
    + + + + + + + + NodeManager for which the NMToken + is used to authenticate. + @return the {@link NodeId} of the NodeManager for which the + NMToken is used to authenticate.]]> + + + + + + + + NodeManager + @return the {@link Token} used for authenticating with NodeManager]]> + + + + + + + + + + + + The NMToken is used for authenticating communication with + NodeManager

    +

    It is issued by ResourceMananger when ApplicationMaster + negotiates resource with ResourceManager and + validated on NodeManager side.

    + @see AllocateResponse#getNMTokens()]]> +
    +
    + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + Node Attribute is a kind of a label which represents one of the + attribute/feature of a Node. Its different from node partition label as + resource guarantees across the queues will not be maintained for these type + of labels. +

    +

    + A given Node can be mapped with any kind of attribute, few examples are + HAS_SSD=true, JAVA_VERSION=JDK1.8, OS_TYPE=WINDOWS. +

    +

    + Its not compulsory for all the attributes to have value, empty string is the + default value of the NodeAttributeType.STRING +

    +

    + Node Attribute Prefix is used as namespace to segregate the attributes. +

    ]]> +
    +
    + + + + + + + + + + + + + + + + + + + + + + + + + Node Attribute Info describes a NodeAttribute. +

    ]]> +
    +
    + + + + + + + + + + + + + + + + + + + + + + + + + Node AttributeKey uniquely identifies a given Node Attribute. Node Attribute + is identified based on attribute prefix and name. +

    +

    + Node Attribute Prefix is used as namespace to segregate the attributes. +

    ]]> +
    +
    + + + + + + + + + + + + + + + + + + + + + + + Type of a node Attribute. +

    + Based on this attribute expressions and values will be evaluated.]]> +
    +
    + + + + + + + + + + + + + hostname of the node. + @return hostname of the node]]> + + + + + port for communicating with the node. + @return port for communicating with the node]]> + + + + + + + + + + + + + + + + + + + NodeId is the unique identifier for a node.

    + +

    It includes the hostname and port to uniquely + identify the node. Thus, it is unique across restarts of any + NodeManager.

    ]]> +
    +
    + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + NodeId of the node. + @return NodeId of the node]]> + + + + + NodeState of the node. + @return NodeState of the node]]> + + + + + http address of the node. + @return http address of the node]]> + + + + + rack name for the node. + @return rack name for the node]]> + + + + + used Resource on the node. + @return used Resource on the node]]> + + + + + total Resource on the node. + @return total Resource on the node]]> + + + + + diagnostic health report of the node. + @return diagnostic health report of the node]]> + + + + + last timestamp at which the health report was received. + @return last timestamp at which the health report was received]]> + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + It includes details such as: +
      +
    • {@link NodeId} of the node.
    • +
    • HTTP Tracking URL of the node.
    • +
    • Rack name for the node.
    • +
    • Used {@link Resource} on the node.
    • +
    • Total available {@link Resource} of the node.
    • +
    • Number of running containers on the node.
    • +
    + + @see ApplicationClientProtocol#getClusterNodes(org.apache.hadoop.yarn.api.protocolrecords.GetClusterNodesRequest)]]> +
    +
    + + + + + + + + + + + + + + + + State of a Node.

    ]]> +
    +
    + + + + + + + + + + + + + + + + + + + + + + Mapping of Attribute Value to a Node. +

    ]]> +
    +
    + + + + + + + + + + + + ResourceManager. + @see PreemptionContract + @see StrictPreemptionContract]]> + + + + + + + + + + ApplicationMaster about resources requested back by the + ResourceManager. + @see AllocateRequest#setAskList(List)]]> + + + + + ApplicationMaster that may be reclaimed by the + ResourceManager. If the AM prefers a different set of + containers, then it may checkpoint or kill containers matching the + description in {@link #getResourceRequest}. + @return Set of containers at risk if the contract is not met.]]> + + + + ResourceManager. + The ApplicationMaster (AM) can satisfy this request according + to its own priorities to prevent containers from being forcibly killed by + the platform. + @see PreemptionMessage]]> + + + + + + + + + + ResourceManager]]> + + + + + + + + + + The AM should decode both parts of the message. The {@link + StrictPreemptionContract} specifies particular allocations that the RM + requires back. The AM can checkpoint containers' state, adjust its execution + plan to move the computation, or take no action and hope that conditions that + caused the RM to ask for the container will change. +

    + In contrast, the {@link PreemptionContract} also includes a description of + resources with a set of containers. If the AM releases containers matching + that profile, then the containers enumerated in {@link + PreemptionContract#getContainers()} may not be killed. +

    + Each preemption message reflects the RM's current understanding of the + cluster state, so a request to return N containers may not + reflect containers the AM is releasing, recently exited containers the RM has + yet to learn about, or new containers allocated before the message was + generated. Conversely, an RM may request a different profile of containers in + subsequent requests. +

    + The policy enforced by the RM is part of the scheduler. Generally, only + containers that have been requested consistently should be killed, but the + details are not specified.]]> + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + The ACL is one of: +

      +
    • + {@link #SUBMIT_APPLICATIONS} - ACL to submit applications to the queue. +
    • +
    • {@link #ADMINISTER_QUEUE} - ACL to administer the queue.
    • +
    + + @see QueueInfo + @see ApplicationClientProtocol#getQueueUserAcls(org.apache.hadoop.yarn.api.protocolrecords.GetQueueUserAclsInfoRequest)]]> +
    +
    + + + + + + + + name of the queue. + @return name of the queue]]> + + + + + configured capacity of the queue. + @return configured capacity of the queue]]> + + + + + maximum capacity of the queue. + @return maximum capacity of the queue]]> + + + + + current capacity of the queue. + @return current capacity of the queue]]> + + + + + child queues of the queue. + @return child queues of the queue]]> + + + + + running applications of the queue. + @return running applications of the queue]]> + + + + + QueueState of the queue. + @return QueueState of the queue]]> + + + + + accessible node labels of the queue. + @return accessible node labels of the queue]]> + + + + + default node label expression of the queue, this takes + affect only when the ApplicationSubmissionContext and + ResourceRequest don't specify their + NodeLabelExpression. + + @return default node label expression of the queue]]> + + + + + + + + queue stats for the queue + + @return queue stats of the queue]]> + + + + + + + + + + + preemption status of the queue. + @return if property is not in proto, return null; + otherwise, return preemption status of the queue]]> + + + + + + + + + + + + + + + It includes information such as: +
      +
    • Queue name.
    • +
    • Capacity of the queue.
    • +
    • Maximum capacity of the queue.
    • +
    • Current capacity of the queue.
    • +
    • Child queues.
    • +
    • Running applications.
    • +
    • {@link QueueState} of the queue.
    • +
    • {@link QueueConfigurations} of the queue.
    • +
    + + @see QueueState + @see QueueConfigurations + @see ApplicationClientProtocol#getQueueInfo(org.apache.hadoop.yarn.api.protocolrecords.GetQueueInfoRequest)]]> +
    +
    + + + + + + + + + + + A queue is in one of: +
      +
    • {@link #RUNNING} - normal state.
    • +
    • {@link #STOPPED} - not accepting new application submissions.
    • +
    • + {@link #DRAINING} - not accepting new application submissions + and waiting for applications finish. +
    • +
    + + @see QueueInfo + @see ApplicationClientProtocol#getQueueInfo(org.apache.hadoop.yarn.api.protocolrecords.GetQueueInfoRequest)]]> +
    +
    + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + queue name of the queue. + @return queue name of the queue]]> + + + + + QueueACL for the given user. + @return list of QueueACL for the given user]]> + + + + QueueUserACLInfo provides information {@link QueueACL} for + the given user.

    + + @see QueueACL + @see ApplicationClientProtocol#getQueueUserAcls(org.apache.hadoop.yarn.api.protocolrecords.GetQueueUserAclsInfoRequest)]]> +
    +
    + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + The ACL is one of: +
      +
    • + {@link #ADMINISTER_RESERVATIONS} - ACL to create, list, update and + delete reservations. +
    • +
    • {@link #LIST_RESERVATIONS} - ACL to list reservations.
    • +
    • {@link #SUBMIT_RESERVATIONS} - ACL to create reservations.
    • +
    + Users can always list, update and delete their own reservations.]]> +
    +
    + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + It includes: +
      +
    • Duration of the reservation.
    • +
    • Acceptance time of the duration.
    • +
    • + List of {@link ResourceAllocationRequest}, which includes the time + interval, and capability of the allocation. + {@code ResourceAllocationRequest} represents an allocation + made for a reservation for the current state of the queue. This can be + changed for reasons such as re-planning, but will always be subject to + the constraints of the user contract as described by + {@link ReservationDefinition} +
    • +
    • {@link ReservationId} of the reservation.
    • +
    • {@link ReservationDefinition} used to make the reservation.
    • +
    + + @see ResourceAllocationRequest + @see ReservationId + @see ReservationDefinition]]> +
    +
    + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + start time of the {@code ResourceManager} which is used to + generate globally unique {@link ReservationId}. + + @return start time of the {@code ResourceManager}]]> + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + {@link ReservationId} represents the globally unique identifier for + a reservation. +

    + +

    + The globally unique nature of the identifier is achieved by using the + cluster timestamp i.e. start-time of the {@code ResourceManager} + along with a monotonically increasing counter for the reservation. +

    ]]> +
    +
    + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + It includes: +
      +
    • {@link Resource} required for each request.
    • +
    • + Number of containers, of above specifications, which are required by the + application. +
    • +
    • Concurrency that indicates the gang size of the request.
    • +
    ]]> +
    +
    + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + memory of the resource. Note - while memory has + never had a unit specified, all YARN configurations have specified memory + in MB. The assumption has been that the daemons and applications are always + using the same units. With the introduction of the ResourceInformation + class we have support for units - so this function will continue to return + memory but in the units of MB + + @return memory(in MB) of the resource]]> + + + + + memory of the resource. Note - while memory has + never had a unit specified, all YARN configurations have specified memory + in MB. The assumption has been that the daemons and applications are always + using the same units. With the introduction of the ResourceInformation + class we have support for units - so this function will continue to return + memory but in the units of MB + + @return memory of the resource]]> + + + + + + memory of the resource. Note - while memory has + never had a unit specified, all YARN configurations have specified memory + in MB. The assumption has been that the daemons and applications are always + using the same units. With the introduction of the ResourceInformation + class we have support for units - so this function will continue to set + memory but the assumption is that the value passed is in units of MB. + + @param memory memory(in MB) of the resource]]> + + + + + + memory of the resource. + @param memory memory of the resource]]> + + + + + number of virtual cpu cores of the resource. + + Virtual cores are a unit for expressing CPU parallelism. A node's capacity + should be configured with virtual cores equal to its number of physical + cores. A container should be requested with the number of cores it can + saturate, i.e. the average number of threads it expects to have runnable + at a time. + + @return num of virtual cpu cores of the resource]]> + + + + + + number of virtual cpu cores of the resource. + + Virtual cores are a unit for expressing CPU parallelism. A node's capacity + should be configured with virtual cores equal to its number of physical + cores. A container should be requested with the number of cores it can + saturate, i.e. the average number of threads it expects to have runnable + at a time. + + @param vCores number of virtual cpu cores of the resource]]> + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + Resource models a set of computer resources in the + cluster.

    + +

    Currently it models both memory and CPU.

    + +

    The unit for memory is megabytes. CPU is modeled with virtual cores + (vcores), a unit for expressing parallelism. A node's capacity should + be configured with virtual cores equal to its number of physical cores. A + container should be requested with the number of cores it can saturate, i.e. + the average number of threads it expects to have runnable at a time.

    + +

    Virtual cores take integer values and thus currently CPU-scheduling is + very coarse. A complementary axis for CPU requests that represents + processing power will likely be added in the future to enable finer-grained + resource configuration.

    + +

    Typically, applications request Resource of suitable + capability to run their component tasks.

    + + @see ResourceRequest + @see ApplicationMasterProtocol#allocate(org.apache.hadoop.yarn.api.protocolrecords.AllocateRequest)]]> +
    +
    + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + It includes: +
      +
    • StartTime of the allocation.
    • +
    • EndTime of the allocation.
    • +
    • {@link Resource} reserved for the allocation.
    • +
    + + @see Resource]]> +
    +
    + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + blacklist of resources + for the application. + + @see ResourceRequest + @see ApplicationMasterProtocol#allocate(org.apache.hadoop.yarn.api.protocolrecords.AllocateRequest)]]> + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + host/rack string represents an arbitrary + host name. + + @param hostName host/rack on which the allocation is desired + @return whether the given host/rack string represents an arbitrary + host name]]> + + + + + Priority of the request. + @return Priority of the request]]> + + + + + + Priority of the request + @param priority Priority of the request]]> + + + + + host/rack) on which the allocation + is desired. + + A special value of * signifies that any resource + (host/rack) is acceptable. + + @return resource (e.g. host/rack) on which the allocation + is desired]]> + + + + + + host/rack) on which the allocation + is desired. + + A special value of * signifies that any resource name + (e.g. host/rack) is acceptable. + + @param resourceName (e.g. host/rack) on which the + allocation is desired]]> + + + + + + + + + + + + + + + + ResourceRequest. Defaults to true. + + @return whether locality relaxation is enabled with this + ResourceRequest.]]> + + + + + + ExecutionTypeRequest of the requested container. + + @param execSpec + ExecutionTypeRequest of the requested container]]> + + + + + ResourceRequest. Defaults to true. + + @return whether locality relaxation is enabled with this + ResourceRequest.]]> + + + + + + For a request at a network hierarchy level, set whether locality can be relaxed + to that level and beyond.

    + +

    If the flag is off on a rack-level ResourceRequest, + containers at that request's priority will not be assigned to nodes on that + request's rack unless requests specifically for those nodes have also been + submitted.

    + +

    If the flag is off on an {@link ResourceRequest#ANY}-level + ResourceRequest, containers at that request's priority will + only be assigned on racks for which specific requests have also been + submitted.

    + +

    For example, to request a container strictly on a specific node, the + corresponding rack-level and any-level requests should have locality + relaxation set to false. Similarly, to request a container strictly on a + specific rack, the corresponding any-level request should have locality + relaxation set to false.

    + + @param relaxLocality whether locality relaxation is enabled with this + ResourceRequest.]]> + + + + + + + + + + + + + + + + ID corresponding to this allocation request. This + ID is an identifier for different {@code ResourceRequest}s from the same + application. The allocated {@code Container}(s) received as part of the + {@code AllocateResponse} response will have the ID corresponding to the + original {@code ResourceRequest} for which the RM made the allocation. +

    + The scheduler may return multiple {@code AllocateResponse}s corresponding + to the same ID as and when scheduler allocates {@code Container}(s). + Applications can continue to completely ignore the returned ID in + the response and use the allocation for any of their outstanding requests. +

    + If one wishes to replace an entire {@code ResourceRequest} corresponding to + a specific ID, they can simply cancel the corresponding {@code + ResourceRequest} and submit a new one afresh. + + @return the ID corresponding to this allocation request.]]> + + + + + + ID corresponding to this allocation request. This + ID is an identifier for different {@code ResourceRequest}s from the same + application. The allocated {@code Container}(s) received as part of the + {@code AllocateResponse} response will have the ID corresponding to the + original {@code ResourceRequest} for which the RM made the allocation. +

    + The scheduler may return multiple {@code AllocateResponse}s corresponding + to the same ID as and when scheduler allocates {@code Container}(s). + Applications can continue to completely ignore the returned ID in + the response and use the allocation for any of their outstanding requests. +

    + If one wishes to replace an entire {@code ResourceRequest} corresponding to + a specific ID, they can simply cancel the corresponding {@code + ResourceRequest} and submit a new one afresh. +

    + If the ID is not set, scheduler will continue to work as previously and all + allocated {@code Container}(s) will have the default ID, -1. + + @param allocationRequestID the ID corresponding to this allocation + request.]]> + + + + + + Resource capability of the request. + @param capability Resource capability of the request]]> + + + + + Resource capability of the request. + @return Resource capability of the request]]> + + + + + + + + + + + + + + + + + + It includes: +

      +
    • {@link Priority} of the request.
    • +
    • + The name of the host or rack on which the allocation is + desired. A special value of * signifies that + any host/rack is acceptable to the application. +
    • +
    • {@link Resource} required for each request.
    • +
    • + Number of containers, of above specifications, which are required + by the application. +
    • +
    • + A boolean relaxLocality flag, defaulting to {@code true}, + which tells the {@code ResourceManager} if the application wants + locality to be loose (i.e. allows fall-through to rack or any) + or strict (i.e. specify hard constraint on resource allocation). +
    • +
    + + @see Resource + @see ApplicationMasterProtocol#allocate(org.apache.hadoop.yarn.api.protocolrecords.AllocateRequest)]]> +
    +
    + + + + + + + priority of the request. + @see ResourceRequest#setPriority(Priority) + @param priority priority of the request + @return {@link ResourceRequestBuilder}]]> + + + + + + resourceName of the request. + @see ResourceRequest#setResourceName(String) + @param resourceName resourceName of the request + @return {@link ResourceRequestBuilder}]]> + + + + + + capability of the request. + @see ResourceRequest#setCapability(Resource) + @param capability capability of the request + @return {@link ResourceRequestBuilder}]]> + + + + + + numContainers of the request. + @see ResourceRequest#setNumContainers(int) + @param numContainers numContainers of the request + @return {@link ResourceRequestBuilder}]]> + + + + + + relaxLocality of the request. + @see ResourceRequest#setRelaxLocality(boolean) + @param relaxLocality relaxLocality of the request + @return {@link ResourceRequestBuilder}]]> + + + + + + nodeLabelExpression of the request. + @see ResourceRequest#setNodeLabelExpression(String) + @param nodeLabelExpression + nodeLabelExpression of the request + @return {@link ResourceRequestBuilder}]]> + + + + + + executionTypeRequest of the request. + @see ResourceRequest#setExecutionTypeRequest( + ExecutionTypeRequest) + @param executionTypeRequest + executionTypeRequest of the request + @return {@link ResourceRequestBuilder}]]> + + + + + + executionTypeRequest of the request with 'ensure + execution type' flag set to true. + @see ResourceRequest#setExecutionTypeRequest( + ExecutionTypeRequest) + @param executionType executionType of the request. + @return {@link ResourceRequestBuilder}]]> + + + + + + allocationRequestId of the request. + @see ResourceRequest#setAllocationRequestId(long) + @param allocationRequestId + allocationRequestId of the request + @return {@link ResourceRequestBuilder}]]> + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + virtual memory. + + @return virtual memory in MB]]> + + + + + + virtual memory. + + @param vmem virtual memory in MB]]> + + + + + physical memory. + + @return physical memory in MB]]> + + + + + + physical memory. + + @param pmem physical memory in MB]]> + + + + + CPU utilization (The amount of vcores used). + + @return CPU utilization]]> + + + + + + CPU utilization (The amount of vcores used). + + @param cpu CPU utilization]]> + + + + + + custom resource utilization + (The amount of custom resource used). + + @param resourceName resourceName of custom resource + @return resourceName utilization]]> + + + + + + + + + + + + custom resource utilization + (The amount of custom resource used). + @param resourceName resourceName + @param utilization utilization of custom resource]]> + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + ResourceUtilization models the utilization of a set of computer + resources in the cluster. +

    ]]> +
    +
    + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + allocationRequestId of the request. + + @see SchedulingRequest#setAllocationRequestId(long) + @param allocationRequestId allocationRequestId of the + request + @return {@link SchedulingRequest.SchedulingRequestBuilder}]]> + + + + + + priority of the request. + + @param priority priority of the request + @return {@link SchedulingRequest.SchedulingRequestBuilder} + @see SchedulingRequest#setPriority(Priority)]]> + + + + + + executionType of the request. + + @see SchedulingRequest#setExecutionType(ExecutionTypeRequest) + @param executionType executionType of the request + @return {@link SchedulingRequest.SchedulingRequestBuilder}]]> + + + + + + allocationTags of the request. + + @see SchedulingRequest#setAllocationTags(Set) + @param allocationTags allocationsTags of the request + @return {@link SchedulingRequest.SchedulingRequestBuilder}]]> + + + + + + executionType of the request. + + @see SchedulingRequest#setResourceSizing(ResourceSizing) + @param resourceSizing resourceSizing of the request + @return {@link SchedulingRequest.SchedulingRequestBuilder}]]> + + + + + + placementConstraintExpression of the request. + + @see SchedulingRequest#setPlacementConstraint( + PlacementConstraint) + @param placementConstraintExpression placementConstraints of + the request + @return {@link SchedulingRequest.SchedulingRequestBuilder}]]> + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + ApplicationMaster that may be reclaimed by the + ResourceManager. + @return the set of {@link ContainerId} to be preempted.]]> + + + + ApplicationMaster (AM) + may attempt to checkpoint work or adjust its execution plan to accommodate + it. In contrast to {@link PreemptionContract}, the AM has no flexibility in + selecting which resources to return to the cluster. + @see PreemptionMessage]]> + + + + + + + + + + + + + + + + + + + + + + + + + + + + + Token is the security entity used by the framework + to verify authenticity of any resource.

    ]]> +
    +
    + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + ContainerId of the container. + @return ContainerId of the container]]> + + + + + + + + + + + ContainerUpdateType of the container. + @return ContainerUpdateType of the container.]]> + + + + + + ContainerUpdateType of the container. + @param updateType of the Container]]> + + + + + ContainerId of the container. + @return ContainerId of the container]]> + + + + + + ContainerId of the container. + @param containerId ContainerId of the container]]> + + + + + ExecutionType of the container. + @return ExecutionType of the container]]> + + + + + + ExecutionType of the container. + @param executionType ExecutionType of the container]]> + + + + + + Resource capability of the request. + @param capability Resource capability of the request]]> + + + + + Resource capability of the request. + @return Resource capability of the request]]> + + + + + + + + + + + + It includes: +
      +
    • version for the container.
    • +
    • {@link ContainerId} for the container.
    • +
    • + {@link Resource} capability of the container after the update request + is completed. +
    • +
    • + {@link ExecutionType} of the container after the update request is + completed. +
    • +
    + + Update rules: +
      +
    • + Currently only ONE aspect of the container can be updated per request + (user can either update Capability OR ExecutionType in one request.. + not both). +
    • +
    • + There must be only 1 update request per container in an allocate call. +
    • +
    • + If a new update request is sent for a container (in a subsequent allocate + call) before the first one is satisfied by the Scheduler, it will + overwrite the previous request. +
    • +
    + @see ApplicationMasterProtocol#allocate(org.apache.hadoop.yarn.api.protocolrecords.AllocateRequest)]]> +
    +
    + + + + + + + + + + + + + + + ContainerUpdateType. + @return ContainerUpdateType]]> + + + + + + ContainerUpdateType. + @param updateType ContainerUpdateType]]> + + + + + Container. + @return Container]]> + + + + + + Container. + @param container Container]]> + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + URL represents a serializable {@link java.net.URL}.

    ]]> +
    +
    + + + + + + + + + + RMAppAttempt.]]> + + + + + + + + + + + + ApplicationMaster.]]> + + + + + + + + + + NodeManagers in the cluster. + @return number of NodeManagers in the cluster]]> + + + + + DecommissionedNodeManagers in the cluster. + + @return number of DecommissionedNodeManagers in the cluster]]> + + + + + ActiveNodeManagers in the cluster. + + @return number of ActiveNodeManagers in the cluster]]> + + + + + LostNodeManagers in the cluster. + + @return number of LostNodeManagers in the cluster]]> + + + + + UnhealthyNodeManagers in the cluster. + + @return number of UnhealthyNodeManagers in the cluster]]> + + + + + RebootedNodeManagers in the cluster. + + @return number of RebootedNodeManagers in the cluster]]> + + + + YarnClusterMetrics represents cluster metrics.

    + +

    Currently only number of NodeManagers is provided.

    ]]> +
    +
    + +
    + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + This class contains the information about a timeline domain, which is used + to a user to host a number of timeline entities, isolating them from others'. + The user can also define the reader and writer users/groups for the the + domain, which is used to control the access to its entities. +

    + +

    + The reader and writer users/groups pattern that the user can supply is the + same as what AccessControlList takes. +

    ]]> +
    +
    + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + The class that contains the the meta information of some conceptual entity + and its related events. The entity can be an application, an application + attempt, a container or whatever the user-defined object. +

    + +

    + Primary filters will be used to index the entities in + TimelineStore, such that users should carefully choose the + information they want to store as the primary filters. The remaining can be + stored as other information. +

    ]]> +
    +
    + + + + + + + + + + + + + ApplicationId of the + TimelineEntityGroupId. + + @return ApplicationId of the + TimelineEntityGroupId]]> + + + + + + + + timelineEntityGroupId. + + @return timelineEntityGroupId]]> + + + + + + + + + + + + + + + + + + + TimelineEntityGroupId is an abstract way for + timeline service users to represent #a group of related timeline data. + For example, all entities that represents one data flow DAG execution + can be grouped into one timeline entity group.

    ]]> +
    +
    + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + +
    + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + This class contains the information about a timeline service domain, which is + used to a user to host a number of timeline entities, isolating them from + others'. The user can also define the reader and writer users/groups for + the domain, which is used to control the access to its entities. +

    +

    + The reader and writer users/groups pattern that the user can supply is the + same as what AccessControlList takes. +

    ]]> +
    +
    + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + The constuctor is used to construct a proxy {@link TimelineEntity} or its + subclass object from the real entity object that carries information. +

    + +

    + It is usually used in the case where we want to recover class polymorphism + after deserializing the entity from its JSON form. +

    + @param entity the real entity that carries information]]> +
    +
    + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + Note: Entities will be stored in the order of idPrefix specified. + If users decide to set idPrefix for an entity, they MUST provide + the same prefix for every update of this entity. +

    + Example:
    + TimelineEntity entity = new TimelineEntity();
    + entity.setIdPrefix(value);
    + 
    + Users can use {@link TimelineServiceHelper#invertLong(long)} to invert + the prefix if necessary. + + @param entityIdPrefix prefix for an entity.]]> +
    +
    + + + + + + + +
    + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + +
    + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + name property as a + InetSocketAddress. On an HA cluster, + this fetches the address corresponding to the RM identified by + {@link #RM_HA_ID}. + @param name property name. + @param defaultAddress the default value + @param defaultPort the default port + @return InetSocketAddress]]> + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + yarn.resourcemanager.scheduler.class + cannot handle placement constraints, the corresponding SchedulingRequests + will be rejected. As of now, only the capacity scheduler supports + SchedulingRequests. In particular, it currently supports anti-affinity + constraints (no affinity or cardinality) and places one container at a + time. The advantage of this handler compared to the placement-processor is + that it follows the same ordering rules for queues (sorted by utilization, + priority) and apps (sorted by FIFO/fairness/priority) as the ones followed + by the main scheduler.]]> + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + OPPORTUNISTIC containers on the NM.]]> + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + +
  • default
  • +
  • docker
  • +
  • javasandbox
  • +
  • runc
  • + ]]> +
    +
    + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + +
  • NONE - the RM will do nothing special.
  • +
  • LENIENT - the RM will generate and provide a keystore and truststore + to the AM, which it is free to use for HTTPS in its tracking URL web + server. The RM proxy will still allow HTTP connections to AMs that opt + not to use HTTPS.
  • +
  • STRICT - this is the same as LENIENT, except that the RM proxy will + only allow HTTPS connections to AMs; HTTP connections will be blocked + and result in a warning page to the user.
  • + ]]> +
    +
    + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + Default platform-specific CLASSPATH for YARN applications. A + comma-separated list of CLASSPATH entries constructed based on the client + OS environment expansion syntax. +

    +

    + Note: Use {@link #DEFAULT_YARN_CROSS_PLATFORM_APPLICATION_CLASSPATH} for + cross-platform practice i.e. submit an application from a Windows client to + a Linux/Unix server or vice versa. +

    ]]> +
    +
    + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + +
    + +
    + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + The information is passed along to applications via + {@link StartContainersResponse#getAllServicesMetaData()} that is returned by + {@link ContainerManagementProtocol#startContainers(StartContainersRequest)} +

    + + @return meta-data for this service that should be made available to + applications.]]> +
    +
    + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + +
    + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + The method used by the NodeManager log aggregation service + to initial the policy object with parameters specified by the application + or the cluster-wide setting. +

    + + @param parameters parameters with scheme defined by the policy class.]]> +
    +
    + + + + + The method used by the NodeManager log aggregation service + to ask the policy object if a given container's logs should be aggregated. +

    + + @param logContext ContainerLogContext + @return Whether or not the container's logs should be aggregated.]]> +
    +
    + + + +
    + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + The method used by administrators to ask SCM to run cleaner task right away +

    + + @param request request SharedCacheManager to run a cleaner task + @return SharedCacheManager returns an empty response + on success and throws an exception on rejecting the request + @throws YarnException + @throws IOException]]> +
    +
    + + + The protocol between administrators and the SharedCacheManager +

    ]]> +
    +
    + +
    + + + + + + + + + + + + + + + + + Tag1(N1),P1:Tag2(N2),P2:...:TagN(Nn),Pn

    + + where TagN(Nn) is a key value pair to determine the source + allocation tag and the number of allocations, such as: + +

    foo(3)

    + + Optional when using NodeAttribute Constraint. + + and where Pn can be any form of a valid constraint expression, + such as: + +
      +
    • in,node,foo,bar
    • +
    • notin,node,foo,bar,1,2
    • +
    • and(notin,node,foo:notin,node,bar)
    • +
    + + and NodeAttribute Constraint such as + +
      +
    • yarn.rm.io/foo=true
    • +
    • java=1.7,1.8
    • +
    + @param expression expression string. + @return a map of source tags to placement constraint mapping. + @throws PlacementConstraintParseException]]> +
    +
    + + + + + +
    + +
    + + + + + +
    diff --git a/hadoop-yarn-project/hadoop-yarn/dev-support/jdiff/Apache_Hadoop_YARN_Client_2.10.2.xml b/hadoop-yarn-project/hadoop-yarn/dev-support/jdiff/Apache_Hadoop_YARN_Client_2.10.2.xml new file mode 100644 index 0000000000000..9dea8f4450715 --- /dev/null +++ b/hadoop-yarn-project/hadoop-yarn/dev-support/jdiff/Apache_Hadoop_YARN_Client_2.10.2.xml @@ -0,0 +1,2832 @@ + + + + + + + + + + + + + + + + + + + + + + + + + + + In secure mode, YARN verifies access to the application, queue + etc. before accepting the request. +

    + If the user does not have VIEW_APP access then the following + fields in the report will be set to stubbed values: +

      +
    • host - set to "N/A"
    • +
    • RPC port - set to -1
    • +
    • client token - set to "N/A"
    • +
    • diagnostics - set to "N/A"
    • +
    • tracking URL - set to "N/A"
    • +
    • original tracking URL - set to "N/A"
    • +
    • resource usage report - all values are -1
    • +
    + + @param appId + {@link ApplicationId} of the application that needs a report + @return application report + @throws YarnException + @throws IOException]]> +
    +
    + + + + + + Get a report (ApplicationReport) of all Applications in the cluster. +

    + +

    + If the user does not have VIEW_APP access for an application + then the corresponding report will be filtered as described in + {@link #getApplicationReport(ApplicationId)}. +

    + + @return a list of reports for all applications + @throws YarnException + @throws IOException]]> +
    +
    + + + + + + + Get a report of the given ApplicationAttempt. +

    + +

    + In secure mode, YARN verifies access to the application, queue + etc. before accepting the request. +

    + + @param applicationAttemptId + {@link ApplicationAttemptId} of the application attempt that needs + a report + @return application attempt report + @throws YarnException + @throws ApplicationAttemptNotFoundException if application attempt + not found + @throws IOException]]> +
    +
    + + + + + + + Get a report of all (ApplicationAttempts) of Application in the cluster. +

    + + @param applicationId + @return a list of reports for all application attempts for specified + application + @throws YarnException + @throws IOException]]> +
    +
    + + + + + + + Get a report of the given Container. +

    + +

    + In secure mode, YARN verifies access to the application, queue + etc. before accepting the request. +

    + + @param containerId + {@link ContainerId} of the container that needs a report + @return container report + @throws YarnException + @throws ContainerNotFoundException if container not found + @throws IOException]]> +
    +
    + + + + + + + Get a report of all (Containers) of ApplicationAttempt in the cluster. +

    + + @param applicationAttemptId + @return a list of reports of all containers for specified application + attempt + @throws YarnException + @throws IOException]]> +
    +
    +
    + + + + + + + + + {@code + AMRMClient.createAMRMClientContainerRequest() + } + @return the newly create AMRMClient instance.]]> + + + + + + + + + + RegisterApplicationMasterResponse + @throws YarnException + @throws IOException]]> + + + + + + + + addContainerRequest are sent to the + ResourceManager. New containers assigned to the master are + retrieved. Status of completed containers and node health updates are also + retrieved. This also doubles up as a heartbeat to the ResourceManager and + must be made periodically. The call may not always return any new + allocations of containers. App should not make concurrent allocate + requests. May cause request loss. + +

    + Note : If the user has not removed container requests that have already + been satisfied, then the re-register may end up sending the entire + container requests to the RM (including matched requests). Which would mean + the RM could end up giving it a lot of new allocated containers. +

    + + @param progressIndicator Indicates progress made by the master + @return the response of the allocate request + @throws YarnException + @throws IOException]]> +
    +
    + + + + + + + + + + + + + + allocate + @param req Resource request]]> + + + + + + + + + + + + + allocate. + Any previous pending resource change request of the same container will be + removed. + + Application that calls this method is expected to maintain the + Containers that are returned from previous successful + allocations or resource changes. By passing in the existing container and a + target resource capability to this method, the application requests the + ResourceManager to change the existing resource allocation to the target + resource allocation. + + @deprecated use + {@link #requestContainerUpdate(Container, UpdateContainerRequest)} + + @param container The container returned from the last successful resource + allocation or resource change + @param capability The target resource capability of the container]]> + + + + + + + allocate. + Any previous pending update request of the same container will be + removed. + + @param container The container returned from the last successful resource + allocation or update + @param updateContainerRequest The UpdateContainerRequest.]]> + + + + + + + + + + + + + + + + + + + + + + + + ContainerRequests matching the given + parameters. These ContainerRequests should have been added via + addContainerRequest earlier in the lifecycle. For performance, + the AMRMClient may return its internal collection directly without creating + a copy. Users should not perform mutable operations on the return value. + Each collection in the list contains requests with identical + Resource size that fit in the given capability. In a + collection, requests will be returned in the same order as they were added. + + NOTE: This API only matches Container requests that were created by the + client WITHOUT the allocationRequestId being set. + + @return Collection of request matching the parameters]]> + + + + + + + + + ContainerRequests matching the given + parameters. These ContainerRequests should have been added via + addContainerRequest earlier in the lifecycle. For performance, + the AMRMClient may return its internal collection directly without creating + a copy. Users should not perform mutable operations on the return value. + Each collection in the list contains requests with identical + Resource size that fit in the given capability. In a + collection, requests will be returned in the same order as they were added. + specify an ExecutionType. + + NOTE: This API only matches Container requests that were created by the + client WITHOUT the allocationRequestId being set. + + @param priority Priority + @param resourceName Location + @param executionType ExecutionType + @param capability Capability + @return Collection of request matching the parameters]]> + + + + + + ContainerRequests matching the given + allocationRequestId. These ContainerRequests should have been added via + addContainerRequest earlier in the lifecycle. For performance, + the AMRMClient may return its internal collection directly without creating + a copy. Users should not perform mutable operations on the return value. + + NOTE: This API only matches Container requests that were created by the + client WITH the allocationRequestId being set to a non-default value. + + @param allocationRequestId Allocation Request Id + @return Collection of request matching the parameters]]> + + + + + + + + + + + + + AMRMClient. This cache must + be shared with the {@link NMClient} used to manage containers for the + AMRMClient +

    + If a NM token cache is not set, the {@link NMTokenCache#getSingleton()} + singleton instance will be used. + + @param nmTokenCache the NM token cache to use.]]> + + + + + AMRMClient. This cache must be + shared with the {@link NMClient} used to manage containers for the + AMRMClient. +

    + If a NM token cache is not set, the {@link NMTokenCache#getSingleton()} + singleton instance will be used. + + @return the NM token cache.]]> + + + + + + + + + + + + + + + + + + + + + + + + check to return true for each 1000 ms. + See also {@link #waitFor(com.google.common.base.Supplier, int)} + and {@link #waitFor(com.google.common.base.Supplier, int, int)} + @param check the condition for which it should wait]]> + + + + + + + + check to return true for each + checkEveryMillis ms. + See also {@link #waitFor(com.google.common.base.Supplier, int, int)} + @param check user defined checker + @param checkEveryMillis interval to call check]]> + + + + + + + + + check to return true for each + checkEveryMillis ms. In the main loop, this method will log + the message "waiting in main loop" for each logInterval times + iteration to confirm the thread is alive. + @param check user defined checker + @param checkEveryMillis interval to call check + @param logInterval interval to log for each]]> + + + + + + + + + + + + + + + + + + + + + + + + + + Start an allocated container.

    + +

    The ApplicationMaster or other applications that use the + client must provide the details of the allocated container, including the + Id, the assigned node's Id and the token via {@link Container}. In + addition, the AM needs to provide the {@link ContainerLaunchContext} as + well.

    + + @param container the allocated container + @param containerLaunchContext the context information needed by the + NodeManager to launch the + container + @return a map between the auxiliary service names and their outputs + @throws YarnException YarnException. + @throws IOException IOException.]]> +
    +
    + + + + + + Increase the resource of a container.

    + +

    The ApplicationMaster or other applications that use the + client must provide the details of the container, including the Id and + the target resource encapsulated in the updated container token via + {@link Container}. +

    + + @param container the container with updated token. + + @throws YarnException YarnException. + @throws IOException IOException.]]> +
    +
    + + + + + + Update the resources of a container.

    + +

    The ApplicationMaster or other applications that use the + client must provide the details of the container, including the Id and + the target resource encapsulated in the updated container token via + {@link Container}. +

    + + @param container the container with updated token. + + @throws YarnException YarnException. + @throws IOException IOException.]]> +
    +
    + + + + + + + Stop an started container.

    + + @param containerId the Id of the started container + @param nodeId the Id of the NodeManager + + @throws YarnException YarnException. + @throws IOException IOException.]]> +
    +
    + + + + + + + Query the status of a container.

    + + @param containerId the Id of the started container + @param nodeId the Id of the NodeManager + + @return the status of a container. + + @throws YarnException YarnException. + @throws IOException IOException.]]> +
    +
    + + + + + + + + Re-Initialize the Container.

    + + @param containerId the Id of the container to Re-Initialize. + @param containerLaunchContex the updated ContainerLaunchContext. + @param autoCommit commit re-initialization automatically ? + + @throws YarnException YarnException. + @throws IOException IOException.]]> +
    +
    + + + + + + Restart the specified container.

    + + @param containerId the Id of the container to restart. + + @throws YarnException YarnException. + @throws IOException IOException.]]> +
    +
    + + + + + + Rollback last reInitialization of the specified container.

    + + @param containerId the Id of the container to restart. + + @throws YarnException YarnException. + @throws IOException IOException.]]> +
    +
    + + + + + + Commit last reInitialization of the specified container.

    + + @param containerId the Id of the container to commit reInitialize. + + @throws YarnException YarnException. + @throws IOException IOException.]]> +
    +
    + + + + Set whether the containers that are started by this client, and are + still running should be stopped when the client stops. By default, the + feature should be enabled.

    However, containers will be stopped only + when service is stopped. i.e. after {@link NMClient#stop()}. + + @param enabled whether the feature is enabled or not]]> +
    +
    + + + + NMClient. This cache must be + shared with the {@link AMRMClient} that requested the containers managed + by this NMClient +

    + If a NM token cache is not set, the {@link NMTokenCache#getSingleton()} + singleton instance will be used. + + @param nmTokenCache the NM token cache to use.]]> + + + + + NMClient. This cache must be + shared with the {@link AMRMClient} that requested the containers managed + by this NMClient +

    + If a NM token cache is not set, the {@link NMTokenCache#getSingleton()} + singleton instance will be used. + + @return the NM token cache]]> + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + By default Yarn client libraries {@link AMRMClient} and {@link NMClient} use + {@link #getSingleton()} instance of the cache. +

      +
    • + Using the singleton instance of the cache is appropriate when running a + single ApplicationMaster in the same JVM. +
    • +
    • + When using the singleton, users don't need to do anything special, + {@link AMRMClient} and {@link NMClient} are already set up to use the + default singleton {@link NMTokenCache} +
    • +
    + If running multiple Application Masters in the same JVM, a different cache + instance should be used for each Application Master. +
      +
    • + If using the {@link AMRMClient} and the {@link NMClient}, setting up + and using an instance cache is as follows: +
      +   NMTokenCache nmTokenCache = new NMTokenCache();
      +   AMRMClient rmClient = AMRMClient.createAMRMClient();
      +   NMClient nmClient = NMClient.createNMClient();
      +   nmClient.setNMTokenCache(nmTokenCache);
      +   ...
      + 
      +
    • +
    • + If using the {@link AMRMClientAsync} and the {@link NMClientAsync}, + setting up and using an instance cache is as follows: +
      +   NMTokenCache nmTokenCache = new NMTokenCache();
      +   AMRMClient rmClient = AMRMClient.createAMRMClient();
      +   NMClient nmClient = NMClient.createNMClient();
      +   nmClient.setNMTokenCache(nmTokenCache);
      +   AMRMClientAsync rmClientAsync = new AMRMClientAsync(rmClient, 1000, [AMRM_CALLBACK]);
      +   NMClientAsync nmClientAsync = new NMClientAsync("nmClient", nmClient, [NM_CALLBACK]);
      +   ...
      + 
      +
    • +
    • + If using {@link ApplicationMasterProtocol} and + {@link ContainerManagementProtocol} directly, setting up and using an + instance cache is as follows: +
      +   NMTokenCache nmTokenCache = new NMTokenCache();
      +   ...
      +   ApplicationMasterProtocol amPro = ClientRMProxy.createRMProxy(conf, ApplicationMasterProtocol.class);
      +   ...
      +   AllocateRequest allocateRequest = ...
      +   ...
      +   AllocateResponse allocateResponse = rmClient.allocate(allocateRequest);
      +   for (NMToken token : allocateResponse.getNMTokens()) {
      +     nmTokenCache.setToken(token.getNodeId().toString(), token.getToken());
      +   }
      +   ...
      +   ContainerManagementProtocolProxy nmPro = ContainerManagementProtocolProxy(conf, nmTokenCache);
      +   ...
      +   nmPro.startContainer(container, containerContext);
      +   ...
      + 
      +
    • +
    + It is also possible to mix the usage of a client ({@code AMRMClient} or + {@code NMClient}, or the async versions of them) with a protocol proxy + ({@code ContainerManagementProtocolProxy} or + {@code ApplicationMasterProtocol}).]]> +
    +
    + + + + + + + + + + + + + + The method to claim a resource with the SharedCacheManager. + The client uses a checksum to identify the resource and an + {@link ApplicationId} to identify which application will be using the + resource. +

    + +

    + The SharedCacheManager responds with whether or not the + resource exists in the cache. If the resource exists, a URL to + the resource in the shared cache is returned. If the resource does not + exist, null is returned instead. +

    + +

    + Once a URL has been returned for a resource, that URL is safe to use for + the lifetime of the application that corresponds to the provided + ApplicationId. +

    + + @param applicationId ApplicationId of the application using the resource + @param resourceKey the key (i.e. checksum) that identifies the resource + @return URL to the resource, or null if it does not exist]]> +
    +
    + + + + + + + The method to release a resource with the SharedCacheManager. + This method is called once an application is no longer using a claimed + resource in the shared cache. The client uses a checksum to identify the + resource and an {@link ApplicationId} to identify which application is + releasing the resource. +

    + +

    + Note: This method is an optimization and the client is not required to call + it for correctness. +

    + + @param applicationId ApplicationId of the application releasing the + resource + @param resourceKey the key (i.e. checksum) that identifies the resource]]> +
    +
    + + + + + + + + + + +
    + + + + + + + + + + + + + + + + Obtain a {@link YarnClientApplication} for a new application, + which in turn contains the {@link ApplicationSubmissionContext} and + {@link org.apache.hadoop.yarn.api.protocolrecords.GetNewApplicationResponse} + objects. +

    + + @return {@link YarnClientApplication} built for a new application + @throws YarnException + @throws IOException]]> +
    +
    + + + + + + + Submit a new application to YARN. It is a blocking call - it + will not return {@link ApplicationId} until the submitted application is + submitted successfully and accepted by the ResourceManager. +

    + +

    + Users should provide an {@link ApplicationId} as part of the parameter + {@link ApplicationSubmissionContext} when submitting a new application, + otherwise it will throw the {@link ApplicationIdNotProvidedException}. +

    + +

    This internally calls {@link ApplicationClientProtocol#submitApplication + (SubmitApplicationRequest)}, and after that, it internally invokes + {@link ApplicationClientProtocol#getApplicationReport + (GetApplicationReportRequest)} and waits till it can make sure that the + application gets properly submitted. If RM fails over or RM restart + happens before ResourceManager saves the application's state, + {@link ApplicationClientProtocol + #getApplicationReport(GetApplicationReportRequest)} will throw + the {@link ApplicationNotFoundException}. This API automatically resubmits + the application with the same {@link ApplicationSubmissionContext} when it + catches the {@link ApplicationNotFoundException}

    + + @param appContext + {@link ApplicationSubmissionContext} containing all the details + needed to submit a new application + @return {@link ApplicationId} of the accepted application + @throws YarnException + @throws IOException + @see #createApplication()]]> +
    +
    + + + + + + + Fail an application attempt identified by given ID. +

    + + @param applicationAttemptId + {@link ApplicationAttemptId} of the attempt to fail. + @throws YarnException + in case of errors or if YARN rejects the request due to + access-control restrictions. + @throws IOException + @see #getQueueAclsInfo()]]> +
    +
    + + + + + + + Kill an application identified by given ID. +

    + + @param applicationId + {@link ApplicationId} of the application that needs to be killed + @throws YarnException + in case of errors or if YARN rejects the request due to + access-control restrictions. + @throws IOException + @see #getQueueAclsInfo()]]> +
    +
    + + + + + + + + Kill an application identified by given ID. +

    + @param applicationId {@link ApplicationId} of the application that needs to + be killed + @param diagnostics for killing an application. + @throws YarnException in case of errors or if YARN rejects the request due + to access-control restrictions. + @throws IOException]]> +
    +
    + + + + + + + Get a report of the given Application. +

    + +

    + In secure mode, YARN verifies access to the application, queue + etc. before accepting the request. +

    + +

    + If the user does not have VIEW_APP access then the following + fields in the report will be set to stubbed values: +

      +
    • host - set to "N/A"
    • +
    • RPC port - set to -1
    • +
    • client token - set to "N/A"
    • +
    • diagnostics - set to "N/A"
    • +
    • tracking URL - set to "N/A"
    • +
    • original tracking URL - set to "N/A"
    • +
    • resource usage report - all values are -1
    • +
    + + @param appId + {@link ApplicationId} of the application that needs a report + @return application report + @throws YarnException + @throws IOException]]> +
    +
    + + + + + + + The AMRM token is required for AM to RM scheduling operations. For + managed Application Masters Yarn takes care of injecting it. For unmanaged + Applications Masters, the token must be obtained via this method and set + in the {@link org.apache.hadoop.security.UserGroupInformation} of the + current user. +

    + The AMRM token will be returned only if all the following conditions are + met: +

      +
    • the requester is the owner of the ApplicationMaster
    • +
    • the application master is an unmanaged ApplicationMaster
    • +
    • the application master is in ACCEPTED state
    • +
    + Else this method returns NULL. + + @param appId {@link ApplicationId} of the application to get the AMRM token + @return the AMRM token if available + @throws YarnException + @throws IOException]]> +
    +
    + + + + + + Get a report (ApplicationReport) of all Applications in the cluster. +

    + +

    + If the user does not have VIEW_APP access for an application + then the corresponding report will be filtered as described in + {@link #getApplicationReport(ApplicationId)}. +

    + + @return a list of reports of all running applications + @throws YarnException + @throws IOException]]> +
    +
    + + + + + + + Get a report (ApplicationReport) of Applications + matching the given application types in the cluster. +

    + +

    + If the user does not have VIEW_APP access for an application + then the corresponding report will be filtered as described in + {@link #getApplicationReport(ApplicationId)}. +

    + + @param applicationTypes set of application types you are interested in + @return a list of reports of applications + @throws YarnException + @throws IOException]]> +
    +
    + + + + + + + Get a report (ApplicationReport) of Applications matching the given + application states in the cluster. +

    + +

    + If the user does not have VIEW_APP access for an application + then the corresponding report will be filtered as described in + {@link #getApplicationReport(ApplicationId)}. +

    + + @param applicationStates set of application states you are interested in + @return a list of reports of applications + @throws YarnException + @throws IOException]]> +
    +
    + + + + + + + + Get a report (ApplicationReport) of Applications matching the given + application types and application states in the cluster. +

    + +

    + If the user does not have VIEW_APP access for an application + then the corresponding report will be filtered as described in + {@link #getApplicationReport(ApplicationId)}. +

    + + @param applicationTypes set of application types you are interested in + @param applicationStates set of application states you are interested in + @return a list of reports of applications + @throws YarnException + @throws IOException]]> +
    +
    + + + + + + + + + Get a report (ApplicationReport) of Applications matching the given + application types, application states and application tags in the cluster. +

    + +

    + If the user does not have VIEW_APP access for an application + then the corresponding report will be filtered as described in + {@link #getApplicationReport(ApplicationId)}. +

    + + @param applicationTypes set of application types you are interested in + @param applicationStates set of application states you are interested in + @param applicationTags set of application tags you are interested in + @return a list of reports of applications + @throws YarnException + @throws IOException]]> +
    +
    + + + + + + + + + + Get a report (ApplicationReport) of Applications matching the given users, + queues, application types and application states in the cluster. If any of + the params is set to null, it is not used when filtering. +

    + +

    + If the user does not have VIEW_APP access for an application + then the corresponding report will be filtered as described in + {@link #getApplicationReport(ApplicationId)}. +

    + + @param queues set of queues you are interested in + @param users set of users you are interested in + @param applicationTypes set of application types you are interested in + @param applicationStates set of application states you are interested in + @return a list of reports of applications + @throws YarnException + @throws IOException]]> +
    +
    + + + + + + + Get a list of ApplicationReports that match the given + {@link GetApplicationsRequest}. +

    + +

    + If the user does not have VIEW_APP access for an application + then the corresponding report will be filtered as described in + {@link #getApplicationReport(ApplicationId)}. +

    + + @param request the request object to get the list of applications. + @return The list of ApplicationReports that match the request + @throws YarnException Exception specific to YARN. + @throws IOException Exception mostly related to connection errors.]]> +
    +
    + + + + + + Get metrics ({@link YarnClusterMetrics}) about the cluster. +

    + + @return cluster metrics + @throws YarnException + @throws IOException]]> +
    +
    + + + + + + + Get a report of nodes ({@link NodeReport}) in the cluster. +

    + + @param states The {@link NodeState}s to filter on. If no filter states are + given, nodes in all states will be returned. + @return A list of node reports + @throws YarnException + @throws IOException]]> +
    +
    + + + + + + + Get a delegation token so as to be able to talk to YARN using those tokens. + + @param renewer + Address of the renewer who can renew these tokens when needed by + securely talking to YARN. + @return a delegation token ({@link Token}) that can be used to + talk to YARN + @throws YarnException + @throws IOException]]> + + + + + + + + + Get information ({@link QueueInfo}) about a given queue. +

    + + @param queueName + Name of the queue whose information is needed + @return queue information + @throws YarnException + in case of errors or if YARN rejects the request due to + access-control restrictions. + @throws IOException]]> +
    +
    + + + + + + Get information ({@link QueueInfo}) about all queues, recursively if there + is a hierarchy +

    + + @return a list of queue-information for all queues + @throws YarnException + @throws IOException]]> +
    +
    + + + + + + Get information ({@link QueueInfo}) about top level queues. +

    + + @return a list of queue-information for all the top-level queues + @throws YarnException + @throws IOException]]> +
    +
    + + + + + + + Get information ({@link QueueInfo}) about all the immediate children queues + of the given queue +

    + + @param parent + Name of the queue whose child-queues' information is needed + @return a list of queue-information for all queues who are direct children + of the given parent queue. + @throws YarnException + @throws IOException]]> +
    +
    + + + + + + Get information about acls for current user on all the + existing queues. +

    + + @return a list of queue acls ({@link QueueUserACLInfo}) for + current user + @throws YarnException + @throws IOException]]> +
    +
    + + + + + + + Get a report of the given ApplicationAttempt. +

    + +

    + In secure mode, YARN verifies access to the application, queue + etc. before accepting the request. +

    + + @param applicationAttemptId + {@link ApplicationAttemptId} of the application attempt that needs + a report + @return application attempt report + @throws YarnException + @throws ApplicationAttemptNotFoundException if application attempt + not found + @throws IOException]]> +
    +
    + + + + + + + Get a report of all (ApplicationAttempts) of Application in the cluster. +

    + + @param applicationId application id of the app + @return a list of reports for all application attempts for specified + application. + @throws YarnException + @throws IOException]]> +
    +
    + + + + + + + Get a report of the given Container. +

    + +

    + In secure mode, YARN verifies access to the application, queue + etc. before accepting the request. +

    + + @param containerId + {@link ContainerId} of the container that needs a report + @return container report + @throws YarnException + @throws ContainerNotFoundException if container not found. + @throws IOException]]> +
    +
    + + + + + + + Get a report of all (Containers) of ApplicationAttempt in the cluster. +

    + + @param applicationAttemptId application attempt id + @return a list of reports of all containers for specified application + attempts + @throws YarnException + @throws IOException]]> +
    +
    + + + + + + + + Attempts to move the given application to the given queue. +

    + + @param appId + Application to move. + @param queue + Queue to place it in to. + @throws YarnException + @throws IOException]]> +
    +
    + + + + + + Obtain a {@link GetNewReservationResponse} for a new reservation, + which contains the {@link ReservationId} object. +

    + + @return The {@link GetNewReservationResponse} containing a new + {@link ReservationId} object. + @throws YarnException if reservation cannot be created. + @throws IOException if reservation cannot be created.]]> +
    +
    + + + + + + + The interface used by clients to submit a new reservation to the + {@code ResourceManager}. +

    + +

    + The client packages all details of its request in a + {@link ReservationSubmissionRequest} object. This contains information + about the amount of capacity, temporal constraints, and gang needs. + Furthermore, the reservation might be composed of multiple stages, with + ordering dependencies among them. +

    + +

    + In order to respond, a new admission control component in the + {@code ResourceManager} performs an analysis of the resources that have + been committed over the period of time the user is requesting, verify that + the user requests can be fulfilled, and that it respect a sharing policy + (e.g., {@code CapacityOverTimePolicy}). Once it has positively determined + that the ReservationRequest is satisfiable the {@code ResourceManager} + answers with a {@link ReservationSubmissionResponse} that includes a + {@link ReservationId}. Upon failure to find a valid allocation the response + is an exception with the message detailing the reason of failure. +

    + +

    + The semantics guarantees that the {@link ReservationId} returned, + corresponds to a valid reservation existing in the time-range request by + the user. The amount of capacity dedicated to such reservation can vary + overtime, depending of the allocation that has been determined. But it is + guaranteed to satisfy all the constraint expressed by the user in the + {@link ReservationDefinition} +

    + + @param request request to submit a new Reservation + @return response contains the {@link ReservationId} on accepting the + submission + @throws YarnException if the reservation cannot be created successfully + @throws IOException]]> +
    +
    + + + + + + + The interface used by clients to update an existing Reservation. This is + referred to as a re-negotiation process, in which a user that has + previously submitted a Reservation. +

    + +

    + The allocation is attempted by virtually substituting all previous + allocations related to this Reservation with new ones, that satisfy the new + {@link ReservationDefinition}. Upon success the previous allocation is + atomically substituted by the new one, and on failure (i.e., if the system + cannot find a valid allocation for the updated request), the previous + allocation remains valid. +

    + + @param request to update an existing Reservation (the + {@link ReservationUpdateRequest} should refer to an existing valid + {@link ReservationId}) + @return response empty on successfully updating the existing reservation + @throws YarnException if the request is invalid or reservation cannot be + updated successfully + @throws IOException]]> +
    +
    + + + + + + + The interface used by clients to remove an existing Reservation. +

    + + @param request to remove an existing Reservation (the + {@link ReservationDeleteRequest} should refer to an existing valid + {@link ReservationId}) + @return response empty on successfully deleting the existing reservation + @throws YarnException if the request is invalid or reservation cannot be + deleted successfully + @throws IOException]]> +
    +
    + + + + + + + The interface used by clients to get the list of reservations in a plan. + The reservationId will be used to search for reservations to list if it is + provided. Otherwise, it will select active reservations within the + startTime and endTime (inclusive). +

    + + @param request to list reservations in a plan. Contains fields to select + String queue, ReservationId reservationId, long startTime, + long endTime, and a bool includeReservationAllocations. + + queue: Required. Cannot be null or empty. Refers to the + reservable queue in the scheduler that was selected when + creating a reservation submission + {@link ReservationSubmissionRequest}. + + reservationId: Optional. If provided, other fields will + be ignored. + + startTime: Optional. If provided, only reservations that + end after the startTime will be selected. This defaults + to 0 if an invalid number is used. + + endTime: Optional. If provided, only reservations that + start on or before endTime will be selected. This defaults + to Long.MAX_VALUE if an invalid number is used. + + includeReservationAllocations: Optional. Flag that + determines whether the entire reservation allocations are + to be returned. Reservation allocations are subject to + change in the event of re-planning as described by + {@link ReservationDefinition}. + + @return response that contains information about reservations that are + being searched for. + @throws YarnException if the request is invalid + @throws IOException if the request failed otherwise]]> +
    +
    + + + + + + The interface used by client to get node to labels mappings in existing cluster +

    + + @return node to labels mappings + @throws YarnException + @throws IOException]]> +
    +
    + + + + + + The interface used by client to get labels to nodes mapping + in existing cluster +

    + + @return node to labels mappings + @throws YarnException + @throws IOException]]> +
    +
    + + + + + + + The interface used by client to get labels to nodes mapping + for specified labels in existing cluster +

    + + @param labels labels for which labels to nodes mapping has to be retrieved + @return labels to nodes mappings for specific labels + @throws YarnException + @throws IOException]]> +
    +
    + + + + + + The interface used by client to get node labels in the cluster +

    + + @return cluster node labels collection + @throws YarnException when there is a failure in + {@link ApplicationClientProtocol} + @throws IOException when there is a failure in + {@link ApplicationClientProtocol}]]> +
    +
    + + + + + + + + The interface used by client to set priority of an application +

    + @param applicationId + @param priority + @return updated priority of an application. + @throws YarnException + @throws IOException]]> +
    +
    + + + + + + + + Signal a container identified by given ID. +

    + + @param containerId + {@link ContainerId} of the container that needs to be signaled + @param command the signal container command + @throws YarnException + @throws IOException]]> +
    +
    + + + + + + + + + + + Get available resource types supported by RM. +

    + @return list of supported resource types with detailed information + @throws YarnException if any issue happens inside YARN + @throws IOException in case of other others]]> +
    +
    +
    + + + + + + + + + + + +
    + + + + + + + + + + + + + + + + Create a new instance of AMRMClientAsync.

    + + @param intervalMs heartbeat interval in milliseconds between AM and RM + @param callbackHandler callback handler that processes responses from + the ResourceManager]]> +
    +
    + + + + + + Create a new instance of AMRMClientAsync.

    + + @param client the AMRMClient instance + @param intervalMs heartbeat interval in milliseconds between AM and RM + @param callbackHandler callback handler that processes responses from + the ResourceManager]]> +
    +
    + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + allocate + @param req Resource request]]> + + + + + + + + + + + + + allocate. + Any previous pending resource change request of the same container will be + removed. + + Application that calls this method is expected to maintain the + Containers that are returned from previous successful + allocations or resource changes. By passing in the existing container and a + target resource capability to this method, the application requests the + ResourceManager to change the existing resource allocation to the target + resource allocation. + + @deprecated use + {@link #requestContainerUpdate(Container, UpdateContainerRequest)} + + @param container The container returned from the last successful resource + allocation or resource change + @param capability The target resource capability of the container]]> + + + + + + + allocate. + Any previous pending update request of the same container will be + removed. + + @param container The container returned from the last successful resource + allocation or update + @param updateContainerRequest The UpdateContainerRequest.]]> + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + check to return true for each 1000 ms. + See also {@link #waitFor(com.google.common.base.Supplier, int)} + and {@link #waitFor(com.google.common.base.Supplier, int, int)} + @param check the condition for which it should wait]]> + + + + + + + + check to return true for each + checkEveryMillis ms. + See also {@link #waitFor(com.google.common.base.Supplier, int, int)} + @param check user defined checker + @param checkEveryMillis interval to call check]]> + + + + + + + + + check to return true for each + checkEveryMillis ms. In the main loop, this method will log + the message "waiting in main loop" for each logInterval times + iteration to confirm the thread is alive. + @param check user defined checker + @param checkEveryMillis interval to call check + @param logInterval interval to log for each]]> + + + + + + + + + + AMRMClientAsync handles communication with the ResourceManager + and provides asynchronous updates on events such as container allocations and + completions. It contains a thread that sends periodic heartbeats to the + ResourceManager. + + It should be used by implementing a CallbackHandler: +
    + {@code
    + class MyCallbackHandler extends AMRMClientAsync.AbstractCallbackHandler {
    +   public void onContainersAllocated(List containers) {
    +     [run tasks on the containers]
    +   }
    +
    +   public void onContainersUpdated(List containers) {
    +     [determine if resource allocation of containers have been increased in
    +      the ResourceManager, and if so, inform the NodeManagers to increase the
    +      resource monitor/enforcement on the containers]
    +   }
    +
    +   public void onContainersCompleted(List statuses) {
    +     [update progress, check whether app is done]
    +   }
    +   
    +   public void onNodesUpdated(List updated) {}
    +   
    +   public void onReboot() {}
    + }
    + }
    + 
    + + The client's lifecycle should be managed similarly to the following: + +
    + {@code
    + AMRMClientAsync asyncClient = 
    +     createAMRMClientAsync(appAttId, 1000, new MyCallbackhandler());
    + asyncClient.init(conf);
    + asyncClient.start();
    + RegisterApplicationMasterResponse response = asyncClient
    +    .registerApplicationMaster(appMasterHostname, appMasterRpcPort,
    +       appMasterTrackingUrl);
    + asyncClient.addContainerRequest(containerRequest);
    + [... wait for application to complete]
    + asyncClient.unregisterApplicationMaster(status, appMsg, trackingUrl);
    + asyncClient.stop();
    + }
    + 
    ]]> +
    +
    + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + Update the resources of a container.

    + +

    The ApplicationMaster or other applications that use the + client must provide the details of the container, including the Id and + the target resource encapsulated in the updated container token via + {@link Container}. +

    + + @param container the container with updated token.]]> +
    +
    + + + + + + Re-Initialize the Container.

    + + @param containerId the Id of the container to Re-Initialize. + @param containerLaunchContex the updated ContainerLaunchContext. + @param autoCommit commit re-initialization automatically ?]]> +
    +
    + + + + Restart the specified container.

    + + @param containerId the Id of the container to restart.]]> +
    +
    + + + + Rollback last reInitialization of the specified container.

    + + @param containerId the Id of the container to restart.]]> +
    +
    + + + + Commit last reInitialization of the specified container.

    + + @param containerId the Id of the container to commit reInitialize.]]> +
    +
    + + + + + + + + + + + + + + + + + + + + + + + + NMClientAsync handles communication with all the NodeManagers + and provides asynchronous updates on getting responses from them. It + maintains a thread pool to communicate with individual NMs where a number of + worker threads process requests to NMs by using {@link NMClientImpl}. The max + size of the thread pool is configurable through + {@link YarnConfiguration#NM_CLIENT_ASYNC_THREAD_POOL_MAX_SIZE}. + + It should be used in conjunction with a CallbackHandler. For example + +
    + {@code
    + class MyCallbackHandler extends NMClientAsync.AbstractCallbackHandler {
    +   public void onContainerStarted(ContainerId containerId,
    +       Map allServiceResponse) {
    +     [post process after the container is started, process the response]
    +   }
    +
    +   public void onContainerResourceIncreased(ContainerId containerId,
    +       Resource resource) {
    +     [post process after the container resource is increased]
    +   }
    +
    +   public void onContainerStatusReceived(ContainerId containerId,
    +       ContainerStatus containerStatus) {
    +     [make use of the status of the container]
    +   }
    +
    +   public void onContainerStopped(ContainerId containerId) {
    +     [post process after the container is stopped]
    +   }
    +
    +   public void onStartContainerError(
    +       ContainerId containerId, Throwable t) {
    +     [handle the raised exception]
    +   }
    +
    +   public void onGetContainerStatusError(
    +       ContainerId containerId, Throwable t) {
    +     [handle the raised exception]
    +   }
    +
    +   public void onStopContainerError(
    +       ContainerId containerId, Throwable t) {
    +     [handle the raised exception]
    +   }
    + }
    + }
    + 
    + + The client's life-cycle should be managed like the following: + +
    + {@code
    + NMClientAsync asyncClient = 
    +     NMClientAsync.createNMClientAsync(new MyCallbackhandler());
    + asyncClient.init(conf);
    + asyncClient.start();
    + asyncClient.startContainer(container, containerLaunchContext);
    + [... wait for container being started]
    + asyncClient.getContainerStatus(container.getId(), container.getNodeId(),
    +     container.getContainerToken());
    + [... handle the status in the callback instance]
    + asyncClient.stopContainer(container.getId(), container.getNodeId(),
    +     container.getContainerToken());
    + [... wait for container being stopped]
    + asyncClient.stop();
    + }
    + 
    ]]> +
    +
    + +
    + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + +
    diff --git a/hadoop-yarn-project/hadoop-yarn/dev-support/jdiff/Apache_Hadoop_YARN_Client_3.3.3.xml b/hadoop-yarn-project/hadoop-yarn/dev-support/jdiff/Apache_Hadoop_YARN_Client_3.3.3.xml new file mode 100644 index 0000000000000..745b4c6a2730e --- /dev/null +++ b/hadoop-yarn-project/hadoop-yarn/dev-support/jdiff/Apache_Hadoop_YARN_Client_3.3.3.xml @@ -0,0 +1,3067 @@ + + + + + + + + + + + + + + + + + + + + + + + + + + + + + In secure mode, YARN verifies access to the application, queue + etc. before accepting the request. +

    + If the user does not have VIEW_APP access then the following + fields in the report will be set to stubbed values: +

      +
    • host - set to "N/A"
    • +
    • RPC port - set to -1
    • +
    • client token - set to "N/A"
    • +
    • diagnostics - set to "N/A"
    • +
    • tracking URL - set to "N/A"
    • +
    • original tracking URL - set to "N/A"
    • +
    • resource usage report - all values are -1
    • +
    + + @param appId + {@link ApplicationId} of the application that needs a report + @return application report + @throws YarnException + @throws IOException]]> +
    +
    + + + + + + Get a report (ApplicationReport) of all Applications in the cluster. +

    + +

    + If the user does not have VIEW_APP access for an application + then the corresponding report will be filtered as described in + {@link #getApplicationReport(ApplicationId)}. +

    + + @return a list of reports for all applications + @throws YarnException + @throws IOException]]> +
    +
    + + + + + + + Get a report of the given ApplicationAttempt. +

    + +

    + In secure mode, YARN verifies access to the application, queue + etc. before accepting the request. +

    + + @param applicationAttemptId + {@link ApplicationAttemptId} of the application attempt that needs + a report + @return application attempt report + @throws YarnException + @throws ApplicationAttemptNotFoundException if application attempt + not found + @throws IOException]]> +
    +
    + + + + + + + Get a report of all (ApplicationAttempts) of Application in the cluster. +

    + + @param applicationId + @return a list of reports for all application attempts for specified + application + @throws YarnException + @throws IOException]]> +
    +
    + + + + + + + Get a report of the given Container. +

    + +

    + In secure mode, YARN verifies access to the application, queue + etc. before accepting the request. +

    + + @param containerId + {@link ContainerId} of the container that needs a report + @return container report + @throws YarnException + @throws ContainerNotFoundException if container not found + @throws IOException]]> +
    +
    + + + + + + + Get a report of all (Containers) of ApplicationAttempt in the cluster. +

    + + @param applicationAttemptId + @return a list of reports of all containers for specified application + attempt + @throws YarnException + @throws IOException]]> +
    +
    +
    + + + + + + + + + {@code + AMRMClient.createAMRMClientContainerRequest() + } + @return the newly create AMRMClient instance.]]> + + + + + + + + + + + + + + + + RegisterApplicationMasterResponse + @throws YarnException + @throws IOException]]> + + + + + + + + + + + RegisterApplicationMasterResponse + @throws YarnException + @throws IOException]]> + + + + + + + + addContainerRequest are sent to the + ResourceManager. New containers assigned to the master are + retrieved. Status of completed containers and node health updates are also + retrieved. This also doubles up as a heartbeat to the ResourceManager and + must be made periodically. The call may not always return any new + allocations of containers. App should not make concurrent allocate + requests. May cause request loss. + +

    + Note : If the user has not removed container requests that have already + been satisfied, then the re-register may end up sending the entire + container requests to the RM (including matched requests). Which would mean + the RM could end up giving it a lot of new allocated containers. +

    + + @param progressIndicator Indicates progress made by the master + @return the response of the allocate request + @throws YarnException + @throws IOException]]> +
    +
    + + + + + + + + + + + + + + allocate + @param req Resource request]]> + + + + + + + + + + + + + allocate. + Any previous pending resource change request of the same container will be + removed. + + Application that calls this method is expected to maintain the + Containers that are returned from previous successful + allocations or resource changes. By passing in the existing container and a + target resource capability to this method, the application requests the + ResourceManager to change the existing resource allocation to the target + resource allocation. + + @deprecated use + {@link #requestContainerUpdate(Container, UpdateContainerRequest)} + + @param container The container returned from the last successful resource + allocation or resource change + @param capability The target resource capability of the container]]> + + + + + + + allocate. + Any previous pending update request of the same container will be + removed. + + @param container The container returned from the last successful resource + allocation or update + @param updateContainerRequest The UpdateContainerRequest.]]> + + + + + + + + + + + + + + + + + + + + + + + + ContainerRequests matching the given + parameters. These ContainerRequests should have been added via + addContainerRequest earlier in the lifecycle. For performance, + the AMRMClient may return its internal collection directly without creating + a copy. Users should not perform mutable operations on the return value. + Each collection in the list contains requests with identical + Resource size that fit in the given capability. In a + collection, requests will be returned in the same order as they were added. + + NOTE: This API only matches Container requests that were created by the + client WITHOUT the allocationRequestId being set. + + @return Collection of request matching the parameters]]> + + + + + + + + + ContainerRequests matching the given + parameters. These ContainerRequests should have been added via + addContainerRequest earlier in the lifecycle. For performance, + the AMRMClient may return its internal collection directly without creating + a copy. Users should not perform mutable operations on the return value. + Each collection in the list contains requests with identical + Resource size that fit in the given capability. In a + collection, requests will be returned in the same order as they were added. + specify an ExecutionType. + + NOTE: This API only matches Container requests that were created by the + client WITHOUT the allocationRequestId being set. + + @param priority Priority + @param resourceName Location + @param executionType ExecutionType + @param capability Capability + @return Collection of request matching the parameters]]> + + + + + + + + + + + + + ContainerRequests matching the given + allocationRequestId. These ContainerRequests should have been added via + addContainerRequest earlier in the lifecycle. For performance, + the AMRMClient may return its internal collection directly without creating + a copy. Users should not perform mutable operations on the return value. + + NOTE: This API only matches Container requests that were created by the + client WITH the allocationRequestId being set to a non-default value. + + @param allocationRequestId Allocation Request Id + @return Collection of request matching the parameters]]> + + + + + + + + + + + + + AMRMClient. This cache must + be shared with the {@link NMClient} used to manage containers for the + AMRMClient +

    + If a NM token cache is not set, the {@link NMTokenCache#getSingleton()} + singleton instance will be used. + + @param nmTokenCache the NM token cache to use.]]> + + + + + AMRMClient. This cache must be + shared with the {@link NMClient} used to manage containers for the + AMRMClient. +

    + If a NM token cache is not set, the {@link NMTokenCache#getSingleton()} + singleton instance will be used. + + @return the NM token cache.]]> + + + + + + + + + + + + + + + + + + + + + + + + check to return true for each 1000 ms. + See also {@link #waitFor(java.util.function.Supplier, int)} + and {@link #waitFor(java.util.function.Supplier, int, int)} + @param check the condition for which it should wait]]> + + + + + + + + check to return true for each + checkEveryMillis ms. + See also {@link #waitFor(java.util.function.Supplier, int, int)} + @param check user defined checker + @param checkEveryMillis interval to call check]]> + + + + + + + + + check to return true for each + checkEveryMillis ms. In the main loop, this method will log + the message "waiting in main loop" for each logInterval times + iteration to confirm the thread is alive. + @param check user defined checker + @param checkEveryMillis interval to call check + @param logInterval interval to log for each]]> + + + + + + + + + + + + + + + + + + + + + + + + + + Start an allocated container.

    + +

    The ApplicationMaster or other applications that use the + client must provide the details of the allocated container, including the + Id, the assigned node's Id and the token via {@link Container}. In + addition, the AM needs to provide the {@link ContainerLaunchContext} as + well.

    + + @param container the allocated container + @param containerLaunchContext the context information needed by the + NodeManager to launch the + container + @return a map between the auxiliary service names and their outputs + @throws YarnException YarnException. + @throws IOException IOException.]]> +
    +
    + + + + + + Increase the resource of a container.

    + +

    The ApplicationMaster or other applications that use the + client must provide the details of the container, including the Id and + the target resource encapsulated in the updated container token via + {@link Container}. +

    + + @param container the container with updated token. + + @throws YarnException YarnException. + @throws IOException IOException.]]> +
    +
    + + + + + + Update the resources of a container.

    + +

    The ApplicationMaster or other applications that use the + client must provide the details of the container, including the Id and + the target resource encapsulated in the updated container token via + {@link Container}. +

    + + @param container the container with updated token. + + @throws YarnException YarnException. + @throws IOException IOException.]]> +
    +
    + + + + + + + Stop an started container.

    + + @param containerId the Id of the started container + @param nodeId the Id of the NodeManager + + @throws YarnException YarnException. + @throws IOException IOException.]]> +
    +
    + + + + + + + Query the status of a container.

    + + @param containerId the Id of the started container + @param nodeId the Id of the NodeManager + + @return the status of a container. + + @throws YarnException YarnException. + @throws IOException IOException.]]> +
    +
    + + + + + + + + Re-Initialize the Container.

    + + @param containerId the Id of the container to Re-Initialize. + @param containerLaunchContex the updated ContainerLaunchContext. + @param autoCommit commit re-initialization automatically ? + + @throws YarnException YarnException. + @throws IOException IOException.]]> +
    +
    + + + + + + Restart the specified container.

    + + @param containerId the Id of the container to restart. + + @throws YarnException YarnException. + @throws IOException IOException.]]> +
    +
    + + + + + + Rollback last reInitialization of the specified container.

    + + @param containerId the Id of the container to restart. + + @throws YarnException YarnException. + @throws IOException IOException.]]> +
    +
    + + + + + + Commit last reInitialization of the specified container.

    + + @param containerId the Id of the container to commit reInitialize. + + @throws YarnException YarnException. + @throws IOException IOException.]]> +
    +
    + + + + Set whether the containers that are started by this client, and are + still running should be stopped when the client stops. By default, the + feature should be enabled.

    However, containers will be stopped only + when service is stopped. i.e. after {@link NMClient#stop()}. + + @param enabled whether the feature is enabled or not]]> +
    +
    + + + + NMClient. This cache must be + shared with the {@link AMRMClient} that requested the containers managed + by this NMClient +

    + If a NM token cache is not set, the {@link NMTokenCache#getSingleton()} + singleton instance will be used. + + @param nmTokenCache the NM token cache to use.]]> + + + + + NMClient. This cache must be + shared with the {@link AMRMClient} that requested the containers managed + by this NMClient +

    + If a NM token cache is not set, the {@link NMTokenCache#getSingleton()} + singleton instance will be used. + + @return the NM token cache]]> + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + By default YARN client libraries {@link AMRMClient} and {@link NMClient} use + {@link #getSingleton()} instance of the cache. +

      +
    • + Using the singleton instance of the cache is appropriate when running a + single ApplicationMaster in the same JVM. +
    • +
    • + When using the singleton, users don't need to do anything special, + {@link AMRMClient} and {@link NMClient} are already set up to use the + default singleton {@link NMTokenCache} +
    • +
    + If running multiple Application Masters in the same JVM, a different cache + instance should be used for each Application Master. +
      +
    • + If using the {@link AMRMClient} and the {@link NMClient}, setting up + and using an instance cache is as follows: +
      +   NMTokenCache nmTokenCache = new NMTokenCache();
      +   AMRMClient rmClient = AMRMClient.createAMRMClient();
      +   NMClient nmClient = NMClient.createNMClient();
      +   nmClient.setNMTokenCache(nmTokenCache);
      +   ...
      + 
      +
    • +
    • + If using the {@link AMRMClientAsync} and the {@link NMClientAsync}, + setting up and using an instance cache is as follows: +
      +   NMTokenCache nmTokenCache = new NMTokenCache();
      +   AMRMClient rmClient = AMRMClient.createAMRMClient();
      +   NMClient nmClient = NMClient.createNMClient();
      +   nmClient.setNMTokenCache(nmTokenCache);
      +   AMRMClientAsync rmClientAsync = new AMRMClientAsync(rmClient, 1000, [AMRM_CALLBACK]);
      +   NMClientAsync nmClientAsync = new NMClientAsync("nmClient", nmClient, [NM_CALLBACK]);
      +   ...
      + 
      +
    • +
    • + If using {@link ApplicationMasterProtocol} and + {@link ContainerManagementProtocol} directly, setting up and using an + instance cache is as follows: +
      +   NMTokenCache nmTokenCache = new NMTokenCache();
      +   ...
      +   ApplicationMasterProtocol amPro = ClientRMProxy.createRMProxy(conf, ApplicationMasterProtocol.class);
      +   ...
      +   AllocateRequest allocateRequest = ...
      +   ...
      +   AllocateResponse allocateResponse = rmClient.allocate(allocateRequest);
      +   for (NMToken token : allocateResponse.getNMTokens()) {
      +     nmTokenCache.setToken(token.getNodeId().toString(), token.getToken());
      +   }
      +   ...
      +   ContainerManagementProtocolProxy nmPro = ContainerManagementProtocolProxy(conf, nmTokenCache);
      +   ...
      +   nmPro.startContainer(container, containerContext);
      +   ...
      + 
      +
    • +
    + It is also possible to mix the usage of a client ({@code AMRMClient} or + {@code NMClient}, or the async versions of them) with a protocol proxy + ({@code ContainerManagementProtocolProxy} or + {@code ApplicationMasterProtocol}).]]> +
    +
    + + + + + + + + + + + + + + The method to claim a resource with the SharedCacheManager. + The client uses a checksum to identify the resource and an + {@link ApplicationId} to identify which application will be using the + resource. +

    + +

    + The SharedCacheManager responds with whether or not the + resource exists in the cache. If the resource exists, a URL to + the resource in the shared cache is returned. If the resource does not + exist, null is returned instead. +

    + +

    + Once a URL has been returned for a resource, that URL is safe to use for + the lifetime of the application that corresponds to the provided + ApplicationId. +

    + + @param applicationId ApplicationId of the application using the resource + @param resourceKey the key (i.e. checksum) that identifies the resource + @return URL to the resource, or null if it does not exist]]> +
    +
    + + + + + + + The method to release a resource with the SharedCacheManager. + This method is called once an application is no longer using a claimed + resource in the shared cache. The client uses a checksum to identify the + resource and an {@link ApplicationId} to identify which application is + releasing the resource. +

    + +

    + Note: This method is an optimization and the client is not required to call + it for correctness. +

    + + @param applicationId ApplicationId of the application releasing the + resource + @param resourceKey the key (i.e. checksum) that identifies the resource]]> +
    +
    + + + + + + + + + + +
    + + + + + + + + + + + + + + + + Obtain a {@link YarnClientApplication} for a new application, + which in turn contains the {@link ApplicationSubmissionContext} and + {@link org.apache.hadoop.yarn.api.protocolrecords.GetNewApplicationResponse} + objects. +

    + + @return {@link YarnClientApplication} built for a new application + @throws YarnException + @throws IOException]]> +
    +
    + + + + + + + Submit a new application to YARN. It is a blocking call - it + will not return {@link ApplicationId} until the submitted application is + submitted successfully and accepted by the ResourceManager. +

    + +

    + Users should provide an {@link ApplicationId} as part of the parameter + {@link ApplicationSubmissionContext} when submitting a new application, + otherwise it will throw the {@link ApplicationIdNotProvidedException}. +

    + +

    This internally calls {@link ApplicationClientProtocol#submitApplication + (SubmitApplicationRequest)}, and after that, it internally invokes + {@link ApplicationClientProtocol#getApplicationReport + (GetApplicationReportRequest)} and waits till it can make sure that the + application gets properly submitted. If RM fails over or RM restart + happens before ResourceManager saves the application's state, + {@link ApplicationClientProtocol + #getApplicationReport(GetApplicationReportRequest)} will throw + the {@link ApplicationNotFoundException}. This API automatically resubmits + the application with the same {@link ApplicationSubmissionContext} when it + catches the {@link ApplicationNotFoundException}

    + + @param appContext + {@link ApplicationSubmissionContext} containing all the details + needed to submit a new application + @return {@link ApplicationId} of the accepted application + @throws YarnException + @throws IOException + @see #createApplication()]]> +
    +
    + + + + + + + Fail an application attempt identified by given ID. +

    + + @param applicationAttemptId + {@link ApplicationAttemptId} of the attempt to fail. + @throws YarnException + in case of errors or if YARN rejects the request due to + access-control restrictions. + @throws IOException + @see #getQueueAclsInfo()]]> +
    +
    + + + + + + + Kill an application identified by given ID. +

    + + @param applicationId + {@link ApplicationId} of the application that needs to be killed + @throws YarnException + in case of errors or if YARN rejects the request due to + access-control restrictions. + @throws IOException + @see #getQueueAclsInfo()]]> +
    +
    + + + + + + + + Kill an application identified by given ID. +

    + @param applicationId {@link ApplicationId} of the application that needs to + be killed + @param diagnostics for killing an application. + @throws YarnException in case of errors or if YARN rejects the request due + to access-control restrictions. + @throws IOException]]> +
    +
    + + + + + + + Get a report of the given Application. +

    + +

    + In secure mode, YARN verifies access to the application, queue + etc. before accepting the request. +

    + +

    + If the user does not have VIEW_APP access then the following + fields in the report will be set to stubbed values: +

      +
    • host - set to "N/A"
    • +
    • RPC port - set to -1
    • +
    • client token - set to "N/A"
    • +
    • diagnostics - set to "N/A"
    • +
    • tracking URL - set to "N/A"
    • +
    • original tracking URL - set to "N/A"
    • +
    • resource usage report - all values are -1
    • +
    + + @param appId + {@link ApplicationId} of the application that needs a report + @return application report + @throws YarnException + @throws IOException]]> +
    +
    + + + + + + + The AMRM token is required for AM to RM scheduling operations. For + managed Application Masters YARN takes care of injecting it. For unmanaged + Applications Masters, the token must be obtained via this method and set + in the {@link org.apache.hadoop.security.UserGroupInformation} of the + current user. +

    + The AMRM token will be returned only if all the following conditions are + met: +

      +
    • the requester is the owner of the ApplicationMaster
    • +
    • the application master is an unmanaged ApplicationMaster
    • +
    • the application master is in ACCEPTED state
    • +
    + Else this method returns NULL. + + @param appId {@link ApplicationId} of the application to get the AMRM token + @return the AMRM token if available + @throws YarnException + @throws IOException]]> +
    +
    + + + + + + Get a report (ApplicationReport) of all Applications in the cluster. +

    + +

    + If the user does not have VIEW_APP access for an application + then the corresponding report will be filtered as described in + {@link #getApplicationReport(ApplicationId)}. +

    + + @return a list of reports of all running applications + @throws YarnException + @throws IOException]]> +
    +
    + + + + + + + Get a report (ApplicationReport) of Applications + matching the given application types in the cluster. +

    + +

    + If the user does not have VIEW_APP access for an application + then the corresponding report will be filtered as described in + {@link #getApplicationReport(ApplicationId)}. +

    + + @param applicationTypes set of application types you are interested in + @return a list of reports of applications + @throws YarnException + @throws IOException]]> +
    +
    + + + + + + + Get a report (ApplicationReport) of Applications matching the given + application states in the cluster. +

    + +

    + If the user does not have VIEW_APP access for an application + then the corresponding report will be filtered as described in + {@link #getApplicationReport(ApplicationId)}. +

    + + @param applicationStates set of application states you are interested in + @return a list of reports of applications + @throws YarnException + @throws IOException]]> +
    +
    + + + + + + + + Get a report (ApplicationReport) of Applications matching the given + application types and application states in the cluster. +

    + +

    + If the user does not have VIEW_APP access for an application + then the corresponding report will be filtered as described in + {@link #getApplicationReport(ApplicationId)}. +

    + + @param applicationTypes set of application types you are interested in + @param applicationStates set of application states you are interested in + @return a list of reports of applications + @throws YarnException + @throws IOException]]> +
    +
    + + + + + + + + + Get a report (ApplicationReport) of Applications matching the given + application types, application states and application tags in the cluster. +

    + +

    + If the user does not have VIEW_APP access for an application + then the corresponding report will be filtered as described in + {@link #getApplicationReport(ApplicationId)}. +

    + + @param applicationTypes set of application types you are interested in + @param applicationStates set of application states you are interested in + @param applicationTags set of application tags you are interested in + @return a list of reports of applications + @throws YarnException + @throws IOException]]> +
    +
    + + + + + + + + + + Get a report (ApplicationReport) of Applications matching the given users, + queues, application types and application states in the cluster. If any of + the params is set to null, it is not used when filtering. +

    + +

    + If the user does not have VIEW_APP access for an application + then the corresponding report will be filtered as described in + {@link #getApplicationReport(ApplicationId)}. +

    + + @param queues set of queues you are interested in + @param users set of users you are interested in + @param applicationTypes set of application types you are interested in + @param applicationStates set of application states you are interested in + @return a list of reports of applications + @throws YarnException + @throws IOException]]> +
    +
    + + + + + + + Get a list of ApplicationReports that match the given + {@link GetApplicationsRequest}. +

    + +

    + If the user does not have VIEW_APP access for an application + then the corresponding report will be filtered as described in + {@link #getApplicationReport(ApplicationId)}. +

    + + @param request the request object to get the list of applications. + @return The list of ApplicationReports that match the request + @throws YarnException Exception specific to YARN. + @throws IOException Exception mostly related to connection errors.]]> +
    +
    + + + + + + Get metrics ({@link YarnClusterMetrics}) about the cluster. +

    + + @return cluster metrics + @throws YarnException + @throws IOException]]> +
    +
    + + + + + + + Get a report of nodes ({@link NodeReport}) in the cluster. +

    + + @param states The {@link NodeState}s to filter on. If no filter states are + given, nodes in all states will be returned. + @return A list of node reports + @throws YarnException + @throws IOException]]> +
    +
    + + + + + + + Get a delegation token so as to be able to talk to YARN using those tokens. + + @param renewer + Address of the renewer who can renew these tokens when needed by + securely talking to YARN. + @return a delegation token ({@link Token}) that can be used to + talk to YARN + @throws YarnException + @throws IOException]]> + + + + + + + + + Get information ({@link QueueInfo}) about a given queue. +

    + + @param queueName + Name of the queue whose information is needed + @return queue information + @throws YarnException + in case of errors or if YARN rejects the request due to + access-control restrictions. + @throws IOException]]> +
    +
    + + + + + + Get information ({@link QueueInfo}) about all queues, recursively if there + is a hierarchy +

    + + @return a list of queue-information for all queues + @throws YarnException + @throws IOException]]> +
    +
    + + + + + + Get information ({@link QueueInfo}) about top level queues. +

    + + @return a list of queue-information for all the top-level queues + @throws YarnException + @throws IOException]]> +
    +
    + + + + + + + Get information ({@link QueueInfo}) about all the immediate children queues + of the given queue +

    + + @param parent + Name of the queue whose child-queues' information is needed + @return a list of queue-information for all queues who are direct children + of the given parent queue. + @throws YarnException + @throws IOException]]> +
    +
    + + + + + + Get information about acls for current user on all the + existing queues. +

    + + @return a list of queue acls ({@link QueueUserACLInfo}) for + current user + @throws YarnException + @throws IOException]]> +
    +
    + + + + + + + Get a report of the given ApplicationAttempt. +

    + +

    + In secure mode, YARN verifies access to the application, queue + etc. before accepting the request. +

    + + @param applicationAttemptId + {@link ApplicationAttemptId} of the application attempt that needs + a report + @return application attempt report + @throws YarnException + @throws ApplicationAttemptNotFoundException if application attempt + not found + @throws IOException]]> +
    +
    + + + + + + + Get a report of all (ApplicationAttempts) of Application in the cluster. +

    + + @param applicationId application id of the app + @return a list of reports for all application attempts for specified + application. + @throws YarnException + @throws IOException]]> +
    +
    + + + + + + + Get a report of the given Container. +

    + +

    + In secure mode, YARN verifies access to the application, queue + etc. before accepting the request. +

    + + @param containerId + {@link ContainerId} of the container that needs a report + @return container report + @throws YarnException + @throws ContainerNotFoundException if container not found. + @throws IOException]]> +
    +
    + + + + + + + Get a report of all (Containers) of ApplicationAttempt in the cluster. +

    + + @param applicationAttemptId application attempt id + @return a list of reports of all containers for specified application + attempts + @throws YarnException + @throws IOException]]> +
    +
    + + + + + + + + Attempts to move the given application to the given queue. +

    + + @param appId + Application to move. + @param queue + Queue to place it in to. + @throws YarnException + @throws IOException]]> +
    +
    + + + + + + Obtain a {@link GetNewReservationResponse} for a new reservation, + which contains the {@link ReservationId} object. +

    + + @return The {@link GetNewReservationResponse} containing a new + {@link ReservationId} object. + @throws YarnException if reservation cannot be created. + @throws IOException if reservation cannot be created.]]> +
    +
    + + + + + + + The interface used by clients to submit a new reservation to the + {@code ResourceManager}. +

    + +

    + The client packages all details of its request in a + {@link ReservationSubmissionRequest} object. This contains information + about the amount of capacity, temporal constraints, and gang needs. + Furthermore, the reservation might be composed of multiple stages, with + ordering dependencies among them. +

    + +

    + In order to respond, a new admission control component in the + {@code ResourceManager} performs an analysis of the resources that have + been committed over the period of time the user is requesting, verify that + the user requests can be fulfilled, and that it respect a sharing policy + (e.g., {@code CapacityOverTimePolicy}). Once it has positively determined + that the ReservationRequest is satisfiable the {@code ResourceManager} + answers with a {@link ReservationSubmissionResponse} that includes a + {@link ReservationId}. Upon failure to find a valid allocation the response + is an exception with the message detailing the reason of failure. +

    + +

    + The semantics guarantees that the {@link ReservationId} returned, + corresponds to a valid reservation existing in the time-range request by + the user. The amount of capacity dedicated to such reservation can vary + overtime, depending of the allocation that has been determined. But it is + guaranteed to satisfy all the constraint expressed by the user in the + {@link ReservationDefinition} +

    + + @param request request to submit a new Reservation + @return response contains the {@link ReservationId} on accepting the + submission + @throws YarnException if the reservation cannot be created successfully + @throws IOException]]> +
    +
    + + + + + + + The interface used by clients to update an existing Reservation. This is + referred to as a re-negotiation process, in which a user that has + previously submitted a Reservation. +

    + +

    + The allocation is attempted by virtually substituting all previous + allocations related to this Reservation with new ones, that satisfy the new + {@link ReservationDefinition}. Upon success the previous allocation is + atomically substituted by the new one, and on failure (i.e., if the system + cannot find a valid allocation for the updated request), the previous + allocation remains valid. +

    + + @param request to update an existing Reservation (the + {@link ReservationUpdateRequest} should refer to an existing valid + {@link ReservationId}) + @return response empty on successfully updating the existing reservation + @throws YarnException if the request is invalid or reservation cannot be + updated successfully + @throws IOException]]> +
    +
    + + + + + + + The interface used by clients to remove an existing Reservation. +

    + + @param request to remove an existing Reservation (the + {@link ReservationDeleteRequest} should refer to an existing valid + {@link ReservationId}) + @return response empty on successfully deleting the existing reservation + @throws YarnException if the request is invalid or reservation cannot be + deleted successfully + @throws IOException]]> +
    +
    + + + + + + + The interface used by clients to get the list of reservations in a plan. + The reservationId will be used to search for reservations to list if it is + provided. Otherwise, it will select active reservations within the + startTime and endTime (inclusive). +

    + + @param request to list reservations in a plan. Contains fields to select + String queue, ReservationId reservationId, long startTime, + long endTime, and a bool includeReservationAllocations. + + queue: Required. Cannot be null or empty. Refers to the + reservable queue in the scheduler that was selected when + creating a reservation submission + {@link ReservationSubmissionRequest}. + + reservationId: Optional. If provided, other fields will + be ignored. + + startTime: Optional. If provided, only reservations that + end after the startTime will be selected. This defaults + to 0 if an invalid number is used. + + endTime: Optional. If provided, only reservations that + start on or before endTime will be selected. This defaults + to Long.MAX_VALUE if an invalid number is used. + + includeReservationAllocations: Optional. Flag that + determines whether the entire reservation allocations are + to be returned. Reservation allocations are subject to + change in the event of re-planning as described by + {@link ReservationDefinition}. + + @return response that contains information about reservations that are + being searched for. + @throws YarnException if the request is invalid + @throws IOException if the request failed otherwise]]> +
    +
    + + + + + + The interface used by client to get node to labels mappings in existing cluster +

    + + @return node to labels mappings + @throws YarnException + @throws IOException]]> +
    +
    + + + + + + The interface used by client to get labels to nodes mapping + in existing cluster +

    + + @return node to labels mappings + @throws YarnException + @throws IOException]]> +
    +
    + + + + + + + The interface used by client to get labels to nodes mapping + for specified labels in existing cluster +

    + + @param labels labels for which labels to nodes mapping has to be retrieved + @return labels to nodes mappings for specific labels + @throws YarnException + @throws IOException]]> +
    +
    + + + + + + The interface used by client to get node labels in the cluster +

    + + @return cluster node labels collection + @throws YarnException when there is a failure in + {@link ApplicationClientProtocol} + @throws IOException when there is a failure in + {@link ApplicationClientProtocol}]]> +
    +
    + + + + + + + + The interface used by client to set priority of an application +

    + @param applicationId + @param priority + @return updated priority of an application. + @throws YarnException + @throws IOException]]> +
    +
    + + + + + + + + Signal a container identified by given ID. +

    + + @param containerId + {@link ContainerId} of the container that needs to be signaled + @param command the signal container command + @throws YarnException + @throws IOException]]> +
    +
    + + + + + + + + + + + Get the resource profiles available in the RM. +

    + @return a Map of the resource profile names to their capabilities + @throws YARNFeatureNotEnabledException if resource-profile is disabled + @throws YarnException if any error happens inside YARN + @throws IOException in case of other errors]]> +
    +
    + + + + + + + Get the details of a specific resource profile from the RM. +

    + @param profile the profile name + @return resource profile name with its capabilities + @throws YARNFeatureNotEnabledException if resource-profile is disabled + @throws YarnException if any error happens inside YARN + @throws IOException in case of other others]]> +
    +
    + + + + + + Get available resource types supported by RM. +

    + @return list of supported resource types with detailed information + @throws YarnException if any issue happens inside YARN + @throws IOException in case of other others]]> +
    +
    + + + + + + The interface used by client to get node attributes in the cluster. +

    + + @return cluster node attributes collection + @throws YarnException when there is a failure in + {@link ApplicationClientProtocol} + @throws IOException when there is a failure in + {@link ApplicationClientProtocol}]]> +
    +
    + + + + + + + The interface used by client to get mapping of AttributeKey to associated + NodeToAttributeValue list for specified node attributeKeys in the cluster. +

    + + @param attributes AttributeKeys for which associated NodeToAttributeValue + mapping value has to be retrieved. If empty or null is set then + will return mapping for all attributeKeys in the cluster + @return mapping of AttributeKey to List of associated + NodeToAttributeValue's. + @throws YarnException + @throws IOException]]> +
    +
    + + + + + + + The interface used by client to get all node to attribute mapping in + existing cluster. +

    + + @param hostNames HostNames for which host to attributes mapping has to + be retrived.If empty or null is set then will return + all nodes to attributes mapping in cluster. + @return Node to attribute mappings + @throws YarnException + @throws IOException]]> +
    +
    + + + + + + + The interface used by client to get a shell to a container. +

    + + @param containerId Container ID + @param command Shell type + @throws IOException if connection fails.]]> +
    +
    +
    + + + + + + + + + + + +
    + + + + + + + + + + + + + + + + Create a new instance of AMRMClientAsync.

    + + @param intervalMs heartbeat interval in milliseconds between AM and RM + @param callbackHandler callback handler that processes responses from + the ResourceManager]]> +
    +
    + + + + + + Create a new instance of AMRMClientAsync.

    + + @param client the AMRMClient instance + @param intervalMs heartbeat interval in milliseconds between AM and RM + @param callbackHandler callback handler that processes responses from + the ResourceManager]]> +
    +
    + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + RegisterApplicationMasterResponse + @throws YarnException + @throws IOException]]> + + + + + + + + + + + + + + + + allocate + @param req Resource request]]> + + + + + + + + + + + + + allocate. + Any previous pending resource change request of the same container will be + removed. + + Application that calls this method is expected to maintain the + Containers that are returned from previous successful + allocations or resource changes. By passing in the existing container and a + target resource capability to this method, the application requests the + ResourceManager to change the existing resource allocation to the target + resource allocation. + + @deprecated use + {@link #requestContainerUpdate(Container, UpdateContainerRequest)} + + @param container The container returned from the last successful resource + allocation or resource change + @param capability The target resource capability of the container]]> + + + + + + + allocate. + Any previous pending update request of the same container will be + removed. + + @param container The container returned from the last successful resource + allocation or update + @param updateContainerRequest The UpdateContainerRequest.]]> + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + check to return true for each 1000 ms. + See also {@link #waitFor(java.util.function.Supplier, int)} + and {@link #waitFor(java.util.function.Supplier, int, int)} + @param check the condition for which it should wait]]> + + + + + + + + check to return true for each + checkEveryMillis ms. + See also {@link #waitFor(java.util.function.Supplier, int, int)} + @param check user defined checker + @param checkEveryMillis interval to call check]]> + + + + + + + + + check to return true for each + checkEveryMillis ms. In the main loop, this method will log + the message "waiting in main loop" for each logInterval times + iteration to confirm the thread is alive. + @param check user defined checker + @param checkEveryMillis interval to call check + @param logInterval interval to log for each]]> + + + + + + + + + + AMRMClientAsync handles communication with the ResourceManager + and provides asynchronous updates on events such as container allocations and + completions. It contains a thread that sends periodic heartbeats to the + ResourceManager. + + It should be used by implementing a CallbackHandler: +
    + {@code
    + class MyCallbackHandler extends AMRMClientAsync.AbstractCallbackHandler {
    +   public void onContainersAllocated(List containers) {
    +     [run tasks on the containers]
    +   }
    +
    +   public void onContainersUpdated(List containers) {
    +     [determine if resource allocation of containers have been increased in
    +      the ResourceManager, and if so, inform the NodeManagers to increase the
    +      resource monitor/enforcement on the containers]
    +   }
    +
    +   public void onContainersCompleted(List statuses) {
    +     [update progress, check whether app is done]
    +   }
    +   
    +   public void onNodesUpdated(List updated) {}
    +   
    +   public void onReboot() {}
    + }
    + }
    + 
    + + The client's lifecycle should be managed similarly to the following: + +
    + {@code
    + AMRMClientAsync asyncClient = 
    +     createAMRMClientAsync(appAttId, 1000, new MyCallbackhandler());
    + asyncClient.init(conf);
    + asyncClient.start();
    + RegisterApplicationMasterResponse response = asyncClient
    +    .registerApplicationMaster(appMasterHostname, appMasterRpcPort,
    +       appMasterTrackingUrl);
    + asyncClient.addContainerRequest(containerRequest);
    + [... wait for application to complete]
    + asyncClient.unregisterApplicationMaster(status, appMsg, trackingUrl);
    + asyncClient.stop();
    + }
    + 
    ]]> +
    +
    + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + Update the resources of a container.

    + +

    The ApplicationMaster or other applications that use the + client must provide the details of the container, including the Id and + the target resource encapsulated in the updated container token via + {@link Container}. +

    + + @param container the container with updated token.]]> +
    +
    + + + + + + Re-Initialize the Container.

    + + @param containerId the Id of the container to Re-Initialize. + @param containerLaunchContex the updated ContainerLaunchContext. + @param autoCommit commit re-initialization automatically ?]]> +
    +
    + + + + Restart the specified container.

    + + @param containerId the Id of the container to restart.]]> +
    +
    + + + + Rollback last reInitialization of the specified container.

    + + @param containerId the Id of the container to restart.]]> +
    +
    + + + + Commit last reInitialization of the specified container.

    + + @param containerId the Id of the container to commit reInitialize.]]> +
    +
    + + + + + + + + + + + + + + + + + + + + + + + + NMClientAsync handles communication with all the NodeManagers + and provides asynchronous updates on getting responses from them. It + maintains a thread pool to communicate with individual NMs where a number of + worker threads process requests to NMs by using {@link NMClientImpl}. The max + size of the thread pool is configurable through + {@link YarnConfiguration#NM_CLIENT_ASYNC_THREAD_POOL_MAX_SIZE}. + + It should be used in conjunction with a CallbackHandler. For example + +
    + {@code
    + class MyCallbackHandler extends NMClientAsync.AbstractCallbackHandler {
    +   public void onContainerStarted(ContainerId containerId,
    +       Map allServiceResponse) {
    +     [post process after the container is started, process the response]
    +   }
    +
    +   public void onContainerResourceIncreased(ContainerId containerId,
    +       Resource resource) {
    +     [post process after the container resource is increased]
    +   }
    +
    +   public void onContainerStatusReceived(ContainerId containerId,
    +       ContainerStatus containerStatus) {
    +     [make use of the status of the container]
    +   }
    +
    +   public void onContainerStopped(ContainerId containerId) {
    +     [post process after the container is stopped]
    +   }
    +
    +   public void onStartContainerError(
    +       ContainerId containerId, Throwable t) {
    +     [handle the raised exception]
    +   }
    +
    +   public void onGetContainerStatusError(
    +       ContainerId containerId, Throwable t) {
    +     [handle the raised exception]
    +   }
    +
    +   public void onStopContainerError(
    +       ContainerId containerId, Throwable t) {
    +     [handle the raised exception]
    +   }
    + }
    + }
    + 
    + + The client's life-cycle should be managed like the following: + +
    + {@code
    + NMClientAsync asyncClient = 
    +     NMClientAsync.createNMClientAsync(new MyCallbackhandler());
    + asyncClient.init(conf);
    + asyncClient.start();
    + asyncClient.startContainer(container, containerLaunchContext);
    + [... wait for container being started]
    + asyncClient.getContainerStatus(container.getId(), container.getNodeId(),
    +     container.getContainerToken());
    + [... handle the status in the callback instance]
    + asyncClient.stopContainer(container.getId(), container.getNodeId(),
    +     container.getContainerToken());
    + [... wait for container being stopped]
    + asyncClient.stop();
    + }
    + 
    ]]> +
    +
    + +
    + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + +
    diff --git a/hadoop-yarn-project/hadoop-yarn/dev-support/jdiff/Apache_Hadoop_YARN_Common_2.10.2.xml b/hadoop-yarn-project/hadoop-yarn/dev-support/jdiff/Apache_Hadoop_YARN_Common_2.10.2.xml new file mode 100644 index 0000000000000..2dfd16a259037 --- /dev/null +++ b/hadoop-yarn-project/hadoop-yarn/dev-support/jdiff/Apache_Hadoop_YARN_Common_2.10.2.xml @@ -0,0 +1,2936 @@ + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + Type of proxy. + @return Proxy to the ResourceManager for the specified client protocol. + @throws IOException]]> + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + Send the information of a number of conceptual entities to the timeline + server. It is a blocking API. The method will not return until it gets the + response from the timeline server. +

    + + @param entities + the collection of {@link TimelineEntity} + @return the error information if the sent entities are not correctly stored + @throws IOException if there are I/O errors + @throws YarnException if entities are incomplete/invalid]]> +
    +
    + + + + + + + + + Send the information of a number of conceptual entities to the timeline + server. It is a blocking API. The method will not return until it gets the + response from the timeline server. + + This API is only for timeline service v1.5 +

    + + @param appAttemptId {@link ApplicationAttemptId} + @param groupId {@link TimelineEntityGroupId} + @param entities + the collection of {@link TimelineEntity} + @return the error information if the sent entities are not correctly stored + @throws IOException if there are I/O errors + @throws YarnException if entities are incomplete/invalid]]> +
    +
    + + + + + + + Send the information of a domain to the timeline server. It is a + blocking API. The method will not return until it gets the response from + the timeline server. +

    + + @param domain + an {@link TimelineDomain} object + @throws IOException + @throws YarnException]]> +
    +
    + + + + + + + + Send the information of a domain to the timeline server. It is a + blocking API. The method will not return until it gets the response from + the timeline server. + + This API is only for timeline service v1.5 +

    + + @param domain + an {@link TimelineDomain} object + @param appAttemptId {@link ApplicationAttemptId} + @throws IOException + @throws YarnException]]> +
    +
    + + + + + + + Get a delegation token so as to be able to talk to the timeline server in a + secure way. +

    + + @param renewer + Address of the renewer who can renew these tokens when needed by + securely talking to the timeline server + @return a delegation token ({@link Token}) that can be used to talk to the + timeline server + @throws IOException + @throws YarnException]]> +
    +
    + + + + + + + Renew a timeline delegation token. +

    + + @param timelineDT + the delegation token to renew + @return the new expiration time + @throws IOException + @throws YarnException]]> +
    +
    + + + + + + + Cancel a timeline delegation token. +

    + + @param timelineDT + the delegation token to cancel + @throws IOException + @throws YarnException]]> +
    +
    + + + +
    + +
    + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + parameterized event of type T]]> + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + InputStream to be checksumed + @return the message digest of the input stream + @throws IOException]]> + + + + + + + + + + + + SharedCacheChecksum object based on the configurable + algorithm implementation + (see yarn.sharedcache.checksum.algo.impl) + + @return SharedCacheChecksum object]]> + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + The object type on which this state machine operates. + @param The state of the entity. + @param The external eventType to be handled. + @param The event object.]]> + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + +
    diff --git a/hadoop-yarn-project/hadoop-yarn/dev-support/jdiff/Apache_Hadoop_YARN_Common_3.3.3.xml b/hadoop-yarn-project/hadoop-yarn/dev-support/jdiff/Apache_Hadoop_YARN_Common_3.3.3.xml new file mode 100644 index 0000000000000..4797c30f5c32a --- /dev/null +++ b/hadoop-yarn-project/hadoop-yarn/dev-support/jdiff/Apache_Hadoop_YARN_Common_3.3.3.xml @@ -0,0 +1,3975 @@ + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + Type of proxy. + @return Proxy to the ResourceManager for the specified client protocol. + @throws IOException]]> + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + Create a new instance of AppAdminClient. +

    + + @param appType application type + @param conf configuration + @return app admin client]]> +
    +
    + + + + + + + + + + Launch a new YARN application. +

    + + @param fileName specification of application + @param appName name of the application + @param lifetime lifetime of the application + @param queue queue of the application + @return exit code + @throws IOException IOException + @throws YarnException exception in client or server]]> +
    +
    + + + + + + + Stop a YARN application (attempt to stop gracefully before killing the + application). In the case of a long-running service, the service may be + restarted later. +

    + + @param appName the name of the application + @return exit code + @throws IOException IOException + @throws YarnException exception in client or server]]> +
    +
    + + + + + + + Start a YARN application from a previously saved specification. In the + case of a long-running service, the service must have been previously + launched/started and then stopped, or previously saved but not started. +

    + + @param appName the name of the application + @return exit code + @throws IOException IOException + @throws YarnException exception in client or server]]> +
    +
    + + + + + + + + + + Save the specification for a YARN application / long-running service. + The application may be started later. +

    + + @param fileName specification of application to save + @param appName name of the application + @param lifetime lifetime of the application + @param queue queue of the application + @return exit code + @throws IOException IOException + @throws YarnException exception in client or server]]> +
    +
    + + + + + + + Remove the specification and all application data for a YARN application. + The application cannot be running. +

    + + @param appName the name of the application + @return exit code + @throws IOException IOException + @throws YarnException exception in client or server]]> +
    +
    + + + + + + + + Change the number of running containers for a component of a YARN + application / long-running service. +

    + + @param appName the name of the application + @param componentCounts map of component name to new component count or + amount to change existing component count (e.g. + 5, +5, -5) + @return exit code + @throws IOException IOException + @throws YarnException exception in client or server]]> +
    +
    + + + + + + + Upload AM dependencies to HDFS. This makes future application launches + faster since the dependencies do not have to be uploaded on each launch. +

    + + @param destinationFolder + an optional HDFS folder where dependency tarball will be uploaded + @return exit code + @throws IOException + IOException + @throws YarnException + exception in client or server]]> +
    +
    + + + + + + + Get detailed app specific status string for a YARN application. +

    + + @param appIdOrName appId or appName + @return status string + @throws IOException IOException + @throws YarnException exception in client or server]]> +
    +
    + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + +
    + + + + + + + + + + + + + + + + + + Send the information of a number of conceptual entities to the timeline + server. It is a blocking API. The method will not return until it gets the + response from the timeline server. +

    + + @param entities + the collection of {@link TimelineEntity} + @return the error information if the sent entities are not correctly stored + @throws IOException if there are I/O errors + @throws YarnException if entities are incomplete/invalid]]> +
    +
    + + + + + + + + + Send the information of a number of conceptual entities to the timeline + server. It is a blocking API. The method will not return until it gets the + response from the timeline server. + + This API is only for timeline service v1.5 +

    + + @param appAttemptId {@link ApplicationAttemptId} + @param groupId {@link TimelineEntityGroupId} + @param entities + the collection of {@link TimelineEntity} + @return the error information if the sent entities are not correctly stored + @throws IOException if there are I/O errors + @throws YarnException if entities are incomplete/invalid]]> +
    +
    + + + + + + + Send the information of a domain to the timeline server. It is a + blocking API. The method will not return until it gets the response from + the timeline server. +

    + + @param domain + an {@link TimelineDomain} object + @throws IOException + @throws YarnException]]> +
    +
    + + + + + + + + Send the information of a domain to the timeline server. It is a + blocking API. The method will not return until it gets the response from + the timeline server. + + This API is only for timeline service v1.5 +

    + + @param domain + an {@link TimelineDomain} object + @param appAttemptId {@link ApplicationAttemptId} + @throws IOException + @throws YarnException]]> +
    +
    + + + + + + + Get a delegation token so as to be able to talk to the timeline server in a + secure way. +

    + + @param renewer + Address of the renewer who can renew these tokens when needed by + securely talking to the timeline server + @return a delegation token ({@link Token}) that can be used to talk to the + timeline server + @throws IOException + @throws YarnException]]> +
    +
    + + + + + + + Renew a timeline delegation token. +

    + + @param timelineDT + the delegation token to renew + @return the new expiration time + @throws IOException + @throws YarnException]]> +
    +
    + + + + + + + Cancel a timeline delegation token. +

    + + @param timelineDT + the delegation token to cancel + @throws IOException + @throws YarnException]]> +
    +
    + + + +
    + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + +
    + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + parameterized event of type T]]> + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + InputStream to be checksumed + @return the message digest of the input stream + @throws IOException]]> + + + + + + + + + + + + SharedCacheChecksum object based on the configurable + algorithm implementation + (see yarn.sharedcache.checksum.algo.impl) + + @return SharedCacheChecksum object]]> + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + The object type on which this state machine operates. + @param The state of the entity. + @param The external eventType to be handled. + @param The event object.]]> + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + When {@link #limit} would be reached on append, past messages will be + truncated from head, and a header telling the user about truncation will be + prepended, with ellipses in between header and messages. +

    + Note that header and ellipses are not counted against {@link #limit}. +

    + An example: + +

    + {@code
    +   // At the beginning it's an empty string
    +   final Appendable shortAppender = new BoundedAppender(80);
    +   // The whole message fits into limit
    +   shortAppender.append(
    +       "message1 this is a very long message but fitting into limit\n");
    +   // The first message is truncated, the second not
    +   shortAppender.append("message2 this is shorter than the previous one\n");
    +   // The first message is deleted, the second truncated, the third
    +   // preserved
    +   shortAppender.append("message3 this is even shorter message, maybe.\n");
    +   // The first two are deleted, the third one truncated, the last preserved
    +   shortAppender.append("message4 the shortest one, yet the greatest :)");
    +   // Current contents are like this:
    +   // Diagnostic messages truncated, showing last 80 chars out of 199:
    +   // ...s is even shorter message, maybe.
    +   // message4 the shortest one, yet the greatest :)
    + }
    + 
    +

    + Note that null values are {@link #append(CharSequence) append}ed + just like in {@link StringBuilder#append(CharSequence) original + implementation}. +

    + Note that this class is not thread safe.]]> + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + diff --git a/hadoop-yarn-project/hadoop-yarn/dev-support/jdiff/Apache_Hadoop_YARN_Server_Common_3.3.3.xml b/hadoop-yarn-project/hadoop-yarn/dev-support/jdiff/Apache_Hadoop_YARN_Server_Common_3.3.3.xml new file mode 100644 index 0000000000000..342de75310203 --- /dev/null +++ b/hadoop-yarn-project/hadoop-yarn/dev-support/jdiff/Apache_Hadoop_YARN_Server_Common_3.3.3.xml @@ -0,0 +1,1456 @@ + + + + + + + + + + + + + + + + + + + + + + + + true if the node is healthy, else false]]> + + + + + diagnostic health report of the node. + @return diagnostic health report of the node]]> + + + + + last timestamp at which the health report was received. + @return last timestamp at which the health report was received]]> + + + + + It includes information such as: +

      +
    • + An indicator of whether the node is healthy, as determined by the + health-check script. +
    • +
    • The previous time at which the health status was reported.
    • +
    • A diagnostic report on the health status.
    • +
    + + @see NodeReport + @see ApplicationClientProtocol#getClusterNodes(org.apache.hadoop.yarn.api.protocolrecords.GetClusterNodesRequest)]]> +
    +
    + +
    + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + type of the proxy + @return the proxy instance + @throws IOException if fails to create the proxy]]> + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + true if the iteration has more elements.]]> + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + +
    diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-api/src/main/java/org/apache/hadoop/yarn/api/protocolrecords/GetLabelsToNodesResponse.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-api/src/main/java/org/apache/hadoop/yarn/api/protocolrecords/GetLabelsToNodesResponse.java index ef0bf60fa9840..f105359110418 100644 --- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-api/src/main/java/org/apache/hadoop/yarn/api/protocolrecords/GetLabelsToNodesResponse.java +++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-api/src/main/java/org/apache/hadoop/yarn/api/protocolrecords/GetLabelsToNodesResponse.java @@ -24,7 +24,6 @@ import org.apache.hadoop.classification.InterfaceAudience.Public; import org.apache.hadoop.classification.InterfaceStability.Evolving; import org.apache.hadoop.yarn.api.records.NodeId; -import org.apache.hadoop.yarn.api.records.NodeLabel; import org.apache.hadoop.yarn.util.Records; public abstract class GetLabelsToNodesResponse { diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-api/src/main/java/org/apache/hadoop/yarn/api/protocolrecords/GetNewReservationResponse.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-api/src/main/java/org/apache/hadoop/yarn/api/protocolrecords/GetNewReservationResponse.java index d7d80bff4b308..ccb22a4672264 100644 --- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-api/src/main/java/org/apache/hadoop/yarn/api/protocolrecords/GetNewReservationResponse.java +++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-api/src/main/java/org/apache/hadoop/yarn/api/protocolrecords/GetNewReservationResponse.java @@ -20,7 +20,6 @@ import org.apache.hadoop.classification.InterfaceAudience.Private; import org.apache.hadoop.classification.InterfaceAudience.Public; -import org.apache.hadoop.classification.InterfaceStability.Stable; import org.apache.hadoop.classification.InterfaceStability.Unstable; import org.apache.hadoop.yarn.api.records.ReservationId; import org.apache.hadoop.yarn.util.Records; diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-api/src/main/java/org/apache/hadoop/yarn/api/records/Resource.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-api/src/main/java/org/apache/hadoop/yarn/api/records/Resource.java index b91658f759c89..e9c7dd4a6d3f2 100644 --- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-api/src/main/java/org/apache/hadoop/yarn/api/records/Resource.java +++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-api/src/main/java/org/apache/hadoop/yarn/api/records/Resource.java @@ -19,7 +19,6 @@ package org.apache.hadoop.yarn.api.records; import java.util.ArrayList; -import java.util.Arrays; import java.util.List; import java.util.Map; @@ -30,7 +29,6 @@ import org.apache.hadoop.classification.InterfaceStability; import org.apache.hadoop.classification.InterfaceStability.Evolving; import org.apache.hadoop.classification.InterfaceStability.Stable; -import org.apache.hadoop.classification.InterfaceStability.Unstable; import org.apache.hadoop.util.StringUtils; import org.apache.hadoop.yarn.api.ApplicationMasterProtocol; import org.apache.hadoop.yarn.api.protocolrecords.ResourceTypes; diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-api/src/main/java/org/apache/hadoop/yarn/api/records/timeline/TimelineHealth.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-api/src/main/java/org/apache/hadoop/yarn/api/records/timeline/TimelineHealth.java index d592167b862cb..3462441161fc2 100644 --- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-api/src/main/java/org/apache/hadoop/yarn/api/records/timeline/TimelineHealth.java +++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-api/src/main/java/org/apache/hadoop/yarn/api/records/timeline/TimelineHealth.java @@ -39,12 +39,12 @@ public class TimelineHealth { * Timline health status. * * RUNNING - Service is up and running - * READER_CONNECTION_FAULURE - isConnectionAlive() of reader implementation + * CONNECTION_FAULURE - isConnectionAlive() of reader / writer implementation * reported an error */ public enum TimelineHealthStatus { RUNNING, - READER_CONNECTION_FAILURE + CONNECTION_FAILURE } private TimelineHealthStatus healthStatus; diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-api/src/main/java/org/apache/hadoop/yarn/exceptions/ApplicationAttemptNotFoundException.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-api/src/main/java/org/apache/hadoop/yarn/exceptions/ApplicationAttemptNotFoundException.java index f2ec38b022826..bbbd2a8bc7c00 100644 --- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-api/src/main/java/org/apache/hadoop/yarn/exceptions/ApplicationAttemptNotFoundException.java +++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-api/src/main/java/org/apache/hadoop/yarn/exceptions/ApplicationAttemptNotFoundException.java @@ -23,7 +23,6 @@ import org.apache.hadoop.yarn.api.ApplicationHistoryProtocol; import org.apache.hadoop.yarn.api.ApplicationMasterProtocol; import org.apache.hadoop.yarn.api.protocolrecords.AllocateRequest; -import org.apache.hadoop.yarn.api.protocolrecords.GetApplicationAttemptReportRequest; /** * This exception is thrown on diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-api/src/main/java/org/apache/hadoop/yarn/exceptions/ApplicationNotFoundException.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-api/src/main/java/org/apache/hadoop/yarn/exceptions/ApplicationNotFoundException.java index da83c397e8f36..939280862ec50 100644 --- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-api/src/main/java/org/apache/hadoop/yarn/exceptions/ApplicationNotFoundException.java +++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-api/src/main/java/org/apache/hadoop/yarn/exceptions/ApplicationNotFoundException.java @@ -21,7 +21,6 @@ import org.apache.hadoop.classification.InterfaceAudience.Public; import org.apache.hadoop.classification.InterfaceStability.Unstable; import org.apache.hadoop.yarn.api.ApplicationClientProtocol; -import org.apache.hadoop.yarn.api.protocolrecords.GetApplicationReportRequest; /** * This exception is thrown on diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-api/src/main/java/org/apache/hadoop/yarn/exceptions/ContainerNotFoundException.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-api/src/main/java/org/apache/hadoop/yarn/exceptions/ContainerNotFoundException.java index d5ef27b4e9b76..03c2001ab8e32 100644 --- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-api/src/main/java/org/apache/hadoop/yarn/exceptions/ContainerNotFoundException.java +++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-api/src/main/java/org/apache/hadoop/yarn/exceptions/ContainerNotFoundException.java @@ -21,7 +21,6 @@ import org.apache.hadoop.classification.InterfaceAudience.Public; import org.apache.hadoop.classification.InterfaceStability.Unstable; import org.apache.hadoop.yarn.api.ApplicationHistoryProtocol; -import org.apache.hadoop.yarn.api.protocolrecords.GetContainerReportRequest; /** * This exception is thrown on diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-applications-catalog/hadoop-yarn-applications-catalog-webapp/src/test/java/org/apache/hadoop/yarn/appcatalog/application/EmbeddedSolrServerFactory.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-applications-catalog/hadoop-yarn-applications-catalog-webapp/src/test/java/org/apache/hadoop/yarn/appcatalog/application/EmbeddedSolrServerFactory.java index eb78266695988..fceca7955ce9c 100644 --- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-applications-catalog/hadoop-yarn-applications-catalog-webapp/src/test/java/org/apache/hadoop/yarn/appcatalog/application/EmbeddedSolrServerFactory.java +++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-applications-catalog/hadoop-yarn-applications-catalog-webapp/src/test/java/org/apache/hadoop/yarn/appcatalog/application/EmbeddedSolrServerFactory.java @@ -22,7 +22,6 @@ import org.apache.solr.client.solrj.embedded.EmbeddedSolrServer; import org.apache.solr.client.solrj.request.CoreAdminRequest; import org.apache.solr.core.NodeConfig; -import org.apache.solr.core.SolrResourceLoader; import java.io.File; import java.io.IOException; diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-applications-distributedshell/src/main/java/org/apache/hadoop/yarn/applications/distributedshell/DistributedShellTimelinePlugin.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-applications-distributedshell/src/main/java/org/apache/hadoop/yarn/applications/distributedshell/DistributedShellTimelinePlugin.java index cf5bc839da1a6..89eea4b67a95d 100644 --- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-applications-distributedshell/src/main/java/org/apache/hadoop/yarn/applications/distributedshell/DistributedShellTimelinePlugin.java +++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-applications-distributedshell/src/main/java/org/apache/hadoop/yarn/applications/distributedshell/DistributedShellTimelinePlugin.java @@ -23,7 +23,6 @@ import org.apache.hadoop.yarn.api.records.timeline.TimelineEntityGroupId; import org.apache.hadoop.yarn.server.timeline.NameValuePair; import org.apache.hadoop.yarn.server.timeline.TimelineEntityGroupPlugin; -import org.apache.hadoop.yarn.util.ConverterUtils; import java.util.Collection; import java.util.HashSet; diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-services/hadoop-yarn-services-core/src/main/java/org/apache/hadoop/yarn/service/monitor/probe/HttpProbe.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-services/hadoop-yarn-services-core/src/main/java/org/apache/hadoop/yarn/service/monitor/probe/HttpProbe.java index 492a11b2c67a3..40a87937629d1 100644 --- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-services/hadoop-yarn-services-core/src/main/java/org/apache/hadoop/yarn/service/monitor/probe/HttpProbe.java +++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-services/hadoop-yarn-services-core/src/main/java/org/apache/hadoop/yarn/service/monitor/probe/HttpProbe.java @@ -88,8 +88,9 @@ public ProbeStatus ping(ComponentInstance instance) { } String ip = instance.getContainerStatus().getIPs().get(0); HttpURLConnection connection = null; + String hostString = urlString.replace(HOST_TOKEN, ip); try { - URL url = new URL(urlString.replace(HOST_TOKEN, ip)); + URL url = new URL(hostString); connection = getConnection(url, this.timeout); int rc = connection.getResponseCode(); if (rc < min || rc > max) { @@ -101,7 +102,8 @@ public ProbeStatus ping(ComponentInstance instance) { status.succeed(this); } } catch (Throwable e) { - String error = "Probe " + urlString + " failed for IP " + ip + ": " + e; + String error = + "Probe " + hostString + " failed for IP " + ip + ": " + e; log.info(error, e); status.fail(this, new IOException(error, e)); diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-services/hadoop-yarn-services-core/src/test/java/org/apache/hadoop/yarn/service/containerlaunch/TestAbstractLauncher.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-services/hadoop-yarn-services-core/src/test/java/org/apache/hadoop/yarn/service/containerlaunch/TestAbstractLauncher.java index 108078ca7f010..31ca38297c856 100644 --- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-services/hadoop-yarn-services-core/src/test/java/org/apache/hadoop/yarn/service/containerlaunch/TestAbstractLauncher.java +++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-services/hadoop-yarn-services-core/src/test/java/org/apache/hadoop/yarn/service/containerlaunch/TestAbstractLauncher.java @@ -42,7 +42,6 @@ .DEFAULT_CONTAINER_RETRY_MAX; import static org.mockito.Mockito.mock; import static org.mockito.Mockito.reset; -import static org.mockito.Mockito.times; import static org.mockito.Mockito.verify; import static org.mockito.Mockito.verifyZeroInteractions; import static org.mockito.Mockito.when; diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-client/src/main/java/org/apache/hadoop/yarn/client/api/impl/AHSv2ClientImpl.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-client/src/main/java/org/apache/hadoop/yarn/client/api/impl/AHSv2ClientImpl.java index e3653baa2b785..26f45baaee019 100644 --- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-client/src/main/java/org/apache/hadoop/yarn/client/api/impl/AHSv2ClientImpl.java +++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-client/src/main/java/org/apache/hadoop/yarn/client/api/impl/AHSv2ClientImpl.java @@ -137,9 +137,8 @@ public List getContainers(ApplicationAttemptId ApplicationId appId = applicationAttemptId.getApplicationId(); ApplicationReport appReport = getApplicationReport(appId); Map filters = new HashMap<>(); - filters.put("infofilters", "SYSTEM_INFO_PARENT_ENTITY eq {\"id\":\"" + - applicationAttemptId.toString() + - "\",\"type\":\"YARN_APPLICATION_ATTEMPT\"}"); + filters.put("infofilters", "SYSTEM_INFO_PARENT_ENTITY eq " + + "{\"type\":\"YARN_APPLICATION_ATTEMPT\",\"id\":\"" + applicationAttemptId + "\"}"); List entities = readerClient.getContainerEntities( appId, "ALL", filters, 0, null); List containers = diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-client/src/test/java/org/apache/hadoop/yarn/client/api/impl/AMRMTokenIdentifierForTest.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-client/src/test/java/org/apache/hadoop/yarn/client/api/impl/AMRMTokenIdentifierForTest.java index 782dc6d644cdd..c493bd2821663 100644 --- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-client/src/test/java/org/apache/hadoop/yarn/client/api/impl/AMRMTokenIdentifierForTest.java +++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-client/src/test/java/org/apache/hadoop/yarn/client/api/impl/AMRMTokenIdentifierForTest.java @@ -25,11 +25,8 @@ import org.apache.commons.io.IOUtils; import org.apache.hadoop.io.Text; -import org.apache.hadoop.security.UserGroupInformation; import org.apache.hadoop.yarn.api.records.ApplicationAttemptId; -import org.apache.hadoop.yarn.api.records.NodeId; import org.apache.hadoop.yarn.api.records.impl.pb.ApplicationAttemptIdPBImpl; -import org.apache.hadoop.yarn.api.records.impl.pb.NodeIdPBImpl; import org.apache.hadoop.yarn.proto.YarnSecurityTestAMRMTokenProtos.AMRMTokenIdentifierForTestProto; import org.apache.hadoop.yarn.security.AMRMTokenIdentifier; diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/pom.xml b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/pom.xml index 27fac5f13f135..c6df7594fcc3f 100644 --- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/pom.xml +++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/pom.xml @@ -258,7 +258,7 @@ src/main/resources/webapps/static/dt-1.10.18/images/Sorting icons.psd src/main/resources/webapps/static/dt-1.10.18/js/jquery.dataTables.min.js src/main/resources/webapps/static/jt/jquery.jstree.js - src/main/resources/webapps/static/jquery/jquery-ui-1.12.1.custom.min.js + src/main/resources/webapps/static/jquery/jquery-ui-1.13.1.custom.min.js src/main/resources/webapps/static/jquery/jquery-3.6.0.min.js src/main/resources/webapps/static/jquery/themes-1.9.1/base/jquery-ui.css src/test/resources/application_1440536969523_0001.har/_index diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/main/java/org/apache/hadoop/yarn/api/protocolrecords/impl/pb/GetLabelsToNodesResponsePBImpl.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/main/java/org/apache/hadoop/yarn/api/protocolrecords/impl/pb/GetLabelsToNodesResponsePBImpl.java index 961b9f1eec5ff..5f4f0394e3fe4 100644 --- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/main/java/org/apache/hadoop/yarn/api/protocolrecords/impl/pb/GetLabelsToNodesResponsePBImpl.java +++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/main/java/org/apache/hadoop/yarn/api/protocolrecords/impl/pb/GetLabelsToNodesResponsePBImpl.java @@ -29,9 +29,7 @@ import org.apache.hadoop.classification.InterfaceAudience.Public; import org.apache.hadoop.classification.InterfaceStability.Evolving; import org.apache.hadoop.yarn.api.records.NodeId; -import org.apache.hadoop.yarn.api.records.NodeLabel; import org.apache.hadoop.yarn.api.records.impl.pb.NodeIdPBImpl; -import org.apache.hadoop.yarn.api.records.impl.pb.NodeLabelPBImpl; import org.apache.hadoop.yarn.proto.YarnProtos.NodeIdProto; import org.apache.hadoop.yarn.api.protocolrecords.GetLabelsToNodesResponse; import org.apache.hadoop.yarn.proto.YarnProtos.LabelsToNodeIdsProto; diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/main/java/org/apache/hadoop/yarn/api/protocolrecords/impl/pb/GetNodesToLabelsResponsePBImpl.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/main/java/org/apache/hadoop/yarn/api/protocolrecords/impl/pb/GetNodesToLabelsResponsePBImpl.java index 6b7c45afaccaa..c9535cacf9adb 100644 --- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/main/java/org/apache/hadoop/yarn/api/protocolrecords/impl/pb/GetNodesToLabelsResponsePBImpl.java +++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/main/java/org/apache/hadoop/yarn/api/protocolrecords/impl/pb/GetNodesToLabelsResponsePBImpl.java @@ -19,7 +19,6 @@ package org.apache.hadoop.yarn.api.protocolrecords.impl.pb; import java.util.HashMap; -import java.util.HashSet; import java.util.Iterator; import java.util.List; import java.util.Map; @@ -28,9 +27,7 @@ import org.apache.hadoop.util.Sets; import org.apache.hadoop.yarn.api.records.NodeId; -import org.apache.hadoop.yarn.api.records.NodeLabel; import org.apache.hadoop.yarn.api.records.impl.pb.NodeIdPBImpl; -import org.apache.hadoop.yarn.api.records.impl.pb.NodeLabelPBImpl; import org.apache.hadoop.yarn.proto.YarnProtos.NodeIdProto; import org.apache.hadoop.yarn.api.protocolrecords.GetNodesToLabelsResponse; import org.apache.hadoop.yarn.proto.YarnProtos.NodeIdToLabelsProto; diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/main/java/org/apache/hadoop/yarn/api/records/impl/pb/QueueConfigurationsPBImpl.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/main/java/org/apache/hadoop/yarn/api/records/impl/pb/QueueConfigurationsPBImpl.java index 1b14f9d700c09..1d213b3059208 100644 --- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/main/java/org/apache/hadoop/yarn/api/records/impl/pb/QueueConfigurationsPBImpl.java +++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/main/java/org/apache/hadoop/yarn/api/records/impl/pb/QueueConfigurationsPBImpl.java @@ -72,11 +72,11 @@ private void mergeLocalToBuilder() { .setEffectiveMaxCapacity(convertToProtoFormat(this.effMaxResource)); } if (this.configuredMinResource != null) { - builder.setEffectiveMinCapacity( + builder.setConfiguredMinCapacity( convertToProtoFormat(this.configuredMinResource)); } if (this.configuredMaxResource != null) { - builder.setEffectiveMaxCapacity( + builder.setConfiguredMaxCapacity( convertToProtoFormat(this.configuredMaxResource)); } } diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/main/java/org/apache/hadoop/yarn/api/records/impl/pb/ResourcePBImpl.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/main/java/org/apache/hadoop/yarn/api/records/impl/pb/ResourcePBImpl.java index 75f87541603b1..850c36b20addc 100644 --- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/main/java/org/apache/hadoop/yarn/api/records/impl/pb/ResourcePBImpl.java +++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/main/java/org/apache/hadoop/yarn/api/records/impl/pb/ResourcePBImpl.java @@ -27,7 +27,6 @@ import org.apache.hadoop.yarn.api.protocolrecords.ResourceTypes; import org.apache.hadoop.yarn.api.records.Resource; import org.apache.hadoop.yarn.api.records.ResourceInformation; -import org.apache.hadoop.yarn.exceptions.ResourceNotFoundException; import org.apache.hadoop.yarn.proto.YarnProtos.ResourceProto; import org.apache.hadoop.yarn.proto.YarnProtos.ResourceProtoOrBuilder; import org.apache.hadoop.yarn.proto.YarnProtos.ResourceInformationProto; diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/main/java/org/apache/hadoop/yarn/client/api/impl/TimelineReaderClientImpl.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/main/java/org/apache/hadoop/yarn/client/api/impl/TimelineReaderClientImpl.java index 29609b955fbec..71bf13220b016 100644 --- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/main/java/org/apache/hadoop/yarn/client/api/impl/TimelineReaderClientImpl.java +++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/main/java/org/apache/hadoop/yarn/client/api/impl/TimelineReaderClientImpl.java @@ -38,7 +38,10 @@ import javax.ws.rs.core.MediaType; import javax.ws.rs.core.MultivaluedMap; import java.io.IOException; +import java.io.UnsupportedEncodingException; import java.net.URI; +import java.net.URLEncoder; +import java.nio.charset.StandardCharsets; import java.util.Arrays; import java.util.List; import java.util.Map; @@ -208,12 +211,21 @@ public List getContainerEntities( return Arrays.asList(entity); } + @VisibleForTesting + protected String encodeValue(String value) throws UnsupportedEncodingException { + // Since URLEncoder doesn't use and doesn't have an option for percent-encoding + // (as specified in RFC 3986) the spaces are encoded to + signs, which need to be replaced + // manually + return URLEncoder.encode(value, StandardCharsets.UTF_8.toString()) + .replaceAll("\\+", "%20"); + } + private void mergeFilters(MultivaluedMap defaults, - Map filters) { + Map filters) throws UnsupportedEncodingException { if (filters != null && !filters.isEmpty()) { for (Map.Entry entry : filters.entrySet()) { if (!defaults.containsKey(entry.getKey())) { - defaults.add(entry.getKey(), filters.get(entry.getValue())); + defaults.add(entry.getKey(), encodeValue(entry.getValue())); } } } diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/main/java/org/apache/hadoop/yarn/logaggregation/AggregatedLogDeletionService.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/main/java/org/apache/hadoop/yarn/logaggregation/AggregatedLogDeletionService.java index 9427068cfc505..4f10b2fd4ce23 100644 --- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/main/java/org/apache/hadoop/yarn/logaggregation/AggregatedLogDeletionService.java +++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/main/java/org/apache/hadoop/yarn/logaggregation/AggregatedLogDeletionService.java @@ -19,6 +19,8 @@ package org.apache.hadoop.yarn.logaggregation; import java.io.IOException; +import java.util.ArrayList; +import java.util.List; import java.util.Timer; import java.util.TimerTask; @@ -57,23 +59,21 @@ public class AggregatedLogDeletionService extends AbstractService { private Timer timer = null; private long checkIntervalMsecs; - private LogDeletionTask task; + private List tasks; - static class LogDeletionTask extends TimerTask { + public static class LogDeletionTask extends TimerTask { private Configuration conf; private long retentionMillis; private String suffix = null; private Path remoteRootLogDir = null; private ApplicationClientProtocol rmClient = null; - public LogDeletionTask(Configuration conf, long retentionSecs, ApplicationClientProtocol rmClient) { + public LogDeletionTask(Configuration conf, long retentionSecs, + ApplicationClientProtocol rmClient, + LogAggregationFileController fileController) { this.conf = conf; this.retentionMillis = retentionSecs * 1000; this.suffix = LogAggregationUtils.getBucketSuffix(); - LogAggregationFileControllerFactory factory = - new LogAggregationFileControllerFactory(conf); - LogAggregationFileController fileController = - factory.getFileControllerForWrite(); this.remoteRootLogDir = fileController.getRemoteRootLogDir(); this.rmClient = rmClient; } @@ -101,7 +101,7 @@ public void run() { } } } catch (Throwable t) { - logException("Error reading root log dir this deletion " + + logException("Error reading root log dir, this deletion " + "attempt is being aborted", t); } LOG.info("aggregated log deletion finished."); @@ -220,7 +220,7 @@ public AggregatedLogDeletionService() { @Override protected void serviceStart() throws Exception { - scheduleLogDeletionTask(); + scheduleLogDeletionTasks(); super.serviceStart(); } @@ -249,13 +249,13 @@ public void refreshLogRetentionSettings() throws IOException { setConfig(conf); stopRMClient(); stopTimer(); - scheduleLogDeletionTask(); + scheduleLogDeletionTasks(); } else { LOG.warn("Failed to execute refreshLogRetentionSettings : Aggregated Log Deletion Service is not started"); } } - private void scheduleLogDeletionTask() throws IOException { + private void scheduleLogDeletionTasks() throws IOException { Configuration conf = getConfig(); if (!conf.getBoolean(YarnConfiguration.LOG_AGGREGATION_ENABLED, YarnConfiguration.DEFAULT_LOG_AGGREGATION_ENABLED)) { @@ -271,9 +271,28 @@ private void scheduleLogDeletionTask() throws IOException { return; } setLogAggCheckIntervalMsecs(retentionSecs); - task = new LogDeletionTask(conf, retentionSecs, createRMClient()); - timer = new Timer(); - timer.scheduleAtFixedRate(task, 0, checkIntervalMsecs); + + tasks = createLogDeletionTasks(conf, retentionSecs, createRMClient()); + for (LogDeletionTask task : tasks) { + timer = new Timer(); + timer.scheduleAtFixedRate(task, 0, checkIntervalMsecs); + } + } + + @VisibleForTesting + public List createLogDeletionTasks(Configuration conf, long retentionSecs, + ApplicationClientProtocol rmClient) + throws IOException { + List tasks = new ArrayList<>(); + LogAggregationFileControllerFactory factory = new LogAggregationFileControllerFactory(conf); + List fileControllers = + factory.getConfiguredLogAggregationFileControllerList(); + for (LogAggregationFileController fileController : fileControllers) { + LogDeletionTask task = new LogDeletionTask(conf, retentionSecs, rmClient, + fileController); + tasks.add(task); + } + return tasks; } private void stopTimer() { @@ -295,14 +314,18 @@ protected Configuration createConf() { // as @Idempotent, it will automatically take care of RM restart/failover. @VisibleForTesting protected ApplicationClientProtocol createRMClient() throws IOException { - return ClientRMProxy.createRMProxy(getConfig(), - ApplicationClientProtocol.class); + return ClientRMProxy.createRMProxy(getConfig(), ApplicationClientProtocol.class); } @VisibleForTesting protected void stopRMClient() { - if (task != null && task.getRMClient() != null) { - RPC.stopProxy(task.getRMClient()); + for (LogDeletionTask task : tasks) { + if (task != null && task.getRMClient() != null) { + RPC.stopProxy(task.getRMClient()); + //The RMClient instance is the same for all deletion tasks. + //It is enough to close the RM client once + break; + } } } } diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/main/java/org/apache/hadoop/yarn/logaggregation/filecontroller/LogAggregationFileControllerFactory.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/main/java/org/apache/hadoop/yarn/logaggregation/filecontroller/LogAggregationFileControllerFactory.java index fea5be66243af..3ded57fe2fb50 100644 --- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/main/java/org/apache/hadoop/yarn/logaggregation/filecontroller/LogAggregationFileControllerFactory.java +++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/main/java/org/apache/hadoop/yarn/logaggregation/filecontroller/LogAggregationFileControllerFactory.java @@ -53,9 +53,8 @@ public class LogAggregationFileControllerFactory { LogAggregationFileControllerFactory.class); private final Pattern p = Pattern.compile( "^[A-Za-z_]+[A-Za-z0-9_]*$"); - private LinkedList controllers - = new LinkedList<>(); - private Configuration conf; + private final LinkedList controllers = new LinkedList<>(); + private final Configuration conf; /** * Construct the LogAggregationFileControllerFactory object. @@ -65,77 +64,59 @@ public LogAggregationFileControllerFactory(Configuration conf) { this.conf = conf; Collection fileControllers = conf.getStringCollection( YarnConfiguration.LOG_AGGREGATION_FILE_FORMATS); - List controllerClassName = new ArrayList<>(); - Map controllerChecker = new HashMap<>(); - for (String fileController : fileControllers) { - Preconditions.checkArgument(validateAggregatedFileControllerName( - fileController), "The FileControllerName: " + fileController - + " set in " + YarnConfiguration.LOG_AGGREGATION_FILE_FORMATS - +" is invalid." + "The valid File Controller name should only " - + "contain a-zA-Z0-9_ and can not start with numbers"); + for (String controllerName : fileControllers) { + validateAggregatedFileControllerName(controllerName); - String remoteDirStr = String.format( - YarnConfiguration.LOG_AGGREGATION_REMOTE_APP_LOG_DIR_FMT, - fileController); - String remoteDir = conf.get(remoteDirStr); - boolean defaultRemoteDir = false; - if (remoteDir == null || remoteDir.isEmpty()) { - remoteDir = conf.get(YarnConfiguration.NM_REMOTE_APP_LOG_DIR, - YarnConfiguration.DEFAULT_NM_REMOTE_APP_LOG_DIR); - defaultRemoteDir = true; - } - String suffixStr = String.format( - YarnConfiguration.LOG_AGGREGATION_REMOTE_APP_LOG_DIR_SUFFIX_FMT, - fileController); - String suffix = conf.get(suffixStr); - boolean defaultSuffix = false; - if (suffix == null || suffix.isEmpty()) { - suffix = conf.get(YarnConfiguration.NM_REMOTE_APP_LOG_DIR_SUFFIX, - YarnConfiguration.DEFAULT_NM_REMOTE_APP_LOG_DIR_SUFFIX); - defaultSuffix = true; - } - String dirSuffix = remoteDir + "-" + suffix; - if (controllerChecker.containsKey(dirSuffix)) { - if (defaultRemoteDir && defaultSuffix) { - String fileControllerStr = controllerChecker.get(dirSuffix); - List controllersList = new ArrayList<>(); - controllersList.add(fileControllerStr); - controllersList.add(fileController); - fileControllerStr = StringUtils.join(controllersList, ","); - controllerChecker.put(dirSuffix, fileControllerStr); - } else { - String conflictController = controllerChecker.get(dirSuffix); - throw new RuntimeException("The combined value of " + remoteDirStr - + " and " + suffixStr + " should not be the same as the value" - + " set for " + conflictController); - } + validateConflictingControllers(conf, controllerChecker, controllerName); + DeterminedControllerClassName className = + new DeterminedControllerClassName(conf, controllerName); + LogAggregationFileController controller = createFileControllerInstance(conf, + controllerName, className); + controller.initialize(conf, controllerName); + controllers.add(controller); + } + } + + private LogAggregationFileController createFileControllerInstance( + Configuration conf, + String fileController, DeterminedControllerClassName className) { + Class clazz = conf.getClass( + className.configKey, null, LogAggregationFileController.class); + if (clazz == null) { + throw new RuntimeException("No class defined for " + fileController); + } + LogAggregationFileController instance = ReflectionUtils.newInstance(clazz, conf); + if (instance == null) { + throw new RuntimeException("No object created for " + className.value); + } + return instance; + } + + private void validateConflictingControllers( + Configuration conf, Map controllerChecker, String fileController) { + DeterminedLogAggregationRemoteDir remoteDir = + new DeterminedLogAggregationRemoteDir(conf, fileController); + DeterminedLogAggregationSuffix suffix = + new DeterminedLogAggregationSuffix(conf, fileController); + String dirSuffix = remoteDir.value + "-" + suffix.value; + if (controllerChecker.containsKey(dirSuffix)) { + if (remoteDir.usingDefault && suffix.usingDefault) { + String fileControllerStr = controllerChecker.get(dirSuffix); + List controllersList = new ArrayList<>(); + controllersList.add(fileControllerStr); + controllersList.add(fileController); + fileControllerStr = StringUtils.join(controllersList, ","); + controllerChecker.put(dirSuffix, fileControllerStr); } else { - controllerChecker.put(dirSuffix, fileController); - } - String classKey = String.format( - YarnConfiguration.LOG_AGGREGATION_FILE_CONTROLLER_FMT, - fileController); - String className = conf.get(classKey); - if (className == null || className.isEmpty()) { - throw new RuntimeException("No class configured for " - + fileController); - } - controllerClassName.add(className); - Class sClass = conf.getClass( - classKey, null, LogAggregationFileController.class); - if (sClass == null) { - throw new RuntimeException("No class defined for " + fileController); - } - LogAggregationFileController s = ReflectionUtils.newInstance( - sClass, conf); - if (s == null) { - throw new RuntimeException("No object created for " - + controllerClassName); + String conflictController = controllerChecker.get(dirSuffix); + throw new RuntimeException(String.format("The combined value of %s " + + "and %s should not be the same as the value set for %s", + remoteDir.configKey, suffix.configKey, conflictController)); } - s.initialize(conf, fileController); - controllers.add(s); + } else { + controllerChecker.put(dirSuffix, fileController); } } @@ -169,8 +150,7 @@ public LogAggregationFileController getFileControllerForRead( return fileController; } } catch (Exception ex) { - diagnosticsMsg.append(ex.getMessage() + "\n"); - continue; + diagnosticsMsg.append(ex.getMessage()).append("\n"); } } } @@ -184,19 +164,26 @@ public LogAggregationFileController getFileControllerForRead( return fileController; } } catch (Exception ex) { - diagnosticsMsg.append(ex.getMessage() + "\n"); - continue; + diagnosticsMsg.append(ex.getMessage()).append("\n"); } } throw new IOException(diagnosticsMsg.toString()); } - private boolean validateAggregatedFileControllerName(String name) { + private void validateAggregatedFileControllerName(String name) { + boolean valid; if (name == null || name.trim().isEmpty()) { - return false; + valid = false; + } else { + valid = p.matcher(name).matches(); } - return p.matcher(name).matches(); + + Preconditions.checkArgument(valid, + String.format("The FileControllerName: %s set in " + + "%s is invalid.The valid File Controller name should only contain " + + "a-zA-Z0-9_ and cannot start with numbers", name, + YarnConfiguration.LOG_AGGREGATION_FILE_FORMATS)); } @Private @@ -205,4 +192,64 @@ private boolean validateAggregatedFileControllerName(String name) { getConfiguredLogAggregationFileControllerList() { return this.controllers; } + + private static class DeterminedLogAggregationRemoteDir { + private String value; + private boolean usingDefault = false; + private final String configKey; + + DeterminedLogAggregationRemoteDir(Configuration conf, + String fileController) { + configKey = String.format( + YarnConfiguration.LOG_AGGREGATION_REMOTE_APP_LOG_DIR_FMT, + fileController); + String remoteDir = conf.get(configKey); + + if (remoteDir == null || remoteDir.isEmpty()) { + this.value = conf.get(YarnConfiguration.NM_REMOTE_APP_LOG_DIR, + YarnConfiguration.DEFAULT_NM_REMOTE_APP_LOG_DIR); + this.usingDefault = true; + } else { + this.value = remoteDir; + } + } + } + + private static class DeterminedLogAggregationSuffix { + private String value; + private boolean usingDefault = false; + private final String configKey; + + DeterminedLogAggregationSuffix(Configuration conf, + String fileController) { + configKey = String.format( + YarnConfiguration.LOG_AGGREGATION_REMOTE_APP_LOG_DIR_SUFFIX_FMT, + fileController); + String suffix = conf.get(configKey); + if (suffix == null || suffix.isEmpty()) { + this.value = conf.get(YarnConfiguration.NM_REMOTE_APP_LOG_DIR_SUFFIX, + YarnConfiguration.DEFAULT_NM_REMOTE_APP_LOG_DIR_SUFFIX); + this.usingDefault = true; + } else { + this.value = suffix; + } + } + } + + private static class DeterminedControllerClassName { + private final String configKey; + private final String value; + + DeterminedControllerClassName(Configuration conf, + String fileController) { + this.configKey = String.format( + YarnConfiguration.LOG_AGGREGATION_FILE_CONTROLLER_FMT, + fileController); + this.value = conf.get(configKey); + if (value == null || value.isEmpty()) { + throw new RuntimeException("No class configured for " + + fileController); + } + } + } } diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/main/java/org/apache/hadoop/yarn/logaggregation/filecontroller/ifile/IndexedFileAggregatedLogsBlock.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/main/java/org/apache/hadoop/yarn/logaggregation/filecontroller/ifile/IndexedFileAggregatedLogsBlock.java index 8390f99daed11..817f53eb1f977 100644 --- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/main/java/org/apache/hadoop/yarn/logaggregation/filecontroller/ifile/IndexedFileAggregatedLogsBlock.java +++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/main/java/org/apache/hadoop/yarn/logaggregation/filecontroller/ifile/IndexedFileAggregatedLogsBlock.java @@ -23,7 +23,6 @@ import com.google.inject.Inject; import java.io.IOException; import java.io.InputStream; -import java.nio.charset.Charset; import java.util.ArrayList; import java.util.List; import java.util.Map; @@ -49,7 +48,6 @@ import org.apache.hadoop.yarn.logaggregation.filecontroller.ifile.LogAggregationIndexedFileController.IndexedLogsMeta; import org.apache.hadoop.yarn.logaggregation.filecontroller.ifile.LogAggregationIndexedFileController.IndexedPerAggregationLogMeta; import org.apache.hadoop.yarn.util.Times; -import org.apache.hadoop.yarn.webapp.hamlet2.Hamlet; /** * The Aggregated Logs Block implementation for Indexed File. diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/main/java/org/apache/hadoop/yarn/logaggregation/filecontroller/ifile/LogAggregationIndexedFileController.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/main/java/org/apache/hadoop/yarn/logaggregation/filecontroller/ifile/LogAggregationIndexedFileController.java index 6cd1f53d4ac8e..d4431d56b39a4 100644 --- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/main/java/org/apache/hadoop/yarn/logaggregation/filecontroller/ifile/LogAggregationIndexedFileController.java +++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/main/java/org/apache/hadoop/yarn/logaggregation/filecontroller/ifile/LogAggregationIndexedFileController.java @@ -282,7 +282,8 @@ private Pair initializeWriterInRolling( checksumFileInputStream = fc.open(remoteLogCheckSumFile); int nameLength = checksumFileInputStream.readInt(); byte[] b = new byte[nameLength]; - int actualLength = checksumFileInputStream.read(b); + checksumFileInputStream.readFully(b); + int actualLength = b.length; if (actualLength == nameLength) { String recoveredLogFile = new String( b, Charset.forName("UTF-8")); @@ -765,7 +766,8 @@ public Map parseCheckSumFiles( checksumFileInputStream = fc.open(file.getPath()); int nameLength = checksumFileInputStream.readInt(); byte[] b = new byte[nameLength]; - int actualLength = checksumFileInputStream.read(b); + checksumFileInputStream.readFully(b); + int actualLength = b.length; if (actualLength == nameLength) { nodeName = new String(b, Charset.forName("UTF-8")); index = checksumFileInputStream.readLong(); @@ -799,7 +801,8 @@ private Long parseChecksum(FileStatus file) { checksumFileInputStream = fileContext.open(file.getPath()); int nameLength = checksumFileInputStream.readInt(); byte[] b = new byte[nameLength]; - int actualLength = checksumFileInputStream.read(b); + checksumFileInputStream.readFully(b); + int actualLength = b.length; if (actualLength == nameLength) { nodeName = new String(b, StandardCharsets.UTF_8); index = checksumFileInputStream.readLong(); @@ -938,7 +941,8 @@ public IndexedLogsMeta loadIndexedLogsMeta(Path remoteLogPath, long end, // Load UUID and make sure the UUID is correct. byte[] uuidRead = new byte[UUID_LENGTH]; - int uuidReadLen = fsDataIStream.read(uuidRead); + fsDataIStream.readFully(uuidRead); + int uuidReadLen = uuidRead.length; if (this.uuid == null) { this.uuid = createUUID(appId); } @@ -1322,7 +1326,8 @@ private byte[] loadUUIDFromLogFile(final FileContext fc, .endsWith(CHECK_SUM_FILE_SUFFIX)) { fsDataInputStream = fc.open(checkPath); byte[] b = new byte[uuid.length]; - int actual = fsDataInputStream.read(b); + fsDataInputStream.readFully(b); + int actual = b.length; if (actual != uuid.length || Arrays.equals(b, uuid)) { deleteFileWithRetries(fc, checkPath); } else if (id == null){ diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/main/java/org/apache/hadoop/yarn/logaggregation/filecontroller/tfile/TFileAggregatedLogsBlock.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/main/java/org/apache/hadoop/yarn/logaggregation/filecontroller/tfile/TFileAggregatedLogsBlock.java index 2c2604b1390a6..d7a44336ef97f 100644 --- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/main/java/org/apache/hadoop/yarn/logaggregation/filecontroller/tfile/TFileAggregatedLogsBlock.java +++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/main/java/org/apache/hadoop/yarn/logaggregation/filecontroller/tfile/TFileAggregatedLogsBlock.java @@ -22,7 +22,6 @@ import com.google.inject.Inject; import java.io.IOException; -import java.io.InputStream; import java.util.Map; import org.apache.hadoop.classification.InterfaceAudience; import org.apache.hadoop.conf.Configuration; @@ -38,8 +37,6 @@ import org.apache.hadoop.yarn.logaggregation.LogAggregationUtils; import org.apache.hadoop.yarn.logaggregation.filecontroller.LogAggregationHtmlBlock; import org.apache.hadoop.yarn.util.Times; -import org.apache.hadoop.yarn.webapp.hamlet2.Hamlet; -import org.apache.hadoop.yarn.webapp.hamlet2.Hamlet.PRE; /** * The Aggregated Logs Block implementation for TFile. diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/main/java/org/apache/hadoop/yarn/server/api/protocolrecords/impl/pb/UpdateNodeResourceRequestPBImpl.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/main/java/org/apache/hadoop/yarn/server/api/protocolrecords/impl/pb/UpdateNodeResourceRequestPBImpl.java index d44599664a28b..0e05e731ad263 100644 --- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/main/java/org/apache/hadoop/yarn/server/api/protocolrecords/impl/pb/UpdateNodeResourceRequestPBImpl.java +++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/main/java/org/apache/hadoop/yarn/server/api/protocolrecords/impl/pb/UpdateNodeResourceRequestPBImpl.java @@ -17,7 +17,6 @@ */ package org.apache.hadoop.yarn.server.api.protocolrecords.impl.pb; -import java.util.ArrayList; import java.util.HashMap; import java.util.Iterator; import java.util.List; diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/main/java/org/apache/hadoop/yarn/server/security/ApplicationACLsManager.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/main/java/org/apache/hadoop/yarn/server/security/ApplicationACLsManager.java index ddbcdae07e512..e7741754899b2 100644 --- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/main/java/org/apache/hadoop/yarn/server/security/ApplicationACLsManager.java +++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/main/java/org/apache/hadoop/yarn/server/security/ApplicationACLsManager.java @@ -27,7 +27,6 @@ import org.slf4j.LoggerFactory; import org.apache.hadoop.classification.InterfaceAudience; import org.apache.hadoop.conf.Configuration; -import org.apache.hadoop.security.AccessControlException; import org.apache.hadoop.security.UserGroupInformation; import org.apache.hadoop.security.authorize.AccessControlList; import org.apache.hadoop.yarn.api.records.ApplicationId; diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/main/java/org/apache/hadoop/yarn/util/ResourceCalculatorProcessTree.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/main/java/org/apache/hadoop/yarn/util/ResourceCalculatorProcessTree.java index 9bbec5f9d7be3..7209f9264747e 100644 --- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/main/java/org/apache/hadoop/yarn/util/ResourceCalculatorProcessTree.java +++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/main/java/org/apache/hadoop/yarn/util/ResourceCalculatorProcessTree.java @@ -23,7 +23,6 @@ import org.slf4j.Logger; import org.slf4j.LoggerFactory; import org.apache.hadoop.classification.InterfaceAudience.Public; -import org.apache.hadoop.classification.InterfaceAudience.Private; import org.apache.hadoop.classification.InterfaceStability.Evolving; import org.apache.hadoop.conf.Configuration; import org.apache.hadoop.conf.Configured; diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/main/java/org/apache/hadoop/yarn/webapp/GenericExceptionHandler.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/main/java/org/apache/hadoop/yarn/webapp/GenericExceptionHandler.java index 2bad02c9f088e..b8fc9e00541d8 100644 --- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/main/java/org/apache/hadoop/yarn/webapp/GenericExceptionHandler.java +++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/main/java/org/apache/hadoop/yarn/webapp/GenericExceptionHandler.java @@ -93,8 +93,8 @@ public Response toResponse(Exception e) { && e.getCause() instanceof UnmarshalException) { s = Response.Status.BAD_REQUEST; } else { - LOG.warn("INTERNAL_SERVER_ERROR", e); - s = Response.Status.INTERNAL_SERVER_ERROR; + LOG.warn("SERVICE_UNAVAILABLE", e); + s = Response.Status.SERVICE_UNAVAILABLE; } // let jaxb handle marshalling data out in the same format requested diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/main/java/org/apache/hadoop/yarn/webapp/WebAppException.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/main/java/org/apache/hadoop/yarn/webapp/WebAppException.java index dbede94f29589..10d947392192f 100644 --- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/main/java/org/apache/hadoop/yarn/webapp/WebAppException.java +++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/main/java/org/apache/hadoop/yarn/webapp/WebAppException.java @@ -25,6 +25,11 @@ public class WebAppException extends YarnRuntimeException { private static final long serialVersionUID = 1L; + private WebApp webApp; + + public WebApp getWebApp() { + return webApp; + } public WebAppException(String msg) { super(msg); @@ -35,6 +40,11 @@ public WebAppException(Throwable cause) { } public WebAppException(String msg, Throwable cause) { + this(msg, cause, null); + } + + public WebAppException(String msg, Throwable cause, WebApp webApp) { super(msg, cause); + this.webApp = webApp; } } diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/main/java/org/apache/hadoop/yarn/webapp/WebApps.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/main/java/org/apache/hadoop/yarn/webapp/WebApps.java index 6bd22b2471986..9fef076196e4e 100644 --- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/main/java/org/apache/hadoop/yarn/webapp/WebApps.java +++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/main/java/org/apache/hadoop/yarn/webapp/WebApps.java @@ -476,7 +476,7 @@ public WebApp start(WebApp webapp, WebAppContext ui2Context) { LOG.info("Web app " + name + " started at " + httpServer.getConnectorAddress(0).getPort()); } catch (IOException e) { - throw new WebAppException("Error starting http server", e); + throw new WebAppException("Error starting http server", e, webApp); } return webApp; } diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/main/java/org/apache/hadoop/yarn/webapp/view/JQueryUI.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/main/java/org/apache/hadoop/yarn/webapp/view/JQueryUI.java index f903fc1595fe2..66ea04c0bf797 100644 --- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/main/java/org/apache/hadoop/yarn/webapp/view/JQueryUI.java +++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/main/java/org/apache/hadoop/yarn/webapp/view/JQueryUI.java @@ -20,15 +20,15 @@ import java.util.List; +import org.apache.hadoop.classification.InterfaceAudience; +import org.apache.hadoop.util.Lists; +import org.apache.hadoop.yarn.webapp.hamlet2.HamletSpec.HTML; + import static org.apache.commons.text.StringEscapeUtils.escapeEcmaScript; import static org.apache.hadoop.yarn.util.StringHelper.djoin; import static org.apache.hadoop.yarn.util.StringHelper.join; import static org.apache.hadoop.yarn.util.StringHelper.split; -import org.apache.hadoop.classification.InterfaceAudience; -import org.apache.hadoop.util.Lists; -import org.apache.hadoop.yarn.webapp.hamlet2.HamletSpec.HTML; - @InterfaceAudience.LimitedPrivate({"YARN", "MapReduce"}) public class JQueryUI extends HtmlBlock { @@ -69,7 +69,7 @@ protected void render(Block html) { .link(root_url("static/dt-1.10.18/css/jui-dt.css")) .link(root_url("static/dt-1.10.18/css/custom_datatable.css")) .script(root_url("static/jquery/jquery-3.6.0.min.js")) - .script(root_url("static/jquery/jquery-ui-1.12.1.custom.min.js")) + .script(root_url("static/jquery/jquery-ui-1.13.1.custom.min.js")) .script(root_url("static/dt-1.10.18/js/jquery.dataTables.min.js")) .script(root_url("static/yarn.dt.plugins.js")) .script(root_url("static/dt-sorting/natural.js")) diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/main/resources/webapps/static/jquery/jquery-ui-1.12.1.custom.min.js b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/main/resources/webapps/static/jquery/jquery-ui-1.12.1.custom.min.js deleted file mode 100644 index 25398a1674150..0000000000000 --- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/main/resources/webapps/static/jquery/jquery-ui-1.12.1.custom.min.js +++ /dev/null @@ -1,13 +0,0 @@ -/*! jQuery UI - v1.12.1 - 2016-09-14 -* http://jqueryui.com -* Includes: widget.js, position.js, data.js, disable-selection.js, effect.js, effects/effect-blind.js, effects/effect-bounce.js, effects/effect-clip.js, effects/effect-drop.js, effects/effect-explode.js, effects/effect-fade.js, effects/effect-fold.js, effects/effect-highlight.js, effects/effect-puff.js, effects/effect-pulsate.js, effects/effect-scale.js, effects/effect-shake.js, effects/effect-size.js, effects/effect-slide.js, effects/effect-transfer.js, focusable.js, form-reset-mixin.js, jquery-1-7.js, keycode.js, labels.js, scroll-parent.js, tabbable.js, unique-id.js, widgets/accordion.js, widgets/autocomplete.js, widgets/button.js, widgets/checkboxradio.js, widgets/controlgroup.js, widgets/datepicker.js, widgets/dialog.js, widgets/draggable.js, widgets/droppable.js, widgets/menu.js, widgets/mouse.js, widgets/progressbar.js, widgets/resizable.js, widgets/selectable.js, widgets/selectmenu.js, widgets/slider.js, widgets/sortable.js, widgets/spinner.js, widgets/tabs.js, widgets/tooltip.js -* Copyright jQuery Foundation and other contributors; Licensed MIT */ - -(function(t){"function"==typeof define&&define.amd?define(["jquery"],t):t(jQuery)})(function(t){function e(t){for(var e=t.css("visibility");"inherit"===e;)t=t.parent(),e=t.css("visibility");return"hidden"!==e}function i(t){for(var e,i;t.length&&t[0]!==document;){if(e=t.css("position"),("absolute"===e||"relative"===e||"fixed"===e)&&(i=parseInt(t.css("zIndex"),10),!isNaN(i)&&0!==i))return i;t=t.parent()}return 0}function s(){this._curInst=null,this._keyEvent=!1,this._disabledInputs=[],this._datepickerShowing=!1,this._inDialog=!1,this._mainDivId="ui-datepicker-div",this._inlineClass="ui-datepicker-inline",this._appendClass="ui-datepicker-append",this._triggerClass="ui-datepicker-trigger",this._dialogClass="ui-datepicker-dialog",this._disableClass="ui-datepicker-disabled",this._unselectableClass="ui-datepicker-unselectable",this._currentClass="ui-datepicker-current-day",this._dayOverClass="ui-datepicker-days-cell-over",this.regional=[],this.regional[""]={closeText:"Done",prevText:"Prev",nextText:"Next",currentText:"Today",monthNames:["January","February","March","April","May","June","July","August","September","October","November","December"],monthNamesShort:["Jan","Feb","Mar","Apr","May","Jun","Jul","Aug","Sep","Oct","Nov","Dec"],dayNames:["Sunday","Monday","Tuesday","Wednesday","Thursday","Friday","Saturday"],dayNamesShort:["Sun","Mon","Tue","Wed","Thu","Fri","Sat"],dayNamesMin:["Su","Mo","Tu","We","Th","Fr","Sa"],weekHeader:"Wk",dateFormat:"mm/dd/yy",firstDay:0,isRTL:!1,showMonthAfterYear:!1,yearSuffix:""},this._defaults={showOn:"focus",showAnim:"fadeIn",showOptions:{},defaultDate:null,appendText:"",buttonText:"...",buttonImage:"",buttonImageOnly:!1,hideIfNoPrevNext:!1,navigationAsDateFormat:!1,gotoCurrent:!1,changeMonth:!1,changeYear:!1,yearRange:"c-10:c+10",showOtherMonths:!1,selectOtherMonths:!1,showWeek:!1,calculateWeek:this.iso8601Week,shortYearCutoff:"+10",minDate:null,maxDate:null,duration:"fast",beforeShowDay:null,beforeShow:null,onSelect:null,onChangeMonthYear:null,onClose:null,numberOfMonths:1,showCurrentAtPos:0,stepMonths:1,stepBigMonths:12,altField:"",altFormat:"",constrainInput:!0,showButtonPanel:!1,autoSize:!1,disabled:!1},t.extend(this._defaults,this.regional[""]),this.regional.en=t.extend(!0,{},this.regional[""]),this.regional["en-US"]=t.extend(!0,{},this.regional.en),this.dpDiv=n(t("
    "))}function n(e){var i="button, .ui-datepicker-prev, .ui-datepicker-next, .ui-datepicker-calendar td a";return e.on("mouseout",i,function(){t(this).removeClass("ui-state-hover"),-1!==this.className.indexOf("ui-datepicker-prev")&&t(this).removeClass("ui-datepicker-prev-hover"),-1!==this.className.indexOf("ui-datepicker-next")&&t(this).removeClass("ui-datepicker-next-hover")}).on("mouseover",i,o)}function o(){t.datepicker._isDisabledDatepicker(m.inline?m.dpDiv.parent()[0]:m.input[0])||(t(this).parents(".ui-datepicker-calendar").find("a").removeClass("ui-state-hover"),t(this).addClass("ui-state-hover"),-1!==this.className.indexOf("ui-datepicker-prev")&&t(this).addClass("ui-datepicker-prev-hover"),-1!==this.className.indexOf("ui-datepicker-next")&&t(this).addClass("ui-datepicker-next-hover"))}function a(e,i){t.extend(e,i);for(var s in i)null==i[s]&&(e[s]=i[s]);return e}function r(t){return function(){var e=this.element.val();t.apply(this,arguments),this._refresh(),e!==this.element.val()&&this._trigger("change")}}t.ui=t.ui||{},t.ui.version="1.12.1";var h=0,l=Array.prototype.slice;t.cleanData=function(e){return function(i){var s,n,o;for(o=0;null!=(n=i[o]);o++)try{s=t._data(n,"events"),s&&s.remove&&t(n).triggerHandler("remove")}catch(a){}e(i)}}(t.cleanData),t.widget=function(e,i,s){var n,o,a,r={},h=e.split(".")[0];e=e.split(".")[1];var l=h+"-"+e;return s||(s=i,i=t.Widget),t.isArray(s)&&(s=t.extend.apply(null,[{}].concat(s))),t.expr[":"][l.toLowerCase()]=function(e){return!!t.data(e,l)},t[h]=t[h]||{},n=t[h][e],o=t[h][e]=function(t,e){return this._createWidget?(arguments.length&&this._createWidget(t,e),void 0):new o(t,e)},t.extend(o,n,{version:s.version,_proto:t.extend({},s),_childConstructors:[]}),a=new i,a.options=t.widget.extend({},a.options),t.each(s,function(e,s){return t.isFunction(s)?(r[e]=function(){function t(){return i.prototype[e].apply(this,arguments)}function n(t){return i.prototype[e].apply(this,t)}return function(){var e,i=this._super,o=this._superApply;return this._super=t,this._superApply=n,e=s.apply(this,arguments),this._super=i,this._superApply=o,e}}(),void 0):(r[e]=s,void 0)}),o.prototype=t.widget.extend(a,{widgetEventPrefix:n?a.widgetEventPrefix||e:e},r,{constructor:o,namespace:h,widgetName:e,widgetFullName:l}),n?(t.each(n._childConstructors,function(e,i){var s=i.prototype;t.widget(s.namespace+"."+s.widgetName,o,i._proto)}),delete n._childConstructors):i._childConstructors.push(o),t.widget.bridge(e,o),o},t.widget.extend=function(e){for(var i,s,n=l.call(arguments,1),o=0,a=n.length;a>o;o++)for(i in n[o])s=n[o][i],n[o].hasOwnProperty(i)&&void 0!==s&&(e[i]=t.isPlainObject(s)?t.isPlainObject(e[i])?t.widget.extend({},e[i],s):t.widget.extend({},s):s);return e},t.widget.bridge=function(e,i){var s=i.prototype.widgetFullName||e;t.fn[e]=function(n){var o="string"==typeof n,a=l.call(arguments,1),r=this;return o?this.length||"instance"!==n?this.each(function(){var i,o=t.data(this,s);return"instance"===n?(r=o,!1):o?t.isFunction(o[n])&&"_"!==n.charAt(0)?(i=o[n].apply(o,a),i!==o&&void 0!==i?(r=i&&i.jquery?r.pushStack(i.get()):i,!1):void 0):t.error("no such method '"+n+"' for "+e+" widget instance"):t.error("cannot call methods on "+e+" prior to initialization; "+"attempted to call method '"+n+"'")}):r=void 0:(a.length&&(n=t.widget.extend.apply(null,[n].concat(a))),this.each(function(){var e=t.data(this,s);e?(e.option(n||{}),e._init&&e._init()):t.data(this,s,new i(n,this))})),r}},t.Widget=function(){},t.Widget._childConstructors=[],t.Widget.prototype={widgetName:"widget",widgetEventPrefix:"",defaultElement:"
    ",options:{classes:{},disabled:!1,create:null},_createWidget:function(e,i){i=t(i||this.defaultElement||this)[0],this.element=t(i),this.uuid=h++,this.eventNamespace="."+this.widgetName+this.uuid,this.bindings=t(),this.hoverable=t(),this.focusable=t(),this.classesElementLookup={},i!==this&&(t.data(i,this.widgetFullName,this),this._on(!0,this.element,{remove:function(t){t.target===i&&this.destroy()}}),this.document=t(i.style?i.ownerDocument:i.document||i),this.window=t(this.document[0].defaultView||this.document[0].parentWindow)),this.options=t.widget.extend({},this.options,this._getCreateOptions(),e),this._create(),this.options.disabled&&this._setOptionDisabled(this.options.disabled),this._trigger("create",null,this._getCreateEventData()),this._init()},_getCreateOptions:function(){return{}},_getCreateEventData:t.noop,_create:t.noop,_init:t.noop,destroy:function(){var e=this;this._destroy(),t.each(this.classesElementLookup,function(t,i){e._removeClass(i,t)}),this.element.off(this.eventNamespace).removeData(this.widgetFullName),this.widget().off(this.eventNamespace).removeAttr("aria-disabled"),this.bindings.off(this.eventNamespace)},_destroy:t.noop,widget:function(){return this.element},option:function(e,i){var s,n,o,a=e;if(0===arguments.length)return t.widget.extend({},this.options);if("string"==typeof e)if(a={},s=e.split("."),e=s.shift(),s.length){for(n=a[e]=t.widget.extend({},this.options[e]),o=0;s.length-1>o;o++)n[s[o]]=n[s[o]]||{},n=n[s[o]];if(e=s.pop(),1===arguments.length)return void 0===n[e]?null:n[e];n[e]=i}else{if(1===arguments.length)return void 0===this.options[e]?null:this.options[e];a[e]=i}return this._setOptions(a),this},_setOptions:function(t){var e;for(e in t)this._setOption(e,t[e]);return this},_setOption:function(t,e){return"classes"===t&&this._setOptionClasses(e),this.options[t]=e,"disabled"===t&&this._setOptionDisabled(e),this},_setOptionClasses:function(e){var i,s,n;for(i in e)n=this.classesElementLookup[i],e[i]!==this.options.classes[i]&&n&&n.length&&(s=t(n.get()),this._removeClass(n,i),s.addClass(this._classes({element:s,keys:i,classes:e,add:!0})))},_setOptionDisabled:function(t){this._toggleClass(this.widget(),this.widgetFullName+"-disabled",null,!!t),t&&(this._removeClass(this.hoverable,null,"ui-state-hover"),this._removeClass(this.focusable,null,"ui-state-focus"))},enable:function(){return this._setOptions({disabled:!1})},disable:function(){return this._setOptions({disabled:!0})},_classes:function(e){function i(i,o){var a,r;for(r=0;i.length>r;r++)a=n.classesElementLookup[i[r]]||t(),a=e.add?t(t.unique(a.get().concat(e.element.get()))):t(a.not(e.element).get()),n.classesElementLookup[i[r]]=a,s.push(i[r]),o&&e.classes[i[r]]&&s.push(e.classes[i[r]])}var s=[],n=this;return e=t.extend({element:this.element,classes:this.options.classes||{}},e),this._on(e.element,{remove:"_untrackClassesElement"}),e.keys&&i(e.keys.match(/\S+/g)||[],!0),e.extra&&i(e.extra.match(/\S+/g)||[]),s.join(" ")},_untrackClassesElement:function(e){var i=this;t.each(i.classesElementLookup,function(s,n){-1!==t.inArray(e.target,n)&&(i.classesElementLookup[s]=t(n.not(e.target).get()))})},_removeClass:function(t,e,i){return this._toggleClass(t,e,i,!1)},_addClass:function(t,e,i){return this._toggleClass(t,e,i,!0)},_toggleClass:function(t,e,i,s){s="boolean"==typeof s?s:i;var n="string"==typeof t||null===t,o={extra:n?e:i,keys:n?t:e,element:n?this.element:t,add:s};return o.element.toggleClass(this._classes(o),s),this},_on:function(e,i,s){var n,o=this;"boolean"!=typeof e&&(s=i,i=e,e=!1),s?(i=n=t(i),this.bindings=this.bindings.add(i)):(s=i,i=this.element,n=this.widget()),t.each(s,function(s,a){function r(){return e||o.options.disabled!==!0&&!t(this).hasClass("ui-state-disabled")?("string"==typeof a?o[a]:a).apply(o,arguments):void 0}"string"!=typeof a&&(r.guid=a.guid=a.guid||r.guid||t.guid++);var h=s.match(/^([\w:-]*)\s*(.*)$/),l=h[1]+o.eventNamespace,c=h[2];c?n.on(l,c,r):i.on(l,r)})},_off:function(e,i){i=(i||"").split(" ").join(this.eventNamespace+" ")+this.eventNamespace,e.off(i).off(i),this.bindings=t(this.bindings.not(e).get()),this.focusable=t(this.focusable.not(e).get()),this.hoverable=t(this.hoverable.not(e).get())},_delay:function(t,e){function i(){return("string"==typeof t?s[t]:t).apply(s,arguments)}var s=this;return setTimeout(i,e||0)},_hoverable:function(e){this.hoverable=this.hoverable.add(e),this._on(e,{mouseenter:function(e){this._addClass(t(e.currentTarget),null,"ui-state-hover")},mouseleave:function(e){this._removeClass(t(e.currentTarget),null,"ui-state-hover")}})},_focusable:function(e){this.focusable=this.focusable.add(e),this._on(e,{focusin:function(e){this._addClass(t(e.currentTarget),null,"ui-state-focus")},focusout:function(e){this._removeClass(t(e.currentTarget),null,"ui-state-focus")}})},_trigger:function(e,i,s){var n,o,a=this.options[e];if(s=s||{},i=t.Event(i),i.type=(e===this.widgetEventPrefix?e:this.widgetEventPrefix+e).toLowerCase(),i.target=this.element[0],o=i.originalEvent)for(n in o)n in i||(i[n]=o[n]);return this.element.trigger(i,s),!(t.isFunction(a)&&a.apply(this.element[0],[i].concat(s))===!1||i.isDefaultPrevented())}},t.each({show:"fadeIn",hide:"fadeOut"},function(e,i){t.Widget.prototype["_"+e]=function(s,n,o){"string"==typeof n&&(n={effect:n});var a,r=n?n===!0||"number"==typeof n?i:n.effect||i:e;n=n||{},"number"==typeof n&&(n={duration:n}),a=!t.isEmptyObject(n),n.complete=o,n.delay&&s.delay(n.delay),a&&t.effects&&t.effects.effect[r]?s[e](n):r!==e&&s[r]?s[r](n.duration,n.easing,o):s.queue(function(i){t(this)[e](),o&&o.call(s[0]),i()})}}),t.widget,function(){function e(t,e,i){return[parseFloat(t[0])*(u.test(t[0])?e/100:1),parseFloat(t[1])*(u.test(t[1])?i/100:1)]}function i(e,i){return parseInt(t.css(e,i),10)||0}function s(e){var i=e[0];return 9===i.nodeType?{width:e.width(),height:e.height(),offset:{top:0,left:0}}:t.isWindow(i)?{width:e.width(),height:e.height(),offset:{top:e.scrollTop(),left:e.scrollLeft()}}:i.preventDefault?{width:0,height:0,offset:{top:i.pageY,left:i.pageX}}:{width:e.outerWidth(),height:e.outerHeight(),offset:e.offset()}}var n,o=Math.max,a=Math.abs,r=/left|center|right/,h=/top|center|bottom/,l=/[\+\-]\d+(\.[\d]+)?%?/,c=/^\w+/,u=/%$/,d=t.fn.position;t.position={scrollbarWidth:function(){if(void 0!==n)return n;var e,i,s=t("
    "),o=s.children()[0];return t("body").append(s),e=o.offsetWidth,s.css("overflow","scroll"),i=o.offsetWidth,e===i&&(i=s[0].clientWidth),s.remove(),n=e-i},getScrollInfo:function(e){var i=e.isWindow||e.isDocument?"":e.element.css("overflow-x"),s=e.isWindow||e.isDocument?"":e.element.css("overflow-y"),n="scroll"===i||"auto"===i&&e.widthi?"left":e>0?"right":"center",vertical:0>r?"top":s>0?"bottom":"middle"};l>p&&p>a(e+i)&&(u.horizontal="center"),c>f&&f>a(s+r)&&(u.vertical="middle"),u.important=o(a(e),a(i))>o(a(s),a(r))?"horizontal":"vertical",n.using.call(this,t,u)}),h.offset(t.extend(D,{using:r}))})},t.ui.position={fit:{left:function(t,e){var i,s=e.within,n=s.isWindow?s.scrollLeft:s.offset.left,a=s.width,r=t.left-e.collisionPosition.marginLeft,h=n-r,l=r+e.collisionWidth-a-n;e.collisionWidth>a?h>0&&0>=l?(i=t.left+h+e.collisionWidth-a-n,t.left+=h-i):t.left=l>0&&0>=h?n:h>l?n+a-e.collisionWidth:n:h>0?t.left+=h:l>0?t.left-=l:t.left=o(t.left-r,t.left)},top:function(t,e){var i,s=e.within,n=s.isWindow?s.scrollTop:s.offset.top,a=e.within.height,r=t.top-e.collisionPosition.marginTop,h=n-r,l=r+e.collisionHeight-a-n;e.collisionHeight>a?h>0&&0>=l?(i=t.top+h+e.collisionHeight-a-n,t.top+=h-i):t.top=l>0&&0>=h?n:h>l?n+a-e.collisionHeight:n:h>0?t.top+=h:l>0?t.top-=l:t.top=o(t.top-r,t.top)}},flip:{left:function(t,e){var i,s,n=e.within,o=n.offset.left+n.scrollLeft,r=n.width,h=n.isWindow?n.scrollLeft:n.offset.left,l=t.left-e.collisionPosition.marginLeft,c=l-h,u=l+e.collisionWidth-r-h,d="left"===e.my[0]?-e.elemWidth:"right"===e.my[0]?e.elemWidth:0,p="left"===e.at[0]?e.targetWidth:"right"===e.at[0]?-e.targetWidth:0,f=-2*e.offset[0];0>c?(i=t.left+d+p+f+e.collisionWidth-r-o,(0>i||a(c)>i)&&(t.left+=d+p+f)):u>0&&(s=t.left-e.collisionPosition.marginLeft+d+p+f-h,(s>0||u>a(s))&&(t.left+=d+p+f))},top:function(t,e){var i,s,n=e.within,o=n.offset.top+n.scrollTop,r=n.height,h=n.isWindow?n.scrollTop:n.offset.top,l=t.top-e.collisionPosition.marginTop,c=l-h,u=l+e.collisionHeight-r-h,d="top"===e.my[1],p=d?-e.elemHeight:"bottom"===e.my[1]?e.elemHeight:0,f="top"===e.at[1]?e.targetHeight:"bottom"===e.at[1]?-e.targetHeight:0,g=-2*e.offset[1];0>c?(s=t.top+p+f+g+e.collisionHeight-r-o,(0>s||a(c)>s)&&(t.top+=p+f+g)):u>0&&(i=t.top-e.collisionPosition.marginTop+p+f+g-h,(i>0||u>a(i))&&(t.top+=p+f+g))}},flipfit:{left:function(){t.ui.position.flip.left.apply(this,arguments),t.ui.position.fit.left.apply(this,arguments)},top:function(){t.ui.position.flip.top.apply(this,arguments),t.ui.position.fit.top.apply(this,arguments)}}}}(),t.ui.position,t.extend(t.expr[":"],{data:t.expr.createPseudo?t.expr.createPseudo(function(e){return function(i){return!!t.data(i,e)}}):function(e,i,s){return!!t.data(e,s[3])}}),t.fn.extend({disableSelection:function(){var t="onselectstart"in document.createElement("div")?"selectstart":"mousedown";return function(){return this.on(t+".ui-disableSelection",function(t){t.preventDefault()})}}(),enableSelection:function(){return this.off(".ui-disableSelection")}});var c="ui-effects-",u="ui-effects-style",d="ui-effects-animated",p=t;t.effects={effect:{}},function(t,e){function i(t,e,i){var s=u[e.type]||{};return null==t?i||!e.def?null:e.def:(t=s.floor?~~t:parseFloat(t),isNaN(t)?e.def:s.mod?(t+s.mod)%s.mod:0>t?0:t>s.max?s.max:t)}function s(i){var s=l(),n=s._rgba=[];return i=i.toLowerCase(),f(h,function(t,o){var a,r=o.re.exec(i),h=r&&o.parse(r),l=o.space||"rgba";return h?(a=s[l](h),s[c[l].cache]=a[c[l].cache],n=s._rgba=a._rgba,!1):e}),n.length?("0,0,0,0"===n.join()&&t.extend(n,o.transparent),s):o[i]}function n(t,e,i){return i=(i+1)%1,1>6*i?t+6*(e-t)*i:1>2*i?e:2>3*i?t+6*(e-t)*(2/3-i):t}var o,a="backgroundColor borderBottomColor borderLeftColor borderRightColor borderTopColor color columnRuleColor outlineColor textDecorationColor textEmphasisColor",r=/^([\-+])=\s*(\d+\.?\d*)/,h=[{re:/rgba?\(\s*(\d{1,3})\s*,\s*(\d{1,3})\s*,\s*(\d{1,3})\s*(?:,\s*(\d?(?:\.\d+)?)\s*)?\)/,parse:function(t){return[t[1],t[2],t[3],t[4]]}},{re:/rgba?\(\s*(\d+(?:\.\d+)?)\%\s*,\s*(\d+(?:\.\d+)?)\%\s*,\s*(\d+(?:\.\d+)?)\%\s*(?:,\s*(\d?(?:\.\d+)?)\s*)?\)/,parse:function(t){return[2.55*t[1],2.55*t[2],2.55*t[3],t[4]]}},{re:/#([a-f0-9]{2})([a-f0-9]{2})([a-f0-9]{2})/,parse:function(t){return[parseInt(t[1],16),parseInt(t[2],16),parseInt(t[3],16)]}},{re:/#([a-f0-9])([a-f0-9])([a-f0-9])/,parse:function(t){return[parseInt(t[1]+t[1],16),parseInt(t[2]+t[2],16),parseInt(t[3]+t[3],16)]}},{re:/hsla?\(\s*(\d+(?:\.\d+)?)\s*,\s*(\d+(?:\.\d+)?)\%\s*,\s*(\d+(?:\.\d+)?)\%\s*(?:,\s*(\d?(?:\.\d+)?)\s*)?\)/,space:"hsla",parse:function(t){return[t[1],t[2]/100,t[3]/100,t[4]]}}],l=t.Color=function(e,i,s,n){return new t.Color.fn.parse(e,i,s,n)},c={rgba:{props:{red:{idx:0,type:"byte"},green:{idx:1,type:"byte"},blue:{idx:2,type:"byte"}}},hsla:{props:{hue:{idx:0,type:"degrees"},saturation:{idx:1,type:"percent"},lightness:{idx:2,type:"percent"}}}},u={"byte":{floor:!0,max:255},percent:{max:1},degrees:{mod:360,floor:!0}},d=l.support={},p=t("

    ")[0],f=t.each;p.style.cssText="background-color:rgba(1,1,1,.5)",d.rgba=p.style.backgroundColor.indexOf("rgba")>-1,f(c,function(t,e){e.cache="_"+t,e.props.alpha={idx:3,type:"percent",def:1}}),l.fn=t.extend(l.prototype,{parse:function(n,a,r,h){if(n===e)return this._rgba=[null,null,null,null],this;(n.jquery||n.nodeType)&&(n=t(n).css(a),a=e);var u=this,d=t.type(n),p=this._rgba=[];return a!==e&&(n=[n,a,r,h],d="array"),"string"===d?this.parse(s(n)||o._default):"array"===d?(f(c.rgba.props,function(t,e){p[e.idx]=i(n[e.idx],e)}),this):"object"===d?(n instanceof l?f(c,function(t,e){n[e.cache]&&(u[e.cache]=n[e.cache].slice())}):f(c,function(e,s){var o=s.cache;f(s.props,function(t,e){if(!u[o]&&s.to){if("alpha"===t||null==n[t])return;u[o]=s.to(u._rgba)}u[o][e.idx]=i(n[t],e,!0)}),u[o]&&0>t.inArray(null,u[o].slice(0,3))&&(u[o][3]=1,s.from&&(u._rgba=s.from(u[o])))}),this):e},is:function(t){var i=l(t),s=!0,n=this;return f(c,function(t,o){var a,r=i[o.cache];return r&&(a=n[o.cache]||o.to&&o.to(n._rgba)||[],f(o.props,function(t,i){return null!=r[i.idx]?s=r[i.idx]===a[i.idx]:e})),s}),s},_space:function(){var t=[],e=this;return f(c,function(i,s){e[s.cache]&&t.push(i)}),t.pop()},transition:function(t,e){var s=l(t),n=s._space(),o=c[n],a=0===this.alpha()?l("transparent"):this,r=a[o.cache]||o.to(a._rgba),h=r.slice();return s=s[o.cache],f(o.props,function(t,n){var o=n.idx,a=r[o],l=s[o],c=u[n.type]||{};null!==l&&(null===a?h[o]=l:(c.mod&&(l-a>c.mod/2?a+=c.mod:a-l>c.mod/2&&(a-=c.mod)),h[o]=i((l-a)*e+a,n)))}),this[n](h)},blend:function(e){if(1===this._rgba[3])return this;var i=this._rgba.slice(),s=i.pop(),n=l(e)._rgba;return l(t.map(i,function(t,e){return(1-s)*n[e]+s*t}))},toRgbaString:function(){var e="rgba(",i=t.map(this._rgba,function(t,e){return null==t?e>2?1:0:t});return 1===i[3]&&(i.pop(),e="rgb("),e+i.join()+")"},toHslaString:function(){var e="hsla(",i=t.map(this.hsla(),function(t,e){return null==t&&(t=e>2?1:0),e&&3>e&&(t=Math.round(100*t)+"%"),t});return 1===i[3]&&(i.pop(),e="hsl("),e+i.join()+")"},toHexString:function(e){var i=this._rgba.slice(),s=i.pop();return e&&i.push(~~(255*s)),"#"+t.map(i,function(t){return t=(t||0).toString(16),1===t.length?"0"+t:t}).join("")},toString:function(){return 0===this._rgba[3]?"transparent":this.toRgbaString()}}),l.fn.parse.prototype=l.fn,c.hsla.to=function(t){if(null==t[0]||null==t[1]||null==t[2])return[null,null,null,t[3]];var e,i,s=t[0]/255,n=t[1]/255,o=t[2]/255,a=t[3],r=Math.max(s,n,o),h=Math.min(s,n,o),l=r-h,c=r+h,u=.5*c;return e=h===r?0:s===r?60*(n-o)/l+360:n===r?60*(o-s)/l+120:60*(s-n)/l+240,i=0===l?0:.5>=u?l/c:l/(2-c),[Math.round(e)%360,i,u,null==a?1:a]},c.hsla.from=function(t){if(null==t[0]||null==t[1]||null==t[2])return[null,null,null,t[3]];var e=t[0]/360,i=t[1],s=t[2],o=t[3],a=.5>=s?s*(1+i):s+i-s*i,r=2*s-a;return[Math.round(255*n(r,a,e+1/3)),Math.round(255*n(r,a,e)),Math.round(255*n(r,a,e-1/3)),o]},f(c,function(s,n){var o=n.props,a=n.cache,h=n.to,c=n.from;l.fn[s]=function(s){if(h&&!this[a]&&(this[a]=h(this._rgba)),s===e)return this[a].slice();var n,r=t.type(s),u="array"===r||"object"===r?s:arguments,d=this[a].slice();return f(o,function(t,e){var s=u["object"===r?t:e.idx];null==s&&(s=d[e.idx]),d[e.idx]=i(s,e)}),c?(n=l(c(d)),n[a]=d,n):l(d)},f(o,function(e,i){l.fn[e]||(l.fn[e]=function(n){var o,a=t.type(n),h="alpha"===e?this._hsla?"hsla":"rgba":s,l=this[h](),c=l[i.idx];return"undefined"===a?c:("function"===a&&(n=n.call(this,c),a=t.type(n)),null==n&&i.empty?this:("string"===a&&(o=r.exec(n),o&&(n=c+parseFloat(o[2])*("+"===o[1]?1:-1))),l[i.idx]=n,this[h](l)))})})}),l.hook=function(e){var i=e.split(" ");f(i,function(e,i){t.cssHooks[i]={set:function(e,n){var o,a,r="";if("transparent"!==n&&("string"!==t.type(n)||(o=s(n)))){if(n=l(o||n),!d.rgba&&1!==n._rgba[3]){for(a="backgroundColor"===i?e.parentNode:e;(""===r||"transparent"===r)&&a&&a.style;)try{r=t.css(a,"backgroundColor"),a=a.parentNode}catch(h){}n=n.blend(r&&"transparent"!==r?r:"_default")}n=n.toRgbaString()}try{e.style[i]=n}catch(h){}}},t.fx.step[i]=function(e){e.colorInit||(e.start=l(e.elem,i),e.end=l(e.end),e.colorInit=!0),t.cssHooks[i].set(e.elem,e.start.transition(e.end,e.pos))}})},l.hook(a),t.cssHooks.borderColor={expand:function(t){var e={};return f(["Top","Right","Bottom","Left"],function(i,s){e["border"+s+"Color"]=t}),e}},o=t.Color.names={aqua:"#00ffff",black:"#000000",blue:"#0000ff",fuchsia:"#ff00ff",gray:"#808080",green:"#008000",lime:"#00ff00",maroon:"#800000",navy:"#000080",olive:"#808000",purple:"#800080",red:"#ff0000",silver:"#c0c0c0",teal:"#008080",white:"#ffffff",yellow:"#ffff00",transparent:[null,null,null,0],_default:"#ffffff"}}(p),function(){function e(e){var i,s,n=e.ownerDocument.defaultView?e.ownerDocument.defaultView.getComputedStyle(e,null):e.currentStyle,o={};if(n&&n.length&&n[0]&&n[n[0]])for(s=n.length;s--;)i=n[s],"string"==typeof n[i]&&(o[t.camelCase(i)]=n[i]);else for(i in n)"string"==typeof n[i]&&(o[i]=n[i]);return o}function i(e,i){var s,o,a={};for(s in i)o=i[s],e[s]!==o&&(n[s]||(t.fx.step[s]||!isNaN(parseFloat(o)))&&(a[s]=o));return a}var s=["add","remove","toggle"],n={border:1,borderBottom:1,borderColor:1,borderLeft:1,borderRight:1,borderTop:1,borderWidth:1,margin:1,padding:1};t.each(["borderLeftStyle","borderRightStyle","borderBottomStyle","borderTopStyle"],function(e,i){t.fx.step[i]=function(t){("none"!==t.end&&!t.setAttr||1===t.pos&&!t.setAttr)&&(p.style(t.elem,i,t.end),t.setAttr=!0)}}),t.fn.addBack||(t.fn.addBack=function(t){return this.add(null==t?this.prevObject:this.prevObject.filter(t))}),t.effects.animateClass=function(n,o,a,r){var h=t.speed(o,a,r);return this.queue(function(){var o,a=t(this),r=a.attr("class")||"",l=h.children?a.find("*").addBack():a;l=l.map(function(){var i=t(this);return{el:i,start:e(this)}}),o=function(){t.each(s,function(t,e){n[e]&&a[e+"Class"](n[e])})},o(),l=l.map(function(){return this.end=e(this.el[0]),this.diff=i(this.start,this.end),this}),a.attr("class",r),l=l.map(function(){var e=this,i=t.Deferred(),s=t.extend({},h,{queue:!1,complete:function(){i.resolve(e)}});return this.el.animate(this.diff,s),i.promise()}),t.when.apply(t,l.get()).done(function(){o(),t.each(arguments,function(){var e=this.el;t.each(this.diff,function(t){e.css(t,"")})}),h.complete.call(a[0])})})},t.fn.extend({addClass:function(e){return function(i,s,n,o){return s?t.effects.animateClass.call(this,{add:i},s,n,o):e.apply(this,arguments)}}(t.fn.addClass),removeClass:function(e){return function(i,s,n,o){return arguments.length>1?t.effects.animateClass.call(this,{remove:i},s,n,o):e.apply(this,arguments)}}(t.fn.removeClass),toggleClass:function(e){return function(i,s,n,o,a){return"boolean"==typeof s||void 0===s?n?t.effects.animateClass.call(this,s?{add:i}:{remove:i},n,o,a):e.apply(this,arguments):t.effects.animateClass.call(this,{toggle:i},s,n,o)}}(t.fn.toggleClass),switchClass:function(e,i,s,n,o){return t.effects.animateClass.call(this,{add:i,remove:e},s,n,o)}})}(),function(){function e(e,i,s,n){return t.isPlainObject(e)&&(i=e,e=e.effect),e={effect:e},null==i&&(i={}),t.isFunction(i)&&(n=i,s=null,i={}),("number"==typeof i||t.fx.speeds[i])&&(n=s,s=i,i={}),t.isFunction(s)&&(n=s,s=null),i&&t.extend(e,i),s=s||i.duration,e.duration=t.fx.off?0:"number"==typeof s?s:s in t.fx.speeds?t.fx.speeds[s]:t.fx.speeds._default,e.complete=n||i.complete,e}function i(e){return!e||"number"==typeof e||t.fx.speeds[e]?!0:"string"!=typeof e||t.effects.effect[e]?t.isFunction(e)?!0:"object"!=typeof e||e.effect?!1:!0:!0}function s(t,e){var i=e.outerWidth(),s=e.outerHeight(),n=/^rect\((-?\d*\.?\d*px|-?\d+%|auto),?\s*(-?\d*\.?\d*px|-?\d+%|auto),?\s*(-?\d*\.?\d*px|-?\d+%|auto),?\s*(-?\d*\.?\d*px|-?\d+%|auto)\)$/,o=n.exec(t)||["",0,i,s,0];return{top:parseFloat(o[1])||0,right:"auto"===o[2]?i:parseFloat(o[2]),bottom:"auto"===o[3]?s:parseFloat(o[3]),left:parseFloat(o[4])||0}}t.expr&&t.expr.filters&&t.expr.filters.animated&&(t.expr.filters.animated=function(e){return function(i){return!!t(i).data(d)||e(i)}}(t.expr.filters.animated)),t.uiBackCompat!==!1&&t.extend(t.effects,{save:function(t,e){for(var i=0,s=e.length;s>i;i++)null!==e[i]&&t.data(c+e[i],t[0].style[e[i]])},restore:function(t,e){for(var i,s=0,n=e.length;n>s;s++)null!==e[s]&&(i=t.data(c+e[s]),t.css(e[s],i))},setMode:function(t,e){return"toggle"===e&&(e=t.is(":hidden")?"show":"hide"),e},createWrapper:function(e){if(e.parent().is(".ui-effects-wrapper"))return e.parent();var i={width:e.outerWidth(!0),height:e.outerHeight(!0),"float":e.css("float")},s=t("

    ").addClass("ui-effects-wrapper").css({fontSize:"100%",background:"transparent",border:"none",margin:0,padding:0}),n={width:e.width(),height:e.height()},o=document.activeElement;try{o.id}catch(a){o=document.body}return e.wrap(s),(e[0]===o||t.contains(e[0],o))&&t(o).trigger("focus"),s=e.parent(),"static"===e.css("position")?(s.css({position:"relative"}),e.css({position:"relative"})):(t.extend(i,{position:e.css("position"),zIndex:e.css("z-index")}),t.each(["top","left","bottom","right"],function(t,s){i[s]=e.css(s),isNaN(parseInt(i[s],10))&&(i[s]="auto")}),e.css({position:"relative",top:0,left:0,right:"auto",bottom:"auto"})),e.css(n),s.css(i).show()},removeWrapper:function(e){var i=document.activeElement;return e.parent().is(".ui-effects-wrapper")&&(e.parent().replaceWith(e),(e[0]===i||t.contains(e[0],i))&&t(i).trigger("focus")),e}}),t.extend(t.effects,{version:"1.12.1",define:function(e,i,s){return s||(s=i,i="effect"),t.effects.effect[e]=s,t.effects.effect[e].mode=i,s},scaledDimensions:function(t,e,i){if(0===e)return{height:0,width:0,outerHeight:0,outerWidth:0};var s="horizontal"!==i?(e||100)/100:1,n="vertical"!==i?(e||100)/100:1;return{height:t.height()*n,width:t.width()*s,outerHeight:t.outerHeight()*n,outerWidth:t.outerWidth()*s}},clipToBox:function(t){return{width:t.clip.right-t.clip.left,height:t.clip.bottom-t.clip.top,left:t.clip.left,top:t.clip.top}},unshift:function(t,e,i){var s=t.queue();e>1&&s.splice.apply(s,[1,0].concat(s.splice(e,i))),t.dequeue()},saveStyle:function(t){t.data(u,t[0].style.cssText)},restoreStyle:function(t){t[0].style.cssText=t.data(u)||"",t.removeData(u)},mode:function(t,e){var i=t.is(":hidden");return"toggle"===e&&(e=i?"show":"hide"),(i?"hide"===e:"show"===e)&&(e="none"),e},getBaseline:function(t,e){var i,s;switch(t[0]){case"top":i=0;break;case"middle":i=.5;break;case"bottom":i=1;break;default:i=t[0]/e.height}switch(t[1]){case"left":s=0;break;case"center":s=.5;break;case"right":s=1;break;default:s=t[1]/e.width}return{x:s,y:i}},createPlaceholder:function(e){var i,s=e.css("position"),n=e.position();return e.css({marginTop:e.css("marginTop"),marginBottom:e.css("marginBottom"),marginLeft:e.css("marginLeft"),marginRight:e.css("marginRight")}).outerWidth(e.outerWidth()).outerHeight(e.outerHeight()),/^(static|relative)/.test(s)&&(s="absolute",i=t("<"+e[0].nodeName+">").insertAfter(e).css({display:/^(inline|ruby)/.test(e.css("display"))?"inline-block":"block",visibility:"hidden",marginTop:e.css("marginTop"),marginBottom:e.css("marginBottom"),marginLeft:e.css("marginLeft"),marginRight:e.css("marginRight"),"float":e.css("float")}).outerWidth(e.outerWidth()).outerHeight(e.outerHeight()).addClass("ui-effects-placeholder"),e.data(c+"placeholder",i)),e.css({position:s,left:n.left,top:n.top}),i},removePlaceholder:function(t){var e=c+"placeholder",i=t.data(e);i&&(i.remove(),t.removeData(e))},cleanUp:function(e){t.effects.restoreStyle(e),t.effects.removePlaceholder(e)},setTransition:function(e,i,s,n){return n=n||{},t.each(i,function(t,i){var o=e.cssUnit(i);o[0]>0&&(n[i]=o[0]*s+o[1])}),n}}),t.fn.extend({effect:function(){function i(e){function i(){r.removeData(d),t.effects.cleanUp(r),"hide"===s.mode&&r.hide(),a()}function a(){t.isFunction(h)&&h.call(r[0]),t.isFunction(e)&&e()}var r=t(this);s.mode=c.shift(),t.uiBackCompat===!1||o?"none"===s.mode?(r[l](),a()):n.call(r[0],s,i):(r.is(":hidden")?"hide"===l:"show"===l)?(r[l](),a()):n.call(r[0],s,a)}var s=e.apply(this,arguments),n=t.effects.effect[s.effect],o=n.mode,a=s.queue,r=a||"fx",h=s.complete,l=s.mode,c=[],u=function(e){var i=t(this),s=t.effects.mode(i,l)||o;i.data(d,!0),c.push(s),o&&("show"===s||s===o&&"hide"===s)&&i.show(),o&&"none"===s||t.effects.saveStyle(i),t.isFunction(e)&&e()};return t.fx.off||!n?l?this[l](s.duration,h):this.each(function(){h&&h.call(this)}):a===!1?this.each(u).each(i):this.queue(r,u).queue(r,i)},show:function(t){return function(s){if(i(s))return t.apply(this,arguments);var n=e.apply(this,arguments);return n.mode="show",this.effect.call(this,n) -}}(t.fn.show),hide:function(t){return function(s){if(i(s))return t.apply(this,arguments);var n=e.apply(this,arguments);return n.mode="hide",this.effect.call(this,n)}}(t.fn.hide),toggle:function(t){return function(s){if(i(s)||"boolean"==typeof s)return t.apply(this,arguments);var n=e.apply(this,arguments);return n.mode="toggle",this.effect.call(this,n)}}(t.fn.toggle),cssUnit:function(e){var i=this.css(e),s=[];return t.each(["em","px","%","pt"],function(t,e){i.indexOf(e)>0&&(s=[parseFloat(i),e])}),s},cssClip:function(t){return t?this.css("clip","rect("+t.top+"px "+t.right+"px "+t.bottom+"px "+t.left+"px)"):s(this.css("clip"),this)},transfer:function(e,i){var s=t(this),n=t(e.to),o="fixed"===n.css("position"),a=t("body"),r=o?a.scrollTop():0,h=o?a.scrollLeft():0,l=n.offset(),c={top:l.top-r,left:l.left-h,height:n.innerHeight(),width:n.innerWidth()},u=s.offset(),d=t("
    ").appendTo("body").addClass(e.className).css({top:u.top-r,left:u.left-h,height:s.innerHeight(),width:s.innerWidth(),position:o?"fixed":"absolute"}).animate(c,e.duration,e.easing,function(){d.remove(),t.isFunction(i)&&i()})}}),t.fx.step.clip=function(e){e.clipInit||(e.start=t(e.elem).cssClip(),"string"==typeof e.end&&(e.end=s(e.end,e.elem)),e.clipInit=!0),t(e.elem).cssClip({top:e.pos*(e.end.top-e.start.top)+e.start.top,right:e.pos*(e.end.right-e.start.right)+e.start.right,bottom:e.pos*(e.end.bottom-e.start.bottom)+e.start.bottom,left:e.pos*(e.end.left-e.start.left)+e.start.left})}}(),function(){var e={};t.each(["Quad","Cubic","Quart","Quint","Expo"],function(t,i){e[i]=function(e){return Math.pow(e,t+2)}}),t.extend(e,{Sine:function(t){return 1-Math.cos(t*Math.PI/2)},Circ:function(t){return 1-Math.sqrt(1-t*t)},Elastic:function(t){return 0===t||1===t?t:-Math.pow(2,8*(t-1))*Math.sin((80*(t-1)-7.5)*Math.PI/15)},Back:function(t){return t*t*(3*t-2)},Bounce:function(t){for(var e,i=4;((e=Math.pow(2,--i))-1)/11>t;);return 1/Math.pow(4,3-i)-7.5625*Math.pow((3*e-2)/22-t,2)}}),t.each(e,function(e,i){t.easing["easeIn"+e]=i,t.easing["easeOut"+e]=function(t){return 1-i(1-t)},t.easing["easeInOut"+e]=function(t){return.5>t?i(2*t)/2:1-i(-2*t+2)/2}})}();var f=t.effects;t.effects.define("blind","hide",function(e,i){var s={up:["bottom","top"],vertical:["bottom","top"],down:["top","bottom"],left:["right","left"],horizontal:["right","left"],right:["left","right"]},n=t(this),o=e.direction||"up",a=n.cssClip(),r={clip:t.extend({},a)},h=t.effects.createPlaceholder(n);r.clip[s[o][0]]=r.clip[s[o][1]],"show"===e.mode&&(n.cssClip(r.clip),h&&h.css(t.effects.clipToBox(r)),r.clip=a),h&&h.animate(t.effects.clipToBox(r),e.duration,e.easing),n.animate(r,{queue:!1,duration:e.duration,easing:e.easing,complete:i})}),t.effects.define("bounce",function(e,i){var s,n,o,a=t(this),r=e.mode,h="hide"===r,l="show"===r,c=e.direction||"up",u=e.distance,d=e.times||5,p=2*d+(l||h?1:0),f=e.duration/p,g=e.easing,m="up"===c||"down"===c?"top":"left",_="up"===c||"left"===c,v=0,b=a.queue().length;for(t.effects.createPlaceholder(a),o=a.css(m),u||(u=a["top"===m?"outerHeight":"outerWidth"]()/3),l&&(n={opacity:1},n[m]=o,a.css("opacity",0).css(m,_?2*-u:2*u).animate(n,f,g)),h&&(u/=Math.pow(2,d-1)),n={},n[m]=o;d>v;v++)s={},s[m]=(_?"-=":"+=")+u,a.animate(s,f,g).animate(n,f,g),u=h?2*u:u/2;h&&(s={opacity:0},s[m]=(_?"-=":"+=")+u,a.animate(s,f,g)),a.queue(i),t.effects.unshift(a,b,p+1)}),t.effects.define("clip","hide",function(e,i){var s,n={},o=t(this),a=e.direction||"vertical",r="both"===a,h=r||"horizontal"===a,l=r||"vertical"===a;s=o.cssClip(),n.clip={top:l?(s.bottom-s.top)/2:s.top,right:h?(s.right-s.left)/2:s.right,bottom:l?(s.bottom-s.top)/2:s.bottom,left:h?(s.right-s.left)/2:s.left},t.effects.createPlaceholder(o),"show"===e.mode&&(o.cssClip(n.clip),n.clip=s),o.animate(n,{queue:!1,duration:e.duration,easing:e.easing,complete:i})}),t.effects.define("drop","hide",function(e,i){var s,n=t(this),o=e.mode,a="show"===o,r=e.direction||"left",h="up"===r||"down"===r?"top":"left",l="up"===r||"left"===r?"-=":"+=",c="+="===l?"-=":"+=",u={opacity:0};t.effects.createPlaceholder(n),s=e.distance||n["top"===h?"outerHeight":"outerWidth"](!0)/2,u[h]=l+s,a&&(n.css(u),u[h]=c+s,u.opacity=1),n.animate(u,{queue:!1,duration:e.duration,easing:e.easing,complete:i})}),t.effects.define("explode","hide",function(e,i){function s(){b.push(this),b.length===u*d&&n()}function n(){p.css({visibility:"visible"}),t(b).remove(),i()}var o,a,r,h,l,c,u=e.pieces?Math.round(Math.sqrt(e.pieces)):3,d=u,p=t(this),f=e.mode,g="show"===f,m=p.show().css("visibility","hidden").offset(),_=Math.ceil(p.outerWidth()/d),v=Math.ceil(p.outerHeight()/u),b=[];for(o=0;u>o;o++)for(h=m.top+o*v,c=o-(u-1)/2,a=0;d>a;a++)r=m.left+a*_,l=a-(d-1)/2,p.clone().appendTo("body").wrap("
    ").css({position:"absolute",visibility:"visible",left:-a*_,top:-o*v}).parent().addClass("ui-effects-explode").css({position:"absolute",overflow:"hidden",width:_,height:v,left:r+(g?l*_:0),top:h+(g?c*v:0),opacity:g?0:1}).animate({left:r+(g?0:l*_),top:h+(g?0:c*v),opacity:g?1:0},e.duration||500,e.easing,s)}),t.effects.define("fade","toggle",function(e,i){var s="show"===e.mode;t(this).css("opacity",s?0:1).animate({opacity:s?1:0},{queue:!1,duration:e.duration,easing:e.easing,complete:i})}),t.effects.define("fold","hide",function(e,i){var s=t(this),n=e.mode,o="show"===n,a="hide"===n,r=e.size||15,h=/([0-9]+)%/.exec(r),l=!!e.horizFirst,c=l?["right","bottom"]:["bottom","right"],u=e.duration/2,d=t.effects.createPlaceholder(s),p=s.cssClip(),f={clip:t.extend({},p)},g={clip:t.extend({},p)},m=[p[c[0]],p[c[1]]],_=s.queue().length;h&&(r=parseInt(h[1],10)/100*m[a?0:1]),f.clip[c[0]]=r,g.clip[c[0]]=r,g.clip[c[1]]=0,o&&(s.cssClip(g.clip),d&&d.css(t.effects.clipToBox(g)),g.clip=p),s.queue(function(i){d&&d.animate(t.effects.clipToBox(f),u,e.easing).animate(t.effects.clipToBox(g),u,e.easing),i()}).animate(f,u,e.easing).animate(g,u,e.easing).queue(i),t.effects.unshift(s,_,4)}),t.effects.define("highlight","show",function(e,i){var s=t(this),n={backgroundColor:s.css("backgroundColor")};"hide"===e.mode&&(n.opacity=0),t.effects.saveStyle(s),s.css({backgroundImage:"none",backgroundColor:e.color||"#ffff99"}).animate(n,{queue:!1,duration:e.duration,easing:e.easing,complete:i})}),t.effects.define("size",function(e,i){var s,n,o,a=t(this),r=["fontSize"],h=["borderTopWidth","borderBottomWidth","paddingTop","paddingBottom"],l=["borderLeftWidth","borderRightWidth","paddingLeft","paddingRight"],c=e.mode,u="effect"!==c,d=e.scale||"both",p=e.origin||["middle","center"],f=a.css("position"),g=a.position(),m=t.effects.scaledDimensions(a),_=e.from||m,v=e.to||t.effects.scaledDimensions(a,0);t.effects.createPlaceholder(a),"show"===c&&(o=_,_=v,v=o),n={from:{y:_.height/m.height,x:_.width/m.width},to:{y:v.height/m.height,x:v.width/m.width}},("box"===d||"both"===d)&&(n.from.y!==n.to.y&&(_=t.effects.setTransition(a,h,n.from.y,_),v=t.effects.setTransition(a,h,n.to.y,v)),n.from.x!==n.to.x&&(_=t.effects.setTransition(a,l,n.from.x,_),v=t.effects.setTransition(a,l,n.to.x,v))),("content"===d||"both"===d)&&n.from.y!==n.to.y&&(_=t.effects.setTransition(a,r,n.from.y,_),v=t.effects.setTransition(a,r,n.to.y,v)),p&&(s=t.effects.getBaseline(p,m),_.top=(m.outerHeight-_.outerHeight)*s.y+g.top,_.left=(m.outerWidth-_.outerWidth)*s.x+g.left,v.top=(m.outerHeight-v.outerHeight)*s.y+g.top,v.left=(m.outerWidth-v.outerWidth)*s.x+g.left),a.css(_),("content"===d||"both"===d)&&(h=h.concat(["marginTop","marginBottom"]).concat(r),l=l.concat(["marginLeft","marginRight"]),a.find("*[width]").each(function(){var i=t(this),s=t.effects.scaledDimensions(i),o={height:s.height*n.from.y,width:s.width*n.from.x,outerHeight:s.outerHeight*n.from.y,outerWidth:s.outerWidth*n.from.x},a={height:s.height*n.to.y,width:s.width*n.to.x,outerHeight:s.height*n.to.y,outerWidth:s.width*n.to.x};n.from.y!==n.to.y&&(o=t.effects.setTransition(i,h,n.from.y,o),a=t.effects.setTransition(i,h,n.to.y,a)),n.from.x!==n.to.x&&(o=t.effects.setTransition(i,l,n.from.x,o),a=t.effects.setTransition(i,l,n.to.x,a)),u&&t.effects.saveStyle(i),i.css(o),i.animate(a,e.duration,e.easing,function(){u&&t.effects.restoreStyle(i)})})),a.animate(v,{queue:!1,duration:e.duration,easing:e.easing,complete:function(){var e=a.offset();0===v.opacity&&a.css("opacity",_.opacity),u||(a.css("position","static"===f?"relative":f).offset(e),t.effects.saveStyle(a)),i()}})}),t.effects.define("scale",function(e,i){var s=t(this),n=e.mode,o=parseInt(e.percent,10)||(0===parseInt(e.percent,10)?0:"effect"!==n?0:100),a=t.extend(!0,{from:t.effects.scaledDimensions(s),to:t.effects.scaledDimensions(s,o,e.direction||"both"),origin:e.origin||["middle","center"]},e);e.fade&&(a.from.opacity=1,a.to.opacity=0),t.effects.effect.size.call(this,a,i)}),t.effects.define("puff","hide",function(e,i){var s=t.extend(!0,{},e,{fade:!0,percent:parseInt(e.percent,10)||150});t.effects.effect.scale.call(this,s,i)}),t.effects.define("pulsate","show",function(e,i){var s=t(this),n=e.mode,o="show"===n,a="hide"===n,r=o||a,h=2*(e.times||5)+(r?1:0),l=e.duration/h,c=0,u=1,d=s.queue().length;for((o||!s.is(":visible"))&&(s.css("opacity",0).show(),c=1);h>u;u++)s.animate({opacity:c},l,e.easing),c=1-c;s.animate({opacity:c},l,e.easing),s.queue(i),t.effects.unshift(s,d,h+1)}),t.effects.define("shake",function(e,i){var s=1,n=t(this),o=e.direction||"left",a=e.distance||20,r=e.times||3,h=2*r+1,l=Math.round(e.duration/h),c="up"===o||"down"===o?"top":"left",u="up"===o||"left"===o,d={},p={},f={},g=n.queue().length;for(t.effects.createPlaceholder(n),d[c]=(u?"-=":"+=")+a,p[c]=(u?"+=":"-=")+2*a,f[c]=(u?"-=":"+=")+2*a,n.animate(d,l,e.easing);r>s;s++)n.animate(p,l,e.easing).animate(f,l,e.easing);n.animate(p,l,e.easing).animate(d,l/2,e.easing).queue(i),t.effects.unshift(n,g,h+1)}),t.effects.define("slide","show",function(e,i){var s,n,o=t(this),a={up:["bottom","top"],down:["top","bottom"],left:["right","left"],right:["left","right"]},r=e.mode,h=e.direction||"left",l="up"===h||"down"===h?"top":"left",c="up"===h||"left"===h,u=e.distance||o["top"===l?"outerHeight":"outerWidth"](!0),d={};t.effects.createPlaceholder(o),s=o.cssClip(),n=o.position()[l],d[l]=(c?-1:1)*u+n,d.clip=o.cssClip(),d.clip[a[h][1]]=d.clip[a[h][0]],"show"===r&&(o.cssClip(d.clip),o.css(l,d[l]),d.clip=s,d[l]=n),o.animate(d,{queue:!1,duration:e.duration,easing:e.easing,complete:i})});var f;t.uiBackCompat!==!1&&(f=t.effects.define("transfer",function(e,i){t(this).transfer(e,i)})),t.ui.focusable=function(i,s){var n,o,a,r,h,l=i.nodeName.toLowerCase();return"area"===l?(n=i.parentNode,o=n.name,i.href&&o&&"map"===n.nodeName.toLowerCase()?(a=t("img[usemap='#"+o+"']"),a.length>0&&a.is(":visible")):!1):(/^(input|select|textarea|button|object)$/.test(l)?(r=!i.disabled,r&&(h=t(i).closest("fieldset")[0],h&&(r=!h.disabled))):r="a"===l?i.href||s:s,r&&t(i).is(":visible")&&e(t(i)))},t.extend(t.expr[":"],{focusable:function(e){return t.ui.focusable(e,null!=t.attr(e,"tabindex"))}}),t.ui.focusable,t.fn.form=function(){return"string"==typeof this[0].form?this.closest("form"):t(this[0].form)},t.ui.formResetMixin={_formResetHandler:function(){var e=t(this);setTimeout(function(){var i=e.data("ui-form-reset-instances");t.each(i,function(){this.refresh()})})},_bindFormResetHandler:function(){if(this.form=this.element.form(),this.form.length){var t=this.form.data("ui-form-reset-instances")||[];t.length||this.form.on("reset.ui-form-reset",this._formResetHandler),t.push(this),this.form.data("ui-form-reset-instances",t)}},_unbindFormResetHandler:function(){if(this.form.length){var e=this.form.data("ui-form-reset-instances");e.splice(t.inArray(this,e),1),e.length?this.form.data("ui-form-reset-instances",e):this.form.removeData("ui-form-reset-instances").off("reset.ui-form-reset")}}},"1.7"===t.fn.jquery.substring(0,3)&&(t.each(["Width","Height"],function(e,i){function s(e,i,s,o){return t.each(n,function(){i-=parseFloat(t.css(e,"padding"+this))||0,s&&(i-=parseFloat(t.css(e,"border"+this+"Width"))||0),o&&(i-=parseFloat(t.css(e,"margin"+this))||0)}),i}var n="Width"===i?["Left","Right"]:["Top","Bottom"],o=i.toLowerCase(),a={innerWidth:t.fn.innerWidth,innerHeight:t.fn.innerHeight,outerWidth:t.fn.outerWidth,outerHeight:t.fn.outerHeight};t.fn["inner"+i]=function(e){return void 0===e?a["inner"+i].call(this):this.each(function(){t(this).css(o,s(this,e)+"px")})},t.fn["outer"+i]=function(e,n){return"number"!=typeof e?a["outer"+i].call(this,e):this.each(function(){t(this).css(o,s(this,e,!0,n)+"px")})}}),t.fn.addBack=function(t){return this.add(null==t?this.prevObject:this.prevObject.filter(t))}),t.ui.keyCode={BACKSPACE:8,COMMA:188,DELETE:46,DOWN:40,END:35,ENTER:13,ESCAPE:27,HOME:36,LEFT:37,PAGE_DOWN:34,PAGE_UP:33,PERIOD:190,RIGHT:39,SPACE:32,TAB:9,UP:38},t.ui.escapeSelector=function(){var t=/([!"#$%&'()*+,.\/:;<=>?@[\]^`{|}~])/g;return function(e){return e.replace(t,"\\$1")}}(),t.fn.labels=function(){var e,i,s,n,o;return this[0].labels&&this[0].labels.length?this.pushStack(this[0].labels):(n=this.eq(0).parents("label"),s=this.attr("id"),s&&(e=this.eq(0).parents().last(),o=e.add(e.length?e.siblings():this.siblings()),i="label[for='"+t.ui.escapeSelector(s)+"']",n=n.add(o.find(i).addBack(i))),this.pushStack(n))},t.fn.scrollParent=function(e){var i=this.css("position"),s="absolute"===i,n=e?/(auto|scroll|hidden)/:/(auto|scroll)/,o=this.parents().filter(function(){var e=t(this);return s&&"static"===e.css("position")?!1:n.test(e.css("overflow")+e.css("overflow-y")+e.css("overflow-x"))}).eq(0);return"fixed"!==i&&o.length?o:t(this[0].ownerDocument||document)},t.extend(t.expr[":"],{tabbable:function(e){var i=t.attr(e,"tabindex"),s=null!=i;return(!s||i>=0)&&t.ui.focusable(e,s)}}),t.fn.extend({uniqueId:function(){var t=0;return function(){return this.each(function(){this.id||(this.id="ui-id-"+ ++t)})}}(),removeUniqueId:function(){return this.each(function(){/^ui-id-\d+$/.test(this.id)&&t(this).removeAttr("id")})}}),t.widget("ui.accordion",{version:"1.12.1",options:{active:0,animate:{},classes:{"ui-accordion-header":"ui-corner-top","ui-accordion-header-collapsed":"ui-corner-all","ui-accordion-content":"ui-corner-bottom"},collapsible:!1,event:"click",header:"> li > :first-child, > :not(li):even",heightStyle:"auto",icons:{activeHeader:"ui-icon-triangle-1-s",header:"ui-icon-triangle-1-e"},activate:null,beforeActivate:null},hideProps:{borderTopWidth:"hide",borderBottomWidth:"hide",paddingTop:"hide",paddingBottom:"hide",height:"hide"},showProps:{borderTopWidth:"show",borderBottomWidth:"show",paddingTop:"show",paddingBottom:"show",height:"show"},_create:function(){var e=this.options;this.prevShow=this.prevHide=t(),this._addClass("ui-accordion","ui-widget ui-helper-reset"),this.element.attr("role","tablist"),e.collapsible||e.active!==!1&&null!=e.active||(e.active=0),this._processPanels(),0>e.active&&(e.active+=this.headers.length),this._refresh()},_getCreateEventData:function(){return{header:this.active,panel:this.active.length?this.active.next():t()}},_createIcons:function(){var e,i,s=this.options.icons;s&&(e=t(""),this._addClass(e,"ui-accordion-header-icon","ui-icon "+s.header),e.prependTo(this.headers),i=this.active.children(".ui-accordion-header-icon"),this._removeClass(i,s.header)._addClass(i,null,s.activeHeader)._addClass(this.headers,"ui-accordion-icons"))},_destroyIcons:function(){this._removeClass(this.headers,"ui-accordion-icons"),this.headers.children(".ui-accordion-header-icon").remove()},_destroy:function(){var t;this.element.removeAttr("role"),this.headers.removeAttr("role aria-expanded aria-selected aria-controls tabIndex").removeUniqueId(),this._destroyIcons(),t=this.headers.next().css("display","").removeAttr("role aria-hidden aria-labelledby").removeUniqueId(),"content"!==this.options.heightStyle&&t.css("height","")},_setOption:function(t,e){return"active"===t?(this._activate(e),void 0):("event"===t&&(this.options.event&&this._off(this.headers,this.options.event),this._setupEvents(e)),this._super(t,e),"collapsible"!==t||e||this.options.active!==!1||this._activate(0),"icons"===t&&(this._destroyIcons(),e&&this._createIcons()),void 0)},_setOptionDisabled:function(t){this._super(t),this.element.attr("aria-disabled",t),this._toggleClass(null,"ui-state-disabled",!!t),this._toggleClass(this.headers.add(this.headers.next()),null,"ui-state-disabled",!!t)},_keydown:function(e){if(!e.altKey&&!e.ctrlKey){var i=t.ui.keyCode,s=this.headers.length,n=this.headers.index(e.target),o=!1;switch(e.keyCode){case i.RIGHT:case i.DOWN:o=this.headers[(n+1)%s];break;case i.LEFT:case i.UP:o=this.headers[(n-1+s)%s];break;case i.SPACE:case i.ENTER:this._eventHandler(e);break;case i.HOME:o=this.headers[0];break;case i.END:o=this.headers[s-1]}o&&(t(e.target).attr("tabIndex",-1),t(o).attr("tabIndex",0),t(o).trigger("focus"),e.preventDefault())}},_panelKeyDown:function(e){e.keyCode===t.ui.keyCode.UP&&e.ctrlKey&&t(e.currentTarget).prev().trigger("focus")},refresh:function(){var e=this.options;this._processPanels(),e.active===!1&&e.collapsible===!0||!this.headers.length?(e.active=!1,this.active=t()):e.active===!1?this._activate(0):this.active.length&&!t.contains(this.element[0],this.active[0])?this.headers.length===this.headers.find(".ui-state-disabled").length?(e.active=!1,this.active=t()):this._activate(Math.max(0,e.active-1)):e.active=this.headers.index(this.active),this._destroyIcons(),this._refresh()},_processPanels:function(){var t=this.headers,e=this.panels;this.headers=this.element.find(this.options.header),this._addClass(this.headers,"ui-accordion-header ui-accordion-header-collapsed","ui-state-default"),this.panels=this.headers.next().filter(":not(.ui-accordion-content-active)").hide(),this._addClass(this.panels,"ui-accordion-content","ui-helper-reset ui-widget-content"),e&&(this._off(t.not(this.headers)),this._off(e.not(this.panels)))},_refresh:function(){var e,i=this.options,s=i.heightStyle,n=this.element.parent();this.active=this._findActive(i.active),this._addClass(this.active,"ui-accordion-header-active","ui-state-active")._removeClass(this.active,"ui-accordion-header-collapsed"),this._addClass(this.active.next(),"ui-accordion-content-active"),this.active.next().show(),this.headers.attr("role","tab").each(function(){var e=t(this),i=e.uniqueId().attr("id"),s=e.next(),n=s.uniqueId().attr("id");e.attr("aria-controls",n),s.attr("aria-labelledby",i)}).next().attr("role","tabpanel"),this.headers.not(this.active).attr({"aria-selected":"false","aria-expanded":"false",tabIndex:-1}).next().attr({"aria-hidden":"true"}).hide(),this.active.length?this.active.attr({"aria-selected":"true","aria-expanded":"true",tabIndex:0}).next().attr({"aria-hidden":"false"}):this.headers.eq(0).attr("tabIndex",0),this._createIcons(),this._setupEvents(i.event),"fill"===s?(e=n.height(),this.element.siblings(":visible").each(function(){var i=t(this),s=i.css("position");"absolute"!==s&&"fixed"!==s&&(e-=i.outerHeight(!0))}),this.headers.each(function(){e-=t(this).outerHeight(!0)}),this.headers.next().each(function(){t(this).height(Math.max(0,e-t(this).innerHeight()+t(this).height()))}).css("overflow","auto")):"auto"===s&&(e=0,this.headers.next().each(function(){var i=t(this).is(":visible");i||t(this).show(),e=Math.max(e,t(this).css("height","").height()),i||t(this).hide()}).height(e))},_activate:function(e){var i=this._findActive(e)[0];i!==this.active[0]&&(i=i||this.active[0],this._eventHandler({target:i,currentTarget:i,preventDefault:t.noop}))},_findActive:function(e){return"number"==typeof e?this.headers.eq(e):t()},_setupEvents:function(e){var i={keydown:"_keydown"};e&&t.each(e.split(" "),function(t,e){i[e]="_eventHandler"}),this._off(this.headers.add(this.headers.next())),this._on(this.headers,i),this._on(this.headers.next(),{keydown:"_panelKeyDown"}),this._hoverable(this.headers),this._focusable(this.headers)},_eventHandler:function(e){var i,s,n=this.options,o=this.active,a=t(e.currentTarget),r=a[0]===o[0],h=r&&n.collapsible,l=h?t():a.next(),c=o.next(),u={oldHeader:o,oldPanel:c,newHeader:h?t():a,newPanel:l};e.preventDefault(),r&&!n.collapsible||this._trigger("beforeActivate",e,u)===!1||(n.active=h?!1:this.headers.index(a),this.active=r?t():a,this._toggle(u),this._removeClass(o,"ui-accordion-header-active","ui-state-active"),n.icons&&(i=o.children(".ui-accordion-header-icon"),this._removeClass(i,null,n.icons.activeHeader)._addClass(i,null,n.icons.header)),r||(this._removeClass(a,"ui-accordion-header-collapsed")._addClass(a,"ui-accordion-header-active","ui-state-active"),n.icons&&(s=a.children(".ui-accordion-header-icon"),this._removeClass(s,null,n.icons.header)._addClass(s,null,n.icons.activeHeader)),this._addClass(a.next(),"ui-accordion-content-active")))},_toggle:function(e){var i=e.newPanel,s=this.prevShow.length?this.prevShow:e.oldPanel;this.prevShow.add(this.prevHide).stop(!0,!0),this.prevShow=i,this.prevHide=s,this.options.animate?this._animate(i,s,e):(s.hide(),i.show(),this._toggleComplete(e)),s.attr({"aria-hidden":"true"}),s.prev().attr({"aria-selected":"false","aria-expanded":"false"}),i.length&&s.length?s.prev().attr({tabIndex:-1,"aria-expanded":"false"}):i.length&&this.headers.filter(function(){return 0===parseInt(t(this).attr("tabIndex"),10)}).attr("tabIndex",-1),i.attr("aria-hidden","false").prev().attr({"aria-selected":"true","aria-expanded":"true",tabIndex:0})},_animate:function(t,e,i){var s,n,o,a=this,r=0,h=t.css("box-sizing"),l=t.length&&(!e.length||t.index()",delay:300,options:{icons:{submenu:"ui-icon-caret-1-e"},items:"> *",menus:"ul",position:{my:"left top",at:"right top"},role:"menu",blur:null,focus:null,select:null},_create:function(){this.activeMenu=this.element,this.mouseHandled=!1,this.element.uniqueId().attr({role:this.options.role,tabIndex:0}),this._addClass("ui-menu","ui-widget ui-widget-content"),this._on({"mousedown .ui-menu-item":function(t){t.preventDefault()},"click .ui-menu-item":function(e){var i=t(e.target),s=t(t.ui.safeActiveElement(this.document[0]));!this.mouseHandled&&i.not(".ui-state-disabled").length&&(this.select(e),e.isPropagationStopped()||(this.mouseHandled=!0),i.has(".ui-menu").length?this.expand(e):!this.element.is(":focus")&&s.closest(".ui-menu").length&&(this.element.trigger("focus",[!0]),this.active&&1===this.active.parents(".ui-menu").length&&clearTimeout(this.timer)))},"mouseenter .ui-menu-item":function(e){if(!this.previousFilter){var i=t(e.target).closest(".ui-menu-item"),s=t(e.currentTarget);i[0]===s[0]&&(this._removeClass(s.siblings().children(".ui-state-active"),null,"ui-state-active"),this.focus(e,s))}},mouseleave:"collapseAll","mouseleave .ui-menu":"collapseAll",focus:function(t,e){var i=this.active||this.element.find(this.options.items).eq(0);e||this.focus(t,i)},blur:function(e){this._delay(function(){var i=!t.contains(this.element[0],t.ui.safeActiveElement(this.document[0]));i&&this.collapseAll(e)})},keydown:"_keydown"}),this.refresh(),this._on(this.document,{click:function(t){this._closeOnDocumentClick(t)&&this.collapseAll(t),this.mouseHandled=!1}})},_destroy:function(){var e=this.element.find(".ui-menu-item").removeAttr("role aria-disabled"),i=e.children(".ui-menu-item-wrapper").removeUniqueId().removeAttr("tabIndex role aria-haspopup");this.element.removeAttr("aria-activedescendant").find(".ui-menu").addBack().removeAttr("role aria-labelledby aria-expanded aria-hidden aria-disabled tabIndex").removeUniqueId().show(),i.children().each(function(){var e=t(this);e.data("ui-menu-submenu-caret")&&e.remove()})},_keydown:function(e){var i,s,n,o,a=!0;switch(e.keyCode){case t.ui.keyCode.PAGE_UP:this.previousPage(e);break;case t.ui.keyCode.PAGE_DOWN:this.nextPage(e);break;case t.ui.keyCode.HOME:this._move("first","first",e);break;case t.ui.keyCode.END:this._move("last","last",e);break;case t.ui.keyCode.UP:this.previous(e);break;case t.ui.keyCode.DOWN:this.next(e);break;case t.ui.keyCode.LEFT:this.collapse(e);break;case t.ui.keyCode.RIGHT:this.active&&!this.active.is(".ui-state-disabled")&&this.expand(e);break;case t.ui.keyCode.ENTER:case t.ui.keyCode.SPACE:this._activate(e);break;case t.ui.keyCode.ESCAPE:this.collapse(e);break;default:a=!1,s=this.previousFilter||"",o=!1,n=e.keyCode>=96&&105>=e.keyCode?""+(e.keyCode-96):String.fromCharCode(e.keyCode),clearTimeout(this.filterTimer),n===s?o=!0:n=s+n,i=this._filterMenuItems(n),i=o&&-1!==i.index(this.active.next())?this.active.nextAll(".ui-menu-item"):i,i.length||(n=String.fromCharCode(e.keyCode),i=this._filterMenuItems(n)),i.length?(this.focus(e,i),this.previousFilter=n,this.filterTimer=this._delay(function(){delete this.previousFilter},1e3)):delete this.previousFilter}a&&e.preventDefault()},_activate:function(t){this.active&&!this.active.is(".ui-state-disabled")&&(this.active.children("[aria-haspopup='true']").length?this.expand(t):this.select(t))},refresh:function(){var e,i,s,n,o,a=this,r=this.options.icons.submenu,h=this.element.find(this.options.menus);this._toggleClass("ui-menu-icons",null,!!this.element.find(".ui-icon").length),s=h.filter(":not(.ui-menu)").hide().attr({role:this.options.role,"aria-hidden":"true","aria-expanded":"false"}).each(function(){var e=t(this),i=e.prev(),s=t("").data("ui-menu-submenu-caret",!0);a._addClass(s,"ui-menu-icon","ui-icon "+r),i.attr("aria-haspopup","true").prepend(s),e.attr("aria-labelledby",i.attr("id"))}),this._addClass(s,"ui-menu","ui-widget ui-widget-content ui-front"),e=h.add(this.element),i=e.find(this.options.items),i.not(".ui-menu-item").each(function(){var e=t(this);a._isDivider(e)&&a._addClass(e,"ui-menu-divider","ui-widget-content")}),n=i.not(".ui-menu-item, .ui-menu-divider"),o=n.children().not(".ui-menu").uniqueId().attr({tabIndex:-1,role:this._itemRole()}),this._addClass(n,"ui-menu-item")._addClass(o,"ui-menu-item-wrapper"),i.filter(".ui-state-disabled").attr("aria-disabled","true"),this.active&&!t.contains(this.element[0],this.active[0])&&this.blur()},_itemRole:function(){return{menu:"menuitem",listbox:"option"}[this.options.role]},_setOption:function(t,e){if("icons"===t){var i=this.element.find(".ui-menu-icon");this._removeClass(i,null,this.options.icons.submenu)._addClass(i,null,e.submenu)}this._super(t,e)},_setOptionDisabled:function(t){this._super(t),this.element.attr("aria-disabled",t+""),this._toggleClass(null,"ui-state-disabled",!!t)},focus:function(t,e){var i,s,n;this.blur(t,t&&"focus"===t.type),this._scrollIntoView(e),this.active=e.first(),s=this.active.children(".ui-menu-item-wrapper"),this._addClass(s,null,"ui-state-active"),this.options.role&&this.element.attr("aria-activedescendant",s.attr("id")),n=this.active.parent().closest(".ui-menu-item").children(".ui-menu-item-wrapper"),this._addClass(n,null,"ui-state-active"),t&&"keydown"===t.type?this._close():this.timer=this._delay(function(){this._close()},this.delay),i=e.children(".ui-menu"),i.length&&t&&/^mouse/.test(t.type)&&this._startOpening(i),this.activeMenu=e.parent(),this._trigger("focus",t,{item:e})},_scrollIntoView:function(e){var i,s,n,o,a,r;this._hasScroll()&&(i=parseFloat(t.css(this.activeMenu[0],"borderTopWidth"))||0,s=parseFloat(t.css(this.activeMenu[0],"paddingTop"))||0,n=e.offset().top-this.activeMenu.offset().top-i-s,o=this.activeMenu.scrollTop(),a=this.activeMenu.height(),r=e.outerHeight(),0>n?this.activeMenu.scrollTop(o+n):n+r>a&&this.activeMenu.scrollTop(o+n-a+r))},blur:function(t,e){e||clearTimeout(this.timer),this.active&&(this._removeClass(this.active.children(".ui-menu-item-wrapper"),null,"ui-state-active"),this._trigger("blur",t,{item:this.active}),this.active=null)},_startOpening:function(t){clearTimeout(this.timer),"true"===t.attr("aria-hidden")&&(this.timer=this._delay(function(){this._close(),this._open(t)},this.delay))},_open:function(e){var i=t.extend({of:this.active},this.options.position);clearTimeout(this.timer),this.element.find(".ui-menu").not(e.parents(".ui-menu")).hide().attr("aria-hidden","true"),e.show().removeAttr("aria-hidden").attr("aria-expanded","true").position(i)},collapseAll:function(e,i){clearTimeout(this.timer),this.timer=this._delay(function(){var s=i?this.element:t(e&&e.target).closest(this.element.find(".ui-menu"));s.length||(s=this.element),this._close(s),this.blur(e),this._removeClass(s.find(".ui-state-active"),null,"ui-state-active"),this.activeMenu=s},this.delay)},_close:function(t){t||(t=this.active?this.active.parent():this.element),t.find(".ui-menu").hide().attr("aria-hidden","true").attr("aria-expanded","false")},_closeOnDocumentClick:function(e){return!t(e.target).closest(".ui-menu").length},_isDivider:function(t){return!/[^\-\u2014\u2013\s]/.test(t.text())},collapse:function(t){var e=this.active&&this.active.parent().closest(".ui-menu-item",this.element);e&&e.length&&(this._close(),this.focus(t,e))},expand:function(t){var e=this.active&&this.active.children(".ui-menu ").find(this.options.items).first();e&&e.length&&(this._open(e.parent()),this._delay(function(){this.focus(t,e)}))},next:function(t){this._move("next","first",t)},previous:function(t){this._move("prev","last",t)},isFirstItem:function(){return this.active&&!this.active.prevAll(".ui-menu-item").length},isLastItem:function(){return this.active&&!this.active.nextAll(".ui-menu-item").length},_move:function(t,e,i){var s;this.active&&(s="first"===t||"last"===t?this.active["first"===t?"prevAll":"nextAll"](".ui-menu-item").eq(-1):this.active[t+"All"](".ui-menu-item").eq(0)),s&&s.length&&this.active||(s=this.activeMenu.find(this.options.items)[e]()),this.focus(i,s)},nextPage:function(e){var i,s,n;return this.active?(this.isLastItem()||(this._hasScroll()?(s=this.active.offset().top,n=this.element.height(),this.active.nextAll(".ui-menu-item").each(function(){return i=t(this),0>i.offset().top-s-n}),this.focus(e,i)):this.focus(e,this.activeMenu.find(this.options.items)[this.active?"last":"first"]())),void 0):(this.next(e),void 0)},previousPage:function(e){var i,s,n;return this.active?(this.isFirstItem()||(this._hasScroll()?(s=this.active.offset().top,n=this.element.height(),this.active.prevAll(".ui-menu-item").each(function(){return i=t(this),i.offset().top-s+n>0}),this.focus(e,i)):this.focus(e,this.activeMenu.find(this.options.items).first())),void 0):(this.next(e),void 0)},_hasScroll:function(){return this.element.outerHeight()",options:{appendTo:null,autoFocus:!1,delay:300,minLength:1,position:{my:"left top",at:"left bottom",collision:"none"},source:null,change:null,close:null,focus:null,open:null,response:null,search:null,select:null},requestIndex:0,pending:0,_create:function(){var e,i,s,n=this.element[0].nodeName.toLowerCase(),o="textarea"===n,a="input"===n; -this.isMultiLine=o||!a&&this._isContentEditable(this.element),this.valueMethod=this.element[o||a?"val":"text"],this.isNewMenu=!0,this._addClass("ui-autocomplete-input"),this.element.attr("autocomplete","off"),this._on(this.element,{keydown:function(n){if(this.element.prop("readOnly"))return e=!0,s=!0,i=!0,void 0;e=!1,s=!1,i=!1;var o=t.ui.keyCode;switch(n.keyCode){case o.PAGE_UP:e=!0,this._move("previousPage",n);break;case o.PAGE_DOWN:e=!0,this._move("nextPage",n);break;case o.UP:e=!0,this._keyEvent("previous",n);break;case o.DOWN:e=!0,this._keyEvent("next",n);break;case o.ENTER:this.menu.active&&(e=!0,n.preventDefault(),this.menu.select(n));break;case o.TAB:this.menu.active&&this.menu.select(n);break;case o.ESCAPE:this.menu.element.is(":visible")&&(this.isMultiLine||this._value(this.term),this.close(n),n.preventDefault());break;default:i=!0,this._searchTimeout(n)}},keypress:function(s){if(e)return e=!1,(!this.isMultiLine||this.menu.element.is(":visible"))&&s.preventDefault(),void 0;if(!i){var n=t.ui.keyCode;switch(s.keyCode){case n.PAGE_UP:this._move("previousPage",s);break;case n.PAGE_DOWN:this._move("nextPage",s);break;case n.UP:this._keyEvent("previous",s);break;case n.DOWN:this._keyEvent("next",s)}}},input:function(t){return s?(s=!1,t.preventDefault(),void 0):(this._searchTimeout(t),void 0)},focus:function(){this.selectedItem=null,this.previous=this._value()},blur:function(t){return this.cancelBlur?(delete this.cancelBlur,void 0):(clearTimeout(this.searching),this.close(t),this._change(t),void 0)}}),this._initSource(),this.menu=t("
     
     
    ",x=o?"":"",g=0;g<7;g++)x+="";for(w+=x+"",D=this._getDaysInMonth(U,K),U===t.selectedYear&&K===t.selectedMonth&&(t.selectedDay=Math.min(t.selectedDay,D)),C=(this._getFirstDayOfMonth(U,K)-n+7)%7,D=Math.ceil((C+D)/7),I=Y&&this.maxRows>D?this.maxRows:D,this.maxRows=I,T=this._daylightSavingAdjust(new Date(U,K,1-C)),P=0;P",M=o?"":"",g=0;g<7;g++)S=c?c.apply(t.input?t.input[0]:null,[T]):[!0,""],z=(H=T.getMonth()!==K)&&!d||!S[0]||j&&T"+(H&&!u?" ":z?""+T.getDate()+"":""+T.getDate()+"")+"",T.setDate(T.getDate()+1),T=this._daylightSavingAdjust(T);w+=M+""}11<++K&&(K=0,U++),_+=w+="
    "+this._get(t,"weekHeader")+""+r[k]+"
    "+this._get(t,"calculateWeek")(T)+"
    "+(Y?""+(0":""):"")}f+=_}return f+=F,t._keyEvent=!1,f},_generateMonthYearHeader:function(t,e,i,s,n,o,a,r){var l,h,c,u,d,p,f=this._get(t,"changeMonth"),g=this._get(t,"changeYear"),m=this._get(t,"showMonthAfterYear"),_=this._get(t,"selectMonthLabel"),v=this._get(t,"selectYearLabel"),b="
    ",y="";if(o||!f)y+=""+a[e]+"";else{for(l=s&&s.getFullYear()===i,h=n&&n.getFullYear()===i,y+=""}if(m||(b+=y+(!o&&f&&g?"":" ")),!t.yearshtml)if(t.yearshtml="",o||!g)b+=""+i+"";else{for(a=this._get(t,"yearRange").split(":"),u=(new Date).getFullYear(),d=(_=function(t){t=t.match(/c[+\-].*/)?i+parseInt(t.substring(1),10):t.match(/[+\-].*/)?u+parseInt(t,10):parseInt(t,10);return isNaN(t)?u:t})(a[0]),p=Math.max(d,_(a[1]||"")),d=s?Math.max(d,s.getFullYear()):d,p=n?Math.min(p,n.getFullYear()):p,t.yearshtml+="",b+=t.yearshtml,t.yearshtml=null}return b+=this._get(t,"yearSuffix"),m&&(b+=(!o&&f&&g?"":" ")+y),b+="
    "},_adjustInstDate:function(t,e,i){var s=t.selectedYear+("Y"===i?e:0),n=t.selectedMonth+("M"===i?e:0),e=Math.min(t.selectedDay,this._getDaysInMonth(s,n))+("D"===i?e:0),e=this._restrictMinMax(t,this._daylightSavingAdjust(new Date(s,n,e)));t.selectedDay=e.getDate(),t.drawMonth=t.selectedMonth=e.getMonth(),t.drawYear=t.selectedYear=e.getFullYear(),"M"!==i&&"Y"!==i||this._notifyChange(t)},_restrictMinMax:function(t,e){var i=this._getMinMaxDate(t,"min"),t=this._getMinMaxDate(t,"max"),e=i&&e=i.getTime())&&(!s||e.getTime()<=s.getTime())&&(!n||e.getFullYear()>=n)&&(!o||e.getFullYear()<=o)},_getFormatConfig:function(t){var e=this._get(t,"shortYearCutoff");return{shortYearCutoff:e="string"!=typeof e?e:(new Date).getFullYear()%100+parseInt(e,10),dayNamesShort:this._get(t,"dayNamesShort"),dayNames:this._get(t,"dayNames"),monthNamesShort:this._get(t,"monthNamesShort"),monthNames:this._get(t,"monthNames")}},_formatDate:function(t,e,i,s){e||(t.currentDay=t.selectedDay,t.currentMonth=t.selectedMonth,t.currentYear=t.selectedYear);e=e?"object"==typeof e?e:this._daylightSavingAdjust(new Date(s,i,e)):this._daylightSavingAdjust(new Date(t.currentYear,t.currentMonth,t.currentDay));return this.formatDate(this._get(t,"dateFormat"),e,this._getFormatConfig(t))}}),V.fn.datepicker=function(t){if(!this.length)return this;V.datepicker.initialized||(V(document).on("mousedown",V.datepicker._checkExternalClick),V.datepicker.initialized=!0),0===V("#"+V.datepicker._mainDivId).length&&V("body").append(V.datepicker.dpDiv);var e=Array.prototype.slice.call(arguments,1);return"string"==typeof t&&("isDisabled"===t||"getDate"===t||"widget"===t)||"option"===t&&2===arguments.length&&"string"==typeof arguments[1]?V.datepicker["_"+t+"Datepicker"].apply(V.datepicker,[this[0]].concat(e)):this.each(function(){"string"==typeof t?V.datepicker["_"+t+"Datepicker"].apply(V.datepicker,[this].concat(e)):V.datepicker._attachDatepicker(this,t)})},V.datepicker=new b,V.datepicker.initialized=!1,V.datepicker.uuid=(new Date).getTime(),V.datepicker.version="1.13.1";V.datepicker;V.widget("ui.dialog",{version:"1.13.1",options:{appendTo:"body",autoOpen:!0,buttons:[],classes:{"ui-dialog":"ui-corner-all","ui-dialog-titlebar":"ui-corner-all"},closeOnEscape:!0,closeText:"Close",draggable:!0,hide:null,height:"auto",maxHeight:null,maxWidth:null,minHeight:150,minWidth:150,modal:!1,position:{my:"center",at:"center",of:window,collision:"fit",using:function(t){var e=V(this).css(t).offset().top;e<0&&V(this).css("top",t.top-e)}},resizable:!0,show:null,title:null,width:300,beforeClose:null,close:null,drag:null,dragStart:null,dragStop:null,focus:null,open:null,resize:null,resizeStart:null,resizeStop:null},sizeRelatedOptions:{buttons:!0,height:!0,maxHeight:!0,maxWidth:!0,minHeight:!0,minWidth:!0,width:!0},resizableRelatedOptions:{maxHeight:!0,maxWidth:!0,minHeight:!0,minWidth:!0},_create:function(){this.originalCss={display:this.element[0].style.display,width:this.element[0].style.width,minHeight:this.element[0].style.minHeight,maxHeight:this.element[0].style.maxHeight,height:this.element[0].style.height},this.originalPosition={parent:this.element.parent(),index:this.element.parent().children().index(this.element)},this.originalTitle=this.element.attr("title"),null==this.options.title&&null!=this.originalTitle&&(this.options.title=this.originalTitle),this.options.disabled&&(this.options.disabled=!1),this._createWrapper(),this.element.show().removeAttr("title").appendTo(this.uiDialog),this._addClass("ui-dialog-content","ui-widget-content"),this._createTitlebar(),this._createButtonPane(),this.options.draggable&&V.fn.draggable&&this._makeDraggable(),this.options.resizable&&V.fn.resizable&&this._makeResizable(),this._isOpen=!1,this._trackFocus()},_init:function(){this.options.autoOpen&&this.open()},_appendTo:function(){var t=this.options.appendTo;return t&&(t.jquery||t.nodeType)?V(t):this.document.find(t||"body").eq(0)},_destroy:function(){var t,e=this.originalPosition;this._untrackInstance(),this._destroyOverlay(),this.element.removeUniqueId().css(this.originalCss).detach(),this.uiDialog.remove(),this.originalTitle&&this.element.attr("title",this.originalTitle),(t=e.parent.children().eq(e.index)).length&&t[0]!==this.element[0]?t.before(this.element):e.parent.append(this.element)},widget:function(){return this.uiDialog},disable:V.noop,enable:V.noop,close:function(t){var e=this;this._isOpen&&!1!==this._trigger("beforeClose",t)&&(this._isOpen=!1,this._focusedElement=null,this._destroyOverlay(),this._untrackInstance(),this.opener.filter(":focusable").trigger("focus").length||V.ui.safeBlur(V.ui.safeActiveElement(this.document[0])),this._hide(this.uiDialog,this.options.hide,function(){e._trigger("close",t)}))},isOpen:function(){return this._isOpen},moveToTop:function(){this._moveToTop()},_moveToTop:function(t,e){var i=!1,s=this.uiDialog.siblings(".ui-front:visible").map(function(){return+V(this).css("z-index")}).get(),s=Math.max.apply(null,s);return s>=+this.uiDialog.css("z-index")&&(this.uiDialog.css("z-index",s+1),i=!0),i&&!e&&this._trigger("focus",t),i},open:function(){var t=this;this._isOpen?this._moveToTop()&&this._focusTabbable():(this._isOpen=!0,this.opener=V(V.ui.safeActiveElement(this.document[0])),this._size(),this._position(),this._createOverlay(),this._moveToTop(null,!0),this.overlay&&this.overlay.css("z-index",this.uiDialog.css("z-index")-1),this._show(this.uiDialog,this.options.show,function(){t._focusTabbable(),t._trigger("focus")}),this._makeFocusTarget(),this._trigger("open"))},_focusTabbable:function(){var t=this._focusedElement;(t=!(t=!(t=!(t=!(t=t||this.element.find("[autofocus]")).length?this.element.find(":tabbable"):t).length?this.uiDialogButtonPane.find(":tabbable"):t).length?this.uiDialogTitlebarClose.filter(":tabbable"):t).length?this.uiDialog:t).eq(0).trigger("focus")},_restoreTabbableFocus:function(){var t=V.ui.safeActiveElement(this.document[0]);this.uiDialog[0]===t||V.contains(this.uiDialog[0],t)||this._focusTabbable()},_keepFocus:function(t){t.preventDefault(),this._restoreTabbableFocus(),this._delay(this._restoreTabbableFocus)},_createWrapper:function(){this.uiDialog=V("
    ").hide().attr({tabIndex:-1,role:"dialog"}).appendTo(this._appendTo()),this._addClass(this.uiDialog,"ui-dialog","ui-widget ui-widget-content ui-front"),this._on(this.uiDialog,{keydown:function(t){if(this.options.closeOnEscape&&!t.isDefaultPrevented()&&t.keyCode&&t.keyCode===V.ui.keyCode.ESCAPE)return t.preventDefault(),void this.close(t);var e,i,s;t.keyCode!==V.ui.keyCode.TAB||t.isDefaultPrevented()||(e=this.uiDialog.find(":tabbable"),i=e.first(),s=e.last(),t.target!==s[0]&&t.target!==this.uiDialog[0]||t.shiftKey?t.target!==i[0]&&t.target!==this.uiDialog[0]||!t.shiftKey||(this._delay(function(){s.trigger("focus")}),t.preventDefault()):(this._delay(function(){i.trigger("focus")}),t.preventDefault()))},mousedown:function(t){this._moveToTop(t)&&this._focusTabbable()}}),this.element.find("[aria-describedby]").length||this.uiDialog.attr({"aria-describedby":this.element.uniqueId().attr("id")})},_createTitlebar:function(){var t;this.uiDialogTitlebar=V("
    "),this._addClass(this.uiDialogTitlebar,"ui-dialog-titlebar","ui-widget-header ui-helper-clearfix"),this._on(this.uiDialogTitlebar,{mousedown:function(t){V(t.target).closest(".ui-dialog-titlebar-close")||this.uiDialog.trigger("focus")}}),this.uiDialogTitlebarClose=V("").button({label:V("").text(this.options.closeText).html(),icon:"ui-icon-closethick",showLabel:!1}).appendTo(this.uiDialogTitlebar),this._addClass(this.uiDialogTitlebarClose,"ui-dialog-titlebar-close"),this._on(this.uiDialogTitlebarClose,{click:function(t){t.preventDefault(),this.close(t)}}),t=V("").uniqueId().prependTo(this.uiDialogTitlebar),this._addClass(t,"ui-dialog-title"),this._title(t),this.uiDialogTitlebar.prependTo(this.uiDialog),this.uiDialog.attr({"aria-labelledby":t.attr("id")})},_title:function(t){this.options.title?t.text(this.options.title):t.html(" ")},_createButtonPane:function(){this.uiDialogButtonPane=V("
    "),this._addClass(this.uiDialogButtonPane,"ui-dialog-buttonpane","ui-widget-content ui-helper-clearfix"),this.uiButtonSet=V("
    ").appendTo(this.uiDialogButtonPane),this._addClass(this.uiButtonSet,"ui-dialog-buttonset"),this._createButtons()},_createButtons:function(){var s=this,t=this.options.buttons;this.uiDialogButtonPane.remove(),this.uiButtonSet.empty(),V.isEmptyObject(t)||Array.isArray(t)&&!t.length?this._removeClass(this.uiDialog,"ui-dialog-buttons"):(V.each(t,function(t,e){var i;e=V.extend({type:"button"},e="function"==typeof e?{click:e,text:t}:e),i=e.click,t={icon:e.icon,iconPosition:e.iconPosition,showLabel:e.showLabel,icons:e.icons,text:e.text},delete e.click,delete e.icon,delete e.iconPosition,delete e.showLabel,delete e.icons,"boolean"==typeof e.text&&delete e.text,V("",e).button(t).appendTo(s.uiButtonSet).on("click",function(){i.apply(s.element[0],arguments)})}),this._addClass(this.uiDialog,"ui-dialog-buttons"),this.uiDialogButtonPane.appendTo(this.uiDialog))},_makeDraggable:function(){var n=this,o=this.options;function a(t){return{position:t.position,offset:t.offset}}this.uiDialog.draggable({cancel:".ui-dialog-content, .ui-dialog-titlebar-close",handle:".ui-dialog-titlebar",containment:"document",start:function(t,e){n._addClass(V(this),"ui-dialog-dragging"),n._blockFrames(),n._trigger("dragStart",t,a(e))},drag:function(t,e){n._trigger("drag",t,a(e))},stop:function(t,e){var i=e.offset.left-n.document.scrollLeft(),s=e.offset.top-n.document.scrollTop();o.position={my:"left top",at:"left"+(0<=i?"+":"")+i+" top"+(0<=s?"+":"")+s,of:n.window},n._removeClass(V(this),"ui-dialog-dragging"),n._unblockFrames(),n._trigger("dragStop",t,a(e))}})},_makeResizable:function(){var n=this,o=this.options,t=o.resizable,e=this.uiDialog.css("position"),t="string"==typeof t?t:"n,e,s,w,se,sw,ne,nw";function a(t){return{originalPosition:t.originalPosition,originalSize:t.originalSize,position:t.position,size:t.size}}this.uiDialog.resizable({cancel:".ui-dialog-content",containment:"document",alsoResize:this.element,maxWidth:o.maxWidth,maxHeight:o.maxHeight,minWidth:o.minWidth,minHeight:this._minHeight(),handles:t,start:function(t,e){n._addClass(V(this),"ui-dialog-resizing"),n._blockFrames(),n._trigger("resizeStart",t,a(e))},resize:function(t,e){n._trigger("resize",t,a(e))},stop:function(t,e){var i=n.uiDialog.offset(),s=i.left-n.document.scrollLeft(),i=i.top-n.document.scrollTop();o.height=n.uiDialog.height(),o.width=n.uiDialog.width(),o.position={my:"left top",at:"left"+(0<=s?"+":"")+s+" top"+(0<=i?"+":"")+i,of:n.window},n._removeClass(V(this),"ui-dialog-resizing"),n._unblockFrames(),n._trigger("resizeStop",t,a(e))}}).css("position",e)},_trackFocus:function(){this._on(this.widget(),{focusin:function(t){this._makeFocusTarget(),this._focusedElement=V(t.target)}})},_makeFocusTarget:function(){this._untrackInstance(),this._trackingInstances().unshift(this)},_untrackInstance:function(){var t=this._trackingInstances(),e=V.inArray(this,t);-1!==e&&t.splice(e,1)},_trackingInstances:function(){var t=this.document.data("ui-dialog-instances");return t||this.document.data("ui-dialog-instances",t=[]),t},_minHeight:function(){var t=this.options;return"auto"===t.height?t.minHeight:Math.min(t.minHeight,t.height)},_position:function(){var t=this.uiDialog.is(":visible");t||this.uiDialog.show(),this.uiDialog.position(this.options.position),t||this.uiDialog.hide()},_setOptions:function(t){var i=this,s=!1,n={};V.each(t,function(t,e){i._setOption(t,e),t in i.sizeRelatedOptions&&(s=!0),t in i.resizableRelatedOptions&&(n[t]=e)}),s&&(this._size(),this._position()),this.uiDialog.is(":data(ui-resizable)")&&this.uiDialog.resizable("option",n)},_setOption:function(t,e){var i,s=this.uiDialog;"disabled"!==t&&(this._super(t,e),"appendTo"===t&&this.uiDialog.appendTo(this._appendTo()),"buttons"===t&&this._createButtons(),"closeText"===t&&this.uiDialogTitlebarClose.button({label:V("").text(""+this.options.closeText).html()}),"draggable"===t&&((i=s.is(":data(ui-draggable)"))&&!e&&s.draggable("destroy"),!i&&e&&this._makeDraggable()),"position"===t&&this._position(),"resizable"===t&&((i=s.is(":data(ui-resizable)"))&&!e&&s.resizable("destroy"),i&&"string"==typeof e&&s.resizable("option","handles",e),i||!1===e||this._makeResizable()),"title"===t&&this._title(this.uiDialogTitlebar.find(".ui-dialog-title")))},_size:function(){var t,e,i,s=this.options;this.element.show().css({width:"auto",minHeight:0,maxHeight:"none",height:0}),s.minWidth>s.width&&(s.width=s.minWidth),t=this.uiDialog.css({height:"auto",width:s.width}).outerHeight(),e=Math.max(0,s.minHeight-t),i="number"==typeof s.maxHeight?Math.max(0,s.maxHeight-t):"none","auto"===s.height?this.element.css({minHeight:e,maxHeight:i,height:"auto"}):this.element.height(Math.max(0,s.height-t)),this.uiDialog.is(":data(ui-resizable)")&&this.uiDialog.resizable("option","minHeight",this._minHeight())},_blockFrames:function(){this.iframeBlocks=this.document.find("iframe").map(function(){var t=V(this);return V("
    ").css({position:"absolute",width:t.outerWidth(),height:t.outerHeight()}).appendTo(t.parent()).offset(t.offset())[0]})},_unblockFrames:function(){this.iframeBlocks&&(this.iframeBlocks.remove(),delete this.iframeBlocks)},_allowInteraction:function(t){return!!V(t.target).closest(".ui-dialog").length||!!V(t.target).closest(".ui-datepicker").length},_createOverlay:function(){var i,s;this.options.modal&&(i=V.fn.jquery.substring(0,4),s=!0,this._delay(function(){s=!1}),this.document.data("ui-dialog-overlays")||this.document.on("focusin.ui-dialog",function(t){var e;s||((e=this._trackingInstances()[0])._allowInteraction(t)||(t.preventDefault(),e._focusTabbable(),"3.4."!==i&&"3.5."!==i||e._delay(e._restoreTabbableFocus)))}.bind(this)),this.overlay=V("
    ").appendTo(this._appendTo()),this._addClass(this.overlay,null,"ui-widget-overlay ui-front"),this._on(this.overlay,{mousedown:"_keepFocus"}),this.document.data("ui-dialog-overlays",(this.document.data("ui-dialog-overlays")||0)+1))},_destroyOverlay:function(){var t;this.options.modal&&this.overlay&&((t=this.document.data("ui-dialog-overlays")-1)?this.document.data("ui-dialog-overlays",t):(this.document.off("focusin.ui-dialog"),this.document.removeData("ui-dialog-overlays")),this.overlay.remove(),this.overlay=null)}}),!1!==V.uiBackCompat&&V.widget("ui.dialog",V.ui.dialog,{options:{dialogClass:""},_createWrapper:function(){this._super(),this.uiDialog.addClass(this.options.dialogClass)},_setOption:function(t,e){"dialogClass"===t&&this.uiDialog.removeClass(this.options.dialogClass).addClass(e),this._superApply(arguments)}});V.ui.dialog,V.widget("ui.progressbar",{version:"1.13.1",options:{classes:{"ui-progressbar":"ui-corner-all","ui-progressbar-value":"ui-corner-left","ui-progressbar-complete":"ui-corner-right"},max:100,value:0,change:null,complete:null},min:0,_create:function(){this.oldValue=this.options.value=this._constrainedValue(),this.element.attr({role:"progressbar","aria-valuemin":this.min}),this._addClass("ui-progressbar","ui-widget ui-widget-content"),this.valueDiv=V("
    ").appendTo(this.element),this._addClass(this.valueDiv,"ui-progressbar-value","ui-widget-header"),this._refreshValue()},_destroy:function(){this.element.removeAttr("role aria-valuemin aria-valuemax aria-valuenow"),this.valueDiv.remove()},value:function(t){if(void 0===t)return this.options.value;this.options.value=this._constrainedValue(t),this._refreshValue()},_constrainedValue:function(t){return void 0===t&&(t=this.options.value),this.indeterminate=!1===t,"number"!=typeof t&&(t=0),!this.indeterminate&&Math.min(this.options.max,Math.max(this.min,t))},_setOptions:function(t){var e=t.value;delete t.value,this._super(t),this.options.value=this._constrainedValue(e),this._refreshValue()},_setOption:function(t,e){"max"===t&&(e=Math.max(this.min,e)),this._super(t,e)},_setOptionDisabled:function(t){this._super(t),this.element.attr("aria-disabled",t),this._toggleClass(null,"ui-state-disabled",!!t)},_percentage:function(){return this.indeterminate?100:100*(this.options.value-this.min)/(this.options.max-this.min)},_refreshValue:function(){var t=this.options.value,e=this._percentage();this.valueDiv.toggle(this.indeterminate||t>this.min).width(e.toFixed(0)+"%"),this._toggleClass(this.valueDiv,"ui-progressbar-complete",null,t===this.options.max)._toggleClass("ui-progressbar-indeterminate",null,this.indeterminate),this.indeterminate?(this.element.removeAttr("aria-valuenow"),this.overlayDiv||(this.overlayDiv=V("
    ").appendTo(this.valueDiv),this._addClass(this.overlayDiv,"ui-progressbar-overlay"))):(this.element.attr({"aria-valuemax":this.options.max,"aria-valuenow":t}),this.overlayDiv&&(this.overlayDiv.remove(),this.overlayDiv=null)),this.oldValue!==t&&(this.oldValue=t,this._trigger("change")),t===this.options.max&&this._trigger("complete")}}),V.widget("ui.selectmenu",[V.ui.formResetMixin,{version:"1.13.1",defaultElement:"",widgetEventPrefix:"spin",options:{classes:{"ui-spinner":"ui-corner-all","ui-spinner-down":"ui-corner-br","ui-spinner-up":"ui-corner-tr"},culture:null,icons:{down:"ui-icon-triangle-1-s",up:"ui-icon-triangle-1-n"},incremental:!0,max:null,min:null,numberFormat:null,page:10,step:1,change:null,spin:null,start:null,stop:null},_create:function(){this._setOption("max",this.options.max),this._setOption("min",this.options.min),this._setOption("step",this.options.step),""!==this.value()&&this._value(this.element.val(),!0),this._draw(),this._on(this._events),this._refresh(),this._on(this.window,{beforeunload:function(){this.element.removeAttr("autocomplete")}})},_getCreateOptions:function(){var s=this._super(),n=this.element;return V.each(["min","max","step"],function(t,e){var i=n.attr(e);null!=i&&i.length&&(s[e]=i)}),s},_events:{keydown:function(t){this._start(t)&&this._keydown(t)&&t.preventDefault()},keyup:"_stop",focus:function(){this.previous=this.element.val()},blur:function(t){this.cancelBlur?delete this.cancelBlur:(this._stop(),this._refresh(),this.previous!==this.element.val()&&this._trigger("change",t))},mousewheel:function(t,e){var i=V.ui.safeActiveElement(this.document[0]);if(this.element[0]===i&&e){if(!this.spinning&&!this._start(t))return!1;this._spin((0").parent().append("")},_draw:function(){this._enhance(),this._addClass(this.uiSpinner,"ui-spinner","ui-widget ui-widget-content"),this._addClass("ui-spinner-input"),this.element.attr("role","spinbutton"),this.buttons=this.uiSpinner.children("a").attr("tabIndex",-1).attr("aria-hidden",!0).button({classes:{"ui-button":""}}),this._removeClass(this.buttons,"ui-corner-all"),this._addClass(this.buttons.first(),"ui-spinner-button ui-spinner-up"),this._addClass(this.buttons.last(),"ui-spinner-button ui-spinner-down"),this.buttons.first().button({icon:this.options.icons.up,showLabel:!1}),this.buttons.last().button({icon:this.options.icons.down,showLabel:!1}),this.buttons.height()>Math.ceil(.5*this.uiSpinner.height())&&0e.max?e.max:null!==e.min&&t"},_buttonHtml:function(){return""}});var S;V.ui.spinner;V.widget("ui.tabs",{version:"1.13.1",delay:300,options:{active:null,classes:{"ui-tabs":"ui-corner-all","ui-tabs-nav":"ui-corner-all","ui-tabs-panel":"ui-corner-bottom","ui-tabs-tab":"ui-corner-top"},collapsible:!1,event:"click",heightStyle:"content",hide:null,show:null,activate:null,beforeActivate:null,beforeLoad:null,load:null},_isLocal:(S=/#.*$/,function(t){var e=t.href.replace(S,""),i=location.href.replace(S,"");try{e=decodeURIComponent(e)}catch(t){}try{i=decodeURIComponent(i)}catch(t){}return 1?@\[\]\^`{|}~]/g,"\\$&"):""},refresh:function(){var t=this.options,e=this.tablist.children(":has(a[href])");t.disabled=V.map(e.filter(".ui-state-disabled"),function(t){return e.index(t)}),this._processTabs(),!1!==t.active&&this.anchors.length?this.active.length&&!V.contains(this.tablist[0],this.active[0])?this.tabs.length===t.disabled.length?(t.active=!1,this.active=V()):this._activate(this._findNextTab(Math.max(0,t.active-1),!1)):t.active=this.tabs.index(this.active):(t.active=!1,this.active=V()),this._refresh()},_refresh:function(){this._setOptionDisabled(this.options.disabled),this._setupEvents(this.options.event),this._setupHeightStyle(this.options.heightStyle),this.tabs.not(this.active).attr({"aria-selected":"false","aria-expanded":"false",tabIndex:-1}),this.panels.not(this._getPanelForTab(this.active)).hide().attr({"aria-hidden":"true"}),this.active.length?(this.active.attr({"aria-selected":"true","aria-expanded":"true",tabIndex:0}),this._addClass(this.active,"ui-tabs-active","ui-state-active"),this._getPanelForTab(this.active).show().attr({"aria-hidden":"false"})):this.tabs.eq(0).attr("tabIndex",0)},_processTabs:function(){var l=this,t=this.tabs,e=this.anchors,i=this.panels;this.tablist=this._getList().attr("role","tablist"),this._addClass(this.tablist,"ui-tabs-nav","ui-helper-reset ui-helper-clearfix ui-widget-header"),this.tablist.on("mousedown"+this.eventNamespace,"> li",function(t){V(this).is(".ui-state-disabled")&&t.preventDefault()}).on("focus"+this.eventNamespace,".ui-tabs-anchor",function(){V(this).closest("li").is(".ui-state-disabled")&&this.blur()}),this.tabs=this.tablist.find("> li:has(a[href])").attr({role:"tab",tabIndex:-1}),this._addClass(this.tabs,"ui-tabs-tab","ui-state-default"),this.anchors=this.tabs.map(function(){return V("a",this)[0]}).attr({tabIndex:-1}),this._addClass(this.anchors,"ui-tabs-anchor"),this.panels=V(),this.anchors.each(function(t,e){var i,s,n,o=V(e).uniqueId().attr("id"),a=V(e).closest("li"),r=a.attr("aria-controls");l._isLocal(e)?(n=(i=e.hash).substring(1),s=l.element.find(l._sanitizeSelector(i))):(n=a.attr("aria-controls")||V({}).uniqueId()[0].id,(s=l.element.find(i="#"+n)).length||(s=l._createPanel(n)).insertAfter(l.panels[t-1]||l.tablist),s.attr("aria-live","polite")),s.length&&(l.panels=l.panels.add(s)),r&&a.data("ui-tabs-aria-controls",r),a.attr({"aria-controls":n,"aria-labelledby":o}),s.attr("aria-labelledby",o)}),this.panels.attr("role","tabpanel"),this._addClass(this.panels,"ui-tabs-panel","ui-widget-content"),t&&(this._off(t.not(this.tabs)),this._off(e.not(this.anchors)),this._off(i.not(this.panels)))},_getList:function(){return this.tablist||this.element.find("ol, ul").eq(0)},_createPanel:function(t){return V("
    ").attr("id",t).data("ui-tabs-destroy",!0)},_setOptionDisabled:function(t){var e,i;for(Array.isArray(t)&&(t.length?t.length===this.anchors.length&&(t=!0):t=!1),i=0;e=this.tabs[i];i++)e=V(e),!0===t||-1!==V.inArray(i,t)?(e.attr("aria-disabled","true"),this._addClass(e,null,"ui-state-disabled")):(e.removeAttr("aria-disabled"),this._removeClass(e,null,"ui-state-disabled"));this.options.disabled=t,this._toggleClass(this.widget(),this.widgetFullName+"-disabled",null,!0===t)},_setupEvents:function(t){var i={};t&&V.each(t.split(" "),function(t,e){i[e]="_eventHandler"}),this._off(this.anchors.add(this.tabs).add(this.panels)),this._on(!0,this.anchors,{click:function(t){t.preventDefault()}}),this._on(this.anchors,i),this._on(this.tabs,{keydown:"_tabKeydown"}),this._on(this.panels,{keydown:"_panelKeydown"}),this._focusable(this.tabs),this._hoverable(this.tabs)},_setupHeightStyle:function(t){var i,e=this.element.parent();"fill"===t?(i=e.height(),i-=this.element.outerHeight()-this.element.height(),this.element.siblings(":visible").each(function(){var t=V(this),e=t.css("position");"absolute"!==e&&"fixed"!==e&&(i-=t.outerHeight(!0))}),this.element.children().not(this.panels).each(function(){i-=V(this).outerHeight(!0)}),this.panels.each(function(){V(this).height(Math.max(0,i-V(this).innerHeight()+V(this).height()))}).css("overflow","auto")):"auto"===t&&(i=0,this.panels.each(function(){i=Math.max(i,V(this).height("").height())}).height(i))},_eventHandler:function(t){var e=this.options,i=this.active,s=V(t.currentTarget).closest("li"),n=s[0]===i[0],o=n&&e.collapsible,a=o?V():this._getPanelForTab(s),r=i.length?this._getPanelForTab(i):V(),i={oldTab:i,oldPanel:r,newTab:o?V():s,newPanel:a};t.preventDefault(),s.hasClass("ui-state-disabled")||s.hasClass("ui-tabs-loading")||this.running||n&&!e.collapsible||!1===this._trigger("beforeActivate",t,i)||(e.active=!o&&this.tabs.index(s),this.active=n?V():s,this.xhr&&this.xhr.abort(),r.length||a.length||V.error("jQuery UI Tabs: Mismatching fragment identifier."),a.length&&this.load(this.tabs.index(s),t),this._toggle(t,i))},_toggle:function(t,e){var i=this,s=e.newPanel,n=e.oldPanel;function o(){i.running=!1,i._trigger("activate",t,e)}function a(){i._addClass(e.newTab.closest("li"),"ui-tabs-active","ui-state-active"),s.length&&i.options.show?i._show(s,i.options.show,o):(s.show(),o())}this.running=!0,n.length&&this.options.hide?this._hide(n,this.options.hide,function(){i._removeClass(e.oldTab.closest("li"),"ui-tabs-active","ui-state-active"),a()}):(this._removeClass(e.oldTab.closest("li"),"ui-tabs-active","ui-state-active"),n.hide(),a()),n.attr("aria-hidden","true"),e.oldTab.attr({"aria-selected":"false","aria-expanded":"false"}),s.length&&n.length?e.oldTab.attr("tabIndex",-1):s.length&&this.tabs.filter(function(){return 0===V(this).attr("tabIndex")}).attr("tabIndex",-1),s.attr("aria-hidden","false"),e.newTab.attr({"aria-selected":"true","aria-expanded":"true",tabIndex:0})},_activate:function(t){var t=this._findActive(t);t[0]!==this.active[0]&&(t=(t=!t.length?this.active:t).find(".ui-tabs-anchor")[0],this._eventHandler({target:t,currentTarget:t,preventDefault:V.noop}))},_findActive:function(t){return!1===t?V():this.tabs.eq(t)},_getIndex:function(t){return t="string"==typeof t?this.anchors.index(this.anchors.filter("[href$='"+V.escapeSelector(t)+"']")):t},_destroy:function(){this.xhr&&this.xhr.abort(),this.tablist.removeAttr("role").off(this.eventNamespace),this.anchors.removeAttr("role tabIndex").removeUniqueId(),this.tabs.add(this.panels).each(function(){V.data(this,"ui-tabs-destroy")?V(this).remove():V(this).removeAttr("role tabIndex aria-live aria-busy aria-selected aria-labelledby aria-hidden aria-expanded")}),this.tabs.each(function(){var t=V(this),e=t.data("ui-tabs-aria-controls");e?t.attr("aria-controls",e).removeData("ui-tabs-aria-controls"):t.removeAttr("aria-controls")}),this.panels.show(),"content"!==this.options.heightStyle&&this.panels.css("height","")},enable:function(i){var t=this.options.disabled;!1!==t&&(t=void 0!==i&&(i=this._getIndex(i),Array.isArray(t)?V.map(t,function(t){return t!==i?t:null}):V.map(this.tabs,function(t,e){return e!==i?e:null})),this._setOptionDisabled(t))},disable:function(t){var e=this.options.disabled;if(!0!==e){if(void 0===t)e=!0;else{if(t=this._getIndex(t),-1!==V.inArray(t,e))return;e=Array.isArray(e)?V.merge([t],e).sort():[t]}this._setOptionDisabled(e)}},load:function(t,s){t=this._getIndex(t);function n(t,e){"abort"===e&&o.panels.stop(!1,!0),o._removeClass(i,"ui-tabs-loading"),a.removeAttr("aria-busy"),t===o.xhr&&delete o.xhr}var o=this,i=this.tabs.eq(t),t=i.find(".ui-tabs-anchor"),a=this._getPanelForTab(i),r={tab:i,panel:a};this._isLocal(t[0])||(this.xhr=V.ajax(this._ajaxSettings(t,s,r)),this.xhr&&"canceled"!==this.xhr.statusText&&(this._addClass(i,"ui-tabs-loading"),a.attr("aria-busy","true"),this.xhr.done(function(t,e,i){setTimeout(function(){a.html(t),o._trigger("load",s,r),n(i,e)},1)}).fail(function(t,e){setTimeout(function(){n(t,e)},1)})))},_ajaxSettings:function(t,i,s){var n=this;return{url:t.attr("href").replace(/#.*$/,""),beforeSend:function(t,e){return n._trigger("beforeLoad",i,V.extend({jqXHR:t,ajaxSettings:e},s))}}},_getPanelForTab:function(t){t=V(t).attr("aria-controls");return this.element.find(this._sanitizeSelector("#"+t))}}),!1!==V.uiBackCompat&&V.widget("ui.tabs",V.ui.tabs,{_processTabs:function(){this._superApply(arguments),this._addClass(this.tabs,"ui-tab")}});V.ui.tabs;V.widget("ui.tooltip",{version:"1.13.1",options:{classes:{"ui-tooltip":"ui-corner-all ui-widget-shadow"},content:function(){var t=V(this).attr("title");return V("").text(t).html()},hide:!0,items:"[title]:not([disabled])",position:{my:"left top+15",at:"left bottom",collision:"flipfit flip"},show:!0,track:!1,close:null,open:null},_addDescribedBy:function(t,e){var i=(t.attr("aria-describedby")||"").split(/\s+/);i.push(e),t.data("ui-tooltip-id",e).attr("aria-describedby",String.prototype.trim.call(i.join(" ")))},_removeDescribedBy:function(t){var e=t.data("ui-tooltip-id"),i=(t.attr("aria-describedby")||"").split(/\s+/),e=V.inArray(e,i);-1!==e&&i.splice(e,1),t.removeData("ui-tooltip-id"),(i=String.prototype.trim.call(i.join(" ")))?t.attr("aria-describedby",i):t.removeAttr("aria-describedby")},_create:function(){this._on({mouseover:"open",focusin:"open"}),this.tooltips={},this.parents={},this.liveRegion=V("
    ").attr({role:"log","aria-live":"assertive","aria-relevant":"additions"}).appendTo(this.document[0].body),this._addClass(this.liveRegion,null,"ui-helper-hidden-accessible"),this.disabledTitles=V([])},_setOption:function(t,e){var i=this;this._super(t,e),"content"===t&&V.each(this.tooltips,function(t,e){i._updateContent(e.element)})},_setOptionDisabled:function(t){this[t?"_disable":"_enable"]()},_disable:function(){var s=this;V.each(this.tooltips,function(t,e){var i=V.Event("blur");i.target=i.currentTarget=e.element[0],s.close(i,!0)}),this.disabledTitles=this.disabledTitles.add(this.element.find(this.options.items).addBack().filter(function(){var t=V(this);if(t.is("[title]"))return t.data("ui-tooltip-title",t.attr("title")).removeAttr("title")}))},_enable:function(){this.disabledTitles.each(function(){var t=V(this);t.data("ui-tooltip-title")&&t.attr("title",t.data("ui-tooltip-title"))}),this.disabledTitles=V([])},open:function(t){var i=this,e=V(t?t.target:this.element).closest(this.options.items);e.length&&!e.data("ui-tooltip-id")&&(e.attr("title")&&e.data("ui-tooltip-title",e.attr("title")),e.data("ui-tooltip-open",!0),t&&"mouseover"===t.type&&e.parents().each(function(){var t,e=V(this);e.data("ui-tooltip-open")&&((t=V.Event("blur")).target=t.currentTarget=this,i.close(t,!0)),e.attr("title")&&(e.uniqueId(),i.parents[this.id]={element:this,title:e.attr("title")},e.attr("title",""))}),this._registerCloseHandlers(t,e),this._updateContent(e,t))},_updateContent:function(e,i){var t=this.options.content,s=this,n=i?i.type:null;if("string"==typeof t||t.nodeType||t.jquery)return this._open(i,e,t);(t=t.call(e[0],function(t){s._delay(function(){e.data("ui-tooltip-open")&&(i&&(i.type=n),this._open(i,e,t))})}))&&this._open(i,e,t)},_open:function(t,e,i){var s,n,o,a=V.extend({},this.options.position);function r(t){a.of=t,n.is(":hidden")||n.position(a)}i&&((s=this._find(e))?s.tooltip.find(".ui-tooltip-content").html(i):(e.is("[title]")&&(t&&"mouseover"===t.type?e.attr("title",""):e.removeAttr("title")),s=this._tooltip(e),n=s.tooltip,this._addDescribedBy(e,n.attr("id")),n.find(".ui-tooltip-content").html(i),this.liveRegion.children().hide(),(i=V("
    ").html(n.find(".ui-tooltip-content").html())).removeAttr("name").find("[name]").removeAttr("name"),i.removeAttr("id").find("[id]").removeAttr("id"),i.appendTo(this.liveRegion),this.options.track&&t&&/^mouse/.test(t.type)?(this._on(this.document,{mousemove:r}),r(t)):n.position(V.extend({of:e},this.options.position)),n.hide(),this._show(n,this.options.show),this.options.track&&this.options.show&&this.options.show.delay&&(o=this.delayedShow=setInterval(function(){n.is(":visible")&&(r(a.of),clearInterval(o))},13)),this._trigger("open",t,{tooltip:n})))},_registerCloseHandlers:function(t,e){var i={keyup:function(t){t.keyCode===V.ui.keyCode.ESCAPE&&((t=V.Event(t)).currentTarget=e[0],this.close(t,!0))}};e[0]!==this.element[0]&&(i.remove=function(){var t=this._find(e);t&&this._removeTooltip(t.tooltip)}),t&&"mouseover"!==t.type||(i.mouseleave="close"),t&&"focusin"!==t.type||(i.focusout="close"),this._on(!0,e,i)},close:function(t){var e,i=this,s=V(t?t.currentTarget:this.element),n=this._find(s);n?(e=n.tooltip,n.closing||(clearInterval(this.delayedShow),s.data("ui-tooltip-title")&&!s.attr("title")&&s.attr("title",s.data("ui-tooltip-title")),this._removeDescribedBy(s),n.hiding=!0,e.stop(!0),this._hide(e,this.options.hide,function(){i._removeTooltip(V(this))}),s.removeData("ui-tooltip-open"),this._off(s,"mouseleave focusout keyup"),s[0]!==this.element[0]&&this._off(s,"remove"),this._off(this.document,"mousemove"),t&&"mouseleave"===t.type&&V.each(this.parents,function(t,e){V(e.element).attr("title",e.title),delete i.parents[t]}),n.closing=!0,this._trigger("close",t,{tooltip:e}),n.hiding||(n.closing=!1))):s.removeData("ui-tooltip-open")},_tooltip:function(t){var e=V("
    ").attr("role","tooltip"),i=V("
    ").appendTo(e),s=e.uniqueId().attr("id");return this._addClass(i,"ui-tooltip-content"),this._addClass(e,"ui-tooltip","ui-widget ui-widget-content"),e.appendTo(this._appendTo(t)),this.tooltips[s]={element:t,tooltip:e}},_find:function(t){t=t.data("ui-tooltip-id");return t?this.tooltips[t]:null},_removeTooltip:function(t){clearInterval(this.delayedShow),t.remove(),delete this.tooltips[t.attr("id")]},_appendTo:function(t){t=t.closest(".ui-front, dialog");return t=!t.length?this.document[0].body:t},_destroy:function(){var s=this;V.each(this.tooltips,function(t,e){var i=V.Event("blur"),e=e.element;i.target=i.currentTarget=e[0],s.close(i,!0),V("#"+t).remove(),e.data("ui-tooltip-title")&&(e.attr("title")||e.attr("title",e.data("ui-tooltip-title")),e.removeData("ui-tooltip-title"))}),this.liveRegion.remove()}}),!1!==V.uiBackCompat&&V.widget("ui.tooltip",V.ui.tooltip,{options:{tooltipClass:null},_tooltip:function(){var t=this._superApply(arguments);return this.options.tooltipClass&&t.tooltip.addClass(this.options.tooltipClass),t}});V.ui.tooltip;var H=V,z={},A=z.toString,O=/^([\-+])=\s*(\d+\.?\d*)/,N=[{re:/rgba?\(\s*(\d{1,3})\s*,\s*(\d{1,3})\s*,\s*(\d{1,3})\s*(?:,\s*(\d?(?:\.\d+)?)\s*)?\)/,parse:function(t){return[t[1],t[2],t[3],t[4]]}},{re:/rgba?\(\s*(\d+(?:\.\d+)?)\%\s*,\s*(\d+(?:\.\d+)?)\%\s*,\s*(\d+(?:\.\d+)?)\%\s*(?:,\s*(\d?(?:\.\d+)?)\s*)?\)/,parse:function(t){return[2.55*t[1],2.55*t[2],2.55*t[3],t[4]]}},{re:/#([a-f0-9]{2})([a-f0-9]{2})([a-f0-9]{2})([a-f0-9]{2})?/,parse:function(t){return[parseInt(t[1],16),parseInt(t[2],16),parseInt(t[3],16),t[4]?(parseInt(t[4],16)/255).toFixed(2):1]}},{re:/#([a-f0-9])([a-f0-9])([a-f0-9])([a-f0-9])?/,parse:function(t){return[parseInt(t[1]+t[1],16),parseInt(t[2]+t[2],16),parseInt(t[3]+t[3],16),t[4]?(parseInt(t[4]+t[4],16)/255).toFixed(2):1]}},{re:/hsla?\(\s*(\d+(?:\.\d+)?)\s*,\s*(\d+(?:\.\d+)?)\%\s*,\s*(\d+(?:\.\d+)?)\%\s*(?:,\s*(\d?(?:\.\d+)?)\s*)?\)/,space:"hsla",parse:function(t){return[t[1],t[2]/100,t[3]/100,t[4]]}}],E=H.Color=function(t,e,i,s){return new H.Color.fn.parse(t,e,i,s)},W={rgba:{props:{red:{idx:0,type:"byte"},green:{idx:1,type:"byte"},blue:{idx:2,type:"byte"}}},hsla:{props:{hue:{idx:0,type:"degrees"},saturation:{idx:1,type:"percent"},lightness:{idx:2,type:"percent"}}}},F={byte:{floor:!0,max:255},percent:{max:1},degrees:{mod:360,floor:!0}},L=E.support={},R=H("

    ")[0],Y=H.each;function B(t){return null==t?t+"":"object"==typeof t?z[A.call(t)]||"object":typeof t}function j(t,e,i){var s=F[e.type]||{};return null==t?i||!e.def?null:e.def:(t=s.floor?~~t:parseFloat(t),isNaN(t)?e.def:s.mod?(t+s.mod)%s.mod:Math.min(s.max,Math.max(0,t)))}function q(s){var n=E(),o=n._rgba=[];return s=s.toLowerCase(),Y(N,function(t,e){var i=e.re.exec(s),i=i&&e.parse(i),e=e.space||"rgba";if(i)return i=n[e](i),n[W[e].cache]=i[W[e].cache],o=n._rgba=i._rgba,!1}),o.length?("0,0,0,0"===o.join()&&H.extend(o,st.transparent),n):st[s]}function K(t,e,i){return 6*(i=(i+1)%1)<1?t+(e-t)*i*6:2*i<1?e:3*i<2?t+(e-t)*(2/3-i)*6:t}R.style.cssText="background-color:rgba(1,1,1,.5)",L.rgba=-1o.mod/2?s+=o.mod:s-n>o.mod/2&&(s-=o.mod)),l[i]=j((n-s)*a+s,e)))}),this[e](l)},blend:function(t){if(1===this._rgba[3])return this;var e=this._rgba.slice(),i=e.pop(),s=E(t)._rgba;return E(H.map(e,function(t,e){return(1-i)*s[e]+i*t}))},toRgbaString:function(){var t="rgba(",e=H.map(this._rgba,function(t,e){return null!=t?t:2

    ").addClass("ui-effects-wrapper").css({fontSize:"100%",background:"transparent",border:"none",margin:0,padding:0}),e={width:i.width(),height:i.height()},n=document.activeElement;try{n.id}catch(t){n=document.body}return i.wrap(t),i[0]!==n&&!V.contains(i[0],n)||V(n).trigger("focus"),t=i.parent(),"static"===i.css("position")?(t.css({position:"relative"}),i.css({position:"relative"})):(V.extend(s,{position:i.css("position"),zIndex:i.css("z-index")}),V.each(["top","left","bottom","right"],function(t,e){s[e]=i.css(e),isNaN(parseInt(s[e],10))&&(s[e]="auto")}),i.css({position:"relative",top:0,left:0,right:"auto",bottom:"auto"})),i.css(e),t.css(s).show()},removeWrapper:function(t){var e=document.activeElement;return t.parent().is(".ui-effects-wrapper")&&(t.parent().replaceWith(t),t[0]!==e&&!V.contains(t[0],e)||V(e).trigger("focus")),t}}),V.extend(V.effects,{version:"1.13.1",define:function(t,e,i){return i||(i=e,e="effect"),V.effects.effect[t]=i,V.effects.effect[t].mode=e,i},scaledDimensions:function(t,e,i){if(0===e)return{height:0,width:0,outerHeight:0,outerWidth:0};var s="horizontal"!==i?(e||100)/100:1,e="vertical"!==i?(e||100)/100:1;return{height:t.height()*e,width:t.width()*s,outerHeight:t.outerHeight()*e,outerWidth:t.outerWidth()*s}},clipToBox:function(t){return{width:t.clip.right-t.clip.left,height:t.clip.bottom-t.clip.top,left:t.clip.left,top:t.clip.top}},unshift:function(t,e,i){var s=t.queue();1").insertAfter(t).css({display:/^(inline|ruby)/.test(t.css("display"))?"inline-block":"block",visibility:"hidden",marginTop:t.css("marginTop"),marginBottom:t.css("marginBottom"),marginLeft:t.css("marginLeft"),marginRight:t.css("marginRight"),float:t.css("float")}).outerWidth(t.outerWidth()).outerHeight(t.outerHeight()).addClass("ui-effects-placeholder"),t.data(nt+"placeholder",e)),t.css({position:i,left:s.left,top:s.top}),e},removePlaceholder:function(t){var e=nt+"placeholder",i=t.data(e);i&&(i.remove(),t.removeData(e))},cleanUp:function(t){V.effects.restoreStyle(t),V.effects.removePlaceholder(t)},setTransition:function(s,t,n,o){return o=o||{},V.each(t,function(t,e){var i=s.cssUnit(e);0
    ");l.appendTo("body").addClass(t.className).css({top:s.top-a,left:s.left-r,height:i.innerHeight(),width:i.innerWidth(),position:n?"fixed":"absolute"}).animate(o,t.duration,t.easing,function(){l.remove(),"function"==typeof e&&e()})}}),V.fx.step.clip=function(t){t.clipInit||(t.start=V(t.elem).cssClip(),"string"==typeof t.end&&(t.end=ct(t.end,t.elem)),t.clipInit=!0),V(t.elem).cssClip({top:t.pos*(t.end.top-t.start.top)+t.start.top,right:t.pos*(t.end.right-t.start.right)+t.start.right,bottom:t.pos*(t.end.bottom-t.start.bottom)+t.start.bottom,left:t.pos*(t.end.left-t.start.left)+t.start.left})},it={},V.each(["Quad","Cubic","Quart","Quint","Expo"],function(e,t){it[t]=function(t){return Math.pow(t,e+2)}}),V.extend(it,{Sine:function(t){return 1-Math.cos(t*Math.PI/2)},Circ:function(t){return 1-Math.sqrt(1-t*t)},Elastic:function(t){return 0===t||1===t?t:-Math.pow(2,8*(t-1))*Math.sin((80*(t-1)-7.5)*Math.PI/15)},Back:function(t){return t*t*(3*t-2)},Bounce:function(t){for(var e,i=4;t<((e=Math.pow(2,--i))-1)/11;);return 1/Math.pow(4,3-i)-7.5625*Math.pow((3*e-2)/22-t,2)}}),V.each(it,function(t,e){V.easing["easeIn"+t]=e,V.easing["easeOut"+t]=function(t){return 1-e(1-t)},V.easing["easeInOut"+t]=function(t){return t<.5?e(2*t)/2:1-e(-2*t+2)/2}});R=V.effects,V.effects.define("blind","hide",function(t,e){var i={up:["bottom","top"],vertical:["bottom","top"],down:["top","bottom"],left:["right","left"],horizontal:["right","left"],right:["left","right"]},s=V(this),n=t.direction||"up",o=s.cssClip(),a={clip:V.extend({},o)},r=V.effects.createPlaceholder(s);a.clip[i[n][0]]=a.clip[i[n][1]],"show"===t.mode&&(s.cssClip(a.clip),r&&r.css(V.effects.clipToBox(a)),a.clip=o),r&&r.animate(V.effects.clipToBox(a),t.duration,t.easing),s.animate(a,{queue:!1,duration:t.duration,easing:t.easing,complete:e})}),V.effects.define("bounce",function(t,e){var i,s,n=V(this),o=t.mode,a="hide"===o,r="show"===o,l=t.direction||"up",h=t.distance,c=t.times||5,o=2*c+(r||a?1:0),u=t.duration/o,d=t.easing,p="up"===l||"down"===l?"top":"left",f="up"===l||"left"===l,g=0,t=n.queue().length;for(V.effects.createPlaceholder(n),l=n.css(p),h=h||n["top"==p?"outerHeight":"outerWidth"]()/3,r&&((s={opacity:1})[p]=l,n.css("opacity",0).css(p,f?2*-h:2*h).animate(s,u,d)),a&&(h/=Math.pow(2,c-1)),(s={})[p]=l;g
    ").css({position:"absolute",visibility:"visible",left:-s*p,top:-i*f}).parent().addClass("ui-effects-explode").css({position:"absolute",overflow:"hidden",width:p,height:f,left:n+(u?a*p:0),top:o+(u?r*f:0),opacity:u?0:1}).animate({left:n+(u?0:a*p),top:o+(u?0:r*f),opacity:u?1:0},t.duration||500,t.easing,m)}),V.effects.define("fade","toggle",function(t,e){var i="show"===t.mode;V(this).css("opacity",i?0:1).animate({opacity:i?1:0},{queue:!1,duration:t.duration,easing:t.easing,complete:e})}),V.effects.define("fold","hide",function(e,t){var i=V(this),s=e.mode,n="show"===s,o="hide"===s,a=e.size||15,r=/([0-9]+)%/.exec(a),l=!!e.horizFirst?["right","bottom"]:["bottom","right"],h=e.duration/2,c=V.effects.createPlaceholder(i),u=i.cssClip(),d={clip:V.extend({},u)},p={clip:V.extend({},u)},f=[u[l[0]],u[l[1]]],s=i.queue().length;r&&(a=parseInt(r[1],10)/100*f[o?0:1]),d.clip[l[0]]=a,p.clip[l[0]]=a,p.clip[l[1]]=0,n&&(i.cssClip(p.clip),c&&c.css(V.effects.clipToBox(p)),p.clip=u),i.queue(function(t){c&&c.animate(V.effects.clipToBox(d),h,e.easing).animate(V.effects.clipToBox(p),h,e.easing),t()}).animate(d,h,e.easing).animate(p,h,e.easing).queue(t),V.effects.unshift(i,s,4)}),V.effects.define("highlight","show",function(t,e){var i=V(this),s={backgroundColor:i.css("backgroundColor")};"hide"===t.mode&&(s.opacity=0),V.effects.saveStyle(i),i.css({backgroundImage:"none",backgroundColor:t.color||"#ffff99"}).animate(s,{queue:!1,duration:t.duration,easing:t.easing,complete:e})}),V.effects.define("size",function(s,e){var n,i=V(this),t=["fontSize"],o=["borderTopWidth","borderBottomWidth","paddingTop","paddingBottom"],a=["borderLeftWidth","borderRightWidth","paddingLeft","paddingRight"],r=s.mode,l="effect"!==r,h=s.scale||"both",c=s.origin||["middle","center"],u=i.css("position"),d=i.position(),p=V.effects.scaledDimensions(i),f=s.from||p,g=s.to||V.effects.scaledDimensions(i,0);V.effects.createPlaceholder(i),"show"===r&&(r=f,f=g,g=r),n={from:{y:f.height/p.height,x:f.width/p.width},to:{y:g.height/p.height,x:g.width/p.width}},"box"!==h&&"both"!==h||(n.from.y!==n.to.y&&(f=V.effects.setTransition(i,o,n.from.y,f),g=V.effects.setTransition(i,o,n.to.y,g)),n.from.x!==n.to.x&&(f=V.effects.setTransition(i,a,n.from.x,f),g=V.effects.setTransition(i,a,n.to.x,g))),"content"!==h&&"both"!==h||n.from.y!==n.to.y&&(f=V.effects.setTransition(i,t,n.from.y,f),g=V.effects.setTransition(i,t,n.to.y,g)),c&&(c=V.effects.getBaseline(c,p),f.top=(p.outerHeight-f.outerHeight)*c.y+d.top,f.left=(p.outerWidth-f.outerWidth)*c.x+d.left,g.top=(p.outerHeight-g.outerHeight)*c.y+d.top,g.left=(p.outerWidth-g.outerWidth)*c.x+d.left),delete f.outerHeight,delete f.outerWidth,i.css(f),"content"!==h&&"both"!==h||(o=o.concat(["marginTop","marginBottom"]).concat(t),a=a.concat(["marginLeft","marginRight"]),i.find("*[width]").each(function(){var t=V(this),e=V.effects.scaledDimensions(t),i={height:e.height*n.from.y,width:e.width*n.from.x,outerHeight:e.outerHeight*n.from.y,outerWidth:e.outerWidth*n.from.x},e={height:e.height*n.to.y,width:e.width*n.to.x,outerHeight:e.height*n.to.y,outerWidth:e.width*n.to.x};n.from.y!==n.to.y&&(i=V.effects.setTransition(t,o,n.from.y,i),e=V.effects.setTransition(t,o,n.to.y,e)),n.from.x!==n.to.x&&(i=V.effects.setTransition(t,a,n.from.x,i),e=V.effects.setTransition(t,a,n.to.x,e)),l&&V.effects.saveStyle(t),t.css(i),t.animate(e,s.duration,s.easing,function(){l&&V.effects.restoreStyle(t)})})),i.animate(g,{queue:!1,duration:s.duration,easing:s.easing,complete:function(){var t=i.offset();0===g.opacity&&i.css("opacity",f.opacity),l||(i.css("position","static"===u?"relative":u).offset(t),V.effects.saveStyle(i)),e()}})}),V.effects.define("scale",function(t,e){var i=V(this),s=t.mode,s=parseInt(t.percent,10)||(0===parseInt(t.percent,10)||"effect"!==s?0:100),s=V.extend(!0,{from:V.effects.scaledDimensions(i),to:V.effects.scaledDimensions(i,s,t.direction||"both"),origin:t.origin||["middle","center"]},t);t.fade&&(s.from.opacity=1,s.to.opacity=0),V.effects.effect.size.call(this,s,e)}),V.effects.define("puff","hide",function(t,e){t=V.extend(!0,{},t,{fade:!0,percent:parseInt(t.percent,10)||150});V.effects.effect.scale.call(this,t,e)}),V.effects.define("pulsate","show",function(t,e){var i=V(this),s=t.mode,n="show"===s,o=2*(t.times||5)+(n||"hide"===s?1:0),a=t.duration/o,r=0,l=1,s=i.queue().length;for(!n&&i.is(":visible")||(i.css("opacity",0).show(),r=1);l entities = client.getContainerEntities(appId, + null, ImmutableMap.of("infofilters", appAttemptInfoFilter), 0, null); + Assert.assertEquals(2, entities.size()); + Assert.assertEquals("mockContainer4", entities.get(1).getId()); + } + @After public void tearDown() { if (client != null) { @@ -135,11 +148,15 @@ private class MockTimelineReaderClient extends TimelineReaderClientImpl { protected ClientResponse doGetUri(URI base, String path, MultivaluedMap params) throws IOException { ClientResponse mockClientResponse = mock(ClientResponse.class); - if (path.contains(YARN_CONTAINER.toString())) { + if (path.contains(YARN_CONTAINER.toString()) && !params.containsKey("infofilters")) { when(mockClientResponse.getEntity(TimelineEntity.class)).thenReturn( createTimelineEntity("mockContainer1")); when(mockClientResponse.getEntity(TimelineEntity[].class)).thenReturn( createTimelineEntities("mockContainer1", "mockContainer2")); + } else if (path.contains(YARN_CONTAINER.toString()) && params.containsKey("infofilters")) { + Assert.assertEquals(encodeValue(appAttemptInfoFilter), params.get("infofilters").get(0)); + when(mockClientResponse.getEntity(TimelineEntity[].class)).thenReturn( + createTimelineEntities("mockContainer3", "mockContainer4")); } else if (path.contains(YARN_APPLICATION_ATTEMPT.toString())) { when(mockClientResponse.getEntity(TimelineEntity.class)).thenReturn( createTimelineEntity("mockAppAttempt1")); @@ -151,6 +168,7 @@ protected ClientResponse doGetUri(URI base, String path, when(mockClientResponse.getEntity(TimelineEntity[].class)).thenReturn( createTimelineEntities("mockApp1", "mockApp2")); } + return mockClientResponse; } } diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/test/java/org/apache/hadoop/yarn/logaggregation/LogAggregationTestUtils.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/test/java/org/apache/hadoop/yarn/logaggregation/LogAggregationTestUtils.java new file mode 100644 index 0000000000000..3cd563a648932 --- /dev/null +++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/test/java/org/apache/hadoop/yarn/logaggregation/LogAggregationTestUtils.java @@ -0,0 +1,68 @@ +/** + * 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.yarn.logaggregation; + +import org.apache.commons.lang3.StringUtils; +import org.apache.hadoop.conf.Configuration; +import org.apache.hadoop.yarn.conf.YarnConfiguration; +import org.apache.hadoop.yarn.logaggregation.filecontroller.LogAggregationFileController; + +import java.util.List; + +import static org.apache.hadoop.yarn.conf.YarnConfiguration.LOG_AGGREGATION_FILE_CONTROLLER_FMT; +import static org.apache.hadoop.yarn.conf.YarnConfiguration.LOG_AGGREGATION_REMOTE_APP_LOG_DIR_FMT; +import static org.apache.hadoop.yarn.conf.YarnConfiguration.LOG_AGGREGATION_REMOTE_APP_LOG_DIR_SUFFIX_FMT; + + +public class LogAggregationTestUtils { + public static final String REMOTE_LOG_ROOT = "target/app-logs/"; + + public static void enableFileControllers(Configuration conf, + List> fileControllers, + List fileControllerNames) { + enableFcs(conf, REMOTE_LOG_ROOT, fileControllers, fileControllerNames); + } + + public static void enableFileControllers(Configuration conf, + String remoteLogRoot, + List> fileControllers, + List fileControllerNames) { + enableFcs(conf, remoteLogRoot, fileControllers, fileControllerNames); + } + + + private static void enableFcs(Configuration conf, + String remoteLogRoot, + List> fileControllers, + List fileControllerNames) { + conf.set(YarnConfiguration.LOG_AGGREGATION_FILE_FORMATS, + StringUtils.join(fileControllerNames, ",")); + for (int i = 0; i < fileControllers.size(); i++) { + Class fileController = fileControllers.get(i); + String controllerName = fileControllerNames.get(i); + + conf.setClass(String.format(LOG_AGGREGATION_FILE_CONTROLLER_FMT, controllerName), + fileController, LogAggregationFileController.class); + conf.set(String.format(LOG_AGGREGATION_REMOTE_APP_LOG_DIR_FMT, controllerName), + remoteLogRoot + controllerName + "/"); + conf.set(String.format(LOG_AGGREGATION_REMOTE_APP_LOG_DIR_SUFFIX_FMT, controllerName), + controllerName); + } + } +} diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/test/java/org/apache/hadoop/yarn/logaggregation/TestAggregatedLogDeletionService.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/test/java/org/apache/hadoop/yarn/logaggregation/TestAggregatedLogDeletionService.java index daa2fc6b01cbc..285ac43322a57 100644 --- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/test/java/org/apache/hadoop/yarn/logaggregation/TestAggregatedLogDeletionService.java +++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/test/java/org/apache/hadoop/yarn/logaggregation/TestAggregatedLogDeletionService.java @@ -18,557 +18,294 @@ package org.apache.hadoop.yarn.logaggregation; -import static org.apache.hadoop.yarn.conf.YarnConfiguration.LOG_AGGREGATION_FILE_CONTROLLER_FMT; - -import java.io.IOException; -import java.net.URI; -import java.util.Arrays; -import java.util.Collections; -import java.util.List; - +import org.apache.commons.lang3.tuple.Pair; import org.apache.hadoop.conf.Configuration; -import org.apache.hadoop.fs.FileStatus; import org.apache.hadoop.fs.FileSystem; import org.apache.hadoop.fs.FilterFileSystem; import org.apache.hadoop.fs.Path; -import org.apache.hadoop.security.AccessControlException; -import org.apache.hadoop.yarn.api.ApplicationClientProtocol; -import org.apache.hadoop.yarn.api.protocolrecords.GetApplicationReportRequest; -import org.apache.hadoop.yarn.api.protocolrecords.GetApplicationReportResponse; -import org.apache.hadoop.yarn.api.records.ApplicationId; -import org.apache.hadoop.yarn.api.records.ApplicationReport; -import org.apache.hadoop.yarn.api.records.YarnApplicationState; +import org.apache.hadoop.util.Lists; import org.apache.hadoop.yarn.conf.YarnConfiguration; +import org.apache.hadoop.yarn.logaggregation.filecontroller.LogAggregationFileController; +import org.apache.hadoop.yarn.logaggregation.filecontroller.ifile.LogAggregationIndexedFileController; import org.apache.hadoop.yarn.logaggregation.filecontroller.tfile.LogAggregationTFileController; +import org.apache.hadoop.yarn.logaggregation.testutils.LogAggregationTestcase; +import org.apache.hadoop.yarn.logaggregation.testutils.LogAggregationTestcaseBuilder; +import org.apache.hadoop.yarn.logaggregation.testutils.LogAggregationTestcaseBuilder.AppDescriptor; +import org.apache.log4j.Level; import org.junit.Before; +import org.junit.BeforeClass; import org.junit.Test; -import org.junit.Assert; -import static org.mockito.Mockito.*; +import java.io.IOException; +import java.net.URI; +import java.util.Arrays; +import java.util.List; + +import static org.apache.hadoop.yarn.conf.YarnConfiguration.LOG_AGGREGATION_FILE_CONTROLLER_FMT; +import static org.apache.hadoop.yarn.logaggregation.LogAggregationTestUtils.enableFileControllers; +import static org.apache.hadoop.yarn.logaggregation.testutils.LogAggregationTestcaseBuilder.NO_TIMEOUT; +import static org.mockito.Mockito.mock; public class TestAggregatedLogDeletionService { - + private static final String T_FILE = "TFile"; + private static final String I_FILE = "IFile"; + private static final String USER_ME = "me"; + private static final String DIR_HOST1 = "host1"; + private static final String DIR_HOST2 = "host2"; + + private static final String ROOT = "mockfs://foo/"; + private static final String REMOTE_ROOT_LOG_DIR = ROOT + "tmp/logs/"; + private static final String SUFFIX = "logs"; + private static final int TEN_DAYS_IN_SECONDS = 10 * 24 * 3600; + + private static final List> + ALL_FILE_CONTROLLERS = Arrays.asList( + LogAggregationIndexedFileController.class, + LogAggregationTFileController.class); + public static final List ALL_FILE_CONTROLLER_NAMES = Arrays.asList(I_FILE, T_FILE); + + @BeforeClass + public static void beforeClass() { + org.apache.log4j.Logger.getRootLogger().setLevel(Level.DEBUG); + } + @Before public void closeFilesystems() throws IOException { // prevent the same mockfs instance from being reused due to FS cache FileSystem.closeAll(); } + private Configuration setupConfiguration(int retainSeconds, int retainCheckIntervalSeconds) { + Configuration conf = new Configuration(); + conf.setClass("fs.mockfs.impl", MockFileSystem.class, FileSystem.class); + conf.setBoolean(YarnConfiguration.LOG_AGGREGATION_ENABLED, true); + conf.setInt(YarnConfiguration.LOG_AGGREGATION_RETAIN_SECONDS, retainSeconds); + conf.setInt(YarnConfiguration.LOG_AGGREGATION_RETAIN_CHECK_INTERVAL_SECONDS, + retainCheckIntervalSeconds); + conf.set(YarnConfiguration.NM_REMOTE_APP_LOG_DIR, REMOTE_ROOT_LOG_DIR); + conf.set(YarnConfiguration.NM_REMOTE_APP_LOG_DIR_SUFFIX, SUFFIX); + conf.set(YarnConfiguration.LOG_AGGREGATION_FILE_FORMATS, T_FILE); + conf.set(String.format(LOG_AGGREGATION_FILE_CONTROLLER_FMT, T_FILE), + LogAggregationTFileController.class.getName()); + return conf; + } + @Test public void testDeletion() throws Exception { long now = System.currentTimeMillis(); - long toDeleteTime = now - (2000*1000); - long toKeepTime = now - (1500*1000); - String root = "mockfs://foo/"; - String remoteRootLogDir = root+"tmp/logs"; - String suffix = "logs"; - String newSuffix = LogAggregationUtils.getBucketSuffix() + suffix; - final Configuration conf = new Configuration(); - conf.setClass("fs.mockfs.impl", MockFileSystem.class, FileSystem.class); - conf.set(YarnConfiguration.LOG_AGGREGATION_ENABLED, "true"); - conf.set(YarnConfiguration.LOG_AGGREGATION_RETAIN_SECONDS, "1800"); - conf.set(YarnConfiguration.NM_REMOTE_APP_LOG_DIR, remoteRootLogDir); - conf.set(YarnConfiguration.NM_REMOTE_APP_LOG_DIR_SUFFIX, suffix); - conf.set(YarnConfiguration.LOG_AGGREGATION_FILE_FORMATS, "TFile"); - conf.set(String.format(LOG_AGGREGATION_FILE_CONTROLLER_FMT, "TFile"), - LogAggregationTFileController.class.getName()); - - - Path rootPath = new Path(root); - FileSystem rootFs = rootPath.getFileSystem(conf); - FileSystem mockFs = ((FilterFileSystem)rootFs).getRawFileSystem(); - - Path remoteRootLogPath = new Path(remoteRootLogDir); - - Path userDir = new Path(remoteRootLogPath, "me"); - FileStatus userDirStatus = new FileStatus(0, true, 0, 0, toKeepTime, userDir); - - when(mockFs.listStatus(remoteRootLogPath)).thenReturn( - new FileStatus[]{userDirStatus}); - - ApplicationId appId1 = - ApplicationId.newInstance(now, 1); - Path suffixDir = new Path(userDir, newSuffix); - FileStatus suffixDirStatus = new FileStatus(0, true, - 0, 0, toDeleteTime, suffixDir); - Path bucketDir = LogAggregationUtils.getRemoteBucketDir( - remoteRootLogPath, "me", suffix, appId1); - FileStatus bucketDirStatus = new FileStatus(0, true, 0, - 0, toDeleteTime, bucketDir); - Path app1Dir = LogAggregationUtils.getRemoteAppLogDir( - remoteRootLogPath, appId1, "me", suffix); - FileStatus app1DirStatus = new FileStatus(0, true, 0, 0, - toDeleteTime, app1Dir); - - ApplicationId appId2 = - ApplicationId.newInstance(now, 2); - Path app2Dir = LogAggregationUtils.getRemoteAppLogDir( - remoteRootLogPath, appId2, "me", suffix); - FileStatus app2DirStatus = new FileStatus(0, true, 0, 0, - toDeleteTime, app2Dir); - - ApplicationId appId3 = - ApplicationId.newInstance(now, 3); - Path app3Dir = LogAggregationUtils.getRemoteAppLogDir( - remoteRootLogPath, appId3, "me", suffix); - FileStatus app3DirStatus = new FileStatus(0, true, 0, 0, - toDeleteTime, app3Dir); - - ApplicationId appId4 = - ApplicationId.newInstance(now, 4); - Path app4Dir = LogAggregationUtils.getRemoteAppLogDir( - remoteRootLogPath, appId4, "me", suffix); - FileStatus app4DirStatus = - new FileStatus(0, true, 0, 0, toDeleteTime, app4Dir); - - when(mockFs.listStatus(userDir)).thenReturn( - new FileStatus[] {suffixDirStatus}); - when(mockFs.listStatus(suffixDir)).thenReturn( - new FileStatus[] {bucketDirStatus}); - when(mockFs.listStatus(bucketDir)).thenReturn( - new FileStatus[] {app1DirStatus, app2DirStatus, - app3DirStatus, app4DirStatus}); - - when(mockFs.listStatus(app1Dir)).thenReturn( - new FileStatus[]{}); - - - Path app2Log1 = new Path(app2Dir, "host1"); - FileStatus app2Log1Status = new FileStatus(10, false, 1, 1, toDeleteTime, app2Log1); - - Path app2Log2 = new Path(app2Dir, "host2"); - FileStatus app2Log2Status = new FileStatus(10, false, 1, 1, toKeepTime, app2Log2); - - when(mockFs.listStatus(app2Dir)).thenReturn( - new FileStatus[]{app2Log1Status, app2Log2Status}); - - Path app3Log1 = new Path(app3Dir, "host1"); - FileStatus app3Log1Status = new FileStatus(10, false, 1, 1, toDeleteTime, app3Log1); - - Path app3Log2 = new Path(app3Dir, "host2"); - FileStatus app3Log2Status = new FileStatus(10, false, 1, 1, toDeleteTime, app3Log2); - - when(mockFs.delete(app3Dir, true)).thenThrow(new AccessControlException("Injected Error\nStack Trace :(")); - - when(mockFs.listStatus(app3Dir)).thenReturn( - new FileStatus[]{app3Log1Status, app3Log2Status}); - - Path app4Log1 = new Path(app4Dir, "host1"); - FileStatus app4Log1Status = new FileStatus(10, false, 1, 1, toDeleteTime, app4Log1); - - Path app4Log2 = new Path(app4Dir, "host2"); - FileStatus app4Log2Status = new FileStatus(10, false, 1, 1, - toKeepTime, app4Log2); - - when(mockFs.listStatus(app4Dir)).thenReturn( - new FileStatus[]{app4Log1Status, app4Log2Status}); - - final List finishedApplications = - Collections.unmodifiableList(Arrays.asList(appId1, appId2, appId3)); - final List runningApplications = - Collections.unmodifiableList(Arrays.asList(appId4)); - - AggregatedLogDeletionService deletionService = - new AggregatedLogDeletionService() { - @Override - protected ApplicationClientProtocol createRMClient() - throws IOException { - try { - return createMockRMClient(finishedApplications, - runningApplications); - } catch (Exception e) { - throw new IOException(e); - } - } - @Override - protected void stopRMClient() { - // DO NOTHING - } - }; - deletionService.init(conf); - deletionService.start(); - - verify(mockFs, timeout(2000)).delete(app1Dir, true); - verify(mockFs, timeout(2000).times(0)).delete(app2Dir, true); - verify(mockFs, timeout(2000)).delete(app3Dir, true); - verify(mockFs, timeout(2000).times(0)).delete(app4Dir, true); - verify(mockFs, timeout(2000)).delete(app4Log1, true); - verify(mockFs, timeout(2000).times(0)).delete(app4Log2, true); - - deletionService.stop(); + long toDeleteTime = now - (2000 * 1000); + long toKeepTime = now - (1500 * 1000); + + Configuration conf = setupConfiguration(1800, -1); + long timeout = 2000L; + LogAggregationTestcaseBuilder.create(conf) + .withRootPath(ROOT) + .withRemoteRootLogPath(REMOTE_ROOT_LOG_DIR) + .withUserDir(USER_ME, toKeepTime) + .withSuffixDir(SUFFIX, toDeleteTime) + .withBucketDir(toDeleteTime) + .withApps(Lists.newArrayList( + new AppDescriptor(toDeleteTime, Lists.newArrayList()), + new AppDescriptor(toDeleteTime, Lists.newArrayList( + Pair.of(DIR_HOST1, toDeleteTime), + Pair.of(DIR_HOST2, toKeepTime))), + new AppDescriptor(toDeleteTime, Lists.newArrayList( + Pair.of(DIR_HOST1, toDeleteTime), + Pair.of(DIR_HOST2, toDeleteTime))), + new AppDescriptor(toDeleteTime, Lists.newArrayList( + Pair.of(DIR_HOST1, toDeleteTime), + Pair.of(DIR_HOST2, toKeepTime))))) + .withFinishedApps(1, 2, 3) + .withRunningApps(4) + .injectExceptionForAppDirDeletion(3) + .build() + .startDeletionService() + .verifyAppDirsDeleted(timeout, 1, 3) + .verifyAppDirsNotDeleted(timeout, 2, 4) + .verifyAppFileDeleted(4, 1, timeout) + .verifyAppFileNotDeleted(4, 2, timeout) + .teardown(1); } @Test public void testRefreshLogRetentionSettings() throws Exception { long now = System.currentTimeMillis(); - //time before 2000 sec long before2000Secs = now - (2000 * 1000); - //time before 50 sec long before50Secs = now - (50 * 1000); - String root = "mockfs://foo/"; - String remoteRootLogDir = root + "tmp/logs"; - String suffix = "logs"; - String newSuffix = LogAggregationUtils.getBucketSuffix() + suffix; - final Configuration conf = new Configuration(); - conf.setClass("fs.mockfs.impl", MockFileSystem.class, FileSystem.class); - conf.set(YarnConfiguration.LOG_AGGREGATION_ENABLED, "true"); - conf.set(YarnConfiguration.LOG_AGGREGATION_RETAIN_SECONDS, "1800"); - conf.set(YarnConfiguration.LOG_AGGREGATION_RETAIN_CHECK_INTERVAL_SECONDS, - "1"); - conf.set(YarnConfiguration.NM_REMOTE_APP_LOG_DIR, remoteRootLogDir); - conf.set(YarnConfiguration.NM_REMOTE_APP_LOG_DIR_SUFFIX, suffix); - conf.set(YarnConfiguration.LOG_AGGREGATION_FILE_FORMATS, "TFile"); - conf.set(String.format(LOG_AGGREGATION_FILE_CONTROLLER_FMT, "TFile"), - LogAggregationTFileController.class.getName()); - - - Path rootPath = new Path(root); - FileSystem rootFs = rootPath.getFileSystem(conf); - FileSystem mockFs = ((FilterFileSystem) rootFs).getRawFileSystem(); - - Path remoteRootLogPath = new Path(remoteRootLogDir); - - Path userDir = new Path(remoteRootLogPath, "me"); - FileStatus userDirStatus = new FileStatus(0, true, 0, 0, before50Secs, - userDir); - - when(mockFs.listStatus(remoteRootLogPath)).thenReturn( - new FileStatus[] { userDirStatus }); - - Path suffixDir = new Path(userDir, newSuffix); - FileStatus suffixStatus = new FileStatus(0, true, 0, 0, before50Secs, - suffixDir); - - ApplicationId appId1 = - ApplicationId.newInstance(System.currentTimeMillis(), 1); - //Set time last modified of app1Dir directory and its files to before2000Secs - Path app1Dir = LogAggregationUtils.getRemoteAppLogDir( - remoteRootLogPath, appId1, "me", suffix); - Path bucketDir = LogAggregationUtils.getRemoteBucketDir( - remoteRootLogPath, "me", suffix, appId1); - FileStatus bucketDirStatus = new FileStatus(0, true, 0, - 0, before50Secs, bucketDir); - FileStatus app1DirStatus = new FileStatus(0, true, 0, 0, before2000Secs, - app1Dir); - - ApplicationId appId2 = - ApplicationId.newInstance(System.currentTimeMillis(), 2); - //Set time last modified of app1Dir directory and its files to before50Secs - Path app2Dir = LogAggregationUtils.getRemoteAppLogDir( - remoteRootLogPath, appId2, "me", suffix); - FileStatus app2DirStatus = new FileStatus(0, true, 0, 0, before50Secs, - app2Dir); - - when(mockFs.listStatus(userDir)).thenReturn( - new FileStatus[] {suffixStatus }); - when(mockFs.listStatus(suffixDir)).thenReturn( - new FileStatus[] {bucketDirStatus }); - when(mockFs.listStatus(bucketDir)).thenReturn( - new FileStatus[] {app1DirStatus, app2DirStatus }); - - Path app1Log1 = new Path(app1Dir, "host1"); - FileStatus app1Log1Status = new FileStatus(10, false, 1, 1, before2000Secs, - app1Log1); - - when(mockFs.listStatus(app1Dir)).thenReturn( - new FileStatus[] { app1Log1Status }); - - Path app2Log1 = new Path(app2Dir, "host1"); - FileStatus app2Log1Status = new FileStatus(10, false, 1, 1, before50Secs, - app2Log1); - - when(mockFs.listStatus(app2Dir)).thenReturn( - new FileStatus[] { app2Log1Status }); - - final List finishedApplications = - Collections.unmodifiableList(Arrays.asList(appId1, appId2)); - - AggregatedLogDeletionService deletionSvc = new AggregatedLogDeletionService() { - @Override - protected Configuration createConf() { - return conf; - } - @Override - protected ApplicationClientProtocol createRMClient() - throws IOException { - try { - return createMockRMClient(finishedApplications, null); - } catch (Exception e) { - throw new IOException(e); - } - } - @Override - protected void stopRMClient() { - // DO NOTHING - } - }; - - deletionSvc.init(conf); - deletionSvc.start(); + int checkIntervalSeconds = 2; + int checkIntervalMilliSeconds = checkIntervalSeconds * 1000; + + Configuration conf = setupConfiguration(1800, 1); + + LogAggregationTestcase testcase = LogAggregationTestcaseBuilder.create(conf) + .withRootPath(ROOT) + .withRemoteRootLogPath(REMOTE_ROOT_LOG_DIR) + .withUserDir(USER_ME, before50Secs) + .withSuffixDir(SUFFIX, before50Secs) + .withBucketDir(before50Secs) + .withApps(Lists.newArrayList( + //Set time last modified of app1Dir directory and its files to before2000Secs + new AppDescriptor(before2000Secs, Lists.newArrayList( + Pair.of(DIR_HOST1, before2000Secs))), + //Set time last modified of app1Dir directory and its files to before50Secs + new AppDescriptor(before50Secs, Lists.newArrayList( + Pair.of(DIR_HOST1, before50Secs)))) + ) + .withFinishedApps(1, 2) + .withRunningApps() + .build(); - //app1Dir would be deleted since its done above log retention period - verify(mockFs, timeout(10000)).delete(app1Dir, true); - //app2Dir is not expected to be deleted since its below the threshold - verify(mockFs, timeout(3000).times(0)).delete(app2Dir, true); - - //Now,lets change the confs - conf.set(YarnConfiguration.LOG_AGGREGATION_RETAIN_SECONDS, "50"); - conf.set(YarnConfiguration.LOG_AGGREGATION_RETAIN_CHECK_INTERVAL_SECONDS, - "2"); - //We have not called refreshLogSettings,hence don't expect to see the changed conf values - Assert.assertTrue(2000l != deletionSvc.getCheckIntervalMsecs()); - - //refresh the log settings - deletionSvc.refreshLogRetentionSettings(); - - //Check interval time should reflect the new value - Assert.assertTrue(2000l == deletionSvc.getCheckIntervalMsecs()); - //app2Dir should be deleted since it falls above the threshold - verify(mockFs, timeout(10000)).delete(app2Dir, true); - deletionSvc.stop(); + testcase + .startDeletionService() + //app1Dir would be deleted since it is done above log retention period + .verifyAppDirDeleted(1, 10000L) + //app2Dir is not expected to be deleted since it is below the threshold + .verifyAppDirNotDeleted(2, 3000L); + + //Now, let's change the log aggregation retention configs + conf.setInt(YarnConfiguration.LOG_AGGREGATION_RETAIN_SECONDS, 50); + conf.setInt(YarnConfiguration.LOG_AGGREGATION_RETAIN_CHECK_INTERVAL_SECONDS, + checkIntervalSeconds); + + testcase + //We have not called refreshLogSettings, hence don't expect to see + // the changed conf values + .verifyCheckIntervalMilliSecondsNotEqualTo(checkIntervalMilliSeconds) + //refresh the log settings + .refreshLogRetentionSettings() + //Check interval time should reflect the new value + .verifyCheckIntervalMilliSecondsEqualTo(checkIntervalMilliSeconds) + //app2Dir should be deleted since it falls above the threshold + .verifyAppDirDeleted(2, 10000L) + //Close expected 2 times: once for refresh and once for stopping + .teardown(2); } @Test public void testCheckInterval() throws Exception { - long RETENTION_SECS = 10 * 24 * 3600; long now = System.currentTimeMillis(); - long toDeleteTime = now - RETENTION_SECS*1000; - - String root = "mockfs://foo/"; - String remoteRootLogDir = root+"tmp/logs"; - String suffix = "logs"; - String newSuffix = LogAggregationUtils.getBucketSuffix() + suffix; - Configuration conf = new Configuration(); - conf.setClass("fs.mockfs.impl", MockFileSystem.class, FileSystem.class); - conf.set(YarnConfiguration.LOG_AGGREGATION_ENABLED, "true"); - conf.set(YarnConfiguration.LOG_AGGREGATION_RETAIN_SECONDS, "864000"); - conf.set(YarnConfiguration.LOG_AGGREGATION_RETAIN_CHECK_INTERVAL_SECONDS, "1"); - conf.set(YarnConfiguration.NM_REMOTE_APP_LOG_DIR, remoteRootLogDir); - conf.set(YarnConfiguration.NM_REMOTE_APP_LOG_DIR_SUFFIX, suffix); - conf.set(YarnConfiguration.LOG_AGGREGATION_FILE_FORMATS, "TFile"); - conf.set(String.format(LOG_AGGREGATION_FILE_CONTROLLER_FMT, "TFile"), - LogAggregationTFileController.class.getName()); + long toDeleteTime = now - TEN_DAYS_IN_SECONDS * 1000; + Configuration conf = setupConfiguration(TEN_DAYS_IN_SECONDS, 1); // prevent us from picking up the same mockfs instance from another test FileSystem.closeAll(); - Path rootPath = new Path(root); - FileSystem rootFs = rootPath.getFileSystem(conf); - FileSystem mockFs = ((FilterFileSystem)rootFs).getRawFileSystem(); - - Path remoteRootLogPath = new Path(remoteRootLogDir); - - Path userDir = new Path(remoteRootLogPath, "me"); - FileStatus userDirStatus = new FileStatus(0, true, 0, 0, now, userDir); - - when(mockFs.listStatus(remoteRootLogPath)).thenReturn( - new FileStatus[]{userDirStatus}); - - ApplicationId appId1 = - ApplicationId.newInstance(System.currentTimeMillis(), 1); - Path suffixDir = new Path(userDir, newSuffix); - FileStatus suffixDirStatus = new FileStatus(0, true, 0, 0, now, - suffixDir); - Path bucketDir = LogAggregationUtils.getRemoteBucketDir( - remoteRootLogPath, "me", suffix, appId1); - Path app1Dir = LogAggregationUtils.getRemoteAppLogDir( - remoteRootLogPath, appId1, "me", suffix); - FileStatus bucketDirStatus = new FileStatus(0, true, 0, - 0, now, bucketDir); - - FileStatus app1DirStatus = new FileStatus(0, true, 0, 0, now, app1Dir); - - when(mockFs.listStatus(userDir)).thenReturn( - new FileStatus[] {suffixDirStatus}); - when(mockFs.listStatus(suffixDir)).thenReturn( - new FileStatus[] {bucketDirStatus}); - when(mockFs.listStatus(bucketDir)).thenReturn( - new FileStatus[] {app1DirStatus}); - Path app1Log1 = new Path(app1Dir, "host1"); - FileStatus app1Log1Status = new FileStatus(10, false, 1, 1, now, app1Log1); - - when(mockFs.listStatus(app1Dir)).thenReturn( - new FileStatus[]{app1Log1Status}); - - final List finishedApplications = - Collections.unmodifiableList(Arrays.asList(appId1)); - - AggregatedLogDeletionService deletionSvc = - new AggregatedLogDeletionService() { - @Override - protected ApplicationClientProtocol createRMClient() - throws IOException { - try { - return createMockRMClient(finishedApplications, null); - } catch (Exception e) { - throw new IOException(e); - } - } - @Override - protected void stopRMClient() { - // DO NOTHING - } - }; - deletionSvc.init(conf); - deletionSvc.start(); - - verify(mockFs, timeout(10000).atLeast(4)).listStatus(any(Path.class)); - verify(mockFs, never()).delete(app1Dir, true); - - // modify the timestamp of the logs and verify it's picked up quickly - bucketDirStatus = new FileStatus(0, true, 0, 0, toDeleteTime, bucketDir); - app1DirStatus = new FileStatus(0, true, 0, 0, toDeleteTime, app1Dir); - app1Log1Status = new FileStatus(10, false, 1, 1, toDeleteTime, app1Log1); - when(mockFs.listStatus(userDir)).thenReturn( - new FileStatus[] {suffixDirStatus}); - when(mockFs.listStatus(suffixDir)).thenReturn( - new FileStatus[] {bucketDirStatus }); - when(mockFs.listStatus(bucketDir)).thenReturn( - new FileStatus[] {app1DirStatus }); - when(mockFs.listStatus(app1Dir)).thenReturn( - new FileStatus[]{app1Log1Status}); - - verify(mockFs, timeout(10000)).delete(app1Dir, true); - - deletionSvc.stop(); + LogAggregationTestcaseBuilder.create(conf) + .withRootPath(ROOT) + .withRemoteRootLogPath(REMOTE_ROOT_LOG_DIR) + .withUserDir(USER_ME, now) + .withSuffixDir(SUFFIX, now) + .withBucketDir(now) + .withApps(Lists.newArrayList( + new AppDescriptor(now, + Lists.newArrayList(Pair.of(DIR_HOST1, now))), + new AppDescriptor(now))) + .withFinishedApps(1) + .withRunningApps() + .build() + .startDeletionService() + .verifyAnyPathListedAtLeast(4, 10000L) + .verifyAppDirNotDeleted(1, NO_TIMEOUT) + // modify the timestamp of the logs and verify if it is picked up quickly + .changeModTimeOfApp(1, toDeleteTime) + .changeModTimeOfAppLogDir(1, 1, toDeleteTime) + .changeModTimeOfBucketDir(toDeleteTime) + .reinitAllPaths() + .verifyAppDirDeleted(1, 10000L) + .teardown(1); } @Test public void testRobustLogDeletion() throws Exception { - final long RETENTION_SECS = 10 * 24 * 3600; - - String root = "mockfs://foo/"; - String remoteRootLogDir = root+"tmp/logs"; - String suffix = "logs"; - String newSuffix = LogAggregationUtils.getBucketSuffix() + suffix; - Configuration conf = new Configuration(); - conf.setClass("fs.mockfs.impl", MockFileSystem.class, - FileSystem.class); - conf.set(YarnConfiguration.LOG_AGGREGATION_ENABLED, "true"); - conf.set(YarnConfiguration.LOG_AGGREGATION_RETAIN_SECONDS, "864000"); - conf.set(YarnConfiguration.LOG_AGGREGATION_RETAIN_CHECK_INTERVAL_SECONDS, - "1"); - conf.set(YarnConfiguration.NM_REMOTE_APP_LOG_DIR, remoteRootLogDir); - conf.set(YarnConfiguration.NM_REMOTE_APP_LOG_DIR_SUFFIX, suffix); - conf.set(YarnConfiguration.LOG_AGGREGATION_FILE_FORMATS, "TFile"); - conf.set(String.format(LOG_AGGREGATION_FILE_CONTROLLER_FMT, "TFile"), - LogAggregationTFileController.class.getName()); + Configuration conf = setupConfiguration(TEN_DAYS_IN_SECONDS, 1); // prevent us from picking up the same mockfs instance from another test FileSystem.closeAll(); - Path rootPath = new Path(root); - FileSystem rootFs = rootPath.getFileSystem(conf); - FileSystem mockFs = ((FilterFileSystem)rootFs).getRawFileSystem(); - - Path remoteRootLogPath = new Path(remoteRootLogDir); - - Path userDir = new Path(remoteRootLogPath, "me"); - Path suffixDir = new Path(userDir, newSuffix); - FileStatus userDirStatus = new FileStatus(0, true, 0, 0, 0, userDir); - FileStatus suffixStatus = new FileStatus(0, true, 0, 0, 0, suffixDir); - Path bucketDir = new Path(suffixDir, String.valueOf(0)); - FileStatus bucketDirStatus = new FileStatus(0, true, 0, 0, 0, bucketDir); - - when(mockFs.listStatus(remoteRootLogPath)).thenReturn( - new FileStatus[]{userDirStatus}); - when(mockFs.listStatus(userDir)).thenReturn( - new FileStatus[]{suffixStatus}); - when(mockFs.listStatus(suffixDir)).thenReturn( - new FileStatus[]{bucketDirStatus}); - - ApplicationId appId1 = - ApplicationId.newInstance(System.currentTimeMillis(), 1); - Path app1Dir = new Path(bucketDir, appId1.toString()); - FileStatus app1DirStatus = new FileStatus(0, true, 0, 0, 0, app1Dir); - ApplicationId appId2 = - ApplicationId.newInstance(System.currentTimeMillis(), 2); - Path app2Dir = new Path(bucketDir, "application_a"); - FileStatus app2DirStatus = new FileStatus(0, true, 0, 0, 0, app2Dir); - ApplicationId appId3 = - ApplicationId.newInstance(System.currentTimeMillis(), 3); - Path app3Dir = new Path(bucketDir, appId3.toString()); - FileStatus app3DirStatus = new FileStatus(0, true, 0, 0, 0, app3Dir); - - when(mockFs.listStatus(bucketDir)).thenReturn( - new FileStatus[]{app1DirStatus, app2DirStatus, app3DirStatus}); - when(mockFs.listStatus(app2Dir)).thenReturn( - new FileStatus[]{}); - - when(mockFs.listStatus(app1Dir)).thenThrow( - new RuntimeException("Should Be Caught and Logged")); - Path app3Log3 = new Path(app3Dir, "host1"); - FileStatus app3Log3Status = new FileStatus(10, false, 1, 1, 0, app3Log3); - when(mockFs.listStatus(app3Dir)).thenReturn( - new FileStatus[]{app3Log3Status}); - - final List finishedApplications = - Collections.unmodifiableList(Arrays.asList(appId1, appId3)); + long modTime = 0L; + + LogAggregationTestcaseBuilder.create(conf) + .withRootPath(ROOT) + .withRemoteRootLogPath(REMOTE_ROOT_LOG_DIR) + .withUserDir(USER_ME, modTime) + .withSuffixDir(SUFFIX, modTime) + .withBucketDir(modTime, "0") + .withApps(Lists.newArrayList( + new AppDescriptor(modTime), + new AppDescriptor(modTime), + new AppDescriptor(modTime, Lists.newArrayList(Pair.of(DIR_HOST1, modTime))))) + .withAdditionalAppDirs(Lists.newArrayList(Pair.of("application_a", modTime))) + .withFinishedApps(1, 3) + .withRunningApps() + .injectExceptionForAppDirDeletion(1) + .build() + .runDeletionTask(TEN_DAYS_IN_SECONDS) + .verifyAppDirDeleted(3, NO_TIMEOUT); + } - ApplicationClientProtocol rmClient = - createMockRMClient(finishedApplications, null); - AggregatedLogDeletionService.LogDeletionTask deletionTask = - new AggregatedLogDeletionService.LogDeletionTask(conf, - RETENTION_SECS, - rmClient); - deletionTask.run(); - verify(mockFs).delete(app3Dir, true); + @Test + public void testDeletionTwoControllers() throws IOException { + long now = System.currentTimeMillis(); + long toDeleteTime = now - (2000 * 1000); + long toKeepTime = now - (1500 * 1000); + + + Configuration conf = setupConfiguration(1800, -1); + enableFileControllers(conf, REMOTE_ROOT_LOG_DIR, ALL_FILE_CONTROLLERS, + ALL_FILE_CONTROLLER_NAMES); + long timeout = 2000L; + LogAggregationTestcaseBuilder.create(conf) + .withRootPath(ROOT) + .withRemoteRootLogPath(REMOTE_ROOT_LOG_DIR) + .withBothFileControllers() + .withUserDir(USER_ME, toKeepTime) + .withSuffixDir(SUFFIX, toDeleteTime) + .withBucketDir(toDeleteTime) + .withApps(//Apps for TFile + Lists.newArrayList( + new AppDescriptor(T_FILE, toDeleteTime, Lists.newArrayList()), + new AppDescriptor(T_FILE, toDeleteTime, Lists.newArrayList( + Pair.of(DIR_HOST1, toDeleteTime), + Pair.of(DIR_HOST2, toKeepTime))), + new AppDescriptor(T_FILE, toDeleteTime, Lists.newArrayList( + Pair.of(DIR_HOST1, toDeleteTime), + Pair.of(DIR_HOST2, toDeleteTime))), + new AppDescriptor(T_FILE, toDeleteTime, Lists.newArrayList( + Pair.of(DIR_HOST1, toDeleteTime), + Pair.of(DIR_HOST2, toKeepTime))), + //Apps for IFile + new AppDescriptor(I_FILE, toDeleteTime, Lists.newArrayList()), + new AppDescriptor(I_FILE, toDeleteTime, Lists.newArrayList( + Pair.of(DIR_HOST1, toDeleteTime), + Pair.of(DIR_HOST2, toKeepTime))), + new AppDescriptor(I_FILE, toDeleteTime, Lists.newArrayList( + Pair.of(DIR_HOST1, toDeleteTime), + Pair.of(DIR_HOST2, toDeleteTime))), + new AppDescriptor(I_FILE, toDeleteTime, Lists.newArrayList( + Pair.of(DIR_HOST1, toDeleteTime), + Pair.of(DIR_HOST2, toKeepTime))))) + .withFinishedApps(1, 2, 3, 5, 6, 7) + .withRunningApps(4, 8) + .injectExceptionForAppDirDeletion(3, 6) + .build() + .startDeletionService() + .verifyAppDirsDeleted(timeout, 1, 3, 5, 7) + .verifyAppDirsNotDeleted(timeout, 2, 4, 6, 8) + .verifyAppFilesDeleted(timeout, Lists.newArrayList(Pair.of(4, 1), Pair.of(8, 1))) + .verifyAppFilesNotDeleted(timeout, Lists.newArrayList(Pair.of(4, 2), Pair.of(8, 2))) + .teardown(1); } static class MockFileSystem extends FilterFileSystem { MockFileSystem() { super(mock(FileSystem.class)); } + public void initialize(URI name, Configuration conf) throws IOException {} - } - private static ApplicationClientProtocol createMockRMClient( - List finishedApplicaitons, - List runningApplications) throws Exception { - final ApplicationClientProtocol mockProtocol = - mock(ApplicationClientProtocol.class); - if (finishedApplicaitons != null && !finishedApplicaitons.isEmpty()) { - for (ApplicationId appId : finishedApplicaitons) { - GetApplicationReportRequest request = - GetApplicationReportRequest.newInstance(appId); - GetApplicationReportResponse response = - createApplicationReportWithFinishedApplication(); - when(mockProtocol.getApplicationReport(request)) - .thenReturn(response); - } - } - if (runningApplications != null && !runningApplications.isEmpty()) { - for (ApplicationId appId : runningApplications) { - GetApplicationReportRequest request = - GetApplicationReportRequest.newInstance(appId); - GetApplicationReportResponse response = - createApplicationReportWithRunningApplication(); - when(mockProtocol.getApplicationReport(request)) - .thenReturn(response); - } + @Override + public boolean hasPathCapability(Path path, String capability) { + return true; } - return mockProtocol; - } - - private static GetApplicationReportResponse - createApplicationReportWithRunningApplication() { - ApplicationReport report = mock(ApplicationReport.class); - when(report.getYarnApplicationState()).thenReturn( - YarnApplicationState.RUNNING); - GetApplicationReportResponse response = - mock(GetApplicationReportResponse.class); - when(response.getApplicationReport()).thenReturn(report); - return response; - } - - private static GetApplicationReportResponse - createApplicationReportWithFinishedApplication() { - ApplicationReport report = mock(ApplicationReport.class); - when(report.getYarnApplicationState()).thenReturn( - YarnApplicationState.FINISHED); - GetApplicationReportResponse response = - mock(GetApplicationReportResponse.class); - when(response.getApplicationReport()).thenReturn(report); - return response; } } diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/test/java/org/apache/hadoop/yarn/logaggregation/filecontroller/TestLogAggregationFileControllerFactory.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/test/java/org/apache/hadoop/yarn/logaggregation/filecontroller/TestLogAggregationFileControllerFactory.java index 2d2fb49c0efc9..c1b991b9bc1b5 100644 --- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/test/java/org/apache/hadoop/yarn/logaggregation/filecontroller/TestLogAggregationFileControllerFactory.java +++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/test/java/org/apache/hadoop/yarn/logaggregation/filecontroller/TestLogAggregationFileControllerFactory.java @@ -18,24 +18,6 @@ package org.apache.hadoop.yarn.logaggregation.filecontroller; -import static org.apache.hadoop.yarn.conf.YarnConfiguration.LOG_AGGREGATION_FILE_CONTROLLER_FMT; -import static org.apache.hadoop.yarn.conf.YarnConfiguration.LOG_AGGREGATION_REMOTE_APP_LOG_DIR_FMT; -import static org.apache.hadoop.yarn.conf.YarnConfiguration.LOG_AGGREGATION_REMOTE_APP_LOG_DIR_SUFFIX_FMT; -import static org.junit.Assert.assertEquals; -import static org.junit.Assert.assertTrue; -import static org.junit.Assert.fail; - -import java.io.File; -import java.io.FileWriter; -import java.io.IOException; -import java.io.OutputStream; -import java.io.Writer; -import java.util.Arrays; -import java.util.Collections; -import java.util.List; -import java.util.Map; - -import org.apache.commons.lang3.StringUtils; import org.apache.hadoop.conf.Configuration; import org.apache.hadoop.conf.Configured; import org.apache.hadoop.fs.FileSystem; @@ -56,6 +38,21 @@ import org.slf4j.Logger; import org.slf4j.LoggerFactory; +import java.io.File; +import java.io.FileWriter; +import java.io.IOException; +import java.io.OutputStream; +import java.io.Writer; +import java.util.Arrays; +import java.util.Collections; +import java.util.List; +import java.util.Map; + +import static org.apache.hadoop.yarn.conf.YarnConfiguration.LOG_AGGREGATION_FILE_FORMATS; +import static org.apache.hadoop.yarn.logaggregation.LogAggregationTestUtils.REMOTE_LOG_ROOT; +import static org.apache.hadoop.yarn.logaggregation.LogAggregationTestUtils.enableFileControllers; +import static org.junit.Assert.*; + /** * Test LogAggregationFileControllerFactory. */ @@ -63,7 +60,6 @@ public class TestLogAggregationFileControllerFactory extends Configured { private static final Logger LOG = LoggerFactory.getLogger( TestLogAggregationFileControllerFactory.class); - private static final String REMOTE_LOG_ROOT = "target/app-logs/"; private static final String REMOTE_DEFAULT_DIR = "default/"; private static final String APP_OWNER = "test"; @@ -87,8 +83,7 @@ public class TestLogAggregationFileControllerFactory extends Configured { public void setup() throws IOException { Configuration conf = new YarnConfiguration(); conf.setBoolean(YarnConfiguration.LOG_AGGREGATION_ENABLED, true); - conf.set(YarnConfiguration.NM_REMOTE_APP_LOG_DIR, REMOTE_LOG_ROOT + - REMOTE_DEFAULT_DIR); + conf.set(YarnConfiguration.NM_REMOTE_APP_LOG_DIR, REMOTE_LOG_ROOT + REMOTE_DEFAULT_DIR); conf.set(YarnConfiguration.NM_REMOTE_APP_LOG_DIR_SUFFIX, "log"); setConf(conf); } @@ -143,36 +138,15 @@ public void testDefaultLogAggregationFileControllerFactory() @Test(expected = Exception.class) public void testLogAggregationFileControllerFactoryClassNotSet() { Configuration conf = getConf(); - conf.set(YarnConfiguration.LOG_AGGREGATION_FILE_FORMATS, - "TestLogAggregationFileController"); + conf.set(LOG_AGGREGATION_FILE_FORMATS, "TestLogAggregationFileController"); new LogAggregationFileControllerFactory(conf); fail("TestLogAggregationFileController's class was not set, " + "but the factory creation did not fail."); } - private void enableFileControllers( - List> fileControllers, - List fileControllerNames) { - Configuration conf = getConf(); - conf.set(YarnConfiguration.LOG_AGGREGATION_FILE_FORMATS, - StringUtils.join(fileControllerNames, ",")); - for (int i = 0; i < fileControllers.size(); i++) { - Class fileController = - fileControllers.get(i); - String controllerName = fileControllerNames.get(i); - - conf.setClass(String.format(LOG_AGGREGATION_FILE_CONTROLLER_FMT, - controllerName), fileController, LogAggregationFileController.class); - conf.set(String.format(LOG_AGGREGATION_REMOTE_APP_LOG_DIR_FMT, - controllerName), REMOTE_LOG_ROOT + controllerName + "/"); - conf.set(String.format(LOG_AGGREGATION_REMOTE_APP_LOG_DIR_SUFFIX_FMT, - controllerName), controllerName); - } - } - @Test public void testLogAggregationFileControllerFactory() throws Exception { - enableFileControllers(ALL_FILE_CONTROLLERS, ALL_FILE_CONTROLLER_NAMES); + enableFileControllers(getConf(), ALL_FILE_CONTROLLERS, ALL_FILE_CONTROLLER_NAMES); LogAggregationFileControllerFactory factory = new LogAggregationFileControllerFactory(getConf()); List list = @@ -199,8 +173,7 @@ public void testLogAggregationFileControllerFactory() throws Exception { @Test public void testClassConfUsed() { - enableFileControllers(Collections.singletonList( - LogAggregationTFileController.class), + enableFileControllers(getConf(), Collections.singletonList(LogAggregationTFileController.class), Collections.singletonList("TFile")); LogAggregationFileControllerFactory factory = new LogAggregationFileControllerFactory(getConf()); @@ -215,7 +188,7 @@ public void testClassConfUsed() { @Test public void testNodemanagerConfigurationIsUsed() { Configuration conf = getConf(); - conf.set(YarnConfiguration.LOG_AGGREGATION_FILE_FORMATS, "TFile"); + conf.set(LOG_AGGREGATION_FILE_FORMATS, "TFile"); LogAggregationFileControllerFactory factory = new LogAggregationFileControllerFactory(conf); LogAggregationFileController fc = factory.getFileControllerForWrite(); @@ -231,7 +204,7 @@ public void testDefaultConfUsed() { Configuration conf = getConf(); conf.unset(YarnConfiguration.NM_REMOTE_APP_LOG_DIR); conf.unset(YarnConfiguration.NM_REMOTE_APP_LOG_DIR_SUFFIX); - conf.set(YarnConfiguration.LOG_AGGREGATION_FILE_FORMATS, "TFile"); + conf.set(LOG_AGGREGATION_FILE_FORMATS, "TFile"); LogAggregationFileControllerFactory factory = new LogAggregationFileControllerFactory(getConf()); @@ -268,20 +241,19 @@ public void postWrite(LogAggregationFileControllerContext record) } @Override - public void initializeWriter(LogAggregationFileControllerContext context) - throws IOException { + public void initializeWriter(LogAggregationFileControllerContext context) { // Do Nothing } @Override public boolean readAggregatedLogs(ContainerLogsRequest logRequest, - OutputStream os) throws IOException { + OutputStream os) { return false; } @Override public List readAggregatedLogsMeta( - ContainerLogsRequest logRequest) throws IOException { + ContainerLogsRequest logRequest) { return null; } diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/test/java/org/apache/hadoop/yarn/logaggregation/testutils/AggregatedLogDeletionServiceForTest.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/test/java/org/apache/hadoop/yarn/logaggregation/testutils/AggregatedLogDeletionServiceForTest.java new file mode 100644 index 0000000000000..76ec8aab537a5 --- /dev/null +++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/test/java/org/apache/hadoop/yarn/logaggregation/testutils/AggregatedLogDeletionServiceForTest.java @@ -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 + * + * 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.yarn.logaggregation.testutils; + +import org.apache.hadoop.conf.Configuration; +import org.apache.hadoop.yarn.api.ApplicationClientProtocol; +import org.apache.hadoop.yarn.api.records.ApplicationId; +import org.apache.hadoop.yarn.logaggregation.AggregatedLogDeletionService; + +import java.io.IOException; +import java.util.List; + +import static org.apache.hadoop.yarn.logaggregation.testutils.MockRMClientUtils.createMockRMClient; + +public class AggregatedLogDeletionServiceForTest extends AggregatedLogDeletionService { + private final List finishedApplications; + private final List runningApplications; + private final Configuration conf; + private ApplicationClientProtocol mockRMClient; + + public AggregatedLogDeletionServiceForTest(List runningApplications, + List finishedApplications) { + this(runningApplications, finishedApplications, null); + } + + public AggregatedLogDeletionServiceForTest(List runningApplications, + List finishedApplications, + Configuration conf) { + this.runningApplications = runningApplications; + this.finishedApplications = finishedApplications; + this.conf = conf; + } + + @Override + protected ApplicationClientProtocol createRMClient() throws IOException { + if (mockRMClient != null) { + return mockRMClient; + } + try { + mockRMClient = + createMockRMClient(finishedApplications, runningApplications); + } catch (Exception e) { + throw new IOException(e); + } + return mockRMClient; + } + + @Override + protected Configuration createConf() { + return conf; + } + + public ApplicationClientProtocol getMockRMClient() { + return mockRMClient; + } +} diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/test/java/org/apache/hadoop/yarn/logaggregation/testutils/FileStatusUtils.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/test/java/org/apache/hadoop/yarn/logaggregation/testutils/FileStatusUtils.java new file mode 100644 index 0000000000000..c4af8618614a1 --- /dev/null +++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/test/java/org/apache/hadoop/yarn/logaggregation/testutils/FileStatusUtils.java @@ -0,0 +1,76 @@ +/** + * 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.yarn.logaggregation.testutils; + +import org.apache.hadoop.fs.FileStatus; +import org.apache.hadoop.fs.Path; +import org.apache.hadoop.yarn.api.records.ApplicationId; +import org.apache.hadoop.yarn.logaggregation.LogAggregationUtils; + +public class FileStatusUtils { + public static PathWithFileStatus createPathWithFileStatusForAppId(Path remoteRootLogDir, + ApplicationId appId, + String user, String suffix, + long modificationTime) { + Path path = LogAggregationUtils.getRemoteAppLogDir( + remoteRootLogDir, appId, user, suffix); + FileStatus fileStatus = createEmptyFileStatus(modificationTime, path); + return new PathWithFileStatus(path, fileStatus); + } + + public static FileStatus createEmptyFileStatus(long modificationTime, Path path) { + return new FileStatus(0, true, 0, 0, modificationTime, path); + } + + public static PathWithFileStatus createFileLogPathWithFileStatus(Path baseDir, String childDir, + long modificationTime) { + Path logPath = new Path(baseDir, childDir); + FileStatus fStatus = createFileStatusWithLengthForFile(10, modificationTime, logPath); + return new PathWithFileStatus(logPath, fStatus); + } + + public static PathWithFileStatus createDirLogPathWithFileStatus(Path baseDir, String childDir, + long modificationTime) { + Path logPath = new Path(baseDir, childDir); + FileStatus fStatus = createFileStatusWithLengthForDir(10, modificationTime, logPath); + return new PathWithFileStatus(logPath, fStatus); + } + + public static PathWithFileStatus createDirBucketDirLogPathWithFileStatus(Path remoteRootLogPath, + String user, + String suffix, + ApplicationId appId, + long modificationTime) { + Path bucketDir = LogAggregationUtils.getRemoteBucketDir(remoteRootLogPath, user, suffix, appId); + FileStatus fStatus = new FileStatus(0, true, 0, 0, modificationTime, bucketDir); + return new PathWithFileStatus(bucketDir, fStatus); + } + + public static FileStatus createFileStatusWithLengthForFile(long length, + long modificationTime, + Path logPath) { + return new FileStatus(length, false, 1, 1, modificationTime, logPath); + } + + public static FileStatus createFileStatusWithLengthForDir(long length, + long modificationTime, + Path logPath) { + return new FileStatus(length, true, 1, 1, modificationTime, logPath); + } +} diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/test/java/org/apache/hadoop/yarn/logaggregation/testutils/LogAggregationTestcase.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/test/java/org/apache/hadoop/yarn/logaggregation/testutils/LogAggregationTestcase.java new file mode 100644 index 0000000000000..f2074f8c8e688 --- /dev/null +++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/test/java/org/apache/hadoop/yarn/logaggregation/testutils/LogAggregationTestcase.java @@ -0,0 +1,444 @@ +/** + * 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.yarn.logaggregation.testutils; + +import org.apache.commons.lang3.tuple.Pair; +import org.apache.hadoop.conf.Configuration; +import org.apache.hadoop.fs.FileStatus; +import org.apache.hadoop.fs.FileSystem; +import org.apache.hadoop.fs.FilterFileSystem; +import org.apache.hadoop.fs.Path; +import org.apache.hadoop.util.Sets; +import org.apache.hadoop.yarn.api.ApplicationClientProtocol; +import org.apache.hadoop.yarn.api.records.ApplicationId; +import org.apache.hadoop.yarn.logaggregation.AggregatedLogDeletionService; +import org.apache.hadoop.yarn.logaggregation.AggregatedLogDeletionService.LogDeletionTask; +import org.apache.hadoop.yarn.logaggregation.testutils.LogAggregationTestcaseBuilder.AppDescriptor; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + +import java.io.Closeable; +import java.io.IOException; +import java.util.ArrayList; +import java.util.HashSet; +import java.util.List; +import java.util.Map; +import java.util.Objects; +import java.util.Set; +import java.util.stream.Collectors; + +import static org.apache.hadoop.yarn.logaggregation.testutils.FileStatusUtils.*; +import static org.apache.hadoop.yarn.logaggregation.testutils.LogAggregationTestcaseBuilder.NO_TIMEOUT; +import static org.apache.hadoop.yarn.logaggregation.testutils.MockRMClientUtils.createMockRMClient; +import static org.junit.Assert.assertEquals; +import static org.junit.Assert.assertTrue; +import static org.mockito.ArgumentMatchers.any; +import static org.mockito.Mockito.*; + +public class LogAggregationTestcase { + private static final Logger LOG = LoggerFactory.getLogger(LogAggregationTestcase.class); + + private final Configuration conf; + private final long now; + private PathWithFileStatus bucketDir; + private final long bucketDirModTime; + private PathWithFileStatus userDir; + private final String userDirName; + private final long userDirModTime; + private PathWithFileStatus suffixDir; + private final String suffix; + private final String suffixDirName; + private final long suffixDirModTime; + private final String bucketId; + private final Path remoteRootLogPath; + private final Map injectedAppDirDeletionExceptions; + private final List fileControllers; + private final List> additionalAppDirs; + + private final List applicationIds = new ArrayList<>(); + private final int[] runningAppIds; + private final int[] finishedAppIds; + private final List> appFiles = new ArrayList<>(); + private final FileSystem mockFs; + private List appDirs; + private final List appDescriptors; + private AggregatedLogDeletionServiceForTest deletionService; + private ApplicationClientProtocol rmClient; + + public LogAggregationTestcase(LogAggregationTestcaseBuilder builder) throws IOException { + conf = builder.conf; + now = builder.now; + bucketDir = builder.bucketDir; + bucketDirModTime = builder.bucketDirModTime; + userDir = builder.userDir; + userDirName = builder.userDirName; + userDirModTime = builder.userDirModTime; + suffix = builder.suffix; + suffixDir = builder.suffixDir; + suffixDirName = builder.suffixDirName; + suffixDirModTime = builder.suffixDirModTime; + bucketId = builder.bucketId; + appDescriptors = builder.apps; + runningAppIds = builder.runningAppIds; + finishedAppIds = builder.finishedAppIds; + remoteRootLogPath = builder.remoteRootLogPath; + injectedAppDirDeletionExceptions = builder.injectedAppDirDeletionExceptions; + fileControllers = builder.fileControllers; + additionalAppDirs = builder.additionalAppDirs; + + mockFs = ((FilterFileSystem) builder.rootFs).getRawFileSystem(); + validateAppControllers(); + setupMocks(); + + setupDeletionService(); + } + + private void validateAppControllers() { + Set controllers = appDescriptors.stream() + .map(a -> a.fileController) + .filter(Objects::nonNull) + .collect(Collectors.toSet()); + Set availableControllers = fileControllers != null ? + new HashSet<>(this.fileControllers) : Sets.newHashSet(); + Set difference = Sets.difference(controllers, availableControllers); + if (!difference.isEmpty()) { + throw new IllegalStateException(String.format("Invalid controller defined!" + + " Available: %s, Actual: %s", availableControllers, controllers)); + } + } + + private void setupMocks() throws IOException { + createApplicationsByDescriptors(); + + List rootPaths = determineRootPaths(); + for (Path rootPath : rootPaths) { + String controllerName = rootPath.getName(); + ApplicationId arbitraryAppIdForBucketDir = this.applicationIds.get(0); + userDir = createDirLogPathWithFileStatus(rootPath, userDirName, userDirModTime); + suffixDir = createDirLogPathWithFileStatus(userDir.path, suffixDirName, suffixDirModTime); + if (bucketId != null) { + bucketDir = createDirLogPathWithFileStatus(suffixDir.path, bucketId, bucketDirModTime); + } else { + bucketDir = createDirBucketDirLogPathWithFileStatus(rootPath, userDirName, suffix, + arbitraryAppIdForBucketDir, bucketDirModTime); + } + setupListStatusForPath(rootPath, userDir); + initFileSystemListings(controllerName); + } + } + + private List determineRootPaths() { + List rootPaths = new ArrayList<>(); + if (fileControllers != null && !fileControllers.isEmpty()) { + for (String fileController : fileControllers) { + //Generic path: //bucket-// + // / + + //remoteRootLogPath: / + //example: mockfs://foo/tmp/logs/ + + //userDir: // + //example: mockfs://foo/tmp/logs/me/ + + //suffixDir: //bucket-/ + //example: mockfs://foo/tmp/logs/me/bucket-logs/ + + //bucketDir: //bucket-// + //example: mockfs://foo/tmp/logs/me/bucket-logs/0001/ + + //remoteRootLogPath with controller: / + //example: mockfs://foo/tmp/logs/IFile + rootPaths.add(new Path(remoteRootLogPath, fileController)); + } + } else { + rootPaths.add(remoteRootLogPath); + } + return rootPaths; + } + + private void initFileSystemListings(String controllerName) throws IOException { + setupListStatusForPath(userDir, suffixDir); + setupListStatusForPath(suffixDir, bucketDir); + setupListStatusForPath(bucketDir, appDirs.stream() + .filter(app -> app.path.toString().contains(controllerName)) + .map(app -> app.fileStatus) + .toArray(FileStatus[]::new)); + + for (Pair appDirPair : additionalAppDirs) { + PathWithFileStatus appDir = createDirLogPathWithFileStatus(bucketDir.path, + appDirPair.getLeft(), appDirPair.getRight()); + setupListStatusForPath(appDir, new FileStatus[] {}); + } + } + + private void createApplicationsByDescriptors() throws IOException { + int len = appDescriptors.size(); + appDirs = new ArrayList<>(len); + + for (int i = 0; i < len; i++) { + AppDescriptor appDesc = appDescriptors.get(i); + ApplicationId applicationId = appDesc.createApplicationId(now, i + 1); + applicationIds.add(applicationId); + Path basePath = this.remoteRootLogPath; + if (appDesc.fileController != null) { + basePath = new Path(basePath, appDesc.fileController); + } + + PathWithFileStatus appDir = createPathWithFileStatusForAppId( + basePath, applicationId, userDirName, suffix, appDesc.modTimeOfAppDir); + LOG.debug("Created application with ID '{}' to path '{}'", applicationId, appDir.path); + appDirs.add(appDir); + addAppChildrenFiles(appDesc, appDir); + } + + setupFsMocksForAppsAndChildrenFiles(); + + for (Map.Entry e : injectedAppDirDeletionExceptions.entrySet()) { + when(mockFs.delete(this.appDirs.get(e.getKey()).path, true)).thenThrow(e.getValue()); + } + } + + private void setupFsMocksForAppsAndChildrenFiles() throws IOException { + for (int i = 0; i < appDirs.size(); i++) { + List appChildren = appFiles.get(i); + Path appPath = appDirs.get(i).path; + setupListStatusForPath(appPath, + appChildren.stream() + .map(child -> child.fileStatus) + .toArray(FileStatus[]::new)); + } + } + + private void setupListStatusForPath(Path dir, PathWithFileStatus pathWithFileStatus) + throws IOException { + setupListStatusForPath(dir, new FileStatus[]{pathWithFileStatus.fileStatus}); + } + + private void setupListStatusForPath(PathWithFileStatus dir, PathWithFileStatus pathWithFileStatus) + throws IOException { + setupListStatusForPath(dir, new FileStatus[]{pathWithFileStatus.fileStatus}); + } + + private void setupListStatusForPath(Path dir, FileStatus[] fileStatuses) throws IOException { + LOG.debug("Setting up listStatus. Parent: {}, files: {}", dir, fileStatuses); + when(mockFs.listStatus(dir)).thenReturn(fileStatuses); + } + + private void setupListStatusForPath(PathWithFileStatus dir, FileStatus[] fileStatuses) + throws IOException { + LOG.debug("Setting up listStatus. Parent: {}, files: {}", dir.path, fileStatuses); + when(mockFs.listStatus(dir.path)).thenReturn(fileStatuses); + } + + private void setupDeletionService() { + List finishedApps = createFinishedAppsList(); + List runningApps = createRunningAppsList(); + deletionService = new AggregatedLogDeletionServiceForTest(runningApps, finishedApps, conf); + } + + public LogAggregationTestcase startDeletionService() { + deletionService.init(conf); + deletionService.start(); + return this; + } + + private List createRunningAppsList() { + List runningApps = new ArrayList<>(); + for (int i : runningAppIds) { + ApplicationId appId = this.applicationIds.get(i - 1); + runningApps.add(appId); + } + return runningApps; + } + + private List createFinishedAppsList() { + List finishedApps = new ArrayList<>(); + for (int i : finishedAppIds) { + ApplicationId appId = this.applicationIds.get(i - 1); + finishedApps.add(appId); + } + return finishedApps; + } + + public LogAggregationTestcase runDeletionTask(long retentionSeconds) throws Exception { + List finishedApps = createFinishedAppsList(); + List runningApps = createRunningAppsList(); + rmClient = createMockRMClient(finishedApps, runningApps); + List tasks = deletionService.createLogDeletionTasks(conf, retentionSeconds, + rmClient); + for (LogDeletionTask deletionTask : tasks) { + deletionTask.run(); + } + + return this; + } + + private void addAppChildrenFiles(AppDescriptor appDesc, PathWithFileStatus appDir) { + List appChildren = new ArrayList<>(); + for (Pair fileWithModDate : appDesc.filesWithModDate) { + PathWithFileStatus appChildFile = createFileLogPathWithFileStatus(appDir.path, + fileWithModDate.getLeft(), + fileWithModDate.getRight()); + appChildren.add(appChildFile); + } + this.appFiles.add(appChildren); + } + + public LogAggregationTestcase verifyAppDirsDeleted(long timeout, int... ids) throws IOException { + for (int id : ids) { + verifyAppDirDeleted(id, timeout); + } + return this; + } + + public LogAggregationTestcase verifyAppDirsNotDeleted(long timeout, int... ids) + throws IOException { + for (int id : ids) { + verifyAppDirNotDeleted(id, timeout); + } + return this; + } + + public LogAggregationTestcase verifyAppDirDeleted(int id, long timeout) throws IOException { + verifyAppDirDeletion(id, 1, timeout); + return this; + } + + public LogAggregationTestcase verifyAppDirNotDeleted(int id, long timeout) throws IOException { + verifyAppDirDeletion(id, 0, timeout); + return this; + } + + public LogAggregationTestcase verifyAppFilesDeleted(long timeout, + List> pairs) + throws IOException { + for (Pair pair : pairs) { + verifyAppFileDeleted(pair.getLeft(), pair.getRight(), timeout); + } + return this; + } + + public LogAggregationTestcase verifyAppFilesNotDeleted(long timeout, + List> pairs) + throws IOException { + for (Pair pair : pairs) { + verifyAppFileNotDeleted(pair.getLeft(), pair.getRight(), timeout); + } + return this; + } + + public LogAggregationTestcase verifyAppFileDeleted(int id, int fileNo, long timeout) + throws IOException { + verifyAppFileDeletion(id, fileNo, 1, timeout); + return this; + } + + public LogAggregationTestcase verifyAppFileNotDeleted(int id, int fileNo, long timeout) + throws IOException { + verifyAppFileDeletion(id, fileNo, 0, timeout); + return this; + } + + private void verifyAppDirDeletion(int id, int times, long timeout) throws IOException { + if (timeout == NO_TIMEOUT) { + verify(mockFs, times(times)).delete(this.appDirs.get(id - 1).path, true); + } else { + verify(mockFs, timeout(timeout).times(times)).delete(this.appDirs.get(id - 1).path, true); + } + } + + private void verifyAppFileDeletion(int appId, int fileNo, int times, long timeout) + throws IOException { + List childrenFiles = this.appFiles.get(appId - 1); + PathWithFileStatus file = childrenFiles.get(fileNo - 1); + verify(mockFs, timeout(timeout).times(times)).delete(file.path, true); + } + + private void verifyMockRmClientWasClosedNTimes(int expectedRmClientCloses) + throws IOException { + ApplicationClientProtocol mockRMClient; + if (deletionService != null) { + mockRMClient = deletionService.getMockRMClient(); + } else { + mockRMClient = rmClient; + } + verify((Closeable)mockRMClient, times(expectedRmClientCloses)).close(); + } + + public void teardown(int expectedRmClientCloses) throws IOException { + deletionService.stop(); + verifyMockRmClientWasClosedNTimes(expectedRmClientCloses); + } + + public LogAggregationTestcase refreshLogRetentionSettings() throws IOException { + deletionService.refreshLogRetentionSettings(); + return this; + } + + public AggregatedLogDeletionService getDeletionService() { + return deletionService; + } + + public LogAggregationTestcase verifyCheckIntervalMilliSecondsEqualTo( + int checkIntervalMilliSeconds) { + assertEquals(checkIntervalMilliSeconds, deletionService.getCheckIntervalMsecs()); + return this; + } + + public LogAggregationTestcase verifyCheckIntervalMilliSecondsNotEqualTo( + int checkIntervalMilliSeconds) { + assertTrue(checkIntervalMilliSeconds != deletionService.getCheckIntervalMsecs()); + return this; + } + + public LogAggregationTestcase verifyAnyPathListedAtLeast(int atLeast, long timeout) + throws IOException { + verify(mockFs, timeout(timeout).atLeast(atLeast)).listStatus(any(Path.class)); + return this; + } + + public LogAggregationTestcase changeModTimeOfApp(int appId, long modTime) { + PathWithFileStatus appDir = appDirs.get(appId - 1); + appDir.changeModificationTime(modTime); + return this; + } + + public LogAggregationTestcase changeModTimeOfAppLogDir(int appId, int fileNo, long modTime) { + List childrenFiles = this.appFiles.get(appId - 1); + PathWithFileStatus file = childrenFiles.get(fileNo - 1); + file.changeModificationTime(modTime); + return this; + } + + public LogAggregationTestcase changeModTimeOfBucketDir(long modTime) { + bucketDir.changeModificationTime(modTime); + return this; + } + + public LogAggregationTestcase reinitAllPaths() throws IOException { + List rootPaths = determineRootPaths(); + for (Path rootPath : rootPaths) { + String controllerName = rootPath.getName(); + initFileSystemListings(controllerName); + } + setupFsMocksForAppsAndChildrenFiles(); + return this; + } + +} diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/test/java/org/apache/hadoop/yarn/logaggregation/testutils/LogAggregationTestcaseBuilder.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/test/java/org/apache/hadoop/yarn/logaggregation/testutils/LogAggregationTestcaseBuilder.java new file mode 100644 index 0000000000000..f532dddce0fd9 --- /dev/null +++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/test/java/org/apache/hadoop/yarn/logaggregation/testutils/LogAggregationTestcaseBuilder.java @@ -0,0 +1,172 @@ +/** + * 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.yarn.logaggregation.testutils; + +import org.apache.commons.compress.utils.Lists; +import org.apache.commons.lang3.tuple.Pair; +import org.apache.hadoop.conf.Configuration; +import org.apache.hadoop.fs.FileSystem; +import org.apache.hadoop.fs.Path; +import org.apache.hadoop.security.AccessControlException; +import org.apache.hadoop.yarn.api.records.ApplicationId; +import org.apache.hadoop.yarn.logaggregation.LogAggregationUtils; + +import java.io.IOException; +import java.util.ArrayList; +import java.util.HashMap; +import java.util.List; +import java.util.Map; + +import static org.apache.hadoop.yarn.logaggregation.TestAggregatedLogDeletionService.ALL_FILE_CONTROLLER_NAMES; + +public class LogAggregationTestcaseBuilder { + public static final long NO_TIMEOUT = -1; + final long now; + final Configuration conf; + Path remoteRootLogPath; + String suffix; + String userDirName; + long userDirModTime; + final Map injectedAppDirDeletionExceptions = new HashMap<>(); + List fileControllers; + long suffixDirModTime; + long bucketDirModTime; + String suffixDirName; + List apps = Lists.newArrayList(); + int[] finishedAppIds; + int[] runningAppIds; + PathWithFileStatus userDir; + PathWithFileStatus suffixDir; + PathWithFileStatus bucketDir; + String bucketId; + List> additionalAppDirs = new ArrayList<>(); + FileSystem rootFs; + + public LogAggregationTestcaseBuilder(Configuration conf) { + this.conf = conf; + this.now = System.currentTimeMillis(); + } + + public static LogAggregationTestcaseBuilder create(Configuration conf) { + return new LogAggregationTestcaseBuilder(conf); + } + + public LogAggregationTestcaseBuilder withRootPath(String root) throws IOException { + Path rootPath = new Path(root); + rootFs = rootPath.getFileSystem(conf); + return this; + } + + public LogAggregationTestcaseBuilder withRemoteRootLogPath(String remoteRootLogDir) { + remoteRootLogPath = new Path(remoteRootLogDir); + return this; + } + + public LogAggregationTestcaseBuilder withUserDir(String userDirName, long modTime) { + this.userDirName = userDirName; + this.userDirModTime = modTime; + return this; + } + + public LogAggregationTestcaseBuilder withSuffixDir(String suffix, long modTime) { + this.suffix = suffix; + this.suffixDirName = LogAggregationUtils.getBucketSuffix() + suffix; + this.suffixDirModTime = modTime; + return this; + } + + /** + * Bucket dir paths will be generated later. + * @param modTime The modification time + * @return The builder + */ + public LogAggregationTestcaseBuilder withBucketDir(long modTime) { + this.bucketDirModTime = modTime; + return this; + } + + public LogAggregationTestcaseBuilder withBucketDir(long modTime, String bucketId) { + this.bucketDirModTime = modTime; + this.bucketId = bucketId; + return this; + } + + public final LogAggregationTestcaseBuilder withApps(List apps) { + this.apps = apps; + return this; + } + + public LogAggregationTestcaseBuilder withFinishedApps(int... apps) { + this.finishedAppIds = apps; + return this; + } + + public LogAggregationTestcaseBuilder withRunningApps(int... apps) { + this.runningAppIds = apps; + return this; + } + + public LogAggregationTestcaseBuilder withBothFileControllers() { + this.fileControllers = ALL_FILE_CONTROLLER_NAMES; + return this; + } + + public LogAggregationTestcaseBuilder withAdditionalAppDirs(List> appDirs) { + this.additionalAppDirs = appDirs; + return this; + } + + public LogAggregationTestcaseBuilder injectExceptionForAppDirDeletion(int... indices) { + for (int i : indices) { + AccessControlException e = new AccessControlException("Injected Error\nStack Trace :("); + this.injectedAppDirDeletionExceptions.put(i, e); + } + return this; + } + + public LogAggregationTestcase build() throws IOException { + return new LogAggregationTestcase(this); + } + + public static final class AppDescriptor { + final long modTimeOfAppDir; + List> filesWithModDate = new ArrayList<>(); + String fileController; + + public AppDescriptor(long modTimeOfAppDir) { + this.modTimeOfAppDir = modTimeOfAppDir; + } + + public AppDescriptor(long modTimeOfAppDir, List> filesWithModDate) { + this.modTimeOfAppDir = modTimeOfAppDir; + this.filesWithModDate = filesWithModDate; + } + + public AppDescriptor(String fileController, long modTimeOfAppDir, + List> filesWithModDate) { + this(modTimeOfAppDir, filesWithModDate); + this.fileController = fileController; + } + + + public ApplicationId createApplicationId(long now, int id) { + return ApplicationId.newInstance(now, id); + } + } +} diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/test/java/org/apache/hadoop/yarn/logaggregation/testutils/MockRMClientUtils.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/test/java/org/apache/hadoop/yarn/logaggregation/testutils/MockRMClientUtils.java new file mode 100644 index 0000000000000..6eb1eb1ecbe0b --- /dev/null +++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/test/java/org/apache/hadoop/yarn/logaggregation/testutils/MockRMClientUtils.java @@ -0,0 +1,74 @@ +/** + * 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.yarn.logaggregation.testutils; + +import org.apache.hadoop.test.MockitoUtil; +import org.apache.hadoop.yarn.api.ApplicationClientProtocol; +import org.apache.hadoop.yarn.api.protocolrecords.GetApplicationReportRequest; +import org.apache.hadoop.yarn.api.protocolrecords.GetApplicationReportResponse; +import org.apache.hadoop.yarn.api.records.ApplicationId; +import org.apache.hadoop.yarn.api.records.ApplicationReport; +import org.apache.hadoop.yarn.api.records.YarnApplicationState; + +import java.util.List; + +import static org.mockito.Mockito.mock; +import static org.mockito.Mockito.when; + +public class MockRMClientUtils { + public static ApplicationClientProtocol createMockRMClient( + List finishedApplications, + List runningApplications) throws Exception { + final ApplicationClientProtocol mockProtocol = + MockitoUtil.mockProtocol(ApplicationClientProtocol.class); + if (finishedApplications != null && !finishedApplications.isEmpty()) { + for (ApplicationId appId : finishedApplications) { + GetApplicationReportRequest request = GetApplicationReportRequest.newInstance(appId); + GetApplicationReportResponse response = createApplicationReportWithFinishedApplication(); + when(mockProtocol.getApplicationReport(request)).thenReturn(response); + } + } + if (runningApplications != null && !runningApplications.isEmpty()) { + for (ApplicationId appId : runningApplications) { + GetApplicationReportRequest request = GetApplicationReportRequest.newInstance(appId); + GetApplicationReportResponse response = createApplicationReportWithRunningApplication(); + when(mockProtocol.getApplicationReport(request)).thenReturn(response); + } + } + return mockProtocol; + } + + public static GetApplicationReportResponse createApplicationReportWithRunningApplication() { + ApplicationReport report = mock(ApplicationReport.class); + when(report.getYarnApplicationState()).thenReturn( + YarnApplicationState.RUNNING); + GetApplicationReportResponse response = + mock(GetApplicationReportResponse.class); + when(response.getApplicationReport()).thenReturn(report); + return response; + } + + public static GetApplicationReportResponse createApplicationReportWithFinishedApplication() { + ApplicationReport report = mock(ApplicationReport.class); + when(report.getYarnApplicationState()).thenReturn(YarnApplicationState.FINISHED); + GetApplicationReportResponse response = mock(GetApplicationReportResponse.class); + when(response.getApplicationReport()).thenReturn(report); + return response; + } +} diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/test/java/org/apache/hadoop/yarn/logaggregation/testutils/PathWithFileStatus.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/test/java/org/apache/hadoop/yarn/logaggregation/testutils/PathWithFileStatus.java new file mode 100644 index 0000000000000..5e743f1138e4e --- /dev/null +++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/test/java/org/apache/hadoop/yarn/logaggregation/testutils/PathWithFileStatus.java @@ -0,0 +1,45 @@ +/** + * 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.yarn.logaggregation.testutils; + +import org.apache.hadoop.fs.FileStatus; +import org.apache.hadoop.fs.Path; + +public class PathWithFileStatus { + public final Path path; + public FileStatus fileStatus; + + public PathWithFileStatus(Path path, FileStatus fileStatus) { + this.path = path; + this.fileStatus = fileStatus; + } + + public void changeModificationTime(long modTime) { + fileStatus = new FileStatus(fileStatus.getLen(), fileStatus.isDirectory(), + fileStatus.getReplication(), + fileStatus.getBlockSize(), modTime, fileStatus.getPath()); + } + + @Override + public String toString() { + return "PathWithFileStatus{" + + "path=" + path + + '}'; + } +} diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/test/java/org/apache/hadoop/yarn/webapp/view/TestCommonViews.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/test/java/org/apache/hadoop/yarn/webapp/view/TestCommonViews.java index b533bce846dbe..a29b152e9f45e 100644 --- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/test/java/org/apache/hadoop/yarn/webapp/view/TestCommonViews.java +++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/test/java/org/apache/hadoop/yarn/webapp/view/TestCommonViews.java @@ -28,7 +28,6 @@ import org.apache.hadoop.yarn.webapp.view.JQueryUI; import org.junit.Test; -import static org.mockito.Mockito.*; public class TestCommonViews { diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-csi/pom.xml b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-csi/pom.xml index 62707eadb9d22..75165f696a169 100644 --- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-csi/pom.xml +++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-csi/pom.xml @@ -27,7 +27,6 @@ 1.26.0 - 1.5.0.Final @@ -149,13 +148,6 @@ - - - kr.motd.maven - os-maven-plugin - ${os-maven-plugin.version} - - org.apache.maven.plugins diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-applicationhistoryservice/src/main/java/org/apache/hadoop/yarn/server/applicationhistoryservice/webapp/NavBlock.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-applicationhistoryservice/src/main/java/org/apache/hadoop/yarn/server/applicationhistoryservice/webapp/NavBlock.java index 10d3b52c7e93b..6b0570a32e19e 100644 --- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-applicationhistoryservice/src/main/java/org/apache/hadoop/yarn/server/applicationhistoryservice/webapp/NavBlock.java +++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-applicationhistoryservice/src/main/java/org/apache/hadoop/yarn/server/applicationhistoryservice/webapp/NavBlock.java @@ -21,7 +21,6 @@ import com.google.inject.Inject; import org.apache.hadoop.conf.Configuration; import org.apache.hadoop.yarn.api.records.YarnApplicationState; -import org.apache.hadoop.yarn.conf.YarnConfiguration; import org.apache.hadoop.yarn.server.webapp.WebPageUtils; import org.apache.hadoop.yarn.util.Log4jWarningErrorMetricsAppender; import org.apache.hadoop.yarn.webapp.hamlet2.Hamlet; diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-applicationhistoryservice/src/test/java/org/apache/hadoop/yarn/server/timeline/TimelineStoreTestUtils.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-applicationhistoryservice/src/test/java/org/apache/hadoop/yarn/server/timeline/TimelineStoreTestUtils.java index 71e298c018654..39b22b1afd5f0 100644 --- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-applicationhistoryservice/src/test/java/org/apache/hadoop/yarn/server/timeline/TimelineStoreTestUtils.java +++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-applicationhistoryservice/src/test/java/org/apache/hadoop/yarn/server/timeline/TimelineStoreTestUtils.java @@ -28,7 +28,6 @@ import java.util.EnumSet; import java.util.HashMap; import java.util.HashSet; -import java.util.Iterator; import java.util.List; import java.util.Map; import java.util.Map.Entry; diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-applicationhistoryservice/src/test/java/org/apache/hadoop/yarn/server/timeline/webapp/TestTimelineWebServicesWithSSL.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-applicationhistoryservice/src/test/java/org/apache/hadoop/yarn/server/timeline/webapp/TestTimelineWebServicesWithSSL.java index 46d5b6b873c74..36209a9be4bde 100644 --- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-applicationhistoryservice/src/test/java/org/apache/hadoop/yarn/server/timeline/webapp/TestTimelineWebServicesWithSSL.java +++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-applicationhistoryservice/src/test/java/org/apache/hadoop/yarn/server/timeline/webapp/TestTimelineWebServicesWithSSL.java @@ -35,7 +35,6 @@ import org.apache.hadoop.yarn.client.api.impl.TimelineWriter; import org.apache.hadoop.yarn.conf.YarnConfiguration; import org.apache.hadoop.yarn.server.applicationhistoryservice.ApplicationHistoryServer; -import org.apache.hadoop.yarn.server.applicationhistoryservice.webapp.AHSWebApp; import org.apache.hadoop.yarn.server.timeline.MemoryTimelineStore; import org.apache.hadoop.yarn.server.timeline.TimelineReader.Field; import org.apache.hadoop.yarn.server.timeline.TimelineStore; diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-common/src/main/java/org/apache/hadoop/yarn/server/metrics/package-info.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-common/src/main/java/org/apache/hadoop/yarn/server/metrics/package-info.java deleted file mode 100644 index a67cf9304c162..0000000000000 --- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-common/src/main/java/org/apache/hadoop/yarn/server/metrics/package-info.java +++ /dev/null @@ -1,18 +0,0 @@ -/** - * 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.yarn.server.metrics; \ No newline at end of file diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/main/java/org/apache/hadoop/yarn/server/nodemanager/containermanager/AuxServicesEvent.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/main/java/org/apache/hadoop/yarn/server/nodemanager/containermanager/AuxServicesEvent.java index 1e5a9a737f588..e47158041dd6e 100644 --- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/main/java/org/apache/hadoop/yarn/server/nodemanager/containermanager/AuxServicesEvent.java +++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/main/java/org/apache/hadoop/yarn/server/nodemanager/containermanager/AuxServicesEvent.java @@ -21,7 +21,6 @@ import java.nio.ByteBuffer; import org.apache.hadoop.yarn.api.records.ApplicationId; -import org.apache.hadoop.yarn.api.records.ContainerId; import org.apache.hadoop.yarn.event.AbstractEvent; import org.apache.hadoop.yarn.server.nodemanager.containermanager.container .Container; diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/main/java/org/apache/hadoop/yarn/server/nodemanager/containermanager/launcher/ContainerCleanup.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/main/java/org/apache/hadoop/yarn/server/nodemanager/containermanager/launcher/ContainerCleanup.java index 63d9374764970..33ae86d0497fb 100644 --- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/main/java/org/apache/hadoop/yarn/server/nodemanager/containermanager/launcher/ContainerCleanup.java +++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/main/java/org/apache/hadoop/yarn/server/nodemanager/containermanager/launcher/ContainerCleanup.java @@ -36,7 +36,6 @@ import org.apache.hadoop.yarn.server.nodemanager.containermanager.container.ContainerExitEvent; import org.apache.hadoop.yarn.server.nodemanager.containermanager.deletion.task.DockerContainerDeletionTask; import org.apache.hadoop.yarn.server.nodemanager.containermanager.linux.runtime.DockerLinuxContainerRuntime; -import org.apache.hadoop.yarn.server.nodemanager.executor.ContainerReapContext; import org.apache.hadoop.yarn.server.nodemanager.executor.ContainerSignalContext; import org.slf4j.Logger; import org.slf4j.LoggerFactory; diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/main/java/org/apache/hadoop/yarn/server/nodemanager/containermanager/launcher/ContainersLauncherEvent.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/main/java/org/apache/hadoop/yarn/server/nodemanager/containermanager/launcher/ContainersLauncherEvent.java index 38bedf246b95e..e32ac370cb72e 100644 --- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/main/java/org/apache/hadoop/yarn/server/nodemanager/containermanager/launcher/ContainersLauncherEvent.java +++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/main/java/org/apache/hadoop/yarn/server/nodemanager/containermanager/launcher/ContainersLauncherEvent.java @@ -19,7 +19,6 @@ package org.apache.hadoop.yarn.server.nodemanager.containermanager.launcher; import org.apache.hadoop.yarn.event.AbstractEvent; -import org.apache.hadoop.yarn.event.Event; import org.apache.hadoop.yarn.server.nodemanager.containermanager.container.Container; public class ContainersLauncherEvent diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/main/java/org/apache/hadoop/yarn/server/nodemanager/containermanager/localizer/ContainerLocalizer.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/main/java/org/apache/hadoop/yarn/server/nodemanager/containermanager/localizer/ContainerLocalizer.java index 1bda631429a3b..604a810ec9468 100644 --- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/main/java/org/apache/hadoop/yarn/server/nodemanager/containermanager/localizer/ContainerLocalizer.java +++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/main/java/org/apache/hadoop/yarn/server/nodemanager/containermanager/localizer/ContainerLocalizer.java @@ -218,7 +218,7 @@ public LocalizationProtocol run() { ExecutorService createDownloadThreadPool() { return HadoopExecutors.newSingleThreadExecutor(new ThreadFactoryBuilder() - .setNameFormat("ContainerLocalizer Downloader").build()); + .setNameFormat("ContainerLocalizer Downloader-" + localizerId).build()); } CompletionService createCompletionService(ExecutorService exec) { diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/main/java/org/apache/hadoop/yarn/server/nodemanager/containermanager/localizer/LocalResourceRequest.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/main/java/org/apache/hadoop/yarn/server/nodemanager/containermanager/localizer/LocalResourceRequest.java index d2e8e22d45964..bd161cc35f82a 100644 --- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/main/java/org/apache/hadoop/yarn/server/nodemanager/containermanager/localizer/LocalResourceRequest.java +++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/main/java/org/apache/hadoop/yarn/server/nodemanager/containermanager/localizer/LocalResourceRequest.java @@ -25,7 +25,6 @@ import org.apache.hadoop.yarn.api.records.LocalResourceType; import org.apache.hadoop.yarn.api.records.LocalResourceVisibility; import org.apache.hadoop.yarn.api.records.URL; -import org.apache.hadoop.yarn.util.ConverterUtils; public class LocalResourceRequest extends LocalResource implements Comparable { diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/main/java/org/apache/hadoop/yarn/server/nodemanager/containermanager/localizer/event/LocalizerResourceRequestEvent.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/main/java/org/apache/hadoop/yarn/server/nodemanager/containermanager/localizer/event/LocalizerResourceRequestEvent.java index 0e732a7ce58a2..857b3314e1adb 100644 --- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/main/java/org/apache/hadoop/yarn/server/nodemanager/containermanager/localizer/event/LocalizerResourceRequestEvent.java +++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/main/java/org/apache/hadoop/yarn/server/nodemanager/containermanager/localizer/event/LocalizerResourceRequestEvent.java @@ -21,7 +21,6 @@ import org.apache.hadoop.yarn.server.nodemanager.containermanager.localizer.LocalizedResource; import org.apache.hadoop.yarn.server.nodemanager.containermanager.localizer.LocalizerContext; import org.apache.hadoop.yarn.server.nodemanager.containermanager.localizer.ResourceLocalizationService; -import org.apache.hadoop.yarn.util.ConverterUtils; /** * Event indicating that the {@link ResourceLocalizationService} diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/main/java/org/apache/hadoop/yarn/server/nodemanager/containermanager/localizer/sharedcache/SharedCacheUploader.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/main/java/org/apache/hadoop/yarn/server/nodemanager/containermanager/localizer/sharedcache/SharedCacheUploader.java index ed75116391537..8fc2e1e91220f 100644 --- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/main/java/org/apache/hadoop/yarn/server/nodemanager/containermanager/localizer/sharedcache/SharedCacheUploader.java +++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/main/java/org/apache/hadoop/yarn/server/nodemanager/containermanager/localizer/sharedcache/SharedCacheUploader.java @@ -44,7 +44,6 @@ import org.apache.hadoop.yarn.server.sharedcache.SharedCacheUtil; import org.apache.hadoop.yarn.sharedcache.SharedCacheChecksum; import org.apache.hadoop.yarn.sharedcache.SharedCacheChecksumFactory; -import org.apache.hadoop.yarn.util.ConverterUtils; import org.apache.hadoop.yarn.util.FSDownload; import org.apache.hadoop.classification.VisibleForTesting; diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/main/java/org/apache/hadoop/yarn/server/nodemanager/containermanager/loghandler/NonAggregatingLogHandler.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/main/java/org/apache/hadoop/yarn/server/nodemanager/containermanager/loghandler/NonAggregatingLogHandler.java index 7d55f2c108c5e..e8b2f472fb46f 100644 --- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/main/java/org/apache/hadoop/yarn/server/nodemanager/containermanager/loghandler/NonAggregatingLogHandler.java +++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/main/java/org/apache/hadoop/yarn/server/nodemanager/containermanager/loghandler/NonAggregatingLogHandler.java @@ -20,8 +20,6 @@ import java.io.IOException; import java.util.ArrayList; import java.util.Collections; -import java.util.HashMap; -import java.util.HashSet; import java.util.List; import java.util.Map; import java.util.Set; diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/main/java/org/apache/hadoop/yarn/server/nodemanager/logaggregation/tracker/NMLogAggregationStatusTracker.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/main/java/org/apache/hadoop/yarn/server/nodemanager/logaggregation/tracker/NMLogAggregationStatusTracker.java index eb2aaf54a3dee..c51d47a7217f5 100644 --- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/main/java/org/apache/hadoop/yarn/server/nodemanager/logaggregation/tracker/NMLogAggregationStatusTracker.java +++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/main/java/org/apache/hadoop/yarn/server/nodemanager/logaggregation/tracker/NMLogAggregationStatusTracker.java @@ -34,7 +34,6 @@ import org.apache.hadoop.yarn.api.records.ApplicationId; import org.apache.hadoop.yarn.api.records.LogAggregationStatus; import org.apache.hadoop.yarn.conf.YarnConfiguration; -import org.apache.hadoop.yarn.exceptions.YarnRuntimeException; import org.apache.hadoop.yarn.server.api.protocolrecords.LogAggregationReport; import org.apache.hadoop.yarn.server.nodemanager.Context; import org.apache.hadoop.yarn.server.nodemanager.containermanager.application.Application; diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/main/java/org/apache/hadoop/yarn/server/nodemanager/webapp/NavBlock.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/main/java/org/apache/hadoop/yarn/server/nodemanager/webapp/NavBlock.java index 86d9ae7687ae9..87d511b1725e4 100644 --- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/main/java/org/apache/hadoop/yarn/server/nodemanager/webapp/NavBlock.java +++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/main/java/org/apache/hadoop/yarn/server/nodemanager/webapp/NavBlock.java @@ -19,7 +19,6 @@ package org.apache.hadoop.yarn.server.nodemanager.webapp; import org.apache.hadoop.conf.Configuration; -import org.apache.hadoop.yarn.conf.YarnConfiguration; import org.apache.hadoop.yarn.server.webapp.WebPageUtils; import org.apache.hadoop.yarn.util.Log4jWarningErrorMetricsAppender; import org.apache.hadoop.yarn.webapp.YarnWebParams; diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/main/java/org/apache/hadoop/yarn/server/nodemanager/webapp/dao/AppInfo.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/main/java/org/apache/hadoop/yarn/server/nodemanager/webapp/dao/AppInfo.java index f55ca810d8d07..3af7c9330fc62 100644 --- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/main/java/org/apache/hadoop/yarn/server/nodemanager/webapp/dao/AppInfo.java +++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/main/java/org/apache/hadoop/yarn/server/nodemanager/webapp/dao/AppInfo.java @@ -27,7 +27,6 @@ import org.apache.hadoop.yarn.api.records.ContainerId; import org.apache.hadoop.yarn.server.nodemanager.containermanager.application.Application; import org.apache.hadoop.yarn.server.nodemanager.containermanager.container.Container; -import org.apache.hadoop.yarn.util.ConverterUtils; @XmlRootElement(name = "app") @XmlAccessorType(XmlAccessType.FIELD) diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/test/java/org/apache/hadoop/yarn/server/nodemanager/TestContainerManagerWithLCE.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/test/java/org/apache/hadoop/yarn/server/nodemanager/TestContainerManagerWithLCE.java index 7bf042f5ebf9d..c3d46bb8095f3 100644 --- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/test/java/org/apache/hadoop/yarn/server/nodemanager/TestContainerManagerWithLCE.java +++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/test/java/org/apache/hadoop/yarn/server/nodemanager/TestContainerManagerWithLCE.java @@ -27,7 +27,6 @@ import org.apache.hadoop.fs.UnsupportedFileSystemException; import org.apache.hadoop.fs.permission.FsPermission; import org.apache.hadoop.yarn.conf.YarnConfiguration; -import org.apache.hadoop.yarn.exceptions.YarnException; import org.apache.hadoop.yarn.server.nodemanager.containermanager.TestContainerManager; import org.junit.After; import org.junit.Assume; diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/test/java/org/apache/hadoop/yarn/server/nodemanager/TestDirectoryCollection.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/test/java/org/apache/hadoop/yarn/server/nodemanager/TestDirectoryCollection.java index 59a30370283de..33bd4d92347ca 100644 --- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/test/java/org/apache/hadoop/yarn/server/nodemanager/TestDirectoryCollection.java +++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/test/java/org/apache/hadoop/yarn/server/nodemanager/TestDirectoryCollection.java @@ -35,8 +35,6 @@ import org.junit.After; import org.junit.Assert; import org.junit.Before; -import org.junit.BeforeClass; -import org.junit.Before; import org.junit.Test; public class TestDirectoryCollection { diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/test/java/org/apache/hadoop/yarn/server/nodemanager/api/protocolrecords/impl/pb/TestPBRecordImpl.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/test/java/org/apache/hadoop/yarn/server/nodemanager/api/protocolrecords/impl/pb/TestPBRecordImpl.java index 05ead9df4ab1b..0fe8f0179bed9 100644 --- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/test/java/org/apache/hadoop/yarn/server/nodemanager/api/protocolrecords/impl/pb/TestPBRecordImpl.java +++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/test/java/org/apache/hadoop/yarn/server/nodemanager/api/protocolrecords/impl/pb/TestPBRecordImpl.java @@ -48,7 +48,7 @@ import org.apache.hadoop.yarn.server.nodemanager.api.protocolrecords.LocalizerHeartbeatResponse; import org.apache.hadoop.yarn.server.nodemanager.api.protocolrecords.LocalizerStatus; import org.apache.hadoop.yarn.server.nodemanager.api.protocolrecords.ResourceStatusType; -import org.apache.hadoop.yarn.util.ConverterUtils; + import org.junit.Test; public class TestPBRecordImpl { diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/test/java/org/apache/hadoop/yarn/server/nodemanager/containermanager/localizer/TestLocalResource.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/test/java/org/apache/hadoop/yarn/server/nodemanager/containermanager/localizer/TestLocalResource.java index 13310ad51009f..daa1033697f5b 100644 --- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/test/java/org/apache/hadoop/yarn/server/nodemanager/containermanager/localizer/TestLocalResource.java +++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/test/java/org/apache/hadoop/yarn/server/nodemanager/containermanager/localizer/TestLocalResource.java @@ -27,7 +27,6 @@ import org.apache.hadoop.yarn.api.records.URL; import org.apache.hadoop.yarn.factory.providers.RecordFactoryProvider; import org.apache.hadoop.yarn.server.nodemanager.containermanager.localizer.LocalResourceRequest; -import org.apache.hadoop.yarn.util.ConverterUtils; import static org.apache.hadoop.yarn.api.records.LocalResourceType.*; import static org.apache.hadoop.yarn.api.records.LocalResourceVisibility.*; diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/test/java/org/apache/hadoop/yarn/server/nodemanager/containermanager/scheduler/TestContainerSchedulerRecovery.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/test/java/org/apache/hadoop/yarn/server/nodemanager/containermanager/scheduler/TestContainerSchedulerRecovery.java index b09a1f260e04f..103bfb3217e00 100644 --- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/test/java/org/apache/hadoop/yarn/server/nodemanager/containermanager/scheduler/TestContainerSchedulerRecovery.java +++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/test/java/org/apache/hadoop/yarn/server/nodemanager/containermanager/scheduler/TestContainerSchedulerRecovery.java @@ -40,7 +40,6 @@ import org.junit.After; import org.junit.Before; import org.junit.Test; -import org.mockito.InjectMocks; import org.mockito.Mock; import org.mockito.MockitoAnnotations; diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/test/java/org/apache/hadoop/yarn/server/nodemanager/webapp/TestNMWebFilter.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/test/java/org/apache/hadoop/yarn/server/nodemanager/webapp/TestNMWebFilter.java index 5dbe244e78662..266c638a89830 100644 --- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/test/java/org/apache/hadoop/yarn/server/nodemanager/webapp/TestNMWebFilter.java +++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/test/java/org/apache/hadoop/yarn/server/nodemanager/webapp/TestNMWebFilter.java @@ -27,7 +27,6 @@ import java.io.IOException; import java.io.PrintWriter; import java.io.StringWriter; -import java.net.HttpURLConnection; import java.util.HashMap; import java.util.Map; import java.util.concurrent.ConcurrentHashMap; diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/ClientRMService.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/ClientRMService.java index 04efc8864139b..6c37b7e9c0a8e 100644 --- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/ClientRMService.java +++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/ClientRMService.java @@ -37,7 +37,6 @@ import java.util.Set; import java.util.concurrent.atomic.AtomicInteger; import java.util.stream.Collectors; - import org.apache.commons.cli.UnrecognizedOptionException; import org.apache.commons.lang3.Range; import org.slf4j.Logger; @@ -913,7 +912,17 @@ public GetApplicationsResponse getApplications(GetApplicationsRequest request) } if (queues != null && !queues.isEmpty()) { - if (!queues.contains(application.getQueue())) { + Map> foundApps = queryApplicationsByQueues(apps, queues); + List runningAppsByQueues = foundApps.entrySet().stream() + .filter(e -> queues.contains(e.getKey())) + .map(Map.Entry::getValue) + .flatMap(Collection::stream) + .collect(Collectors.toList()); + List runningAppsById = runningAppsByQueues.stream() + .filter(app -> app.getApplicationId().equals(application.getApplicationId())) + .collect(Collectors.toList()); + + if (runningAppsById.isEmpty() && !queues.contains(application.getQueue())) { continue; } } @@ -992,6 +1001,22 @@ public GetApplicationsResponse getApplications(GetApplicationsRequest request) return response; } + private Map> queryApplicationsByQueues( + Map apps, Set queues) { + final Map> appsToQueues = new HashMap<>(); + for (String queue : queues) { + List appsInQueue = scheduler.getAppsInQueue(queue); + if (appsInQueue != null && !appsInQueue.isEmpty()) { + for (ApplicationAttemptId appAttemptId : appsInQueue) { + RMApp rmApp = apps.get(appAttemptId.getApplicationId()); + appsToQueues.putIfAbsent(queue, new ArrayList<>()); + appsToQueues.get(queue).add(rmApp); + } + } + } + return appsToQueues; + } + private Set getLowerCasedAppTypes(GetApplicationsRequest request) { Set applicationTypes = new HashSet<>(); if (request.getApplicationTypes() != null && !request.getApplicationTypes() diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/ClusterMetrics.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/ClusterMetrics.java index c56a1e7b3d92d..b31d7c0e6f439 100644 --- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/ClusterMetrics.java +++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/ClusterMetrics.java @@ -39,10 +39,8 @@ import org.apache.hadoop.metrics2.lib.MutableRate; import org.apache.hadoop.yarn.api.records.Resource; import org.apache.hadoop.classification.VisibleForTesting; -import org.apache.hadoop.yarn.api.records.ResourceInformation; import org.apache.hadoop.yarn.metrics.CustomResourceMetricValue; import org.apache.hadoop.yarn.metrics.CustomResourceMetrics; -import org.apache.hadoop.yarn.server.resourcemanager.scheduler.QueueMetricsForCustomResources; import org.apache.hadoop.yarn.util.resource.ResourceUtils; @InterfaceAudience.Private diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/DefaultAMSProcessor.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/DefaultAMSProcessor.java index 3797a6ed3a189..bab7e521dc0c0 100644 --- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/DefaultAMSProcessor.java +++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/DefaultAMSProcessor.java @@ -55,8 +55,6 @@ import org.apache.hadoop.yarn.exceptions.InvalidContainerReleaseException; import org.apache.hadoop.yarn.exceptions.InvalidResourceBlacklistRequestException; import org.apache.hadoop.yarn.exceptions.InvalidResourceRequestException; -import org.apache.hadoop.yarn.exceptions.InvalidResourceRequestException - .InvalidResourceType; import org.apache.hadoop.yarn.exceptions.SchedulerInvalidResoureRequestException; import org.apache.hadoop.yarn.exceptions.YarnException; import org.apache.hadoop.yarn.factories.RecordFactory; diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/ResourceManager.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/ResourceManager.java index 3bd6a0fe43a8c..8adcff42a695d 100644 --- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/ResourceManager.java +++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/ResourceManager.java @@ -23,6 +23,7 @@ import com.sun.jersey.spi.container.servlet.ServletContainer; import org.apache.hadoop.yarn.metrics.GenericEventTypeMetrics; +import org.apache.hadoop.yarn.webapp.WebAppException; import org.slf4j.Logger; import org.slf4j.LoggerFactory; import org.slf4j.Marker; @@ -1440,7 +1441,12 @@ protected void startWepApp() { IsResourceManagerActiveServlet.PATH_SPEC, IsResourceManagerActiveServlet.class); - webApp = builder.start(new RMWebApp(this), uiWebAppContext); + try { + webApp = builder.start(new RMWebApp(this), uiWebAppContext); + } catch (WebAppException e) { + webApp = e.getWebApp(); + throw e; + } } private String getWebAppsPath(String appName) { diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/ahs/RMApplicationHistoryWriter.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/ahs/RMApplicationHistoryWriter.java index a532a4ab9d975..a9e73b8e0b745 100644 --- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/ahs/RMApplicationHistoryWriter.java +++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/ahs/RMApplicationHistoryWriter.java @@ -36,7 +36,6 @@ import org.apache.hadoop.yarn.exceptions.YarnRuntimeException; import org.apache.hadoop.yarn.server.applicationhistoryservice.ApplicationHistoryStore; import org.apache.hadoop.yarn.server.applicationhistoryservice.ApplicationHistoryWriter; -import org.apache.hadoop.yarn.server.applicationhistoryservice.FileSystemApplicationHistoryStore; import org.apache.hadoop.yarn.server.applicationhistoryservice.NullApplicationHistoryStore; import org.apache.hadoop.yarn.server.applicationhistoryservice.records.ApplicationAttemptFinishData; import org.apache.hadoop.yarn.server.applicationhistoryservice.records.ApplicationAttemptStartData; diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/monitor/capacity/AbstractPreemptableResourceCalculator.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/monitor/capacity/AbstractPreemptableResourceCalculator.java index c8f68a26a07a4..c3c594512b07c 100644 --- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/monitor/capacity/AbstractPreemptableResourceCalculator.java +++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/monitor/capacity/AbstractPreemptableResourceCalculator.java @@ -31,12 +31,16 @@ import java.util.Comparator; import java.util.Iterator; import java.util.PriorityQueue; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; /** * Calculate how much resources need to be preempted for each queue, * will be used by {@link PreemptionCandidatesSelector}. */ public class AbstractPreemptableResourceCalculator { + private static final Logger LOG = LoggerFactory.getLogger( + AbstractPreemptableResourceCalculator.class); protected final CapacitySchedulerPreemptionContext context; protected final ResourceCalculator rc; @@ -76,6 +80,34 @@ private double getIdealPctOfGuaranteed(TempQueuePerPartition q) { } } + private static class NormalizationTuple { + private Resource numerator; + private Resource denominator; + + NormalizationTuple(Resource numer, Resource denom) { + this.numerator = numer; + this.denominator = denom; + } + + long getNumeratorValue(int i) { + return numerator.getResourceInformation(i).getValue(); + } + + long getDenominatorValue(int i) { + String nUnits = numerator.getResourceInformation(i).getUnits(); + ResourceInformation dResourceInformation = denominator + .getResourceInformation(i); + return UnitsConversionUtil.convert( + dResourceInformation.getUnits(), nUnits, dResourceInformation.getValue()); + } + + float getNormalizedValue(int i) { + long nValue = getNumeratorValue(i); + long dValue = getDenominatorValue(i); + return dValue == 0 ? 0.0f : (float) nValue / dValue; + } + } + /** * PreemptableResourceCalculator constructor. * @@ -175,7 +207,7 @@ protected void computeFixpointAllocation(Resource totGuarant, unassigned, Resources.none())) { // we compute normalizedGuarantees capacity based on currently active // queues - resetCapacity(unassigned, orderedByNeed, ignoreGuarantee); + resetCapacity(orderedByNeed, ignoreGuarantee); // For each underserved queue (or set of queues if multiple are equally // underserved), offer its share of the unassigned resources based on its @@ -252,47 +284,146 @@ protected void initIdealAssignment(Resource totGuarant, /** * Computes a normalizedGuaranteed capacity based on active queues. * - * @param clusterResource - * the total amount of resources in the cluster * @param queues * the list of queues to consider * @param ignoreGuar * ignore guarantee. */ - private void resetCapacity(Resource clusterResource, - Collection queues, boolean ignoreGuar) { + private void resetCapacity(Collection queues, + boolean ignoreGuar) { Resource activeCap = Resource.newInstance(0, 0); + float activeTotalAbsCap = 0.0f; int maxLength = ResourceUtils.getNumberOfCountableResourceTypes(); if (ignoreGuar) { - for (TempQueuePerPartition q : queues) { - for (int i = 0; i < maxLength; i++) { - q.normalizedGuarantee[i] = 1.0f / queues.size(); + for (int i = 0; i < maxLength; i++) { + for (TempQueuePerPartition q : queues) { + computeNormGuarEvenly(q, queues.size(), i); } } } else { for (TempQueuePerPartition q : queues) { Resources.addTo(activeCap, q.getGuaranteed()); + activeTotalAbsCap += q.getAbsCapacity(); } - for (TempQueuePerPartition q : queues) { - for (int i = 0; i < maxLength; i++) { - ResourceInformation nResourceInformation = q.getGuaranteed() - .getResourceInformation(i); - ResourceInformation dResourceInformation = activeCap - .getResourceInformation(i); - - long nValue = nResourceInformation.getValue(); - long dValue = UnitsConversionUtil.convert( - dResourceInformation.getUnits(), nResourceInformation.getUnits(), - dResourceInformation.getValue()); - if (dValue != 0) { - q.normalizedGuarantee[i] = (float) nValue / dValue; + + // loop through all resource types and normalize guaranteed capacity for all queues + for (int i = 0; i < maxLength; i++) { + boolean useAbsCapBasedNorm = false; + // if the sum of absolute capacity of all queues involved is 0, + // we should normalize evenly + boolean useEvenlyDistNorm = activeTotalAbsCap == 0; + + // loop through all the queues once to determine the + // right normalization strategy for current processing resource type + for (TempQueuePerPartition q : queues) { + NormalizationTuple normTuple = new NormalizationTuple( + q.getGuaranteed(), activeCap); + long queueGuaranValue = normTuple.getNumeratorValue(i); + long totalActiveGuaranValue = normTuple.getDenominatorValue(i); + + if (queueGuaranValue == 0 && q.getAbsCapacity() != 0 && totalActiveGuaranValue != 0) { + // when the rounded value of a resource type is 0 but its absolute capacity is not 0, + // we should consider taking the normalized guarantee based on absolute capacity + useAbsCapBasedNorm = true; + break; + } + + if (totalActiveGuaranValue == 0) { + // If totalActiveGuaranValue from activeCap is zero, that means the guaranteed capacity + // of this resource dimension for all active queues is tiny (close to 0). + // For example, if a queue has 1% of minCapacity on a cluster with a totalVcores of 48, + // then the idealAssigned Vcores for this queue is (48 * 0.01)=0.48 which then + // get rounded/casted into 0 (double -> long) + // In this scenario where the denominator is 0, we can just spread resources across + // all tiny queues evenly since their absoluteCapacity are roughly the same + useEvenlyDistNorm = true; + } + } + + if (LOG.isDebugEnabled()) { + LOG.debug("Queue normalization strategy: " + + "absoluteCapacityBasedNormalization(" + useAbsCapBasedNorm + + "), evenlyDistributedNormalization(" + useEvenlyDistNorm + + "), defaultNormalization(" + !(useAbsCapBasedNorm || useEvenlyDistNorm) + ")"); + } + + // loop through all the queues again to apply normalization strategy + for (TempQueuePerPartition q : queues) { + if (useAbsCapBasedNorm) { + computeNormGuarFromAbsCapacity(q, activeTotalAbsCap, i); + } else if (useEvenlyDistNorm) { + computeNormGuarEvenly(q, queues.size(), i); + } else { + computeDefaultNormGuar(q, activeCap, i); } } } } } + /** + * Computes the normalized guaranteed capacity based on the weight of a queue's abs capacity. + * + * Example: + * There are two active queues: queueA & queueB, and + * their configured absolute minimum capacity is 1% and 3% respectively. + * + * Then their normalized guaranteed capacity are: + * normalized_guar_queueA = 0.01 / (0.01 + 0.03) = 0.25 + * normalized_guar_queueB = 0.03 / (0.01 + 0.03) = 0.75 + * + * @param q + * the queue to consider + * @param activeTotalAbsCap + * the sum of absolute capacity of all active queues + * @param resourceTypeIdx + * index of the processing resource type + */ + private static void computeNormGuarFromAbsCapacity(TempQueuePerPartition q, + float activeTotalAbsCap, + int resourceTypeIdx) { + if (activeTotalAbsCap != 0) { + q.normalizedGuarantee[resourceTypeIdx] = q.getAbsCapacity() / activeTotalAbsCap; + } + } + + /** + * Computes the normalized guaranteed capacity evenly based on num of active queues. + * + * @param q + * the queue to consider + * @param numOfActiveQueues + * number of active queues + * @param resourceTypeIdx + * index of the processing resource type + */ + private static void computeNormGuarEvenly(TempQueuePerPartition q, + int numOfActiveQueues, + int resourceTypeIdx) { + q.normalizedGuarantee[resourceTypeIdx] = 1.0f / numOfActiveQueues; + } + + /** + * The default way to compute a queue's normalized guaranteed capacity. + * + * For each resource type, divide a queue's configured guaranteed amount (MBs/Vcores) by + * the total amount of guaranteed resource of all active queues + * + * @param q + * the queue to consider + * @param activeCap + * total guaranteed resources of all active queues + * @param resourceTypeIdx + * index of the processing resource type + */ + private static void computeDefaultNormGuar(TempQueuePerPartition q, + Resource activeCap, + int resourceTypeIdx) { + NormalizationTuple normTuple = new NormalizationTuple(q.getGuaranteed(), activeCap); + q.normalizedGuarantee[resourceTypeIdx] = normTuple.getNormalizedValue(resourceTypeIdx); + } + // Take the most underserved TempQueue (the one on the head). Collect and // return the list of all queues that have the same idealAssigned // percentage of guaranteed. diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/monitor/capacity/FifoIntraQueuePreemptionPlugin.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/monitor/capacity/FifoIntraQueuePreemptionPlugin.java index 188e619891ee7..4cc1c8cee9dd5 100644 --- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/monitor/capacity/FifoIntraQueuePreemptionPlugin.java +++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/monitor/capacity/FifoIntraQueuePreemptionPlugin.java @@ -23,7 +23,6 @@ import java.util.Collections; import java.util.Comparator; import java.util.HashMap; -import java.util.HashSet; import java.util.List; import java.util.Map; import java.util.PriorityQueue; @@ -402,8 +401,7 @@ private PriorityQueue createTempAppForResCalculation( pending = (pending == null) ? Resources.createResource(0, 0) : pending; reserved = (reserved == null) ? Resources.createResource(0, 0) : reserved; - HashSet partitions = new HashSet( - app.getAppAttemptResourceUsage().getNodePartitionsSet()); + Set partitions = app.getAppAttemptResourceUsage().getExistingNodeLabels(); partitions.addAll(app.getTotalPendingRequestsPerPartition().keySet()); // Create TempAppPerQueue for further calculation. diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/monitor/capacity/TempQueuePerPartition.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/monitor/capacity/TempQueuePerPartition.java index 958c08e8038af..78075bb5c17c0 100644 --- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/monitor/capacity/TempQueuePerPartition.java +++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/monitor/capacity/TempQueuePerPartition.java @@ -201,6 +201,10 @@ Resource offer(Resource avail, ResourceCalculator rc, return remain; } + public float getAbsCapacity() { + return absCapacity; + } + public Resource getGuaranteed() { if(!effMinRes.equals(Resources.none())) { return Resources.clone(effMinRes); diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/nodelabels/NodeAttributesManagerImpl.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/nodelabels/NodeAttributesManagerImpl.java index 6769e83669fd4..b0b98ad9b481a 100644 --- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/nodelabels/NodeAttributesManagerImpl.java +++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/nodelabels/NodeAttributesManagerImpl.java @@ -61,7 +61,6 @@ import org.apache.hadoop.yarn.server.resourcemanager.RMContext; import org.apache.hadoop.yarn.server.resourcemanager.scheduler.event.NodeAttributesUpdateSchedulerEvent; -import org.apache.hadoop.thirdparty.com.google.common.base.Strings; /** * Manager holding the attributes to Labels. diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/resource/DynamicResourceConfiguration.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/resource/DynamicResourceConfiguration.java index 3869a008d9add..e0c498661c5ed 100644 --- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/resource/DynamicResourceConfiguration.java +++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/resource/DynamicResourceConfiguration.java @@ -32,7 +32,6 @@ import org.apache.hadoop.yarn.api.records.Resource; import org.apache.hadoop.yarn.api.records.ResourceOption; import org.apache.hadoop.yarn.conf.YarnConfiguration; -import org.apache.hadoop.yarn.util.ConverterUtils; import org.apache.hadoop.yarn.util.resource.Resources; public class DynamicResourceConfiguration extends Configuration { diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/rmcontainer/RMContainer.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/rmcontainer/RMContainer.java index bcee51d65c330..8c6784b12bd04 100644 --- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/rmcontainer/RMContainer.java +++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/rmcontainer/RMContainer.java @@ -31,7 +31,6 @@ import org.apache.hadoop.yarn.api.records.NodeId; import org.apache.hadoop.yarn.api.records.Priority; import org.apache.hadoop.yarn.api.records.Resource; -import org.apache.hadoop.yarn.api.records.ResourceRequest; import org.apache.hadoop.yarn.api.records.SchedulingRequest; import org.apache.hadoop.yarn.event.EventHandler; import org.apache.hadoop.yarn.server.resourcemanager.scheduler.common.ContainerRequest; diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/rmnode/RMNodeUpdateContainerEvent.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/rmnode/RMNodeUpdateContainerEvent.java index b8f8e734f9b19..6bbe39e89fc12 100644 --- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/rmnode/RMNodeUpdateContainerEvent.java +++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/rmnode/RMNodeUpdateContainerEvent.java @@ -18,7 +18,6 @@ package org.apache.hadoop.yarn.server.resourcemanager.rmnode; -import java.util.List; import java.util.Map; import org.apache.hadoop.yarn.api.records.Container; diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/AbstractResourceUsage.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/AbstractResourceUsage.java index ad3bfb18ef3b8..b95f192febe53 100644 --- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/AbstractResourceUsage.java +++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/AbstractResourceUsage.java @@ -19,6 +19,7 @@ package org.apache.hadoop.yarn.server.resourcemanager.scheduler; import java.util.HashMap; +import java.util.HashSet; import java.util.Map; import java.util.Set; import java.util.concurrent.atomic.AtomicReferenceArray; @@ -200,10 +201,10 @@ public String toString() { } } - public Set getNodePartitionsSet() { + public Set getExistingNodeLabels() { readLock.lock(); try { - return usages.keySet(); + return new HashSet<>(usages.keySet()); } finally { readLock.unlock(); } diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/PartitionQueueMetrics.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/PartitionQueueMetrics.java index f43131809a0ca..02eaa7bd9b7de 100644 --- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/PartitionQueueMetrics.java +++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/PartitionQueueMetrics.java @@ -40,6 +40,7 @@ protected PartitionQueueMetrics(MetricsSystem ms, String queueName, String parentMetricName = partition + METRIC_NAME_DELIMITER + newQueueName; setParent(getQueueMetrics().get(parentMetricName)); + storedPartitionMetrics = null; } } diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/QueueMetrics.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/QueueMetrics.java index 6f9b1ab47b2b1..d41cbed0b107a 100644 --- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/QueueMetrics.java +++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/QueueMetrics.java @@ -23,6 +23,7 @@ import java.util.ArrayList; import java.util.HashMap; import java.util.Map; +import java.util.Set; import org.apache.hadoop.classification.InterfaceAudience; import org.apache.hadoop.classification.InterfaceAudience.Private; @@ -40,6 +41,7 @@ import org.apache.hadoop.metrics2.lib.MutableGaugeInt; import org.apache.hadoop.metrics2.lib.MutableGaugeLong; import org.apache.hadoop.metrics2.lib.MutableRate; +import org.apache.hadoop.util.Sets; import org.apache.hadoop.yarn.api.records.ApplicationId; import org.apache.hadoop.yarn.api.records.Resource; import org.apache.hadoop.yarn.conf.YarnConfiguration; @@ -133,7 +135,7 @@ public class QueueMetrics implements MetricsSource { protected final MetricsRegistry registry; protected final String queueName; private QueueMetrics parent; - private final Queue parentQueue; + private Queue parentQueue; protected final MetricsSystem metricsSystem; protected final Map users; protected final Configuration conf; @@ -177,6 +179,7 @@ public class QueueMetrics implements MetricsSource { "AggregatePreemptedSeconds."; private static final String AGGREGATE_PREEMPTED_SECONDS_METRIC_DESC = "Aggregate Preempted Seconds for NAME"; + protected Set storedPartitionMetrics = Sets.newConcurrentHashSet(); public QueueMetrics(MetricsSystem ms, String queueName, Queue parent, boolean enableUserMetrics, Configuration conf) { @@ -338,6 +341,7 @@ public synchronized QueueMetrics getPartitionQueueMetrics(String partition) { queueMetrics.tag(PARTITION_INFO, partitionJMXStr).tag(QUEUE_INFO, this.queueName)); getQueueMetrics().put(metricName, queueMetrics); + registerPartitionMetricsCreation(metricName); return queueMetrics; } else { return metrics; @@ -380,6 +384,7 @@ private QueueMetrics getPartitionMetrics(String partition) { partitionJMXStr)); } getQueueMetrics().put(metricName, metrics); + registerPartitionMetricsCreation(metricName); } return metrics; } @@ -1332,4 +1337,26 @@ public void setParent(QueueMetrics parent) { public Queue getParentQueue() { return parentQueue; } + + protected void registerPartitionMetricsCreation(String metricName) { + if (storedPartitionMetrics != null) { + storedPartitionMetrics.add(metricName); + } + } + + public void setParentQueue(Queue parentQueue) { + this.parentQueue = parentQueue; + + if (storedPartitionMetrics == null) { + return; + } + + for (String partitionMetric : storedPartitionMetrics) { + QueueMetrics metric = getQueueMetrics().get(partitionMetric); + + if (metric != null && metric.parentQueue != null) { + metric.parentQueue = parentQueue; + } + } + } } \ No newline at end of file diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/activities/ActivitiesManager.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/activities/ActivitiesManager.java index 6285cc941e92b..01a18db786f13 100644 --- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/activities/ActivitiesManager.java +++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/activities/ActivitiesManager.java @@ -50,10 +50,7 @@ import java.util.concurrent.ConcurrentLinkedQueue; import java.util.concurrent.ConcurrentMap; import java.util.concurrent.atomic.AtomicInteger; -import java.util.List; -import java.util.Set; import java.util.*; -import java.util.ArrayList; import java.util.stream.Collectors; /** diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/capacity/AbstractCSQueue.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/capacity/AbstractCSQueue.java index 6caca65ffb0e2..1a5a1ce0fd461 100644 --- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/capacity/AbstractCSQueue.java +++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/capacity/AbstractCSQueue.java @@ -262,6 +262,7 @@ public CSQueue getParent() { @Override public void setParent(CSQueue newParentQueue) { this.parent = newParentQueue; + getMetrics().setParentQueue(newParentQueue); } @Override @@ -1000,8 +1001,8 @@ public Set getNodeLabelsForQueue() { Set nodeLabels = new HashSet(); if (this.getAccessibleNodeLabels() != null && this.getAccessibleNodeLabels() .contains(RMNodeLabelsManager.ANY)) { - nodeLabels.addAll(Sets.union(this.getQueueCapacities().getNodePartitionsSet(), - this.getQueueResourceUsage().getNodePartitionsSet())); + nodeLabels.addAll(Sets.union(this.getQueueCapacities().getExistingNodeLabels(), + this.getQueueResourceUsage().getExistingNodeLabels())); } else { nodeLabels.addAll(this.getAccessibleNodeLabels()); } diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/capacity/AbstractLeafQueue.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/capacity/AbstractLeafQueue.java index 3d6863335de89..08fedb578cab9 100644 --- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/capacity/AbstractLeafQueue.java +++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/capacity/AbstractLeafQueue.java @@ -578,6 +578,8 @@ public void submitApplicationAttempt(FiCaSchedulerApp application, public void submitApplicationAttempt(FiCaSchedulerApp application, String userName, boolean isMoveApp) { // Careful! Locking order is important! + boolean isAppAlreadySubmitted = applicationAttemptMap.containsKey( + application.getApplicationAttemptId()); writeLock.lock(); try { // TODO, should use getUser, use this method just to avoid UT failure @@ -591,7 +593,7 @@ public void submitApplicationAttempt(FiCaSchedulerApp application, } // We don't want to update metrics for move app - if (!isMoveApp) { + if (!isMoveApp && !isAppAlreadySubmitted) { boolean unmanagedAM = application.getAppSchedulingInfo() != null && application.getAppSchedulingInfo().isUnmanagedAM(); usageTracker.getMetrics().submitAppAttempt(userName, unmanagedAM); @@ -1748,8 +1750,8 @@ public void recalculateQueueUsageRatio(Resource clusterResource, if (nodePartition == null) { for (String partition : Sets.union( - getQueueCapacities().getNodePartitionsSet(), - queueResourceUsage.getNodePartitionsSet())) { + getQueueCapacities().getExistingNodeLabels(), + queueResourceUsage.getExistingNodeLabels())) { usersManager.updateUsageRatio(partition, clusterResource); } } else { diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/capacity/CSQueueUtils.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/capacity/CSQueueUtils.java index c6d50a1cfb7c5..ecb285748b510 100644 --- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/capacity/CSQueueUtils.java +++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/capacity/CSQueueUtils.java @@ -19,9 +19,7 @@ import java.util.Set; -import org.apache.hadoop.thirdparty.com.google.common.collect.ImmutableSet; import org.apache.hadoop.util.Sets; -import org.apache.hadoop.yarn.api.records.QueueState; import org.apache.hadoop.yarn.api.records.Resource; import org.apache.hadoop.yarn.nodelabels.CommonNodeLabelsManager; import org.apache.hadoop.yarn.server.resourcemanager.nodelabels.RMNodeLabelsManager; @@ -224,8 +222,8 @@ public static void updateQueueStatistics( ResourceUsage queueResourceUsage = childQueue.getQueueResourceUsage(); if (nodePartition == null) { - for (String partition : Sets.union(queueCapacities.getNodePartitionsSet(), - queueResourceUsage.getNodePartitionsSet())) { + for (String partition : Sets.union(queueCapacities.getExistingNodeLabels(), + queueResourceUsage.getExistingNodeLabels())) { updateUsedCapacity(rc, nlm.getResourceByLabel(partition, cluster), partition, childQueue); diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/capacity/CapacityScheduler.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/capacity/CapacityScheduler.java index 44e80a6c23487..cf5034ba22877 100644 --- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/capacity/CapacityScheduler.java +++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/capacity/CapacityScheduler.java @@ -2678,7 +2678,12 @@ public String moveApplication(ApplicationId appId, if (application == null) { throw new YarnException("App to be moved " + appId + " not found."); } - String sourceQueueName = application.getQueue().getQueueName(); + if (!(application.getQueue() instanceof CSQueue)) { + throw new YarnException("Source queue is not a Capacity Scheduler queue"); + } + + CSQueue csQueue = (CSQueue) application.getQueue(); + String sourceQueueName = csQueue.getQueuePath(); AbstractLeafQueue source = this.queueManager.getAndCheckLeafQueue(sourceQueueName); String destQueueName = handleMoveToPlanQueue(targetQueueName); diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/capacity/QueueCapacities.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/capacity/QueueCapacities.java index 0edf83f72694d..87d4094f89384 100644 --- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/capacity/QueueCapacities.java +++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/capacity/QueueCapacities.java @@ -337,13 +337,4 @@ public String toString() { readLock.unlock(); } } - - public Set getNodePartitionsSet() { - readLock.lock(); - try { - return capacitiesMap.keySet(); - } finally { - readLock.unlock(); - } - } } diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/capacity/QueueManagementDynamicEditPolicy.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/capacity/QueueManagementDynamicEditPolicy.java index 5ddb6a2ee42b8..593cc874f6d64 100644 --- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/capacity/QueueManagementDynamicEditPolicy.java +++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/capacity/QueueManagementDynamicEditPolicy.java @@ -37,7 +37,6 @@ import org.apache.hadoop.yarn.util.SystemClock; import org.apache.hadoop.yarn.util.resource.ResourceCalculator; -import java.text.MessageFormat; import java.util.ArrayList; import java.util.Collections; import java.util.HashSet; diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/capacity/UsersManager.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/capacity/UsersManager.java index e8c99408ffeec..479bfec6d9440 100644 --- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/capacity/UsersManager.java +++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/capacity/UsersManager.java @@ -972,7 +972,7 @@ private void updateActiveUsersResourceUsage(String userName) { // Update total resource usage of active and non-active after user // is moved from non-active to active. - for (String partition : resourceUsage.getNodePartitionsSet()) { + for (String partition : resourceUsage.getExistingNodeLabels()) { totalResUsageForNonActiveUsers.decUsed(partition, resourceUsage.getUsed(partition)); totalResUsageForActiveUsers.incUsed(partition, @@ -1013,7 +1013,7 @@ private void updateNonActiveUsersResourceUsage(String userName) { // Update total resource usage of active and non-active after user is // moved from active to non-active. - for (String partition : resourceUsage.getNodePartitionsSet()) { + for (String partition : resourceUsage.getExistingNodeLabels()) { totalResUsageForActiveUsers.decUsed(partition, resourceUsage.getUsed(partition)); totalResUsageForNonActiveUsers.incUsed(partition, diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/capacity/allocator/ContainerAllocation.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/capacity/allocator/ContainerAllocation.java index b9b9bcff2f607..94bd4cb3c96d1 100644 --- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/capacity/allocator/ContainerAllocation.java +++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/capacity/allocator/ContainerAllocation.java @@ -18,12 +18,9 @@ package org.apache.hadoop.yarn.server.resourcemanager.scheduler.capacity.allocator; -import org.apache.hadoop.yarn.api.records.Container; -import org.apache.hadoop.yarn.api.records.ContainerId; import org.apache.hadoop.yarn.api.records.Resource; import org.apache.hadoop.yarn.server.resourcemanager.rmcontainer.RMContainer; import org.apache.hadoop.yarn.server.resourcemanager.scheduler.NodeType; -import org.apache.hadoop.yarn.server.resourcemanager.scheduler.common.fica.FiCaSchedulerNode; import org.apache.hadoop.yarn.util.resource.Resources; import java.util.List; diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/capacity/conf/LeveldbConfigurationStore.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/capacity/conf/LeveldbConfigurationStore.java index a351799872fe6..6aa37f399e41a 100644 --- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/capacity/conf/LeveldbConfigurationStore.java +++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/capacity/conf/LeveldbConfigurationStore.java @@ -204,8 +204,8 @@ public void logMutation(LogMutation logMutation) throws IOException { @Override public void confirmMutation(LogMutation pendingMutation, boolean isValid) { - WriteBatch updateBatch = db.createWriteBatch(); if (isValid) { + WriteBatch updateBatch = db.createWriteBatch(); for (Map.Entry changes : pendingMutation.getUpdates().entrySet()) { if (changes.getValue() == null || changes.getValue().isEmpty()) { @@ -215,8 +215,8 @@ public void confirmMutation(LogMutation pendingMutation, } } increaseConfigVersion(); + db.write(updateBatch); } - db.write(updateBatch); } private byte[] serLogMutations(LinkedList mutations) throws diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/capacity/conf/ZKConfigurationStore.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/capacity/conf/ZKConfigurationStore.java index 71226c300a840..492ff43dce72b 100644 --- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/capacity/conf/ZKConfigurationStore.java +++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/capacity/conf/ZKConfigurationStore.java @@ -18,6 +18,7 @@ package org.apache.hadoop.yarn.server.resourcemanager.scheduler.capacity.conf; +import org.apache.commons.io.serialization.ValidatingObjectInputStream; import org.apache.hadoop.classification.VisibleForTesting; import org.apache.zookeeper.KeeperException.NodeExistsException; import org.slf4j.Logger; @@ -35,7 +36,6 @@ import java.io.IOException; import java.io.ByteArrayInputStream; import java.io.ByteArrayOutputStream; -import java.io.ObjectInputStream; import java.io.ObjectOutputStream; import java.util.HashMap; import java.util.LinkedList; @@ -251,7 +251,7 @@ public List getConfirmedConfHistory(long fromId) { private boolean createNewZkPath(String path) throws Exception { if (!zkManager.exists(path)) { try { - zkManager.create(path); + zkManager.create(path, zkAcl); } catch(NodeExistsException e) { LOG.warn(NODEEXISTS_MSG, e); return false; @@ -314,7 +314,8 @@ private static byte[] serializeObject(Object o) throws Exception { private static Object deserializeObject(byte[] bytes) throws Exception { try (ByteArrayInputStream bais = new ByteArrayInputStream(bytes); - ObjectInputStream ois = new ObjectInputStream(bais);) { + ValidatingObjectInputStream ois = new ValidatingObjectInputStream(bais);) { + ois.accept(LinkedList.class, LogMutation.class, HashMap.class, String.class); return ois.readObject(); } } diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/capacity/preemption/PreemptableQueue.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/capacity/preemption/PreemptableQueue.java index fefb56ad730cf..72d5bc58b2007 100644 --- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/capacity/preemption/PreemptableQueue.java +++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/capacity/preemption/PreemptableQueue.java @@ -23,7 +23,6 @@ import org.apache.hadoop.yarn.server.resourcemanager.rmcontainer.RMContainer; import org.apache.hadoop.yarn.util.resource.Resources; -import java.util.Collections; import java.util.HashMap; import java.util.Map; import java.util.concurrent.ConcurrentSkipListMap; diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/fifo/FifoAppAttempt.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/fifo/FifoAppAttempt.java index fb5850d61b553..a4b2e2a1d5a60 100644 --- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/fifo/FifoAppAttempt.java +++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/fifo/FifoAppAttempt.java @@ -23,7 +23,6 @@ import org.apache.hadoop.yarn.api.records.ApplicationAttemptId; import org.apache.hadoop.yarn.api.records.Container; import org.apache.hadoop.yarn.api.records.ContainerId; -import org.apache.hadoop.yarn.api.records.ResourceRequest; import org.apache.hadoop.yarn.server.resourcemanager.RMAuditLogger; import org.apache.hadoop.yarn.server.resourcemanager.RMContext; import org.apache.hadoop.yarn.server.resourcemanager.rmcontainer.RMContainer; @@ -40,8 +39,6 @@ import org.apache.hadoop.yarn.server.scheduler.SchedulerRequestKey; -import java.util.List; - public class FifoAppAttempt extends FiCaSchedulerApp { private static final Logger LOG = LoggerFactory.getLogger(FifoAppAttempt.class); diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/webapp/CapacitySchedulerPage.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/webapp/CapacitySchedulerPage.java index 9ce37ac028e63..1018dc818dbf8 100644 --- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/webapp/CapacitySchedulerPage.java +++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/webapp/CapacitySchedulerPage.java @@ -103,7 +103,7 @@ private void renderLeafQueueInfoWithPartition(Block html) { ? NodeLabel.DEFAULT_NODE_LABEL_PARTITION : nodeLabel; // first display the queue's label specific details : ResponseInfo ri = - info("\'" + lqinfo.getQueuePath().substring(5) + info("\'" + lqinfo.getQueuePath() + "\' Queue Status for Partition \'" + nodeLabelDisplay + "\'"); renderQueueCapacityInfo(ri, nodeLabel); html.__(InfoBlock.class); @@ -113,7 +113,7 @@ private void renderLeafQueueInfoWithPartition(Block html) { // second display the queue specific details : ri = - info("\'" + lqinfo.getQueuePath().substring(5) + "\' Queue Status") + info("\'" + lqinfo.getQueuePath() + "\' Queue Status") .__("Queue State:", lqinfo.getQueueState()); renderCommonLeafQueueInfo(ri); @@ -125,7 +125,7 @@ private void renderLeafQueueInfoWithPartition(Block html) { private void renderLeafQueueInfoWithoutParition(Block html) { ResponseInfo ri = - info("\'" + lqinfo.getQueuePath().substring(5) + "\' Queue Status") + info("\'" + lqinfo.getQueuePath() + "\' Queue Status") .__("Queue State:", lqinfo.getQueueState()); renderQueueCapacityInfo(ri, ""); renderCommonLeafQueueInfo(ri); @@ -348,7 +348,7 @@ public void render(Block html) { span().$style(join(width(usedCapPercent), ";font-size:1px;left:0%;", absUsedCap > absCap ? Q_OVER : Q_UNDER)). __('.').__(). - span(".q", "Queue: "+info.getQueuePath().substring(5)).__(). + span(".q", info.getQueuePath()).__(). span().$class("qstats").$style(left(Q_STATS_POS)). __(join(percent(used), " used")).__(); @@ -492,7 +492,7 @@ public void render(Block html) { a(_Q).$style(width(Q_MAX_WIDTH)). span().$style(join(width(used), ";left:0%;", used > 1 ? Q_OVER : Q_UNDER)).__(".").__(). - span(".q", "Queue: root").__(). + span(".q", "root").__(). span().$class("qstats").$style(left(Q_STATS_POS)). __(join(percent(used), " used")).__(). __(QueueBlock.class).__(); @@ -522,7 +522,7 @@ public void render(Block html) { a(_Q).$style(width(Q_MAX_WIDTH)). span().$style(join(width(used), ";left:0%;", used > 1 ? Q_OVER : Q_UNDER)).__(".").__(). - span(".q", "Queue: root").__(). + span(".q", "root").__(). span().$class("qstats").$style(left(Q_STATS_POS)). __(join(percent(used), " used")).__(). __(QueueBlock.class).__().__(); @@ -656,12 +656,9 @@ public void render(HtmlBlock.Block html) { " }", " });", " $('#cs').bind('select_node.jstree', function(e, data) {", - " var q = $('.q', data.rslt.obj).first().text();", - " if (q == 'Queue: root') q = '';", - " else {", - " q = q.substr(q.lastIndexOf(':') + 2);", - " q = '^' + q.substr(q.lastIndexOf('.') + 1) + '$';", - " }", + " var queues = $('.q', data.rslt.obj);", + " var q = '^' + queues.first().text();", + " q += queues.length == 1 ? '$' : '\\\\.';", // Update this filter column index for queue if new columns are added // Current index for queue column is 5 " $('#apps').dataTable().fnFilter(q, 5, true);", diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/webapp/NavBlock.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/webapp/NavBlock.java index d6d7e54c34da7..c17d4f6d7b0e4 100644 --- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/webapp/NavBlock.java +++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/webapp/NavBlock.java @@ -21,7 +21,6 @@ import com.google.inject.Inject; import org.apache.hadoop.conf.Configuration; import org.apache.hadoop.yarn.api.records.YarnApplicationState; -import org.apache.hadoop.yarn.conf.YarnConfiguration; import org.apache.hadoop.yarn.server.webapp.WebPageUtils; import org.apache.hadoop.yarn.util.Log4jWarningErrorMetricsAppender; import org.apache.hadoop.yarn.webapp.hamlet2.Hamlet; diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/webapp/RMWebAppFilter.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/webapp/RMWebAppFilter.java index 6870317f9ae30..e70266f368a8b 100644 --- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/webapp/RMWebAppFilter.java +++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/webapp/RMWebAppFilter.java @@ -47,7 +47,6 @@ import org.apache.hadoop.yarn.server.resourcemanager.RMContext; import org.apache.hadoop.yarn.server.webproxy.ProxyUriUtils; import org.apache.hadoop.yarn.util.Apps; -import org.apache.hadoop.yarn.util.ConverterUtils; import org.apache.hadoop.yarn.webapp.YarnWebParams; import org.apache.hadoop.yarn.webapp.util.WebAppUtils; import org.slf4j.Logger; diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/webapp/dao/ClusterInfo.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/webapp/dao/ClusterInfo.java index d815315b26027..a429215dd5d79 100644 --- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/webapp/dao/ClusterInfo.java +++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/webapp/dao/ClusterInfo.java @@ -25,7 +25,6 @@ import org.apache.hadoop.service.Service.STATE; import org.apache.hadoop.util.VersionInfo; import org.apache.hadoop.yarn.server.resourcemanager.ResourceManager; -import org.apache.hadoop.yarn.server.resourcemanager.recovery.RMStateStore; import org.apache.hadoop.yarn.util.YarnVersionInfo; @XmlRootElement diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/webapp/dao/ResourcesInfo.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/webapp/dao/ResourcesInfo.java index 1d96f79081e31..775810348c291 100644 --- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/webapp/dao/ResourcesInfo.java +++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/webapp/dao/ResourcesInfo.java @@ -43,7 +43,7 @@ public ResourcesInfo(ResourceUsage resourceUsage, if (resourceUsage == null) { return; } - for (String partitionName : resourceUsage.getNodePartitionsSet()) { + for (String partitionName : resourceUsage.getExistingNodeLabels()) { resourceUsagesByPartition.add(new PartitionResourcesInfo(partitionName, new ResourceInfo(resourceUsage.getUsed(partitionName)), new ResourceInfo(resourceUsage.getReserved(partitionName)), diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/webapp/dao/UserMetricsInfo.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/webapp/dao/UserMetricsInfo.java index bfa5bd2937da3..b39c283029e38 100644 --- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/webapp/dao/UserMetricsInfo.java +++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/webapp/dao/UserMetricsInfo.java @@ -22,7 +22,6 @@ import javax.xml.bind.annotation.XmlRootElement; import javax.xml.bind.annotation.XmlTransient; -import org.apache.hadoop.yarn.server.resourcemanager.RMContext; import org.apache.hadoop.yarn.server.resourcemanager.ResourceManager; import org.apache.hadoop.yarn.server.resourcemanager.scheduler.QueueMetrics; import org.apache.hadoop.yarn.server.resourcemanager.scheduler.ResourceScheduler; diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/TestClientRMService.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/TestClientRMService.java index 7806845a2edeb..9f4e9433b144b 100644 --- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/TestClientRMService.java +++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/TestClientRMService.java @@ -1402,9 +1402,9 @@ public void handle(Event event) {} request.setQueues(queueSet); queueSet.add(queues[0]); - assertEquals("Incorrect number of applications in queue", 2, + assertEquals("Incorrect number of applications in queue", 3, rmService.getApplications(request).getApplicationList().size()); - assertEquals("Incorrect number of applications in queue", 2, + assertEquals("Incorrect number of applications in queue", 3, rmService.getApplications(request).getApplicationList().size()); queueSet.add(queues[1]); diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/TestClientRMTokens.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/TestClientRMTokens.java index 50afced670e25..556fd5bdf00d8 100644 --- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/TestClientRMTokens.java +++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/TestClientRMTokens.java @@ -18,7 +18,6 @@ package org.apache.hadoop.yarn.server.resourcemanager; import static org.junit.Assert.assertEquals; -import static org.junit.Assert.assertTrue; import static org.junit.Assert.fail; import static org.mockito.ArgumentMatchers.any; import static org.mockito.ArgumentMatchers.anyString; @@ -36,10 +35,10 @@ import java.security.PrivilegedAction; import java.security.PrivilegedExceptionAction; +import org.apache.hadoop.test.LambdaTestUtils; import org.apache.hadoop.thirdparty.protobuf.InvalidProtocolBufferException; import org.apache.hadoop.io.DataOutputBuffer; import org.apache.hadoop.net.NetUtils; -import org.apache.hadoop.security.token.delegation.TestDelegationToken; import org.apache.hadoop.yarn.server.resourcemanager.recovery.records.RMDelegationTokenIdentifierData; import org.junit.AfterClass; import org.junit.Assert; @@ -112,7 +111,7 @@ public void resetSecretManager() { } @Test - public void testDelegationToken() throws IOException, InterruptedException { + public void testDelegationToken() throws Exception { final YarnConfiguration conf = new YarnConfiguration(); conf.set(YarnConfiguration.RM_PRINCIPAL, "testuser/localhost@apache.org"); @@ -199,14 +198,11 @@ public void testDelegationToken() throws IOException, InterruptedException { } Thread.sleep(50l); LOG.info("At time: " + System.currentTimeMillis() + ", token should be invalid"); - // Token should have expired. - try { - clientRMWithDT.getNewApplication(request); - fail("Should not have succeeded with an expired token"); - } catch (Exception e) { - assertEquals(InvalidToken.class.getName(), e.getClass().getName()); - assertTrue(e.getMessage().contains("is expired")); - } + // Token should have expired. + final ApplicationClientProtocol finalClientRMWithDT = clientRMWithDT; + final GetNewApplicationRequest finalRequest = request; + LambdaTestUtils.intercept(InvalidToken.class, "Token has expired", + () -> finalClientRMWithDT.getNewApplication(finalRequest)); // Test cancellation // Stop the existing proxy, start another. diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/monitor/capacity/TestProportionalCapacityPreemptionPolicyInterQueueWithDRF.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/monitor/capacity/TestProportionalCapacityPreemptionPolicyInterQueueWithDRF.java index 27208020185f2..efd2ffc991cbd 100644 --- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/monitor/capacity/TestProportionalCapacityPreemptionPolicyInterQueueWithDRF.java +++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/monitor/capacity/TestProportionalCapacityPreemptionPolicyInterQueueWithDRF.java @@ -18,6 +18,11 @@ package org.apache.hadoop.yarn.server.resourcemanager.monitor.capacity; +import java.io.IOException; + +import org.junit.Before; +import org.junit.Test; + import org.apache.hadoop.yarn.api.protocolrecords.ResourceTypes; import org.apache.hadoop.yarn.api.records.Resource; import org.apache.hadoop.yarn.api.records.ResourceInformation; @@ -26,10 +31,6 @@ import org.apache.hadoop.yarn.server.resourcemanager.scheduler.capacity.LeafQueue; import org.apache.hadoop.yarn.util.resource.DominantResourceCalculator; import org.apache.hadoop.yarn.util.resource.ResourceUtils; -import org.junit.Before; -import org.junit.Test; - -import java.io.IOException; import static org.mockito.ArgumentMatchers.any; import static org.mockito.ArgumentMatchers.argThat; @@ -210,7 +211,7 @@ public void testInterQueuePreemptionWithStrictAndRelaxedDRF() "-b(=[20480:200 61440:600 0:0 20480:20 0]);" + // a "-c(=[20480:200 61440:600 0:0 0:0 0])"; // c String appsConfig = - //queueName\t(priority,resource,host,expression,#repeat,reserved) + // queueName\t(priority,resource,host,expression,#repeat,reserved,pending,user) "a\t" + "(1,1024:1,n1,,60,false,0:0,user1);" + // app1 in a "b\t" + "(1,0:0,n1,,0,false,20480:20,user2);"; // app2 in b diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/monitor/capacity/TestProportionalCapacityPreemptionPolicyIntraQueue.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/monitor/capacity/TestProportionalCapacityPreemptionPolicyIntraQueue.java index 807796242744e..2fb69f555728e 100644 --- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/monitor/capacity/TestProportionalCapacityPreemptionPolicyIntraQueue.java +++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/monitor/capacity/TestProportionalCapacityPreemptionPolicyIntraQueue.java @@ -18,13 +18,14 @@ package org.apache.hadoop.yarn.server.resourcemanager.monitor.capacity; -import org.apache.hadoop.yarn.server.resourcemanager.monitor.capacity.TestProportionalCapacityPreemptionPolicy.IsPreemptionRequestFor; -import org.apache.hadoop.yarn.server.resourcemanager.monitor.capacity.mockframework.ProportionalCapacityPreemptionPolicyMockFramework; -import org.apache.hadoop.yarn.server.resourcemanager.scheduler.capacity.CapacitySchedulerConfiguration; +import java.io.IOException; + import org.junit.Before; import org.junit.Test; -import java.io.IOException; +import org.apache.hadoop.yarn.server.resourcemanager.monitor.capacity.TestProportionalCapacityPreemptionPolicy.IsPreemptionRequestFor; +import org.apache.hadoop.yarn.server.resourcemanager.monitor.capacity.mockframework.ProportionalCapacityPreemptionPolicyMockFramework; +import org.apache.hadoop.yarn.server.resourcemanager.scheduler.capacity.CapacitySchedulerConfiguration; import static org.mockito.ArgumentMatchers.argThat; import static org.mockito.Mockito.never; @@ -845,7 +846,7 @@ public void testIntraQueuePreemptionWithTwoUsers() "-b(=[40 100 40 120 0])"; // b String appsConfig = - // queueName\t(priority,resource,host,expression,#repeat,reserved,pending) + // queueName\t(priority,resource,host,expression,#repeat,reserved,pending,user(optional)) "a\t" // app1 in a + "(1,1,n1,,5,false,25);" + // app1 a "a\t" // app2 in a diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/monitor/capacity/TestProportionalCapacityPreemptionPolicyIntraQueueFairOrdering.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/monitor/capacity/TestProportionalCapacityPreemptionPolicyIntraQueueFairOrdering.java index eb9d21836da51..def4554ec5b24 100644 --- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/monitor/capacity/TestProportionalCapacityPreemptionPolicyIntraQueueFairOrdering.java +++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/monitor/capacity/TestProportionalCapacityPreemptionPolicyIntraQueueFairOrdering.java @@ -74,7 +74,7 @@ public void testIntraQueuePreemptionFairOrderingPolicyEnabledOneAppPerUser() // With FairOrderingPolicy enabled on queue a, all 20 resources should be // preempted from app1 String appsConfig = - // queueName\t(prio,resource,host,expression,#repeat,reserved,pending,user) + // queueName\t(priority,resource,host,expression,#repeat,reserved,pending,user) "a\t" // app1, user1 in a + "(1,1,n1,,60,false,0,user1);" + "a\t" // app2, user2 in a @@ -122,7 +122,7 @@ public void testIntraQueuePreemptionFifoOrderingPolicyEnabled() // With FifoOrderingPolicy enabled on queue a, the first 5 should come from // the youngest app, app2, until app2 is reduced to the user limit of 35. String appsConfig = - // queueName\t(prio,resource,host,expression,#repeat,reserved,pending,user) + // queueName\t(priority,resource,host,expression,#repeat,reserved,pending,user) "a\t" // app1, user1 in a + "(1,1,n1,,60,false,0,user1);" + "a\t" // app2, user2 in a @@ -145,7 +145,7 @@ public void testIntraQueuePreemptionFifoOrderingPolicyEnabled() // The next 15 should come from app1 even though app2 is younger since app2 // has already been reduced to its user limit. appsConfig = - // queueName\t(prio,resource,host,expression,#repeat,reserved,pending,user) + // queueName\t(priority,resource,host,expression,#repeat,reserved,pending,user) "a\t" // app1, user1 in a + "(1,1,n1,,60,false,0,user1);" + "a\t" // app2, user2 in a @@ -195,7 +195,7 @@ public void testIntraQueuePreemptionFairOrderingPolicyMulitipleAppsPerUser() // preempted from app1 since it's the most over served app from the most // over served user String appsConfig = - // queueName\t(prio,resource,host,expression,#repeat,reserved,pending,user) + // queueName\t(priority,resource,host,expression,#repeat,reserved,pending,user) "a\t" // app1 and app2, user1 in a + "(1,1,n1,,35,false,0,user1);" + "a\t" @@ -243,7 +243,7 @@ public void testIntraQueuePreemptionFifoOrderingPolicyMultipleAppsPerUser() // user4/app4 is requesting 20 resources in queue a // With 3 users, preemptable user limit should be around 35 resources each. String appsConfig = - // queueName\t(prio,resource,host,expression,#repeat,reserved,pending,user) + // queueName\t(priority,resource,host,expression,#repeat,reserved,pending,user) "a\t" // app1, user1 in a + "(1,1,n1,,40,false,0,user1);" + "a\t" // app2, user1 in a diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/monitor/capacity/TestProportionalCapacityPreemptionPolicyIntraQueueUserLimit.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/monitor/capacity/TestProportionalCapacityPreemptionPolicyIntraQueueUserLimit.java index 9fb08cd3bc041..2a0838b83f453 100644 --- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/monitor/capacity/TestProportionalCapacityPreemptionPolicyIntraQueueUserLimit.java +++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/monitor/capacity/TestProportionalCapacityPreemptionPolicyIntraQueueUserLimit.java @@ -18,12 +18,13 @@ package org.apache.hadoop.yarn.server.resourcemanager.monitor.capacity; -import org.apache.hadoop.yarn.server.resourcemanager.monitor.capacity.mockframework.ProportionalCapacityPreemptionPolicyMockFramework; -import org.apache.hadoop.yarn.server.resourcemanager.scheduler.capacity.CapacitySchedulerConfiguration; +import java.io.IOException; + import org.junit.Before; import org.junit.Test; -import java.io.IOException; +import org.apache.hadoop.yarn.server.resourcemanager.monitor.capacity.mockframework.ProportionalCapacityPreemptionPolicyMockFramework; +import org.apache.hadoop.yarn.server.resourcemanager.scheduler.capacity.CapacitySchedulerConfiguration; import static org.mockito.ArgumentMatchers.argThat; import static org.mockito.Mockito.times; @@ -84,7 +85,7 @@ public void testSimpleIntraQueuePreemptionWithTwoUsers() "-a(=[100 100 100 30 0])"; // a String appsConfig = - // queueName\t(priority,resource,host,expression,#repeat,reserved,pending) + // queueName\t(priority,resource,host,expression,#repeat,reserved,pending,user) "a\t" // app1 in a + "(1,1,n1,,100,false,0,user1);" + // app1 a "a\t" // app2 in a @@ -139,7 +140,7 @@ public void testNoIntraQueuePreemptionWithSingleUser() "-a(=[100 100 100 30 0])"; // a String appsConfig = - // queueName\t(priority,resource,host,expression,#repeat,reserved,pending) + // queueName\t(priority,resource,host,expression,#repeat,reserved,pending,user) "a\t" // app1 in a + "(1,1,n1,,100,false,0,user1);" + // app1 a "a\t" // app2 in a @@ -196,7 +197,7 @@ public void testNoIntraQueuePreemptionWithTwoUserUnderUserLimit() "-a(=[100 100 80 30 0])"; // a String appsConfig = - // queueName\t(priority,resource,host,expression,#repeat,reserved,pending) + // queueName\t(priority,resource,host,expression,#repeat,reserved,pending,user) "a\t" // app1 in a + "(1,1,n1,,50,false,0,user1);" + // app1 a "a\t" // app2 in a @@ -252,7 +253,7 @@ public void testSimpleIntraQueuePreemptionWithTwoUsersWithAppPriority() "-a(=[100 100 100 30 0])"; // a String appsConfig = - // queueName\t(priority,resource,host,expression,#repeat,reserved,pending) + // queueName\t(priority,resource,host,expression,#repeat,reserved,pending,user) "a\t" // app1 in a + "(2,1,n1,,100,false,0,user1);" + // app1 a "a\t" // app2 in a @@ -309,7 +310,7 @@ public void testIntraQueuePreemptionOfUserLimitWithMultipleApps() "-a(=[100 100 80 90 0])"; // a String appsConfig = - // queueName\t(priority,resource,host,expression,#repeat,reserved,pending) + // queueName\t(priority,resource,host,expression,#repeat,reserved,pending,user) "a\t" // app1 in a + "(1,1,n1,,30,false,30,user1);" + // app1 a "a\t" // app2 in a @@ -370,7 +371,7 @@ public void testNoPreemptionOfUserLimitWithMultipleAppsAndSameUser() "-a(=[100 100 80 90 0])"; // a String appsConfig = - // queueName\t(priority,resource,host,expression,#repeat,reserved,pending) + // queueName\t(priority,resource,host,expression,#repeat,reserved,pending,user) "a\t" // app1 in a + "(1,1,n1,,30,false,20,user1);" + // app1 a "a\t" // app2 in a @@ -439,7 +440,7 @@ public void testIntraQueuePreemptionOfUserLimitWitAppsOfDifferentPriority() "-a(=[100 100 80 60 0])"; // b String appsConfig = - // queueName\t(priority,resource,host,expression,#repeat,reserved,pending) + // queueName\t(priority,resource,host,expression,#repeat,reserved,pending,user) "a\t" // app1 in a + "(3,1,n1,,30,false,30,user1);" + // app1 a "a\t" // app2 in a @@ -490,7 +491,7 @@ public void testIntraQueuePreemptionOfUserLimitInTwoQueues() "-b(=[40 100 35 20 0])"; // b String appsConfig = - // queueName\t(priority,resource,host,expression,#repeat,reserved,pending) + // queueName\t(priority,resource,host,expression,#repeat,reserved,pending,user) "a\t" // app1 in a + "(3,1,n1,,20,false,30,user1);" + // app1 a "a\t" // app2 in a @@ -556,7 +557,7 @@ public void testIntraQueuePreemptionWithTwoRequestingUsers() "-a(=[100 100 100 20 0])"; // a String appsConfig = - // queueName\t(priority,resource,host,expression,#repeat,reserved,pending) + // queueName\t(priority,resource,host,expression,#repeat,reserved,pending,user) "a\t" // app1 in a + "(1,1,n1,,60,false,10,user1);" + // app1 a "a\t" // app2 in a @@ -795,7 +796,7 @@ public void testNoIntraQueuePreemptionWithUserLimitDeadzone() "-a(=[100 100 100 20 0])"; // a String appsConfig = - // queueName\t(priority,resource,host,expression,#repeat,reserved,pending) + // queueName\t(priority,resource,host,expression,#repeat,reserved,pending,user) "a\t" // app1 in a + "(1,3,n1,,20,false,10,user1);" + // app1 a "a\t" // app2 in a @@ -855,7 +856,7 @@ public void testIntraQueuePreemptionWithUserLimitDeadzoneAndPriority() "-a(=[100 100 100 20 0])"; // a String appsConfig = - // queueName\t(priority,resource,host,expression,#repeat,reserved,pending) + // queueName\t(priority,resource,host,expression,#repeat,reserved,pending,user) "a\t" // app1 in a + "(1,3,n1,,20,false,10,user1);" + // app1 a "a\t" // app2 in a @@ -879,7 +880,7 @@ public void testIntraQueuePreemptionWithUserLimitDeadzoneAndPriority() // After first round, 3 containers were preempted from app1 and resource // distribution will be like below. appsConfig = - // queueName\t(priority,resource,host,expression,#repeat,reserved,pending) + // queueName\t(priority,resource,host,expression,#repeat,reserved,pending,user) "a\t" // app1 in a + "(1,3,n1,,17,false,10,user1);" + // app1 a "a\t" // app2 in a @@ -914,7 +915,7 @@ public void testSimpleIntraQueuePreemptionOneUserUnderOneUserAtOneUserAbove() "-a(=[100 100 100 1 0])"; // a String appsConfig = - // queueName\t(priority,resource,host,expression,#repeat,reserved,pending) + // queueName\t(priority,resource,host,expression,#repeat,reserved,pending,user) "a\t" // app1 in a + "(1,1,n1,,65,false,0,user1);" + "a\t" // app2 in a diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/recovery/RMStateStoreTestBase.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/recovery/RMStateStoreTestBase.java index 55654fb7b5ac6..1773c7aaa88e7 100644 --- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/recovery/RMStateStoreTestBase.java +++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/recovery/RMStateStoreTestBase.java @@ -84,7 +84,6 @@ import org.apache.hadoop.yarn.server.resourcemanager.security.ClientToAMTokenSecretManagerInRM; import org.apache.hadoop.yarn.server.security.MasterKeyData; import org.apache.hadoop.yarn.server.webproxy.ProxyCA; -import org.apache.hadoop.yarn.util.ConverterUtils; import org.apache.hadoop.yarn.util.resource.DefaultResourceCalculator; import org.apache.hadoop.yarn.util.resource.ResourceCalculator; import org.junit.Assert; diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/recovery/TestFSRMStateStore.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/recovery/TestFSRMStateStore.java index 13faff0fc8ac1..b775f4c4e7977 100644 --- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/recovery/TestFSRMStateStore.java +++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/recovery/TestFSRMStateStore.java @@ -49,7 +49,7 @@ import org.apache.hadoop.yarn.server.resourcemanager.rmapp.RMApp; import org.apache.hadoop.yarn.server.resourcemanager.rmapp.RMAppState; import org.apache.hadoop.yarn.server.resourcemanager.rmapp.attempt.RMAppAttempt; -import org.apache.hadoop.yarn.util.ConverterUtils; + import org.junit.Assert; import org.junit.Test; diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/reservation/TestCapacityOverTimePolicy.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/reservation/TestCapacityOverTimePolicy.java index 4ad4c3cdcb9d4..a988211d78f14 100644 --- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/reservation/TestCapacityOverTimePolicy.java +++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/reservation/TestCapacityOverTimePolicy.java @@ -22,7 +22,7 @@ import java.util.Collection; import net.jcip.annotations.NotThreadSafe; -import org.apache.hadoop.yarn.server.resourcemanager.reservation.exceptions.ContractValidationException; + import org.apache.hadoop.yarn.server.resourcemanager.reservation.exceptions.PlanningException; import org.apache.hadoop.yarn.server.resourcemanager.reservation.exceptions.PlanningQuotaException; import org.junit.Test; diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/rmapp/attempt/TestAMLivelinessMonitor.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/rmapp/attempt/TestAMLivelinessMonitor.java index 14c14df0afb51..33253ecb3bffc 100644 --- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/rmapp/attempt/TestAMLivelinessMonitor.java +++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/rmapp/attempt/TestAMLivelinessMonitor.java @@ -25,7 +25,7 @@ import org.apache.hadoop.yarn.server.resourcemanager.MockRM; import org.apache.hadoop.yarn.server.resourcemanager.recovery.MemoryRMStateStore; import org.apache.hadoop.yarn.util.ControlledClock; -import org.apache.hadoop.yarn.util.SystemClock; + import org.junit.Assert; import org.junit.Test; diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/capacity/CapacitySchedulerPreemptionTestBase.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/capacity/CapacitySchedulerPreemptionTestBase.java index 55ccb8afca952..41ee40ebcd2bc 100644 --- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/capacity/CapacitySchedulerPreemptionTestBase.java +++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/capacity/CapacitySchedulerPreemptionTestBase.java @@ -18,11 +18,9 @@ package org.apache.hadoop.yarn.server.resourcemanager.scheduler.capacity; -import org.apache.hadoop.conf.Configuration; import org.apache.hadoop.service.Service; import org.apache.hadoop.yarn.api.records.ApplicationAttemptId; import org.apache.hadoop.yarn.conf.YarnConfiguration; -import org.apache.hadoop.yarn.server.resourcemanager.Application; import org.apache.hadoop.yarn.server.resourcemanager.MockRM; import org.apache.hadoop.yarn.server.resourcemanager.ResourceManager; import org.apache.hadoop.yarn.server.resourcemanager.monitor.SchedulingEditPolicy; @@ -32,7 +30,6 @@ import org.apache.hadoop.yarn.server.resourcemanager.nodelabels.RMNodeLabelsManager; import org.apache.hadoop.yarn.server.resourcemanager.rmcontainer.RMContainer; import org.apache.hadoop.yarn.server.resourcemanager.scheduler.ResourceScheduler; -import org.apache.hadoop.yarn.server.resourcemanager.scheduler.SchedulerNode; import org.apache.hadoop.yarn.server.resourcemanager.scheduler.common.fica.FiCaSchedulerApp; import org.apache.hadoop.yarn.server.resourcemanager.scheduler.common.fica.FiCaSchedulerNode; import org.apache.hadoop.yarn.util.Clock; diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/capacity/CapacitySchedulerQueueHelpers.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/capacity/CapacitySchedulerQueueHelpers.java index bc7e2b317ac1f..1adde7d8d196e 100644 --- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/capacity/CapacitySchedulerQueueHelpers.java +++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/capacity/CapacitySchedulerQueueHelpers.java @@ -29,6 +29,7 @@ public final class CapacitySchedulerQueueHelpers { public static final String A = CapacitySchedulerConfiguration.ROOT + ".a"; public static final String B = CapacitySchedulerConfiguration.ROOT + ".b"; + public static final String A_CHILD = A + ".a"; public static final String A1 = A + ".a1"; public static final String A2 = A + ".a2"; public static final String B1 = B + ".b1"; @@ -89,6 +90,64 @@ public static CapacitySchedulerConfiguration setupQueueConfiguration( return conf; } + public static CapacitySchedulerConfiguration setupAdditionalQueues( + CapacitySchedulerConfiguration conf) { + + // Define top-level queues + conf.setQueues(CapacitySchedulerConfiguration.ROOT, new String[]{"a", "b"}); + + conf.setCapacity(A, A_CAPACITY); + conf.setCapacity(B, B_CAPACITY); + + // Define 2nd-level queues + conf.setQueues(A, new String[]{"a1", "a2", "a3"}); + conf.setCapacity(A1, 30.0f); + conf.setUserLimitFactor(A1, 100.0f); + conf.setCapacity(A2, 30.0f); + conf.setUserLimitFactor(A2, 100.0f); + conf.setCapacity("root.a.a3", 40.0f); + conf.setUserLimitFactor("root.a.a3", 100.0f); + + conf.setQueues(B, new String[]{"b1", "b2", "b3"}); + conf.setCapacity(B1, B1_CAPACITY); + conf.setUserLimitFactor(B1, 100.0f); + conf.setCapacity(B2, B2_CAPACITY); + conf.setUserLimitFactor(B2, 100.0f); + conf.setCapacity(B3, B3_CAPACITY); + conf.setUserLimitFactor(B3, 100.0f); + + return conf; + } + + /** + * @param conf, to be modified + * @return CS configuration which has deleted all children of queue(b) + * root + * / \ + * a b + * / \ + * a1 a2 + */ + public static CapacitySchedulerConfiguration setupQueueConfAmbiguousQueue( + CapacitySchedulerConfiguration conf) { + + // Define top-level queues + conf.setQueues(CapacitySchedulerConfiguration.ROOT, + new String[]{"a", "b"}); + + conf.setCapacity(A, A_CAPACITY); + conf.setCapacity(B, B_CAPACITY); + + // Define 2nd-level queues + conf.setQueues(A, new String[]{"a", "a1"}); + conf.setCapacity(A_CHILD, A1_CAPACITY); + conf.setUserLimitFactor(A1, 100.0f); + conf.setCapacity(A1, A2_CAPACITY); + conf.setUserLimitFactor(A2, 100.0f); + + return conf; + } + /** * @param conf, to be modified * @return CS configuration which has deleted all childred of queue(b) diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/capacity/CapacitySchedulerTestUtilities.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/capacity/CapacitySchedulerTestUtilities.java index b2c654891ebfc..b85874c62ca9e 100644 --- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/capacity/CapacitySchedulerTestUtilities.java +++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/capacity/CapacitySchedulerTestUtilities.java @@ -56,6 +56,7 @@ import java.io.IOException; import java.util.Set; +import static org.apache.hadoop.yarn.server.resourcemanager.scheduler.capacity.CapacitySchedulerQueueHelpers.setupQueueConfAmbiguousQueue; import static org.apache.hadoop.yarn.server.resourcemanager.scheduler.capacity.CapacitySchedulerQueueHelpers.setupQueueConfiguration; import static org.mockito.Mockito.mock; import static org.mockito.Mockito.when; @@ -174,6 +175,16 @@ public static ApplicationAttemptId appHelper(MockRM rm, CapacityScheduler cs, return appAttemptId1; } + public static MockRM setUpMoveAmbiguousQueue() { + CapacitySchedulerConfiguration conf = new CapacitySchedulerConfiguration(); + setupQueueConfAmbiguousQueue(conf); + conf.setClass(YarnConfiguration.RM_SCHEDULER, CapacityScheduler.class, + ResourceScheduler.class); + MockRM rm = new MockRM(conf); + rm.start(); + return rm; + } + public static MockRM setUpMove() { CapacitySchedulerConfiguration conf = new CapacitySchedulerConfiguration(); return setUpMove(conf); diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/capacity/TestCapacityScheduler.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/capacity/TestCapacityScheduler.java index 4a9e45e756f28..7c407dfd4ad4b 100644 --- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/capacity/TestCapacityScheduler.java +++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/capacity/TestCapacityScheduler.java @@ -24,6 +24,7 @@ import static org.apache.hadoop.yarn.server.resourcemanager.scheduler.capacity.CapacitySchedulerConfigGeneratorForTest.setMinAllocMb; import static org.apache.hadoop.yarn.server.resourcemanager.scheduler.capacity.CapacitySchedulerConfigGeneratorForTest.setMinAllocVcores; import static org.apache.hadoop.yarn.server.resourcemanager.scheduler.capacity.CapacitySchedulerQueueHelpers.findQueue; +import static org.apache.hadoop.yarn.server.resourcemanager.scheduler.capacity.CapacitySchedulerQueueHelpers.setupAdditionalQueues; import static org.apache.hadoop.yarn.server.resourcemanager.scheduler.capacity.CapacitySchedulerQueueHelpers.setupBlockedQueueConfiguration; import static org.apache.hadoop.yarn.server.resourcemanager.scheduler.capacity.CapacitySchedulerQueueHelpers.setupOtherBlockedQueueConfiguration; import static org.apache.hadoop.yarn.server.resourcemanager.scheduler.capacity.CapacitySchedulerQueueHelpers.setupQueueConfiguration; @@ -81,6 +82,7 @@ import org.apache.hadoop.util.Sets; import org.apache.hadoop.service.ServiceStateException; import org.apache.hadoop.yarn.server.api.records.NodeStatus; +import org.apache.hadoop.yarn.server.resourcemanager.scheduler.QueueMetrics; import org.slf4j.Logger; import org.slf4j.LoggerFactory; import org.apache.hadoop.conf.Configuration; @@ -2663,6 +2665,64 @@ public void testContainerAllocationLocalitySkipped() throws Exception { ContainerAllocation.QUEUE_SKIPPED.getAllocationState()); } + /** + * Tests + * @throws Exception + */ + @Test + public void testCSQueueMetricsDoesNotLeakOnReinit() throws Exception { + // Initialize resource map + Map riMap = new HashMap<>(); + + // Initialize mandatory resources + ResourceInformation memory = + ResourceInformation.newInstance(ResourceInformation.MEMORY_MB.getName(), + ResourceInformation.MEMORY_MB.getUnits(), + YarnConfiguration.DEFAULT_RM_SCHEDULER_MINIMUM_ALLOCATION_MB, + YarnConfiguration.DEFAULT_RM_SCHEDULER_MAXIMUM_ALLOCATION_MB); + ResourceInformation vcores = + ResourceInformation.newInstance(ResourceInformation.VCORES.getName(), + ResourceInformation.VCORES.getUnits(), + YarnConfiguration.DEFAULT_RM_SCHEDULER_MINIMUM_ALLOCATION_VCORES, + YarnConfiguration.DEFAULT_RM_SCHEDULER_MAXIMUM_ALLOCATION_VCORES); + riMap.put(ResourceInformation.MEMORY_URI, memory); + riMap.put(ResourceInformation.VCORES_URI, vcores); + + ResourceUtils.initializeResourcesFromResourceInformationMap(riMap); + + CapacitySchedulerConfiguration csConf = + new CapacitySchedulerConfiguration(); + csConf.setResourceComparator(DominantResourceCalculator.class); + + setupQueueConfiguration(csConf); + + YarnConfiguration conf = new YarnConfiguration(csConf); + + // Don't reset resource types since we have already configured resource + // types + conf.setBoolean(TestResourceProfiles.TEST_CONF_RESET_RESOURCE_TYPES, false); + conf.setClass(YarnConfiguration.RM_SCHEDULER, CapacityScheduler.class, + ResourceScheduler.class); + + MockRM rm = new MockRM(conf); + rm.start(); + + CapacityScheduler cs = (CapacityScheduler) rm.getResourceScheduler(); + csConf = new CapacitySchedulerConfiguration(); + setupAdditionalQueues(csConf); + cs.reinitialize(csConf, cs.getRMContext()); + QueueMetrics a3DefaultPartitionMetrics = QueueMetrics.getQueueMetrics().get( + "default.root.a.a3"); + + Assert.assertSame("Different ParentQueue of siblings is a sign of a memory leak", + QueueMetrics.getQueueMetrics().get("root.a.a1").getParentQueue(), + QueueMetrics.getQueueMetrics().get("root.a.a3").getParentQueue()); + + Assert.assertSame("Different ParentQueue of partition metrics is a sign of a memory leak", + QueueMetrics.getQueueMetrics().get("root.a.a1").getParentQueue(), + a3DefaultPartitionMetrics.getParentQueue()); + } + @Test public void testCSQueueMetrics() throws Exception { diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/capacity/TestCapacitySchedulerApps.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/capacity/TestCapacitySchedulerApps.java index 9943e03e4ad74..d192e7dcc6933 100644 --- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/capacity/TestCapacitySchedulerApps.java +++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/capacity/TestCapacitySchedulerApps.java @@ -18,6 +18,7 @@ package org.apache.hadoop.yarn.server.resourcemanager.scheduler.capacity; +import java.util.ArrayList; import java.util.List; import org.apache.hadoop.conf.Configuration; @@ -106,6 +107,7 @@ import static org.apache.hadoop.yarn.server.resourcemanager.scheduler.capacity.CapacitySchedulerTestUtilities.nodeUpdate; import static org.apache.hadoop.yarn.server.resourcemanager.scheduler.capacity.CapacitySchedulerTestUtilities.registerNode; import static org.apache.hadoop.yarn.server.resourcemanager.scheduler.capacity.CapacitySchedulerTestUtilities.setUpMove; +import static org.apache.hadoop.yarn.server.resourcemanager.scheduler.capacity.CapacitySchedulerTestUtilities.setUpMoveAmbiguousQueue; import static org.apache.hadoop.yarn.server.resourcemanager.scheduler.capacity.CapacitySchedulerTestUtilities.stopResourceManager; import static org.junit.Assert.assertEquals; import static org.junit.Assert.assertTrue; @@ -115,6 +117,8 @@ public class TestCapacitySchedulerApps { + public static final int MAX_PARALLEL_APPS = 5; + public static final String USER_0 = "user_0"; private ResourceManager resourceManager = null; private RMContext mockContext; @@ -236,18 +240,7 @@ public void testKillAllAppsInvalidSource() throws Exception { YarnScheduler scheduler = rm.getResourceScheduler(); // submit an app - MockRMAppSubmissionData data = - MockRMAppSubmissionData.Builder.createWithMemory(GB, rm) - .withAppName("test-move-1") - .withUser("user_0") - .withAcls(null) - .withQueue("a1") - .withUnmanagedAM(false) - .build(); - RMApp app = MockRMAppSubmitter.submit(rm, data); - ApplicationAttemptId appAttemptId = - rm.getApplicationReport(app.getApplicationId()) - .getCurrentApplicationAttemptId(); + ApplicationAttemptId appAttemptId = submitApp(rm); // check preconditions List appsInA1 = scheduler.getAppsInQueue("a1"); @@ -347,6 +340,46 @@ public void testAppReservationWithDominantResourceCalculator() throws Exception rm.stop(); } + @Test + public void testMoveAppAmbiguousQueue() throws Exception { + MockRM rm = setUpMoveAmbiguousQueue(); + AbstractYarnScheduler scheduler = + (AbstractYarnScheduler) rm.getResourceScheduler(); + QueueMetrics metrics = scheduler.getRootQueueMetrics(); + Assert.assertEquals(0, metrics.getAppsPending()); + // submit an app + MockRMAppSubmissionData data = + MockRMAppSubmissionData.Builder.createWithMemory(GB, rm) + .withAppName("test-move-1") + .withUser("user_0") + .withAcls(null) + .withQueue("root.a.a") + .withUnmanagedAM(false) + .build(); + RMApp app = MockRMAppSubmitter.submit(rm, data); + // check preconditions + List appsInA = scheduler.getAppsInQueue("root.a.a"); + assertEquals(1, appsInA.size()); + String queue = + scheduler.getApplicationAttempt(appsInA.get(0)).getQueue() + .getQueueName(); + Assert.assertEquals("a", queue); + + // now move the app + scheduler.moveApplication(app.getApplicationId(), "a1"); + List appsInA1 = scheduler.getAppsInQueue("root.a.a1"); + assertEquals(1, appsInA1.size()); + queue = + scheduler.getApplicationAttempt(appsInA1.get(0)).getQueue() + .getQueueName(); + Assert.assertEquals("a1", queue); + + appsInA = scheduler.getAppsInQueue("root.a.a"); + assertTrue(appsInA.isEmpty()); + + rm.stop(); + } + @Test public void testMoveAppBasic() throws Exception { MockRM rm = setUpMove(); @@ -979,18 +1012,7 @@ public void testMoveAllApps() throws Exception { (AbstractYarnScheduler) rm.getResourceScheduler(); // submit an app - MockRMAppSubmissionData data = - MockRMAppSubmissionData.Builder.createWithMemory(GB, rm) - .withAppName("test-move-1") - .withUser("user_0") - .withAcls(null) - .withQueue("a1") - .withUnmanagedAM(false) - .build(); - RMApp app = MockRMAppSubmitter.submit(rm, data); - ApplicationAttemptId appAttemptId = - rm.getApplicationReport(app.getApplicationId()) - .getCurrentApplicationAttemptId(); + ApplicationAttemptId appAttemptId = submitApp(rm); // check preconditions assertOneAppInQueue(scheduler, "a1"); @@ -1016,23 +1038,61 @@ public void testMoveAllApps() throws Exception { } @Test - public void testMoveAllAppsInvalidDestination() throws Exception { + public void testMaxParallelAppsPendingQueueMetrics() throws Exception { MockRM rm = setUpMove(); ResourceScheduler scheduler = rm.getResourceScheduler(); + CapacityScheduler cs = (CapacityScheduler) scheduler; + cs.getQueueContext().getConfiguration().setInt(CapacitySchedulerConfiguration.getQueuePrefix(A1) + + CapacitySchedulerConfiguration.MAX_PARALLEL_APPLICATIONS, MAX_PARALLEL_APPS); + cs.reinitialize(cs.getQueueContext().getConfiguration(), mockContext); + List attemptIds = new ArrayList<>(); + + for (int i = 0; i < 2 * MAX_PARALLEL_APPS; i++) { + attemptIds.add(submitApp(rm)); + } + + // Finish first batch to allow the other batch to run + for (int i = 0; i < MAX_PARALLEL_APPS; i++) { + cs.handle(new AppAttemptRemovedSchedulerEvent(attemptIds.get(i), + RMAppAttemptState.FINISHED, true)); + } + + // Finish the remaining apps + for (int i = MAX_PARALLEL_APPS; i < 2 * MAX_PARALLEL_APPS; i++) { + cs.handle(new AppAttemptRemovedSchedulerEvent(attemptIds.get(i), + RMAppAttemptState.FINISHED, true)); + } + + Assert.assertEquals("No pending app should remain for root queue", 0, + cs.getRootQueueMetrics().getAppsPending()); + Assert.assertEquals("No running application should remain for root queue", 0, + cs.getRootQueueMetrics().getAppsRunning()); + + rm.stop(); + } + private ApplicationAttemptId submitApp(MockRM rm) throws Exception { // submit an app MockRMAppSubmissionData data = MockRMAppSubmissionData.Builder.createWithMemory(GB, rm) .withAppName("test-move-1") - .withUser("user_0") + .withUser(USER_0) .withAcls(null) .withQueue("a1") .withUnmanagedAM(false) .build(); RMApp app = MockRMAppSubmitter.submit(rm, data); - ApplicationAttemptId appAttemptId = - rm.getApplicationReport(app.getApplicationId()) - .getCurrentApplicationAttemptId(); + return rm.getApplicationReport(app.getApplicationId()) + .getCurrentApplicationAttemptId(); + } + + @Test + public void testMoveAllAppsInvalidDestination() throws Exception { + MockRM rm = setUpMove(); + ResourceScheduler scheduler = rm.getResourceScheduler(); + + // submit an app + ApplicationAttemptId appAttemptId = submitApp(rm); // check preconditions assertApps(scheduler, "root", appAttemptId); diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/capacity/TestNodeLabelContainerAllocation.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/capacity/TestNodeLabelContainerAllocation.java index 30ee24d6e5532..87aa156f29207 100644 --- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/capacity/TestNodeLabelContainerAllocation.java +++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/capacity/TestNodeLabelContainerAllocation.java @@ -25,7 +25,6 @@ import java.util.Collections; import java.util.HashSet; import java.util.List; -import java.util.Map; import java.util.Set; import org.apache.hadoop.conf.Configuration; diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/capacity/TestQueueCapacities.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/capacity/TestQueueCapacities.java index 248831f03d15e..c3f505bdf085e 100644 --- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/capacity/TestQueueCapacities.java +++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/capacity/TestQueueCapacities.java @@ -24,7 +24,6 @@ import org.slf4j.Logger; import org.slf4j.LoggerFactory; -import org.apache.hadoop.yarn.api.records.Resource; import org.junit.Assert; import org.junit.Test; import org.junit.runner.RunWith; diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/capacity/conf/TestZKConfigurationStore.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/capacity/conf/TestZKConfigurationStore.java index 880ba77fa513b..155996d11feb7 100644 --- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/capacity/conf/TestZKConfigurationStore.java +++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/capacity/conf/TestZKConfigurationStore.java @@ -42,15 +42,18 @@ import org.apache.hadoop.yarn.webapp.dao.QueueConfigInfo; import org.apache.hadoop.yarn.webapp.dao.SchedConfUpdateInfo; import org.junit.After; +import org.junit.Assert; import org.junit.Before; import org.junit.Test; import org.slf4j.Logger; import org.slf4j.LoggerFactory; +import java.io.File; import java.io.IOException; import java.io.ByteArrayOutputStream; import java.io.ObjectOutputStream; import java.util.Arrays; +import java.util.Base64; import java.util.HashMap; import java.util.Map; @@ -67,6 +70,9 @@ public class TestZKConfigurationStore extends LoggerFactory.getLogger(TestZKConfigurationStore.class); private static final int ZK_TIMEOUT_MS = 10000; + private static final String DESERIALIZATION_VULNERABILITY_FILEPATH = + "/tmp/ZK_DESERIALIZATION_VULNERABILITY"; + private TestingServer curatorTestingServer; private CuratorFramework curatorFramework; private ResourceManager rm; @@ -401,6 +407,35 @@ public void testFailoverAfterRemoveQueue() throws Exception { rm2.close(); } + @Test(timeout = 3000) + @SuppressWarnings("checkstyle:linelength") + public void testDeserializationIsNotVulnerable() throws Exception { + confStore.initialize(conf, schedConf, rmContext); + String confStorePath = getZkPath("CONF_STORE"); + + File flagFile = new File(DESERIALIZATION_VULNERABILITY_FILEPATH); + if (flagFile.exists()) { + Assert.assertTrue(flagFile.delete()); + } + + // Generated using ysoserial (https://github.com/frohoff/ysoserial) + // java -jar ysoserial.jar CommonsBeanutils1 'touch /tmp/ZK_DESERIALIZATION_VULNERABILITY' | base64 + ((ZKConfigurationStore) confStore).setZkData(confStorePath, Base64.getDecoder().decode("rO0ABXNyABdqYXZhLnV0aWwuUHJpb3JpdHlRdWV1ZZTaMLT7P4KxAwACSQAEc2l6ZUwACmNvbXBhcmF0b3J0ABZMamF2YS91dGlsL0NvbXBhcmF0b3I7eHAAAAACc3IAK29yZy5hcGFjaGUuY29tbW9ucy5iZWFudXRpbHMuQmVhbkNvbXBhcmF0b3LjoYjqcyKkSAIAAkwACmNvbXBhcmF0b3JxAH4AAUwACHByb3BlcnR5dAASTGphdmEvbGFuZy9TdHJpbmc7eHBzcgA/b3JnLmFwYWNoZS5jb21tb25zLmNvbGxlY3Rpb25zLmNvbXBhcmF0b3JzLkNvbXBhcmFibGVDb21wYXJhdG9y+/SZJbhusTcCAAB4cHQAEG91dHB1dFByb3BlcnRpZXN3BAAAAANzcgA6Y29tLnN1bi5vcmcuYXBhY2hlLnhhbGFuLmludGVybmFsLnhzbHRjLnRyYXguVGVtcGxhdGVzSW1wbAlXT8FurKszAwAGSQANX2luZGVudE51bWJlckkADl90cmFuc2xldEluZGV4WwAKX2J5dGVjb2Rlc3QAA1tbQlsABl9jbGFzc3QAEltMamF2YS9sYW5nL0NsYXNzO0wABV9uYW1lcQB+AARMABFfb3V0cHV0UHJvcGVydGllc3QAFkxqYXZhL3V0aWwvUHJvcGVydGllczt4cAAAAAD/////dXIAA1tbQkv9GRVnZ9s3AgAAeHAAAAACdXIAAltCrPMX+AYIVOACAAB4cAAABsHK/rq+AAAAMgA5CgADACIHADcHACUHACYBABBzZXJpYWxWZXJzaW9uVUlEAQABSgEADUNvbnN0YW50VmFsdWUFrSCT85Hd7z4BAAY8aW5pdD4BAAMoKVYBAARDb2RlAQAPTGluZU51bWJlclRhYmxlAQASTG9jYWxWYXJpYWJsZVRhYmxlAQAEdGhpcwEAE1N0dWJUcmFuc2xldFBheWxvYWQBAAxJbm5lckNsYXNzZXMBADVMeXNvc2VyaWFsL3BheWxvYWRzL3V0aWwvR2FkZ2V0cyRTdHViVHJhbnNsZXRQYXlsb2FkOwEACXRyYW5zZm9ybQEAcihMY29tL3N1bi9vcmcvYXBhY2hlL3hhbGFuL2ludGVybmFsL3hzbHRjL0RPTTtbTGNvbS9zdW4vb3JnL2FwYWNoZS94bWwvaW50ZXJuYWwvc2VyaWFsaXplci9TZXJpYWxpemF0aW9uSGFuZGxlcjspVgEACGRvY3VtZW50AQAtTGNvbS9zdW4vb3JnL2FwYWNoZS94YWxhbi9pbnRlcm5hbC94c2x0Yy9ET007AQAIaGFuZGxlcnMBAEJbTGNvbS9zdW4vb3JnL2FwYWNoZS94bWwvaW50ZXJuYWwvc2VyaWFsaXplci9TZXJpYWxpemF0aW9uSGFuZGxlcjsBAApFeGNlcHRpb25zBwAnAQCmKExjb20vc3VuL29yZy9hcGFjaGUveGFsYW4vaW50ZXJuYWwveHNsdGMvRE9NO0xjb20vc3VuL29yZy9hcGFjaGUveG1sL2ludGVybmFsL2R0bS9EVE1BeGlzSXRlcmF0b3I7TGNvbS9zdW4vb3JnL2FwYWNoZS94bWwvaW50ZXJuYWwvc2VyaWFsaXplci9TZXJpYWxpemF0aW9uSGFuZGxlcjspVgEACGl0ZXJhdG9yAQA1TGNvbS9zdW4vb3JnL2FwYWNoZS94bWwvaW50ZXJuYWwvZHRtL0RUTUF4aXNJdGVyYXRvcjsBAAdoYW5kbGVyAQBBTGNvbS9zdW4vb3JnL2FwYWNoZS94bWwvaW50ZXJuYWwvc2VyaWFsaXplci9TZXJpYWxpemF0aW9uSGFuZGxlcjsBAApTb3VyY2VGaWxlAQAMR2FkZ2V0cy5qYXZhDAAKAAsHACgBADN5c29zZXJpYWwvcGF5bG9hZHMvdXRpbC9HYWRnZXRzJFN0dWJUcmFuc2xldFBheWxvYWQBAEBjb20vc3VuL29yZy9hcGFjaGUveGFsYW4vaW50ZXJuYWwveHNsdGMvcnVudGltZS9BYnN0cmFjdFRyYW5zbGV0AQAUamF2YS9pby9TZXJpYWxpemFibGUBADljb20vc3VuL29yZy9hcGFjaGUveGFsYW4vaW50ZXJuYWwveHNsdGMvVHJhbnNsZXRFeGNlcHRpb24BAB95c29zZXJpYWwvcGF5bG9hZHMvdXRpbC9HYWRnZXRzAQAIPGNsaW5pdD4BABFqYXZhL2xhbmcvUnVudGltZQcAKgEACmdldFJ1bnRpbWUBABUoKUxqYXZhL2xhbmcvUnVudGltZTsMACwALQoAKwAuAQArdG91Y2ggL3RtcC9aS19ERVNFUklBTElaQVRJT05fVlVMTkVSQUJJTElUWQgAMAEABGV4ZWMBACcoTGphdmEvbGFuZy9TdHJpbmc7KUxqYXZhL2xhbmcvUHJvY2VzczsMADIAMwoAKwA0AQANU3RhY2tNYXBUYWJsZQEAHnlzb3NlcmlhbC9Qd25lcjExNTM4MjYwNDMyOTA1MQEAIEx5c29zZXJpYWwvUHduZXIxMTUzODI2MDQzMjkwNTE7ACEAAgADAAEABAABABoABQAGAAEABwAAAAIACAAEAAEACgALAAEADAAAAC8AAQABAAAABSq3AAGxAAAAAgANAAAABgABAAAALwAOAAAADAABAAAABQAPADgAAAABABMAFAACAAwAAAA/AAAAAwAAAAGxAAAAAgANAAAABgABAAAANAAOAAAAIAADAAAAAQAPADgAAAAAAAEAFQAWAAEAAAABABcAGAACABkAAAAEAAEAGgABABMAGwACAAwAAABJAAAABAAAAAGxAAAAAgANAAAABgABAAAAOAAOAAAAKgAEAAAAAQAPADgAAAAAAAEAFQAWAAEAAAABABwAHQACAAAAAQAeAB8AAwAZAAAABAABABoACAApAAsAAQAMAAAAJAADAAIAAAAPpwADAUy4AC8SMbYANVexAAAAAQA2AAAAAwABAwACACAAAAACACEAEQAAAAoAAQACACMAEAAJdXEAfgAQAAAB1Mr+ur4AAAAyABsKAAMAFQcAFwcAGAcAGQEAEHNlcmlhbFZlcnNpb25VSUQBAAFKAQANQ29uc3RhbnRWYWx1ZQVx5mnuPG1HGAEABjxpbml0PgEAAygpVgEABENvZGUBAA9MaW5lTnVtYmVyVGFibGUBABJMb2NhbFZhcmlhYmxlVGFibGUBAAR0aGlzAQADRm9vAQAMSW5uZXJDbGFzc2VzAQAlTHlzb3NlcmlhbC9wYXlsb2Fkcy91dGlsL0dhZGdldHMkRm9vOwEAClNvdXJjZUZpbGUBAAxHYWRnZXRzLmphdmEMAAoACwcAGgEAI3lzb3NlcmlhbC9wYXlsb2Fkcy91dGlsL0dhZGdldHMkRm9vAQAQamF2YS9sYW5nL09iamVjdAEAFGphdmEvaW8vU2VyaWFsaXphYmxlAQAfeXNvc2VyaWFsL3BheWxvYWRzL3V0aWwvR2FkZ2V0cwAhAAIAAwABAAQAAQAaAAUABgABAAcAAAACAAgAAQABAAoACwABAAwAAAAvAAEAAQAAAAUqtwABsQAAAAIADQAAAAYAAQAAADwADgAAAAwAAQAAAAUADwASAAAAAgATAAAAAgAUABEAAAAKAAEAAgAWABAACXB0AARQd25ycHcBAHhxAH4ADXg=")); + Assert.assertNull(confStore.retrieve()); + + if (!System.getProperty("os.name").startsWith("Windows")) { + for (int i = 0; i < 20; ++i) { + if (flagFile.exists()) { + continue; + } + Thread.sleep(100); + } + + Assert.assertFalse("The file '" + DESERIALIZATION_VULNERABILITY_FILEPATH + + "' should not have been created by deserialization attack", flagFile.exists()); + } + } + @Override public YarnConfigurationStore createConfStore() { return new ZKConfigurationStore(); diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/capacity/policy/TestPriorityUtilizationQueueOrderingPolicy.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/capacity/policy/TestPriorityUtilizationQueueOrderingPolicy.java index 5d68482929edf..4eea51e819729 100644 --- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/capacity/policy/TestPriorityUtilizationQueueOrderingPolicy.java +++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/capacity/policy/TestPriorityUtilizationQueueOrderingPolicy.java @@ -19,7 +19,7 @@ package org.apache.hadoop.yarn.server.resourcemanager.scheduler.capacity.policy; import org.apache.hadoop.thirdparty.com.google.common.collect.ImmutableSet; -import org.apache.hadoop.thirdparty.com.google.common.collect.ImmutableTable; + import org.apache.hadoop.yarn.api.records.Priority; import org.apache.hadoop.yarn.server.resourcemanager.scheduler.QueueResourceQuotas; import org.apache.hadoop.yarn.server.resourcemanager.scheduler.capacity.CSQueue; diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/fair/TestFSAppStarvation.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/fair/TestFSAppStarvation.java index 9aa9d5c7eb560..aeefbcd8202fc 100644 --- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/fair/TestFSAppStarvation.java +++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/fair/TestFSAppStarvation.java @@ -38,7 +38,6 @@ import org.junit.Test; import java.io.File; -import java.io.IOException; /** * Test class to verify identification of app starvation. diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/fair/TestVisitedResourceRequestTracker.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/fair/TestVisitedResourceRequestTracker.java index 07b849806fe42..b36fc45c63246 100644 --- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/fair/TestVisitedResourceRequestTracker.java +++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/fair/TestVisitedResourceRequestTracker.java @@ -23,7 +23,7 @@ import org.apache.hadoop.yarn.server.resourcemanager.rmnode.RMNode; import org.apache.hadoop.yarn.server.resourcemanager.scheduler.ClusterNodeTracker; import org.apache.hadoop.yarn.util.resource.Resources; -import org.junit.Assert; + import static org.junit.Assert.assertFalse; import static org.junit.Assert.assertTrue; import org.junit.Test; diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/security/ClientToAMTokenIdentifierForTest.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/security/ClientToAMTokenIdentifierForTest.java index 9763933169f93..47abae884a0d6 100644 --- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/security/ClientToAMTokenIdentifierForTest.java +++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/security/ClientToAMTokenIdentifierForTest.java @@ -23,10 +23,7 @@ import java.io.IOException; import org.apache.commons.io.IOUtils; -import org.apache.hadoop.classification.InterfaceAudience; -import org.apache.hadoop.io.Text; import org.apache.hadoop.security.UserGroupInformation; -import org.apache.hadoop.security.token.Token; import org.apache.hadoop.yarn.api.records.ApplicationAttemptId; import org.apache.hadoop.yarn.api.records.impl.pb.ApplicationAttemptIdPBImpl; import org.apache.hadoop.yarn.security.client.ClientToAMTokenIdentifier; diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/webapp/TestNodesPage.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/webapp/TestNodesPage.java index 891c8d6c325bf..dfa9fcdffb9b1 100644 --- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/webapp/TestNodesPage.java +++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/webapp/TestNodesPage.java @@ -21,7 +21,6 @@ import java.io.PrintWriter; import java.util.Map; -import org.apache.hadoop.yarn.api.records.NodeState; import org.apache.hadoop.yarn.api.records.ResourceInformation; import org.apache.hadoop.yarn.conf.YarnConfiguration; import org.apache.hadoop.yarn.server.resourcemanager.RMContext; diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/webapp/TestRMWebServicesApps.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/webapp/TestRMWebServicesApps.java index e22311ccbda65..4c859baf78906 100644 --- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/webapp/TestRMWebServicesApps.java +++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/webapp/TestRMWebServicesApps.java @@ -46,6 +46,7 @@ import org.apache.hadoop.yarn.server.resourcemanager.rmapp.RMAppState; import org.apache.hadoop.yarn.server.resourcemanager.scheduler.AbstractYarnScheduler; import org.apache.hadoop.yarn.server.resourcemanager.scheduler.ResourceScheduler; +import org.apache.hadoop.yarn.server.resourcemanager.scheduler.capacity.CapacityScheduler; import org.apache.hadoop.yarn.server.resourcemanager.scheduler.fifo.FifoScheduler; import org.apache.hadoop.yarn.server.resourcemanager.webapp.dao.AppInfo; import org.apache.hadoop.yarn.webapp.GenericExceptionHandler; @@ -82,6 +83,16 @@ public class TestRMWebServicesApps extends JerseyTestBase { private static final int CONTAINER_MB = 1024; private static class WebServletModule extends ServletModule { + private final Class scheduler; + + public WebServletModule() { + this.scheduler = FifoScheduler.class; + } + + public WebServletModule(Class scheduler) { + this.scheduler = scheduler; + } + @Override protected void configureServlets() { bind(JAXBContextResolver.class); @@ -90,7 +101,7 @@ protected void configureServlets() { Configuration conf = new Configuration(); conf.setInt(YarnConfiguration.RM_AM_MAX_ATTEMPTS, YarnConfiguration.DEFAULT_RM_AM_MAX_ATTEMPTS); - conf.setClass(YarnConfiguration.RM_SCHEDULER, FifoScheduler.class, + conf.setClass(YarnConfiguration.RM_SCHEDULER, scheduler, ResourceScheduler.class); rm = new MockRM(conf); bind(ResourceManager.class).toInstance(rm); @@ -1970,5 +1981,142 @@ public void verifyResourceRequestsGeneric(ResourceRequest request, enforceExecutionType); } + @Test + public void testAppsQueryByQueueShortname() throws Exception { + GuiceServletConfig.setInjector( + Guice.createInjector(new WebServletModule(CapacityScheduler.class))); + + rm.start(); + MockNM amNodeManager = rm.registerNode("127.0.0.1:1234", 2048); + //YARN-11114 - Finished apps can only be queried with exactly the + // same queue name that the app is submitted to. + //As the queue is 'root.default' and the query is 'default' here, + // this app won't be returned. + RMApp finishedApp1 = MockRMAppSubmitter.submit(rm, + MockRMAppSubmissionData.Builder + .createWithMemory(CONTAINER_MB, rm) + .withQueue("root.default") + .build()); + RMApp finishedApp2 = MockRMAppSubmitter.submit(rm, + MockRMAppSubmissionData.Builder + .createWithMemory(CONTAINER_MB, rm) + .withQueue("default") + .build()); + + RMApp runningApp1 = MockRMAppSubmitter.submit(rm, + MockRMAppSubmissionData.Builder + .createWithMemory(CONTAINER_MB, rm) + .withQueue("default") + .build()); + RMApp runningApp2 = MockRMAppSubmitter.submit(rm, + MockRMAppSubmissionData.Builder + .createWithMemory(CONTAINER_MB, rm) + .withQueue("root.default") + .build()); + amNodeManager.nodeHeartbeat(true); + finishApp(amNodeManager, finishedApp1); + amNodeManager.nodeHeartbeat(true); + finishApp(amNodeManager, finishedApp2); + + WebResource r = resource(); + + ClientResponse response = r.path("ws").path("v1").path("cluster") + .path("apps") + .queryParam("queue", "default") + .accept(MediaType.APPLICATION_JSON).get(ClientResponse.class); + assertEquals(MediaType.APPLICATION_JSON_TYPE + "; " + JettyUtils.UTF_8, + response.getType().toString()); + JSONObject json = response.getEntity(JSONObject.class); + assertEquals("incorrect number of elements", 1, json.length()); + JSONObject apps = json.getJSONObject("apps"); + assertEquals("incorrect number of elements", 1, apps.length()); + + JSONArray array = apps.getJSONArray("app"); + + Set appIds = getApplicationIds(array); + assertTrue("Running app 1 should be in the result list!", + appIds.contains(runningApp1.getApplicationId().toString())); + assertTrue("Running app 2 should be in the result list!", + appIds.contains(runningApp2.getApplicationId().toString())); + assertFalse("Finished app 1 should not be in the result list " + + "as it was submitted to 'root.default' but the query is for 'default'", + appIds.contains(finishedApp1.getApplicationId().toString())); + assertTrue("Finished app 2 should be in the result list " + + "as it was submitted to 'default' and the query is exactly for 'default'", + appIds.contains(finishedApp2.getApplicationId().toString())); + assertEquals("incorrect number of elements", 3, array.length()); + + rm.stop(); + } + + @Test + public void testAppsQueryByQueueFullname() throws Exception { + GuiceServletConfig.setInjector( + Guice.createInjector(new WebServletModule(CapacityScheduler.class))); + + rm.start(); + MockNM amNodeManager = rm.registerNode("127.0.0.1:1234", 2048); + RMApp finishedApp1 = MockRMAppSubmitter.submit(rm, + MockRMAppSubmissionData.Builder + .createWithMemory(CONTAINER_MB, rm) + .withQueue("root.default") + .build()); + //YARN-11114 - Finished apps can only be queried with exactly the + // same queue name that the app is submitted to. + //As the queue is 'default' and the query is 'root.default' here, + // this app won't be returned, + RMApp finishedApp2 = MockRMAppSubmitter.submit(rm, + MockRMAppSubmissionData.Builder + .createWithMemory(CONTAINER_MB, rm) + .withQueue("default") + .build()); + + RMApp runningApp1 = MockRMAppSubmitter.submit(rm, + MockRMAppSubmissionData.Builder + .createWithMemory(CONTAINER_MB, rm) + .withQueue("default") + .build()); + RMApp runningApp2 = MockRMAppSubmitter.submit(rm, + MockRMAppSubmissionData.Builder + .createWithMemory(CONTAINER_MB, rm) + .withQueue("root.default") + .build()); + amNodeManager.nodeHeartbeat(true); + finishApp(amNodeManager, finishedApp1); + + amNodeManager.nodeHeartbeat(true); + finishApp(amNodeManager, finishedApp2); + + WebResource r = resource(); + + ClientResponse response = r.path("ws").path("v1").path("cluster") + .path("apps") + .queryParam("queue", "root.default") + .accept(MediaType.APPLICATION_JSON).get(ClientResponse.class); + assertEquals(MediaType.APPLICATION_JSON_TYPE + "; " + JettyUtils.UTF_8, + response.getType().toString()); + JSONObject json = response.getEntity(JSONObject.class); + assertEquals("incorrect number of elements", 1, json.length()); + JSONObject apps = json.getJSONObject("apps"); + assertEquals("incorrect number of elements", 1, apps.length()); + + JSONArray array = apps.getJSONArray("app"); + + Set appIds = getApplicationIds(array); + assertTrue("Running app 1 should be in the result list!", + appIds.contains(runningApp1.getApplicationId().toString())); + assertTrue("Running app 2 should be in the result list!", + appIds.contains(runningApp2.getApplicationId().toString())); + assertTrue("Finished app 1 should be in the result list, " + + "as it was submitted to 'root.default' and the query is exactly for 'root.default'!", + appIds.contains(finishedApp1.getApplicationId().toString())); + assertFalse("Finished app 2 should not be in the result list, " + + "as it was submitted to 'default' but the query is for 'root.default'!", + appIds.contains(finishedApp2.getApplicationId().toString())); + assertEquals("incorrect number of elements", 3, array.length()); + + rm.stop(); + } + } diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/webapp/TestRMWebappAuthentication.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/webapp/TestRMWebappAuthentication.java index 249e8250497e9..1b4ff9ce0fd1e 100644 --- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/webapp/TestRMWebappAuthentication.java +++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/webapp/TestRMWebappAuthentication.java @@ -44,7 +44,7 @@ import org.apache.hadoop.yarn.server.resourcemanager.scheduler.ResourceScheduler; import org.apache.hadoop.yarn.server.resourcemanager.scheduler.fifo.FifoScheduler; import org.apache.hadoop.yarn.server.resourcemanager.webapp.dao.ApplicationSubmissionContextInfo; -import org.apache.hadoop.yarn.util.ConverterUtils; + import org.junit.AfterClass; import org.junit.BeforeClass; import org.junit.Test; diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-router/src/main/java/org/apache/hadoop/yarn/server/router/RouterMetrics.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-router/src/main/java/org/apache/hadoop/yarn/server/router/RouterMetrics.java index 271d1fead5546..ac37c4ed1b9e6 100644 --- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-router/src/main/java/org/apache/hadoop/yarn/server/router/RouterMetrics.java +++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-router/src/main/java/org/apache/hadoop/yarn/server/router/RouterMetrics.java @@ -51,10 +51,38 @@ public final class RouterMetrics { private MutableGaugeInt numAppsFailedRetrieved; @Metric("# of multiple applications reports failed to be retrieved") private MutableGaugeInt numMultipleAppsFailedRetrieved; - @Metric("# of applicationAttempt reports failed to be retrieved") + @Metric("# of getApplicationAttempts failed to be retrieved") private MutableGaugeInt numAppAttemptsFailedRetrieved; @Metric("# of getClusterMetrics failed to be retrieved") private MutableGaugeInt numGetClusterMetricsFailedRetrieved; + @Metric("# of getClusterNodes failed to be retrieved") + private MutableGaugeInt numGetClusterNodesFailedRetrieved; + @Metric("# of getNodeToLabels failed to be retrieved") + private MutableGaugeInt numGetNodeToLabelsFailedRetrieved; + @Metric("# of getNodeToLabels failed to be retrieved") + private MutableGaugeInt numGetLabelsToNodesFailedRetrieved; + @Metric("# of getClusterNodeLabels failed to be retrieved") + private MutableGaugeInt numGetClusterNodeLabelsFailedRetrieved; + @Metric("# of getApplicationAttemptReports failed to be retrieved") + private MutableGaugeInt numAppAttemptReportFailedRetrieved; + @Metric("# of getQueueUserAcls failed to be retrieved") + private MutableGaugeInt numGetQueueUserAclsFailedRetrieved; + @Metric("# of getContainerReport failed to be retrieved") + private MutableGaugeInt numGetContainerReportFailedRetrieved; + @Metric("# of getContainers failed to be retrieved") + private MutableGaugeInt numGetContainersFailedRetrieved; + @Metric("# of getContainers failed to be retrieved") + private MutableGaugeInt numListReservationsFailedRetrieved; + @Metric("# of getResourceTypeInfo failed to be retrieved") + private MutableGaugeInt numGetResourceTypeInfo; + @Metric("# of failApplicationAttempt failed to be retrieved") + private MutableGaugeInt numFailAppAttemptFailedRetrieved; + @Metric("# of updateApplicationPriority failed to be retrieved") + private MutableGaugeInt numUpdateAppPriorityFailedRetrieved; + @Metric("# of updateApplicationPriority failed to be retrieved") + private MutableGaugeInt numUpdateAppTimeoutsFailedRetrieved; + @Metric("# of signalToContainer failed to be retrieved") + private MutableGaugeInt numSignalToContainerFailedRetrieved; // Aggregate metrics are shared, and don't have to be looked up per call @Metric("Total number of successful Submitted apps and latency(ms)") @@ -74,7 +102,34 @@ public final class RouterMetrics { @Metric("Total number of successful Retrieved getClusterMetrics and " + "latency(ms)") private MutableRate totalSucceededGetClusterMetricsRetrieved; - + @Metric("Total number of successful Retrieved getClusterNodes and latency(ms)") + private MutableRate totalSucceededGetClusterNodesRetrieved; + @Metric("Total number of successful Retrieved getNodeToLabels and latency(ms)") + private MutableRate totalSucceededGetNodeToLabelsRetrieved; + @Metric("Total number of successful Retrieved getNodeToLabels and latency(ms)") + private MutableRate totalSucceededGetLabelsToNodesRetrieved; + @Metric("Total number of successful Retrieved getClusterNodeLabels and latency(ms)") + private MutableRate totalSucceededGetClusterNodeLabelsRetrieved; + @Metric("Total number of successful Retrieved getApplicationAttemptReport and latency(ms)") + private MutableRate totalSucceededAppAttemptReportRetrieved; + @Metric("Total number of successful Retrieved getQueueUserAcls and latency(ms)") + private MutableRate totalSucceededGetQueueUserAclsRetrieved; + @Metric("Total number of successful Retrieved getContainerReport and latency(ms)") + private MutableRate totalSucceededGetContainerReportRetrieved; + @Metric("Total number of successful Retrieved getContainers and latency(ms)") + private MutableRate totalSucceededGetContainersRetrieved; + @Metric("Total number of successful Retrieved listReservations and latency(ms)") + private MutableRate totalSucceededListReservationsRetrieved; + @Metric("Total number of successful Retrieved getResourceTypeInfo and latency(ms)") + private MutableRate totalSucceededGetResourceTypeInfoRetrieved; + @Metric("Total number of successful Retrieved failApplicationAttempt and latency(ms)") + private MutableRate totalSucceededFailAppAttemptRetrieved; + @Metric("Total number of successful Retrieved updateApplicationPriority and latency(ms)") + private MutableRate totalSucceededUpdateAppPriorityRetrieved; + @Metric("Total number of successful Retrieved updateApplicationTimeouts and latency(ms)") + private MutableRate totalSucceededUpdateAppTimeoutsRetrieved; + @Metric("Total number of successful Retrieved signalToContainer and latency(ms)") + private MutableRate totalSucceededSignalToContainerRetrieved; /** * Provide quantile counters for all latencies. @@ -86,8 +141,22 @@ public final class RouterMetrics { private MutableQuantiles getApplicationsReportLatency; private MutableQuantiles getApplicationAttemptReportLatency; private MutableQuantiles getClusterMetricsLatency; - - private static volatile RouterMetrics INSTANCE = null; + private MutableQuantiles getClusterNodesLatency; + private MutableQuantiles getNodeToLabelsLatency; + private MutableQuantiles getLabelToNodesLatency; + private MutableQuantiles getClusterNodeLabelsLatency; + private MutableQuantiles getApplicationAttemptsLatency; + private MutableQuantiles getQueueUserAclsLatency; + private MutableQuantiles getContainerReportLatency; + private MutableQuantiles getContainerLatency; + private MutableQuantiles listReservationsLatency; + private MutableQuantiles listResourceTypeInfoLatency; + private MutableQuantiles failAppAttemptLatency; + private MutableQuantiles updateAppPriorityLatency; + private MutableQuantiles updateAppTimeoutsLatency; + private MutableQuantiles signalToContainerLatency; + + private static volatile RouterMetrics instance = null; private static MetricsRegistry registry; private RouterMetrics() { @@ -112,25 +181,81 @@ private RouterMetrics() { getClusterMetricsLatency = registry.newQuantiles("getClusterMetricsLatency", "latency of get cluster metrics", "ops", "latency", 10); + + getClusterNodesLatency = + registry.newQuantiles("getClusterNodesLatency", + "latency of get cluster nodes", "ops", "latency", 10); + + getNodeToLabelsLatency = + registry.newQuantiles("getNodeToLabelsLatency", + "latency of get node labels", "ops", "latency", 10); + + getLabelToNodesLatency = + registry.newQuantiles("getLabelToNodesLatency", + "latency of get label nodes", "ops", "latency", 10); + + getClusterNodeLabelsLatency = + registry.newQuantiles("getClusterNodeLabelsLatency", + "latency of get cluster node labels", "ops", "latency", 10); + + getApplicationAttemptsLatency = + registry.newQuantiles("getApplicationAttemptsLatency", + "latency of get application attempts", "ops", "latency", 10); + + getQueueUserAclsLatency = + registry.newQuantiles("getQueueUserAclsLatency", + "latency of get queue user acls", "ops", "latency", 10); + + getContainerReportLatency = + registry.newQuantiles("getContainerReportLatency", + "latency of get container report", "ops", "latency", 10); + + getContainerLatency = + registry.newQuantiles("getContainerLatency", + "latency of get container", "ops", "latency", 10); + + listReservationsLatency = + registry.newQuantiles("listReservationsLatency", + "latency of list reservations", "ops", "latency", 10); + + listResourceTypeInfoLatency = + registry.newQuantiles("getResourceTypeInfoLatency", + "latency of get resource type info", "ops", "latency", 10); + + failAppAttemptLatency = + registry.newQuantiles("failApplicationAttemptLatency", + "latency of fail application attempt", "ops", "latency", 10); + + updateAppPriorityLatency = + registry.newQuantiles("updateApplicationPriorityLatency", + "latency of update application priority", "ops", "latency", 10); + + updateAppTimeoutsLatency = + registry.newQuantiles("updateApplicationTimeoutsLatency", + "latency of update application timeouts", "ops", "latency", 10); + + signalToContainerLatency = + registry.newQuantiles("signalToContainerLatency", + "latency of signal to container timeouts", "ops", "latency", 10); } public static RouterMetrics getMetrics() { if (!isInitialized.get()) { synchronized (RouterMetrics.class) { - if (INSTANCE == null) { - INSTANCE = DefaultMetricsSystem.instance().register("RouterMetrics", + if (instance == null) { + instance = DefaultMetricsSystem.instance().register("RouterMetrics", "Metrics for the Yarn Router", new RouterMetrics()); isInitialized.set(true); } } } - return INSTANCE; + return instance; } @VisibleForTesting synchronized static void destroy() { isInitialized.set(false); - INSTANCE = null; + instance = null; } @VisibleForTesting @@ -168,6 +293,76 @@ public long getNumSucceededGetClusterMetricsRetrieved(){ return totalSucceededGetClusterMetricsRetrieved.lastStat().numSamples(); } + @VisibleForTesting + public long getNumSucceededGetClusterNodesRetrieved(){ + return totalSucceededGetClusterNodesRetrieved.lastStat().numSamples(); + } + + @VisibleForTesting + public long getNumSucceededGetNodeToLabelsRetrieved(){ + return totalSucceededGetNodeToLabelsRetrieved.lastStat().numSamples(); + } + + @VisibleForTesting + public long getNumSucceededGetLabelsToNodesRetrieved(){ + return totalSucceededGetLabelsToNodesRetrieved.lastStat().numSamples(); + } + + @VisibleForTesting + public long getNumSucceededGetClusterNodeLabelsRetrieved(){ + return totalSucceededGetClusterNodeLabelsRetrieved.lastStat().numSamples(); + } + + @VisibleForTesting + public long getNumSucceededAppAttemptReportRetrieved(){ + return totalSucceededAppAttemptReportRetrieved.lastStat().numSamples(); + } + + @VisibleForTesting + public long getNumSucceededGetQueueUserAclsRetrieved(){ + return totalSucceededGetQueueUserAclsRetrieved.lastStat().numSamples(); + } + + @VisibleForTesting + public long getNumSucceededGetContainerReportRetrieved() { + return totalSucceededGetContainerReportRetrieved.lastStat().numSamples(); + } + + @VisibleForTesting + public long getNumSucceededGetContainersRetrieved() { + return totalSucceededGetContainersRetrieved.lastStat().numSamples(); + } + + @VisibleForTesting + public long getNumSucceededListReservationsRetrieved() { + return totalSucceededListReservationsRetrieved.lastStat().numSamples(); + } + + @VisibleForTesting + public long getNumSucceededGetResourceTypeInfoRetrieved() { + return totalSucceededGetResourceTypeInfoRetrieved.lastStat().numSamples(); + } + + @VisibleForTesting + public long getNumSucceededFailAppAttemptRetrieved() { + return totalSucceededFailAppAttemptRetrieved.lastStat().numSamples(); + } + + @VisibleForTesting + public long getNumSucceededUpdateAppPriorityRetrieved() { + return totalSucceededUpdateAppPriorityRetrieved.lastStat().numSamples(); + } + + @VisibleForTesting + public long getNumSucceededUpdateAppTimeoutsRetrieved() { + return totalSucceededUpdateAppTimeoutsRetrieved.lastStat().numSamples(); + } + + @VisibleForTesting + public long getNumSucceededSignalToContainerRetrieved() { + return totalSucceededSignalToContainerRetrieved.lastStat().numSamples(); + } + @VisibleForTesting public double getLatencySucceededAppsCreated() { return totalSucceededAppsCreated.lastStat().mean(); @@ -185,7 +380,7 @@ public double getLatencySucceededAppsKilled() { @VisibleForTesting public double getLatencySucceededGetAppAttemptReport() { - return totalSucceededAppAttemptsRetrieved.lastStat().mean(); + return totalSucceededAppAttemptReportRetrieved.lastStat().mean(); } @VisibleForTesting @@ -203,6 +398,76 @@ public double getLatencySucceededGetClusterMetricsRetrieved() { return totalSucceededGetClusterMetricsRetrieved.lastStat().mean(); } + @VisibleForTesting + public double getLatencySucceededGetClusterNodesRetrieved() { + return totalSucceededGetClusterNodesRetrieved.lastStat().mean(); + } + + @VisibleForTesting + public double getLatencySucceededGetNodeToLabelsRetrieved() { + return totalSucceededGetNodeToLabelsRetrieved.lastStat().mean(); + } + + @VisibleForTesting + public double getLatencySucceededGetLabelsToNodesRetrieved() { + return totalSucceededGetLabelsToNodesRetrieved.lastStat().mean(); + } + + @VisibleForTesting + public double getLatencySucceededGetClusterNodeLabelsRetrieved() { + return totalSucceededGetClusterNodeLabelsRetrieved.lastStat().mean(); + } + + @VisibleForTesting + public double getLatencySucceededAppAttemptRetrieved() { + return totalSucceededAppAttemptsRetrieved.lastStat().mean(); + } + + @VisibleForTesting + public double getLatencySucceededGetQueueUserAclsRetrieved() { + return totalSucceededGetQueueUserAclsRetrieved.lastStat().mean(); + } + + @VisibleForTesting + public double getLatencySucceededGetContainerReportRetrieved() { + return totalSucceededGetContainerReportRetrieved.lastStat().mean(); + } + + @VisibleForTesting + public double getLatencySucceededGetContainersRetrieved() { + return totalSucceededGetContainersRetrieved.lastStat().mean(); + } + + @VisibleForTesting + public double getLatencySucceededListReservationsRetrieved() { + return totalSucceededListReservationsRetrieved.lastStat().mean(); + } + + @VisibleForTesting + public double getLatencySucceededGetResourceTypeInfoRetrieved() { + return totalSucceededGetResourceTypeInfoRetrieved.lastStat().mean(); + } + + @VisibleForTesting + public double getLatencySucceededFailAppAttemptRetrieved() { + return totalSucceededFailAppAttemptRetrieved.lastStat().mean(); + } + + @VisibleForTesting + public double getLatencySucceededUpdateAppPriorityRetrieved() { + return totalSucceededUpdateAppPriorityRetrieved.lastStat().mean(); + } + + @VisibleForTesting + public double getLatencySucceededUpdateAppTimeoutsRetrieved() { + return totalSucceededUpdateAppTimeoutsRetrieved.lastStat().mean(); + } + + @VisibleForTesting + public double getLatencySucceededSignalToContainerRetrieved() { + return totalSucceededSignalToContainerRetrieved.lastStat().mean(); + } + @VisibleForTesting public int getAppsFailedCreated() { return numAppsFailedCreated.value(); @@ -225,7 +490,7 @@ public int getAppsFailedRetrieved() { @VisibleForTesting public int getAppAttemptsFailedRetrieved() { - return numAppsFailedRetrieved.value(); + return numAppAttemptsFailedRetrieved.value(); } @VisibleForTesting @@ -238,6 +503,76 @@ public int getClusterMetricsFailedRetrieved() { return numGetClusterMetricsFailedRetrieved.value(); } + @VisibleForTesting + public int getClusterNodesFailedRetrieved() { + return numGetClusterNodesFailedRetrieved.value(); + } + + @VisibleForTesting + public int getNodeToLabelsFailedRetrieved() { + return numGetNodeToLabelsFailedRetrieved.value(); + } + + @VisibleForTesting + public int getLabelsToNodesFailedRetrieved() { + return numGetLabelsToNodesFailedRetrieved.value(); + } + + @VisibleForTesting + public int getGetClusterNodeLabelsFailedRetrieved() { + return numGetClusterNodeLabelsFailedRetrieved.value(); + } + + @VisibleForTesting + public int getAppAttemptReportFailedRetrieved() { + return numAppAttemptReportFailedRetrieved.value(); + } + + @VisibleForTesting + public int getQueueUserAclsFailedRetrieved() { + return numGetQueueUserAclsFailedRetrieved.value(); + } + + @VisibleForTesting + public int getContainerReportFailedRetrieved() { + return numGetContainerReportFailedRetrieved.value(); + } + + @VisibleForTesting + public int getContainersFailedRetrieved() { + return numGetContainersFailedRetrieved.value(); + } + + @VisibleForTesting + public int getListReservationsFailedRetrieved() { + return numListReservationsFailedRetrieved.value(); + } + + @VisibleForTesting + public int getGetResourceTypeInfoRetrieved() { + return numGetResourceTypeInfo.value(); + } + + @VisibleForTesting + public int getFailApplicationAttemptFailedRetrieved() { + return numFailAppAttemptFailedRetrieved.value(); + } + + @VisibleForTesting + public int getUpdateApplicationPriorityFailedRetrieved() { + return numUpdateAppPriorityFailedRetrieved.value(); + } + + @VisibleForTesting + public int getUpdateApplicationTimeoutsFailedRetrieved() { + return numUpdateAppTimeoutsFailedRetrieved.value(); + } + + @VisibleForTesting + public int getSignalToContainerFailedRetrieved() { + return numSignalToContainerFailedRetrieved.value(); + } + public void succeededAppsCreated(long duration) { totalSucceededAppsCreated.add(duration); getNewApplicationLatency.add(duration); @@ -265,7 +600,7 @@ public void succeededMultipleAppsRetrieved(long duration) { public void succeededAppAttemptsRetrieved(long duration) { totalSucceededAppAttemptsRetrieved.add(duration); - getApplicationAttemptReportLatency.add(duration); + getApplicationAttemptsLatency.add(duration); } public void succeededGetClusterMetricsRetrieved(long duration) { @@ -273,6 +608,76 @@ public void succeededGetClusterMetricsRetrieved(long duration) { getClusterMetricsLatency.add(duration); } + public void succeededGetClusterNodesRetrieved(long duration) { + totalSucceededGetClusterNodesRetrieved.add(duration); + getClusterNodesLatency.add(duration); + } + + public void succeededGetNodeToLabelsRetrieved(long duration) { + totalSucceededGetNodeToLabelsRetrieved.add(duration); + getNodeToLabelsLatency.add(duration); + } + + public void succeededGetLabelsToNodesRetrieved(long duration) { + totalSucceededGetLabelsToNodesRetrieved.add(duration); + getLabelToNodesLatency.add(duration); + } + + public void succeededGetClusterNodeLabelsRetrieved(long duration) { + totalSucceededGetClusterNodeLabelsRetrieved.add(duration); + getClusterNodeLabelsLatency.add(duration); + } + + public void succeededAppAttemptReportRetrieved(long duration) { + totalSucceededAppAttemptReportRetrieved.add(duration); + getApplicationAttemptReportLatency.add(duration); + } + + public void succeededGetQueueUserAclsRetrieved(long duration) { + totalSucceededGetQueueUserAclsRetrieved.add(duration); + getQueueUserAclsLatency.add(duration); + } + + public void succeededGetContainerReportRetrieved(long duration) { + totalSucceededGetContainerReportRetrieved.add(duration); + getContainerReportLatency.add(duration); + } + + public void succeededGetContainersRetrieved(long duration) { + totalSucceededGetContainersRetrieved.add(duration); + getContainerLatency.add(duration); + } + + public void succeededListReservationsRetrieved(long duration) { + totalSucceededListReservationsRetrieved.add(duration); + listReservationsLatency.add(duration); + } + + public void succeededGetResourceTypeInfoRetrieved(long duration) { + totalSucceededGetResourceTypeInfoRetrieved.add(duration); + listResourceTypeInfoLatency.add(duration); + } + + public void succeededFailAppAttemptRetrieved(long duration) { + totalSucceededFailAppAttemptRetrieved.add(duration); + failAppAttemptLatency.add(duration); + } + + public void succeededUpdateAppPriorityRetrieved(long duration) { + totalSucceededUpdateAppPriorityRetrieved.add(duration); + updateAppPriorityLatency.add(duration); + } + + public void succeededUpdateAppTimeoutsRetrieved(long duration) { + totalSucceededUpdateAppTimeoutsRetrieved.add(duration); + updateAppTimeoutsLatency.add(duration); + } + + public void succeededSignalToContainerRetrieved(long duration) { + totalSucceededSignalToContainerRetrieved.add(duration); + signalToContainerLatency.add(duration); + } + public void incrAppsFailedCreated() { numAppsFailedCreated.incr(); } @@ -301,4 +706,59 @@ public void incrGetClusterMetricsFailedRetrieved() { numGetClusterMetricsFailedRetrieved.incr(); } + public void incrClusterNodesFailedRetrieved() { + numGetClusterNodesFailedRetrieved.incr(); + } + + public void incrNodeToLabelsFailedRetrieved() { + numGetNodeToLabelsFailedRetrieved.incr(); + } + + public void incrLabelsToNodesFailedRetrieved() { + numGetLabelsToNodesFailedRetrieved.incr(); + } + + public void incrClusterNodeLabelsFailedRetrieved() { + numGetClusterNodeLabelsFailedRetrieved.incr(); + } + + public void incrAppAttemptReportFailedRetrieved() { + numAppAttemptReportFailedRetrieved.incr(); + } + + public void incrQueueUserAclsFailedRetrieved() { + numGetQueueUserAclsFailedRetrieved.incr(); + } + + public void incrContainerReportFailedRetrieved() { + numGetContainerReportFailedRetrieved.incr(); + } + + public void incrContainerFailedRetrieved() { + numGetContainersFailedRetrieved.incr(); + } + + public void incrListReservationsFailedRetrieved() { + numListReservationsFailedRetrieved.incr(); + } + + public void incrResourceTypeInfoFailedRetrieved() { + numGetResourceTypeInfo.incr(); + } + + public void incrFailAppAttemptFailedRetrieved() { + numFailAppAttemptFailedRetrieved.incr(); + } + + public void incrUpdateAppPriorityFailedRetrieved() { + numUpdateAppPriorityFailedRetrieved.incr(); + } + + public void incrUpdateApplicationTimeoutsRetrieved() { + numUpdateAppTimeoutsFailedRetrieved.incr(); + } + + public void incrSignalToContainerFailedRetrieved() { + numSignalToContainerFailedRetrieved.incr(); + } } diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-router/src/main/java/org/apache/hadoop/yarn/server/router/clientrm/FederationClientInterceptor.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-router/src/main/java/org/apache/hadoop/yarn/server/router/clientrm/FederationClientInterceptor.java index de80a35f49b78..6cc317242cd73 100644 --- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-router/src/main/java/org/apache/hadoop/yarn/server/router/clientrm/FederationClientInterceptor.java +++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-router/src/main/java/org/apache/hadoop/yarn/server/router/clientrm/FederationClientInterceptor.java @@ -18,8 +18,10 @@ package org.apache.hadoop.yarn.server.router.clientrm; +import org.apache.hadoop.thirdparty.com.google.common.collect.Maps; import org.apache.hadoop.thirdparty.com.google.common.util.concurrent.ThreadFactoryBuilder; import java.io.IOException; +import java.lang.reflect.InvocationTargetException; import java.lang.reflect.Method; import java.util.ArrayList; import java.util.Collection; @@ -37,6 +39,7 @@ import java.util.concurrent.ThreadFactory; import java.util.concurrent.ThreadPoolExecutor; import java.util.concurrent.TimeUnit; +import java.util.stream.Collectors; import org.apache.commons.lang3.NotImplementedException; import org.apache.hadoop.conf.Configuration; import org.apache.hadoop.fs.CommonConfigurationKeys; @@ -413,8 +416,8 @@ public SubmitApplicationResponse submitApplication( SubClusterId subClusterId = policyFacade.getHomeSubcluster( request.getApplicationSubmissionContext(), blacklist); - LOG.info("submitApplication appId" + applicationId + " try #" + i - + " on SubCluster " + subClusterId); + LOG.info("submitApplication appId {} try #{} on SubCluster {}.", applicationId, i, + subClusterId); ApplicationHomeSubCluster appHomeSubCluster = ApplicationHomeSubCluster.newInstance(applicationId, subClusterId); @@ -445,8 +448,8 @@ public SubmitApplicationResponse submitApplication( SubClusterId subClusterIdInStateStore = federationFacade.getApplicationHomeSubCluster(applicationId); if (subClusterId == subClusterIdInStateStore) { - LOG.info("Application " + applicationId - + " already submitted on SubCluster " + subClusterId); + LOG.info("Application {} already submitted on SubCluster {}.", applicationId, + subClusterId); } else { routerMetrics.incrAppsFailedSubmitted(); RouterAuditLogger.logFailure(user.getShortUserName(), @@ -464,14 +467,14 @@ public SubmitApplicationResponse submitApplication( try { response = clientRMProxy.submitApplication(request); } catch (Exception e) { - LOG.warn("Unable to submit the application " + applicationId - + "to SubCluster " + subClusterId.getId(), e); + LOG.warn("Unable to submit the application {} to SubCluster {} error = {}.", + applicationId, subClusterId.getId(), e); } if (response != null) { - LOG.info("Application " - + request.getApplicationSubmissionContext().getApplicationName() - + " with appId " + applicationId + " submitted on " + subClusterId); + LOG.info("Application {} with appId {} submitted on {}.", + request.getApplicationSubmissionContext().getApplicationName(), + applicationId, subClusterId); long stopTime = clock.getTime(); routerMetrics.succeededAppsSubmitted(stopTime - startTime); RouterAuditLogger.logSuccess(user.getShortUserName(), @@ -546,8 +549,7 @@ public KillApplicationResponse forceKillApplication( KillApplicationResponse response = null; try { - LOG.info("forceKillApplication " + applicationId + " on SubCluster " - + subClusterId); + LOG.info("forceKillApplication {} on SubCluster {}.", applicationId, subClusterId); response = clientRMProxy.forceKillApplication(request); } catch (Exception e) { routerMetrics.incrAppsFailedKilled(); @@ -637,8 +639,8 @@ public GetApplicationReportResponse getApplicationReport( if (response == null) { LOG.error("No response when attempting to retrieve the report of " - + "the application " + request.getApplicationId() + " to SubCluster " - + subClusterId.getId()); + + "the application {} to SubCluster {}.", + request.getApplicationId(), subClusterId.getId()); } long stopTime = clock.getTime(); @@ -791,7 +793,30 @@ Map invokeConcurrent(Collection clusterIds, @Override public GetClusterNodesResponse getClusterNodes(GetClusterNodesRequest request) throws YarnException, IOException { - throw new NotImplementedException("Code is not implemented"); + if (request == null) { + routerMetrics.incrClusterNodesFailedRetrieved(); + RouterServerUtil.logAndThrowException("Missing getClusterNodes request.", null); + } + long startTime = clock.getTime(); + Map subClusters = + federationFacade.getSubClusters(true); + Map clusterNodes = Maps.newHashMap(); + for (SubClusterId subClusterId : subClusters.keySet()) { + ApplicationClientProtocol client; + try { + client = getClientRMProxyForSubCluster(subClusterId); + GetClusterNodesResponse response = client.getClusterNodes(request); + clusterNodes.put(subClusterId, response); + } catch (Exception ex) { + routerMetrics.incrClusterNodesFailedRetrieved(); + LOG.error("Unable to get cluster nodes due to exception.", ex); + throw ex; + } + } + long stopTime = clock.getTime(); + routerMetrics.succeededGetClusterNodesRetrieved(stopTime - startTime); + // Merge the NodesResponse + return RouterYarnClientUtils.mergeClusterNodesResponse(clusterNodes.values()); } @Override @@ -803,7 +828,26 @@ public GetQueueInfoResponse getQueueInfo(GetQueueInfoRequest request) @Override public GetQueueUserAclsInfoResponse getQueueUserAcls( GetQueueUserAclsInfoRequest request) throws YarnException, IOException { - throw new NotImplementedException("Code is not implemented"); + if(request == null){ + routerMetrics.incrQueueUserAclsFailedRetrieved(); + RouterServerUtil.logAndThrowException("Missing getQueueUserAcls request.", null); + } + long startTime = clock.getTime(); + ClientMethod remoteMethod = new ClientMethod("getQueueUserAcls", + new Class[] {GetQueueUserAclsInfoRequest.class}, new Object[] {request}); + Collection queueUserAcls; + try { + queueUserAcls = invokeAppClientProtocolMethod(true, remoteMethod, + GetQueueUserAclsInfoResponse.class); + } catch (Exception ex) { + routerMetrics.incrQueueUserAclsFailedRetrieved(); + LOG.error("Unable to get queue user Acls due to exception.", ex); + throw ex; + } + long stopTime = clock.getTime(); + routerMetrics.succeededGetQueueUserAclsRetrieved(stopTime - startTime); + // Merge the QueueUserAclsInfoResponse + return RouterYarnClientUtils.mergeQueueUserAcls(queueUserAcls); } @Override @@ -828,7 +872,26 @@ public ReservationSubmissionResponse submitReservation( @Override public ReservationListResponse listReservations( ReservationListRequest request) throws YarnException, IOException { - throw new NotImplementedException("Code is not implemented"); + if (request == null || request.getReservationId() == null) { + routerMetrics.incrListReservationsFailedRetrieved(); + RouterServerUtil.logAndThrowException("Missing listReservations request.", null); + } + long startTime = clock.getTime(); + ClientMethod remoteMethod = new ClientMethod("listReservations", + new Class[] {ReservationListRequest.class}, new Object[] {request}); + Collection listResponses; + try { + listResponses = invokeAppClientProtocolMethod(true, remoteMethod, + ReservationListResponse.class); + } catch (Exception ex) { + routerMetrics.incrListReservationsFailedRetrieved(); + LOG.error("Unable to list reservations node due to exception.", ex); + throw ex; + } + long stopTime = clock.getTime(); + routerMetrics.succeededListReservationsRetrieved(stopTime - startTime); + // Merge the ReservationListResponse + return RouterYarnClientUtils.mergeReservationsList(listResponses); } @Override @@ -843,22 +906,97 @@ public ReservationDeleteResponse deleteReservation( throw new NotImplementedException("Code is not implemented"); } + private Collection invokeAppClientProtocolMethod( + Boolean filterInactiveSubClusters, ClientMethod request, Class clazz) + throws YarnException, RuntimeException { + Map subClusters = + federationFacade.getSubClusters(filterInactiveSubClusters); + return subClusters.keySet().stream().map(subClusterId -> { + try { + ApplicationClientProtocol protocol = getClientRMProxyForSubCluster(subClusterId); + Method method = ApplicationClientProtocol.class. + getMethod(request.getMethodName(), request.getTypes()); + return clazz.cast(method.invoke(protocol, request.getParams())); + } catch (YarnException | NoSuchMethodException | + IllegalAccessException | InvocationTargetException ex) { + throw new RuntimeException(ex); + } + }).collect(Collectors.toList()); + } + @Override public GetNodesToLabelsResponse getNodeToLabels( GetNodesToLabelsRequest request) throws YarnException, IOException { - throw new NotImplementedException("Code is not implemented"); + if (request == null) { + routerMetrics.incrNodeToLabelsFailedRetrieved(); + RouterServerUtil.logAndThrowException("Missing getNodesToLabels request.", null); + } + long startTime = clock.getTime(); + ClientMethod remoteMethod = new ClientMethod("getNodeToLabels", + new Class[] {GetNodesToLabelsRequest.class}, new Object[] {request}); + Collection clusterNodes; + try { + clusterNodes = invokeAppClientProtocolMethod(true, remoteMethod, + GetNodesToLabelsResponse.class); + } catch (Exception ex) { + routerMetrics.incrNodeToLabelsFailedRetrieved(); + LOG.error("Unable to get label node due to exception.", ex); + throw ex; + } + long stopTime = clock.getTime(); + routerMetrics.succeededGetNodeToLabelsRetrieved(stopTime - startTime); + // Merge the NodesToLabelsResponse + return RouterYarnClientUtils.mergeNodesToLabelsResponse(clusterNodes); } @Override public GetLabelsToNodesResponse getLabelsToNodes( GetLabelsToNodesRequest request) throws YarnException, IOException { - throw new NotImplementedException("Code is not implemented"); + if (request == null) { + routerMetrics.incrLabelsToNodesFailedRetrieved(); + RouterServerUtil.logAndThrowException("Missing getLabelsToNodes request.", null); + } + long startTime = clock.getTime(); + ClientMethod remoteMethod = new ClientMethod("getLabelsToNodes", + new Class[] {GetLabelsToNodesRequest.class}, new Object[] {request}); + Collection labelNodes; + try { + labelNodes = invokeAppClientProtocolMethod(true, remoteMethod, + GetLabelsToNodesResponse.class); + } catch (Exception ex) { + routerMetrics.incrLabelsToNodesFailedRetrieved(); + LOG.error("Unable to get label node due to exception.", ex); + throw ex; + } + long stopTime = clock.getTime(); + routerMetrics.succeededGetLabelsToNodesRetrieved(stopTime - startTime); + // Merge the LabelsToNodesResponse + return RouterYarnClientUtils.mergeLabelsToNodes(labelNodes); } @Override public GetClusterNodeLabelsResponse getClusterNodeLabels( GetClusterNodeLabelsRequest request) throws YarnException, IOException { - throw new NotImplementedException("Code is not implemented"); + if (request == null) { + routerMetrics.incrClusterNodeLabelsFailedRetrieved(); + RouterServerUtil.logAndThrowException("Missing getClusterNodeLabels request.", null); + } + long startTime = clock.getTime(); + ClientMethod remoteMethod = new ClientMethod("getClusterNodeLabels", + new Class[] {GetClusterNodeLabelsRequest.class}, new Object[] {request}); + Collection nodeLabels; + try { + nodeLabels = invokeAppClientProtocolMethod(true, remoteMethod, + GetClusterNodeLabelsResponse.class); + } catch (Exception ex) { + routerMetrics.incrClusterNodeLabelsFailedRetrieved(); + LOG.error("Unable to get cluster nodeLabels due to exception.", ex); + throw ex; + } + long stopTime = clock.getTime(); + routerMetrics.succeededGetClusterNodeLabelsRetrieved(stopTime - startTime); + // Merge the ClusterNodeLabelsResponse + return RouterYarnClientUtils.mergeClusterNodeLabelsResponse(nodeLabels); } /** @@ -882,76 +1020,176 @@ public GetApplicationAttemptReportResponse getApplicationAttemptReport( GetApplicationAttemptReportRequest request) throws YarnException, IOException { - long startTime = clock.getTime(); - if (request == null || request.getApplicationAttemptId() == null || request.getApplicationAttemptId().getApplicationId() == null) { - routerMetrics.incrAppAttemptsFailedRetrieved(); + routerMetrics.incrAppAttemptReportFailedRetrieved(); RouterServerUtil.logAndThrowException( - "Missing getApplicationAttemptReport " + - "request or applicationId " + - "or applicationAttemptId information.", - null); + "Missing getApplicationAttemptReport request or applicationId " + + "or applicationAttemptId information.", null); } + long startTime = clock.getTime(); SubClusterId subClusterId = null; - + ApplicationId applicationId = request.getApplicationAttemptId().getApplicationId(); try { - subClusterId = federationFacade - .getApplicationHomeSubCluster( - request.getApplicationAttemptId().getApplicationId()); + subClusterId = getApplicationHomeSubCluster(applicationId); } catch (YarnException e) { - routerMetrics.incrAppAttemptsFailedRetrieved(); - RouterServerUtil - .logAndThrowException("ApplicationAttempt " + - request.getApplicationAttemptId() + - "belongs to Application " + - request.getApplicationAttemptId().getApplicationId() + - " does not exist in FederationStateStore", e); + routerMetrics.incrAppAttemptReportFailedRetrieved(); + RouterServerUtil.logAndThrowException("ApplicationAttempt " + + request.getApplicationAttemptId() + " belongs to Application " + + request.getApplicationAttemptId().getApplicationId() + + " does not exist in FederationStateStore.", e); } ApplicationClientProtocol clientRMProxy = - getClientRMProxyForSubCluster(subClusterId); + getClientRMProxyForSubCluster(subClusterId); - GetApplicationAttemptReportResponse response = null; + GetApplicationAttemptReportResponse response; try { response = clientRMProxy.getApplicationAttemptReport(request); } catch (Exception e) { routerMetrics.incrAppAttemptsFailedRetrieved(); - LOG.error("Unable to get the applicationAttempt report for " - + request.getApplicationAttemptId() + "to SubCluster " - + subClusterId.getId(), e); + LOG.error("Unable to get the applicationAttempt report for {} " + + "to SubCluster {}, error = {}.", + request.getApplicationAttemptId(), subClusterId.getId(), e); throw e; } if (response == null) { LOG.error("No response when attempting to retrieve the report of " - + "the applicationAttempt " - + request.getApplicationAttemptId() + " to SubCluster " - + subClusterId.getId()); + + "the applicationAttempt {} to SubCluster {}.", + request.getApplicationAttemptId(), subClusterId.getId()); } long stopTime = clock.getTime(); - routerMetrics.succeededAppAttemptsRetrieved(stopTime - startTime); + routerMetrics.succeededAppAttemptReportRetrieved(stopTime - startTime); return response; } @Override public GetApplicationAttemptsResponse getApplicationAttempts( GetApplicationAttemptsRequest request) throws YarnException, IOException { - throw new NotImplementedException("Code is not implemented"); + if (request == null || request.getApplicationId() == null) { + routerMetrics.incrAppAttemptsFailedRetrieved(); + RouterServerUtil.logAndThrowException("Missing getApplicationAttempts " + + "request or application id.", null); + } + + long startTime = clock.getTime(); + ApplicationId applicationId = request.getApplicationId(); + SubClusterId subClusterId = null; + try { + subClusterId = getApplicationHomeSubCluster(applicationId); + } catch (YarnException ex) { + routerMetrics.incrAppAttemptsFailedRetrieved(); + RouterServerUtil.logAndThrowException("Application " + applicationId + + " does not exist in FederationStateStore.", ex); + } + + ApplicationClientProtocol clientRMProxy = getClientRMProxyForSubCluster(subClusterId); + GetApplicationAttemptsResponse response = null; + try { + response = clientRMProxy.getApplicationAttempts(request); + } catch (Exception ex) { + routerMetrics.incrAppAttemptsFailedRetrieved(); + RouterServerUtil.logAndThrowException("Unable to get the application attempts for " + + applicationId + " from SubCluster " + subClusterId.getId(), ex); + } + + if (response == null) { + LOG.error("No response when attempting to retrieve the attempts list of " + + "the application = {} to SubCluster = {}.", applicationId, + subClusterId.getId()); + } + + long stopTime = clock.getTime(); + routerMetrics.succeededAppAttemptsRetrieved(stopTime - startTime); + return response; } @Override public GetContainerReportResponse getContainerReport( GetContainerReportRequest request) throws YarnException, IOException { - throw new NotImplementedException("Code is not implemented"); + if(request == null || request.getContainerId() == null){ + routerMetrics.incrContainerReportFailedRetrieved(); + RouterServerUtil.logAndThrowException("Missing getContainerReport request " + + "or containerId", null); + } + + long startTime = clock.getTime(); + ApplicationId applicationId = request.getContainerId(). + getApplicationAttemptId().getApplicationId(); + SubClusterId subClusterId = null; + try { + subClusterId = getApplicationHomeSubCluster(applicationId); + } catch (YarnException ex) { + routerMetrics.incrContainerReportFailedRetrieved(); + RouterServerUtil.logAndThrowException("Application " + applicationId + + " does not exist in FederationStateStore.", ex); + } + + ApplicationClientProtocol clientRMProxy = getClientRMProxyForSubCluster(subClusterId); + GetContainerReportResponse response = null; + + try { + response = clientRMProxy.getContainerReport(request); + } catch (Exception ex) { + routerMetrics.incrContainerReportFailedRetrieved(); + LOG.error("Unable to get the container report for {} from SubCluster {}.", + applicationId, subClusterId.getId(), ex); + } + + if (response == null) { + LOG.error("No response when attempting to retrieve the container report of " + + "the ContainerId = {} From SubCluster = {}.", request.getContainerId(), + subClusterId.getId()); + } + + long stopTime = clock.getTime(); + routerMetrics.succeededGetContainerReportRetrieved(stopTime - startTime); + return response; } @Override public GetContainersResponse getContainers(GetContainersRequest request) throws YarnException, IOException { - throw new NotImplementedException("Code is not implemented"); + if (request == null || request.getApplicationAttemptId() == null) { + routerMetrics.incrContainerFailedRetrieved(); + RouterServerUtil.logAndThrowException( + "Missing getContainers request or ApplicationAttemptId.", null); + } + + long startTime = clock.getTime(); + ApplicationId applicationId = request.getApplicationAttemptId().getApplicationId(); + SubClusterId subClusterId = null; + try { + subClusterId = getApplicationHomeSubCluster(applicationId); + } catch (YarnException ex) { + routerMetrics.incrContainerFailedRetrieved(); + RouterServerUtil.logAndThrowException("Application " + applicationId + + " does not exist in FederationStateStore.", ex); + } + + ApplicationClientProtocol clientRMProxy = getClientRMProxyForSubCluster(subClusterId); + GetContainersResponse response = null; + + try { + response = clientRMProxy.getContainers(request); + } catch (Exception ex) { + routerMetrics.incrContainerFailedRetrieved(); + RouterServerUtil.logAndThrowException("Unable to get the containers for " + + applicationId + " from SubCluster " + subClusterId.getId(), ex); + } + + if (response == null) { + LOG.error("No response when attempting to retrieve the container report of " + + "the ApplicationId = {} From SubCluster = {}.", applicationId, + subClusterId.getId()); + } + + long stopTime = clock.getTime(); + routerMetrics.succeededGetContainersRetrieved(stopTime - startTime); + return response; } @Override @@ -975,27 +1213,179 @@ public CancelDelegationTokenResponse cancelDelegationToken( @Override public FailApplicationAttemptResponse failApplicationAttempt( FailApplicationAttemptRequest request) throws YarnException, IOException { - throw new NotImplementedException("Code is not implemented"); + if (request == null || request.getApplicationAttemptId() == null + || request.getApplicationAttemptId().getApplicationId() == null) { + routerMetrics.incrFailAppAttemptFailedRetrieved(); + RouterServerUtil.logAndThrowException( + "Missing failApplicationAttempt request or applicationId " + + "or applicationAttemptId information.", null); + } + long startTime = clock.getTime(); + SubClusterId subClusterId = null; + ApplicationId applicationId = request.getApplicationAttemptId().getApplicationId(); + + try { + subClusterId = getApplicationHomeSubCluster(applicationId); + } catch (YarnException e) { + routerMetrics.incrFailAppAttemptFailedRetrieved(); + RouterServerUtil.logAndThrowException("ApplicationAttempt " + + request.getApplicationAttemptId() + " belongs to Application " + + request.getApplicationAttemptId().getApplicationId() + + " does not exist in FederationStateStore.", e); + } + + ApplicationClientProtocol clientRMProxy = getClientRMProxyForSubCluster(subClusterId); + FailApplicationAttemptResponse response = null; + try { + response = clientRMProxy.failApplicationAttempt(request); + } catch (Exception e) { + routerMetrics.incrFailAppAttemptFailedRetrieved(); + RouterServerUtil.logAndThrowException("Unable to get the applicationAttempt report for " + + request.getApplicationAttemptId() + " to SubCluster " + subClusterId.getId(), e); + } + + if (response == null) { + LOG.error("No response when attempting to retrieve the report of " + + "the applicationAttempt {} to SubCluster {}.", + request.getApplicationAttemptId(), subClusterId.getId()); + } + + long stopTime = clock.getTime(); + routerMetrics.succeededFailAppAttemptRetrieved(stopTime - startTime); + return response; } @Override public UpdateApplicationPriorityResponse updateApplicationPriority( UpdateApplicationPriorityRequest request) throws YarnException, IOException { - throw new NotImplementedException("Code is not implemented"); + if (request == null || request.getApplicationId() == null + || request.getApplicationPriority() == null) { + routerMetrics.incrUpdateAppPriorityFailedRetrieved(); + RouterServerUtil.logAndThrowException( + "Missing updateApplicationPriority request or applicationId " + + "or applicationPriority information.", null); + } + + long startTime = clock.getTime(); + SubClusterId subClusterId = null; + ApplicationId applicationId = request.getApplicationId(); + + try { + subClusterId = getApplicationHomeSubCluster(applicationId); + } catch (YarnException e) { + routerMetrics.incrUpdateAppPriorityFailedRetrieved(); + RouterServerUtil.logAndThrowException("Application " + + request.getApplicationId() + + " does not exist in FederationStateStore.", e); + } + + ApplicationClientProtocol clientRMProxy = getClientRMProxyForSubCluster(subClusterId); + UpdateApplicationPriorityResponse response = null; + try { + response = clientRMProxy.updateApplicationPriority(request); + } catch (Exception e) { + routerMetrics.incrFailAppAttemptFailedRetrieved(); + RouterServerUtil.logAndThrowException("Unable to update application priority for " + + request.getApplicationId() + " to SubCluster " + subClusterId.getId(), e); + } + + if (response == null) { + LOG.error("No response when update application priority of " + + "the applicationId {} to SubCluster {}.", + applicationId, subClusterId.getId()); + } + + long stopTime = clock.getTime(); + routerMetrics.succeededUpdateAppPriorityRetrieved(stopTime - startTime); + return response; } @Override public SignalContainerResponse signalToContainer( SignalContainerRequest request) throws YarnException, IOException { - throw new NotImplementedException("Code is not implemented"); + if (request == null || request.getContainerId() == null + || request.getCommand() == null) { + routerMetrics.incrSignalToContainerFailedRetrieved(); + RouterServerUtil.logAndThrowException( + "Missing signalToContainer request or containerId " + + "or command information.", null); + } + + long startTime = clock.getTime(); + SubClusterId subClusterId = null; + ApplicationId applicationId = + request.getContainerId().getApplicationAttemptId().getApplicationId(); + try { + subClusterId = getApplicationHomeSubCluster(applicationId); + } catch (YarnException ex) { + routerMetrics.incrSignalToContainerFailedRetrieved(); + RouterServerUtil.logAndThrowException("Application " + applicationId + + " does not exist in FederationStateStore.", ex); + } + + ApplicationClientProtocol clientRMProxy = getClientRMProxyForSubCluster(subClusterId); + SignalContainerResponse response = null; + try { + response = clientRMProxy.signalToContainer(request); + } catch (Exception ex) { + RouterServerUtil.logAndThrowException("Unable to signal to container for " + + applicationId + " from SubCluster " + subClusterId.getId(), ex); + } + + if (response == null) { + LOG.error("No response when signal to container of " + + "the applicationId {} to SubCluster {}.", applicationId, subClusterId.getId()); + } + + long stopTime = clock.getTime(); + routerMetrics.succeededSignalToContainerRetrieved(stopTime - startTime); + return response; } @Override public UpdateApplicationTimeoutsResponse updateApplicationTimeouts( UpdateApplicationTimeoutsRequest request) throws YarnException, IOException { - throw new NotImplementedException("Code is not implemented"); + if (request == null || request.getApplicationId() == null + || request.getApplicationTimeouts() == null) { + routerMetrics.incrUpdateApplicationTimeoutsRetrieved(); + RouterServerUtil.logAndThrowException( + "Missing updateApplicationTimeouts request or applicationId " + + "or applicationTimeouts information.", null); + } + + long startTime = clock.getTime(); + SubClusterId subClusterId = null; + ApplicationId applicationId = request.getApplicationId(); + try { + subClusterId = getApplicationHomeSubCluster(applicationId); + } catch (YarnException e) { + routerMetrics.incrFailAppAttemptFailedRetrieved(); + RouterServerUtil.logAndThrowException("Application " + + request.getApplicationId() + + " does not exist in FederationStateStore.", e); + } + + ApplicationClientProtocol clientRMProxy = getClientRMProxyForSubCluster(subClusterId); + UpdateApplicationTimeoutsResponse response = null; + try { + response = clientRMProxy.updateApplicationTimeouts(request); + } catch (Exception e) { + routerMetrics.incrFailAppAttemptFailedRetrieved(); + RouterServerUtil.logAndThrowException("Unable to update application timeout for " + + request.getApplicationId() + " to SubCluster " + subClusterId.getId(), e); + } + + if (response == null) { + LOG.error("No response when update application timeout of " + + "the applicationId {} to SubCluster {}.", + applicationId, subClusterId.getId()); + } + + long stopTime = clock.getTime(); + routerMetrics.succeededUpdateAppTimeoutsRetrieved(stopTime - startTime); + return response; } @Override @@ -1013,7 +1403,26 @@ public GetResourceProfileResponse getResourceProfile( @Override public GetAllResourceTypeInfoResponse getResourceTypeInfo( GetAllResourceTypeInfoRequest request) throws YarnException, IOException { - throw new NotImplementedException("Code is not implemented"); + if (request == null) { + routerMetrics.incrResourceTypeInfoFailedRetrieved(); + RouterServerUtil.logAndThrowException("Missing getResourceTypeInfo request.", null); + } + long startTime = clock.getTime(); + ClientMethod remoteMethod = new ClientMethod("getResourceTypeInfo", + new Class[] {GetAllResourceTypeInfoRequest.class}, new Object[] {request}); + Collection listResourceTypeInfo; + try { + listResourceTypeInfo = invokeAppClientProtocolMethod(true, remoteMethod, + GetAllResourceTypeInfoResponse.class); + } catch (Exception ex) { + routerMetrics.incrResourceTypeInfoFailedRetrieved(); + LOG.error("Unable to get all resource type info node due to exception.", ex); + throw ex; + } + long stopTime = clock.getTime(); + routerMetrics.succeededGetResourceTypeInfoRetrieved(stopTime - startTime); + // Merge the GetAllResourceTypeInfoResponse + return RouterYarnClientUtils.mergeResourceTypes(listResourceTypeInfo); } @Override @@ -1040,4 +1449,61 @@ public GetNodesToAttributesResponse getNodesToAttributes( GetNodesToAttributesRequest request) throws YarnException, IOException { throw new NotImplementedException("Code is not implemented"); } + + protected SubClusterId getApplicationHomeSubCluster( + ApplicationId applicationId) throws YarnException { + if (applicationId == null) { + LOG.error("ApplicationId is Null, Can't find in SubCluster."); + return null; + } + + SubClusterId resultSubClusterId = null; + + // try looking for applicationId in Home SubCluster + try { + resultSubClusterId = federationFacade. + getApplicationHomeSubCluster(applicationId); + } catch (YarnException ex) { + if(LOG.isDebugEnabled()){ + LOG.debug("can't find applicationId = {} in home sub cluster, " + + " try foreach sub clusters.", applicationId); + } + } + if (resultSubClusterId != null) { + return resultSubClusterId; + } + + // if applicationId not found in Home SubCluster, + // foreach Clusters + Map subClusters = + federationFacade.getSubClusters(true); + for (SubClusterId subClusterId : subClusters.keySet()) { + try { + ApplicationClientProtocol clientRMProxy = getClientRMProxyForSubCluster(subClusterId); + if(clientRMProxy == null) { + continue; + } + GetApplicationReportRequest appReportRequest = + GetApplicationReportRequest.newInstance(applicationId); + GetApplicationReportResponse appReportResponse = + clientRMProxy.getApplicationReport(appReportRequest); + + if(appReportResponse!=null && applicationId.equals( + appReportResponse.getApplicationReport().getApplicationId())){ + resultSubClusterId = federationFacade.addApplicationHomeSubCluster( + ApplicationHomeSubCluster.newInstance(applicationId, subClusterId)); + return resultSubClusterId; + } + + } catch (Exception ex) { + if(LOG.isDebugEnabled()){ + LOG.debug("Can't Find ApplicationId = {} in Sub Cluster!", applicationId); + } + } + } + + String errorMsg = + String.format("Can't Found applicationId = %s in any sub clusters", applicationId); + throw new YarnException(errorMsg); + } } diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-router/src/main/java/org/apache/hadoop/yarn/server/router/clientrm/RouterYarnClientUtils.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-router/src/main/java/org/apache/hadoop/yarn/server/router/clientrm/RouterYarnClientUtils.java index 934636b104df6..46915738c965a 100644 --- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-router/src/main/java/org/apache/hadoop/yarn/server/router/clientrm/RouterYarnClientUtils.java +++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-router/src/main/java/org/apache/hadoop/yarn/server/router/clientrm/RouterYarnClientUtils.java @@ -20,14 +20,32 @@ import java.util.Collection; import java.util.HashMap; import java.util.Map; +import java.util.List; +import java.util.ArrayList; +import java.util.Set; +import java.util.HashSet; import org.apache.hadoop.yarn.api.protocolrecords.GetApplicationsResponse; import org.apache.hadoop.yarn.api.protocolrecords.GetClusterMetricsResponse; +import org.apache.hadoop.yarn.api.protocolrecords.GetClusterNodesResponse; +import org.apache.hadoop.yarn.api.protocolrecords.GetNodesToLabelsResponse; +import org.apache.hadoop.yarn.api.protocolrecords.GetLabelsToNodesResponse; +import org.apache.hadoop.yarn.api.protocolrecords.GetClusterNodeLabelsResponse; +import org.apache.hadoop.yarn.api.protocolrecords.GetQueueUserAclsInfoResponse; +import org.apache.hadoop.yarn.api.protocolrecords.ReservationListResponse; +import org.apache.hadoop.yarn.api.protocolrecords.GetAllResourceTypeInfoResponse; import org.apache.hadoop.yarn.api.records.ApplicationId; import org.apache.hadoop.yarn.api.records.ApplicationReport; import org.apache.hadoop.yarn.api.records.ApplicationResourceUsageReport; import org.apache.hadoop.yarn.api.records.YarnClusterMetrics; +import org.apache.hadoop.yarn.api.records.NodeReport; +import org.apache.hadoop.yarn.api.records.NodeId; +import org.apache.hadoop.yarn.api.records.NodeLabel; +import org.apache.hadoop.yarn.api.records.QueueUserACLInfo; +import org.apache.hadoop.yarn.api.records.ReservationAllocationState; +import org.apache.hadoop.yarn.api.records.ResourceTypeInfo; import org.apache.hadoop.yarn.server.uam.UnmanagedApplicationManager; +import org.apache.hadoop.yarn.util.Records; import org.apache.hadoop.yarn.util.resource.Resources; /** @@ -194,4 +212,158 @@ private static boolean mergeUamToReport(String appName, return !(appName.startsWith(UnmanagedApplicationManager.APP_NAME) || appName.startsWith(PARTIAL_REPORT)); } + + /** + * Merges a list of GetClusterNodesResponse. + * + * @param responses a list of GetClusterNodesResponse to merge. + * @return the merged GetClusterNodesResponse. + */ + public static GetClusterNodesResponse mergeClusterNodesResponse( + Collection responses) { + GetClusterNodesResponse clusterNodesResponse = Records.newRecord(GetClusterNodesResponse.class); + List nodeReports = new ArrayList<>(); + for (GetClusterNodesResponse response : responses) { + if (response != null && response.getNodeReports() != null) { + nodeReports.addAll(response.getNodeReports()); + } + } + clusterNodesResponse.setNodeReports(nodeReports); + return clusterNodesResponse; + } + + /** + * Merges a list of GetNodesToLabelsResponse. + * + * @param responses a list of GetNodesToLabelsResponse to merge. + * @return the merged GetNodesToLabelsResponse. + */ + public static GetNodesToLabelsResponse mergeNodesToLabelsResponse( + Collection responses) { + GetNodesToLabelsResponse nodesToLabelsResponse = Records.newRecord( + GetNodesToLabelsResponse.class); + Map> nodesToLabelMap = new HashMap<>(); + for (GetNodesToLabelsResponse response : responses) { + if (response != null && response.getNodeToLabels() != null) { + nodesToLabelMap.putAll(response.getNodeToLabels()); + } + } + nodesToLabelsResponse.setNodeToLabels(nodesToLabelMap); + return nodesToLabelsResponse; + } + + /** + * Merges a list of GetLabelsToNodesResponse. + * + * @param responses a list of GetLabelsToNodesResponse to merge. + * @return the merged GetLabelsToNodesResponse. + */ + public static GetLabelsToNodesResponse mergeLabelsToNodes( + Collection responses){ + GetLabelsToNodesResponse labelsToNodesResponse = Records.newRecord( + GetLabelsToNodesResponse.class); + Map> labelsToNodesMap = new HashMap<>(); + for (GetLabelsToNodesResponse response : responses) { + if (response != null && response.getLabelsToNodes() != null) { + Map> clusterLabelsToNodesMap = response.getLabelsToNodes(); + for (Map.Entry> entry : clusterLabelsToNodesMap.entrySet()) { + String label = entry.getKey(); + Set clusterNodes = entry.getValue(); + if (labelsToNodesMap.containsKey(label)) { + Set allNodes = labelsToNodesMap.get(label); + allNodes.addAll(clusterNodes); + } else { + labelsToNodesMap.put(label, clusterNodes); + } + } + } + } + labelsToNodesResponse.setLabelsToNodes(labelsToNodesMap); + return labelsToNodesResponse; + } + + /** + * Merges a list of GetClusterNodeLabelsResponse. + * + * @param responses a list of GetClusterNodeLabelsResponse to merge. + * @return the merged GetClusterNodeLabelsResponse. + */ + public static GetClusterNodeLabelsResponse mergeClusterNodeLabelsResponse( + Collection responses) { + GetClusterNodeLabelsResponse nodeLabelsResponse = Records.newRecord( + GetClusterNodeLabelsResponse.class); + Set nodeLabelsList = new HashSet<>(); + for (GetClusterNodeLabelsResponse response : responses) { + if (response != null && response.getNodeLabelList() != null) { + nodeLabelsList.addAll(response.getNodeLabelList()); + } + } + nodeLabelsResponse.setNodeLabelList(new ArrayList<>(nodeLabelsList)); + return nodeLabelsResponse; + } + + /** + * Merges a list of GetQueueUserAclsInfoResponse. + * + * @param responses a list of GetQueueUserAclsInfoResponse to merge. + * @return the merged GetQueueUserAclsInfoResponse. + */ + public static GetQueueUserAclsInfoResponse mergeQueueUserAcls( + Collection responses) { + GetQueueUserAclsInfoResponse aclsInfoResponse = Records.newRecord( + GetQueueUserAclsInfoResponse.class); + Set queueUserACLInfos = new HashSet<>(); + for (GetQueueUserAclsInfoResponse response : responses) { + if (response != null && response.getUserAclsInfoList() != null) { + queueUserACLInfos.addAll(response.getUserAclsInfoList()); + } + } + aclsInfoResponse.setUserAclsInfoList(new ArrayList<>(queueUserACLInfos)); + return aclsInfoResponse; + } + + /** + * Merges a list of ReservationListResponse. + * + * @param responses a list of ReservationListResponse to merge. + * @return the merged ReservationListResponse. + */ + public static ReservationListResponse mergeReservationsList( + Collection responses) { + ReservationListResponse reservationListResponse = + Records.newRecord(ReservationListResponse.class); + List reservationAllocationStates = + new ArrayList<>(); + for (ReservationListResponse response : responses) { + if (response != null && response.getReservationAllocationState() != null) { + reservationAllocationStates.addAll( + response.getReservationAllocationState()); + } + } + reservationListResponse.setReservationAllocationState( + reservationAllocationStates); + return reservationListResponse; + } + + /** + * Merges a list of GetAllResourceTypeInfoResponse. + * + * @param responses a list of GetAllResourceTypeInfoResponse to merge. + * @return the merged GetAllResourceTypeInfoResponse. + */ + public static GetAllResourceTypeInfoResponse mergeResourceTypes( + Collection responses) { + GetAllResourceTypeInfoResponse resourceTypeInfoResponse = + Records.newRecord(GetAllResourceTypeInfoResponse.class); + Set resourceTypeInfoSet = new HashSet<>(); + for (GetAllResourceTypeInfoResponse response : responses) { + if (response != null && response.getResourceTypeInfo() != null) { + resourceTypeInfoSet.addAll(response.getResourceTypeInfo()); + } + } + resourceTypeInfoResponse.setResourceTypeInfo( + new ArrayList<>(resourceTypeInfoSet)); + return resourceTypeInfoResponse; + } } + diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-router/src/test/java/org/apache/hadoop/yarn/server/router/TestRouterMetrics.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-router/src/test/java/org/apache/hadoop/yarn/server/router/TestRouterMetrics.java index 4d4838a560b48..eddd2a0ab4816 100644 --- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-router/src/test/java/org/apache/hadoop/yarn/server/router/TestRouterMetrics.java +++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-router/src/test/java/org/apache/hadoop/yarn/server/router/TestRouterMetrics.java @@ -38,6 +38,8 @@ public class TestRouterMetrics { private static RouterMetrics metrics = RouterMetrics.getMetrics(); + private static final Double ASSERT_DOUBLE_DELTA = 0.01; + @BeforeClass public static void init() { @@ -208,21 +210,21 @@ public void testAppsReportFailed() { @Test public void testSucceededAppAttemptReport() { - long totalGoodBefore = metrics.getNumSucceededAppAttemptsRetrieved(); + long totalGoodBefore = metrics.getNumSucceededAppAttemptReportRetrieved(); goodSubCluster.getApplicationAttemptReport(100); Assert.assertEquals(totalGoodBefore + 1, - metrics.getNumSucceededAppAttemptsRetrieved()); + metrics.getNumSucceededAppAttemptReportRetrieved()); Assert.assertEquals(100, - metrics.getLatencySucceededGetAppAttemptReport(), 0); + metrics.getLatencySucceededGetAppAttemptReport(), ASSERT_DOUBLE_DELTA); goodSubCluster.getApplicationAttemptReport(200); Assert.assertEquals(totalGoodBefore + 2, - metrics.getNumSucceededAppAttemptsRetrieved()); + metrics.getNumSucceededAppAttemptReportRetrieved()); Assert.assertEquals(150, - metrics.getLatencySucceededGetAppAttemptReport(), 0); + metrics.getLatencySucceededGetAppAttemptReport(), ASSERT_DOUBLE_DELTA); } /** @@ -232,12 +234,12 @@ public void testSucceededAppAttemptReport() { @Test public void testAppAttemptReportFailed() { - long totalBadbefore = metrics.getAppAttemptsFailedRetrieved(); + long totalBadBefore = metrics.getAppAttemptReportFailedRetrieved(); badSubCluster.getApplicationAttemptReport(); - Assert.assertEquals(totalBadbefore + 1, - metrics.getAppAttemptsFailedRetrieved()); + Assert.assertEquals(totalBadBefore + 1, + metrics.getAppAttemptReportFailedRetrieved()); } /** @@ -334,7 +336,7 @@ public void getApplicationReport() { public void getApplicationAttemptReport() { LOG.info("Mocked: failed getApplicationAttemptReport call"); - metrics.incrAppsFailedRetrieved(); + metrics.incrAppAttemptReportFailedRetrieved(); } public void getApplicationsReport() { @@ -346,6 +348,76 @@ public void getClusterMetrics() { LOG.info("Mocked: failed getClusterMetrics call"); metrics.incrGetClusterMetricsFailedRetrieved(); } + + public void getClusterNodes() { + LOG.info("Mocked: failed getClusterNodes call"); + metrics.incrClusterNodesFailedRetrieved(); + } + + public void getNodeToLabels() { + LOG.info("Mocked: failed getNodeToLabels call"); + metrics.incrNodeToLabelsFailedRetrieved(); + } + + public void getLabelToNodes() { + LOG.info("Mocked: failed getLabelToNodes call"); + metrics.incrLabelsToNodesFailedRetrieved(); + } + + public void getClusterNodeLabels() { + LOG.info("Mocked: failed getClusterNodeLabels call"); + metrics.incrClusterNodeLabelsFailedRetrieved(); + } + + public void getQueueUserAcls() { + LOG.info("Mocked: failed getQueueUserAcls call"); + metrics.incrQueueUserAclsFailedRetrieved(); + } + + public void getListReservations() { + LOG.info("Mocked: failed listReservations call"); + metrics.incrListReservationsFailedRetrieved(); + } + + public void getApplicationAttempts() { + LOG.info("Mocked: failed getApplicationAttempts call"); + metrics.incrAppAttemptsFailedRetrieved(); + } + + public void getContainerReport() { + LOG.info("Mocked: failed getContainerReport call"); + metrics.incrContainerReportFailedRetrieved(); + } + + public void getContainer() { + LOG.info("Mocked: failed getContainer call"); + metrics.incrContainerFailedRetrieved(); + } + + public void getResourceTypeInfo() { + LOG.info("Mocked: failed getResourceTypeInfo call"); + metrics.incrResourceTypeInfoFailedRetrieved(); + } + + public void getFailApplicationAttempt() { + LOG.info("Mocked: failed failApplicationAttempt call"); + metrics.incrFailAppAttemptFailedRetrieved(); + } + + public void getUpdateApplicationPriority() { + LOG.info("Mocked: failed updateApplicationPriority call"); + metrics.incrUpdateAppPriorityFailedRetrieved(); + } + + public void getUpdateApplicationTimeouts() { + LOG.info("Mocked: failed updateApplicationTimeouts call"); + metrics.incrUpdateApplicationTimeoutsRetrieved(); + } + + public void getSignalContainer() { + LOG.info("Mocked: failed signalContainer call"); + metrics.incrSignalToContainerFailedRetrieved(); + } } // Records successes for all calls @@ -375,10 +447,9 @@ public void getApplicationReport(long duration) { } public void getApplicationAttemptReport(long duration) { - LOG.info("Mocked: successful " + - "getApplicationAttemptReport call with duration {}", - duration); - metrics.succeededAppAttemptsRetrieved(duration); + LOG.info("Mocked: successful getApplicationAttemptReport call " + + "with duration {}", duration); + metrics.succeededAppAttemptReportRetrieved(duration); } public void getApplicationsReport(long duration) { @@ -392,5 +463,380 @@ public void getClusterMetrics(long duration){ duration); metrics.succeededGetClusterMetricsRetrieved(duration); } + + public void getClusterNodes(long duration) { + LOG.info("Mocked: successful getClusterNodes call with duration {}", duration); + metrics.succeededGetClusterNodesRetrieved(duration); + } + + public void getNodeToLabels(long duration) { + LOG.info("Mocked: successful getNodeToLabels call with duration {}", duration); + metrics.succeededGetNodeToLabelsRetrieved(duration); + } + + public void getLabelToNodes(long duration) { + LOG.info("Mocked: successful getLabelToNodes call with duration {}", duration); + metrics.succeededGetLabelsToNodesRetrieved(duration); + } + + public void getClusterNodeLabels(long duration) { + LOG.info("Mocked: successful getClusterNodeLabels call with duration {}", duration); + metrics.succeededGetClusterNodeLabelsRetrieved(duration); + } + + public void getQueueUserAcls(long duration) { + LOG.info("Mocked: successful getQueueUserAcls call with duration {}", duration); + metrics.succeededGetQueueUserAclsRetrieved(duration); + } + + public void getListReservations(long duration) { + LOG.info("Mocked: successful listReservations call with duration {}", duration); + metrics.succeededListReservationsRetrieved(duration); + } + + public void getApplicationAttempts(long duration) { + LOG.info("Mocked: successful getApplicationAttempts call with duration {}", duration); + metrics.succeededAppAttemptsRetrieved(duration); + } + + public void getContainerReport(long duration) { + LOG.info("Mocked: successful getContainerReport call with duration {}", duration); + metrics.succeededGetContainerReportRetrieved(duration); + } + + public void getContainer(long duration) { + LOG.info("Mocked: successful getContainer call with duration {}", duration); + metrics.succeededGetContainersRetrieved(duration); + } + + public void getResourceTypeInfo(long duration) { + LOG.info("Mocked: successful getResourceTypeInfo call with duration {}", duration); + metrics.succeededGetResourceTypeInfoRetrieved(duration); + } + + public void getFailApplicationAttempt(long duration) { + LOG.info("Mocked: successful failApplicationAttempt call with duration {}", duration); + metrics.succeededFailAppAttemptRetrieved(duration); + } + + public void getUpdateApplicationPriority(long duration) { + LOG.info("Mocked: successful updateApplicationPriority call with duration {}", duration); + metrics.succeededUpdateAppPriorityRetrieved(duration); + } + + public void getUpdateApplicationTimeouts(long duration) { + LOG.info("Mocked: successful updateApplicationTimeouts call with duration {}", duration); + metrics.succeededUpdateAppTimeoutsRetrieved(duration); + } + + public void getSignalToContainerTimeouts(long duration) { + LOG.info("Mocked: successful signalToContainer call with duration {}", duration); + metrics.succeededSignalToContainerRetrieved(duration); + } + } + + @Test + public void testSucceededGetClusterNodes() { + long totalGoodBefore = metrics.getNumSucceededGetClusterNodesRetrieved(); + goodSubCluster.getClusterNodes(150); + Assert.assertEquals(totalGoodBefore + 1, metrics.getNumSucceededGetClusterNodesRetrieved()); + Assert.assertEquals(150, metrics.getLatencySucceededGetClusterNodesRetrieved(), + ASSERT_DOUBLE_DELTA); + goodSubCluster.getClusterNodes(300); + Assert.assertEquals(totalGoodBefore + 2, metrics.getNumSucceededGetClusterNodesRetrieved()); + Assert.assertEquals(225, metrics.getLatencySucceededGetClusterNodesRetrieved(), + ASSERT_DOUBLE_DELTA); + } + + @Test + public void testGetClusterNodesFailed() { + long totalBadBefore = metrics.getClusterNodesFailedRetrieved(); + badSubCluster.getClusterNodes(); + Assert.assertEquals(totalBadBefore + 1, metrics.getClusterNodesFailedRetrieved()); + } + + @Test + public void testSucceededGetNodeToLabels() { + long totalGoodBefore = metrics.getNumSucceededGetNodeToLabelsRetrieved(); + goodSubCluster.getNodeToLabels(150); + Assert.assertEquals(totalGoodBefore + 1, metrics.getNumSucceededGetNodeToLabelsRetrieved()); + Assert.assertEquals(150, metrics.getLatencySucceededGetNodeToLabelsRetrieved(), + ASSERT_DOUBLE_DELTA); + goodSubCluster.getNodeToLabels(300); + Assert.assertEquals(totalGoodBefore + 2, metrics.getNumSucceededGetNodeToLabelsRetrieved()); + Assert.assertEquals(225, metrics.getLatencySucceededGetNodeToLabelsRetrieved(), + ASSERT_DOUBLE_DELTA); + } + + @Test + public void testGetNodeToLabelsFailed() { + long totalBadBefore = metrics.getNodeToLabelsFailedRetrieved(); + badSubCluster.getNodeToLabels(); + Assert.assertEquals(totalBadBefore + 1, metrics.getNodeToLabelsFailedRetrieved()); + } + + @Test + public void testSucceededLabelsToNodes() { + long totalGoodBefore = metrics.getNumSucceededGetLabelsToNodesRetrieved(); + goodSubCluster.getLabelToNodes(150); + Assert.assertEquals(totalGoodBefore + 1, metrics.getNumSucceededGetLabelsToNodesRetrieved()); + Assert.assertEquals(150, metrics.getLatencySucceededGetLabelsToNodesRetrieved(), + ASSERT_DOUBLE_DELTA); + goodSubCluster.getLabelToNodes(300); + Assert.assertEquals(totalGoodBefore + 2, metrics.getNumSucceededGetLabelsToNodesRetrieved()); + Assert.assertEquals(225, metrics.getLatencySucceededGetLabelsToNodesRetrieved(), + ASSERT_DOUBLE_DELTA); + } + + @Test + public void testGetLabelsToNodesFailed() { + long totalBadBefore = metrics.getLabelsToNodesFailedRetrieved(); + badSubCluster.getLabelToNodes(); + Assert.assertEquals(totalBadBefore + 1, metrics.getLabelsToNodesFailedRetrieved()); + } + + @Test + public void testSucceededClusterNodeLabels() { + long totalGoodBefore = metrics.getNumSucceededGetClusterNodeLabelsRetrieved(); + goodSubCluster.getClusterNodeLabels(150); + Assert.assertEquals(totalGoodBefore + 1, + metrics.getNumSucceededGetClusterNodeLabelsRetrieved()); + Assert.assertEquals(150, + metrics.getLatencySucceededGetClusterNodeLabelsRetrieved(), ASSERT_DOUBLE_DELTA); + goodSubCluster.getClusterNodeLabels(300); + Assert.assertEquals(totalGoodBefore + 2, + metrics.getNumSucceededGetClusterNodeLabelsRetrieved()); + Assert.assertEquals(225, metrics.getLatencySucceededGetClusterNodeLabelsRetrieved(), + ASSERT_DOUBLE_DELTA); + } + + @Test + public void testClusterNodeLabelsFailed() { + long totalBadBefore = metrics.getGetClusterNodeLabelsFailedRetrieved(); + badSubCluster.getClusterNodeLabels(); + Assert.assertEquals(totalBadBefore + 1, metrics.getGetClusterNodeLabelsFailedRetrieved()); } + + @Test + public void testSucceededQueueUserAcls() { + long totalGoodBefore = metrics.getNumSucceededGetQueueUserAclsRetrieved(); + goodSubCluster.getQueueUserAcls(150); + Assert.assertEquals(totalGoodBefore + 1, + metrics.getNumSucceededGetQueueUserAclsRetrieved()); + Assert.assertEquals(150, + metrics.getLatencySucceededGetQueueUserAclsRetrieved(), ASSERT_DOUBLE_DELTA); + goodSubCluster.getQueueUserAcls(300); + Assert.assertEquals(totalGoodBefore + 2, + metrics.getNumSucceededGetQueueUserAclsRetrieved()); + Assert.assertEquals(225, metrics.getLatencySucceededGetQueueUserAclsRetrieved(), + ASSERT_DOUBLE_DELTA); + } + + @Test + public void testQueueUserAclsFailed() { + long totalBadBefore = metrics.getQueueUserAclsFailedRetrieved(); + badSubCluster.getQueueUserAcls(); + Assert.assertEquals(totalBadBefore + 1, metrics.getQueueUserAclsFailedRetrieved()); + } + @Test + public void testSucceededListReservations() { + long totalGoodBefore = metrics.getNumSucceededListReservationsRetrieved(); + goodSubCluster.getListReservations(150); + Assert.assertEquals(totalGoodBefore + 1, + metrics.getNumSucceededListReservationsRetrieved()); + Assert.assertEquals(150, + metrics.getLatencySucceededListReservationsRetrieved(), ASSERT_DOUBLE_DELTA); + goodSubCluster.getListReservations(300); + Assert.assertEquals(totalGoodBefore + 2, + metrics.getNumSucceededListReservationsRetrieved()); + Assert.assertEquals(225, metrics.getLatencySucceededListReservationsRetrieved(), + ASSERT_DOUBLE_DELTA); + } + + @Test + public void testListReservationsFailed() { + long totalBadBefore = metrics.getListReservationsFailedRetrieved(); + badSubCluster.getListReservations(); + Assert.assertEquals(totalBadBefore + 1, metrics.getListReservationsFailedRetrieved()); + } + + @Test + public void testSucceededGetApplicationAttempts() { + long totalGoodBefore = metrics.getNumSucceededAppAttemptsRetrieved(); + goodSubCluster.getApplicationAttempts(150); + Assert.assertEquals(totalGoodBefore + 1, + metrics.getNumSucceededAppAttemptsRetrieved()); + Assert.assertEquals(150, + metrics.getLatencySucceededAppAttemptRetrieved(), ASSERT_DOUBLE_DELTA); + goodSubCluster.getApplicationAttempts(300); + Assert.assertEquals(totalGoodBefore + 2, + metrics.getNumSucceededAppAttemptsRetrieved()); + Assert.assertEquals(225, metrics.getLatencySucceededAppAttemptRetrieved(), + ASSERT_DOUBLE_DELTA); + } + + @Test + public void testGetApplicationAttemptsFailed() { + long totalBadBefore = metrics.getAppAttemptsFailedRetrieved(); + badSubCluster.getApplicationAttempts(); + Assert.assertEquals(totalBadBefore + 1, metrics.getAppAttemptsFailedRetrieved()); + } + + @Test + public void testSucceededGetContainerReport() { + long totalGoodBefore = metrics.getNumSucceededGetContainerReportRetrieved(); + goodSubCluster.getContainerReport(150); + Assert.assertEquals(totalGoodBefore + 1, + metrics.getNumSucceededGetContainerReportRetrieved()); + Assert.assertEquals(150, + metrics.getLatencySucceededGetContainerReportRetrieved(), ASSERT_DOUBLE_DELTA); + goodSubCluster.getContainerReport(300); + Assert.assertEquals(totalGoodBefore + 2, + metrics.getNumSucceededGetContainerReportRetrieved()); + Assert.assertEquals(225, metrics.getLatencySucceededGetContainerReportRetrieved(), + ASSERT_DOUBLE_DELTA); + } + + @Test + public void testGetContainerReportFailed() { + long totalBadBefore = metrics.getContainerReportFailedRetrieved(); + badSubCluster.getContainerReport(); + Assert.assertEquals(totalBadBefore + 1, metrics.getContainerReportFailedRetrieved()); + } + + @Test + public void testSucceededGetContainers() { + long totalGoodBefore = metrics.getNumSucceededGetContainersRetrieved(); + goodSubCluster.getContainer(150); + Assert.assertEquals(totalGoodBefore + 1, + metrics.getNumSucceededGetContainersRetrieved()); + Assert.assertEquals(150, + metrics.getLatencySucceededGetContainersRetrieved(), ASSERT_DOUBLE_DELTA); + goodSubCluster.getContainer(300); + Assert.assertEquals(totalGoodBefore + 2, + metrics.getNumSucceededGetContainersRetrieved()); + Assert.assertEquals(225, metrics.getLatencySucceededGetContainersRetrieved(), + ASSERT_DOUBLE_DELTA); + } + + @Test + public void testGetContainerFailed() { + long totalBadBefore = metrics.getContainersFailedRetrieved(); + badSubCluster.getContainer(); + Assert.assertEquals(totalBadBefore + 1, metrics.getContainersFailedRetrieved()); + } + + @Test + public void testSucceededGetResourceTypeInfo() { + long totalGoodBefore = metrics.getNumSucceededGetResourceTypeInfoRetrieved(); + goodSubCluster.getResourceTypeInfo(150); + Assert.assertEquals(totalGoodBefore + 1, + metrics.getNumSucceededGetResourceTypeInfoRetrieved()); + Assert.assertEquals(150, + metrics.getLatencySucceededGetResourceTypeInfoRetrieved(), ASSERT_DOUBLE_DELTA); + goodSubCluster.getResourceTypeInfo(300); + Assert.assertEquals(totalGoodBefore + 2, + metrics.getNumSucceededGetResourceTypeInfoRetrieved()); + Assert.assertEquals(225, + metrics.getLatencySucceededGetResourceTypeInfoRetrieved(), ASSERT_DOUBLE_DELTA); + } + + @Test + public void testGetResourceTypeInfoFailed() { + long totalBadBefore = metrics.getGetResourceTypeInfoRetrieved(); + badSubCluster.getResourceTypeInfo(); + Assert.assertEquals(totalBadBefore + 1, metrics.getGetResourceTypeInfoRetrieved()); + } + + @Test + public void testSucceededFailApplicationAttempt() { + long totalGoodBefore = metrics.getNumSucceededFailAppAttemptRetrieved(); + goodSubCluster.getFailApplicationAttempt(150); + Assert.assertEquals(totalGoodBefore + 1, + metrics.getNumSucceededFailAppAttemptRetrieved()); + Assert.assertEquals(150, + metrics.getLatencySucceededFailAppAttemptRetrieved(), ASSERT_DOUBLE_DELTA); + goodSubCluster.getFailApplicationAttempt(300); + Assert.assertEquals(totalGoodBefore + 2, + metrics.getNumSucceededFailAppAttemptRetrieved()); + Assert.assertEquals(225, + metrics.getLatencySucceededFailAppAttemptRetrieved(), ASSERT_DOUBLE_DELTA); + } + + @Test + public void testFailApplicationAttemptFailed() { + long totalBadBefore = metrics.getFailApplicationAttemptFailedRetrieved(); + badSubCluster.getFailApplicationAttempt(); + Assert.assertEquals(totalBadBefore + 1, metrics.getFailApplicationAttemptFailedRetrieved()); + } + + @Test + public void testSucceededUpdateApplicationPriority() { + long totalGoodBefore = metrics.getNumSucceededUpdateAppPriorityRetrieved(); + goodSubCluster.getUpdateApplicationPriority(150); + Assert.assertEquals(totalGoodBefore + 1, + metrics.getNumSucceededUpdateAppPriorityRetrieved()); + Assert.assertEquals(150, + metrics.getLatencySucceededUpdateAppPriorityRetrieved(), ASSERT_DOUBLE_DELTA); + goodSubCluster.getUpdateApplicationPriority(300); + Assert.assertEquals(totalGoodBefore + 2, + metrics.getNumSucceededUpdateAppPriorityRetrieved()); + Assert.assertEquals(225, + metrics.getLatencySucceededUpdateAppPriorityRetrieved(), ASSERT_DOUBLE_DELTA); + } + + @Test + public void testUpdateApplicationPriorityFailed() { + long totalBadBefore = metrics.getUpdateApplicationPriorityFailedRetrieved(); + badSubCluster.getUpdateApplicationPriority(); + Assert.assertEquals(totalBadBefore + 1, + metrics.getUpdateApplicationPriorityFailedRetrieved()); + } + + @Test + public void testSucceededUpdateAppTimeoutsRetrieved() { + long totalGoodBefore = metrics.getNumSucceededUpdateAppTimeoutsRetrieved(); + goodSubCluster.getUpdateApplicationTimeouts(150); + Assert.assertEquals(totalGoodBefore + 1, + metrics.getNumSucceededUpdateAppTimeoutsRetrieved()); + Assert.assertEquals(150, + metrics.getLatencySucceededUpdateAppTimeoutsRetrieved(), ASSERT_DOUBLE_DELTA); + goodSubCluster.getUpdateApplicationTimeouts(300); + Assert.assertEquals(totalGoodBefore + 2, + metrics.getNumSucceededUpdateAppTimeoutsRetrieved()); + Assert.assertEquals(225, + metrics.getLatencySucceededUpdateAppTimeoutsRetrieved(), ASSERT_DOUBLE_DELTA); + } + + @Test + public void testUpdateAppTimeoutsFailed() { + long totalBadBefore = metrics.getUpdateApplicationTimeoutsFailedRetrieved(); + badSubCluster.getUpdateApplicationTimeouts(); + Assert.assertEquals(totalBadBefore + 1, + metrics.getUpdateApplicationTimeoutsFailedRetrieved()); + } + + @Test + public void testSucceededSignalToContainerRetrieved() { + long totalGoodBefore = metrics.getNumSucceededSignalToContainerRetrieved(); + goodSubCluster.getSignalToContainerTimeouts(150); + Assert.assertEquals(totalGoodBefore + 1, + metrics.getNumSucceededSignalToContainerRetrieved()); + Assert.assertEquals(150, + metrics.getLatencySucceededSignalToContainerRetrieved(), ASSERT_DOUBLE_DELTA); + goodSubCluster.getSignalToContainerTimeouts(300); + Assert.assertEquals(totalGoodBefore + 2, + metrics.getNumSucceededSignalToContainerRetrieved()); + Assert.assertEquals(225, + metrics.getLatencySucceededSignalToContainerRetrieved(), ASSERT_DOUBLE_DELTA); + } + + @Test + public void testSignalToContainerFailed() { + long totalBadBefore = metrics.getSignalToContainerFailedRetrieved(); + badSubCluster.getSignalContainer(); + Assert.assertEquals(totalBadBefore + 1, + metrics.getSignalToContainerFailedRetrieved()); + } + } diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-router/src/test/java/org/apache/hadoop/yarn/server/router/clientrm/TestFederationClientInterceptor.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-router/src/test/java/org/apache/hadoop/yarn/server/router/clientrm/TestFederationClientInterceptor.java index 74d10a44c52b5..3037738240266 100644 --- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-router/src/test/java/org/apache/hadoop/yarn/server/router/clientrm/TestFederationClientInterceptor.java +++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-router/src/test/java/org/apache/hadoop/yarn/server/router/clientrm/TestFederationClientInterceptor.java @@ -26,7 +26,9 @@ import java.util.EnumSet; import java.util.List; import java.util.Map; +import java.util.HashMap; import java.util.Set; +import java.util.stream.Collectors; import org.apache.hadoop.test.LambdaTestUtils; import org.apache.hadoop.yarn.MockApps; @@ -44,12 +46,46 @@ import org.apache.hadoop.yarn.api.protocolrecords.KillApplicationResponse; import org.apache.hadoop.yarn.api.protocolrecords.SubmitApplicationRequest; import org.apache.hadoop.yarn.api.protocolrecords.SubmitApplicationResponse; +import org.apache.hadoop.yarn.api.protocolrecords.GetClusterNodesResponse; +import org.apache.hadoop.yarn.api.protocolrecords.GetClusterNodesRequest; +import org.apache.hadoop.yarn.api.protocolrecords.GetNodesToLabelsResponse; +import org.apache.hadoop.yarn.api.protocolrecords.GetNodesToLabelsRequest; +import org.apache.hadoop.yarn.api.protocolrecords.GetLabelsToNodesResponse; +import org.apache.hadoop.yarn.api.protocolrecords.GetLabelsToNodesRequest; +import org.apache.hadoop.yarn.api.protocolrecords.GetClusterNodeLabelsResponse; +import org.apache.hadoop.yarn.api.protocolrecords.GetClusterNodeLabelsRequest; +import org.apache.hadoop.yarn.api.protocolrecords.GetApplicationAttemptsRequest; +import org.apache.hadoop.yarn.api.protocolrecords.GetApplicationAttemptsResponse; +import org.apache.hadoop.yarn.api.protocolrecords.GetQueueUserAclsInfoResponse; +import org.apache.hadoop.yarn.api.protocolrecords.GetQueueUserAclsInfoRequest; +import org.apache.hadoop.yarn.api.protocolrecords.ReservationListResponse; +import org.apache.hadoop.yarn.api.protocolrecords.ReservationListRequest; +import org.apache.hadoop.yarn.api.protocolrecords.GetContainersRequest; +import org.apache.hadoop.yarn.api.protocolrecords.GetContainersResponse; +import org.apache.hadoop.yarn.api.protocolrecords.GetContainerReportRequest; +import org.apache.hadoop.yarn.api.protocolrecords.GetContainerReportResponse; +import org.apache.hadoop.yarn.api.protocolrecords.GetAllResourceTypeInfoResponse; +import org.apache.hadoop.yarn.api.protocolrecords.GetAllResourceTypeInfoRequest; +import org.apache.hadoop.yarn.api.protocolrecords.FailApplicationAttemptRequest; +import org.apache.hadoop.yarn.api.protocolrecords.FailApplicationAttemptResponse; +import org.apache.hadoop.yarn.api.protocolrecords.UpdateApplicationPriorityRequest; +import org.apache.hadoop.yarn.api.protocolrecords.UpdateApplicationPriorityResponse; +import org.apache.hadoop.yarn.api.protocolrecords.UpdateApplicationTimeoutsRequest; +import org.apache.hadoop.yarn.api.protocolrecords.UpdateApplicationTimeoutsResponse; +import org.apache.hadoop.yarn.api.protocolrecords.SignalContainerRequest; +import org.apache.hadoop.yarn.api.protocolrecords.SignalContainerResponse; import org.apache.hadoop.yarn.api.records.ApplicationAttemptId; import org.apache.hadoop.yarn.api.records.ApplicationId; import org.apache.hadoop.yarn.api.records.ApplicationSubmissionContext; import org.apache.hadoop.yarn.api.records.ContainerLaunchContext; import org.apache.hadoop.yarn.api.records.Priority; import org.apache.hadoop.yarn.api.records.YarnApplicationState; +import org.apache.hadoop.yarn.api.records.QueueACL; +import org.apache.hadoop.yarn.api.records.QueueUserACLInfo; +import org.apache.hadoop.yarn.api.records.ReservationId; +import org.apache.hadoop.yarn.api.records.ContainerId; +import org.apache.hadoop.yarn.api.records.ApplicationTimeoutType; +import org.apache.hadoop.yarn.api.records.SignalContainerCommand; import org.apache.hadoop.yarn.conf.YarnConfiguration; import org.apache.hadoop.yarn.exceptions.YarnException; import org.apache.hadoop.yarn.server.federation.policies.manager.UniformBroadcastPolicyManager; @@ -57,7 +93,13 @@ import org.apache.hadoop.yarn.server.federation.store.records.SubClusterId; import org.apache.hadoop.yarn.server.federation.utils.FederationStateStoreFacade; import org.apache.hadoop.yarn.server.federation.utils.FederationStateStoreTestUtil; +import org.apache.hadoop.yarn.server.resourcemanager.MockRM; +import org.apache.hadoop.yarn.server.resourcemanager.MockNM; import org.apache.hadoop.yarn.server.resourcemanager.ResourceManager; +import org.apache.hadoop.yarn.server.resourcemanager.rmapp.RMApp; +import org.apache.hadoop.yarn.server.resourcemanager.rmapp.RMAppState; +import org.apache.hadoop.yarn.server.resourcemanager.rmapp.attempt.RMAppAttemptState; +import org.apache.hadoop.yarn.util.Times; import org.apache.hadoop.yarn.util.resource.Resources; import org.junit.Assert; import org.junit.Test; @@ -84,6 +126,8 @@ public class TestFederationClientInterceptor extends BaseRouterClientRMTest { private final static int NUM_SUBCLUSTER = 4; + private final static int APP_PRIORITY_ZERO = 0; + @Override public void setUp() { super.setUpConfig(); @@ -167,7 +211,7 @@ public void testGetNewApplication() */ @Test public void testSubmitApplication() - throws YarnException, IOException, InterruptedException { + throws YarnException, IOException { LOG.info("Test FederationClientInterceptor: Submit Application"); ApplicationId appId = ApplicationId.newInstance(System.currentTimeMillis(), @@ -187,7 +231,7 @@ private SubmitApplicationRequest mockSubmitApplicationRequest( ContainerLaunchContext amContainerSpec = mock(ContainerLaunchContext.class); ApplicationSubmissionContext context = ApplicationSubmissionContext .newInstance(appId, MockApps.newAppName(), "default", - Priority.newInstance(0), amContainerSpec, false, false, -1, + Priority.newInstance(APP_PRIORITY_ZERO), amContainerSpec, false, false, -1, Resources.createResource( YarnConfiguration.DEFAULT_RM_SCHEDULER_MINIMUM_ALLOCATION_MB), "MockApp"); @@ -427,13 +471,10 @@ public void testGetApplicationEmptyRequest() @Test public void testGetApplicationAttemptReport() throws YarnException, IOException, InterruptedException { - LOG.info("Test FederationClientInterceptor: " + - "Get ApplicationAttempt Report"); + LOG.info("Test FederationClientInterceptor: Get ApplicationAttempt Report."); ApplicationId appId = - ApplicationId.newInstance(System.currentTimeMillis(), 1); - ApplicationAttemptId appAttemptId = - ApplicationAttemptId.newInstance(appId, 1); + ApplicationId.newInstance(System.currentTimeMillis(), 1); SubmitApplicationRequest request = mockSubmitApplicationRequest(appId); @@ -443,11 +484,26 @@ public void testGetApplicationAttemptReport() Assert.assertNotNull(response); Assert.assertNotNull(stateStoreUtil.queryApplicationHomeSC(appId)); + // Call GetApplicationAttempts Get ApplicationAttemptId + GetApplicationAttemptsRequest attemptsRequest = + GetApplicationAttemptsRequest.newInstance(appId); + GetApplicationAttemptsResponse attemptsResponse = + interceptor.getApplicationAttempts(attemptsRequest); + + // Wait for app to start + while(attemptsResponse.getApplicationAttemptList().size() == 0) { + attemptsResponse = + interceptor.getApplicationAttempts(attemptsRequest); + } + + Assert.assertNotNull(attemptsResponse); + GetApplicationAttemptReportRequest requestGet = - GetApplicationAttemptReportRequest.newInstance(appAttemptId); + GetApplicationAttemptReportRequest.newInstance( + attemptsResponse.getApplicationAttemptList().get(0).getApplicationAttemptId()); GetApplicationAttemptReportResponse responseGet = - interceptor.getApplicationAttemptReport(requestGet); + interceptor.getApplicationAttemptReport(requestGet); Assert.assertNotNull(responseGet); } @@ -471,8 +527,8 @@ public void testGetApplicationAttemptNotExists() GetApplicationAttemptReportRequest.newInstance(appAttemptID); LambdaTestUtils.intercept(YarnException.class, "ApplicationAttempt " + - appAttemptID + "belongs to Application " + - appId + " does not exist in FederationStateStore", + appAttemptID + " belongs to Application " + + appId + " does not exist in FederationStateStore.", () -> interceptor.getApplicationAttemptReport(requestGet)); } @@ -615,7 +671,7 @@ public void testGetApplicationsApplicationTypeNotExists() throws Exception{ * with given YarnApplicationState does not exist. */ @Test - public void testGetApplicationsApplicationStateNotExists() throws Exception{ + public void testGetApplicationsApplicationStateNotExists() throws Exception { LOG.info("Test FederationClientInterceptor:" + " Application with state does not exist"); @@ -641,4 +697,408 @@ public void testGetApplicationsApplicationStateNotExists() throws Exception{ Assert.assertNotNull(responseGet); Assert.assertTrue(responseGet.getApplicationList().isEmpty()); } + + @Test + public void testGetClusterNodesRequest() throws Exception { + LOG.info("Test FederationClientInterceptor : Get Cluster Nodeds request"); + // null request + LambdaTestUtils.intercept(YarnException.class, "Missing getClusterNodes request.", + () -> interceptor.getClusterNodes(null)); + // normal request. + GetClusterNodesResponse response = + interceptor.getClusterNodes(GetClusterNodesRequest.newInstance()); + Assert.assertEquals(subClusters.size(), response.getNodeReports().size()); + } + + @Test + public void testGetNodeToLabelsRequest() throws Exception { + LOG.info("Test FederationClientInterceptor : Get Node To Labels request"); + // null request + LambdaTestUtils.intercept(YarnException.class, "Missing getNodesToLabels request.", + () -> interceptor.getNodeToLabels(null)); + // normal request. + GetNodesToLabelsResponse response = + interceptor.getNodeToLabels(GetNodesToLabelsRequest.newInstance()); + Assert.assertEquals(0, response.getNodeToLabels().size()); + } + + @Test + public void testGetLabelsToNodesRequest() throws Exception { + LOG.info("Test FederationClientInterceptor : Get Labels To Node request"); + // null request + LambdaTestUtils.intercept(YarnException.class, "Missing getLabelsToNodes request.", + () -> interceptor.getLabelsToNodes(null)); + // normal request. + GetLabelsToNodesResponse response = + interceptor.getLabelsToNodes(GetLabelsToNodesRequest.newInstance()); + Assert.assertEquals(0, response.getLabelsToNodes().size()); + } + + @Test + public void testClusterNodeLabelsRequest() throws Exception { + LOG.info("Test FederationClientInterceptor : Get Cluster NodeLabels request"); + // null request + LambdaTestUtils.intercept(YarnException.class, "Missing getClusterNodeLabels request.", + () -> interceptor.getClusterNodeLabels(null)); + // normal request. + GetClusterNodeLabelsResponse response = + interceptor.getClusterNodeLabels(GetClusterNodeLabelsRequest.newInstance()); + Assert.assertEquals(0, response.getNodeLabelList().size()); + } + + @Test + public void testGetQueueUserAcls() throws Exception { + LOG.info("Test FederationClientInterceptor : Get QueueUserAcls request"); + + // null request + LambdaTestUtils.intercept(YarnException.class, "Missing getQueueUserAcls request.", + () -> interceptor.getQueueUserAcls(null)); + + // noraml request + GetQueueUserAclsInfoResponse response = interceptor.getQueueUserAcls( + GetQueueUserAclsInfoRequest.newInstance()); + + Assert.assertNotNull(response); + + List submitAndAdministerAcl = new ArrayList<>(); + submitAndAdministerAcl.add(QueueACL.SUBMIT_APPLICATIONS); + submitAndAdministerAcl.add(QueueACL.ADMINISTER_QUEUE); + + QueueUserACLInfo exceptRootQueueACLInfo = QueueUserACLInfo.newInstance("root", + submitAndAdministerAcl); + + QueueUserACLInfo queueRootQueueACLInfo = response.getUserAclsInfoList().stream(). + filter(acl->acl.getQueueName().equals("root")). + collect(Collectors.toList()).get(0); + + Assert.assertEquals(exceptRootQueueACLInfo, queueRootQueueACLInfo); + } + + @Test + public void testListReservations() throws Exception { + LOG.info("Test FederationClientInterceptor : Get ListReservations request"); + + // null request + LambdaTestUtils.intercept(YarnException.class, "Missing listReservations request.", + () -> interceptor.listReservations(null)); + + // normal request + ReservationId reservationId = ReservationId.newInstance(1653487680L, 1L); + ReservationListResponse response = interceptor.listReservations( + ReservationListRequest.newInstance("root.decided", reservationId.toString())); + Assert.assertNotNull(response); + Assert.assertEquals(0, response.getReservationAllocationState().size()); + } + + @Test + public void testGetContainersRequest() throws Exception { + LOG.info("Test FederationClientInterceptor : Get Containers request"); + + // null request + LambdaTestUtils.intercept(YarnException.class, "Missing getContainers request " + + "or ApplicationAttemptId.", () -> interceptor.getContainers(null)); + + // normal request + ApplicationId appId = + ApplicationId.newInstance(System.currentTimeMillis(), 1); + SubmitApplicationRequest request = mockSubmitApplicationRequest(appId); + + // Submit the application + SubmitApplicationResponse response = interceptor.submitApplication(request); + + Assert.assertNotNull(response); + Assert.assertNotNull(stateStoreUtil.queryApplicationHomeSC(appId)); + + // Call GetApplicationAttempts + GetApplicationAttemptsRequest attemptsRequest = + GetApplicationAttemptsRequest.newInstance(appId); + GetApplicationAttemptsResponse attemptsResponse = + interceptor.getApplicationAttempts(attemptsRequest); + + // Wait for app to start + while(attemptsResponse.getApplicationAttemptList().size() == 0) { + attemptsResponse = + interceptor.getApplicationAttempts(attemptsRequest); + } + + Assert.assertNotNull(attemptsResponse); + + // Call GetContainers + GetContainersRequest containersRequest = + GetContainersRequest.newInstance( + attemptsResponse.getApplicationAttemptList().get(0).getApplicationAttemptId()); + GetContainersResponse containersResponse = + interceptor.getContainers(containersRequest); + + Assert.assertNotNull(containersResponse); + } + + @Test + public void testGetContainerReportRequest() throws Exception { + LOG.info("Test FederationClientInterceptor : Get Container Report request."); + + // null request + LambdaTestUtils.intercept(YarnException.class, "Missing getContainerReport request " + + "or containerId", () -> interceptor.getContainerReport(null)); + + // normal request + ApplicationId appId = + ApplicationId.newInstance(System.currentTimeMillis(), 1); + SubmitApplicationRequest request = mockSubmitApplicationRequest(appId); + + // Submit the application + SubmitApplicationResponse response = interceptor.submitApplication(request); + + Assert.assertNotNull(response); + Assert.assertNotNull(stateStoreUtil.queryApplicationHomeSC(appId)); + + // Call GetApplicationAttempts + GetApplicationAttemptsRequest attemptsRequest = + GetApplicationAttemptsRequest.newInstance(appId); + GetApplicationAttemptsResponse attemptsResponse = + interceptor.getApplicationAttempts(attemptsRequest); + + // Wait for app to start + while(attemptsResponse.getApplicationAttemptList().size() == 0) { + attemptsResponse = + interceptor.getApplicationAttempts(attemptsRequest); + } + Assert.assertNotNull(attemptsResponse); + + ApplicationAttemptId attemptId = attemptsResponse.getApplicationAttemptList(). + get(0).getApplicationAttemptId(); + ContainerId containerId = ContainerId.newContainerId(attemptId, 1); + + // Call ContainerReport, RM does not allocate Container, Here is null + GetContainerReportRequest containerReportRequest = + GetContainerReportRequest.newInstance(containerId); + GetContainerReportResponse containerReportResponse = + interceptor.getContainerReport(containerReportRequest); + + Assert.assertEquals(containerReportResponse, null); + } + + @Test + public void getApplicationAttempts() throws Exception { + LOG.info("Test FederationClientInterceptor : Get Application Attempts request."); + + // null request + LambdaTestUtils.intercept(YarnException.class, "Missing getApplicationAttempts " + + "request or application id.", () -> interceptor.getApplicationAttempts(null)); + + // normal request + ApplicationId appId = + ApplicationId.newInstance(System.currentTimeMillis(), 1); + SubmitApplicationRequest request = mockSubmitApplicationRequest(appId); + + // Submit the application + SubmitApplicationResponse response = interceptor.submitApplication(request); + + Assert.assertNotNull(response); + Assert.assertNotNull(stateStoreUtil.queryApplicationHomeSC(appId)); + + // Call GetApplicationAttempts + GetApplicationAttemptsRequest attemptsRequest = + GetApplicationAttemptsRequest.newInstance(appId); + GetApplicationAttemptsResponse attemptsResponse = + interceptor.getApplicationAttempts(attemptsRequest); + + Assert.assertNotNull(attemptsResponse); + } + + @Test + public void testGetResourceTypeInfoRequest() throws Exception { + LOG.info("Test FederationClientInterceptor : Get Resource TypeInfo request"); + // null request + LambdaTestUtils.intercept(YarnException.class, "Missing getResourceTypeInfo request.", + () -> interceptor.getResourceTypeInfo(null)); + // normal request. + GetAllResourceTypeInfoResponse response = + interceptor.getResourceTypeInfo(GetAllResourceTypeInfoRequest.newInstance()); + Assert.assertEquals(2, response.getResourceTypeInfo().size()); + } + + @Test + public void testFailApplicationAttempt() throws Exception { + LOG.info("Test FederationClientInterceptor : Fail Application Attempt request."); + + // null request + LambdaTestUtils.intercept(YarnException.class, "Missing failApplicationAttempt request " + + "or applicationId or applicationAttemptId information.", + () -> interceptor.failApplicationAttempt(null)); + + // normal request + ApplicationId appId = ApplicationId.newInstance(System.currentTimeMillis(), 1); + SubmitApplicationRequest request = mockSubmitApplicationRequest(appId); + + // Submit the application + SubmitApplicationResponse response = interceptor.submitApplication(request); + + Assert.assertNotNull(response); + Assert.assertNotNull(stateStoreUtil.queryApplicationHomeSC(appId)); + + SubClusterId subClusterId = interceptor.getApplicationHomeSubCluster(appId); + Assert.assertNotNull(subClusterId); + + MockRM mockRM = interceptor.getMockRMs().get(subClusterId); + mockRM.waitForState(appId, RMAppState.ACCEPTED); + RMApp rmApp = mockRM.getRMContext().getRMApps().get(appId); + mockRM.waitForState(rmApp.getCurrentAppAttempt().getAppAttemptId(), + RMAppAttemptState.SCHEDULED); + + // Call GetApplicationAttempts + GetApplicationAttemptsRequest attemptsRequest = + GetApplicationAttemptsRequest.newInstance(appId); + GetApplicationAttemptsResponse attemptsResponse = + interceptor.getApplicationAttempts(attemptsRequest); + Assert.assertNotNull(attemptsResponse); + + ApplicationAttemptId attemptId = attemptsResponse.getApplicationAttemptList(). + get(0).getApplicationAttemptId(); + + FailApplicationAttemptRequest requestFailAppAttempt = + FailApplicationAttemptRequest.newInstance(attemptId); + FailApplicationAttemptResponse responseFailAppAttempt = + interceptor.failApplicationAttempt(requestFailAppAttempt); + + Assert.assertNotNull(responseFailAppAttempt); + } + + @Test + public void testUpdateApplicationPriority() throws Exception { + LOG.info("Test FederationClientInterceptor : Update Application Priority request."); + + // null request + LambdaTestUtils.intercept(YarnException.class, "Missing updateApplicationPriority request " + + "or applicationId or applicationPriority information.", + () -> interceptor.updateApplicationPriority(null)); + + // normal request + ApplicationId appId = ApplicationId.newInstance(System.currentTimeMillis(), 1); + SubmitApplicationRequest request = mockSubmitApplicationRequest(appId); + + // Submit the application + SubmitApplicationResponse response = interceptor.submitApplication(request); + + Assert.assertNotNull(response); + Assert.assertNotNull(stateStoreUtil.queryApplicationHomeSC(appId)); + + SubClusterId subClusterId = interceptor.getApplicationHomeSubCluster(appId); + Assert.assertNotNull(subClusterId); + + MockRM mockRM = interceptor.getMockRMs().get(subClusterId); + mockRM.waitForState(appId, RMAppState.ACCEPTED); + RMApp rmApp = mockRM.getRMContext().getRMApps().get(appId); + mockRM.waitForState(rmApp.getCurrentAppAttempt().getAppAttemptId(), + RMAppAttemptState.SCHEDULED); + + // Call GetApplicationAttempts + GetApplicationAttemptsRequest attemptsRequest = + GetApplicationAttemptsRequest.newInstance(appId); + GetApplicationAttemptsResponse attemptsResponse = + interceptor.getApplicationAttempts(attemptsRequest); + Assert.assertNotNull(attemptsResponse); + + Priority priority = Priority.newInstance(20); + UpdateApplicationPriorityRequest requestUpdateAppPriority = + UpdateApplicationPriorityRequest.newInstance(appId, priority); + UpdateApplicationPriorityResponse responseAppPriority = + interceptor.updateApplicationPriority(requestUpdateAppPriority); + + Assert.assertNotNull(responseAppPriority); + Assert.assertEquals(20, + responseAppPriority.getApplicationPriority().getPriority()); + } + + @Test + public void testUpdateApplicationTimeouts() throws Exception { + LOG.info("Test FederationClientInterceptor : Update Application Timeouts request."); + + // null request + LambdaTestUtils.intercept(YarnException.class, "Missing updateApplicationTimeouts request " + + "or applicationId or applicationTimeouts information.", + () -> interceptor.updateApplicationTimeouts(null)); + + // normal request + ApplicationId appId = ApplicationId.newInstance(System.currentTimeMillis(), 1); + SubmitApplicationRequest request = mockSubmitApplicationRequest(appId); + + // Submit the application + SubmitApplicationResponse response = interceptor.submitApplication(request); + + Assert.assertNotNull(response); + Assert.assertNotNull(stateStoreUtil.queryApplicationHomeSC(appId)); + + SubClusterId subClusterId = interceptor.getApplicationHomeSubCluster(appId); + Assert.assertNotNull(subClusterId); + + MockRM mockRM = interceptor.getMockRMs().get(subClusterId); + mockRM.waitForState(appId, RMAppState.ACCEPTED); + RMApp rmApp = mockRM.getRMContext().getRMApps().get(appId); + mockRM.waitForState(rmApp.getCurrentAppAttempt().getAppAttemptId(), + RMAppAttemptState.SCHEDULED); + + // Call GetApplicationAttempts + GetApplicationAttemptsRequest attemptsRequest = + GetApplicationAttemptsRequest.newInstance(appId); + GetApplicationAttemptsResponse attemptsResponse = + interceptor.getApplicationAttempts(attemptsRequest); + Assert.assertNotNull(attemptsResponse); + + String appTimeout = + Times.formatISO8601(System.currentTimeMillis() + 5 * 1000); + Map applicationTimeouts = new HashMap<>(); + applicationTimeouts.put(ApplicationTimeoutType.LIFETIME, appTimeout); + + UpdateApplicationTimeoutsRequest timeoutsRequest = + UpdateApplicationTimeoutsRequest.newInstance(appId, applicationTimeouts); + UpdateApplicationTimeoutsResponse timeoutsResponse = + interceptor.updateApplicationTimeouts(timeoutsRequest); + + String responseTimeOut = + timeoutsResponse.getApplicationTimeouts().get(ApplicationTimeoutType.LIFETIME); + Assert.assertNotNull(timeoutsResponse); + Assert.assertEquals(appTimeout, responseTimeOut); + } + + @Test + public void testSignalContainer() throws Exception { + LOG.info("Test FederationClientInterceptor : Signal Container request."); + + // null request + LambdaTestUtils.intercept(YarnException.class, "Missing signalToContainer request " + + "or containerId or command information.", () -> interceptor.signalToContainer(null)); + + // normal request + ApplicationId appId = + ApplicationId.newInstance(System.currentTimeMillis(), 1); + SubmitApplicationRequest request = mockSubmitApplicationRequest(appId); + + // Submit the application + SubmitApplicationResponse response = interceptor.submitApplication(request); + Assert.assertNotNull(response); + Assert.assertNotNull(stateStoreUtil.queryApplicationHomeSC(appId)); + + SubClusterId subClusterId = interceptor.getApplicationHomeSubCluster(appId); + Assert.assertNotNull(subClusterId); + + MockRM mockRM = interceptor.getMockRMs().get(subClusterId); + mockRM.waitForState(appId, RMAppState.ACCEPTED); + RMApp rmApp = mockRM.getRMContext().getRMApps().get(appId); + mockRM.waitForState(rmApp.getCurrentAppAttempt().getAppAttemptId(), + RMAppAttemptState.SCHEDULED); + MockNM nm = interceptor.getMockNMs().get(subClusterId); + nm.nodeHeartbeat(true); + mockRM.waitForState(rmApp.getCurrentAppAttempt(), RMAppAttemptState.ALLOCATED); + mockRM.sendAMLaunched(rmApp.getCurrentAppAttempt().getAppAttemptId()); + + ContainerId containerId = rmApp.getCurrentAppAttempt().getMasterContainer().getId(); + + SignalContainerRequest signalContainerRequest = + SignalContainerRequest.newInstance(containerId, SignalContainerCommand.GRACEFUL_SHUTDOWN); + SignalContainerResponse signalContainerResponse = + interceptor.signalToContainer(signalContainerRequest); + + Assert.assertNotNull(signalContainerResponse); + } } diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-router/src/test/java/org/apache/hadoop/yarn/server/router/clientrm/TestRouterYarnClientUtils.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-router/src/test/java/org/apache/hadoop/yarn/server/router/clientrm/TestRouterYarnClientUtils.java index c91da3527932f..435a8bd85176b 100644 --- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-router/src/test/java/org/apache/hadoop/yarn/server/router/clientrm/TestRouterYarnClientUtils.java +++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-router/src/test/java/org/apache/hadoop/yarn/server/router/clientrm/TestRouterYarnClientUtils.java @@ -20,9 +20,21 @@ import java.util.ArrayList; import java.util.List; +import java.util.Set; +import java.util.Map; +import java.util.HashMap; +import java.util.HashSet; +import org.apache.commons.collections.CollectionUtils; +import org.apache.hadoop.thirdparty.com.google.common.collect.ImmutableSet; import org.apache.hadoop.yarn.api.protocolrecords.GetApplicationsResponse; import org.apache.hadoop.yarn.api.protocolrecords.GetClusterMetricsResponse; +import org.apache.hadoop.yarn.api.protocolrecords.GetNodesToLabelsResponse; +import org.apache.hadoop.yarn.api.protocolrecords.GetClusterNodeLabelsResponse; +import org.apache.hadoop.yarn.api.protocolrecords.GetLabelsToNodesResponse; +import org.apache.hadoop.yarn.api.protocolrecords.GetQueueUserAclsInfoResponse; +import org.apache.hadoop.yarn.api.protocolrecords.ReservationListResponse; +import org.apache.hadoop.yarn.api.protocolrecords.GetAllResourceTypeInfoResponse; import org.apache.hadoop.yarn.api.records.ApplicationAttemptId; import org.apache.hadoop.yarn.api.records.ApplicationId; import org.apache.hadoop.yarn.api.records.ApplicationReport; @@ -31,6 +43,15 @@ import org.apache.hadoop.yarn.api.records.Resource; import org.apache.hadoop.yarn.api.records.YarnApplicationState; import org.apache.hadoop.yarn.api.records.YarnClusterMetrics; +import org.apache.hadoop.yarn.api.records.NodeId; +import org.apache.hadoop.yarn.api.records.NodeLabel; +import org.apache.hadoop.yarn.api.records.QueueACL; +import org.apache.hadoop.yarn.api.records.QueueUserACLInfo; +import org.apache.hadoop.yarn.api.records.ReservationAllocationState; +import org.apache.hadoop.yarn.api.records.ReservationDefinition; +import org.apache.hadoop.yarn.api.records.ReservationId; +import org.apache.hadoop.yarn.api.records.ResourceTypeInfo; +import org.apache.hadoop.yarn.util.Records; import org.apache.hadoop.yarn.server.uam.UnmanagedApplicationManager; import org.junit.Assert; import org.junit.Test; @@ -213,4 +234,309 @@ private GetApplicationsResponse getApplicationsResponse(int value, return GetApplicationsResponse.newInstance(applications); } + + @Test + public void testMergeNodesToLabelsResponse() { + NodeId node1 = NodeId.fromString("SubCluster1Node1:1111"); + NodeId node2 = NodeId.fromString("SubCluster1Node2:2222"); + NodeId node3 = NodeId.fromString("SubCluster2Node1:1111"); + + Map> nodeLabelsMapSC1 = new HashMap<>(); + nodeLabelsMapSC1.put(node1, ImmutableSet.of("node1")); + nodeLabelsMapSC1.put(node2, ImmutableSet.of("node2")); + nodeLabelsMapSC1.put(node3, ImmutableSet.of("node3")); + + // normal response + GetNodesToLabelsResponse response1 = Records.newRecord( + GetNodesToLabelsResponse.class); + response1.setNodeToLabels(nodeLabelsMapSC1); + + // empty response + Map> nodeLabelsMapSC2 = new HashMap<>(); + GetNodesToLabelsResponse response2 = Records.newRecord( + GetNodesToLabelsResponse.class); + response2.setNodeToLabels(nodeLabelsMapSC2); + + // null response + GetNodesToLabelsResponse response3 = null; + + Map> expectedResponse = new HashMap<>(); + expectedResponse.put(node1, ImmutableSet.of("node1")); + expectedResponse.put(node2, ImmutableSet.of("node2")); + expectedResponse.put(node3, ImmutableSet.of("node3")); + + List responses = new ArrayList<>(); + responses.add(response1); + responses.add(response2); + responses.add(response3); + + GetNodesToLabelsResponse response = RouterYarnClientUtils. + mergeNodesToLabelsResponse(responses); + Assert.assertEquals(expectedResponse, response.getNodeToLabels()); + } + + @Test + public void testMergeClusterNodeLabelsResponse() { + NodeLabel nodeLabel1 = NodeLabel.newInstance("nodeLabel1"); + NodeLabel nodeLabel2 = NodeLabel.newInstance("nodeLabel2"); + NodeLabel nodeLabel3 = NodeLabel.newInstance("nodeLabel3"); + + // normal response + List nodeLabelListSC1 = new ArrayList<>(); + nodeLabelListSC1.add(nodeLabel1); + nodeLabelListSC1.add(nodeLabel2); + nodeLabelListSC1.add(nodeLabel3); + + GetClusterNodeLabelsResponse response1 = Records.newRecord( + GetClusterNodeLabelsResponse.class); + response1.setNodeLabelList(nodeLabelListSC1); + + // empty response + List nodeLabelListSC2 = new ArrayList<>(); + + GetClusterNodeLabelsResponse response2 = Records.newRecord( + GetClusterNodeLabelsResponse.class); + response2.setNodeLabelList(nodeLabelListSC2); + + // null response + GetClusterNodeLabelsResponse response3 = null; + + List responses = new ArrayList<>(); + responses.add(response1); + responses.add(response2); + responses.add(response3); + + List expectedResponse = new ArrayList<>(); + expectedResponse.add(nodeLabel1); + expectedResponse.add(nodeLabel2); + expectedResponse.add(nodeLabel3); + + GetClusterNodeLabelsResponse response = RouterYarnClientUtils. + mergeClusterNodeLabelsResponse(responses); + Assert.assertTrue(CollectionUtils.isEqualCollection(expectedResponse, + response.getNodeLabelList())); + } + + @Test + public void testMergeLabelsToNodes(){ + NodeId node1 = NodeId.fromString("SubCluster1Node1:1111"); + NodeId node2 = NodeId.fromString("SubCluster1Node2:2222"); + NodeId node3 = NodeId.fromString("SubCluster2node1:1111"); + NodeId node4 = NodeId.fromString("SubCluster2node2:2222"); + + Map> labelsToNodesSC1 = new HashMap<>(); + + Set nodeIdSet1 = new HashSet<>(); + nodeIdSet1.add(node1); + nodeIdSet1.add(node2); + labelsToNodesSC1.put("Label1", nodeIdSet1); + + // normal response + GetLabelsToNodesResponse response1 = Records.newRecord( + GetLabelsToNodesResponse.class); + response1.setLabelsToNodes(labelsToNodesSC1); + Map> labelsToNodesSC2 = new HashMap<>(); + Set nodeIdSet2 = new HashSet<>(); + nodeIdSet2.add(node3); + Set nodeIdSet3 = new HashSet<>(); + nodeIdSet3.add(node4); + labelsToNodesSC2.put("Label1", nodeIdSet2); + labelsToNodesSC2.put("Label2", nodeIdSet3); + + GetLabelsToNodesResponse response2 = Records.newRecord( + GetLabelsToNodesResponse.class); + response2.setLabelsToNodes(labelsToNodesSC2); + + // empty response + GetLabelsToNodesResponse response3 = Records.newRecord( + GetLabelsToNodesResponse.class); + + // null response + GetLabelsToNodesResponse response4 = null; + + List responses = new ArrayList<>(); + responses.add(response1); + responses.add(response2); + responses.add(response3); + responses.add(response4); + + Map> expectedResponse = new HashMap<>(); + Set nodeIdMergedSet1 = new HashSet<>(); + nodeIdMergedSet1.add(node1); + nodeIdMergedSet1.add(node2); + nodeIdMergedSet1.add(node3); + + Set nodeIdMergedSet2 = new HashSet<>(); + nodeIdMergedSet2.add(node4); + expectedResponse.put("Label1", nodeIdMergedSet1); + expectedResponse.put("Label2", nodeIdMergedSet2); + + GetLabelsToNodesResponse response = RouterYarnClientUtils. + mergeLabelsToNodes(responses); + + Assert.assertEquals(expectedResponse, response.getLabelsToNodes()); + } + + @Test + public void testMergeQueueUserAclsResponse() { + + List submitOnlyAcl = new ArrayList<>(); + submitOnlyAcl.add(QueueACL.SUBMIT_APPLICATIONS); + + List administerOnlyAcl = new ArrayList<>(); + administerOnlyAcl.add(QueueACL.ADMINISTER_QUEUE); + + List submitAndAdministerAcl = new ArrayList<>(); + submitAndAdministerAcl.add(QueueACL.ADMINISTER_QUEUE); + submitAndAdministerAcl.add(QueueACL.SUBMIT_APPLICATIONS); + + QueueUserACLInfo queueUserACLInfo1 = QueueUserACLInfo.newInstance( + "root", submitAndAdministerAcl); + + QueueUserACLInfo queueUserACLInfo2 = QueueUserACLInfo.newInstance( + "default", submitOnlyAcl); + + QueueUserACLInfo queueUserACLInfo3 = QueueUserACLInfo.newInstance( + "root", submitAndAdministerAcl); + + QueueUserACLInfo queueUserACLInfo4 = QueueUserACLInfo.newInstance( + "yarn", administerOnlyAcl); + + List queueUserACLInfoList1 = new ArrayList<>(); + List queueUserACLInfoList2 = new ArrayList<>(); + + queueUserACLInfoList1.add(queueUserACLInfo1); + queueUserACLInfoList1.add(queueUserACLInfo2); + queueUserACLInfoList2.add(queueUserACLInfo3); + queueUserACLInfoList2.add(queueUserACLInfo4); + + // normal response + GetQueueUserAclsInfoResponse response1 = Records.newRecord( + GetQueueUserAclsInfoResponse.class); + response1.setUserAclsInfoList(queueUserACLInfoList1); + GetQueueUserAclsInfoResponse response2 = Records.newRecord( + GetQueueUserAclsInfoResponse.class); + response2.setUserAclsInfoList(queueUserACLInfoList2); + + // empty response + GetQueueUserAclsInfoResponse response3 = Records.newRecord( + GetQueueUserAclsInfoResponse.class); + + // null responce + GetQueueUserAclsInfoResponse response4 = null; + + List responses = new ArrayList<>(); + responses.add(response1); + responses.add(response2); + responses.add(response3); + responses.add(response4); + + // expected user acls + List expectedOutput = new ArrayList<>(); + expectedOutput.add(queueUserACLInfo1); + expectedOutput.add(queueUserACLInfo2); + expectedOutput.add(queueUserACLInfo4); + + GetQueueUserAclsInfoResponse response = + RouterYarnClientUtils.mergeQueueUserAcls(responses); + Assert.assertTrue(CollectionUtils.isEqualCollection(expectedOutput, + response.getUserAclsInfoList())); + } + + @Test + public void testMergeReservationsList() { + + // normal response + ReservationListResponse response1 = createReservationListResponse( + 165348678000L, 165348690000L, 165348678000L, 1L); + + ReservationListResponse response2 = createReservationListResponse( + 165348750000L, 165348768000L, 165348750000L, 1L); + + // empty response + ReservationListResponse response3 = ReservationListResponse.newInstance(new ArrayList<>()); + + // null response + ReservationListResponse response4 = null; + + List responses = new ArrayList<>(); + responses.add(response1); + responses.add(response2); + responses.add(response3); + responses.add(response4); + + // expected response + List expectedResponse = new ArrayList<>(); + expectedResponse.addAll(response1.getReservationAllocationState()); + expectedResponse.addAll(response2.getReservationAllocationState()); + + ReservationListResponse response = + RouterYarnClientUtils.mergeReservationsList(responses); + Assert.assertEquals(expectedResponse, response.getReservationAllocationState()); + } + + private ReservationListResponse createReservationListResponse(long startTime, + long endTime, long reservationTime, long reservationNumber) { + List reservationsList = new ArrayList<>(); + ReservationDefinition reservationDefinition = + Records.newRecord(ReservationDefinition.class); + reservationDefinition.setArrival(startTime); + reservationDefinition.setDeadline(endTime); + ReservationAllocationState reservationAllocationState = + Records.newRecord(ReservationAllocationState.class); + ReservationId reservationId = ReservationId.newInstance(reservationTime, + reservationNumber); + reservationAllocationState.setReservationDefinition(reservationDefinition); + reservationAllocationState.setReservationId(reservationId); + reservationsList.add(reservationAllocationState); + return ReservationListResponse.newInstance(reservationsList); + } + + @Test + public void testMergeResourceTypes() { + + ResourceTypeInfo resourceTypeInfo1 = ResourceTypeInfo.newInstance("vcores"); + ResourceTypeInfo resourceTypeInfo2 = ResourceTypeInfo.newInstance("gpu"); + ResourceTypeInfo resourceTypeInfo3 = ResourceTypeInfo.newInstance("memory-mb"); + + List resourceTypeInfoList1 = new ArrayList<>(); + resourceTypeInfoList1.add(resourceTypeInfo1); + resourceTypeInfoList1.add(resourceTypeInfo3); + + List resourceTypeInfoList2 = new ArrayList<>(); + resourceTypeInfoList2.add(resourceTypeInfo3); + resourceTypeInfoList2.add(resourceTypeInfo2); + + // normal response + GetAllResourceTypeInfoResponse response1 = + Records.newRecord(GetAllResourceTypeInfoResponse.class); + response1.setResourceTypeInfo(resourceTypeInfoList1); + + GetAllResourceTypeInfoResponse response2 = + Records.newRecord(GetAllResourceTypeInfoResponse.class); + response2.setResourceTypeInfo(resourceTypeInfoList2); + + // empty response + GetAllResourceTypeInfoResponse response3 = + Records.newRecord(GetAllResourceTypeInfoResponse.class); + + // null response + GetAllResourceTypeInfoResponse response4 = null; + + List responses = new ArrayList<>(); + responses.add(response1); + responses.add(response2); + responses.add(response3); + responses.add(response4); + + // expected response + List expectedResponse = new ArrayList<>(); + expectedResponse.add(resourceTypeInfo1); + expectedResponse.add(resourceTypeInfo2); + expectedResponse.add(resourceTypeInfo3); + GetAllResourceTypeInfoResponse response = + RouterYarnClientUtils.mergeResourceTypes(responses); + Assert.assertTrue(CollectionUtils.isEqualCollection(expectedResponse, + response.getResourceTypeInfo())); + } } diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-router/src/test/java/org/apache/hadoop/yarn/server/router/clientrm/TestableFederationClientInterceptor.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-router/src/test/java/org/apache/hadoop/yarn/server/router/clientrm/TestableFederationClientInterceptor.java index c97d05324259b..af1f45924c19c 100644 --- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-router/src/test/java/org/apache/hadoop/yarn/server/router/clientrm/TestableFederationClientInterceptor.java +++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-router/src/test/java/org/apache/hadoop/yarn/server/router/clientrm/TestableFederationClientInterceptor.java @@ -33,6 +33,7 @@ import org.apache.hadoop.yarn.server.federation.store.records.SubClusterId; import org.apache.hadoop.yarn.server.resourcemanager.ClientRMService; import org.apache.hadoop.yarn.server.resourcemanager.MockRM; +import org.apache.hadoop.yarn.server.resourcemanager.MockNM; import org.apache.hadoop.yarn.server.resourcemanager.RMAppManager; import org.apache.hadoop.yarn.server.resourcemanager.RMContext; import org.apache.hadoop.yarn.server.resourcemanager.scheduler.YarnScheduler; @@ -51,6 +52,9 @@ public class TestableFederationClientInterceptor private ConcurrentHashMap mockRMs = new ConcurrentHashMap<>(); + private ConcurrentHashMap mockNMs = + new ConcurrentHashMap<>(); + private List badSubCluster = new ArrayList(); @Override @@ -71,7 +75,8 @@ protected ApplicationClientProtocol getClientRMProxyForSubCluster( mockRM.init(super.getConf()); mockRM.start(); try { - mockRM.registerNode("h1:1234", 1024); + MockNM nm = mockRM.registerNode("127.0.0.1:1234", 8*1024, 4); + mockNMs.put(subClusterId, nm); } catch (Exception e) { Assert.fail(e.getMessage()); } @@ -115,4 +120,11 @@ protected void registerBadSubCluster(SubClusterId badSC) throws IOException { } } + public ConcurrentHashMap getMockRMs() { + return mockRMs; + } + + public ConcurrentHashMap getMockNMs() { + return mockNMs; + } } diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-router/src/test/java/org/apache/hadoop/yarn/server/router/webapp/TestRouterWebServicesREST.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-router/src/test/java/org/apache/hadoop/yarn/server/router/webapp/TestRouterWebServicesREST.java index 868a953e9adfd..b345ebdd90202 100644 --- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-router/src/test/java/org/apache/hadoop/yarn/server/router/webapp/TestRouterWebServicesREST.java +++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-router/src/test/java/org/apache/hadoop/yarn/server/router/webapp/TestRouterWebServicesREST.java @@ -20,7 +20,7 @@ import static javax.servlet.http.HttpServletResponse.SC_ACCEPTED; import static javax.servlet.http.HttpServletResponse.SC_BAD_REQUEST; -import static javax.servlet.http.HttpServletResponse.SC_INTERNAL_SERVER_ERROR; +import static javax.servlet.http.HttpServletResponse.SC_SERVICE_UNAVAILABLE; import static javax.servlet.http.HttpServletResponse.SC_OK; import static javax.ws.rs.core.MediaType.APPLICATION_JSON; import static javax.ws.rs.core.MediaType.APPLICATION_XML; @@ -143,6 +143,8 @@ import net.jcip.annotations.NotThreadSafe; +import javax.servlet.http.HttpServletRequest; + /** * This test validate E2E the correctness of the RouterWebServices. It starts * Router, RM and NM in 3 different processes to avoid servlet conflicts. Each @@ -423,7 +425,7 @@ public void testSchedulerInfoXML() throws Exception { /** * This test validates the correctness of - * {@link RMWebServiceProtocol#getNodes()} inside Router. + * {@link RMWebServiceProtocol#getNodes(String)} inside Router. */ @Test(timeout = 2000) public void testNodesEmptyXML() throws Exception { @@ -444,7 +446,7 @@ public void testNodesEmptyXML() throws Exception { /** * This test validates the correctness of - * {@link RMWebServiceProtocol#getNodes()} inside Router. + * {@link RMWebServiceProtocol#getNodes(String)} inside Router. */ @Test(timeout = 2000) public void testNodesXML() throws Exception { @@ -465,7 +467,7 @@ public void testNodesXML() throws Exception { /** * This test validates the correctness of - * {@link RMWebServiceProtocol#getNode()} inside Router. + * {@link RMWebServiceProtocol#getNode(String)} inside Router. */ @Test(timeout = 2000) public void testNodeXML() throws Exception { @@ -528,7 +530,7 @@ public void testUpdateNodeResource() throws Exception { /** * This test validates the correctness of - * {@link RMWebServiceProtocol#getActivities()} inside Router. + * {@link RMWebServiceProtocol#getActivities(HttpServletRequest, String, String)} inside Router. */ @Test(timeout = 2000) public void testActiviesXML() throws Exception { @@ -600,7 +602,7 @@ public void testDumpSchedulerLogsXML() throws Exception { performCall(RM_WEB_SERVICE_PATH + SCHEDULER_LOGS, null, null, null, PUT); - assertEquals(SC_INTERNAL_SERVER_ERROR, badResponse.getStatus()); + assertEquals(SC_SERVICE_UNAVAILABLE, badResponse.getStatus()); // Test with the correct HTTP method ClientResponse response = performCall( @@ -623,7 +625,7 @@ public void testNewApplicationXML() throws Exception { RM_WEB_SERVICE_PATH + APPS_NEW_APPLICATION, null, null, null, PUT); - assertEquals(SC_INTERNAL_SERVER_ERROR, badResponse.getStatus()); + assertEquals(SC_SERVICE_UNAVAILABLE, badResponse.getStatus()); // Test with the correct HTTP method ClientResponse response = performCall( @@ -646,7 +648,7 @@ public void testSubmitApplicationXML() throws Exception { ClientResponse badResponse = performCall( RM_WEB_SERVICE_PATH + APPS, null, null, null, PUT); - assertEquals(SC_INTERNAL_SERVER_ERROR, badResponse.getStatus()); + assertEquals(SC_SERVICE_UNAVAILABLE, badResponse.getStatus()); // Test with the correct HTTP method ApplicationSubmissionContextInfo context = @@ -771,7 +773,7 @@ public void testUpdateAppStateXML() throws Exception { ClientResponse badResponse = performCall( pathApp, null, null, null, POST); - assertEquals(SC_INTERNAL_SERVER_ERROR, badResponse.getStatus()); + assertEquals(SC_SERVICE_UNAVAILABLE, badResponse.getStatus()); // Test with the correct HTTP method AppState appState = new AppState("KILLED"); @@ -820,7 +822,7 @@ public void testUpdateAppPriorityXML() throws Exception { RM_WEB_SERVICE_PATH + format(APPS_APPID_PRIORITY, appId), null, null, null, POST); - assertEquals(SC_INTERNAL_SERVER_ERROR, badResponse.getStatus()); + assertEquals(SC_SERVICE_UNAVAILABLE, badResponse.getStatus()); // Test with the correct HTTP method AppPriority appPriority = new AppPriority(1); @@ -870,7 +872,7 @@ public void testUpdateAppQueueXML() throws Exception { RM_WEB_SERVICE_PATH + format(APPS_APPID_QUEUE, appId), null, null, null, POST); - assertEquals(SC_INTERNAL_SERVER_ERROR, badResponse.getStatus()); + assertEquals(SC_SERVICE_UNAVAILABLE, badResponse.getStatus()); // Test with the correct HTTP method AppQueue appQueue = new AppQueue("default"); @@ -945,7 +947,7 @@ public void testUpdateAppTimeoutsXML() throws Exception { RM_WEB_SERVICE_PATH + format(APPS_TIMEOUT, appId), null, null, null, POST); - assertEquals(SC_INTERNAL_SERVER_ERROR, badResponse.getStatus()); + assertEquals(SC_SERVICE_UNAVAILABLE, badResponse.getStatus()); // Test with a bad request AppTimeoutInfo appTimeoutInfo = new AppTimeoutInfo(); @@ -971,7 +973,7 @@ public void testNewReservationXML() throws Exception { RM_WEB_SERVICE_PATH + RESERVATION_NEW, null, null, null, PUT); - assertEquals(SC_INTERNAL_SERVER_ERROR, badResponse.getStatus()); + assertEquals(SC_SERVICE_UNAVAILABLE, badResponse.getStatus()); // Test with the correct HTTP method ClientResponse response = performCall( @@ -995,7 +997,7 @@ public void testSubmitReservationXML() throws Exception { RM_WEB_SERVICE_PATH + RESERVATION_SUBMIT, null, null, null, PUT); - assertEquals(SC_INTERNAL_SERVER_ERROR, badResponse.getStatus()); + assertEquals(SC_SERVICE_UNAVAILABLE, badResponse.getStatus()); // Test with the correct HTTP method ReservationSubmissionRequestInfo context = @@ -1022,7 +1024,7 @@ public void testUpdateReservationXML() throws Exception { ClientResponse badResponse = performCall( RM_WEB_SERVICE_PATH + RESERVATION_UPDATE, null, null, null, PUT); - assertEquals(SC_INTERNAL_SERVER_ERROR, badResponse.getStatus()); + assertEquals(SC_SERVICE_UNAVAILABLE, badResponse.getStatus()); // Test with the correct HTTP method String reservationId = getNewReservationId().getReservationId(); @@ -1048,7 +1050,7 @@ public void testDeleteReservationXML() throws Exception { ClientResponse badResponse = performCall( RM_WEB_SERVICE_PATH + RESERVATION_DELETE, null, null, null, PUT); - assertEquals(SC_INTERNAL_SERVER_ERROR, badResponse.getStatus()); + assertEquals(SC_SERVICE_UNAVAILABLE, badResponse.getStatus()); // Test with the correct HTTP method String reservationId = getNewReservationId().getReservationId(); @@ -1185,7 +1187,7 @@ public void testAddToClusterNodeLabelsXML() throws Exception { RM_WEB_SERVICE_PATH + ADD_NODE_LABELS, null, null, null, PUT); - assertEquals(SC_INTERNAL_SERVER_ERROR, badResponse.getStatus()); + assertEquals(SC_SERVICE_UNAVAILABLE, badResponse.getStatus()); // Test with the correct HTTP method @@ -1213,7 +1215,7 @@ public void testRemoveFromClusterNodeLabelsXML() ClientResponse badResponse = performCall( RM_WEB_SERVICE_PATH + REMOVE_NODE_LABELS, null, null, null, PUT); - assertEquals(SC_INTERNAL_SERVER_ERROR, badResponse.getStatus()); + assertEquals(SC_SERVICE_UNAVAILABLE, badResponse.getStatus()); // Test with the correct HTTP method addNodeLabel(); @@ -1238,7 +1240,7 @@ public void testReplaceLabelsOnNodesXML() throws Exception { ClientResponse badResponse = performCall( RM_WEB_SERVICE_PATH + REPLACE_NODE_TO_LABELS, null, null, null, PUT); - assertEquals(SC_INTERNAL_SERVER_ERROR, badResponse.getStatus()); + assertEquals(SC_SERVICE_UNAVAILABLE, badResponse.getStatus()); // Test with the correct HTTP method addNodeLabel(); @@ -1267,7 +1269,7 @@ public void testReplaceLabelsOnNodeXML() throws Exception { ClientResponse badResponse = performCall( pathNode, null, null, null, PUT); - assertEquals(SC_INTERNAL_SERVER_ERROR, badResponse.getStatus()); + assertEquals(SC_SERVICE_UNAVAILABLE, badResponse.getStatus()); // Test with the correct HTTP method addNodeLabel(); diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-router/src/test/resources/capacity-scheduler.xml b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-router/src/test/resources/capacity-scheduler.xml index 90c5eeb097e8b..ffe1a5d4ca48c 100644 --- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-router/src/test/resources/capacity-scheduler.xml +++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-router/src/test/resources/capacity-scheduler.xml @@ -45,7 +45,7 @@ yarn.scheduler.capacity.root.queues - default + default,decided The queues at the this level (root is the root queue). @@ -97,6 +97,15 @@ + + yarn.scheduler.capacity.root.decided.reservable + true + + indicates to the ReservationSystem that the queue’s resources + is available for users to reserve. + + + yarn.scheduler.capacity.node-locality-delay -1 diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-router/src/test/resources/yarn-site.xml b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-router/src/test/resources/yarn-site.xml index f3e0de3604b60..310a1612486bf 100644 --- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-router/src/test/resources/yarn-site.xml +++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-router/src/test/resources/yarn-site.xml @@ -27,4 +27,8 @@ yarn.resourcemanager.webapp.address 0.0.0.0:8080 + + yarn.cluster.max-application-priority + 50 + diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-tests/src/test/java/org/apache/hadoop/yarn/server/TestMiniYARNClusterForHA.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-tests/src/test/java/org/apache/hadoop/yarn/server/TestMiniYARNClusterForHA.java index 384d1cd78f2e5..ed596340d7c55 100644 --- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-tests/src/test/java/org/apache/hadoop/yarn/server/TestMiniYARNClusterForHA.java +++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-tests/src/test/java/org/apache/hadoop/yarn/server/TestMiniYARNClusterForHA.java @@ -24,7 +24,6 @@ import java.io.IOException; import org.apache.hadoop.conf.Configuration; -import org.apache.hadoop.ha.HAServiceProtocol; import org.apache.hadoop.yarn.conf.YarnConfiguration; import org.apache.hadoop.yarn.exceptions.YarnException; import org.junit.Before; diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-timeline-pluginstorage/src/test/java/org/apache/hadoop/yarn/server/timeline/EntityGroupPlugInForTest.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-timeline-pluginstorage/src/test/java/org/apache/hadoop/yarn/server/timeline/EntityGroupPlugInForTest.java index f239fac32395b..89208054de3f7 100644 --- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-timeline-pluginstorage/src/test/java/org/apache/hadoop/yarn/server/timeline/EntityGroupPlugInForTest.java +++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-timeline-pluginstorage/src/test/java/org/apache/hadoop/yarn/server/timeline/EntityGroupPlugInForTest.java @@ -20,7 +20,6 @@ import org.apache.hadoop.util.Sets; import org.apache.hadoop.yarn.api.records.ApplicationId; import org.apache.hadoop.yarn.api.records.timeline.TimelineEntityGroupId; -import org.apache.hadoop.yarn.util.ConverterUtils; import java.util.Collection; import java.util.Set; diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-timeline-pluginstorage/src/test/java/org/apache/hadoop/yarn/server/timeline/TestLogInfo.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-timeline-pluginstorage/src/test/java/org/apache/hadoop/yarn/server/timeline/TestLogInfo.java index dd38565cc5962..9182b26c6abea 100644 --- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-timeline-pluginstorage/src/test/java/org/apache/hadoop/yarn/server/timeline/TestLogInfo.java +++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-timeline-pluginstorage/src/test/java/org/apache/hadoop/yarn/server/timeline/TestLogInfo.java @@ -41,7 +41,6 @@ import java.io.IOException; import java.io.OutputStream; import java.nio.charset.Charset; -import java.util.EnumSet; import static org.junit.Assert.assertEquals; import static org.junit.Assert.assertFalse; diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-timelineservice-documentstore/src/main/java/org/apache/hadoop/yarn/server/timelineservice/documentstore/DocumentStoreTimelineReaderImpl.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-timelineservice-documentstore/src/main/java/org/apache/hadoop/yarn/server/timelineservice/documentstore/DocumentStoreTimelineReaderImpl.java index 8de3b8645ac50..7d8098e40d7fd 100755 --- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-timelineservice-documentstore/src/main/java/org/apache/hadoop/yarn/server/timelineservice/documentstore/DocumentStoreTimelineReaderImpl.java +++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-timelineservice-documentstore/src/main/java/org/apache/hadoop/yarn/server/timelineservice/documentstore/DocumentStoreTimelineReaderImpl.java @@ -108,7 +108,7 @@ public TimelineHealth getHealthStatus() { ""); } else { return new TimelineHealth( - TimelineHealth.TimelineHealthStatus.READER_CONNECTION_FAILURE, + TimelineHealth.TimelineHealthStatus.CONNECTION_FAILURE, "Timeline store reader not initialized."); } } @@ -131,4 +131,4 @@ private Set applyFilters(TimelineEntityFilters filters, } return timelineEntities; } -} \ No newline at end of file +} diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-timelineservice-documentstore/src/main/java/org/apache/hadoop/yarn/server/timelineservice/documentstore/DocumentStoreTimelineWriterImpl.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-timelineservice-documentstore/src/main/java/org/apache/hadoop/yarn/server/timelineservice/documentstore/DocumentStoreTimelineWriterImpl.java index 572d888b5299e..0ea70f9dcf922 100755 --- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-timelineservice-documentstore/src/main/java/org/apache/hadoop/yarn/server/timelineservice/documentstore/DocumentStoreTimelineWriterImpl.java +++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-timelineservice-documentstore/src/main/java/org/apache/hadoop/yarn/server/timelineservice/documentstore/DocumentStoreTimelineWriterImpl.java @@ -25,6 +25,7 @@ import org.apache.hadoop.yarn.server.metrics.AppAttemptMetricsConstants; import org.apache.hadoop.yarn.server.metrics.ApplicationMetricsConstants; import org.apache.hadoop.yarn.server.metrics.ContainerMetricsConstants; +import org.apache.hadoop.yarn.api.records.timeline.TimelineHealth; import org.apache.hadoop.yarn.server.timelineservice.collector.TimelineCollectorContext; import org.apache.hadoop.yarn.server.timelineservice.documentstore.lib.DocumentStoreVendor; import org.apache.hadoop.yarn.server.timelineservice.storage.TimelineAggregationTrack; @@ -151,6 +152,11 @@ public TimelineWriteResponse write(TimelineCollectorContext context, return null; } + @Override + public TimelineHealth getHealthStatus() { + return new TimelineHealth(TimelineHealth.TimelineHealthStatus.RUNNING, ""); + } + private void appendSubAppUserIfExists(TimelineCollectorContext context, String subApplicationUser) { String userId = context.getUserId(); @@ -282,4 +288,4 @@ public TimelineWriteResponse aggregate(TimelineEntity data, @Override public void flush() { } -} \ No newline at end of file +} diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-timelineservice-hbase/hadoop-yarn-server-timelineservice-hbase-client/src/main/java/org/apache/hadoop/yarn/server/timelineservice/storage/HBaseTimelineReaderImpl.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-timelineservice-hbase/hadoop-yarn-server-timelineservice-hbase-client/src/main/java/org/apache/hadoop/yarn/server/timelineservice/storage/HBaseTimelineReaderImpl.java index f3592d2924088..b6e1d76902841 100644 --- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-timelineservice-hbase/hadoop-yarn-server-timelineservice-hbase-client/src/main/java/org/apache/hadoop/yarn/server/timelineservice/storage/HBaseTimelineReaderImpl.java +++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-timelineservice-hbase/hadoop-yarn-server-timelineservice-hbase-client/src/main/java/org/apache/hadoop/yarn/server/timelineservice/storage/HBaseTimelineReaderImpl.java @@ -115,7 +115,7 @@ public TimelineHealth getHealthStatus() { ""); } catch (IOException e){ return new TimelineHealth( - TimelineHealth.TimelineHealthStatus.READER_CONNECTION_FAILURE, + TimelineHealth.TimelineHealthStatus.CONNECTION_FAILURE, "HBase connection is down"); } } diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-timelineservice-hbase/hadoop-yarn-server-timelineservice-hbase-client/src/main/java/org/apache/hadoop/yarn/server/timelineservice/storage/HBaseTimelineWriterImpl.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-timelineservice-hbase/hadoop-yarn-server-timelineservice-hbase-client/src/main/java/org/apache/hadoop/yarn/server/timelineservice/storage/HBaseTimelineWriterImpl.java index dda004d7ec928..7233dab34572b 100644 --- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-timelineservice-hbase/hadoop-yarn-server-timelineservice-hbase-client/src/main/java/org/apache/hadoop/yarn/server/timelineservice/storage/HBaseTimelineWriterImpl.java +++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-timelineservice-hbase/hadoop-yarn-server-timelineservice-hbase-client/src/main/java/org/apache/hadoop/yarn/server/timelineservice/storage/HBaseTimelineWriterImpl.java @@ -28,6 +28,7 @@ import org.apache.hadoop.hbase.client.ConnectionFactory; import org.apache.hadoop.security.UserGroupInformation; import org.apache.hadoop.service.AbstractService; +import org.apache.hadoop.yarn.api.records.timeline.TimelineHealth; import org.apache.hadoop.yarn.api.records.timelineservice.ApplicationEntity; import org.apache.hadoop.yarn.api.records.timelineservice.SubApplicationEntity; import org.apache.hadoop.yarn.api.records.timelineservice.TimelineDomain; @@ -604,6 +605,19 @@ public TimelineWriteResponse aggregate(TimelineEntity data, return null; } + @Override + public TimelineHealth getHealthStatus() { + try { + storageMonitor.checkStorageIsUp(); + return new TimelineHealth(TimelineHealth.TimelineHealthStatus.RUNNING, + ""); + } catch (IOException e){ + return new TimelineHealth( + TimelineHealth.TimelineHealthStatus.CONNECTION_FAILURE, + "HBase connection is down"); + } + } + /* * (non-Javadoc) * diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-timelineservice-hbase/hadoop-yarn-server-timelineservice-hbase-client/src/main/java/org/apache/hadoop/yarn/server/timelineservice/storage/apptoflow/AppToFlowTableRW.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-timelineservice-hbase/hadoop-yarn-server-timelineservice-hbase-client/src/main/java/org/apache/hadoop/yarn/server/timelineservice/storage/apptoflow/AppToFlowTableRW.java index 05c4e570d65b7..3ea18adb94328 100644 --- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-timelineservice-hbase/hadoop-yarn-server-timelineservice-hbase-client/src/main/java/org/apache/hadoop/yarn/server/timelineservice/storage/apptoflow/AppToFlowTableRW.java +++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-timelineservice-hbase/hadoop-yarn-server-timelineservice-hbase-client/src/main/java/org/apache/hadoop/yarn/server/timelineservice/storage/apptoflow/AppToFlowTableRW.java @@ -26,7 +26,7 @@ import org.apache.hadoop.hbase.regionserver.BloomType; import org.apache.hadoop.yarn.conf.YarnConfiguration; import org.apache.hadoop.yarn.server.timelineservice.storage.common.BaseTableRW; -import org.apache.hadoop.yarn.server.timelineservice.storage.common.TimelineHBaseSchemaConstants; + import org.slf4j.Logger; import org.slf4j.LoggerFactory; diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-timelineservice/src/main/java/org/apache/hadoop/yarn/server/timelineservice/collector/TimelineCollector.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-timelineservice/src/main/java/org/apache/hadoop/yarn/server/timelineservice/collector/TimelineCollector.java index 0c54ed03f9edc..1c1fe118b843c 100644 --- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-timelineservice/src/main/java/org/apache/hadoop/yarn/server/timelineservice/collector/TimelineCollector.java +++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-timelineservice/src/main/java/org/apache/hadoop/yarn/server/timelineservice/collector/TimelineCollector.java @@ -34,6 +34,7 @@ import org.apache.hadoop.conf.Configuration; import org.apache.hadoop.security.UserGroupInformation; import org.apache.hadoop.service.CompositeService; +import org.apache.hadoop.yarn.api.records.timeline.TimelineHealth; import org.apache.hadoop.yarn.api.records.timelineservice.TimelineDomain; import org.apache.hadoop.yarn.api.records.timelineservice.TimelineMetricOperation; import org.apache.hadoop.yarn.api.records.timelineservice.TimelineEntities; @@ -71,6 +72,9 @@ public abstract class TimelineCollector extends CompositeService { private volatile boolean isStopped = false; + private int maxWriteRetries; + private long writeRetryInterval; + public TimelineCollector(String name) { super(name); } @@ -86,6 +90,13 @@ protected void serviceInit(Configuration conf) throws Exception { new ArrayBlockingQueue<>(capacity)); pool.setRejectedExecutionHandler( new ThreadPoolExecutor.DiscardOldestPolicy()); + + maxWriteRetries = + conf.getInt(YarnConfiguration.TIMELINE_SERVICE_CLIENT_MAX_RETRIES, + YarnConfiguration.DEFAULT_TIMELINE_SERVICE_CLIENT_MAX_RETRIES); + writeRetryInterval = conf.getLong( + YarnConfiguration.TIMELINE_SERVICE_CLIENT_RETRY_INTERVAL_MS, + YarnConfiguration.DEFAULT_TIMELINE_SERVICE_CLIENT_RETRY_INTERVAL_MS); } @Override @@ -153,18 +164,54 @@ public TimelineWriteResponse putEntities(TimelineEntities entities, UserGroupInformation callerUgi) throws IOException { LOG.debug("putEntities(entities={}, callerUgi={})", entities, callerUgi); - TimelineWriteResponse response; - // synchronize on the writer object so that no other threads can - // flush the writer buffer concurrently and swallow any exception - // caused by the timeline enitites that are being put here. - synchronized (writer) { - response = writeTimelineEntities(entities, callerUgi); - flushBufferedTimelineEntities(); + TimelineWriteResponse response = null; + try { + boolean isStorageUp = checkRetryWithSleep(); + if (isStorageUp) { + // synchronize on the writer object so that no other threads can + // flush the writer buffer concurrently and swallow any exception + // caused by the timeline enitites that are being put here. + synchronized (writer) { + response = writeTimelineEntities(entities, callerUgi); + flushBufferedTimelineEntities(); + } + } else { + String msg = String.format("Failed to putEntities(" + + "entities=%s, callerUgi=%s) as Timeline Storage is Down", + entities, callerUgi); + throw new IOException(msg); + } + } catch (InterruptedException ex) { + String msg = String.format("Interrupted while retrying to putEntities(" + + "entities=%s, callerUgi=%s)", entities, callerUgi); + throw new IOException(msg); } return response; } + + private boolean checkRetryWithSleep() throws InterruptedException { + int retries = maxWriteRetries; + while (retries > 0) { + TimelineHealth timelineHealth = writer.getHealthStatus(); + if (timelineHealth.getHealthStatus().equals( + TimelineHealth.TimelineHealthStatus.RUNNING)) { + return true; + } else { + try { + Thread.sleep(writeRetryInterval); + } catch (InterruptedException ex) { + Thread.currentThread().interrupt(); + throw ex; + } + retries--; + } + } + return false; + } + + /** * Add or update an domain. If the domain already exists, only the owner * and the admin can update it. @@ -179,11 +226,25 @@ public TimelineWriteResponse putDomain(TimelineDomain domain, UserGroupInformation callerUgi) throws IOException { LOG.debug("putDomain(domain={}, callerUgi={})", domain, callerUgi); - TimelineWriteResponse response; - synchronized (writer) { - final TimelineCollectorContext context = getTimelineEntityContext(); - response = writer.write(context, domain); - flushBufferedTimelineEntities(); + TimelineWriteResponse response = null; + try { + boolean isStorageUp = checkRetryWithSleep(); + if (isStorageUp) { + synchronized (writer) { + final TimelineCollectorContext context = getTimelineEntityContext(); + response = writer.write(context, domain); + flushBufferedTimelineEntities(); + } + } else { + String msg = String.format("Failed to putDomain(" + + "domain=%s, callerUgi=%s) as Timeline Storage is Down", + domain, callerUgi); + throw new IOException(msg); + } + } catch (InterruptedException ex) { + String msg = String.format("Interrupted while retrying to putDomain(" + + "domain=%s, callerUgi=%s)", domain, callerUgi); + throw new IOException(msg); } return response; diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-timelineservice/src/main/java/org/apache/hadoop/yarn/server/timelineservice/storage/FileSystemTimelineReaderImpl.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-timelineservice/src/main/java/org/apache/hadoop/yarn/server/timelineservice/storage/FileSystemTimelineReaderImpl.java index 3dd7396cfcaf5..197fa5a4072a8 100644 --- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-timelineservice/src/main/java/org/apache/hadoop/yarn/server/timelineservice/storage/FileSystemTimelineReaderImpl.java +++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-timelineservice/src/main/java/org/apache/hadoop/yarn/server/timelineservice/storage/FileSystemTimelineReaderImpl.java @@ -454,7 +454,7 @@ public TimelineHealth getHealthStatus() { fs.exists(rootPath); } catch (IOException e) { return new TimelineHealth( - TimelineHealth.TimelineHealthStatus.READER_CONNECTION_FAILURE, + TimelineHealth.TimelineHealthStatus.CONNECTION_FAILURE, e.getMessage() ); } diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-timelineservice/src/main/java/org/apache/hadoop/yarn/server/timelineservice/storage/FileSystemTimelineWriterImpl.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-timelineservice/src/main/java/org/apache/hadoop/yarn/server/timelineservice/storage/FileSystemTimelineWriterImpl.java index d5c70a0607ebb..d407636cb6af6 100644 --- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-timelineservice/src/main/java/org/apache/hadoop/yarn/server/timelineservice/storage/FileSystemTimelineWriterImpl.java +++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-timelineservice/src/main/java/org/apache/hadoop/yarn/server/timelineservice/storage/FileSystemTimelineWriterImpl.java @@ -31,6 +31,7 @@ import org.apache.hadoop.io.IOUtils; import org.apache.hadoop.security.UserGroupInformation; import org.apache.hadoop.service.AbstractService; +import org.apache.hadoop.yarn.api.records.timeline.TimelineHealth; import org.apache.hadoop.yarn.api.records.timelineservice.TimelineDomain; import org.apache.hadoop.yarn.api.records.timelineservice.TimelineEntities; import org.apache.hadoop.yarn.api.records.timelineservice.TimelineEntity; @@ -194,6 +195,20 @@ public void flush() throws IOException { // no op } + @Override + public TimelineHealth getHealthStatus() { + try { + fs.exists(rootPath); + } catch (IOException e) { + return new TimelineHealth( + TimelineHealth.TimelineHealthStatus.CONNECTION_FAILURE, + e.getMessage() + ); + } + return new TimelineHealth(TimelineHealth.TimelineHealthStatus.RUNNING, + ""); + } + private void mkdirs(Path... paths) throws IOException, InterruptedException { for (Path path: paths) { if (!existsWithRetries(path)) { diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-timelineservice/src/main/java/org/apache/hadoop/yarn/server/timelineservice/storage/NoOpTimelineWriterImpl.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-timelineservice/src/main/java/org/apache/hadoop/yarn/server/timelineservice/storage/NoOpTimelineWriterImpl.java index 48b334800f457..fd3120969391b 100644 --- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-timelineservice/src/main/java/org/apache/hadoop/yarn/server/timelineservice/storage/NoOpTimelineWriterImpl.java +++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-timelineservice/src/main/java/org/apache/hadoop/yarn/server/timelineservice/storage/NoOpTimelineWriterImpl.java @@ -19,6 +19,7 @@ import org.apache.hadoop.security.UserGroupInformation; import org.apache.hadoop.service.AbstractService; +import org.apache.hadoop.yarn.api.records.timeline.TimelineHealth; import org.apache.hadoop.yarn.api.records.timelineservice.TimelineDomain; import org.apache.hadoop.yarn.api.records.timelineservice.TimelineEntities; import org.apache.hadoop.yarn.api.records.timelineservice.TimelineEntity; @@ -77,4 +78,10 @@ public TimelineWriteResponse aggregate(TimelineEntity data, public void flush() throws IOException { LOG.debug("NoOpTimelineWriter is configured. Ignoring flush call"); } + + @Override + public TimelineHealth getHealthStatus() { + return new TimelineHealth(TimelineHealth.TimelineHealthStatus.RUNNING, + "NoOpTimelineWriter is configured. "); + } } diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-timelineservice/src/main/java/org/apache/hadoop/yarn/server/timelineservice/storage/TimelineWriter.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-timelineservice/src/main/java/org/apache/hadoop/yarn/server/timelineservice/storage/TimelineWriter.java index 08cfc8becb2d2..ccc7491037710 100644 --- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-timelineservice/src/main/java/org/apache/hadoop/yarn/server/timelineservice/storage/TimelineWriter.java +++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-timelineservice/src/main/java/org/apache/hadoop/yarn/server/timelineservice/storage/TimelineWriter.java @@ -23,6 +23,7 @@ import org.apache.hadoop.classification.InterfaceStability; import org.apache.hadoop.security.UserGroupInformation; import org.apache.hadoop.service.Service; +import org.apache.hadoop.yarn.api.records.timeline.TimelineHealth; import org.apache.hadoop.yarn.api.records.timelineservice.TimelineDomain; import org.apache.hadoop.yarn.api.records.timelineservice.TimelineEntities; import org.apache.hadoop.yarn.api.records.timelineservice.TimelineEntity; @@ -95,4 +96,13 @@ TimelineWriteResponse aggregate(TimelineEntity data, * entities to the backend storage. */ void flush() throws IOException; + + /** + * Check if writer connection is working properly. + * + * @return True if writer connection works as expected, false otherwise. + */ + TimelineHealth getHealthStatus(); + + } \ No newline at end of file diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-timelineservice/src/test/java/org/apache/hadoop/yarn/server/timelineservice/collector/TestTimelineCollector.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-timelineservice/src/test/java/org/apache/hadoop/yarn/server/timelineservice/collector/TestTimelineCollector.java index 96636f8bf8c46..09a1e6cb3d3f2 100644 --- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-timelineservice/src/test/java/org/apache/hadoop/yarn/server/timelineservice/collector/TestTimelineCollector.java +++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-timelineservice/src/test/java/org/apache/hadoop/yarn/server/timelineservice/collector/TestTimelineCollector.java @@ -21,6 +21,7 @@ import org.apache.hadoop.conf.Configuration; import org.apache.hadoop.security.UserGroupInformation; import org.apache.hadoop.util.Sets; +import org.apache.hadoop.yarn.api.records.timeline.TimelineHealth; import org.apache.hadoop.yarn.api.records.timelineservice.TimelineDomain; import org.apache.hadoop.yarn.api.records.timelineservice.TimelineMetricOperation; import org.apache.hadoop.yarn.api.records.ApplicationId; @@ -155,7 +156,17 @@ public void testAggregation() throws Exception { @Test public void testPutEntity() throws IOException { TimelineWriter writer = mock(TimelineWriter.class); + TimelineHealth timelineHealth = new TimelineHealth(TimelineHealth. + TimelineHealthStatus.RUNNING, ""); + when(writer.getHealthStatus()).thenReturn(timelineHealth); + + Configuration conf = new Configuration(); + conf.setInt(YarnConfiguration.TIMELINE_SERVICE_CLIENT_MAX_RETRIES, 5); + conf.setLong(YarnConfiguration.TIMELINE_SERVICE_CLIENT_RETRY_INTERVAL_MS, + 500L); + TimelineCollector collector = new TimelineCollectorForTest(writer); + collector.init(conf); TimelineEntities entities = generateTestEntities(1, 1); collector.putEntities( @@ -166,6 +177,36 @@ public void testPutEntity() throws IOException { verify(writer, times(1)).flush(); } + + @Test + public void testPutEntityWithStorageDown() throws IOException { + TimelineWriter writer = mock(TimelineWriter.class); + TimelineHealth timelineHealth = new TimelineHealth(TimelineHealth. + TimelineHealthStatus.CONNECTION_FAILURE, ""); + when(writer.getHealthStatus()).thenReturn(timelineHealth); + + Configuration conf = new Configuration(); + conf.setInt(YarnConfiguration.TIMELINE_SERVICE_CLIENT_MAX_RETRIES, 5); + conf.setLong(YarnConfiguration.TIMELINE_SERVICE_CLIENT_RETRY_INTERVAL_MS, + 500L); + + TimelineCollector collector = new TimelineCollectorForTest(writer); + collector.init(conf); + + TimelineEntities entities = generateTestEntities(1, 1); + boolean exceptionCaught = false; + try { + collector.putEntities(entities, UserGroupInformation. + createRemoteUser("test-user")); + } catch (Exception e) { + if (e.getMessage().contains("Failed to putEntities")) { + exceptionCaught = true; + } + } + assertTrue("TimelineCollector putEntity failed to " + + "handle storage down", exceptionCaught); + } + /** * Test TimelineCollector's interaction with TimelineWriter upon * putEntityAsync() calls. @@ -222,7 +263,17 @@ public void testAsyncEntityDiscard() throws Exception { */ @Test public void testPutDomain() throws IOException { TimelineWriter writer = mock(TimelineWriter.class); + TimelineHealth timelineHealth = new TimelineHealth(TimelineHealth. + TimelineHealthStatus.RUNNING, ""); + when(writer.getHealthStatus()).thenReturn(timelineHealth); + + Configuration conf = new Configuration(); + conf.setInt(YarnConfiguration.TIMELINE_SERVICE_CLIENT_MAX_RETRIES, 5); + conf.setLong(YarnConfiguration.TIMELINE_SERVICE_CLIENT_RETRY_INTERVAL_MS, + 500L); + TimelineCollector collector = new TimelineCollectorForTest(writer); + collector.init(conf); TimelineDomain domain = generateDomain("id", "desc", "owner", "reader1,reader2", "writer", 0L, @@ -287,8 +338,19 @@ public TimelineCollectorContext getTimelineEntityContext() { 1L, ApplicationId.newInstance(ts, 1).toString()); } }; - collector.init(new Configuration()); - collector.setWriter(mock(TimelineWriter.class)); + + TimelineWriter writer = mock(TimelineWriter.class); + TimelineHealth timelineHealth = new TimelineHealth(TimelineHealth. + TimelineHealthStatus.RUNNING, ""); + when(writer.getHealthStatus()).thenReturn(timelineHealth); + + Configuration conf = new Configuration(); + conf.setInt(YarnConfiguration.TIMELINE_SERVICE_CLIENT_MAX_RETRIES, 5); + conf.setLong(YarnConfiguration.TIMELINE_SERVICE_CLIENT_RETRY_INTERVAL_MS, + 500L); + + collector.init(conf); + collector.setWriter(writer); // Put 5 entities with different metric values. TimelineEntities entities = new TimelineEntities(); diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-web-proxy/src/test/java/org/apache/hadoop/yarn/server/webproxy/TestAppReportFetcher.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-web-proxy/src/test/java/org/apache/hadoop/yarn/server/webproxy/TestAppReportFetcher.java index bcab33fd66d75..4bf0db6b7d7af 100644 --- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-web-proxy/src/test/java/org/apache/hadoop/yarn/server/webproxy/TestAppReportFetcher.java +++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-web-proxy/src/test/java/org/apache/hadoop/yarn/server/webproxy/TestAppReportFetcher.java @@ -26,7 +26,6 @@ import org.apache.hadoop.yarn.api.protocolrecords.GetApplicationReportRequest; import org.apache.hadoop.yarn.api.protocolrecords.GetApplicationReportResponse; import org.apache.hadoop.yarn.api.records.ApplicationId; -import org.apache.hadoop.yarn.api.records.ApplicationReport; import org.apache.hadoop.yarn.conf.YarnConfiguration; import org.apache.hadoop.yarn.exceptions.ApplicationNotFoundException; import org.apache.hadoop.yarn.exceptions.YarnException; diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-web-proxy/src/test/java/org/apache/hadoop/yarn/server/webproxy/amfilter/TestAmFilter.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-web-proxy/src/test/java/org/apache/hadoop/yarn/server/webproxy/amfilter/TestAmFilter.java index 5fcdb62ea33ab..857e24fdb5d09 100644 --- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-web-proxy/src/test/java/org/apache/hadoop/yarn/server/webproxy/amfilter/TestAmFilter.java +++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-web-proxy/src/test/java/org/apache/hadoop/yarn/server/webproxy/amfilter/TestAmFilter.java @@ -25,12 +25,9 @@ import java.util.Set; import java.util.HashSet; import java.util.Enumeration; -import java.util.Collection; import java.util.Collections; import java.util.Map; import java.util.HashMap; -import java.util.ArrayList; -import java.util.Arrays; import java.util.concurrent.atomic.AtomicBoolean; import javax.servlet.FilterConfig; diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-ui/pom.xml b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-ui/pom.xml index ca291ec85d1fd..e18a7eac3387d 100644 --- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-ui/pom.xml +++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-ui/pom.xml @@ -38,6 +38,34 @@ + + + org.apache.maven.plugins + maven-antrun-plugin + 1.8 + + + validate + + run + + + true + + + + + + + + Ember build script to use = ${emberBuildScript} + + + + + + org.apache.rat @@ -229,7 +257,7 @@ ${packageManagerScript} run - build:mvn + ${emberBuildScript} diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-ui/src/main/webapp/package.json b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-ui/src/main/webapp/package.json index 91c5a0f3e671b..deb79a722e693 100644 --- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-ui/src/main/webapp/package.json +++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-ui/src/main/webapp/package.json @@ -10,7 +10,8 @@ "scripts": { "build": "TMPDIR=tmp node ./node_modules/ember-cli/bin/ember build", "start": "TMPDIR=tmp node ./node_modules/ember-cli/bin/ember server", - "build:mvn": "TMPDIR=tmp node/node ./node_modules/ember-cli/bin/ember build -prod" + "build:mvn": "TMPDIR=tmp node/node ./node_modules/ember-cli/bin/ember build -prod", + "build:mvn:windows": "set TMPDIR=tmp; node/node ./node_modules/ember-cli/bin/ember build -prod" }, "repository": "", "engines": { diff --git a/pom.xml b/pom.xml index 1f6aa885666c4..ed869625de18f 100644 --- a/pom.xml +++ b/pom.xml @@ -117,6 +117,7 @@ xsi:schemaLocation="http://maven.apache.org/POM/4.0.0 https://maven.apache.org/x 4.2.2 4.2.0 1.1.1 + 3.10.1 bash @@ -478,6 +479,11 @@ xsi:schemaLocation="http://maven.apache.org/POM/4.0.0 https://maven.apache.org/x jsonschema2pojo-maven-plugin ${jsonschema2pojo-maven-plugin.version} + + org.apache.maven.plugins + maven-compiler-plugin + ${maven-compiler-plugin.version} + @@ -544,6 +550,7 @@ xsi:schemaLocation="http://maven.apache.org/POM/4.0.0 https://maven.apache.org/x licenses-binary/** dev-support/docker/pkg-resolver/packages.json dev-support/docker/pkg-resolver/platforms.json + **/target/** From c27c28dd3f51159c7a4e72e33fd7805c48c47f55 Mon Sep 17 00:00:00 2001 From: Pranav Saxena Date: Fri, 8 Jul 2022 10:50:03 +0530 Subject: [PATCH 31/77] backmerged pranavsaxena:trunk --- .../util/JaasConfiguration.java | 77 +++++++ .../util/ZKSignerSecretProvider.java | 61 ------ .../util/TestJaasConfiguration.java | 4 +- .../hadoop/fs/CommonConfigurationKeys.java | 4 + .../java/org/apache/hadoop/ha/NodeFencer.java | 1 + .../apache/hadoop/ha/PowerShellFencer.java | 4 + .../ZKDelegationTokenSecretManager.java | 67 +----- .../hadoop/util/curator/ZKCuratorManager.java | 39 +++- .../src/main/resources/core-default.xml | 2 +- .../apache/hadoop/fs/TestLocalFileSystem.java | 2 +- .../tfile/TestTFileSeqFileComparison.java | 2 +- .../apache/hadoop/ipc/RPCCallBenchmark.java | 2 +- .../security/TestWhitelistBasedResolver.java | 16 +- .../util/curator/TestZKCuratorManager.java | 51 +++++ .../client/impl/zk/RegistrySecurity.java | 63 +----- .../qjournal/client/IPCLoggerChannel.java | 4 +- .../server/namenode/SecondaryNameNode.java | 2 +- .../markdown/HDFSHighAvailabilityWithNFS.md | 26 ++- .../markdown/HDFSHighAvailabilityWithQJM.md | 24 +- .../server/namenode/TestAddBlockRetry.java | 2 +- .../TestCheckPointForSecurityTokens.java | 2 +- .../hdfs/server/namenode/TestCheckpoint.java | 2 +- .../namenode/TestFavoredNodesEndToEnd.java | 2 +- .../ha/TestDelegationTokensWithHA.java | 2 +- .../org/apache/hadoop/mapred/MapTask.java | 8 +- .../mapreduce/lib/aggregate/LongValueMax.java | 2 +- .../mapreduce/v2/hs/webapp/HsTasksBlock.java | 2 +- .../apache/hadoop/mapred/BigMapOutput.java | 2 +- .../hadoop/mapred/NotificationTestCase.java | 6 +- .../hadoop/mapreduce/TestValueIterReset.java | 2 +- .../hadoop/examples/pi/math/LongLong.java | 2 +- .../fs/azurebfs/services/AbfsClient.java | 1 - .../hadoop/yarn/client/cli/TestYarnCLI.java | 2 +- .../pb/ReservationAllocationStatePBImpl.java | 2 +- .../AggregatedLogDeletionService.java | 2 +- .../security/ConfiguredYarnAuthorizer.java | 2 +- .../WindowsSecureContainerExecutor.java | 2 +- .../ProportionalCapacityPreemptionPolicy.java | 2 +- .../resourcemanager/TestAppManager.java | 2 +- .../resourcemanager/TestRMAdminService.java | 2 +- .../src/site/markdown/UsingNuma.md | 206 ++++++++++++++++++ 41 files changed, 469 insertions(+), 239 deletions(-) create mode 100644 hadoop-common-project/hadoop-auth/src/main/java/org/apache/hadoop/security/authentication/util/JaasConfiguration.java create mode 100644 hadoop-yarn-project/hadoop-yarn/hadoop-yarn-site/src/site/markdown/UsingNuma.md diff --git a/hadoop-common-project/hadoop-auth/src/main/java/org/apache/hadoop/security/authentication/util/JaasConfiguration.java b/hadoop-common-project/hadoop-auth/src/main/java/org/apache/hadoop/security/authentication/util/JaasConfiguration.java new file mode 100644 index 0000000000000..d03e630cedf7e --- /dev/null +++ b/hadoop-common-project/hadoop-auth/src/main/java/org/apache/hadoop/security/authentication/util/JaasConfiguration.java @@ -0,0 +1,77 @@ +/** + * Licensed 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. See accompanying LICENSE file. + */ +package org.apache.hadoop.security.authentication.util; + +import java.util.HashMap; +import java.util.Map; +import javax.security.auth.login.AppConfigurationEntry; +import javax.security.auth.login.Configuration; + + +/** + * Creates a programmatic version of a jaas.conf file. This can be used + * instead of writing a jaas.conf file and setting the system property, + * "java.security.auth.login.config", to point to that file. It is meant to be + * used for connecting to ZooKeeper. + */ +public class JaasConfiguration extends Configuration { + + private final javax.security.auth.login.Configuration baseConfig = + javax.security.auth.login.Configuration.getConfiguration(); + private final AppConfigurationEntry[] entry; + private final String entryName; + + /** + * Add an entry to the jaas configuration with the passed in name, + * principal, and keytab. The other necessary options will be set for you. + * + * @param entryName The name of the entry (e.g. "Client") + * @param principal The principal of the user + * @param keytab The location of the keytab + */ + public JaasConfiguration(String entryName, String principal, String keytab) { + this.entryName = entryName; + Map options = new HashMap<>(); + options.put("keyTab", keytab); + options.put("principal", principal); + options.put("useKeyTab", "true"); + options.put("storeKey", "true"); + options.put("useTicketCache", "false"); + options.put("refreshKrb5Config", "true"); + String jaasEnvVar = System.getenv("HADOOP_JAAS_DEBUG"); + if ("true".equalsIgnoreCase(jaasEnvVar)) { + options.put("debug", "true"); + } + entry = new AppConfigurationEntry[]{ + new AppConfigurationEntry(getKrb5LoginModuleName(), + AppConfigurationEntry.LoginModuleControlFlag.REQUIRED, + options)}; + } + + @Override + public AppConfigurationEntry[] getAppConfigurationEntry(String name) { + return (entryName.equals(name)) ? entry : ((baseConfig != null) + ? baseConfig.getAppConfigurationEntry(name) : null); + } + + private String getKrb5LoginModuleName() { + String krb5LoginModuleName; + if (System.getProperty("java.vendor").contains("IBM")) { + krb5LoginModuleName = "com.ibm.security.auth.module.Krb5LoginModule"; + } else { + krb5LoginModuleName = "com.sun.security.auth.module.Krb5LoginModule"; + } + return krb5LoginModuleName; + } +} diff --git a/hadoop-common-project/hadoop-auth/src/main/java/org/apache/hadoop/security/authentication/util/ZKSignerSecretProvider.java b/hadoop-common-project/hadoop-auth/src/main/java/org/apache/hadoop/security/authentication/util/ZKSignerSecretProvider.java index 374f4a5665796..53b6f4d239f98 100644 --- a/hadoop-common-project/hadoop-auth/src/main/java/org/apache/hadoop/security/authentication/util/ZKSignerSecretProvider.java +++ b/hadoop-common-project/hadoop-auth/src/main/java/org/apache/hadoop/security/authentication/util/ZKSignerSecretProvider.java @@ -17,12 +17,9 @@ import java.nio.ByteBuffer; import java.security.SecureRandom; import java.util.Collections; -import java.util.HashMap; import java.util.List; -import java.util.Map; import java.util.Properties; import java.util.Random; -import javax.security.auth.login.AppConfigurationEntry; import javax.security.auth.login.Configuration; import javax.servlet.ServletContext; import org.apache.curator.RetryPolicy; @@ -429,62 +426,4 @@ public List getAclForPath(String path) { return saslACL; } } - - /** - * Creates a programmatic version of a jaas.conf file. This can be used - * instead of writing a jaas.conf file and setting the system property, - * "java.security.auth.login.config", to point to that file. It is meant to be - * used for connecting to ZooKeeper. - */ - @InterfaceAudience.Private - public static class JaasConfiguration extends Configuration { - - private final javax.security.auth.login.Configuration baseConfig = - javax.security.auth.login.Configuration.getConfiguration(); - private static AppConfigurationEntry[] entry; - private String entryName; - - /** - * Add an entry to the jaas configuration with the passed in name, - * principal, and keytab. The other necessary options will be set for you. - * - * @param entryName The name of the entry (e.g. "Client") - * @param principal The principal of the user - * @param keytab The location of the keytab - */ - public JaasConfiguration(String entryName, String principal, String keytab) { - this.entryName = entryName; - Map options = new HashMap(); - options.put("keyTab", keytab); - options.put("principal", principal); - options.put("useKeyTab", "true"); - options.put("storeKey", "true"); - options.put("useTicketCache", "false"); - options.put("refreshKrb5Config", "true"); - String jaasEnvVar = System.getenv("HADOOP_JAAS_DEBUG"); - if (jaasEnvVar != null && "true".equalsIgnoreCase(jaasEnvVar)) { - options.put("debug", "true"); - } - entry = new AppConfigurationEntry[]{ - new AppConfigurationEntry(getKrb5LoginModuleName(), - AppConfigurationEntry.LoginModuleControlFlag.REQUIRED, - options)}; - } - - @Override - public AppConfigurationEntry[] getAppConfigurationEntry(String name) { - return (entryName.equals(name)) ? entry : ((baseConfig != null) - ? baseConfig.getAppConfigurationEntry(name) : null); - } - - private String getKrb5LoginModuleName() { - String krb5LoginModuleName; - if (System.getProperty("java.vendor").contains("IBM")) { - krb5LoginModuleName = "com.ibm.security.auth.module.Krb5LoginModule"; - } else { - krb5LoginModuleName = "com.sun.security.auth.module.Krb5LoginModule"; - } - return krb5LoginModuleName; - } - } } diff --git a/hadoop-common-project/hadoop-auth/src/test/java/org/apache/hadoop/security/authentication/util/TestJaasConfiguration.java b/hadoop-common-project/hadoop-auth/src/test/java/org/apache/hadoop/security/authentication/util/TestJaasConfiguration.java index 2b70135800be0..5de4122471f50 100644 --- a/hadoop-common-project/hadoop-auth/src/test/java/org/apache/hadoop/security/authentication/util/TestJaasConfiguration.java +++ b/hadoop-common-project/hadoop-auth/src/test/java/org/apache/hadoop/security/authentication/util/TestJaasConfiguration.java @@ -32,8 +32,8 @@ public void test() throws Exception { krb5LoginModuleName = "com.sun.security.auth.module.Krb5LoginModule"; } - ZKSignerSecretProvider.JaasConfiguration jConf = - new ZKSignerSecretProvider.JaasConfiguration("foo", "foo/localhost", + JaasConfiguration jConf = + new JaasConfiguration("foo", "foo/localhost", "/some/location/foo.keytab"); AppConfigurationEntry[] entries = jConf.getAppConfigurationEntry("bar"); Assert.assertNull(entries); diff --git a/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/fs/CommonConfigurationKeys.java b/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/fs/CommonConfigurationKeys.java index db82498b3c15e..16144206eeea4 100644 --- a/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/fs/CommonConfigurationKeys.java +++ b/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/fs/CommonConfigurationKeys.java @@ -401,6 +401,10 @@ public class CommonConfigurationKeys extends CommonConfigurationKeysPublic { public static final String ZK_AUTH = ZK_PREFIX + "auth"; /** Principal name for zookeeper servers. */ public static final String ZK_SERVER_PRINCIPAL = ZK_PREFIX + "server.principal"; + /** Kerberos principal name for zookeeper connection. */ + public static final String ZK_KERBEROS_PRINCIPAL = ZK_PREFIX + "kerberos.principal"; + /** Kerberos keytab for zookeeper connection. */ + public static final String ZK_KERBEROS_KEYTAB = ZK_PREFIX + "kerberos.keytab"; /** Address of the ZooKeeper ensemble. */ public static final String ZK_ADDRESS = ZK_PREFIX + "address"; diff --git a/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/ha/NodeFencer.java b/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/ha/NodeFencer.java index fb78a4c47dcde..f42db1d0c5561 100644 --- a/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/ha/NodeFencer.java +++ b/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/ha/NodeFencer.java @@ -48,6 +48,7 @@ *
      *
    • shell(/path/to/some/script.sh args...)
    • *
    • sshfence(...) (see {@link SshFenceByTcpPort}) + *
    • powershell(...) (see {@link PowerShellFencer}) *
    */ @InterfaceAudience.Private diff --git a/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/ha/PowerShellFencer.java b/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/ha/PowerShellFencer.java index 6de618c8fcc52..47e526f65e658 100644 --- a/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/ha/PowerShellFencer.java +++ b/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/ha/PowerShellFencer.java @@ -34,6 +34,10 @@ * Fencer method that uses PowerShell to remotely connect to a machine and kill * the required process. This only works in Windows. * + * Fencing method uses PowerShell command. It should be specified + * in the fencing configuration like:
    + * powershell(NameNode) + * * The argument passed to this fencer should be a unique string in the * "CommandLine" attribute for the "java.exe" process. For example, the full * path for the Namenode: "org.apache.hadoop.hdfs.server.namenode.NameNode". diff --git a/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/security/token/delegation/ZKDelegationTokenSecretManager.java b/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/security/token/delegation/ZKDelegationTokenSecretManager.java index 452565676a028..d0630e38b4ddb 100644 --- a/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/security/token/delegation/ZKDelegationTokenSecretManager.java +++ b/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/security/token/delegation/ZKDelegationTokenSecretManager.java @@ -25,14 +25,10 @@ import java.io.IOException; import java.io.UncheckedIOException; import java.util.Collections; -import java.util.HashMap; import java.util.List; -import java.util.Map; import java.util.concurrent.atomic.AtomicInteger; import java.util.stream.Stream; -import javax.security.auth.login.AppConfigurationEntry; - import org.apache.curator.ensemble.fixed.FixedEnsembleProvider; import org.apache.curator.framework.CuratorFramework; import org.apache.curator.framework.CuratorFrameworkFactory; @@ -52,6 +48,7 @@ import org.apache.hadoop.classification.InterfaceStability.Unstable; import org.apache.hadoop.conf.Configuration; import org.apache.hadoop.security.SecurityUtil; +import org.apache.hadoop.security.authentication.util.JaasConfiguration; import org.apache.hadoop.security.token.Token; import org.apache.hadoop.security.token.delegation.web.DelegationTokenManager; import static org.apache.hadoop.util.Time.now; @@ -251,68 +248,6 @@ private String setJaasConfiguration(Configuration config) throws Exception { return principal.split("[/@]")[0]; } - /** - * Creates a programmatic version of a jaas.conf file. This can be used - * instead of writing a jaas.conf file and setting the system property, - * "java.security.auth.login.config", to point to that file. It is meant to be - * used for connecting to ZooKeeper. - */ - @InterfaceAudience.Private - public static class JaasConfiguration extends - javax.security.auth.login.Configuration { - - private final javax.security.auth.login.Configuration baseConfig = - javax.security.auth.login.Configuration.getConfiguration(); - private static AppConfigurationEntry[] entry; - private String entryName; - - /** - * Add an entry to the jaas configuration with the passed in name, - * principal, and keytab. The other necessary options will be set for you. - * - * @param entryName - * The name of the entry (e.g. "Client") - * @param principal - * The principal of the user - * @param keytab - * The location of the keytab - */ - public JaasConfiguration(String entryName, String principal, String keytab) { - this.entryName = entryName; - Map options = new HashMap(); - options.put("keyTab", keytab); - options.put("principal", principal); - options.put("useKeyTab", "true"); - options.put("storeKey", "true"); - options.put("useTicketCache", "false"); - options.put("refreshKrb5Config", "true"); - String jaasEnvVar = System.getenv("HADOOP_JAAS_DEBUG"); - if (jaasEnvVar != null && "true".equalsIgnoreCase(jaasEnvVar)) { - options.put("debug", "true"); - } - entry = new AppConfigurationEntry[] { - new AppConfigurationEntry(getKrb5LoginModuleName(), - AppConfigurationEntry.LoginModuleControlFlag.REQUIRED, - options) }; - } - - @Override - public AppConfigurationEntry[] getAppConfigurationEntry(String name) { - return (entryName.equals(name)) ? entry : ((baseConfig != null) - ? baseConfig.getAppConfigurationEntry(name) : null); - } - - private String getKrb5LoginModuleName() { - String krb5LoginModuleName; - if (System.getProperty("java.vendor").contains("IBM")) { - krb5LoginModuleName = "com.ibm.security.auth.module.Krb5LoginModule"; - } else { - krb5LoginModuleName = "com.sun.security.auth.module.Krb5LoginModule"; - } - return krb5LoginModuleName; - } - } - @Override public void startThreads() throws IOException { if (!isExternalClient) { diff --git a/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/util/curator/ZKCuratorManager.java b/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/util/curator/ZKCuratorManager.java index 7d35977e5b00d..c11b86838673a 100644 --- a/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/util/curator/ZKCuratorManager.java +++ b/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/util/curator/ZKCuratorManager.java @@ -33,6 +33,7 @@ import org.apache.hadoop.conf.Configuration; import org.apache.hadoop.fs.CommonConfigurationKeys; import org.apache.hadoop.security.SecurityUtil; +import org.apache.hadoop.security.authentication.util.JaasConfiguration; import org.apache.hadoop.util.ZKUtil; import org.apache.zookeeper.CreateMode; import org.apache.zookeeper.Watcher; @@ -159,7 +160,9 @@ public void start(List authInfos) throws IOException { CuratorFramework client = CuratorFrameworkFactory.builder() .connectString(zkHostPort) .zookeeperFactory(new HadoopZookeeperFactory( - conf.get(CommonConfigurationKeys.ZK_SERVER_PRINCIPAL))) + conf.get(CommonConfigurationKeys.ZK_SERVER_PRINCIPAL), + conf.get(CommonConfigurationKeys.ZK_KERBEROS_PRINCIPAL), + conf.get(CommonConfigurationKeys.ZK_KERBEROS_KEYTAB))) .sessionTimeoutMs(zkSessionTimeout) .retryPolicy(retryPolicy) .authorization(authInfos) @@ -445,10 +448,20 @@ public void setData(String path, byte[] data, int version) } public static class HadoopZookeeperFactory implements ZookeeperFactory { + public final static String JAAS_CLIENT_ENTRY = "HadoopZookeeperClient"; private final String zkPrincipal; + private final String kerberosPrincipal; + private final String kerberosKeytab; public HadoopZookeeperFactory(String zkPrincipal) { + this(zkPrincipal, null, null); + } + + public HadoopZookeeperFactory(String zkPrincipal, String kerberosPrincipal, + String kerberosKeytab) { this.zkPrincipal = zkPrincipal; + this.kerberosPrincipal = kerberosPrincipal; + this.kerberosKeytab = kerberosKeytab; } @Override @@ -462,8 +475,32 @@ public ZooKeeper newZooKeeper(String connectString, int sessionTimeout, zkClientConfig.setProperty(ZKClientConfig.ZK_SASL_CLIENT_USERNAME, zkPrincipal); } + if (zkClientConfig.isSaslClientEnabled() && !isJaasConfigurationSet(zkClientConfig)) { + setJaasConfiguration(zkClientConfig); + } return new ZooKeeper(connectString, sessionTimeout, watcher, canBeReadOnly, zkClientConfig); } + + private boolean isJaasConfigurationSet(ZKClientConfig zkClientConfig) { + String clientConfig = zkClientConfig.getProperty(ZKClientConfig.LOGIN_CONTEXT_NAME_KEY, + ZKClientConfig.LOGIN_CONTEXT_NAME_KEY_DEFAULT); + return javax.security.auth.login.Configuration.getConfiguration() + .getAppConfigurationEntry(clientConfig) != null; + } + + private void setJaasConfiguration(ZKClientConfig zkClientConfig) throws IOException { + if (kerberosPrincipal == null || kerberosKeytab == null) { + LOG.warn("JaasConfiguration has not been set since kerberos principal " + + "or keytab is not specified"); + return; + } + + String principal = SecurityUtil.getServerPrincipal(kerberosPrincipal, ""); + JaasConfiguration jconf = new JaasConfiguration(JAAS_CLIENT_ENTRY, principal, + kerberosKeytab); + javax.security.auth.login.Configuration.setConfiguration(jconf); + zkClientConfig.setProperty(ZKClientConfig.LOGIN_CONTEXT_NAME_KEY, JAAS_CLIENT_ENTRY); + } } } \ No newline at end of file diff --git a/hadoop-common-project/hadoop-common/src/main/resources/core-default.xml b/hadoop-common-project/hadoop-common/src/main/resources/core-default.xml index 85138611c3b4e..5a1c09f0141ed 100644 --- a/hadoop-common-project/hadoop-common/src/main/resources/core-default.xml +++ b/hadoop-common-project/hadoop-common/src/main/resources/core-default.xml @@ -2930,7 +2930,7 @@ The switch to turn S3A auditing on or off. List of fencing methods to use for service fencing. May contain - builtin methods (eg shell and sshfence) or user-defined method. + builtin methods (eg shell, sshfence and powershell) or user-defined method. diff --git a/hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/fs/TestLocalFileSystem.java b/hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/fs/TestLocalFileSystem.java index 949ea42331527..29ef6ca6c7afd 100644 --- a/hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/fs/TestLocalFileSystem.java +++ b/hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/fs/TestLocalFileSystem.java @@ -164,7 +164,7 @@ public void testWorkingDirectory() throws IOException { public void testSyncable() throws IOException { FileSystem fs = fileSys.getRawFileSystem(); Path file = new Path(TEST_ROOT_DIR, "syncable"); - FSDataOutputStream out = fs.create(file);; + FSDataOutputStream out = fs.create(file); final int bytesWritten = 1; byte[] expectedBuf = new byte[] {'0', '1', '2', '3'}; try { diff --git a/hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/io/file/tfile/TestTFileSeqFileComparison.java b/hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/io/file/tfile/TestTFileSeqFileComparison.java index 48924203f34e7..fdd622a7f6a37 100644 --- a/hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/io/file/tfile/TestTFileSeqFileComparison.java +++ b/hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/io/file/tfile/TestTFileSeqFileComparison.java @@ -516,7 +516,7 @@ private static String parameters2String(MyOptions options) { } private static class MyOptions { - String rootDir = GenericTestUtils.getTestDir().getAbsolutePath();; + String rootDir = GenericTestUtils.getTestDir().getAbsolutePath(); String compress = "gz"; String format = "tfile"; int dictSize = 1000; diff --git a/hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/ipc/RPCCallBenchmark.java b/hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/ipc/RPCCallBenchmark.java index 6742425e766e8..cca40f97c570b 100644 --- a/hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/ipc/RPCCallBenchmark.java +++ b/hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/ipc/RPCCallBenchmark.java @@ -289,7 +289,7 @@ public int run(String[] args) throws Exception { long cpuNanosClient = getTotalCpuTime(ctx.getTestThreads()); long cpuNanosServer = -1; if (server != null) { - cpuNanosServer = getTotalCpuTime(server.getHandlers());; + cpuNanosServer = getTotalCpuTime(server.getHandlers()); } System.out.println("====== Results ======"); System.out.println("Options:\n" + opts); diff --git a/hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/security/TestWhitelistBasedResolver.java b/hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/security/TestWhitelistBasedResolver.java index 03fc4cbb54c16..81abc42a023f3 100644 --- a/hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/security/TestWhitelistBasedResolver.java +++ b/hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/security/TestWhitelistBasedResolver.java @@ -111,16 +111,16 @@ public void testFixedAndLocalWhiteList() throws IOException { assertEquals (wqr.getDefaultProperties(), wqr.getServerProperties(InetAddress.getByName("10.119.103.112"))); - assertEquals (SASL_PRIVACY_PROPS, wqr.getServerProperties("10.119.103.113")); + assertEquals(SASL_PRIVACY_PROPS, wqr.getServerProperties("10.119.103.113")); - assertEquals (wqr.getDefaultProperties(), wqr.getServerProperties("10.221.103.121")); + assertEquals(wqr.getDefaultProperties(), wqr.getServerProperties("10.221.103.121")); - assertEquals (SASL_PRIVACY_PROPS, wqr.getServerProperties("10.221.104.0")); - assertEquals (SASL_PRIVACY_PROPS, wqr.getServerProperties("10.222.103.121")); - assertEquals (SASL_PRIVACY_PROPS, wqr.getServerProperties("10.223.104.0")); - assertEquals (SASL_PRIVACY_PROPS, wqr.getServerProperties("10.113.221.221")); - assertEquals (SASL_PRIVACY_PROPS, wqr.getServerProperties("10.113.221.222")); - assertEquals (wqr.getDefaultProperties(), wqr.getServerProperties("127.0.0.1"));; + assertEquals(SASL_PRIVACY_PROPS, wqr.getServerProperties("10.221.104.0")); + assertEquals(SASL_PRIVACY_PROPS, wqr.getServerProperties("10.222.103.121")); + assertEquals(SASL_PRIVACY_PROPS, wqr.getServerProperties("10.223.104.0")); + assertEquals(SASL_PRIVACY_PROPS, wqr.getServerProperties("10.113.221.221")); + assertEquals(SASL_PRIVACY_PROPS, wqr.getServerProperties("10.113.221.222")); + assertEquals(wqr.getDefaultProperties(), wqr.getServerProperties("127.0.0.1")); TestFileBasedIPList.removeFile("fixedwhitelist.txt"); TestFileBasedIPList.removeFile("variablewhitelist.txt"); diff --git a/hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/util/curator/TestZKCuratorManager.java b/hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/util/curator/TestZKCuratorManager.java index a2156ee6d93af..fd15a0c2b1bf4 100644 --- a/hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/util/curator/TestZKCuratorManager.java +++ b/hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/util/curator/TestZKCuratorManager.java @@ -25,11 +25,15 @@ import java.util.Arrays; import java.util.List; +import javax.security.auth.login.AppConfigurationEntry; import org.apache.curator.test.TestingServer; import org.apache.hadoop.conf.Configuration; import org.apache.hadoop.fs.CommonConfigurationKeys; +import org.apache.hadoop.security.authentication.util.JaasConfiguration; import org.apache.hadoop.util.ZKUtil; import org.apache.zookeeper.CreateMode; +import org.apache.zookeeper.ZooKeeper; +import org.apache.zookeeper.client.ZKClientConfig; import org.apache.zookeeper.data.ACL; import org.apache.zookeeper.data.Stat; import org.junit.After; @@ -154,4 +158,51 @@ public void testTransaction() throws Exception { assertFalse(curator.exists(node2)); assertTrue(Arrays.equals(setData, curator.getData(node1))); } + + @Test + public void testJaasConfiguration() throws Exception { + // Validate that HadoopZooKeeperFactory will set ZKConfig with given principals + ZKCuratorManager.HadoopZookeeperFactory factory1 = + new ZKCuratorManager.HadoopZookeeperFactory("foo1", "bar1", "bar1.keytab"); + ZooKeeper zk1 = factory1.newZooKeeper("connString", 1000, null, false); + validateJaasConfiguration(ZKCuratorManager.HadoopZookeeperFactory.JAAS_CLIENT_ENTRY, + "bar1", "bar1.keytab", zk1); + + // Validate that a new HadoopZooKeeperFactory will use the new principals + ZKCuratorManager.HadoopZookeeperFactory factory2 = + new ZKCuratorManager.HadoopZookeeperFactory("foo2", "bar2", "bar2.keytab"); + ZooKeeper zk2 = factory2.newZooKeeper("connString", 1000, null, false); + validateJaasConfiguration(ZKCuratorManager.HadoopZookeeperFactory.JAAS_CLIENT_ENTRY, + "bar2", "bar2.keytab", zk2); + + try { + // Setting global configuration + String testClientConfig = "TestClientConfig"; + JaasConfiguration jconf = new JaasConfiguration(testClientConfig, "test", "test.keytab"); + javax.security.auth.login.Configuration.setConfiguration(jconf); + System.setProperty(ZKClientConfig.LOGIN_CONTEXT_NAME_KEY, testClientConfig); + + // Validate that a new HadoopZooKeeperFactory will use the global principals + ZKCuratorManager.HadoopZookeeperFactory factory3 = + new ZKCuratorManager.HadoopZookeeperFactory("foo3", "bar3", "bar3.keytab"); + ZooKeeper zk3 = factory3.newZooKeeper("connString", 1000, null, false); + validateJaasConfiguration(testClientConfig, "test", "test.keytab", zk3); + } finally { + // Remove global configuration + System.clearProperty(ZKClientConfig.LOGIN_CONTEXT_NAME_KEY); + } + } + + private void validateJaasConfiguration(String clientConfig, String principal, String keytab, + ZooKeeper zk) { + assertEquals("Validate that expected clientConfig is set in ZK config", clientConfig, + zk.getClientConfig().getProperty(ZKClientConfig.LOGIN_CONTEXT_NAME_KEY)); + + AppConfigurationEntry[] entries = javax.security.auth.login.Configuration.getConfiguration() + .getAppConfigurationEntry(clientConfig); + assertEquals("Validate that expected principal is set in Jaas config", principal, + entries[0].getOptions().get("principal")); + assertEquals("Validate that expected keytab is set in Jaas config", keytab, + entries[0].getOptions().get("keyTab")); + } } \ No newline at end of file diff --git a/hadoop-common-project/hadoop-registry/src/main/java/org/apache/hadoop/registry/client/impl/zk/RegistrySecurity.java b/hadoop-common-project/hadoop-registry/src/main/java/org/apache/hadoop/registry/client/impl/zk/RegistrySecurity.java index e500ba6617b5f..d48920a222333 100644 --- a/hadoop-common-project/hadoop-registry/src/main/java/org/apache/hadoop/registry/client/impl/zk/RegistrySecurity.java +++ b/hadoop-common-project/hadoop-registry/src/main/java/org/apache/hadoop/registry/client/impl/zk/RegistrySecurity.java @@ -18,11 +18,11 @@ package org.apache.hadoop.registry.client.impl.zk; +import org.apache.hadoop.security.authentication.util.JaasConfiguration; import org.apache.hadoop.util.Preconditions; import org.apache.hadoop.thirdparty.com.google.common.base.Splitter; import org.apache.commons.lang3.StringUtils; import org.apache.curator.framework.CuratorFrameworkFactory; -import org.apache.hadoop.classification.InterfaceAudience; import org.apache.hadoop.conf.Configuration; import org.apache.hadoop.security.UserGroupInformation; import org.apache.hadoop.security.authentication.util.KerberosUtil; @@ -45,11 +45,9 @@ import java.security.NoSuchAlgorithmException; import java.util.ArrayList; import java.util.Collections; -import java.util.HashMap; import java.util.List; import java.util.ListIterator; import java.util.Locale; -import java.util.Map; import java.util.concurrent.CopyOnWriteArrayList; import static org.apache.hadoop.registry.client.impl.zk.ZookeeperConfigOptions.*; @@ -797,65 +795,6 @@ public void setKerberosPrincipalAndKeytab(String principal, String keytab) { this.keytab = keytab; } - /** - * Creates a programmatic version of a jaas.conf file. This can be used - * instead of writing a jaas.conf file and setting the system property, - * "java.security.auth.login.config", to point to that file. It is meant to be - * used for connecting to ZooKeeper. - */ - @InterfaceAudience.Private - public static class JaasConfiguration extends - javax.security.auth.login.Configuration { - - private final javax.security.auth.login.Configuration baseConfig = - javax.security.auth.login.Configuration.getConfiguration(); - private static AppConfigurationEntry[] entry; - private String entryName; - - /** - * Add an entry to the jaas configuration with the passed in name, - * principal, and keytab. The other necessary options will be set for you. - * - * @param entryName The name of the entry (e.g. "Client") - * @param principal The principal of the user - * @param keytab The location of the keytab - */ - public JaasConfiguration(String entryName, String principal, String keytab) { - this.entryName = entryName; - Map options = new HashMap(); - options.put("keyTab", keytab); - options.put("principal", principal); - options.put("useKeyTab", "true"); - options.put("storeKey", "true"); - options.put("useTicketCache", "false"); - options.put("refreshKrb5Config", "true"); - String jaasEnvVar = System.getenv("HADOOP_JAAS_DEBUG"); - if (jaasEnvVar != null && "true".equalsIgnoreCase(jaasEnvVar)) { - options.put("debug", "true"); - } - entry = new AppConfigurationEntry[]{ - new AppConfigurationEntry(getKrb5LoginModuleName(), - AppConfigurationEntry.LoginModuleControlFlag.REQUIRED, - options)}; - } - - @Override - public AppConfigurationEntry[] getAppConfigurationEntry(String name) { - return (entryName.equals(name)) ? entry : ((baseConfig != null) - ? baseConfig.getAppConfigurationEntry(name) : null); - } - - private String getKrb5LoginModuleName() { - String krb5LoginModuleName; - if (System.getProperty("java.vendor").contains("IBM")) { - krb5LoginModuleName = "com.ibm.security.auth.module.Krb5LoginModule"; - } else { - krb5LoginModuleName = "com.sun.security.auth.module.Krb5LoginModule"; - } - return krb5LoginModuleName; - } - } - /** * Set the client properties. This forces the ZK client into * failing if it can't auth. diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/qjournal/client/IPCLoggerChannel.java b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/qjournal/client/IPCLoggerChannel.java index 548525ba012b8..58c5ad39b99ab 100644 --- a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/qjournal/client/IPCLoggerChannel.java +++ b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/qjournal/client/IPCLoggerChannel.java @@ -745,7 +745,7 @@ private void constructHttpServerURI(GetEditLogManifestResponseProto ret) { URI uri = URI.create(ret.getFromURL()); httpServerURL = getHttpServerURI(uri.getScheme(), uri.getPort()); } else { - httpServerURL = getHttpServerURI("http", ret.getHttpPort());; + httpServerURL = getHttpServerURI("http", ret.getHttpPort()); } } @@ -754,7 +754,7 @@ private void constructHttpServerURI(GetJournalStateResponseProto ret) { URI uri = URI.create(ret.getFromURL()); httpServerURL = getHttpServerURI(uri.getScheme(), uri.getPort()); } else { - httpServerURL = getHttpServerURI("http", ret.getHttpPort());; + httpServerURL = getHttpServerURI("http", ret.getHttpPort()); } } diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/SecondaryNameNode.java b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/SecondaryNameNode.java index a0c8e9840f4b3..e95200b35aaa5 100644 --- a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/SecondaryNameNode.java +++ b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/SecondaryNameNode.java @@ -803,7 +803,7 @@ enum Command { geteditsizeOpt = new Option("geteditsize", "return the number of uncheckpointed transactions on the NameNode"); checkpointOpt = OptionBuilder.withArgName("force") - .hasOptionalArg().withDescription("checkpoint on startup").create("checkpoint");; + .hasOptionalArg().withDescription("checkpoint on startup").create("checkpoint"); formatOpt = new Option("format", "format the local storage during startup"); helpOpt = new Option("h", "help", false, "get help information"); diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/site/markdown/HDFSHighAvailabilityWithNFS.md b/hadoop-hdfs-project/hadoop-hdfs/src/site/markdown/HDFSHighAvailabilityWithNFS.md index 06cda83651e56..df1ab68afca5b 100644 --- a/hadoop-hdfs-project/hadoop-hdfs/src/site/markdown/HDFSHighAvailabilityWithNFS.md +++ b/hadoop-hdfs-project/hadoop-hdfs/src/site/markdown/HDFSHighAvailabilityWithNFS.md @@ -192,8 +192,8 @@ The order in which you set these configurations is unimportant, but the values y terminated, before transitioning another NameNode to the Active state. In order to do this, you must configure at least one **fencing method.** These are configured as a carriage-return-separated list, which will be attempted in order - until one indicates that fencing has succeeded. There are two methods which - ship with Hadoop: *shell* and *sshfence*. For information on implementing + until one indicates that fencing has succeeded. There are three methods which + ship with Hadoop: *shell*, *sshfence* and *powershell*. For information on implementing your own custom fencing method, see the *org.apache.hadoop.ha.NodeFencer* class. - - - @@ -283,7 +283,27 @@ The order in which you set these configurations is unimportant, but the values y a subshell to kill its parent in some number of seconds). - - - - + + **powershell** - use PowerShell to remotely connect to a machine and kill + the required process + + The *powershell* fencing method uses PowerShell command. It may be + configured like so: + + + dfs.ha.fencing.methods + powershell(NameNode) + + + The argument passed to this fencer should be a unique string in the "CommandLine" + attribute for the "java.exe" process. For example, the full path for the Namenode: + "org.apache.hadoop.hdfs.server.namenode.NameNode". + The administrator can also shorten the name to "Namenode" if it's unique. + + **Note:** This only works in Windows. + + - - - + * **fs.defaultFS** - the default path prefix used by the Hadoop FS client when none is given Optionally, you may now configure the default path for Hadoop clients to use diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/site/markdown/HDFSHighAvailabilityWithQJM.md b/hadoop-hdfs-project/hadoop-hdfs/src/site/markdown/HDFSHighAvailabilityWithQJM.md index 4bb5971107eb7..6bdd4e1c52927 100644 --- a/hadoop-hdfs-project/hadoop-hdfs/src/site/markdown/HDFSHighAvailabilityWithQJM.md +++ b/hadoop-hdfs-project/hadoop-hdfs/src/site/markdown/HDFSHighAvailabilityWithQJM.md @@ -239,8 +239,8 @@ The order in which you set these configurations is unimportant, but the values y The fencing methods used during a failover are configured as a carriage-return-separated list, which will be attempted in order until - one indicates that fencing has succeeded. There are two methods which ship with - Hadoop: *shell* and *sshfence*. For information on implementing your own custom + one indicates that fencing has succeeded. There are three methods which ship with + Hadoop: *shell*, *sshfence* and *powershell*. For information on implementing your own custom fencing method, see the *org.apache.hadoop.ha.NodeFencer* class. - - - @@ -331,6 +331,26 @@ The order in which you set these configurations is unimportant, but the values y - - - + **powershell** - use PowerShell to remotely connect to a machine and kill + the required process + + The *powershell* fencing method uses PowerShell command. It may be + configured like so: + + + dfs.ha.fencing.methods + powershell(NameNode) + + + The argument passed to this fencer should be a unique string in the "CommandLine" + attribute for the "java.exe" process. For example, the full path for the Namenode: + "org.apache.hadoop.hdfs.server.namenode.NameNode". + The administrator can also shorten the name to "Namenode" if it's unique. + + **Note:** This only works in Windows. + + - - - + * **fs.defaultFS** - the default path prefix used by the Hadoop FS client when none is given Optionally, you may now configure the default path for Hadoop clients to use diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/TestAddBlockRetry.java b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/TestAddBlockRetry.java index 088a47e893650..1a763b5bae3bd 100644 --- a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/TestAddBlockRetry.java +++ b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/TestAddBlockRetry.java @@ -99,7 +99,7 @@ public void testRetryAddBlockWhileInChooseTarget() throws Exception { HdfsConstants.GRANDFATHER_INODE_ID, "clientName", null, onRetryBlock); } finally { - ns.readUnlock();; + ns.readUnlock(); } DatanodeStorageInfo targets[] = FSDirWriteFileOp.chooseTargetForNewBlock( ns.getBlockManager(), src, null, null, null, r); diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/TestCheckPointForSecurityTokens.java b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/TestCheckPointForSecurityTokens.java index cff4e1f32f5b6..33fefca5fda8d 100644 --- a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/TestCheckPointForSecurityTokens.java +++ b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/TestCheckPointForSecurityTokens.java @@ -91,7 +91,7 @@ public void testSaveNamespace() throws IOException { log.scanLog(Long.MAX_VALUE, true); long numTransactions = (log.getLastTxId() - log.getFirstTxId()) + 1; assertEquals("In-progress log " + log + " should have 5 transactions", - 5, numTransactions);; + 5, numTransactions); } // Saving image in safe mode should succeed diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/TestCheckpoint.java b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/TestCheckpoint.java index 064d5ae83957a..1265863e11515 100644 --- a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/TestCheckpoint.java +++ b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/TestCheckpoint.java @@ -2613,7 +2613,7 @@ private static List getCheckpointCurrentDirs(SecondaryNameNode secondary) } private static CheckpointStorage spyOnSecondaryImage(SecondaryNameNode secondary1) { - CheckpointStorage spy = Mockito.spy((CheckpointStorage)secondary1.getFSImage());; + CheckpointStorage spy = Mockito.spy((CheckpointStorage)secondary1.getFSImage()); secondary1.setFSImage(spy); return spy; } diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/TestFavoredNodesEndToEnd.java b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/TestFavoredNodesEndToEnd.java index f5a112c7acd54..b8cc32e43e2d8 100644 --- a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/TestFavoredNodesEndToEnd.java +++ b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/TestFavoredNodesEndToEnd.java @@ -150,7 +150,7 @@ public void testWhenSomeNodesAreNotGood() throws Exception { d.stopDecommission(); BlockLocation[] locations = getBlockLocations(p); - Assert.assertEquals(replication, locations[0].getNames().length);; + Assert.assertEquals(replication, locations[0].getNames().length); //also make sure that the datanode[0] is not in the list of hosts for (int i = 0; i < replication; i++) { final String loc = locations[0].getNames()[i]; diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/ha/TestDelegationTokensWithHA.java b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/ha/TestDelegationTokensWithHA.java index 2ef48a31f9dc3..bd54ba2e9896c 100644 --- a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/ha/TestDelegationTokensWithHA.java +++ b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/ha/TestDelegationTokensWithHA.java @@ -317,7 +317,7 @@ public Void run() throws Exception { longUgi.doAs(new PrivilegedExceptionAction() { @Override public Void run() throws Exception { - token.cancel(conf);; + token.cancel(conf); return null; } }); diff --git a/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-core/src/main/java/org/apache/hadoop/mapred/MapTask.java b/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-core/src/main/java/org/apache/hadoop/mapred/MapTask.java index fa4396d77f458..06d9fbbe7a323 100644 --- a/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-core/src/main/java/org/apache/hadoop/mapred/MapTask.java +++ b/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-core/src/main/java/org/apache/hadoop/mapred/MapTask.java @@ -150,8 +150,8 @@ public void readFields(DataInput in) throws IOException { * @param */ class TrackedRecordReader - implements RecordReader { - private RecordReader rawIn; + implements RecordReader { + private RecordReader rawIn; private Counters.Counter fileInputByteCounter; private Counters.Counter inputRecordCounter; private TaskReporter reporter; @@ -240,7 +240,7 @@ private long getInputBytes(List stats) { * This class skips the records based on the failed ranges from previous * attempts. */ - class SkippingRecordReader extends TrackedRecordReader { + class SkippingRecordReader extends TrackedRecordReader { private SkipRangeIterator skipIt; private SequenceFile.Writer skipWriter; private boolean toWriteSkipRecs; @@ -930,7 +930,7 @@ public static class MapOutputBuffer // spill accounting private int maxRec; private int softLimit; - boolean spillInProgress;; + boolean spillInProgress; int bufferRemaining; volatile Throwable sortSpillException = null; diff --git a/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-core/src/main/java/org/apache/hadoop/mapreduce/lib/aggregate/LongValueMax.java b/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-core/src/main/java/org/apache/hadoop/mapreduce/lib/aggregate/LongValueMax.java index 90b4ae0f54442..d574bf694f2de 100644 --- a/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-core/src/main/java/org/apache/hadoop/mapreduce/lib/aggregate/LongValueMax.java +++ b/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-core/src/main/java/org/apache/hadoop/mapreduce/lib/aggregate/LongValueMax.java @@ -96,7 +96,7 @@ public void reset() { * expected to be used by the a combiner. */ public ArrayList getCombinerOutput() { - ArrayList retv = new ArrayList(1);; + ArrayList retv = new ArrayList(1); retv.add("" + maxVal); return retv; } diff --git a/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-hs/src/main/java/org/apache/hadoop/mapreduce/v2/hs/webapp/HsTasksBlock.java b/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-hs/src/main/java/org/apache/hadoop/mapreduce/v2/hs/webapp/HsTasksBlock.java index a00146e680bc0..aa3acfbc54641 100644 --- a/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-hs/src/main/java/org/apache/hadoop/mapreduce/v2/hs/webapp/HsTasksBlock.java +++ b/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-hs/src/main/java/org/apache/hadoop/mapreduce/v2/hs/webapp/HsTasksBlock.java @@ -123,7 +123,7 @@ public class HsTasksBlock extends HtmlBlock { long sortFinishTime = -1; long attemptFinishTime = -1; long elapsedShuffleTime = -1; - long elapsedSortTime = -1;; + long elapsedSortTime = -1; long elapsedReduceTime = -1; long attemptElapsed = -1; TaskAttempt successful = info.getSuccessful(); diff --git a/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-jobclient/src/test/java/org/apache/hadoop/mapred/BigMapOutput.java b/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-jobclient/src/test/java/org/apache/hadoop/mapred/BigMapOutput.java index 35992f5de007f..3aedfbf1ecb2a 100644 --- a/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-jobclient/src/test/java/org/apache/hadoop/mapred/BigMapOutput.java +++ b/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-jobclient/src/test/java/org/apache/hadoop/mapred/BigMapOutput.java @@ -70,7 +70,7 @@ private static void createBigMapInputFile(Configuration conf, FileSystem fs, BytesWritable.class, BytesWritable.class, CompressionType.NONE); long numBytesToWrite = fileSizeInMB * 1024 * 1024; - int minKeySize = conf.getInt(MIN_KEY, 10);; + int minKeySize = conf.getInt(MIN_KEY, 10); int keySizeRange = conf.getInt(MAX_KEY, 1000) - minKeySize; int minValueSize = conf.getInt(MIN_VALUE, 0); diff --git a/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-jobclient/src/test/java/org/apache/hadoop/mapred/NotificationTestCase.java b/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-jobclient/src/test/java/org/apache/hadoop/mapred/NotificationTestCase.java index 5df1af5cce1d1..3372c8f28b6ff 100644 --- a/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-jobclient/src/test/java/org/apache/hadoop/mapred/NotificationTestCase.java +++ b/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-jobclient/src/test/java/org/apache/hadoop/mapred/NotificationTestCase.java @@ -179,8 +179,7 @@ public void testMR() throws Exception { // Hack for local FS that does not have the concept of a 'mounting point' if (isLocalFS()) { - String localPathRoot = System.getProperty("test.build.data","/tmp") - .toString().replace(' ', '+');; + String localPathRoot = System.getProperty("test.build.data", "/tmp").replace(' ', '+'); inDir = new Path(localPathRoot, inDir); outDir = new Path(localPathRoot, outDir); } @@ -217,8 +216,7 @@ private String launchWordCount(JobConf conf, // Hack for local FS that does not have the concept of a 'mounting point' if (isLocalFS()) { - String localPathRoot = System.getProperty("test.build.data","/tmp") - .toString().replace(' ', '+');; + String localPathRoot = System.getProperty("test.build.data", "/tmp").replace(' ', '+'); inDir = new Path(localPathRoot, inDir); outDir = new Path(localPathRoot, outDir); } diff --git a/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-jobclient/src/test/java/org/apache/hadoop/mapreduce/TestValueIterReset.java b/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-jobclient/src/test/java/org/apache/hadoop/mapreduce/TestValueIterReset.java index 4bcacd8ec4858..343e81f163128 100644 --- a/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-jobclient/src/test/java/org/apache/hadoop/mapreduce/TestValueIterReset.java +++ b/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-jobclient/src/test/java/org/apache/hadoop/mapreduce/TestValueIterReset.java @@ -437,7 +437,7 @@ private static int test3(IntWritable key, int count = 0; while (values.hasNext()) { - i = values.next();; + i = values.next(); LOG.info(key + ":" + i); if (count == 5) { diff --git a/hadoop-mapreduce-project/hadoop-mapreduce-examples/src/main/java/org/apache/hadoop/examples/pi/math/LongLong.java b/hadoop-mapreduce-project/hadoop-mapreduce-examples/src/main/java/org/apache/hadoop/examples/pi/math/LongLong.java index 8d7e7a0a9713d..8299ba161a80b 100644 --- a/hadoop-mapreduce-project/hadoop-mapreduce-examples/src/main/java/org/apache/hadoop/examples/pi/math/LongLong.java +++ b/hadoop-mapreduce-project/hadoop-mapreduce-examples/src/main/java/org/apache/hadoop/examples/pi/math/LongLong.java @@ -84,7 +84,7 @@ static LongLong multiplication(final LongLong r, final long a, final long b) { final long v = x1*y1; final long tmp = (t - u)>>>1; - result.d0 = ((t + u)>>>1) - v + ((tmp << MID) & FULL_MASK);; + result.d0 = ((t + u)>>>1) - v + ((tmp << MID) & FULL_MASK); result.d1 = v + (tmp >> MID); return result; */ diff --git a/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/services/AbfsClient.java b/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/services/AbfsClient.java index c9d6a613327d0..a8d4eb9c73b58 100644 --- a/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/services/AbfsClient.java +++ b/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/services/AbfsClient.java @@ -117,7 +117,6 @@ public class AbfsClient implements Closeable { private static final LogExactlyOnce ABFS_METADATA_INCOMPLETE_RENAME_FAILURE = new LogExactlyOnce(LOG); - private AbfsClient(final URL baseUrl, final SharedKeyCredentials sharedKeyCredentials, final AbfsConfiguration abfsConfiguration, diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-client/src/test/java/org/apache/hadoop/yarn/client/cli/TestYarnCLI.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-client/src/test/java/org/apache/hadoop/yarn/client/cli/TestYarnCLI.java index 621540677cf04..22c7cc34bcf2b 100644 --- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-client/src/test/java/org/apache/hadoop/yarn/client/cli/TestYarnCLI.java +++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-client/src/test/java/org/apache/hadoop/yarn/client/cli/TestYarnCLI.java @@ -1928,7 +1928,7 @@ public void testGetQueueInfoWithNonExistedQueue() throws Exception { QueueCLI cli = createAndGetQueueCLI(); when(client.getQueueInfo(any(String.class))).thenReturn(null); int result = cli.run(new String[] { "-status", queueName }); - assertEquals(-1, result);; + assertEquals(-1, result); ByteArrayOutputStream baos = new ByteArrayOutputStream(); PrintWriter pw = new PrintWriter(baos); pw.println("Cannot get queue from RM by queueName = " + queueName diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/main/java/org/apache/hadoop/yarn/api/records/impl/pb/ReservationAllocationStatePBImpl.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/main/java/org/apache/hadoop/yarn/api/records/impl/pb/ReservationAllocationStatePBImpl.java index 88e39ec994fa5..48f62c1bae721 100644 --- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/main/java/org/apache/hadoop/yarn/api/records/impl/pb/ReservationAllocationStatePBImpl.java +++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/main/java/org/apache/hadoop/yarn/api/records/impl/pb/ReservationAllocationStatePBImpl.java @@ -62,7 +62,7 @@ public class ReservationAllocationStatePBImpl extends ReservationAllocationState { private ReservationAllocationStateProto proto = - ReservationAllocationStateProto.getDefaultInstance();; + ReservationAllocationStateProto.getDefaultInstance(); private ReservationAllocationStateProto.Builder builder = null; private boolean viaProto = false; diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/main/java/org/apache/hadoop/yarn/logaggregation/AggregatedLogDeletionService.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/main/java/org/apache/hadoop/yarn/logaggregation/AggregatedLogDeletionService.java index 4f10b2fd4ce23..c7e871d0f5034 100644 --- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/main/java/org/apache/hadoop/yarn/logaggregation/AggregatedLogDeletionService.java +++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/main/java/org/apache/hadoop/yarn/logaggregation/AggregatedLogDeletionService.java @@ -59,7 +59,7 @@ public class AggregatedLogDeletionService extends AbstractService { private Timer timer = null; private long checkIntervalMsecs; - private List tasks; + private List tasks = new ArrayList<>(); public static class LogDeletionTask extends TimerTask { private Configuration conf; diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/main/java/org/apache/hadoop/yarn/security/ConfiguredYarnAuthorizer.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/main/java/org/apache/hadoop/yarn/security/ConfiguredYarnAuthorizer.java index 615ecb0106e08..227f1c960a435 100644 --- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/main/java/org/apache/hadoop/yarn/security/ConfiguredYarnAuthorizer.java +++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/main/java/org/apache/hadoop/yarn/security/ConfiguredYarnAuthorizer.java @@ -43,7 +43,7 @@ public class ConfiguredYarnAuthorizer extends YarnAuthorizationProvider { private final ConcurrentMap> allAcls = new ConcurrentHashMap<>(); private volatile AccessControlList adminAcl = null; - private final ReentrantReadWriteLock lock = new ReentrantReadWriteLock();; + private final ReentrantReadWriteLock lock = new ReentrantReadWriteLock(); private final ReentrantReadWriteLock.ReadLock readLock = lock.readLock(); private final ReentrantReadWriteLock.WriteLock writeLock = lock.writeLock(); diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/main/java/org/apache/hadoop/yarn/server/nodemanager/WindowsSecureContainerExecutor.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/main/java/org/apache/hadoop/yarn/server/nodemanager/WindowsSecureContainerExecutor.java index c13efadff1b33..6132e579ef982 100644 --- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/main/java/org/apache/hadoop/yarn/server/nodemanager/WindowsSecureContainerExecutor.java +++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/main/java/org/apache/hadoop/yarn/server/nodemanager/WindowsSecureContainerExecutor.java @@ -449,7 +449,7 @@ private enum State { COMPLETE }; - private State state;; + private State state; private final String cwd; private final String jobName; diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/monitor/capacity/ProportionalCapacityPreemptionPolicy.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/monitor/capacity/ProportionalCapacityPreemptionPolicy.java index 9439c538af52f..1c4d60962eef5 100644 --- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/monitor/capacity/ProportionalCapacityPreemptionPolicy.java +++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/monitor/capacity/ProportionalCapacityPreemptionPolicy.java @@ -496,7 +496,7 @@ private void containerBasedPreemptOrKill(CSQueue root, Map> toPreempt = new HashMap<>(); Map>> toPreemptPerSelector = new HashMap<>();; + Set>> toPreemptPerSelector = new HashMap<>(); for (PreemptionCandidatesSelector selector : candidatesSelectionPolicies) { long startTime = 0; diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/TestAppManager.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/TestAppManager.java index cc5e8af176285..be0dbf2a1ba41 100644 --- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/TestAppManager.java +++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/TestAppManager.java @@ -1772,7 +1772,7 @@ private static ContainerLaunchContext mockContainerLaunchContext( RecordFactory recordFactory) { ContainerLaunchContext amContainer = recordFactory.newRecordInstance( ContainerLaunchContext.class); - amContainer.setApplicationACLs(new HashMap());; + amContainer.setApplicationACLs(new HashMap()); return amContainer; } diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/TestRMAdminService.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/TestRMAdminService.java index bc540b0ba7f32..457e9d47dd0b4 100644 --- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/TestRMAdminService.java +++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/TestRMAdminService.java @@ -109,7 +109,7 @@ public class TestRMAdminService { - private Configuration configuration;; + private Configuration configuration; private MockRM rm = null; private FileSystem fs; private Path workingPath; diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-site/src/site/markdown/UsingNuma.md b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-site/src/site/markdown/UsingNuma.md new file mode 100644 index 0000000000000..c93469cae3f42 --- /dev/null +++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-site/src/site/markdown/UsingNuma.md @@ -0,0 +1,206 @@ + + +# NUMA + +Non-uniform memory access (NUMA) is a computer memory design used in multiprocessing, +where the memory access time depends on the memory location relative to the processor. +Under NUMA, a processor can access its own local memory faster than non-local memory +(memory local to another processor or memory shared between processors). +Yarn Containers can make benefit of this NUMA design to get better performance by binding to a +specific NUMA node and all subsequent memory allocations will be served by the same node, +reducing remote memory accesses. NUMA support for YARN Container has to be enabled only if worker +node machines has NUMA support. + +# Enabling NUMA + +### Prerequisites + +- As of now, NUMA awareness works only with `LinuxContainerExecutor` (LCE) +- To use the feature of NUMA awareness in the cluster,It must be enabled with + LinuxContainerExecutor (LCE) +- Steps to enable SecureContainer (LCE) for cluster is documented [here](SecureContainer.md) + +## Configurations + +**1) Enable/Disable the NUMA awareness** + +This property enables the NUMA awareness feature in the Node Manager +for the containers. By default, the value of this property is false which means it is disabled. +If this property is `true` then only the below configurations will be applicable otherwise they +will be ignored. + +In `yarn-site.xml` add + +``` + + yarn.nodemanager.numa-awareness.enabled + true + +``` + +**2) NUMA topology** + +This property decides whether to read the NUMA topology from the system or from the +configurations. If this property value is true then the topology will be read from the system using +`numactl --hardware` command in UNIX systems and similar way in windows. +If this property is false then the topology will be read using the below configurations. +Default value of this configuration is false which means NodeManager will read the NUMA topology +from the below configurations. + +In `yarn-site.xml` add + +``` + + yarn.nodemanager.numa-awareness.read-topology + false + +``` + +**3) Numa command** + +This property is passed when `yarn.nodemanager.numa-awareness.read-topology` is set to true. +It is recommended to verify the installation of `numactl` command in the Linux OS of every node. + +Use `/usr/bin/numactl --hardware` to verify. +Sample output of `/usr/bin/numactl --hardware` + +``` +available: 2 nodes (0-1) +node 0 cpus: 0 1 2 3 4 5 6 7 8 9 10 11 12 13 14 15 16 17 18 19 20 21 22 23 48 49 50 51 52 53 54 55 56 57 58 59 60 61 62 63 64 65 66 67 68 69 70 71 +node 0 size: 191297 MB +node 0 free: 186539 MB +node 1 cpus: 24 25 26 27 28 29 30 31 32 33 34 35 36 37 38 39 40 41 42 43 44 45 46 47 72 73 74 75 76 77 78 79 80 81 82 83 84 85 86 87 88 89 90 91 92 93 94 95 +node 1 size: 191383 MB +node 1 free: 185914 MB +node distances: +node 0 1 + 0: 10 21 + 1: 21 10 +``` + +In `yarn-site.xml` add + +``` + + yarn.nodemanager.numa-awareness.numactl.cmd + /usr/bin/numactl + +``` + +**4) NUMA nodes id’s** + +This property is used to provide the NUMA node ids as comma separated values.It will be read only +when the `yarn.nodemanager.numa-awareness.read-topology` is false. + +In ```yarn-site.xml``` add + +``` + + yarn.nodemanager.numa-awareness.node-ids + 0,1 + +``` + +**5) NUMA Node memory** + +This property will be used to read the memory(in MB) configured for each NUMA node specified in +`yarn.nodemanager.numa-awareness.node-ids` by substituting the node id in the place of +``.It will be read only when the `yarn.nodemanager.numa-awareness.read-topology` +is false. + +In `yarn-site.xml` add + +``` + + yarn.nodemanager.numa-awareness..memory + 191297 + +``` + +The value passed is the per node memory available , from the above sample output of +`numactl --hardware` the value passed for the property is the memory available i.e `191297` + +**6) NUMA Node CPUs** + +This property will be used to read the number of CPUs configured for each node specified in +`yarn.nodemanager.numa-awareness.node-ids` by substituting the node id in the place of +``.It will be read only when the `yarn.nodemanager.numa-awareness.read-topology` is false. + +In ```yarn-site.xml``` add + +``` + + yarn.nodemanager.numa-awareness..cpus + 48 + +``` + +referring to the `numactl --hardware` output , number of cpu's in a node is `48`. + +**7) Passing java_opts for map/reduce** + +Every container has to be aware of NUMA and the JVM can be notified via passing NUMA flag. +Spark, Tez and other YARN Applications also need to set the container JVM Opts to leverage +NUMA Support. + +In ```mapred-site.xml``` add + +``` + + mapreduce.reduce.java.opts + -XX:+UseNUMA + + + mapreduce.map.java.opts + -XX:+UseNUMA + +``` + +# Default configuration + +| Property | Default value | +| --- |-----| +|yarn.nodemanager.numa-awareness.enabled|false| +|yarn.nodemanager.numa-awareness.read-topology|false| + +# Enable numa balancing at OS Level (Optional) + +In linux, by default numa balancing is by default off. For more performance improvement, +NumaBalancing can be turned on for all the nodes in cluster + +``` +echo 1 | sudo tee /proc/sys/kernel/numa_balancing +``` + +# Verify + +**1) NodeManager log** + +In any of the NodeManager, grep log file using below command + +`grep "NUMA resources allocation is enabled," *` + +Sample log with `LinuxContainerExecutor` enabled message + +``` +.log.2022-06-24-19.gz:2022-06-24 19:16:40,178 INFO org.apache.hadoop.yarn.server.nodemanager.containermanager.linux.resources.numa.NumaResourceHandlerImpl (main): NUMA resources allocation is enabled, initializing NUMA resources allocator. +``` + +**2) Container Log** + +Grep the NodeManager log using below grep command to check if a container is assigned with NUMA node +resources. + +`grep "NUMA node" | grep ` \ No newline at end of file From 4164ee2e1535d85f7a39260fbe4d70b0fa3697b9 Mon Sep 17 00:00:00 2001 From: Pranav Saxena Date: Fri, 8 Jul 2022 11:56:37 +0530 Subject: [PATCH 32/77] remove additional space --- .../java/org/apache/hadoop/yarn/server/router/RouterMetrics.java | 1 - 1 file changed, 1 deletion(-) diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-router/src/main/java/org/apache/hadoop/yarn/server/router/RouterMetrics.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-router/src/main/java/org/apache/hadoop/yarn/server/router/RouterMetrics.java index 629116e9d72ff..e6e20e2af8c1a 100644 --- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-router/src/main/java/org/apache/hadoop/yarn/server/router/RouterMetrics.java +++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-router/src/main/java/org/apache/hadoop/yarn/server/router/RouterMetrics.java @@ -825,5 +825,4 @@ public void incrGetQueueInfoFailedRetrieved() { public void incrMoveApplicationAcrossQueuesFailedRetrieved() { numMoveApplicationAcrossQueuesFailedRetrieved.incr(); } - } From 6c05dac00aa056254c63e46c8e89e80294088ea0 Mon Sep 17 00:00:00 2001 From: Pranav Saxena Date: Fri, 8 Jul 2022 18:18:15 +0530 Subject: [PATCH 33/77] Findbug resolution --- .../fs/azurebfs/security/EncryptionAdapter.java | 17 +++++++++++++++-- .../hadoop/fs/azurebfs/services/AbfsClient.java | 8 +++++++- 2 files changed, 22 insertions(+), 3 deletions(-) diff --git a/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/security/EncryptionAdapter.java b/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/security/EncryptionAdapter.java index 4a3e517de02dd..676fc27e3108f 100644 --- a/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/security/EncryptionAdapter.java +++ b/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/security/EncryptionAdapter.java @@ -71,6 +71,11 @@ public SecretKey createEncryptionContext() throws IOException { public void computeKeys() throws IOException { SecretKey key = getEncryptionKey(); Preconditions.checkNotNull(key, "Encryption key should not be null."); + if(key.getClass() == ABFSKey.class) { + encodedKey = ((ABFSKey) key).getBase64EncodedString(); + encodedKeySHA = getBase64EncodedString(((ABFSKey) key).getSHA256Hash()); + return; + } encodedKey = getBase64EncodedString(key.getEncoded()); encodedKeySHA = getBase64EncodedString(getSHA256Hash(key.getEncoded())); } @@ -97,7 +102,7 @@ public void destroy() throws DestroyFailedException { public class ABFSKey implements SecretKey { private final byte[] bytes; public ABFSKey(byte[] bytes) { - this.bytes = bytes; + this.bytes = (bytes != null) ? bytes.clone() : null; } @Override @@ -112,7 +117,15 @@ public String getFormat() { @Override public byte[] getEncoded() { - return bytes; + return bytes.clone(); + } + + public String getBase64EncodedString() { + return EncryptionAdapter.getBase64EncodedString(bytes); + } + + public byte[] getSHA256Hash() throws IOException { + return EncryptionAdapter.getSHA256Hash(bytes); } @Override diff --git a/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/services/AbfsClient.java b/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/services/AbfsClient.java index a8d4eb9c73b58..8c33b9fa39ffb 100644 --- a/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/services/AbfsClient.java +++ b/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/services/AbfsClient.java @@ -251,8 +251,14 @@ private void addEncryptionKeyRequestHeaders(String path, SecretKey encryptionContext = encryptionAdapter.createEncryptionContext(); encryptionAdapter.computeKeys(); + String base64EncodedSecret; + if(encryptionContext.getClass() == EncryptionAdapter.ABFSKey.class) { + base64EncodedSecret = ((EncryptionAdapter.ABFSKey)encryptionContext).getBase64EncodedString(); + } else { + base64EncodedSecret = Base64.getEncoder().encodeToString(encryptionContext.getEncoded()); + } requestHeaders.add(new AbfsHttpHeader(X_MS_ENCRYPTION_CONTEXT, - Base64.getEncoder().encodeToString(encryptionContext.getEncoded()))); + base64EncodedSecret)); try { encryptionContext.destroy(); } catch (DestroyFailedException e) { From cd7d05557286c7ab6ec1705f69e928e3752cac5c Mon Sep 17 00:00:00 2001 From: Pranav Saxena Date: Tue, 12 Jul 2022 12:17:23 +0530 Subject: [PATCH 34/77] take out ABFSKey class out; Ensure that ABFSKey is used by the EncryptionContextProvider implementation --- .../extensions/EncryptionContextProvider.java | 5 +- .../hadoop/fs/azurebfs/security/ABFSKey.java | 53 ++++++++++++++ .../fs/azurebfs/security/EncodingHelper.java | 26 +++++++ .../azurebfs/security/EncryptionAdapter.java | 69 ++----------------- .../fs/azurebfs/services/AbfsClient.java | 16 ++--- .../azurebfs/ITestAbfsCustomEncryption.java | 16 +++-- .../MockEncryptionContextProvider.java | 14 ++-- 7 files changed, 109 insertions(+), 90 deletions(-) create mode 100644 hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/security/ABFSKey.java create mode 100644 hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/security/EncodingHelper.java diff --git a/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/extensions/EncryptionContextProvider.java b/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/extensions/EncryptionContextProvider.java index cce6c902615a7..2c7add7e6e49f 100644 --- a/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/extensions/EncryptionContextProvider.java +++ b/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/extensions/EncryptionContextProvider.java @@ -23,6 +23,7 @@ import java.io.IOException; import org.apache.hadoop.conf.Configuration; +import org.apache.hadoop.fs.azurebfs.security.ABFSKey; public interface EncryptionContextProvider extends Destroyable { /** @@ -42,7 +43,7 @@ public interface EncryptionContextProvider extends Destroyable { * @return encryptionContext string * @throws IOException error in fetching encryption context */ - SecretKey getEncryptionContext(String path) throws IOException; + ABFSKey getEncryptionContext(String path) throws IOException; /** * Fetch encryption key in-exchange for encryption context @@ -52,7 +53,7 @@ public interface EncryptionContextProvider extends Destroyable { * @return Encryption key * @throws IOException error in fetching encryption key */ - SecretKey getEncryptionKey(String path, SecretKey encryptionContext) throws IOException; + ABFSKey getEncryptionKey(String path, ABFSKey encryptionContext) throws IOException; void destroy(); } diff --git a/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/security/ABFSKey.java b/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/security/ABFSKey.java new file mode 100644 index 0000000000000..cbd473440d358 --- /dev/null +++ b/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/security/ABFSKey.java @@ -0,0 +1,53 @@ +package org.apache.hadoop.fs.azurebfs.security; + +import javax.crypto.SecretKey; +import java.io.IOException; +import java.util.Arrays; + +public class ABFSKey implements SecretKey { + private byte[] bytes; + private String base64Encoding; + private byte[] sha256Hash; + public ABFSKey(byte[] bytes) { + if(bytes != null) { + this.bytes = bytes.clone(); + base64Encoding = EncodingHelper.getBase64EncodedString(this.bytes); + sha256Hash = EncodingHelper.getSHA256Hash(this.bytes); + } + } + + @Override + public String getAlgorithm() { + return null; + } + + @Override + public String getFormat() { + return null; + } + + /** + * This method to be called by implementations of EncryptionContextProvider interface. + * Method returns clone of the original bytes array to prevent findbugs flags. + * */ + @Override + public byte[] getEncoded() { + if(bytes == null) { + return null; + } + return bytes.clone(); + } + + public String getBase64EncodedString() { + return base64Encoding; + } + + public byte[] getSHA256Hash() { + return sha256Hash.clone(); + } + + @Override + public void destroy() { + Arrays.fill(bytes, (byte) 0); + } +} diff --git a/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/security/EncodingHelper.java b/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/security/EncodingHelper.java new file mode 100644 index 0000000000000..a2360991b0bd8 --- /dev/null +++ b/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/security/EncodingHelper.java @@ -0,0 +1,26 @@ +package org.apache.hadoop.fs.azurebfs.security; + +import java.io.IOException; +import java.security.MessageDigest; +import java.security.NoSuchAlgorithmException; +import java.util.Base64; + +public class EncodingHelper { + + public static byte[] getSHA256Hash(byte[] key) { + try { + final MessageDigest digester = MessageDigest.getInstance("SHA-256"); + return digester.digest(key); + } catch (NoSuchAlgorithmException ignored) { + /** + * This exception can be ignored. Reason being SHA-256 is a valid algorithm, and it is constant for all + * method calls. + */ + return null; + } + } + + public static String getBase64EncodedString(byte[] bytes) { + return Base64.getEncoder().encodeToString(bytes); + } +} diff --git a/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/security/EncryptionAdapter.java b/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/security/EncryptionAdapter.java index 676fc27e3108f..cf0eaa3040439 100644 --- a/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/security/EncryptionAdapter.java +++ b/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/security/EncryptionAdapter.java @@ -21,9 +21,7 @@ import java.io.IOException; import java.security.MessageDigest; import java.security.NoSuchAlgorithmException; -import java.util.Arrays; import java.util.Base64; -import javax.crypto.SecretKey; import javax.security.auth.DestroyFailedException; import javax.security.auth.Destroyable; @@ -33,8 +31,8 @@ public class EncryptionAdapter implements Destroyable { private final String path; - private SecretKey encryptionContext; - private SecretKey encryptionKey; + private ABFSKey encryptionContext; + private ABFSKey encryptionKey; private final EncryptionContextProvider provider; private String encodedKey = null; private String encodedKeySHA = null; @@ -53,7 +51,7 @@ public EncryptionAdapter(EncryptionContextProvider provider, String path) this.path = path; } - public SecretKey getEncryptionKey() throws IOException { + public ABFSKey getEncryptionKey() throws IOException { if (encryptionKey != null) { return encryptionKey; } @@ -61,7 +59,7 @@ public SecretKey getEncryptionKey() throws IOException { return encryptionKey; } - public SecretKey createEncryptionContext() throws IOException { + public ABFSKey createEncryptionContext() throws IOException { encryptionContext = provider.getEncryptionContext(path); Preconditions.checkNotNull(encryptionContext, "Encryption context should not be null."); @@ -69,15 +67,10 @@ public SecretKey createEncryptionContext() throws IOException { } public void computeKeys() throws IOException { - SecretKey key = getEncryptionKey(); + ABFSKey key = getEncryptionKey(); Preconditions.checkNotNull(key, "Encryption key should not be null."); - if(key.getClass() == ABFSKey.class) { - encodedKey = ((ABFSKey) key).getBase64EncodedString(); - encodedKeySHA = getBase64EncodedString(((ABFSKey) key).getSHA256Hash()); - return; - } - encodedKey = getBase64EncodedString(key.getEncoded()); - encodedKeySHA = getBase64EncodedString(getSHA256Hash(key.getEncoded())); + encodedKey = key.getBase64EncodedString(); + encodedKeySHA = EncodingHelper.getBase64EncodedString(key.getSHA256Hash()); } public String getEncodedKey() throws IOException { @@ -98,52 +91,4 @@ public void destroy() throws DestroyFailedException { encryptionKey.destroy(); provider.destroy(); } - - public class ABFSKey implements SecretKey { - private final byte[] bytes; - public ABFSKey(byte[] bytes) { - this.bytes = (bytes != null) ? bytes.clone() : null; - } - - @Override - public String getAlgorithm() { - return null; - } - - @Override - public String getFormat() { - return null; - } - - @Override - public byte[] getEncoded() { - return bytes.clone(); - } - - public String getBase64EncodedString() { - return EncryptionAdapter.getBase64EncodedString(bytes); - } - - public byte[] getSHA256Hash() throws IOException { - return EncryptionAdapter.getSHA256Hash(bytes); - } - - @Override - public void destroy() { - Arrays.fill(bytes, (byte) 0); - } - } - - public static byte[] getSHA256Hash(byte[] key) throws IOException { - try { - final MessageDigest digester = MessageDigest.getInstance("SHA-256"); - return digester.digest(key); - } catch (NoSuchAlgorithmException e) { - throw new IOException(e); - } - } - - public static String getBase64EncodedString(byte[] bytes) { - return Base64.getEncoder().encodeToString(bytes); - } } diff --git a/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/services/AbfsClient.java b/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/services/AbfsClient.java index 8c33b9fa39ffb..3a6e37dc65553 100644 --- a/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/services/AbfsClient.java +++ b/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/services/AbfsClient.java @@ -36,6 +36,7 @@ import java.util.concurrent.TimeUnit; import org.apache.hadoop.classification.VisibleForTesting; +import org.apache.hadoop.fs.azurebfs.security.ABFSKey; import org.apache.hadoop.fs.store.LogExactlyOnce; import org.apache.hadoop.util.Preconditions; import org.apache.hadoop.fs.Path; @@ -248,23 +249,14 @@ private void addEncryptionKeyRequestHeaders(String path, case ENCRYPTION_CONTEXT: if (isCreateFileRequest) { // get new context for create file request - SecretKey encryptionContext = + final ABFSKey encryptionContext = encryptionAdapter.createEncryptionContext(); encryptionAdapter.computeKeys(); String base64EncodedSecret; - if(encryptionContext.getClass() == EncryptionAdapter.ABFSKey.class) { - base64EncodedSecret = ((EncryptionAdapter.ABFSKey)encryptionContext).getBase64EncodedString(); - } else { - base64EncodedSecret = Base64.getEncoder().encodeToString(encryptionContext.getEncoded()); - } + base64EncodedSecret = encryptionContext.getBase64EncodedString(); requestHeaders.add(new AbfsHttpHeader(X_MS_ENCRYPTION_CONTEXT, base64EncodedSecret)); - try { - encryptionContext.destroy(); - } catch (DestroyFailedException e) { - throw new IOException( - "Could not destroy encryptionContext: " + e.getMessage()); - } + encryptionContext.destroy(); } else if (encryptionAdapter == null) { // get encryption context from GetPathStatus response header byte[] encryptionContext; diff --git a/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/ITestAbfsCustomEncryption.java b/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/ITestAbfsCustomEncryption.java index 9fc71ec5fb349..8d318de415023 100644 --- a/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/ITestAbfsCustomEncryption.java +++ b/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/ITestAbfsCustomEncryption.java @@ -25,6 +25,8 @@ import java.util.Hashtable; import java.util.Random; +import org.apache.hadoop.fs.azurebfs.security.ABFSKey; +import org.apache.hadoop.fs.azurebfs.security.EncodingHelper; import org.assertj.core.api.Assertions; import org.junit.Assume; import org.junit.Test; @@ -146,8 +148,8 @@ public ITestAbfsCustomEncryption() throws Exception { getConfiguration().getBoolean(FS_AZURE_TEST_NAMESPACE_ENABLED_ACCOUNT, false)); new Random().nextBytes(cpk); - cpkSHAEncoded = EncryptionAdapter.getBase64EncodedString( - EncryptionAdapter.getSHA256Hash(cpk)); + cpkSHAEncoded = EncodingHelper.getBase64EncodedString( + EncodingHelper.getSHA256Hash(cpk)); } @Test @@ -165,8 +167,8 @@ public void testCustomEncryptionCombinations() throws Exception { if (isCpkResponseHdrExpected) { if (requestEncryptionType == ENCRYPTION_CONTEXT) { String encryptionContext = ecp.getEncryptionContextForTest(relativePath); - String expectedKeySHA = EncryptionAdapter.getBase64EncodedString( - EncryptionAdapter.getSHA256Hash( + String expectedKeySHA = EncodingHelper.getBase64EncodedString( + EncodingHelper.getSHA256Hash( ecp.getEncryptionKeyForTest(encryptionContext))); Assertions.assertThat(httpOp.getResponseHeader(X_MS_ENCRYPTION_KEY_SHA256)) .isEqualTo(expectedKeySHA); @@ -273,9 +275,9 @@ private AzureBlobFileSystem getECProviderEnabledFS() throws Exception { private AzureBlobFileSystem getCPKEnabledFS() throws IOException { Configuration conf = getRawConfiguration(); - String cpkEncoded = EncryptionAdapter.getBase64EncodedString(cpk); - String cpkEncodedSHA = EncryptionAdapter.getBase64EncodedString( - EncryptionAdapter.getSHA256Hash(cpk)); + String cpkEncoded = EncodingHelper.getBase64EncodedString(cpk); + String cpkEncodedSHA = EncodingHelper.getBase64EncodedString( + EncodingHelper.getSHA256Hash(cpk)); conf.set(FS_AZURE_ENCRYPTION_ENCODED_CLIENT_PROVIDED_KEY + "." + getAccountName(), cpkEncoded); conf.set(FS_AZURE_ENCRYPTION_ENCODED_CLIENT_PROVIDED_KEY_SHA + "." diff --git a/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/extensions/MockEncryptionContextProvider.java b/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/extensions/MockEncryptionContextProvider.java index 74ff51eb7c9f0..d54487da2a906 100644 --- a/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/extensions/MockEncryptionContextProvider.java +++ b/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/extensions/MockEncryptionContextProvider.java @@ -28,32 +28,32 @@ import org.apache.hadoop.conf.Configuration; import org.apache.hadoop.classification.VisibleForTesting; +import org.apache.hadoop.fs.azurebfs.security.ABFSKey; import static org.apache.hadoop.fs.azurebfs.constants.TestConfigurationKeys.ENCRYPTION_KEY_LEN; public class MockEncryptionContextProvider implements EncryptionContextProvider { private HashMap pathToContextMap = new HashMap<>(); - private HashMap contextToKeyMap = new HashMap<>(); + private HashMap contextToKeyMap = new HashMap<>(); @Override public void initialize(Configuration configuration, String accountName, String fileSystem) throws IOException { } @Override - public SecretKey getEncryptionContext(String path) + public ABFSKey getEncryptionContext(String path) throws IOException { String newContext = UUID.randomUUID().toString(); pathToContextMap.put(path, newContext); byte[] newKey = new byte[ENCRYPTION_KEY_LEN]; new Random().nextBytes(newKey); - Key key = new Key(newKey); + ABFSKey key = new ABFSKey(newKey); contextToKeyMap.put(newContext, key); - return new Key(newContext.getBytes(StandardCharsets.UTF_8)); + return new ABFSKey(newContext.getBytes(StandardCharsets.UTF_8)); } @Override - public SecretKey getEncryptionKey(String path, - SecretKey encryptionContext) throws IOException { + public ABFSKey getEncryptionKey(String path, ABFSKey encryptionContext) throws IOException { String encryptionContextString = new String(encryptionContext.getEncoded(), StandardCharsets.UTF_8); if (!encryptionContextString.equals(pathToContextMap.get(path))) { @@ -65,7 +65,7 @@ public SecretKey getEncryptionKey(String path, @Override public void destroy() { pathToContextMap = null; - for (Key encryptionKey : contextToKeyMap.values()) { + for (ABFSKey encryptionKey : contextToKeyMap.values()) { encryptionKey.destroy(); } contextToKeyMap = null; From 926e7d97f99d01e5439a4ece3520b3f7fb5e6ed7 Mon Sep 17 00:00:00 2001 From: Pranav Saxena Date: Tue, 12 Jul 2022 16:02:07 +0530 Subject: [PATCH 35/77] refactor --- .../java/org/apache/hadoop/fs/azurebfs/security/ABFSKey.java | 3 +++ .../apache/hadoop/fs/azurebfs/security/EncryptionAdapter.java | 4 ++-- 2 files changed, 5 insertions(+), 2 deletions(-) diff --git a/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/security/ABFSKey.java b/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/security/ABFSKey.java index cbd473440d358..0333052f07be7 100644 --- a/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/security/ABFSKey.java +++ b/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/security/ABFSKey.java @@ -43,6 +43,9 @@ public String getBase64EncodedString() { } public byte[] getSHA256Hash() { + if(sha256Hash == null) { + return null; + } return sha256Hash.clone(); } diff --git a/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/security/EncryptionAdapter.java b/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/security/EncryptionAdapter.java index cf0eaa3040439..391ec68aea039 100644 --- a/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/security/EncryptionAdapter.java +++ b/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/security/EncryptionAdapter.java @@ -69,8 +69,8 @@ public ABFSKey createEncryptionContext() throws IOException { public void computeKeys() throws IOException { ABFSKey key = getEncryptionKey(); Preconditions.checkNotNull(key, "Encryption key should not be null."); - encodedKey = key.getBase64EncodedString(); - encodedKeySHA = EncodingHelper.getBase64EncodedString(key.getSHA256Hash()); + encodedKey = key.getBase64EncodedString(); + encodedKeySHA = EncodingHelper.getBase64EncodedString(key.getSHA256Hash()); } public String getEncodedKey() throws IOException { From af5e90b0197ce606263487f00cdf3b843d8cf96b Mon Sep 17 00:00:00 2001 From: Pranav Saxena Date: Tue, 19 Jul 2022 12:33:50 +0530 Subject: [PATCH 36/77] Added Apache license --- .../hadoop/fs/azurebfs/security/ABFSKey.java | 18 ++++++++++++++++++ .../fs/azurebfs/security/EncodingHelper.java | 18 ++++++++++++++++++ 2 files changed, 36 insertions(+) diff --git a/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/security/ABFSKey.java b/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/security/ABFSKey.java index 0333052f07be7..ac9001ca66413 100644 --- a/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/security/ABFSKey.java +++ b/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/security/ABFSKey.java @@ -1,3 +1,21 @@ +/** + * 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.fs.azurebfs.security; import javax.crypto.SecretKey; diff --git a/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/security/EncodingHelper.java b/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/security/EncodingHelper.java index a2360991b0bd8..c9ee75ecd00e8 100644 --- a/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/security/EncodingHelper.java +++ b/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/security/EncodingHelper.java @@ -1,3 +1,21 @@ +/** + * 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.fs.azurebfs.security; import java.io.IOException; From 9505ed9de76275a649e99f1d75353fe906007aaf Mon Sep 17 00:00:00 2001 From: Marios Meimaris Date: Thu, 1 Sep 2022 15:29:56 +0200 Subject: [PATCH 37/77] initial commit --- .../extensions/EncryptionContextProvider.java | 2 +- .../azurebfs/security/EncryptionAdapter.java | 58 ++++++++----------- .../fs/azurebfs/services/AbfsClient.java | 20 ++++--- .../fs/azurebfs/services/AbfsInputStream.java | 13 ++++- .../azurebfs/services/AbfsOutputStream.java | 29 ++++++---- hadoop-tools/pom.xml | 2 +- 6 files changed, 69 insertions(+), 55 deletions(-) diff --git a/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/extensions/EncryptionContextProvider.java b/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/extensions/EncryptionContextProvider.java index 2c7add7e6e49f..d1c178629b3c0 100644 --- a/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/extensions/EncryptionContextProvider.java +++ b/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/extensions/EncryptionContextProvider.java @@ -40,7 +40,7 @@ public interface EncryptionContextProvider extends Destroyable { * Fetch encryption context for a given path * * @param path file path from filesystem root - * @return encryptionContext string + * @return encryptionContext key * @throws IOException error in fetching encryption context */ ABFSKey getEncryptionContext(String path) throws IOException; diff --git a/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/security/EncryptionAdapter.java b/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/security/EncryptionAdapter.java index 391ec68aea039..9e269ed1e6160 100644 --- a/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/security/EncryptionAdapter.java +++ b/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/security/EncryptionAdapter.java @@ -19,10 +19,8 @@ package org.apache.hadoop.fs.azurebfs.security; import java.io.IOException; -import java.security.MessageDigest; -import java.security.NoSuchAlgorithmException; +import java.util.Arrays; import java.util.Base64; -import javax.security.auth.DestroyFailedException; import javax.security.auth.Destroyable; import org.apache.hadoop.util.Preconditions; @@ -34,8 +32,6 @@ public class EncryptionAdapter implements Destroyable { private ABFSKey encryptionContext; private ABFSKey encryptionKey; private final EncryptionContextProvider provider; - private String encodedKey = null; - private String encodedKeySHA = null; public EncryptionAdapter(EncryptionContextProvider provider, String path, byte[] encryptionContext) throws IOException { @@ -43,6 +39,7 @@ public EncryptionAdapter(EncryptionContextProvider provider, String path, Preconditions.checkNotNull(encryptionContext, "Encryption context should not be null."); this.encryptionContext = new ABFSKey(Base64.getDecoder().decode(encryptionContext)); + Arrays.fill(encryptionContext, (byte) 0); } public EncryptionAdapter(EncryptionContextProvider provider, String path) @@ -51,44 +48,39 @@ public EncryptionAdapter(EncryptionContextProvider provider, String path) this.path = path; } - public ABFSKey getEncryptionKey() throws IOException { - if (encryptionKey != null) { - return encryptionKey; + private void computeKeys() throws IOException { + if (encryptionContext == null) { + encryptionContext = provider.getEncryptionContext(path); } - encryptionKey = provider.getEncryptionKey(path, encryptionContext); - return encryptionKey; + Preconditions.checkNotNull(encryptionContext, + "Encryption context should not be null."); + if (encryptionKey == null) { + encryptionKey = provider.getEncryptionKey(path, encryptionContext); + } + Preconditions.checkNotNull(encryptionKey, "Encryption key should not be null."); } - public ABFSKey createEncryptionContext() throws IOException { - encryptionContext = provider.getEncryptionContext(path); - Preconditions.checkNotNull(encryptionContext, - "Encryption context should not be null."); - return encryptionContext; + public String getEncodedKey() throws IOException { + computeKeys(); + return getBase64EncodedString(encryptionKey.getEncoded()); } - public void computeKeys() throws IOException { - ABFSKey key = getEncryptionKey(); - Preconditions.checkNotNull(key, "Encryption key should not be null."); - encodedKey = key.getBase64EncodedString(); - encodedKeySHA = EncodingHelper.getBase64EncodedString(key.getSHA256Hash()); + public String getEncodedKeySHA() throws IOException { + computeKeys(); + return getBase64EncodedString(EncodingHelper.getSHA256Hash(encryptionKey.getEncoded())); } - public String getEncodedKey() throws IOException { - if (encodedKey == null) { - computeKeys(); - } - return encodedKey; + public String getEncodedContext() throws IOException { + computeKeys(); + return getBase64EncodedString(encryptionContext.getEncoded()); } - public String getEncodedKeySHA() throws IOException { - if (encodedKeySHA == null) { - computeKeys(); - } - return encodedKeySHA; + public void destroy() { + if (encryptionContext != null) encryptionContext.destroy(); + if (encryptionKey != null) encryptionKey.destroy(); } - public void destroy() throws DestroyFailedException { - encryptionKey.destroy(); - provider.destroy(); + public static String getBase64EncodedString(byte[] bytes) { + return Base64.getEncoder().encodeToString(bytes); } } diff --git a/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/services/AbfsClient.java b/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/services/AbfsClient.java index 3a6e37dc65553..bb5fdc64988f1 100644 --- a/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/services/AbfsClient.java +++ b/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/services/AbfsClient.java @@ -240,6 +240,7 @@ private void addEncryptionKeyRequestHeaders(String path, EncryptionAdapter encryptionAdapter, TracingContext tracingContext) throws IOException { String encodedKey, encodedKeySHA256; + boolean encryptionAdapterCreated = false; switch (encryptionType) { case GLOBAL_KEY: encodedKey = clientProvidedEncryptionKey; @@ -249,14 +250,8 @@ private void addEncryptionKeyRequestHeaders(String path, case ENCRYPTION_CONTEXT: if (isCreateFileRequest) { // get new context for create file request - final ABFSKey encryptionContext = - encryptionAdapter.createEncryptionContext(); - encryptionAdapter.computeKeys(); - String base64EncodedSecret; - base64EncodedSecret = encryptionContext.getBase64EncodedString(); requestHeaders.add(new AbfsHttpHeader(X_MS_ENCRYPTION_CONTEXT, - base64EncodedSecret)); - encryptionContext.destroy(); + encryptionAdapter.getEncodedContext())); } else if (encryptionAdapter == null) { // get encryption context from GetPathStatus response header byte[] encryptionContext; @@ -272,6 +267,7 @@ private void addEncryptionKeyRequestHeaders(String path, try { encryptionAdapter = new EncryptionAdapter(encryptionContextProvider, new Path(path).toUri().getPath(), encryptionContext); + encryptionAdapterCreated = true; } catch (IOException e) { LOG.debug("Could not initialize EncryptionAdapter"); throw e; @@ -284,6 +280,16 @@ private void addEncryptionKeyRequestHeaders(String path, default: return; // no client-provided encryption keys } + + if (encryptionAdapterCreated && encryptionAdapter != null) { + try { + encryptionAdapter.destroy(); + } catch (Exception e) { + throw new IOException( + "Could not destroy encryptionContext: " + e.getMessage()); + } + } + requestHeaders.add(new AbfsHttpHeader(X_MS_ENCRYPTION_KEY, encodedKey)); requestHeaders.add(new AbfsHttpHeader(X_MS_ENCRYPTION_KEY_SHA256, encodedKeySHA256)); requestHeaders.add(new AbfsHttpHeader(X_MS_ENCRYPTION_ALGORITHM, diff --git a/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/services/AbfsInputStream.java b/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/services/AbfsInputStream.java index a94656081adf2..42fbb4088ac1e 100644 --- a/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/services/AbfsInputStream.java +++ b/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/services/AbfsInputStream.java @@ -46,6 +46,8 @@ import org.apache.hadoop.fs.statistics.IOStatistics; import org.apache.hadoop.fs.statistics.IOStatisticsSource; +import javax.security.auth.DestroyFailedException; + import static java.lang.Math.max; import static java.lang.Math.min; @@ -699,8 +701,15 @@ public boolean seekToNewSource(long l) throws IOException { public synchronized void close() throws IOException { LOG.debug("Closing {}", this); closed = true; - buffer = null; // de-reference the buffer so it can be GC'ed sooner - ReadBufferManager.getBufferManager().purgeBuffersForStream(this); + try { + if (encryptionAdapter != null) encryptionAdapter.destroy(); + } catch (Exception e) { + throw new IOException( + "Could not destroy encryptionContext: " + e.getMessage()); + } finally { + buffer = null; // de-reference the buffer so it can be GC'ed sooner + ReadBufferManager.getBufferManager().purgeBuffersForStream(this); + } } /** diff --git a/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/services/AbfsOutputStream.java b/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/services/AbfsOutputStream.java index 49d4a595950cd..0485bff267847 100644 --- a/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/services/AbfsOutputStream.java +++ b/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/services/AbfsOutputStream.java @@ -497,17 +497,24 @@ public synchronized void close() throws IOException { // See HADOOP-16785 throw wrapException(path, e.getMessage(), e); } finally { - if (hasLease()) { - lease.free(); - lease = null; - } - lastError = new IOException(FSExceptionMessages.STREAM_IS_CLOSED); - buffer = null; - bufferIndex = 0; - closed = true; - writeOperations.clear(); - if (hasActiveBlock()) { - clearActiveBlock(); + try { + if (encryptionAdapter != null) encryptionAdapter.destroy(); + } catch (Exception e) { + throw new IOException( + "Could not destroy encryptionContext: " + e.getMessage()); + } finally { + if (hasLease()) { + lease.free(); + lease = null; + } + lastError = new IOException(FSExceptionMessages.STREAM_IS_CLOSED); + buffer = null; + bufferIndex = 0; + closed = true; + writeOperations.clear(); + if (hasActiveBlock()) { + clearActiveBlock(); + } } } LOG.debug("Closing AbfsOutputStream : {}", this); diff --git a/hadoop-tools/pom.xml b/hadoop-tools/pom.xml index 4e934cd101f85..d238bd751cd6d 100644 --- a/hadoop-tools/pom.xml +++ b/hadoop-tools/pom.xml @@ -50,7 +50,7 @@ hadoop-kafka hadoop-azure-datalake hadoop-aliyun - hadoop-fs2img + hadoop-benchmark From f3abbb928e51239ca7ecefb06bf1e899880d09c8 Mon Sep 17 00:00:00 2001 From: Marios Meimaris Date: Thu, 1 Sep 2022 15:31:45 +0200 Subject: [PATCH 38/77] revert unrelated change --- hadoop-tools/pom.xml | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/hadoop-tools/pom.xml b/hadoop-tools/pom.xml index d238bd751cd6d..4e934cd101f85 100644 --- a/hadoop-tools/pom.xml +++ b/hadoop-tools/pom.xml @@ -50,7 +50,7 @@ hadoop-kafka hadoop-azure-datalake hadoop-aliyun - + hadoop-fs2img hadoop-benchmark From a3d31f1edf2707080e76371f814963985210e84f Mon Sep 17 00:00:00 2001 From: Marios Meimaris Date: Fri, 2 Sep 2022 09:07:28 +0200 Subject: [PATCH 39/77] remove unused import --- .../org/apache/hadoop/fs/azurebfs/services/AbfsInputStream.java | 2 -- 1 file changed, 2 deletions(-) diff --git a/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/services/AbfsInputStream.java b/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/services/AbfsInputStream.java index 42fbb4088ac1e..2760e066b6793 100644 --- a/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/services/AbfsInputStream.java +++ b/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/services/AbfsInputStream.java @@ -46,8 +46,6 @@ import org.apache.hadoop.fs.statistics.IOStatistics; import org.apache.hadoop.fs.statistics.IOStatisticsSource; -import javax.security.auth.DestroyFailedException; - import static java.lang.Math.max; import static java.lang.Math.min; From 5842b28edd73d95b38d3ddae2717a99c62f948b1 Mon Sep 17 00:00:00 2001 From: Marios Meimaris Date: Fri, 2 Sep 2022 09:11:00 +0200 Subject: [PATCH 40/77] minor --- .../apache/hadoop/fs/azurebfs/security/EncryptionAdapter.java | 3 ++- .../apache/hadoop/fs/azurebfs/services/AbfsInputStream.java | 4 +++- .../apache/hadoop/fs/azurebfs/services/AbfsOutputStream.java | 4 +++- 3 files changed, 8 insertions(+), 3 deletions(-) diff --git a/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/security/EncryptionAdapter.java b/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/security/EncryptionAdapter.java index 9e269ed1e6160..4fecbca177bd7 100644 --- a/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/security/EncryptionAdapter.java +++ b/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/security/EncryptionAdapter.java @@ -21,6 +21,7 @@ import java.io.IOException; import java.util.Arrays; import java.util.Base64; +import javax.security.auth.DestroyFailedException; import javax.security.auth.Destroyable; import org.apache.hadoop.util.Preconditions; @@ -75,7 +76,7 @@ public String getEncodedContext() throws IOException { return getBase64EncodedString(encryptionContext.getEncoded()); } - public void destroy() { + public void destroy() throws DestroyFailedException { if (encryptionContext != null) encryptionContext.destroy(); if (encryptionKey != null) encryptionKey.destroy(); } diff --git a/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/services/AbfsInputStream.java b/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/services/AbfsInputStream.java index 2760e066b6793..0163689a93b41 100644 --- a/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/services/AbfsInputStream.java +++ b/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/services/AbfsInputStream.java @@ -46,6 +46,8 @@ import org.apache.hadoop.fs.statistics.IOStatistics; import org.apache.hadoop.fs.statistics.IOStatisticsSource; +import javax.security.auth.DestroyFailedException; + import static java.lang.Math.max; import static java.lang.Math.min; @@ -701,7 +703,7 @@ public synchronized void close() throws IOException { closed = true; try { if (encryptionAdapter != null) encryptionAdapter.destroy(); - } catch (Exception e) { + } catch (DestroyFailedException e) { throw new IOException( "Could not destroy encryptionContext: " + e.getMessage()); } finally { diff --git a/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/services/AbfsOutputStream.java b/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/services/AbfsOutputStream.java index 0485bff267847..602c30cbfc95f 100644 --- a/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/services/AbfsOutputStream.java +++ b/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/services/AbfsOutputStream.java @@ -52,6 +52,8 @@ import org.apache.hadoop.fs.StreamCapabilities; import org.apache.hadoop.fs.Syncable; +import javax.security.auth.DestroyFailedException; + import static org.apache.hadoop.fs.azurebfs.constants.FileSystemConfigurations.STREAM_ID_LEN; import static org.apache.hadoop.fs.azurebfs.services.AbfsErrors.ERR_WRITE_WITHOUT_LEASE; import static org.apache.hadoop.fs.impl.StoreImplementationUtils.isProbeForSyncable; @@ -499,7 +501,7 @@ public synchronized void close() throws IOException { } finally { try { if (encryptionAdapter != null) encryptionAdapter.destroy(); - } catch (Exception e) { + } catch (DestroyFailedException e) { throw new IOException( "Could not destroy encryptionContext: " + e.getMessage()); } finally { From 1f1d9d1cffd9ca40f79be4ada6959821275607c7 Mon Sep 17 00:00:00 2001 From: Marios Meimaris Date: Fri, 2 Sep 2022 09:13:26 +0200 Subject: [PATCH 41/77] clean up deps --- .../org/apache/hadoop/fs/azurebfs/services/AbfsClient.java | 6 +----- 1 file changed, 1 insertion(+), 5 deletions(-) diff --git a/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/services/AbfsClient.java b/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/services/AbfsClient.java index bb5fdc64988f1..684fd848a1cd5 100644 --- a/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/services/AbfsClient.java +++ b/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/services/AbfsClient.java @@ -27,7 +27,6 @@ import java.net.URLEncoder; import java.nio.charset.StandardCharsets; import java.util.ArrayList; -import java.util.Base64; import java.util.List; import java.util.Locale; import java.util.UUID; @@ -36,9 +35,7 @@ import java.util.concurrent.TimeUnit; import org.apache.hadoop.classification.VisibleForTesting; -import org.apache.hadoop.fs.azurebfs.security.ABFSKey; import org.apache.hadoop.fs.store.LogExactlyOnce; -import org.apache.hadoop.util.Preconditions; import org.apache.hadoop.fs.Path; import org.apache.hadoop.fs.azurebfs.AzureBlobFileSystemStore.Permissions; import org.apache.hadoop.fs.azurebfs.extensions.EncryptionContextProvider; @@ -72,7 +69,6 @@ import org.apache.hadoop.security.ssl.DelegatingSSLSocketFactory; import org.apache.hadoop.util.concurrent.HadoopExecutors; -import javax.crypto.SecretKey; import javax.security.auth.DestroyFailedException; import static org.apache.commons.lang3.StringUtils.isNotEmpty; @@ -284,7 +280,7 @@ private void addEncryptionKeyRequestHeaders(String path, if (encryptionAdapterCreated && encryptionAdapter != null) { try { encryptionAdapter.destroy(); - } catch (Exception e) { + } catch (DestroyFailedException e) { throw new IOException( "Could not destroy encryptionContext: " + e.getMessage()); } From 4f8622462410a90e67f854229caed72e46715afc Mon Sep 17 00:00:00 2001 From: Marios Meimaris Date: Wed, 7 Sep 2022 16:45:08 +0200 Subject: [PATCH 42/77] fix test --- .../MockEncryptionContextProvider.java | 28 ++++--------------- 1 file changed, 5 insertions(+), 23 deletions(-) diff --git a/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/extensions/MockEncryptionContextProvider.java b/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/extensions/MockEncryptionContextProvider.java index d54487da2a906..72b91f157a5a4 100644 --- a/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/extensions/MockEncryptionContextProvider.java +++ b/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/extensions/MockEncryptionContextProvider.java @@ -47,9 +47,9 @@ public ABFSKey getEncryptionContext(String path) pathToContextMap.put(path, newContext); byte[] newKey = new byte[ENCRYPTION_KEY_LEN]; new Random().nextBytes(newKey); - ABFSKey key = new ABFSKey(newKey); + ABFSKey key = new MockABFSKey(newKey); contextToKeyMap.put(newContext, key); - return new ABFSKey(newContext.getBytes(StandardCharsets.UTF_8)); + return new MockABFSKey(newContext.getBytes(StandardCharsets.UTF_8)); } @Override @@ -71,31 +71,13 @@ public void destroy() { contextToKeyMap = null; } - class Key implements SecretKey { - - private final byte[] key; - - Key(byte[] secret) { - key = secret; - } - @Override - public String getAlgorithm() { - return null; - } - - @Override - public String getFormat() { - return null; - } - - @Override - public byte[] getEncoded() { - return key; + class MockABFSKey extends ABFSKey { + public MockABFSKey(byte[] bytes) { + super(bytes); } @Override public void destroy() { - Arrays.fill(key, (byte) 0); } } From 873b3fe8dae0625c30e19d1e2cd486d66a902376 Mon Sep 17 00:00:00 2001 From: Marios Meimaris Date: Wed, 7 Sep 2022 16:51:32 +0200 Subject: [PATCH 43/77] purge buffers before destroying adapter --- .../org/apache/hadoop/fs/azurebfs/services/AbfsInputStream.java | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/services/AbfsInputStream.java b/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/services/AbfsInputStream.java index 0163689a93b41..1da3859735298 100644 --- a/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/services/AbfsInputStream.java +++ b/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/services/AbfsInputStream.java @@ -701,6 +701,7 @@ public boolean seekToNewSource(long l) throws IOException { public synchronized void close() throws IOException { LOG.debug("Closing {}", this); closed = true; + ReadBufferManager.getBufferManager().purgeBuffersForStream(this); try { if (encryptionAdapter != null) encryptionAdapter.destroy(); } catch (DestroyFailedException e) { @@ -708,7 +709,6 @@ public synchronized void close() throws IOException { "Could not destroy encryptionContext: " + e.getMessage()); } finally { buffer = null; // de-reference the buffer so it can be GC'ed sooner - ReadBufferManager.getBufferManager().purgeBuffersForStream(this); } } From e8c1fb5dc5c4585dc61cb37f68cc9d97d6f4d80e Mon Sep 17 00:00:00 2001 From: Pranav Saxena Date: Thu, 15 Sep 2022 19:16:33 +0530 Subject: [PATCH 44/77] checkstyle --- .../extensions/EncryptionContextProvider.java | 1 - .../hadoop/fs/azurebfs/security/ABFSKey.java | 23 +++++-------------- .../fs/azurebfs/security/EncodingHelper.java | 1 - .../azurebfs/security/EncryptionAdapter.java | 18 +++++++-------- .../fs/azurebfs/services/AbfsInputStream.java | 4 +++- .../azurebfs/services/AbfsOutputStream.java | 4 +++- .../azurebfs/ITestAbfsCustomEncryption.java | 1 - .../MockEncryptionContextProvider.java | 4 +--- 8 files changed, 22 insertions(+), 34 deletions(-) diff --git a/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/extensions/EncryptionContextProvider.java b/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/extensions/EncryptionContextProvider.java index d1c178629b3c0..343f193cbbd60 100644 --- a/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/extensions/EncryptionContextProvider.java +++ b/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/extensions/EncryptionContextProvider.java @@ -18,7 +18,6 @@ package org.apache.hadoop.fs.azurebfs.extensions; -import javax.crypto.SecretKey; import javax.security.auth.Destroyable; import java.io.IOException; diff --git a/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/security/ABFSKey.java b/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/security/ABFSKey.java index ac9001ca66413..e4d0310a2feef 100644 --- a/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/security/ABFSKey.java +++ b/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/security/ABFSKey.java @@ -6,9 +6,9 @@ * 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 - * + *

    + * 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. @@ -19,15 +19,15 @@ package org.apache.hadoop.fs.azurebfs.security; import javax.crypto.SecretKey; -import java.io.IOException; import java.util.Arrays; public class ABFSKey implements SecretKey { private byte[] bytes; private String base64Encoding; private byte[] sha256Hash; + public ABFSKey(byte[] bytes) { - if(bytes != null) { + if (bytes != null) { this.bytes = bytes.clone(); base64Encoding = EncodingHelper.getBase64EncodedString(this.bytes); sha256Hash = EncodingHelper.getSHA256Hash(this.bytes); @@ -50,23 +50,12 @@ public String getFormat() { * */ @Override public byte[] getEncoded() { - if(bytes == null) { + if (bytes == null) { return null; } return bytes.clone(); } - public String getBase64EncodedString() { - return base64Encoding; - } - - public byte[] getSHA256Hash() { - if(sha256Hash == null) { - return null; - } - return sha256Hash.clone(); - } - @Override public void destroy() { Arrays.fill(bytes, (byte) 0); diff --git a/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/security/EncodingHelper.java b/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/security/EncodingHelper.java index c9ee75ecd00e8..9608b7518661b 100644 --- a/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/security/EncodingHelper.java +++ b/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/security/EncodingHelper.java @@ -18,7 +18,6 @@ package org.apache.hadoop.fs.azurebfs.security; -import java.io.IOException; import java.security.MessageDigest; import java.security.NoSuchAlgorithmException; import java.util.Base64; diff --git a/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/security/EncryptionAdapter.java b/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/security/EncryptionAdapter.java index 4fecbca177bd7..5cddfe90fd64e 100644 --- a/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/security/EncryptionAdapter.java +++ b/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/security/EncryptionAdapter.java @@ -63,25 +63,25 @@ private void computeKeys() throws IOException { public String getEncodedKey() throws IOException { computeKeys(); - return getBase64EncodedString(encryptionKey.getEncoded()); + return EncodingHelper.getBase64EncodedString(encryptionKey.getEncoded()); } public String getEncodedKeySHA() throws IOException { computeKeys(); - return getBase64EncodedString(EncodingHelper.getSHA256Hash(encryptionKey.getEncoded())); + return EncodingHelper.getBase64EncodedString(EncodingHelper.getSHA256Hash(encryptionKey.getEncoded())); } public String getEncodedContext() throws IOException { computeKeys(); - return getBase64EncodedString(encryptionContext.getEncoded()); + return EncodingHelper.getBase64EncodedString(encryptionContext.getEncoded()); } public void destroy() throws DestroyFailedException { - if (encryptionContext != null) encryptionContext.destroy(); - if (encryptionKey != null) encryptionKey.destroy(); - } - - public static String getBase64EncodedString(byte[] bytes) { - return Base64.getEncoder().encodeToString(bytes); + if (encryptionContext != null) { + encryptionContext.destroy(); + } + if (encryptionKey != null) { + encryptionKey.destroy(); + } } } diff --git a/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/services/AbfsInputStream.java b/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/services/AbfsInputStream.java index 1da3859735298..dc1a7ec7fe063 100644 --- a/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/services/AbfsInputStream.java +++ b/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/services/AbfsInputStream.java @@ -703,7 +703,9 @@ public synchronized void close() throws IOException { closed = true; ReadBufferManager.getBufferManager().purgeBuffersForStream(this); try { - if (encryptionAdapter != null) encryptionAdapter.destroy(); + if (encryptionAdapter != null) { + encryptionAdapter.destroy(); + } } catch (DestroyFailedException e) { throw new IOException( "Could not destroy encryptionContext: " + e.getMessage()); diff --git a/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/services/AbfsOutputStream.java b/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/services/AbfsOutputStream.java index 602c30cbfc95f..5d155b3d80f72 100644 --- a/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/services/AbfsOutputStream.java +++ b/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/services/AbfsOutputStream.java @@ -500,7 +500,9 @@ public synchronized void close() throws IOException { throw wrapException(path, e.getMessage(), e); } finally { try { - if (encryptionAdapter != null) encryptionAdapter.destroy(); + if (encryptionAdapter != null) { + encryptionAdapter.destroy(); + } } catch (DestroyFailedException e) { throw new IOException( "Could not destroy encryptionContext: " + e.getMessage()); diff --git a/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/ITestAbfsCustomEncryption.java b/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/ITestAbfsCustomEncryption.java index 8d318de415023..73cdc584e6529 100644 --- a/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/ITestAbfsCustomEncryption.java +++ b/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/ITestAbfsCustomEncryption.java @@ -25,7 +25,6 @@ import java.util.Hashtable; import java.util.Random; -import org.apache.hadoop.fs.azurebfs.security.ABFSKey; import org.apache.hadoop.fs.azurebfs.security.EncodingHelper; import org.assertj.core.api.Assertions; import org.junit.Assume; diff --git a/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/extensions/MockEncryptionContextProvider.java b/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/extensions/MockEncryptionContextProvider.java index 72b91f157a5a4..399f21b21f65d 100644 --- a/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/extensions/MockEncryptionContextProvider.java +++ b/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/extensions/MockEncryptionContextProvider.java @@ -18,10 +18,8 @@ package org.apache.hadoop.fs.azurebfs.extensions; -import javax.crypto.SecretKey; import java.io.IOException; import java.nio.charset.StandardCharsets; -import java.util.Arrays; import java.util.HashMap; import java.util.Random; import java.util.UUID; @@ -72,7 +70,7 @@ public void destroy() { } class MockABFSKey extends ABFSKey { - public MockABFSKey(byte[] bytes) { + MockABFSKey(byte[] bytes) { super(bytes); } From ae2d065b4416b8130e721703f919244f2240c3db Mon Sep 17 00:00:00 2001 From: Pranav Saxena Date: Thu, 15 Sep 2022 19:21:49 +0530 Subject: [PATCH 45/77] checkstyle-2 --- .../java/org/apache/hadoop/fs/azurebfs/security/ABFSKey.java | 4 ---- 1 file changed, 4 deletions(-) diff --git a/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/security/ABFSKey.java b/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/security/ABFSKey.java index e4d0310a2feef..2ca2c3af63a01 100644 --- a/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/security/ABFSKey.java +++ b/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/security/ABFSKey.java @@ -23,14 +23,10 @@ public class ABFSKey implements SecretKey { private byte[] bytes; - private String base64Encoding; - private byte[] sha256Hash; public ABFSKey(byte[] bytes) { if (bytes != null) { this.bytes = bytes.clone(); - base64Encoding = EncodingHelper.getBase64EncodedString(this.bytes); - sha256Hash = EncodingHelper.getSHA256Hash(this.bytes); } } From 4f1e33bf20e5fe2f69dcdee1c3a45c7a8d699c9d Mon Sep 17 00:00:00 2001 From: Pranav Saxena Date: Fri, 16 Sep 2022 15:29:51 +0530 Subject: [PATCH 46/77] read in test should have etag --- .../fs/azurebfs/ITestAbfsCustomEncryption.java | 15 +++++++++------ 1 file changed, 9 insertions(+), 6 deletions(-) diff --git a/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/ITestAbfsCustomEncryption.java b/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/ITestAbfsCustomEncryption.java index 73cdc584e6529..d68b82da64419 100644 --- a/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/ITestAbfsCustomEncryption.java +++ b/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/ITestAbfsCustomEncryption.java @@ -20,12 +20,11 @@ import java.io.IOException; import java.nio.charset.StandardCharsets; -import java.util.Arrays; -import java.util.Base64; -import java.util.Hashtable; -import java.util.Random; +import java.util.*; +import org.apache.hadoop.fs.azurebfs.constants.HttpHeaderConfigurations; import org.apache.hadoop.fs.azurebfs.security.EncodingHelper; +import org.apache.hadoop.fs.azurebfs.utils.TracingContext; import org.assertj.core.api.Assertions; import org.junit.Assume; import org.junit.Test; @@ -229,8 +228,12 @@ private AbfsRestOperation callOperation(AzureBlobFileSystem fs, } String path = testPath.toString(); switch (operation) { - case READ: return client.read(path, 0, new byte[5], 0, 5, null, - null, encryptionAdapter, getTestTracingContext(fs, true)); + case READ: + TracingContext tracingContext = getTestTracingContext(fs, true); + AbfsHttpOperation statusOp = client.getPathStatus(path, false, + tracingContext).getResult(); + return client.read(path, 0, new byte[5], 0, 5, statusOp.getResponseHeader(HttpHeaderConfigurations.ETAG), + null, encryptionAdapter, tracingContext); case WRITE: return client.flush(path, 3, false, false, null, null, encryptionAdapter, getTestTracingContext(fs, false)); case APPEND: return client.append(path, "val".getBytes(), From 8806ceb86dedd57bd355ed639234953ab6530359 Mon Sep 17 00:00:00 2001 From: Pranav Saxena Date: Thu, 6 Oct 2022 22:05:11 -0700 Subject: [PATCH 47/77] general code refactor --- .../hadoop/fs/azurebfs/AbfsConfiguration.java | 2 +- .../fs/azurebfs/AzureBlobFileSystemStore.java | 27 +++++++++++-------- .../azurebfs/ITestAbfsCustomEncryption.java | 5 +++- .../fs/azurebfs/TestTracingContext.java | 3 +-- 4 files changed, 22 insertions(+), 15 deletions(-) diff --git a/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/AbfsConfiguration.java b/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/AbfsConfiguration.java index 2c4fe90e47c53..e92f6bb9a4663 100644 --- a/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/AbfsConfiguration.java +++ b/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/AbfsConfiguration.java @@ -21,7 +21,6 @@ import java.io.IOException; import java.lang.reflect.Field; -import org.apache.hadoop.fs.azurebfs.extensions.EncryptionContextProvider; import org.apache.hadoop.classification.VisibleForTesting; import org.apache.hadoop.util.Preconditions; @@ -50,6 +49,7 @@ import org.apache.hadoop.fs.azurebfs.diagnostics.StringConfigurationBasicValidator; import org.apache.hadoop.fs.azurebfs.enums.Trilean; import org.apache.hadoop.fs.azurebfs.extensions.CustomTokenProviderAdaptee; +import org.apache.hadoop.fs.azurebfs.extensions.EncryptionContextProvider; import org.apache.hadoop.fs.azurebfs.extensions.SASTokenProvider; import org.apache.hadoop.fs.azurebfs.oauth2.AccessTokenProvider; import org.apache.hadoop.fs.azurebfs.oauth2.ClientCredsTokenProvider; diff --git a/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/AzureBlobFileSystemStore.java b/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/AzureBlobFileSystemStore.java index fa9e6eef3526d..6d4b93ba2c575 100644 --- a/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/AzureBlobFileSystemStore.java +++ b/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/AzureBlobFileSystemStore.java @@ -54,6 +54,7 @@ import java.util.concurrent.ExecutorService; import java.util.concurrent.TimeUnit; +import org.apache.hadoop.fs.PathIOException; import org.apache.hadoop.fs.azurebfs.extensions.EncryptionContextProvider; import org.apache.hadoop.fs.azurebfs.security.EncryptionAdapter; import org.apache.hadoop.fs.azurebfs.utils.EncryptionType; @@ -612,10 +613,10 @@ public OutputStream createFile(final Path path, */ private AbfsRestOperation conditionalCreateOverwriteFile(final String relativePath, final FileSystem.Statistics statistics, - Permissions permissions, + final Permissions permissions, final boolean isAppendBlob, - EncryptionAdapter encryptionAdapter, - TracingContext tracingContext) throws IOException { + final EncryptionAdapter encryptionAdapter, + final TracingContext tracingContext) throws IOException { AbfsRestOperation op; try { @@ -787,17 +788,21 @@ public AbfsInputStream openFileForRead(Path path, contentLength = Long.parseLong( op.getResponseHeader(HttpHeaderConfigurations.CONTENT_LENGTH)); eTag = op.getResponseHeader(HttpHeaderConfigurations.ETAG); + /** + * For file created with ENCRYPTION_CONTEXT, client shall receive + * encryptionContext from header field: X_MS_ENCRYPTION_CONTEXT. + * */ if (client.getEncryptionType() == EncryptionType.ENCRYPTION_CONTEXT) { - try { - encryptionAdapter = new EncryptionAdapter( - client.getEncryptionContextProvider(), getRelativePath(path), - op.getResponseHeader(HttpHeaderConfigurations.X_MS_ENCRYPTION_CONTEXT) - .getBytes(StandardCharsets.UTF_8)); - } catch (NullPointerException ex) { + final String fileEncryptionContext = op.getResponseHeader( + HttpHeaderConfigurations.X_MS_ENCRYPTION_CONTEXT); + if (fileEncryptionContext == null) { LOG.debug("EncryptionContext missing in GetPathStatus response"); - throw new IOException( - "EncryptionContext not present in GetPathStatus response headers", ex); + throw new PathIOException(path.toString(), + "EncryptionContext not present in GetPathStatus response headers"); } + encryptionAdapter = new EncryptionAdapter( + client.getEncryptionContextProvider(), getRelativePath(path), + fileEncryptionContext.getBytes(StandardCharsets.UTF_8)); } } diff --git a/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/ITestAbfsCustomEncryption.java b/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/ITestAbfsCustomEncryption.java index d68b82da64419..4fbccb1be4575 100644 --- a/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/ITestAbfsCustomEncryption.java +++ b/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/ITestAbfsCustomEncryption.java @@ -20,7 +20,10 @@ import java.io.IOException; import java.nio.charset.StandardCharsets; -import java.util.*; +import java.util.Arrays; +import java.util.Base64; +import java.util.Hashtable; +import java.util.Random; import org.apache.hadoop.fs.azurebfs.constants.HttpHeaderConfigurations; import org.apache.hadoop.fs.azurebfs.security.EncodingHelper; diff --git a/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/TestTracingContext.java b/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/TestTracingContext.java index 19b7801907504..89ca34b4847ad 100644 --- a/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/TestTracingContext.java +++ b/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/TestTracingContext.java @@ -26,7 +26,6 @@ import org.apache.hadoop.conf.Configuration; import org.apache.hadoop.fs.FileSystem; -import org.apache.hadoop.fs.azurebfs.AzureBlobFileSystemStore.Permissions; import org.assertj.core.api.Assertions; import org.junit.Assume; import org.junit.AssumptionViolatedException; @@ -99,7 +98,7 @@ public void checkCorrelationConfigValidation(String clientCorrelationId, TracingHeaderFormat.ALL_ID_FORMAT, null); boolean isNamespaceEnabled = fs.getIsNamespaceEnabled(tracingContext); String path = getRelativePath(new Path("/testDir")); - Permissions permissions = new Permissions(isNamespaceEnabled, + AzureBlobFileSystemStore.Permissions permissions = new AzureBlobFileSystemStore.Permissions(isNamespaceEnabled, FsPermission.getDefault(), FsPermission.getUMask(fs.getConf())); //request should not fail for invalid clientCorrelationID From ef092dd8cc5ceba008f4b938d3d53b59b23d3a13 Mon Sep 17 00:00:00 2001 From: Pranav Saxena Date: Fri, 7 Oct 2022 02:16:23 -0700 Subject: [PATCH 48/77] general refactor --- .../fs/azurebfs/AzureBlobFileSystemStore.java | 40 ++++++++++++++----- .../azurebfs/constants/ConfigurationKeys.java | 2 +- .../extensions/EncryptionContextProvider.java | 11 +++++ .../fs/azurebfs/services/AbfsClient.java | 4 +- 4 files changed, 43 insertions(+), 14 deletions(-) diff --git a/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/AzureBlobFileSystemStore.java b/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/AzureBlobFileSystemStore.java index 6d4b93ba2c575..2136f4acc9a2d 100644 --- a/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/AzureBlobFileSystemStore.java +++ b/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/AzureBlobFileSystemStore.java @@ -819,7 +819,8 @@ public AbfsInputStream openFileForRead(Path path, // Add statistics for InputStream return new AbfsInputStream(client, statistics, relativePath, contentLength, populateAbfsInputStreamContext( - parameters.map(OpenFileParameters::getOptions), encryptionAdapter), + parameters.map(OpenFileParameters::getOptions), + encryptionAdapter), eTag, tracingContext); } } @@ -883,12 +884,16 @@ public OutputStream openFileForWrite(final Path path, AbfsLease lease = maybeCreateLease(relativePath, tracingContext); EncryptionAdapter encryptionAdapter = null; if (client.getEncryptionType() == EncryptionType.ENCRYPTION_CONTEXT) { - byte[] encryptionContext = op.getResult() - .getResponseHeader(HttpHeaderConfigurations.X_MS_ENCRYPTION_CONTEXT) - .getBytes(StandardCharsets.UTF_8); + final String encryptionContext = op.getResult() + .getResponseHeader( + HttpHeaderConfigurations.X_MS_ENCRYPTION_CONTEXT); + if (encryptionContext == null) { + throw new PathIOException(path.toString(), + "File doesn't have encryptionContext."); + } encryptionAdapter = new EncryptionAdapter( client.getEncryptionContextProvider(), getRelativePath(path), - encryptionContext); + encryptionContext.getBytes(StandardCharsets.UTF_8)); } return new AbfsOutputStream( @@ -1659,16 +1664,15 @@ private void initializeClient(URI uri, String fileSystemName, abfsConfiguration.createEncryptionContextProvider(); if (encryptionContextProvider != null) { if (abfsConfiguration.getEncodedClientProvidedEncryptionKey() != null) { - throw new IOException( + throw new PathIOException(uri.getPath(), "Both global key and encryption context are set, only one allowed"); } encryptionContextProvider.initialize( abfsConfiguration.getRawConfiguration(), accountName, fileSystemName); } else if (abfsConfiguration.getEncodedClientProvidedEncryptionKey() != null) { - if (abfsConfiguration.getEncodedClientProvidedEncryptionKeySHA() != null) { - } else { - throw new IOException( + if (abfsConfiguration.getEncodedClientProvidedEncryptionKeySHA() == null) { + throw new PathIOException(uri.getPath(), "Encoded SHA256 hash must be provided for global encryption"); } } @@ -1720,7 +1724,8 @@ private boolean parseIsDirectory(final String resourceType) { && resourceType.equalsIgnoreCase(AbfsHttpConstants.DIRECTORY); } - public String convertXmsPropertiesToCommaSeparatedString(final Hashtable properties) throws CharacterCodingException { StringBuilder commaSeparatedProperties = new StringBuilder(); @@ -1911,7 +1916,12 @@ public String toString() { } } - public static class Permissions { + /** + * Permissions class contain provided permission and umask in octalNotation. + * If the object is created for namespace-disabled, the permission and umask + * would be null. + * */ + public static final class Permissions { private final String permission; private final String umask; @@ -1931,6 +1941,14 @@ private String getOctalNotation(FsPermission fsPermission) { return String.format(AbfsHttpConstants.PERMISSION_FORMAT, fsPermission.toOctal()); } + public Boolean hasPermission() { + return permission != null && !permission.isEmpty(); + } + + public Boolean hasUmask() { + return umask != null && !umask.isEmpty(); + } + public String getPermission() { return permission; } diff --git a/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/constants/ConfigurationKeys.java b/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/constants/ConfigurationKeys.java index c48ed0d97298c..31fa1aa54c2da 100644 --- a/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/constants/ConfigurationKeys.java +++ b/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/constants/ConfigurationKeys.java @@ -188,7 +188,7 @@ public final class ConfigurationKeys { public static final String AZURE_KEY_ACCOUNT_SHELLKEYPROVIDER_SCRIPT = "fs.azure.shellkeyprovider.script"; /** Setting this true will make the driver use it's own RemoteIterator implementation */ public static final String FS_AZURE_ENABLE_ABFS_LIST_ITERATOR = "fs.azure.enable.abfslistiterator"; - /** Server side encryption key encoded in Base6format */ + /** Server side encryption key encoded in Base6format {@value}.*/ public static final String FS_AZURE_ENCRYPTION_ENCODED_CLIENT_PROVIDED_KEY = "fs.azure.encryption.encoded.client-provided-key"; /** SHA256 hash of encryption key encoded in Base64format */ diff --git a/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/extensions/EncryptionContextProvider.java b/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/extensions/EncryptionContextProvider.java index 343f193cbbd60..abd5f1bddc2c5 100644 --- a/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/extensions/EncryptionContextProvider.java +++ b/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/extensions/EncryptionContextProvider.java @@ -24,6 +24,17 @@ import org.apache.hadoop.conf.Configuration; import org.apache.hadoop.fs.azurebfs.security.ABFSKey; +/** + * This interface has two roles:
    + *

      + *
    • + * To create new encryptionContext from a given path: To be used in case of + * create file as there is no encryptionContext in remote server to refer to + * for encryptionKey creation + *
    • + *
    • To create encryptionKey using encryptionContext
    • + *
    + * */ public interface EncryptionContextProvider extends Destroyable { /** * Initialize instance diff --git a/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/services/AbfsClient.java b/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/services/AbfsClient.java index 684fd848a1cd5..cc9ed10ada3b3 100644 --- a/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/services/AbfsClient.java +++ b/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/services/AbfsClient.java @@ -412,13 +412,13 @@ public AbfsRestOperation createPath(final String path, final boolean isFile, requestHeaders.add(new AbfsHttpHeader(IF_NONE_MATCH, AbfsHttpConstants.STAR)); } - if (permissions.getPermission() != null && !permissions.getPermission().isEmpty()) { + if (permissions.hasPermission()) { requestHeaders.add( new AbfsHttpHeader(HttpHeaderConfigurations.X_MS_PERMISSIONS, permissions.getPermission())); } - if (permissions.getUmask() != null && !permissions.getUmask().isEmpty()) { + if (permissions.hasUmask()) { requestHeaders.add(new AbfsHttpHeader(HttpHeaderConfigurations.X_MS_UMASK, permissions.getUmask())); } From fec93f433d49752d7ea01c0d3e07bbd3155eccb2 Mon Sep 17 00:00:00 2001 From: Pranav Saxena Date: Fri, 7 Oct 2022 03:00:24 -0700 Subject: [PATCH 49/77] Made ABFSKey final --- .../extensions/EncryptionContextProvider.java | 2 - .../hadoop/fs/azurebfs/security/ABFSKey.java | 61 ++++++++++--------- .../MockEncryptionContextProvider.java | 31 ++-------- 3 files changed, 39 insertions(+), 55 deletions(-) diff --git a/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/extensions/EncryptionContextProvider.java b/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/extensions/EncryptionContextProvider.java index abd5f1bddc2c5..2c62bcdb3c6dd 100644 --- a/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/extensions/EncryptionContextProvider.java +++ b/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/extensions/EncryptionContextProvider.java @@ -64,6 +64,4 @@ public interface EncryptionContextProvider extends Destroyable { * @throws IOException error in fetching encryption key */ ABFSKey getEncryptionKey(String path, ABFSKey encryptionContext) throws IOException; - - void destroy(); } diff --git a/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/security/ABFSKey.java b/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/security/ABFSKey.java index 2ca2c3af63a01..c3a06ea56625d 100644 --- a/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/security/ABFSKey.java +++ b/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/security/ABFSKey.java @@ -21,39 +21,44 @@ import javax.crypto.SecretKey; import java.util.Arrays; -public class ABFSKey implements SecretKey { - private byte[] bytes; +/** + * Implementation of SecretKey that would be used by EncryptionAdapter object, + * implementations of encryptionContextProvider to maintain the byteArrays of + * encryptionContext and encryptionKey. + * */ +public final class ABFSKey implements SecretKey { + private byte[] bytes; - public ABFSKey(byte[] bytes) { - if (bytes != null) { - this.bytes = bytes.clone(); - } + public ABFSKey(byte[] bytes) { + if (bytes != null) { + this.bytes = bytes.clone(); } + } - @Override - public String getAlgorithm() { - return null; - } + @Override + public String getAlgorithm() { + return null; + } - @Override - public String getFormat() { - return null; - } + @Override + public String getFormat() { + return null; + } - /** - * This method to be called by implementations of EncryptionContextProvider interface. - * Method returns clone of the original bytes array to prevent findbugs flags. - * */ - @Override - public byte[] getEncoded() { - if (bytes == null) { - return null; - } - return bytes.clone(); + /** + * This method to be called by implementations of EncryptionContextProvider interface. + * Method returns clone of the original bytes array to prevent findbugs flags. + * */ + @Override + public byte[] getEncoded() { + if (bytes == null) { + return null; } + return bytes.clone(); + } - @Override - public void destroy() { - Arrays.fill(bytes, (byte) 0); - } + @Override + public void destroy() { + Arrays.fill(bytes, (byte) 0); + } } diff --git a/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/extensions/MockEncryptionContextProvider.java b/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/extensions/MockEncryptionContextProvider.java index 399f21b21f65d..ff388411e0b4d 100644 --- a/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/extensions/MockEncryptionContextProvider.java +++ b/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/extensions/MockEncryptionContextProvider.java @@ -32,7 +32,7 @@ public class MockEncryptionContextProvider implements EncryptionContextProvider { private HashMap pathToContextMap = new HashMap<>(); - private HashMap contextToKeyMap = new HashMap<>(); + private HashMap contextToKeyByteMap = new HashMap<>(); @Override public void initialize(Configuration configuration, String accountName, String fileSystem) throws IOException { @@ -45,9 +45,9 @@ public ABFSKey getEncryptionContext(String path) pathToContextMap.put(path, newContext); byte[] newKey = new byte[ENCRYPTION_KEY_LEN]; new Random().nextBytes(newKey); - ABFSKey key = new MockABFSKey(newKey); - contextToKeyMap.put(newContext, key); - return new MockABFSKey(newContext.getBytes(StandardCharsets.UTF_8)); + ABFSKey key = new ABFSKey(newKey); + contextToKeyByteMap.put(newContext, key.getEncoded()); + return new ABFSKey(newContext.getBytes(StandardCharsets.UTF_8)); } @Override @@ -57,31 +57,12 @@ public ABFSKey getEncryptionKey(String path, ABFSKey encryptionContext) throws I if (!encryptionContextString.equals(pathToContextMap.get(path))) { throw new IOException("encryption context does not match path"); } - return contextToKeyMap.get(encryptionContextString); - } - - @Override - public void destroy() { - pathToContextMap = null; - for (ABFSKey encryptionKey : contextToKeyMap.values()) { - encryptionKey.destroy(); - } - contextToKeyMap = null; - } - - class MockABFSKey extends ABFSKey { - MockABFSKey(byte[] bytes) { - super(bytes); - } - - @Override - public void destroy() { - } + return new ABFSKey(contextToKeyByteMap.get(encryptionContextString)); } @VisibleForTesting public byte[] getEncryptionKeyForTest(String encryptionContext) { - return contextToKeyMap.get(encryptionContext).getEncoded(); + return contextToKeyByteMap.get(encryptionContext); } @VisibleForTesting From c2d35d8a778c280656ec6d4153eb41fa021a697c Mon Sep 17 00:00:00 2001 From: Pranav Saxena Date: Fri, 7 Oct 2022 03:46:43 -0700 Subject: [PATCH 50/77] added javadocs --- .../fs/azurebfs/AzureBlobFileSystemStore.java | 7 +++- .../fs/azurebfs/security/EncodingHelper.java | 32 ++++++++------- .../azurebfs/security/EncryptionAdapter.java | 41 +++++++++++++++---- 3 files changed, 55 insertions(+), 25 deletions(-) diff --git a/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/AzureBlobFileSystemStore.java b/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/AzureBlobFileSystemStore.java index 2136f4acc9a2d..9d83064244638 100644 --- a/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/AzureBlobFileSystemStore.java +++ b/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/AzureBlobFileSystemStore.java @@ -562,8 +562,11 @@ public OutputStream createFile(final Path path, triggerConditionalCreateOverwrite = true; } - EncryptionAdapter encryptionAdapter = new EncryptionAdapter( - client.getEncryptionContextProvider(), getRelativePath(path)); + EncryptionAdapter encryptionAdapter = null; + if(client.getEncryptionType() == EncryptionType.ENCRYPTION_CONTEXT) { + encryptionAdapter = new EncryptionAdapter( + client.getEncryptionContextProvider(), getRelativePath(path)); + } AbfsRestOperation op; if (triggerConditionalCreateOverwrite) { op = conditionalCreateOverwriteFile(relativePath, diff --git a/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/security/EncodingHelper.java b/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/security/EncodingHelper.java index 9608b7518661b..1e2ec3948d487 100644 --- a/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/security/EncodingHelper.java +++ b/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/security/EncodingHelper.java @@ -22,22 +22,24 @@ import java.security.NoSuchAlgorithmException; import java.util.Base64; -public class EncodingHelper { +/** + * Utility class for managing encryption of bytes or base64String conversion of + * bytes. + * */ +public final class EncodingHelper { - public static byte[] getSHA256Hash(byte[] key) { - try { - final MessageDigest digester = MessageDigest.getInstance("SHA-256"); - return digester.digest(key); - } catch (NoSuchAlgorithmException ignored) { - /** - * This exception can be ignored. Reason being SHA-256 is a valid algorithm, and it is constant for all - * method calls. - */ - return null; - } + public static byte[] getSHA256Hash(byte[] key) { + try { + final MessageDigest digester = MessageDigest.getInstance("SHA-256"); + return digester.digest(key); + } catch (NoSuchAlgorithmException ignored) { + /*This exception can be ignored. Reason being SHA-256 is a valid algorithm, + and it is constant for all method calls.*/ + throw new RuntimeException("SHA-256 algorithm not found in MessageDigest", ignored); } + } - public static String getBase64EncodedString(byte[] bytes) { - return Base64.getEncoder().encodeToString(bytes); - } + public static String getBase64EncodedString(byte[] bytes) { + return Base64.getEncoder().encodeToString(bytes); + } } diff --git a/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/security/EncryptionAdapter.java b/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/security/EncryptionAdapter.java index 5cddfe90fd64e..3c6d2e8fe18fa 100644 --- a/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/security/EncryptionAdapter.java +++ b/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/security/EncryptionAdapter.java @@ -18,35 +18,63 @@ package org.apache.hadoop.fs.azurebfs.security; -import java.io.IOException; -import java.util.Arrays; -import java.util.Base64; + import javax.security.auth.DestroyFailedException; import javax.security.auth.Destroyable; -import org.apache.hadoop.util.Preconditions; +import java.io.IOException; +import java.util.Arrays; +import java.util.Base64; import org.apache.hadoop.fs.azurebfs.extensions.EncryptionContextProvider; +import org.apache.hadoop.fs.azurebfs.services.AbfsRestOperationType; +import org.apache.hadoop.util.Preconditions; +/** + * Class manages the encryptionContext and encryptionKey that needs to be added + * to the headers to server request if Customer-Encryption-Context is enabled in + * the configuration. + *
    + * For fileCreation, the object helps in creating encryptionContext. + *
    + * For all operations, the object helps in converting encryptionContext to + * encryptionKey through the implementation of EncryptionContextProvider. + * */ public class EncryptionAdapter implements Destroyable { private final String path; private ABFSKey encryptionContext; private ABFSKey encryptionKey; private final EncryptionContextProvider provider; + /** + * Following constructor called when the encryptionContext of file is known. + * The server shall send encryptionContext as a String, the constructor shall + * convert the string into a byte-array. The converted byte-array would be used + * by the implementation of EncryptionContextProvider to create byte-array of + * encryptionKey. + * */ public EncryptionAdapter(EncryptionContextProvider provider, String path, byte[] encryptionContext) throws IOException { - this(provider, path); + this.provider = provider; + this.path = path; Preconditions.checkNotNull(encryptionContext, "Encryption context should not be null."); this.encryptionContext = new ABFSKey(Base64.getDecoder().decode(encryptionContext)); Arrays.fill(encryptionContext, (byte) 0); + computeKeys(); } + /** + * Following constructor called in case of createPath. Since, the path is not + * on the server, encryptionContext is not there for the path. Implementation + * of the EncryptionContextProvider would be used to create encryptionContext + * from the path. + * */ public EncryptionAdapter(EncryptionContextProvider provider, String path) throws IOException { this.provider = provider; this.path = path; + computeKeys(); } private void computeKeys() throws IOException { @@ -62,17 +90,14 @@ private void computeKeys() throws IOException { } public String getEncodedKey() throws IOException { - computeKeys(); return EncodingHelper.getBase64EncodedString(encryptionKey.getEncoded()); } public String getEncodedKeySHA() throws IOException { - computeKeys(); return EncodingHelper.getBase64EncodedString(EncodingHelper.getSHA256Hash(encryptionKey.getEncoded())); } public String getEncodedContext() throws IOException { - computeKeys(); return EncodingHelper.getBase64EncodedString(encryptionContext.getEncoded()); } From 1e987a55c09c51d9064e8606770160d41d940ac9 Mon Sep 17 00:00:00 2001 From: Pranav Saxena Date: Fri, 7 Oct 2022 05:52:22 -0700 Subject: [PATCH 51/77] refactors --- .../azurebfs/constants/AbfsHttpConstants.java | 2 + .../azurebfs/security/EncryptionAdapter.java | 29 ++++------ .../fs/azurebfs/services/AbfsClient.java | 18 ++---- .../fs/azurebfs/services/AbfsInputStream.java | 19 ++---- .../azurebfs/services/AbfsOutputStream.java | 35 +++++------ .../fs/azurebfs/utils/EncryptionType.java | 11 ++-- .../azurebfs/ITestAbfsCustomEncryption.java | 58 ++++++++++++++----- .../ITestAzureBlobFileSystemCreate.java | 5 +- .../MockEncryptionContextProvider.java | 3 - 9 files changed, 90 insertions(+), 90 deletions(-) diff --git a/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/constants/AbfsHttpConstants.java b/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/constants/AbfsHttpConstants.java index 5cf7ec565b59e..1451c80b3dc8a 100644 --- a/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/constants/AbfsHttpConstants.java +++ b/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/constants/AbfsHttpConstants.java @@ -111,6 +111,8 @@ public final class AbfsHttpConstants { public static final char CHAR_EQUALS = '='; public static final char CHAR_STAR = '*'; public static final char CHAR_PLUS = '+'; + public static final String DECEMBER_2019_API_VERSION = "2019-12-12"; + public static final String APRIL_2021_API_VERSION = "2021-04-10"; private AbfsHttpConstants() {} } diff --git a/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/security/EncryptionAdapter.java b/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/security/EncryptionAdapter.java index 3c6d2e8fe18fa..981889bcec3cd 100644 --- a/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/security/EncryptionAdapter.java +++ b/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/security/EncryptionAdapter.java @@ -18,17 +18,12 @@ package org.apache.hadoop.fs.azurebfs.security; - -import javax.security.auth.DestroyFailedException; -import javax.security.auth.Destroyable; - import java.io.IOException; import java.util.Arrays; import java.util.Base64; +import java.util.Objects; import org.apache.hadoop.fs.azurebfs.extensions.EncryptionContextProvider; -import org.apache.hadoop.fs.azurebfs.services.AbfsRestOperationType; -import org.apache.hadoop.util.Preconditions; /** * Class manages the encryptionContext and encryptionKey that needs to be added @@ -40,9 +35,9 @@ * For all operations, the object helps in converting encryptionContext to * encryptionKey through the implementation of EncryptionContextProvider. * */ -public class EncryptionAdapter implements Destroyable { +public class EncryptionAdapter { private final String path; - private ABFSKey encryptionContext; + private final ABFSKey encryptionContext; private ABFSKey encryptionKey; private final EncryptionContextProvider provider; @@ -57,7 +52,7 @@ public EncryptionAdapter(EncryptionContextProvider provider, String path, byte[] encryptionContext) throws IOException { this.provider = provider; this.path = path; - Preconditions.checkNotNull(encryptionContext, + Objects.requireNonNull(encryptionContext, "Encryption context should not be null."); this.encryptionContext = new ABFSKey(Base64.getDecoder().decode(encryptionContext)); Arrays.fill(encryptionContext, (byte) 0); @@ -74,19 +69,15 @@ public EncryptionAdapter(EncryptionContextProvider provider, String path) throws IOException { this.provider = provider; this.path = path; + encryptionContext = provider.getEncryptionContext(path); + Objects.requireNonNull(encryptionContext, + "Encryption context should not be null."); computeKeys(); } private void computeKeys() throws IOException { - if (encryptionContext == null) { - encryptionContext = provider.getEncryptionContext(path); - } - Preconditions.checkNotNull(encryptionContext, - "Encryption context should not be null."); - if (encryptionKey == null) { - encryptionKey = provider.getEncryptionKey(path, encryptionContext); - } - Preconditions.checkNotNull(encryptionKey, "Encryption key should not be null."); + encryptionKey = provider.getEncryptionKey(path, encryptionContext); + Objects.requireNonNull(encryptionKey, "Encryption key should not be null."); } public String getEncodedKey() throws IOException { @@ -101,7 +92,7 @@ public String getEncodedContext() throws IOException { return EncodingHelper.getBase64EncodedString(encryptionContext.getEncoded()); } - public void destroy() throws DestroyFailedException { + public void destroy() { if (encryptionContext != null) { encryptionContext.destroy(); } diff --git a/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/services/AbfsClient.java b/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/services/AbfsClient.java index cc9ed10ada3b3..d5e46fc1be95e 100644 --- a/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/services/AbfsClient.java +++ b/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/services/AbfsClient.java @@ -36,11 +36,11 @@ import org.apache.hadoop.classification.VisibleForTesting; import org.apache.hadoop.fs.store.LogExactlyOnce; -import org.apache.hadoop.fs.Path; import org.apache.hadoop.fs.azurebfs.AzureBlobFileSystemStore.Permissions; import org.apache.hadoop.fs.azurebfs.extensions.EncryptionContextProvider; import org.apache.hadoop.fs.azurebfs.security.EncryptionAdapter; import org.apache.hadoop.fs.azurebfs.utils.EncryptionType; +import org.apache.hadoop.io.IOUtils; import org.apache.hadoop.thirdparty.com.google.common.base.Strings; import org.apache.hadoop.thirdparty.com.google.common.util.concurrent.FutureCallback; import org.apache.hadoop.thirdparty.com.google.common.util.concurrent.Futures; @@ -66,11 +66,10 @@ import org.apache.hadoop.fs.azurebfs.contracts.services.AppendRequestParameters; import org.apache.hadoop.fs.azurebfs.oauth2.AccessTokenProvider; import org.apache.hadoop.fs.azurebfs.utils.TracingContext; +import org.apache.hadoop.fs.Path; import org.apache.hadoop.security.ssl.DelegatingSSLSocketFactory; import org.apache.hadoop.util.concurrent.HadoopExecutors; -import javax.security.auth.DestroyFailedException; - import static org.apache.commons.lang3.StringUtils.isNotEmpty; import static org.apache.hadoop.fs.azurebfs.AbfsStatistic.RENAME_PATH_ATTEMPTS; import static org.apache.hadoop.fs.azurebfs.AzureBlobFileSystemStore.extractEtagHeader; @@ -90,7 +89,7 @@ public class AbfsClient implements Closeable { private final URL baseUrl; private final SharedKeyCredentials sharedKeyCredentials; - private String xMsVersion = "2019-12-12"; + private String xMsVersion = DECEMBER_2019_API_VERSION; private final ExponentialRetryPolicy retryPolicy; private final String filesystem; private final AbfsConfiguration abfsConfiguration; @@ -130,7 +129,7 @@ private AbfsClient(final URL baseUrl, if (encryptionContextProvider != null) { this.encryptionContextProvider = encryptionContextProvider; - xMsVersion = "2021-04-10"; // will be default once server change deployed + xMsVersion = APRIL_2021_API_VERSION; // will be default once server change deployed encryptionType = EncryptionType.ENCRYPTION_CONTEXT; } else if (abfsConfiguration.getEncodedClientProvidedEncryptionKey() != null) { clientProvidedEncryptionKey = @@ -190,7 +189,7 @@ public AbfsClient(final URL baseUrl, final SharedKeyCredentials sharedKeyCredent @Override public void close() throws IOException { if (tokenProvider instanceof Closeable) { - org.apache.hadoop.io.IOUtils.cleanupWithLogger(LOG, + IOUtils.cleanupWithLogger(LOG, (Closeable) tokenProvider); } HadoopExecutors.shutdown(executorService, LOG, 0, TimeUnit.SECONDS); @@ -278,12 +277,7 @@ private void addEncryptionKeyRequestHeaders(String path, } if (encryptionAdapterCreated && encryptionAdapter != null) { - try { - encryptionAdapter.destroy(); - } catch (DestroyFailedException e) { - throw new IOException( - "Could not destroy encryptionContext: " + e.getMessage()); - } + encryptionAdapter.destroy(); } requestHeaders.add(new AbfsHttpHeader(X_MS_ENCRYPTION_KEY, encodedKey)); diff --git a/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/services/AbfsInputStream.java b/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/services/AbfsInputStream.java index dc1a7ec7fe063..8140369cece3a 100644 --- a/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/services/AbfsInputStream.java +++ b/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/services/AbfsInputStream.java @@ -18,6 +18,7 @@ package org.apache.hadoop.fs.azurebfs.services; +import javax.security.auth.DestroyFailedException; import java.io.EOFException; import java.io.FileNotFoundException; import java.io.IOException; @@ -25,8 +26,8 @@ import java.util.UUID; import org.apache.commons.lang3.StringUtils; -import org.apache.hadoop.fs.azurebfs.security.EncryptionAdapter; import org.apache.hadoop.classification.VisibleForTesting; +import org.apache.hadoop.fs.azurebfs.security.EncryptionAdapter; import org.apache.hadoop.util.Preconditions; import org.slf4j.Logger; @@ -46,8 +47,6 @@ import org.apache.hadoop.fs.statistics.IOStatistics; import org.apache.hadoop.fs.statistics.IOStatisticsSource; -import javax.security.auth.DestroyFailedException; - import static java.lang.Math.max; import static java.lang.Math.min; @@ -702,16 +701,10 @@ public synchronized void close() throws IOException { LOG.debug("Closing {}", this); closed = true; ReadBufferManager.getBufferManager().purgeBuffersForStream(this); - try { - if (encryptionAdapter != null) { - encryptionAdapter.destroy(); - } - } catch (DestroyFailedException e) { - throw new IOException( - "Could not destroy encryptionContext: " + e.getMessage()); - } finally { - buffer = null; // de-reference the buffer so it can be GC'ed sooner - } + buffer = null; // de-reference the buffer so it can be GC'ed sooner + if (encryptionAdapter != null) { + encryptionAdapter.destroy(); + } } /** diff --git a/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/services/AbfsOutputStream.java b/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/services/AbfsOutputStream.java index 5d155b3d80f72..7625c254529b2 100644 --- a/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/services/AbfsOutputStream.java +++ b/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/services/AbfsOutputStream.java @@ -499,26 +499,21 @@ public synchronized void close() throws IOException { // See HADOOP-16785 throw wrapException(path, e.getMessage(), e); } finally { - try { - if (encryptionAdapter != null) { - encryptionAdapter.destroy(); - } - } catch (DestroyFailedException e) { - throw new IOException( - "Could not destroy encryptionContext: " + e.getMessage()); - } finally { - if (hasLease()) { - lease.free(); - lease = null; - } - lastError = new IOException(FSExceptionMessages.STREAM_IS_CLOSED); - buffer = null; - bufferIndex = 0; - closed = true; - writeOperations.clear(); - if (hasActiveBlock()) { - clearActiveBlock(); - } + if (encryptionAdapter != null) { + encryptionAdapter.destroy(); + } + + if (hasLease()) { + lease.free(); + lease = null; + } + lastError = new IOException(FSExceptionMessages.STREAM_IS_CLOSED); + buffer = null; + bufferIndex = 0; + closed = true; + writeOperations.clear(); + if (hasActiveBlock()) { + clearActiveBlock(); } } LOG.debug("Closing AbfsOutputStream : {}", this); diff --git a/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/utils/EncryptionType.java b/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/utils/EncryptionType.java index f7a06e1487c17..880dc8fc5a279 100644 --- a/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/utils/EncryptionType.java +++ b/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/utils/EncryptionType.java @@ -19,10 +19,13 @@ package org.apache.hadoop.fs.azurebfs.utils; /** - * Enum EncryptionType to represent the level of encryption applied + * Enum EncryptionType to represent the level of encryption applied. + * GLOBAL_KEY: encrypt all files with the same client-provided key + * ENCRYPTION_CONTEXT: uses client-provided implementation to generate keys + * NONE: encryption handled entirely at server */ public enum EncryptionType { - GLOBAL_KEY, // encrypt all files with the same client-provided key - ENCRYPTION_CONTEXT, // uses client-provided implementation to generate keys - NONE // encryption handled entirely at server + GLOBAL_KEY, + ENCRYPTION_CONTEXT, + NONE } diff --git a/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/ITestAbfsCustomEncryption.java b/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/ITestAbfsCustomEncryption.java index 4fbccb1be4575..e7474be9e84c4 100644 --- a/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/ITestAbfsCustomEncryption.java +++ b/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/ITestAbfsCustomEncryption.java @@ -20,9 +20,11 @@ import java.io.IOException; import java.nio.charset.StandardCharsets; +import java.util.ArrayList; import java.util.Arrays; import java.util.Base64; import java.util.Hashtable; +import java.util.List; import java.util.Random; import org.apache.hadoop.fs.azurebfs.constants.HttpHeaderConfigurations; @@ -75,6 +77,8 @@ public class ITestAbfsCustomEncryption extends AbstractAbfsIntegrationTest { private final byte[] cpk = new byte[ENCRYPTION_KEY_LEN]; private final String cpkSHAEncoded; + private List fileSystemsOpenedInTest = new ArrayList<>(); + // Encryption type used by filesystem while creating file @Parameterized.Parameter public EncryptionType fileEncryptionType; @@ -144,7 +148,6 @@ public static Iterable params() { } public ITestAbfsCustomEncryption() throws Exception { - super(); Assume.assumeTrue("Account should be HNS enabled for CPK", getConfiguration().getBoolean(FS_AZURE_TEST_NAMESPACE_ENABLED_ACCOUNT, false)); @@ -155,7 +158,7 @@ public ITestAbfsCustomEncryption() throws Exception { @Test public void testCustomEncryptionCombinations() throws Exception { - AzureBlobFileSystem fs = getFS(); + AzureBlobFileSystem fs = getOrCreateFS(); Path testPath = path("/testFile"); String relativePath = fs.getAbfsStore().getRelativePath(testPath); MockEncryptionContextProvider ecp = @@ -216,6 +219,7 @@ private AbfsRestOperation callOperation(AzureBlobFileSystem fs, case SET_ATTR: fs.setXAttr(testPath, "attribute", "value".getBytes()); break; case GET_ATTR: fs.getXAttr(testPath, "attribute"); + break; default: throw new NoSuchFieldException(); } }); @@ -237,21 +241,28 @@ private AbfsRestOperation callOperation(AzureBlobFileSystem fs, tracingContext).getResult(); return client.read(path, 0, new byte[5], 0, 5, statusOp.getResponseHeader(HttpHeaderConfigurations.ETAG), null, encryptionAdapter, tracingContext); - case WRITE: return client.flush(path, 3, false, false, null, + case WRITE: + return client.flush(path, 3, false, false, null, null, encryptionAdapter, getTestTracingContext(fs, false)); - case APPEND: return client.append(path, "val".getBytes(), + case APPEND: + return client.append(path, "val".getBytes(), new AppendRequestParameters(3, 0, 3, APPEND_MODE, false, null), null, encryptionAdapter, getTestTracingContext(fs, false)); - case SET_ACL: return client.setAcl(path, AclEntry.aclSpecToString( + case SET_ACL: + return client.setAcl(path, AclEntry.aclSpecToString( Lists.newArrayList(aclEntry(ACCESS, USER, ALL))), getTestTracingContext(fs, false)); - case LISTSTATUS: return client.listPath(path, false, 5, null, + case LISTSTATUS: + return client.listPath(path, false, 5, null, getTestTracingContext(fs, true)); - case RENAME: return client.renamePath(path, new Path(path + "_2").toString(), + case RENAME: + return client.renamePath(path, new Path(path + "_2").toString(), null, getTestTracingContext(fs, true), null, false).getOp(); - case DELETE: return client.deletePath(path, false, null, + case DELETE: + return client.deletePath(path, false, null, getTestTracingContext(fs, false)); - case GET_ATTR: return client.getPathStatus(path, true, + case GET_ATTR: + return client.getPathStatus(path, true, getTestTracingContext(fs, false)); case SET_ATTR: Hashtable properties = new Hashtable<>(); @@ -275,7 +286,9 @@ private AzureBlobFileSystem getECProviderEnabledFS() throws Exception { + getAccountName()); configuration.unset(FS_AZURE_ENCRYPTION_ENCODED_CLIENT_PROVIDED_KEY_SHA + "." + getAccountName()); - return (AzureBlobFileSystem) FileSystem.newInstance(configuration); + AzureBlobFileSystem fs = (AzureBlobFileSystem) FileSystem.newInstance(configuration); + fileSystemsOpenedInTest.add(fs); + return fs; } private AzureBlobFileSystem getCPKEnabledFS() throws IOException { @@ -288,10 +301,12 @@ private AzureBlobFileSystem getCPKEnabledFS() throws IOException { conf.set(FS_AZURE_ENCRYPTION_ENCODED_CLIENT_PROVIDED_KEY_SHA + "." + getAccountName(), cpkEncodedSHA); conf.unset(FS_AZURE_ENCRYPTION_CONTEXT_PROVIDER_TYPE); - return (AzureBlobFileSystem) FileSystem.newInstance(conf); + AzureBlobFileSystem fs = (AzureBlobFileSystem) FileSystem.newInstance(conf); + fileSystemsOpenedInTest.add(fs); + return fs; } - private AzureBlobFileSystem getFS() throws Exception { + private AzureBlobFileSystem getOrCreateFS() throws Exception { if (getFileSystem().getAbfsClient().getEncryptionType() == requestEncryptionType) { return getFileSystem(); } @@ -302,7 +317,9 @@ private AzureBlobFileSystem getFS() throws Exception { } else { Configuration conf = getRawConfiguration(); conf.unset(FS_AZURE_ENCRYPTION_CONTEXT_PROVIDER_TYPE); - return (AzureBlobFileSystem) FileSystem.newInstance(conf); + AzureBlobFileSystem fs = (AzureBlobFileSystem) FileSystem.newInstance(conf); + fileSystemsOpenedInTest.add(fs); + return fs; } } @@ -322,12 +339,21 @@ private EncryptionContextProvider createEncryptedFile(Path testPath) throws Exce // verify file is encrypted by calling getPathStatus (with properties) // without encryption headers in request if (fileEncryptionType != EncryptionType.NONE) { - fs.getAbfsClient().setEncryptionType(EncryptionType.NONE); - LambdaTestUtils.intercept(IOException.class, () -> fs.getAbfsClient() - .getPathStatus(fs.getAbfsStore().getRelativePath(testPath), true, + final AbfsClient abfsClient = fs.getAbfsClient(); + abfsClient.setEncryptionType(EncryptionType.NONE); + LambdaTestUtils.intercept(IOException.class, () -> + abfsClient.getPathStatus(fs.getAbfsStore().getRelativePath(testPath), + true, getTestTracingContext(fs, false))); fs.getAbfsClient().setEncryptionType(fileEncryptionType); } return fs.getAbfsClient().getEncryptionContextProvider(); } + + @Override + protected void finalize() throws Throwable { + for(AzureBlobFileSystem azureBlobFileSystem : fileSystemsOpenedInTest) { + azureBlobFileSystem.close(); + } + } } diff --git a/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/ITestAzureBlobFileSystemCreate.java b/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/ITestAzureBlobFileSystemCreate.java index 44e6a0f08d0dd..4a103d2c0c053 100644 --- a/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/ITestAzureBlobFileSystemCreate.java +++ b/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/ITestAzureBlobFileSystemCreate.java @@ -25,7 +25,6 @@ import java.util.EnumSet; import java.util.UUID; -import org.apache.hadoop.fs.azurebfs.AzureBlobFileSystemStore.Permissions; import org.junit.Test; import org.apache.hadoop.conf.Configuration; @@ -399,7 +398,7 @@ public void testNegativeScenariosForCreateOverwriteDisabled() serverErrorResponseEx) // Scn5: create overwrite=false fails with Http500 .when(mockClient) .createPath(any(String.class), eq(true), eq(false), - any(Permissions.class), any(boolean.class), eq(null), any(), + any(AzureBlobFileSystemStore.Permissions.class), any(boolean.class), eq(null), any(), any(TracingContext.class)); doThrow(fileNotFoundResponseEx) // Scn1: GFS fails with Http404 @@ -416,7 +415,7 @@ public void testNegativeScenariosForCreateOverwriteDisabled() serverErrorResponseEx) // Scn4: create overwrite=true fails with Http500 .when(mockClient) .createPath(any(String.class), eq(true), eq(true), - any(Permissions.class), any(boolean.class), eq(null), any(), + any(AzureBlobFileSystemStore.Permissions.class), any(boolean.class), eq(null), any(), any(TracingContext.class)); // Scn1: GFS fails with Http404 diff --git a/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/extensions/MockEncryptionContextProvider.java b/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/extensions/MockEncryptionContextProvider.java index ff388411e0b4d..2e94d2648eba5 100644 --- a/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/extensions/MockEncryptionContextProvider.java +++ b/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/extensions/MockEncryptionContextProvider.java @@ -25,7 +25,6 @@ import java.util.UUID; import org.apache.hadoop.conf.Configuration; -import org.apache.hadoop.classification.VisibleForTesting; import org.apache.hadoop.fs.azurebfs.security.ABFSKey; import static org.apache.hadoop.fs.azurebfs.constants.TestConfigurationKeys.ENCRYPTION_KEY_LEN; @@ -60,12 +59,10 @@ public ABFSKey getEncryptionKey(String path, ABFSKey encryptionContext) throws I return new ABFSKey(contextToKeyByteMap.get(encryptionContextString)); } - @VisibleForTesting public byte[] getEncryptionKeyForTest(String encryptionContext) { return contextToKeyByteMap.get(encryptionContext); } - @VisibleForTesting public String getEncryptionContextForTest(String path) { return pathToContextMap.get(path); } From 6364cfb7ad87fba04e8fb5a6fecd25bfffcaf67c Mon Sep 17 00:00:00 2001 From: Pranav Saxena Date: Sun, 9 Oct 2022 21:19:23 -0700 Subject: [PATCH 52/77] added javadocs. --- .../fs/azurebfs/services/AbfsClient.java | 56 ++++++++++++++++--- 1 file changed, 48 insertions(+), 8 deletions(-) diff --git a/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/services/AbfsClient.java b/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/services/AbfsClient.java index d5e46fc1be95e..2673ff8bfc015 100644 --- a/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/services/AbfsClient.java +++ b/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/services/AbfsClient.java @@ -230,6 +230,23 @@ List createDefaultHeaders() { return requestHeaders; } + /** + * This method adds following headers: + *
      + *
    1. X_MS_ENCRYPTION_KEY
    2. + *
    3. X_MS_ENCRYPTION_KEY_SHA256
    4. + *
    5. X_MS_ENCRYPTION_ALGORITHM
    6. + *
    + * Above headers have to be added in following operations: + *
      + *
    1. createPath
    2. + *
    3. append
    4. + *
    5. flush
    6. + *
    7. setPathProperties
    8. + *
    9. getPathStatus for fs.setXAttr and fs.getXAttr
    10. + *
    11. read
    12. + *
    + * */ private void addEncryptionKeyRequestHeaders(String path, List requestHeaders, boolean isCreateFileRequest, EncryptionAdapter encryptionAdapter, TracingContext tracingContext) @@ -250,15 +267,16 @@ private void addEncryptionKeyRequestHeaders(String path, } else if (encryptionAdapter == null) { // get encryption context from GetPathStatus response header byte[] encryptionContext; - try { - encryptionContext = getPathStatus(path, false, tracingContext) - .getResult().getResponseHeader(X_MS_ENCRYPTION_CONTEXT) - .getBytes(StandardCharsets.UTF_8); - } catch (NullPointerException e) { - LOG.debug("EncryptionContext not present in GetPathStatus response."); + final String responseHeaderEncryptionContext = getPathStatus(path, + false, tracingContext).getResult() + .getResponseHeader(X_MS_ENCRYPTION_CONTEXT); + if (responseHeaderEncryptionContext == null) { throw new IOException( - "EncryptionContext not present in GetPathStatus response", e); + "EncryptionContext not present in GetPathStatus response"); } + encryptionContext = responseHeaderEncryptionContext.getBytes( + StandardCharsets.UTF_8); + try { encryptionAdapter = new EncryptionAdapter(encryptionContextProvider, new Path(path).toUri().getPath(), encryptionContext); @@ -392,10 +410,32 @@ public AbfsRestOperation deleteFilesystem(TracingContext tracingContext) throws return op; } + /** + * Method for calling createPath API to the backend. Method can be called from: + *
      + *
    1. create new file
    2. + *
    3. overwrite file
    4. + *
    5. create new directory
    6. + *
    + * + * @param path: path of the file / directory to be created / overwritten. + * @param isFile: defines if file or directory has to be created / overwritten. + * @param overwrite: defines if the file / directory to be overwritten. + * @param permissions: contains permission and umask + * @param isAppendBlob: defines if directory in the path is enabled for appendBlob + * @param eTag: required in case of overwrite of file / directory. Path would be + * overwritten only if the provided eTag is equal to the one present in backend for + * the path. + * @param: encryptionAdapter: object that contains the encryptionContext and + * encryptionKey created from the developer provided implementation of + * {@link org.apache.hadoop.fs.azurebfs.extensions.EncryptionContextProvider} + * @param: tracingContext: Object of {@link org.apache.hadoop.fs.azurebfs.utils.TracingContext} + * correlating to the current fs.create() request. + * */ public AbfsRestOperation createPath(final String path, final boolean isFile, final boolean overwrite, final Permissions permissions, final boolean isAppendBlob, final String eTag, - EncryptionAdapter encryptionAdapter, TracingContext tracingContext) + final EncryptionAdapter encryptionAdapter, final TracingContext tracingContext) throws IOException { final List requestHeaders = createDefaultHeaders(); if (isFile) { From a1b19063619c155f11da083aa2c0baef29a74362 Mon Sep 17 00:00:00 2001 From: Pranav Saxena Date: Sun, 9 Oct 2022 22:32:36 -0700 Subject: [PATCH 53/77] small refactors + javadocs --- .../hadoop/fs/azurebfs/AzureBlobFileSystemStore.java | 9 +++++++-- .../azurebfs/extensions/EncryptionContextProvider.java | 4 ++-- .../apache/hadoop/fs/azurebfs/services/AbfsClient.java | 7 ++++--- 3 files changed, 13 insertions(+), 7 deletions(-) diff --git a/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/AzureBlobFileSystemStore.java b/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/AzureBlobFileSystemStore.java index 9d83064244638..1b947d8d39956 100644 --- a/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/AzureBlobFileSystemStore.java +++ b/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/AzureBlobFileSystemStore.java @@ -791,7 +791,7 @@ public AbfsInputStream openFileForRead(Path path, contentLength = Long.parseLong( op.getResponseHeader(HttpHeaderConfigurations.CONTENT_LENGTH)); eTag = op.getResponseHeader(HttpHeaderConfigurations.ETAG); - /** + /* * For file created with ENCRYPTION_CONTEXT, client shall receive * encryptionContext from header field: X_MS_ENCRYPTION_CONTEXT. * */ @@ -1727,7 +1727,12 @@ private boolean parseIsDirectory(final String resourceType) { && resourceType.equalsIgnoreCase(AbfsHttpConstants.DIRECTORY); } - @VisibleForTesting + /** + * Convert properties stored in a Map into a comma separated string. For map + * , method would convert to: + * key1=value1,key2=value,...,keyN=valueN + * */ + @VisibleForTesting String convertXmsPropertiesToCommaSeparatedString(final Map properties) throws CharacterCodingException { diff --git a/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/extensions/EncryptionContextProvider.java b/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/extensions/EncryptionContextProvider.java index 2c62bcdb3c6dd..2b567fa9abe73 100644 --- a/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/extensions/EncryptionContextProvider.java +++ b/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/extensions/EncryptionContextProvider.java @@ -30,9 +30,9 @@ *
  • * To create new encryptionContext from a given path: To be used in case of * create file as there is no encryptionContext in remote server to refer to - * for encryptionKey creation + * for encryptionKey creation. *
  • - *
  • To create encryptionKey using encryptionContext
  • + *
  • To create encryptionKey using encryptionContext.
  • * * */ public interface EncryptionContextProvider extends Destroyable { diff --git a/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/services/AbfsClient.java b/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/services/AbfsClient.java index 2673ff8bfc015..a1537e169ec57 100644 --- a/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/services/AbfsClient.java +++ b/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/services/AbfsClient.java @@ -35,6 +35,7 @@ import java.util.concurrent.TimeUnit; import org.apache.hadoop.classification.VisibleForTesting; +import org.apache.hadoop.fs.PathIOException; import org.apache.hadoop.fs.store.LogExactlyOnce; import org.apache.hadoop.fs.azurebfs.AzureBlobFileSystemStore.Permissions; import org.apache.hadoop.fs.azurebfs.extensions.EncryptionContextProvider; @@ -271,7 +272,7 @@ private void addEncryptionKeyRequestHeaders(String path, false, tracingContext).getResult() .getResponseHeader(X_MS_ENCRYPTION_CONTEXT); if (responseHeaderEncryptionContext == null) { - throw new IOException( + throw new PathIOException(path, "EncryptionContext not present in GetPathStatus response"); } encryptionContext = responseHeaderEncryptionContext.getBytes( @@ -426,10 +427,10 @@ public AbfsRestOperation deleteFilesystem(TracingContext tracingContext) throws * @param eTag: required in case of overwrite of file / directory. Path would be * overwritten only if the provided eTag is equal to the one present in backend for * the path. - * @param: encryptionAdapter: object that contains the encryptionContext and + * @param encryptionAdapter: object that contains the encryptionContext and * encryptionKey created from the developer provided implementation of * {@link org.apache.hadoop.fs.azurebfs.extensions.EncryptionContextProvider} - * @param: tracingContext: Object of {@link org.apache.hadoop.fs.azurebfs.utils.TracingContext} + * @param tracingContext: Object of {@link org.apache.hadoop.fs.azurebfs.utils.TracingContext} * correlating to the current fs.create() request. * */ public AbfsRestOperation createPath(final String path, final boolean isFile, From d35a2760c8dfb7cd9180774d9e5317829798bdb8 Mon Sep 17 00:00:00 2001 From: Pranav Saxena Date: Tue, 11 Oct 2022 06:02:12 -0700 Subject: [PATCH 54/77] removed null handling of encryptionAdapter in abfsclient.addEncryptionKeyRequestHeaders --- .../fs/azurebfs/AzureBlobFileSystemStore.java | 44 ++++++++++++++++--- .../fs/azurebfs/services/AbfsClient.java | 36 ++++----------- .../azurebfs/ITestAbfsCustomEncryption.java | 38 +++++++++++++--- .../ITestAbfsInputStreamStatistics.java | 2 +- .../ITestAzureBlobFileSystemCreate.java | 4 +- .../ITestAzureBlobFileSystemRandomRead.java | 2 +- .../services/TestAbfsInputStream.java | 6 ++- 7 files changed, 88 insertions(+), 44 deletions(-) diff --git a/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/AzureBlobFileSystemStore.java b/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/AzureBlobFileSystemStore.java index 1b947d8d39956..75d4554b76515 100644 --- a/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/AzureBlobFileSystemStore.java +++ b/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/AzureBlobFileSystemStore.java @@ -152,6 +152,7 @@ import static org.apache.hadoop.fs.azurebfs.constants.ConfigurationKeys.AZURE_ABFS_ENDPOINT; import static org.apache.hadoop.fs.azurebfs.constants.ConfigurationKeys.FS_AZURE_BUFFERED_PREAD_DISABLE; import static org.apache.hadoop.fs.azurebfs.constants.ConfigurationKeys.FS_AZURE_IDENTITY_TRANSFORM_CLASS; +import static org.apache.hadoop.fs.azurebfs.constants.HttpHeaderConfigurations.X_MS_ENCRYPTION_CONTEXT; /** * Provides the bridging logic between Hadoop's abstract filesystem and Azure Storage. @@ -475,9 +476,14 @@ public Hashtable getPathStatus(final Path path, path); final Hashtable parsedXmsProperties; + final String relativePath = getRelativePath(path); + final EncryptionAdapter encryptionAdapter + = createEncryptionAdapterFromServerStoreContext(relativePath, + tracingContext); final AbfsRestOperation op = client - .getPathStatus(getRelativePath(path), true, tracingContext); + .getPathStatus(relativePath, true, tracingContext, encryptionAdapter); perfInfo.registerResult(op.getResult()); + encryptionAdapter.destroy(); final String xMsProperties = op.getResult().getResponseHeader(HttpHeaderConfigurations.X_MS_PROPERTIES); @@ -489,6 +495,27 @@ public Hashtable getPathStatus(final Path path, } } + private EncryptionAdapter createEncryptionAdapterFromServerStoreContext(final String path, + final TracingContext tracingContext) throws IOException { + final String responseHeaderEncryptionContext = client.getPathStatus(path, + false, tracingContext, null).getResult() + .getResponseHeader(X_MS_ENCRYPTION_CONTEXT); + if (responseHeaderEncryptionContext == null) { + throw new PathIOException(path, + "EncryptionContext not present in GetPathStatus response"); + } + byte[] encryptionContext = responseHeaderEncryptionContext.getBytes( + StandardCharsets.UTF_8); + + try { + return new EncryptionAdapter(client.getEncryptionContextProvider(), + new Path(path).toUri().getPath(), encryptionContext); + } catch (IOException e) { + LOG.debug("Could not initialize EncryptionAdapter"); + throw e; + } + } + public void setPathProperties(final Path path, final Hashtable properties, TracingContext tracingContext) throws IOException { @@ -504,9 +531,14 @@ public void setPathProperties(final Path path, } catch (CharacterCodingException ex) { throw new InvalidAbfsRestOperationException(ex); } + final String relativePath = getRelativePath(path); + final EncryptionAdapter encryptionAdapter + = createEncryptionAdapterFromServerStoreContext(relativePath, + tracingContext); final AbfsRestOperation op = client .setPathProperties(getRelativePath(path), commaSeparatedProperties, - tracingContext); + tracingContext, encryptionAdapter); + encryptionAdapter.destroy(); perfInfo.registerResult(op.getResult()).registerSuccess(true); } } @@ -633,7 +665,7 @@ private AbfsRestOperation conditionalCreateOverwriteFile(final String relativePa if (e.getStatusCode() == HttpURLConnection.HTTP_CONFLICT) { // File pre-exists, fetch eTag try { - op = client.getPathStatus(relativePath, false, tracingContext); + op = client.getPathStatus(relativePath, false, tracingContext, null); } catch (AbfsRestOperationException ex) { if (ex.getStatusCode() == HttpURLConnection.HTTP_NOT_FOUND) { // Is a parallel access case, as file which was found to be @@ -785,7 +817,7 @@ public AbfsInputStream openFileForRead(Path path, + "is not of type VersionedFileStatus, or file is encrypted"); } AbfsHttpOperation op = client.getPathStatus(relativePath, false, - tracingContext).getResult(); + tracingContext, null).getResult(); resourceType = op.getResponseHeader( HttpHeaderConfigurations.X_MS_RESOURCE_TYPE); contentLength = Long.parseLong( @@ -861,7 +893,7 @@ public OutputStream openFileForWrite(final Path path, String relativePath = getRelativePath(path); final AbfsRestOperation op = client - .getPathStatus(relativePath, false, tracingContext); + .getPathStatus(relativePath, false, tracingContext, null); perfInfo.registerResult(op.getResult()); final String resourceType = op.getResult().getResponseHeader(HttpHeaderConfigurations.X_MS_RESOURCE_TYPE); @@ -1038,7 +1070,7 @@ public FileStatus getFileStatus(final Path path, } } else { perfInfo.registerCallee("getPathStatus"); - op = client.getPathStatus(getRelativePath(path), false, tracingContext); + op = client.getPathStatus(getRelativePath(path), false, tracingContext, null); } perfInfo.registerResult(op.getResult()); diff --git a/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/services/AbfsClient.java b/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/services/AbfsClient.java index a1537e169ec57..1f66a948bda54 100644 --- a/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/services/AbfsClient.java +++ b/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/services/AbfsClient.java @@ -265,27 +265,6 @@ private void addEncryptionKeyRequestHeaders(String path, // get new context for create file request requestHeaders.add(new AbfsHttpHeader(X_MS_ENCRYPTION_CONTEXT, encryptionAdapter.getEncodedContext())); - } else if (encryptionAdapter == null) { - // get encryption context from GetPathStatus response header - byte[] encryptionContext; - final String responseHeaderEncryptionContext = getPathStatus(path, - false, tracingContext).getResult() - .getResponseHeader(X_MS_ENCRYPTION_CONTEXT); - if (responseHeaderEncryptionContext == null) { - throw new PathIOException(path, - "EncryptionContext not present in GetPathStatus response"); - } - encryptionContext = responseHeaderEncryptionContext.getBytes( - StandardCharsets.UTF_8); - - try { - encryptionAdapter = new EncryptionAdapter(encryptionContextProvider, - new Path(path).toUri().getPath(), encryptionContext); - encryptionAdapterCreated = true; - } catch (IOException e) { - LOG.debug("Could not initialize EncryptionAdapter"); - throw e; - } } // else use cached encryption keys from input/output streams encodedKey = encryptionAdapter.getEncodedKey(); @@ -658,7 +637,7 @@ public AbfsClientRenameResult renamePath( // Doing a HEAD call resolves the incomplete metadata state and // then we can retry the rename operation. AbfsRestOperation sourceStatusOp = getPathStatus(source, false, - tracingContext); + tracingContext, null); isMetadataIncompleteState = true; // Extract the sourceEtag, using the status Op, and set it // for future rename recovery. @@ -724,7 +703,7 @@ && isNotEmpty(sourceEtag)) { try { final AbfsRestOperation destStatusOp = getPathStatus(destination, - false, tracingContext); + false, tracingContext, null); final AbfsHttpOperation result = destStatusOp.getResult(); return result.getStatusCode() == HttpURLConnection.HTTP_OK @@ -819,7 +798,7 @@ public boolean appendSuccessCheckOp(AbfsRestOperation op, final String path, throws IOException { if ((op.isARetriedRequest()) && (op.getResult().getStatusCode() == HttpURLConnection.HTTP_BAD_REQUEST)) { - final AbfsRestOperation destStatusOp = getPathStatus(path, false, tracingContext); + final AbfsRestOperation destStatusOp = getPathStatus(path, false, tracingContext, null); if (destStatusOp.getResult().getStatusCode() == HttpURLConnection.HTTP_OK) { String fileLength = destStatusOp.getResult().getResponseHeader( HttpHeaderConfigurations.CONTENT_LENGTH); @@ -869,11 +848,11 @@ public AbfsRestOperation flush(final String path, final long position, } public AbfsRestOperation setPathProperties(final String path, final String properties, - TracingContext tracingContext) + final TracingContext tracingContext, final EncryptionAdapter encryptionAdapter) throws IOException { final List requestHeaders = createDefaultHeaders(); addEncryptionKeyRequestHeaders(path, requestHeaders, false, - null, tracingContext); + encryptionAdapter, tracingContext); // JDK7 does not support PATCH, so to workaround the issue we will use // PUT and specify the real method in the X-Http-Method-Override header. requestHeaders.add(new AbfsHttpHeader(X_HTTP_METHOD_OVERRIDE, @@ -897,7 +876,8 @@ public AbfsRestOperation setPathProperties(final String path, final String prope } public AbfsRestOperation getPathStatus(final String path, - final boolean includeProperties, TracingContext tracingContext) + final boolean includeProperties, final TracingContext tracingContext, + final EncryptionAdapter encryptionAdapter) throws IOException { final List requestHeaders = createDefaultHeaders(); @@ -910,7 +890,7 @@ public AbfsRestOperation getPathStatus(final String path, abfsUriQueryBuilder.addQuery(HttpQueryParams.QUERY_PARAM_ACTION, AbfsHttpConstants.GET_STATUS); operation = SASTokenProvider.GET_STATUS_OPERATION; } else { - addEncryptionKeyRequestHeaders(path, requestHeaders, false, null, + addEncryptionKeyRequestHeaders(path, requestHeaders, false, encryptionAdapter, tracingContext); } abfsUriQueryBuilder.addQuery(HttpQueryParams.QUERY_PARAM_UPN, String.valueOf(abfsConfiguration.isUpnUsed())); diff --git a/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/ITestAbfsCustomEncryption.java b/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/ITestAbfsCustomEncryption.java index e7474be9e84c4..32ea3f33273c6 100644 --- a/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/ITestAbfsCustomEncryption.java +++ b/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/ITestAbfsCustomEncryption.java @@ -27,6 +27,7 @@ import java.util.List; import java.util.Random; +import org.apache.hadoop.fs.PathIOException; import org.apache.hadoop.fs.azurebfs.constants.HttpHeaderConfigurations; import org.apache.hadoop.fs.azurebfs.security.EncodingHelper; import org.apache.hadoop.fs.azurebfs.utils.TracingContext; @@ -58,6 +59,7 @@ import static org.apache.hadoop.fs.azurebfs.constants.ConfigurationKeys.FS_AZURE_ENCRYPTION_CONTEXT_PROVIDER_TYPE; import static org.apache.hadoop.fs.azurebfs.constants.ConfigurationKeys.FS_AZURE_ENCRYPTION_ENCODED_CLIENT_PROVIDED_KEY; import static org.apache.hadoop.fs.azurebfs.constants.ConfigurationKeys.FS_AZURE_ENCRYPTION_ENCODED_CLIENT_PROVIDED_KEY_SHA; +import static org.apache.hadoop.fs.azurebfs.constants.HttpHeaderConfigurations.X_MS_ENCRYPTION_CONTEXT; import static org.apache.hadoop.fs.azurebfs.constants.HttpHeaderConfigurations.X_MS_ENCRYPTION_KEY_SHA256; import static org.apache.hadoop.fs.azurebfs.constants.HttpHeaderConfigurations.X_MS_REQUEST_SERVER_ENCRYPTED; import static org.apache.hadoop.fs.azurebfs.constants.HttpHeaderConfigurations.X_MS_SERVER_ENCRYPTED; @@ -238,7 +240,7 @@ private AbfsRestOperation callOperation(AzureBlobFileSystem fs, case READ: TracingContext tracingContext = getTestTracingContext(fs, true); AbfsHttpOperation statusOp = client.getPathStatus(path, false, - tracingContext).getResult(); + tracingContext, null).getResult(); return client.read(path, 0, new byte[5], 0, 5, statusOp.getResponseHeader(HttpHeaderConfigurations.ETAG), null, encryptionAdapter, tracingContext); case WRITE: @@ -263,13 +265,17 @@ private AbfsRestOperation callOperation(AzureBlobFileSystem fs, getTestTracingContext(fs, false)); case GET_ATTR: return client.getPathStatus(path, true, - getTestTracingContext(fs, false)); + getTestTracingContext(fs, false), + createEncryptionAdapterFromServerStoreContext(path, + getTestTracingContext(fs, false), client)); case SET_ATTR: Hashtable properties = new Hashtable<>(); properties.put("key", "{ value: valueTest }"); return client.setPathProperties(path, fs.getAbfsStore() .convertXmsPropertiesToCommaSeparatedString(properties), - getTestTracingContext(fs, false)); + getTestTracingContext(fs, false), + createEncryptionAdapterFromServerStoreContext(path, + getTestTracingContext(fs, false), client)); case SET_PERMISSION: return client.setPermission(path, FsPermission.getDefault().toString(), getTestTracingContext(fs, false)); @@ -278,6 +284,26 @@ private AbfsRestOperation callOperation(AzureBlobFileSystem fs, } } + private EncryptionAdapter createEncryptionAdapterFromServerStoreContext(final String path, + final TracingContext tracingContext, + final AbfsClient client) throws IOException { + if(client.getEncryptionType() != ENCRYPTION_CONTEXT) { + return null; + } + final String responseHeaderEncryptionContext = client.getPathStatus(path, + false, tracingContext, null).getResult() + .getResponseHeader(X_MS_ENCRYPTION_CONTEXT); + if (responseHeaderEncryptionContext == null) { + throw new PathIOException(path, + "EncryptionContext not present in GetPathStatus response"); + } + byte[] encryptionContext = responseHeaderEncryptionContext.getBytes( + StandardCharsets.UTF_8); + + return new EncryptionAdapter(client.getEncryptionContextProvider(), + new Path(path).toUri().getPath(), encryptionContext); + } + private AzureBlobFileSystem getECProviderEnabledFS() throws Exception { Configuration configuration = getRawConfiguration(); configuration.set(FS_AZURE_ENCRYPTION_CONTEXT_PROVIDER_TYPE + "." @@ -342,9 +368,11 @@ private EncryptionContextProvider createEncryptedFile(Path testPath) throws Exce final AbfsClient abfsClient = fs.getAbfsClient(); abfsClient.setEncryptionType(EncryptionType.NONE); LambdaTestUtils.intercept(IOException.class, () -> - abfsClient.getPathStatus(fs.getAbfsStore().getRelativePath(testPath), + abfsClient.getPathStatus(relativePath, true, - getTestTracingContext(fs, false))); + getTestTracingContext(fs, false), + createEncryptionAdapterFromServerStoreContext(relativePath, + getTestTracingContext(fs, false), abfsClient))); fs.getAbfsClient().setEncryptionType(fileEncryptionType); } return fs.getAbfsClient().getEncryptionContextProvider(); diff --git a/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/ITestAbfsInputStreamStatistics.java b/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/ITestAbfsInputStreamStatistics.java index d96f1a283609f..e8cbeb1255209 100644 --- a/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/ITestAbfsInputStreamStatistics.java +++ b/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/ITestAbfsInputStreamStatistics.java @@ -280,7 +280,7 @@ public void testWithNullStreamStatistics() throws IOException { // AbfsRestOperation Instance required for eTag. AbfsRestOperation abfsRestOperation = fs.getAbfsClient() .getPathStatus(nullStatFilePath.toUri().getPath(), false, - getTestTracingContext(fs, false)); + getTestTracingContext(fs, false), null); // AbfsInputStream with no StreamStatistics. in = new AbfsInputStream(fs.getAbfsClient(), null, diff --git a/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/ITestAzureBlobFileSystemCreate.java b/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/ITestAzureBlobFileSystemCreate.java index 4a103d2c0c053..222d71b1fcc22 100644 --- a/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/ITestAzureBlobFileSystemCreate.java +++ b/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/ITestAzureBlobFileSystemCreate.java @@ -33,6 +33,7 @@ import org.apache.hadoop.fs.FileSystem; import org.apache.hadoop.fs.FSDataOutputStream; import org.apache.hadoop.fs.Path; +import org.apache.hadoop.fs.azurebfs.security.EncryptionAdapter; import org.apache.hadoop.fs.permission.FsAction; import org.apache.hadoop.fs.permission.FsPermission; import org.apache.hadoop.test.GenericTestUtils; @@ -406,7 +407,8 @@ public void testNegativeScenariosForCreateOverwriteDisabled() .doReturn(successOp) // Scn3: create overwrite=true fails with Http412 .doReturn(successOp) // Scn4: create overwrite=true fails with Http500 .when(mockClient) - .getPathStatus(any(String.class), eq(false), any(TracingContext.class)); + .getPathStatus(any(String.class), eq(false), any(TracingContext.class), any( + EncryptionAdapter.class)); // mock for overwrite=true doThrow( diff --git a/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/ITestAzureBlobFileSystemRandomRead.java b/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/ITestAzureBlobFileSystemRandomRead.java index c1f0e06439950..940d56fecb438 100644 --- a/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/ITestAzureBlobFileSystemRandomRead.java +++ b/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/ITestAzureBlobFileSystemRandomRead.java @@ -508,7 +508,7 @@ public void testAlwaysReadBufferSizeConfig(boolean alwaysReadBufferSizeConfigVal 1 * MEGABYTE, config); String eTag = fs.getAbfsClient() .getPathStatus(testFile.toUri().getPath(), false, - getTestTracingContext(fs, false)) + getTestTracingContext(fs, false), null) .getResult() .getResponseHeader(ETAG); diff --git a/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/services/TestAbfsInputStream.java b/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/services/TestAbfsInputStream.java index c50c192add7f0..6d1ecc2936127 100644 --- a/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/services/TestAbfsInputStream.java +++ b/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/services/TestAbfsInputStream.java @@ -41,6 +41,7 @@ import org.apache.hadoop.fs.azurebfs.AzureBlobFileSystemStore; import org.apache.hadoop.fs.azurebfs.contracts.exceptions.TimeoutException; import org.apache.hadoop.fs.azurebfs.contracts.services.ReadBufferStatus; +import org.apache.hadoop.fs.azurebfs.security.EncryptionAdapter; import org.apache.hadoop.fs.azurebfs.utils.TestCachedSASToken; import org.apache.hadoop.fs.azurebfs.utils.TracingContext; import org.apache.hadoop.fs.impl.OpenFileParameters; @@ -234,7 +235,8 @@ private void checkGetPathStatusCalls(Path testFile, FileStatus fileStatus, .ofNullable(new OpenFileParameters().withStatus(fileStatus)), null, tracingContext); verify(mockClient, times(0).description((String.format( "FileStatus [from %s result] provided, GetFileStatus should not be invoked", - source)))).getPathStatus(anyString(), anyBoolean(), any(TracingContext.class)); + source)))).getPathStatus(anyString(), anyBoolean(), any(TracingContext.class), any( + EncryptionAdapter.class)); // verify GetPathStatus invoked when FileStatus not provided abfsStore.openFileForRead(testFile, @@ -242,7 +244,7 @@ private void checkGetPathStatusCalls(Path testFile, FileStatus fileStatus, tracingContext); verify(mockClient, times(1).description( "GetPathStatus should be invoked when FileStatus not provided")) - .getPathStatus(anyString(), anyBoolean(), any(TracingContext.class)); + .getPathStatus(anyString(), anyBoolean(), any(TracingContext.class), any(EncryptionAdapter.class)); Mockito.reset(mockClient); //clears invocation count for next test case } From 450324a59813b2a8bf578a7a1a279170b4cbbee3 Mon Sep 17 00:00:00 2001 From: Pranav Saxena Date: Tue, 11 Oct 2022 06:49:00 -0700 Subject: [PATCH 55/77] refactored how encryptionAdapter was created in case of versionFileStatus object --- .../fs/azurebfs/AzureBlobFileSystemStore.java | 48 +++++++++++-------- 1 file changed, 29 insertions(+), 19 deletions(-) diff --git a/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/AzureBlobFileSystemStore.java b/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/AzureBlobFileSystemStore.java index 75d4554b76515..6623084c0f001 100644 --- a/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/AzureBlobFileSystemStore.java +++ b/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/AzureBlobFileSystemStore.java @@ -800,22 +800,7 @@ public AbfsInputStream openFileForRead(Path path, String resourceType, eTag; long contentLength; EncryptionAdapter encryptionAdapter = null; - if (fileStatus instanceof VersionedFileStatus - && client.getEncryptionType() != EncryptionType.ENCRYPTION_CONTEXT) { - path = path.makeQualified(this.uri, path); - Preconditions.checkArgument(fileStatus.getPath().equals(path), - String.format( - "Filestatus path [%s] does not match with given path [%s]", - fileStatus.getPath(), path)); - resourceType = fileStatus.isFile() ? FILE : DIRECTORY; - contentLength = fileStatus.getLen(); - eTag = ((VersionedFileStatus) fileStatus).getVersion(); - } else { - if (fileStatus != null) { - LOG.debug( - "Fallback to getPathStatus REST call as provided filestatus " - + "is not of type VersionedFileStatus, or file is encrypted"); - } + if(!(fileStatus instanceof VersionedFileStatus) || (client.getEncryptionType() == EncryptionType.ENCRYPTION_CONTEXT && ((VersionedFileStatus) fileStatus).getEncryptionContext() == null)) { AbfsHttpOperation op = client.getPathStatus(relativePath, false, tracingContext, null).getResult(); resourceType = op.getResponseHeader( @@ -839,6 +824,21 @@ public AbfsInputStream openFileForRead(Path path, client.getEncryptionContextProvider(), getRelativePath(path), fileEncryptionContext.getBytes(StandardCharsets.UTF_8)); } + } else { + path = path.makeQualified(this.uri, path); + Preconditions.checkArgument(fileStatus.getPath().equals(path), + String.format( + "Filestatus path [%s] does not match with given path [%s]", + fileStatus.getPath(), path)); + resourceType = fileStatus.isFile() ? FILE : DIRECTORY; + contentLength = fileStatus.getLen(); + eTag = ((VersionedFileStatus) fileStatus).getVersion(); + final String encryptionContext = ((VersionedFileStatus) fileStatus).getEncryptionContext(); + if(client.getEncryptionType() == EncryptionType.ENCRYPTION_CONTEXT) { + encryptionAdapter = new EncryptionAdapter( + client.getEncryptionContextProvider(), getRelativePath(path), + encryptionContext.getBytes(StandardCharsets.UTF_8)); + } } if (parseIsDirectory(resourceType)) { @@ -1080,6 +1080,7 @@ public FileStatus getFileStatus(final Path path, String eTag = extractEtagHeader(result); final String lastModified = result.getResponseHeader(HttpHeaderConfigurations.LAST_MODIFIED); final String permissions = result.getResponseHeader((HttpHeaderConfigurations.X_MS_PERMISSIONS)); + final String encryptionContext = op.getResult().getResponseHeader(X_MS_ENCRYPTION_CONTEXT); final boolean hasAcl = AbfsPermission.isExtendedAcl(permissions); final long contentLength; final boolean resourceIsDir; @@ -1116,7 +1117,8 @@ public FileStatus getFileStatus(final Path path, blockSize, DateTimeUtils.parseLastModifiedTime(lastModified), path, - eTag); + eTag, + encryptionContext); } } @@ -1222,7 +1224,8 @@ public String listStatus(final Path path, final String startFrom, blockSize, lastModifiedMillis, entryPath, - entry.eTag())); + entry.eTag(), + null)); } perfInfo.registerSuccess(true); @@ -1879,11 +1882,13 @@ private static final class VersionedFileStatus extends FileStatus */ private String version; + private String encryptionContext; + private VersionedFileStatus( final String owner, final String group, final FsPermission fsPermission, final boolean hasAcl, final long length, final boolean isdir, final int blockReplication, final long blocksize, final long modificationTime, final Path path, - String version) { + final String version, final String encryptionContext) { super(length, isdir, blockReplication, blocksize, modificationTime, 0, fsPermission, owner, @@ -1893,6 +1898,7 @@ private VersionedFileStatus( hasAcl, false, false); this.version = version; + this.encryptionContext = encryptionContext; } /** Compare if this object is equal to another object. @@ -1945,6 +1951,10 @@ public String getEtag() { return getVersion(); } + public String getEncryptionContext() { + return encryptionContext; + } + @Override public String toString() { final StringBuilder sb = new StringBuilder( From 183f1bc4409f166b822f7dfb11271222fc7b1f81 Mon Sep 17 00:00:00 2001 From: Pranav Saxena Date: Wed, 12 Oct 2022 00:05:18 -0700 Subject: [PATCH 56/77] small refactors --- .../fs/azurebfs/AzureBlobFileSystemStore.java | 28 +++++++++++++++---- .../fs/azurebfs/security/EncodingHelper.java | 5 ++-- .../azurebfs/security/EncryptionAdapter.java | 11 +++++--- .../fs/azurebfs/services/AbfsInputStream.java | 3 +- 4 files changed, 34 insertions(+), 13 deletions(-) diff --git a/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/AzureBlobFileSystemStore.java b/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/AzureBlobFileSystemStore.java index 6623084c0f001..9e41948c6a6d7 100644 --- a/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/AzureBlobFileSystemStore.java +++ b/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/AzureBlobFileSystemStore.java @@ -800,7 +800,18 @@ public AbfsInputStream openFileForRead(Path path, String resourceType, eTag; long contentLength; EncryptionAdapter encryptionAdapter = null; - if(!(fileStatus instanceof VersionedFileStatus) || (client.getEncryptionType() == EncryptionType.ENCRYPTION_CONTEXT && ((VersionedFileStatus) fileStatus).getEncryptionContext() == null)) { + /* + * GetPathStatus API has to be called in case of: + * 1. fileStatus is null or not an object of VersionedFileStatus: as eTag + * would not be there in the fileStatus object. + * 2. fileStatus is an object of VersionedFileStatus and the object doesn't + * have encryptionContext field when client's encryptionType is + * ENCRYPTION_CONTEXT. + * */ + if (!(fileStatus instanceof VersionedFileStatus) || ( + client.getEncryptionType() == EncryptionType.ENCRYPTION_CONTEXT + && ((VersionedFileStatus) fileStatus).getEncryptionContext() + == null)) { AbfsHttpOperation op = client.getPathStatus(relativePath, false, tracingContext, null).getResult(); resourceType = op.getResponseHeader( @@ -833,8 +844,9 @@ public AbfsInputStream openFileForRead(Path path, resourceType = fileStatus.isFile() ? FILE : DIRECTORY; contentLength = fileStatus.getLen(); eTag = ((VersionedFileStatus) fileStatus).getVersion(); - final String encryptionContext = ((VersionedFileStatus) fileStatus).getEncryptionContext(); - if(client.getEncryptionType() == EncryptionType.ENCRYPTION_CONTEXT) { + final String encryptionContext + = ((VersionedFileStatus) fileStatus).getEncryptionContext(); + if (client.getEncryptionType() == EncryptionType.ENCRYPTION_CONTEXT) { encryptionAdapter = new EncryptionAdapter( client.getEncryptionContextProvider(), getRelativePath(path), encryptionContext.getBytes(StandardCharsets.UTF_8)); @@ -1968,8 +1980,8 @@ public String toString() { /** * Permissions class contain provided permission and umask in octalNotation. - * If the object is created for namespace-disabled, the permission and umask - * would be null. + * If the object is created for namespace-disabled account, the permission and + * umask would be null. * */ public static final class Permissions { private final String permission; @@ -2006,6 +2018,12 @@ public String getPermission() { public String getUmask() { return umask; } + + @Override + public String toString() { + return String.format("{\"permission\":%s, \"umask\":%s}", permission, + umask); + } } /** diff --git a/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/security/EncodingHelper.java b/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/security/EncodingHelper.java index 1e2ec3948d487..9a90cf05bb927 100644 --- a/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/security/EncodingHelper.java +++ b/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/security/EncodingHelper.java @@ -32,10 +32,11 @@ public static byte[] getSHA256Hash(byte[] key) { try { final MessageDigest digester = MessageDigest.getInstance("SHA-256"); return digester.digest(key); - } catch (NoSuchAlgorithmException ignored) { + } catch (NoSuchAlgorithmException noSuchAlgorithmException) { /*This exception can be ignored. Reason being SHA-256 is a valid algorithm, and it is constant for all method calls.*/ - throw new RuntimeException("SHA-256 algorithm not found in MessageDigest", ignored); + throw new RuntimeException("SHA-256 algorithm not found in MessageDigest", + noSuchAlgorithmException); } } diff --git a/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/security/EncryptionAdapter.java b/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/security/EncryptionAdapter.java index 981889bcec3cd..8020c138310f4 100644 --- a/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/security/EncryptionAdapter.java +++ b/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/security/EncryptionAdapter.java @@ -25,12 +25,15 @@ import org.apache.hadoop.fs.azurebfs.extensions.EncryptionContextProvider; +import static org.apache.hadoop.fs.azurebfs.security.EncodingHelper.getBase64EncodedString; + /** * Class manages the encryptionContext and encryptionKey that needs to be added * to the headers to server request if Customer-Encryption-Context is enabled in * the configuration. *
    - * For fileCreation, the object helps in creating encryptionContext. + * For fileCreation, the object helps in creating encryptionContext through the + * implementation of EncryptionContextProvider. *
    * For all operations, the object helps in converting encryptionContext to * encryptionKey through the implementation of EncryptionContextProvider. @@ -81,15 +84,15 @@ private void computeKeys() throws IOException { } public String getEncodedKey() throws IOException { - return EncodingHelper.getBase64EncodedString(encryptionKey.getEncoded()); + return getBase64EncodedString(encryptionKey.getEncoded()); } public String getEncodedKeySHA() throws IOException { - return EncodingHelper.getBase64EncodedString(EncodingHelper.getSHA256Hash(encryptionKey.getEncoded())); + return getBase64EncodedString(EncodingHelper.getSHA256Hash(encryptionKey.getEncoded())); } public String getEncodedContext() throws IOException { - return EncodingHelper.getBase64EncodedString(encryptionContext.getEncoded()); + return getBase64EncodedString(encryptionContext.getEncoded()); } public void destroy() { diff --git a/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/services/AbfsInputStream.java b/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/services/AbfsInputStream.java index 8140369cece3a..48630e7b0bc86 100644 --- a/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/services/AbfsInputStream.java +++ b/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/services/AbfsInputStream.java @@ -18,7 +18,6 @@ package org.apache.hadoop.fs.azurebfs.services; -import javax.security.auth.DestroyFailedException; import java.io.EOFException; import java.io.FileNotFoundException; import java.io.IOException; @@ -27,7 +26,6 @@ import org.apache.commons.lang3.StringUtils; import org.apache.hadoop.classification.VisibleForTesting; -import org.apache.hadoop.fs.azurebfs.security.EncryptionAdapter; import org.apache.hadoop.util.Preconditions; import org.slf4j.Logger; @@ -44,6 +42,7 @@ import org.apache.hadoop.fs.azurebfs.utils.CachedSASToken; import org.apache.hadoop.fs.azurebfs.utils.Listener; import org.apache.hadoop.fs.azurebfs.utils.TracingContext; +import org.apache.hadoop.fs.azurebfs.security.EncryptionAdapter; import org.apache.hadoop.fs.statistics.IOStatistics; import org.apache.hadoop.fs.statistics.IOStatisticsSource; From d0a16bde279ec0f10688a99dc751509bf850f086 Mon Sep 17 00:00:00 2001 From: Pranav Saxena Date: Wed, 12 Oct 2022 06:10:38 -0700 Subject: [PATCH 57/77] fixed tests --- .../hadoop/fs/azurebfs/AzureBlobFileSystemStore.java | 11 +++++++++-- .../fs/azurebfs/ITestAzureBlobFileSystemCreate.java | 3 ++- 2 files changed, 11 insertions(+), 3 deletions(-) diff --git a/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/AzureBlobFileSystemStore.java b/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/AzureBlobFileSystemStore.java index 9e41948c6a6d7..1952ce64f209e 100644 --- a/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/AzureBlobFileSystemStore.java +++ b/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/AzureBlobFileSystemStore.java @@ -483,7 +483,9 @@ public Hashtable getPathStatus(final Path path, final AbfsRestOperation op = client .getPathStatus(relativePath, true, tracingContext, encryptionAdapter); perfInfo.registerResult(op.getResult()); - encryptionAdapter.destroy(); + if(encryptionAdapter != null) { + encryptionAdapter.destroy(); + } final String xMsProperties = op.getResult().getResponseHeader(HttpHeaderConfigurations.X_MS_PROPERTIES); @@ -497,6 +499,9 @@ public Hashtable getPathStatus(final Path path, private EncryptionAdapter createEncryptionAdapterFromServerStoreContext(final String path, final TracingContext tracingContext) throws IOException { + if(client.getEncryptionType() != EncryptionType.ENCRYPTION_CONTEXT) { + return null; + } final String responseHeaderEncryptionContext = client.getPathStatus(path, false, tracingContext, null).getResult() .getResponseHeader(X_MS_ENCRYPTION_CONTEXT); @@ -538,7 +543,9 @@ public void setPathProperties(final Path path, final AbfsRestOperation op = client .setPathProperties(getRelativePath(path), commaSeparatedProperties, tracingContext, encryptionAdapter); - encryptionAdapter.destroy(); + if(encryptionAdapter != null) { + encryptionAdapter.destroy(); + } perfInfo.registerResult(op.getResult()).registerSuccess(true); } } diff --git a/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/ITestAzureBlobFileSystemCreate.java b/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/ITestAzureBlobFileSystemCreate.java index 222d71b1fcc22..b98d3cc9cd4b8 100644 --- a/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/ITestAzureBlobFileSystemCreate.java +++ b/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/ITestAzureBlobFileSystemCreate.java @@ -56,6 +56,7 @@ import static org.mockito.ArgumentMatchers.any; import static org.mockito.ArgumentMatchers.eq; +import static org.mockito.ArgumentMatchers.nullable; import static org.mockito.Mockito.doThrow; import static org.mockito.Mockito.mock; import static org.mockito.Mockito.when; @@ -407,7 +408,7 @@ public void testNegativeScenariosForCreateOverwriteDisabled() .doReturn(successOp) // Scn3: create overwrite=true fails with Http412 .doReturn(successOp) // Scn4: create overwrite=true fails with Http500 .when(mockClient) - .getPathStatus(any(String.class), eq(false), any(TracingContext.class), any( + .getPathStatus(any(String.class), eq(false), any(TracingContext.class), nullable( EncryptionAdapter.class)); // mock for overwrite=true From a69ee4a81eda01ed87d87529770141861e8cb2fd Mon Sep 17 00:00:00 2001 From: Pranav Saxena Date: Mon, 17 Oct 2022 04:08:38 -0700 Subject: [PATCH 58/77] refactored condition in openFileForRead --- .../fs/azurebfs/AzureBlobFileSystemStore.java | 40 +++++++++---------- 1 file changed, 20 insertions(+), 20 deletions(-) diff --git a/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/AzureBlobFileSystemStore.java b/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/AzureBlobFileSystemStore.java index 1952ce64f209e..5f31a758cf9f9 100644 --- a/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/AzureBlobFileSystemStore.java +++ b/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/AzureBlobFileSystemStore.java @@ -815,10 +815,26 @@ public AbfsInputStream openFileForRead(Path path, * have encryptionContext field when client's encryptionType is * ENCRYPTION_CONTEXT. * */ - if (!(fileStatus instanceof VersionedFileStatus) || ( - client.getEncryptionType() == EncryptionType.ENCRYPTION_CONTEXT - && ((VersionedFileStatus) fileStatus).getEncryptionContext() - == null)) { + if ((fileStatus instanceof VersionedFileStatus) && ( + client.getEncryptionType() != EncryptionType.ENCRYPTION_CONTEXT + || ((VersionedFileStatus) fileStatus).getEncryptionContext() + != null)) { + path = path.makeQualified(this.uri, path); + Preconditions.checkArgument(fileStatus.getPath().equals(path), + String.format( + "Filestatus path [%s] does not match with given path [%s]", + fileStatus.getPath(), path)); + resourceType = fileStatus.isFile() ? FILE : DIRECTORY; + contentLength = fileStatus.getLen(); + eTag = ((VersionedFileStatus) fileStatus).getVersion(); + final String encryptionContext + = ((VersionedFileStatus) fileStatus).getEncryptionContext(); + if (client.getEncryptionType() == EncryptionType.ENCRYPTION_CONTEXT) { + encryptionAdapter = new EncryptionAdapter( + client.getEncryptionContextProvider(), getRelativePath(path), + encryptionContext.getBytes(StandardCharsets.UTF_8)); + } + } else { AbfsHttpOperation op = client.getPathStatus(relativePath, false, tracingContext, null).getResult(); resourceType = op.getResponseHeader( @@ -842,22 +858,6 @@ public AbfsInputStream openFileForRead(Path path, client.getEncryptionContextProvider(), getRelativePath(path), fileEncryptionContext.getBytes(StandardCharsets.UTF_8)); } - } else { - path = path.makeQualified(this.uri, path); - Preconditions.checkArgument(fileStatus.getPath().equals(path), - String.format( - "Filestatus path [%s] does not match with given path [%s]", - fileStatus.getPath(), path)); - resourceType = fileStatus.isFile() ? FILE : DIRECTORY; - contentLength = fileStatus.getLen(); - eTag = ((VersionedFileStatus) fileStatus).getVersion(); - final String encryptionContext - = ((VersionedFileStatus) fileStatus).getEncryptionContext(); - if (client.getEncryptionType() == EncryptionType.ENCRYPTION_CONTEXT) { - encryptionAdapter = new EncryptionAdapter( - client.getEncryptionContextProvider(), getRelativePath(path), - encryptionContext.getBytes(StandardCharsets.UTF_8)); - } } if (parseIsDirectory(resourceType)) { From 061930b28d06087df6247370680b153f8a540b45 Mon Sep 17 00:00:00 2001 From: Pranav Saxena Date: Wed, 16 Nov 2022 23:25:56 -0800 Subject: [PATCH 59/77] getIsNamespace in abfsclient --- .../fs/azurebfs/services/AbfsClient.java | 27 +++++++++++++++++++ 1 file changed, 27 insertions(+) diff --git a/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/services/AbfsClient.java b/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/services/AbfsClient.java index 1f66a948bda54..b35c804c539d3 100644 --- a/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/services/AbfsClient.java +++ b/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/services/AbfsClient.java @@ -36,6 +36,7 @@ import org.apache.hadoop.classification.VisibleForTesting; import org.apache.hadoop.fs.PathIOException; +import org.apache.hadoop.fs.azurebfs.contracts.exceptions.AbfsRestOperationException; import org.apache.hadoop.fs.store.LogExactlyOnce; import org.apache.hadoop.fs.azurebfs.AzureBlobFileSystemStore.Permissions; import org.apache.hadoop.fs.azurebfs.extensions.EncryptionContextProvider; @@ -108,6 +109,7 @@ public class AbfsClient implements Closeable { private EncryptionType encryptionType = EncryptionType.NONE; private final ListeningScheduledExecutorService executorService; + private Boolean isNamespaceEnabled; /** logging the rename failure if metadata is in an incomplete state. */ @@ -252,6 +254,9 @@ private void addEncryptionKeyRequestHeaders(String path, List requestHeaders, boolean isCreateFileRequest, EncryptionAdapter encryptionAdapter, TracingContext tracingContext) throws IOException { + if(!getIsNamespaceEnabled(tracingContext)) { + return; + } String encodedKey, encodedKeySHA256; boolean encryptionAdapterCreated = false; switch (encryptionType) { @@ -1276,6 +1281,28 @@ public synchronized String getAccessToken() throws IOException { } } + private synchronized Boolean getIsNamespaceEnabled(TracingContext tracingContext) + throws AzureBlobFileSystemException { + if(isNamespaceEnabled != null) { + return isNamespaceEnabled; + } + try { + this.getAclStatus(AbfsHttpConstants.ROOT_PATH, tracingContext); + isNamespaceEnabled = true; + } catch (AbfsRestOperationException ex) { + // Get ACL status is a HEAD request, its response doesn't contain + // errorCode + // So can only rely on its status code to determine its account type. + if (HttpURLConnection.HTTP_BAD_REQUEST != ex.getStatusCode()) { + throw ex; + } + isNamespaceEnabled = false; + } catch (AzureBlobFileSystemException ex) { + throw ex; + } + return isNamespaceEnabled; + } + public AuthType getAuthType() { return authType; } From 00bd8c3bd4a40090fc2fb1f2266b75f32af239d2 Mon Sep 17 00:00:00 2001 From: Pranav Saxena Date: Wed, 16 Nov 2022 23:37:32 -0800 Subject: [PATCH 60/77] Added NamespaceUtil --- .../fs/azurebfs/AzureBlobFileSystemStore.java | 21 +--------- .../fs/azurebfs/services/AbfsClient.java | 17 ++------ .../fs/azurebfs/utils/NamespaceUtil.java | 42 +++++++++++++++++++ 3 files changed, 47 insertions(+), 33 deletions(-) create mode 100644 hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/utils/NamespaceUtil.java diff --git a/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/AzureBlobFileSystemStore.java b/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/AzureBlobFileSystemStore.java index 5f31a758cf9f9..133ca0692f148 100644 --- a/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/AzureBlobFileSystemStore.java +++ b/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/AzureBlobFileSystemStore.java @@ -59,6 +59,7 @@ import org.apache.hadoop.fs.azurebfs.security.EncryptionAdapter; import org.apache.hadoop.fs.azurebfs.utils.EncryptionType; import org.apache.hadoop.classification.VisibleForTesting; +import org.apache.hadoop.fs.azurebfs.utils.NamespaceUtil; import org.apache.hadoop.util.Preconditions; import org.apache.hadoop.thirdparty.com.google.common.base.Strings; import org.apache.hadoop.thirdparty.com.google.common.util.concurrent.Futures; @@ -362,25 +363,7 @@ public boolean getIsNamespaceEnabled(TracingContext tracingContext) + " getAcl server call", e); } - LOG.debug("Get root ACL status"); - try (AbfsPerfInfo perfInfo = startTracking("getIsNamespaceEnabled", - "getAclStatus")) { - AbfsRestOperation op = client - .getAclStatus(AbfsHttpConstants.ROOT_PATH, tracingContext); - perfInfo.registerResult(op.getResult()); - isNamespaceEnabled = Trilean.getTrilean(true); - perfInfo.registerSuccess(true); - } catch (AbfsRestOperationException ex) { - // Get ACL status is a HEAD request, its response doesn't contain - // errorCode - // So can only rely on its status code to determine its account type. - if (HttpURLConnection.HTTP_BAD_REQUEST != ex.getStatusCode()) { - throw ex; - } - - isNamespaceEnabled = Trilean.getTrilean(false); - } - + isNamespaceEnabled = Trilean.getTrilean(NamespaceUtil.isNamespaceEnabled(client, tracingContext)); return isNamespaceEnabled.toBoolean(); } diff --git a/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/services/AbfsClient.java b/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/services/AbfsClient.java index b35c804c539d3..79a0b919c6dbc 100644 --- a/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/services/AbfsClient.java +++ b/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/services/AbfsClient.java @@ -37,6 +37,7 @@ import org.apache.hadoop.classification.VisibleForTesting; import org.apache.hadoop.fs.PathIOException; import org.apache.hadoop.fs.azurebfs.contracts.exceptions.AbfsRestOperationException; +import org.apache.hadoop.fs.azurebfs.utils.NamespaceUtil; import org.apache.hadoop.fs.store.LogExactlyOnce; import org.apache.hadoop.fs.azurebfs.AzureBlobFileSystemStore.Permissions; import org.apache.hadoop.fs.azurebfs.extensions.EncryptionContextProvider; @@ -1286,20 +1287,8 @@ private synchronized Boolean getIsNamespaceEnabled(TracingContext tracingContext if(isNamespaceEnabled != null) { return isNamespaceEnabled; } - try { - this.getAclStatus(AbfsHttpConstants.ROOT_PATH, tracingContext); - isNamespaceEnabled = true; - } catch (AbfsRestOperationException ex) { - // Get ACL status is a HEAD request, its response doesn't contain - // errorCode - // So can only rely on its status code to determine its account type. - if (HttpURLConnection.HTTP_BAD_REQUEST != ex.getStatusCode()) { - throw ex; - } - isNamespaceEnabled = false; - } catch (AzureBlobFileSystemException ex) { - throw ex; - } + + isNamespaceEnabled = NamespaceUtil.isNamespaceEnabled(this, tracingContext); return isNamespaceEnabled; } diff --git a/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/utils/NamespaceUtil.java b/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/utils/NamespaceUtil.java new file mode 100644 index 0000000000000..2a81867f1135a --- /dev/null +++ b/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/utils/NamespaceUtil.java @@ -0,0 +1,42 @@ +package org.apache.hadoop.fs.azurebfs.utils; + +import java.net.HttpURLConnection; + +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + +import org.apache.hadoop.fs.azurebfs.constants.AbfsHttpConstants; +import org.apache.hadoop.fs.azurebfs.contracts.exceptions.AbfsRestOperationException; +import org.apache.hadoop.fs.azurebfs.contracts.exceptions.AzureBlobFileSystemException; +import org.apache.hadoop.fs.azurebfs.services.AbfsClient; + +public class NamespaceUtil { + + public static final Logger LOG = LoggerFactory.getLogger(NamespaceUtil.class); + + private NamespaceUtil() { + + } + + public static Boolean isNamespaceEnabled(final AbfsClient abfsClient, final TracingContext tracingContext) + throws AzureBlobFileSystemException { + Boolean isNamespaceEnabled; + try { + LOG.debug("Get root ACL status"); + abfsClient.getAclStatus(AbfsHttpConstants.ROOT_PATH, tracingContext); + isNamespaceEnabled = true; + } catch (AbfsRestOperationException ex) { + // Get ACL status is a HEAD request, its response doesn't contain + // errorCode + // So can only rely on its status code to determine its account type. + if (HttpURLConnection.HTTP_BAD_REQUEST != ex.getStatusCode()) { + throw ex; + } + isNamespaceEnabled = false; + } catch (AzureBlobFileSystemException ex) { + throw ex; + } + return isNamespaceEnabled; + } + +} From a642ef32842f2b7aadee1eeaa36ee3955e5094f8 Mon Sep 17 00:00:00 2001 From: Pranav Saxena Date: Sun, 20 Nov 2022 22:17:29 -0800 Subject: [PATCH 61/77] ListResultEntrySchema to have value for x-ms-encryption-context: https://learn.microsoft.com/en-us/rest/api/storageservices/datalakestoragegen2/path/list#pathlist --- .../fs/azurebfs/AzureBlobFileSystemStore.java | 3 ++- .../contracts/services/ListResultEntrySchema.java | 13 +++++++++++++ 2 files changed, 15 insertions(+), 1 deletion(-) diff --git a/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/AzureBlobFileSystemStore.java b/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/AzureBlobFileSystemStore.java index 133ca0692f148..43ad8f9b88c9b 100644 --- a/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/AzureBlobFileSystemStore.java +++ b/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/AzureBlobFileSystemStore.java @@ -1198,6 +1198,7 @@ public String listStatus(final Path path, final String startFrom, for (ListResultEntrySchema entry : retrievedSchema.paths()) { final String owner = identityTransformer.transformIdentityForGetRequest(entry.owner(), true, userName); final String group = identityTransformer.transformIdentityForGetRequest(entry.group(), false, primaryUserGroup); + final String encryptionContext = entry.xMsEncryptionContext(); final FsPermission fsPermission = entry.permissions() == null ? new AbfsPermission(FsAction.ALL, FsAction.ALL, FsAction.ALL) : AbfsPermission.valueOf(entry.permissions()); @@ -1227,7 +1228,7 @@ public String listStatus(final Path path, final String startFrom, lastModifiedMillis, entryPath, entry.eTag(), - null)); + encryptionContext)); } perfInfo.registerSuccess(true); diff --git a/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/contracts/services/ListResultEntrySchema.java b/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/contracts/services/ListResultEntrySchema.java index a9883dd2ce5fc..18455f748072b 100644 --- a/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/contracts/services/ListResultEntrySchema.java +++ b/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/contracts/services/ListResultEntrySchema.java @@ -77,6 +77,12 @@ public class ListResultEntrySchema { @JsonProperty(value = "permissions") private String permissions; + /** + * The encryption context property + */ + @JsonProperty(value = "x-ms-encryption-context") + private String xMsEncryptionContext; + /** * Get the name value. * @@ -238,4 +244,11 @@ public ListResultEntrySchema withPermissions(final String permissions) { return this; } + /** + * Get the x-ms-encryption-context value. + * @return the x-ms-encryption-context value + * */ + public String xMsEncryptionContext() { + return xMsEncryptionContext; + } } \ No newline at end of file From 56f3efa053fd74cc5744c1f7c277409d594ae216 Mon Sep 17 00:00:00 2001 From: Pranav Saxena Date: Sun, 20 Nov 2022 23:47:34 -0800 Subject: [PATCH 62/77] added test for the listPath --- .../fs/azurebfs/AzureBlobFileSystemStore.java | 2 +- .../services/ListResultEntrySchema.java | 20 ++++- .../azurebfs/ITestAbfsCustomEncryption.java | 88 +++++++++++-------- 3 files changed, 69 insertions(+), 41 deletions(-) diff --git a/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/AzureBlobFileSystemStore.java b/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/AzureBlobFileSystemStore.java index 43ad8f9b88c9b..d856ad38d9e9f 100644 --- a/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/AzureBlobFileSystemStore.java +++ b/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/AzureBlobFileSystemStore.java @@ -1198,7 +1198,7 @@ public String listStatus(final Path path, final String startFrom, for (ListResultEntrySchema entry : retrievedSchema.paths()) { final String owner = identityTransformer.transformIdentityForGetRequest(entry.owner(), true, userName); final String group = identityTransformer.transformIdentityForGetRequest(entry.group(), false, primaryUserGroup); - final String encryptionContext = entry.xMsEncryptionContext(); + final String encryptionContext = entry.getXMsEncryptionContext(); final FsPermission fsPermission = entry.permissions() == null ? new AbfsPermission(FsAction.ALL, FsAction.ALL, FsAction.ALL) : AbfsPermission.valueOf(entry.permissions()); diff --git a/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/contracts/services/ListResultEntrySchema.java b/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/contracts/services/ListResultEntrySchema.java index 18455f748072b..d35b9cfea238c 100644 --- a/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/contracts/services/ListResultEntrySchema.java +++ b/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/contracts/services/ListResultEntrySchema.java @@ -80,9 +80,15 @@ public class ListResultEntrySchema { /** * The encryption context property */ - @JsonProperty(value = "x-ms-encryption-context") + @JsonProperty(value = "EncryptionContext") private String xMsEncryptionContext; + /** + * The customer-provided encryption-256 value + * */ + @JsonProperty(value = "CustomerProvidedKeySha256") + private String customerProvidedKeySha256; + /** * Get the name value. * @@ -246,9 +252,17 @@ public ListResultEntrySchema withPermissions(final String permissions) { /** * Get the x-ms-encryption-context value. - * @return the x-ms-encryption-context value + * @return the x-ms-encryption-context value. * */ - public String xMsEncryptionContext() { + public String getXMsEncryptionContext() { return xMsEncryptionContext; } + + /** + * Get the customer-provided sha-256 key + * @return the x-ms-encryption-key-sha256 value used by client. + * */ + public String getCustomerProvidedKeySha256() { + return customerProvidedKeySha256; + } } \ No newline at end of file diff --git a/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/ITestAbfsCustomEncryption.java b/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/ITestAbfsCustomEncryption.java index 32ea3f33273c6..1284b79a01ed0 100644 --- a/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/ITestAbfsCustomEncryption.java +++ b/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/ITestAbfsCustomEncryption.java @@ -104,48 +104,50 @@ public class ITestAbfsCustomEncryption extends AbstractAbfsIntegrationTest { @Parameterized.Parameter(6) public boolean isCpkResponseHdrExpected; + @Parameterized.Parameter(7) + public Boolean isCpkResponseKeyExpected = false; @Parameterized.Parameters(name = "{0} mode, {2}") public static Iterable params() { return Arrays.asList(new Object[][] { - {ENCRYPTION_CONTEXT, ENCRYPTION_CONTEXT, FSOperationType.READ, true, false, false, true}, - {ENCRYPTION_CONTEXT, ENCRYPTION_CONTEXT, FSOperationType.WRITE, false, true, false, true}, - {ENCRYPTION_CONTEXT, ENCRYPTION_CONTEXT, FSOperationType.APPEND, false, true, false, true}, - {ENCRYPTION_CONTEXT, ENCRYPTION_CONTEXT, FSOperationType.SET_ACL, false, false, false, false}, - {ENCRYPTION_CONTEXT, ENCRYPTION_CONTEXT, FSOperationType.GET_ATTR, true, false, false, true}, - {ENCRYPTION_CONTEXT, ENCRYPTION_CONTEXT, FSOperationType.SET_ATTR, false, true, false, true}, - {ENCRYPTION_CONTEXT, ENCRYPTION_CONTEXT, FSOperationType.LISTSTATUS, false, false, false, false}, - {ENCRYPTION_CONTEXT, ENCRYPTION_CONTEXT, FSOperationType.RENAME, false, false, false, false}, - {ENCRYPTION_CONTEXT, ENCRYPTION_CONTEXT, FSOperationType.DELETE, false, false, false, false}, + {ENCRYPTION_CONTEXT, ENCRYPTION_CONTEXT, FSOperationType.READ, true, false, false, true, false}, + {ENCRYPTION_CONTEXT, ENCRYPTION_CONTEXT, FSOperationType.WRITE, false, true, false, true, false}, + {ENCRYPTION_CONTEXT, ENCRYPTION_CONTEXT, FSOperationType.APPEND, false, true, false, true, false}, + {ENCRYPTION_CONTEXT, ENCRYPTION_CONTEXT, FSOperationType.SET_ACL, false, false, false, false, false}, + {ENCRYPTION_CONTEXT, ENCRYPTION_CONTEXT, FSOperationType.GET_ATTR, true, false, false, true, false}, + {ENCRYPTION_CONTEXT, ENCRYPTION_CONTEXT, FSOperationType.SET_ATTR, false, true, false, true, false}, + {ENCRYPTION_CONTEXT, ENCRYPTION_CONTEXT, FSOperationType.LISTSTATUS, false, false, false, false, true}, + {ENCRYPTION_CONTEXT, ENCRYPTION_CONTEXT, FSOperationType.RENAME, false, false, false, false, false}, + {ENCRYPTION_CONTEXT, ENCRYPTION_CONTEXT, FSOperationType.DELETE, false, false, false, false, false}, - {ENCRYPTION_CONTEXT, NONE, FSOperationType.WRITE, false, false, true, false}, - {ENCRYPTION_CONTEXT, NONE, FSOperationType.GET_ATTR, true, false, true, false}, - {ENCRYPTION_CONTEXT, NONE, FSOperationType.READ, false, false, true, false}, - {ENCRYPTION_CONTEXT, NONE, FSOperationType.SET_ATTR, false, true, true, false}, - {ENCRYPTION_CONTEXT, NONE, FSOperationType.RENAME, false, false, false, false}, - {ENCRYPTION_CONTEXT, NONE, FSOperationType.LISTSTATUS, false, false, false, false}, - {ENCRYPTION_CONTEXT, NONE, FSOperationType.DELETE, false, false, false, false}, - {ENCRYPTION_CONTEXT, NONE, FSOperationType.SET_ACL, false, false, false, false}, - {ENCRYPTION_CONTEXT, NONE, FSOperationType.SET_PERMISSION, false, false, false, false}, + {ENCRYPTION_CONTEXT, NONE, FSOperationType.WRITE, false, false, true, false, false}, + {ENCRYPTION_CONTEXT, NONE, FSOperationType.GET_ATTR, true, false, true, false, false}, + {ENCRYPTION_CONTEXT, NONE, FSOperationType.READ, false, false, true, false, false}, + {ENCRYPTION_CONTEXT, NONE, FSOperationType.SET_ATTR, false, true, true, false, false}, + {ENCRYPTION_CONTEXT, NONE, FSOperationType.RENAME, false, false, false, false, false}, + {ENCRYPTION_CONTEXT, NONE, FSOperationType.LISTSTATUS, false, false, false, false, false}, + {ENCRYPTION_CONTEXT, NONE, FSOperationType.DELETE, false, false, false, false, false}, + {ENCRYPTION_CONTEXT, NONE, FSOperationType.SET_ACL, false, false, false, false, false}, + {ENCRYPTION_CONTEXT, NONE, FSOperationType.SET_PERMISSION, false, false, false, false, false}, - {GLOBAL_KEY, GLOBAL_KEY, FSOperationType.READ, true, false, false, true}, - {GLOBAL_KEY, GLOBAL_KEY, FSOperationType.WRITE, false, true, false, true}, - {GLOBAL_KEY, GLOBAL_KEY, FSOperationType.APPEND, false, true, false, true}, - {GLOBAL_KEY, GLOBAL_KEY, FSOperationType.SET_ACL, false, false, false, false}, - {GLOBAL_KEY, GLOBAL_KEY, FSOperationType.LISTSTATUS, false, false, false, false}, - {GLOBAL_KEY, GLOBAL_KEY, FSOperationType.RENAME, false, false, false, false}, - {GLOBAL_KEY, GLOBAL_KEY, FSOperationType.DELETE, false, false, false, false}, - {GLOBAL_KEY, GLOBAL_KEY, FSOperationType.GET_ATTR, true, false, false, true}, - {GLOBAL_KEY, GLOBAL_KEY, FSOperationType.SET_ATTR, false, true, false, true}, + {GLOBAL_KEY, GLOBAL_KEY, FSOperationType.READ, true, false, false, true, false}, + {GLOBAL_KEY, GLOBAL_KEY, FSOperationType.WRITE, false, true, false, true, false}, + {GLOBAL_KEY, GLOBAL_KEY, FSOperationType.APPEND, false, true, false, true, false}, + {GLOBAL_KEY, GLOBAL_KEY, FSOperationType.SET_ACL, false, false, false, false, false}, + {GLOBAL_KEY, GLOBAL_KEY, FSOperationType.LISTSTATUS, false, false, false, false, false}, + {GLOBAL_KEY, GLOBAL_KEY, FSOperationType.RENAME, false, false, false, false, false}, + {GLOBAL_KEY, GLOBAL_KEY, FSOperationType.DELETE, false, false, false, false, false}, + {GLOBAL_KEY, GLOBAL_KEY, FSOperationType.GET_ATTR, true, false, false, true, false}, + {GLOBAL_KEY, GLOBAL_KEY, FSOperationType.SET_ATTR, false, true, false, true, false}, - {GLOBAL_KEY, NONE, FSOperationType.READ, true, false, true, true}, - {GLOBAL_KEY, NONE, FSOperationType.WRITE, false, true, true, true}, - {GLOBAL_KEY, NONE, FSOperationType.SET_ATTR, false, false, true, true}, - {GLOBAL_KEY, NONE, FSOperationType.SET_ACL, false, false, false, false}, - {GLOBAL_KEY, NONE, FSOperationType.RENAME, false, false, false, false}, - {GLOBAL_KEY, NONE, FSOperationType.LISTSTATUS, false, false, false, false}, - {GLOBAL_KEY, NONE, FSOperationType.DELETE, false, false, false, false}, - {GLOBAL_KEY, NONE, FSOperationType.SET_PERMISSION, false, false, false, false}, + {GLOBAL_KEY, NONE, FSOperationType.READ, true, false, true, true, false}, + {GLOBAL_KEY, NONE, FSOperationType.WRITE, false, true, true, true, false}, + {GLOBAL_KEY, NONE, FSOperationType.SET_ATTR, false, false, true, true, false}, + {GLOBAL_KEY, NONE, FSOperationType.SET_ACL, false, false, false, false, false}, + {GLOBAL_KEY, NONE, FSOperationType.RENAME, false, false, false, false, false}, + {GLOBAL_KEY, NONE, FSOperationType.LISTSTATUS, false, false, false, false, false}, + {GLOBAL_KEY, NONE, FSOperationType.DELETE, false, false, false, false, false}, + {GLOBAL_KEY, NONE, FSOperationType.SET_PERMISSION, false, false, false, false, false}, }); } @@ -183,8 +185,20 @@ public void testCustomEncryptionCombinations() throws Exception { .isEqualTo(cpkSHAEncoded); } } else { - Assertions.assertThat(httpOp.getResponseHeader(X_MS_ENCRYPTION_KEY_SHA256)) - .isEqualTo(null); + if(isCpkResponseKeyExpected) { + if (requestEncryptionType == ENCRYPTION_CONTEXT) { + String encryptionContext = ecp.getEncryptionContextForTest(relativePath); + String expectedKeySHA = EncodingHelper.getBase64EncodedString( + EncodingHelper.getSHA256Hash( + ecp.getEncryptionKeyForTest(encryptionContext))); + Assertions.assertThat(httpOp.getListResultSchema().paths().get(0) + .getCustomerProvidedKeySha256()).isEqualTo(expectedKeySHA); + } + } else { + Assertions.assertThat( + httpOp.getResponseHeader(X_MS_ENCRYPTION_KEY_SHA256)) + .isEqualTo(null); + } } Assertions.assertThat(httpOp.getResponseHeader(X_MS_SERVER_ENCRYPTED)) .isEqualTo(responseHeaderServerEnc? "true" : null); From adc82f67caea228f67dafc73ff7d0ef70be28583 Mon Sep 17 00:00:00 2001 From: Pranav Saxena Date: Wed, 7 Dec 2022 23:40:49 -0800 Subject: [PATCH 63/77] test for: using result of listStatus to open a file in case the file is of type ENCYPTION_CONTEXT. --- .../fs/azurebfs/AzureBlobFileSystemStore.java | 2 +- .../azurebfs/ITestAbfsCustomEncryption.java | 126 ++++++++++++------ 2 files changed, 83 insertions(+), 45 deletions(-) diff --git a/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/AzureBlobFileSystemStore.java b/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/AzureBlobFileSystemStore.java index d856ad38d9e9f..9554d72b916a5 100644 --- a/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/AzureBlobFileSystemStore.java +++ b/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/AzureBlobFileSystemStore.java @@ -1870,7 +1870,7 @@ private AbfsPerfInfo startTracking(String callerName, String calleeName) { * in a LIST or HEAD request. * The etag is included in the java serialization. */ - private static final class VersionedFileStatus extends FileStatus + static final class VersionedFileStatus extends FileStatus implements EtagSource { /** diff --git a/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/ITestAbfsCustomEncryption.java b/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/ITestAbfsCustomEncryption.java index 1284b79a01ed0..9422d27be87a9 100644 --- a/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/ITestAbfsCustomEncryption.java +++ b/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/ITestAbfsCustomEncryption.java @@ -23,10 +23,12 @@ import java.util.ArrayList; import java.util.Arrays; import java.util.Base64; +import java.util.HashSet; import java.util.Hashtable; import java.util.List; import java.util.Random; +import org.apache.hadoop.fs.FileStatus; import org.apache.hadoop.fs.PathIOException; import org.apache.hadoop.fs.azurebfs.constants.HttpHeaderConfigurations; import org.apache.hadoop.fs.azurebfs.security.EncodingHelper; @@ -51,6 +53,7 @@ import org.apache.hadoop.fs.azurebfs.services.AbfsHttpOperation; import org.apache.hadoop.fs.azurebfs.services.AbfsRestOperation; import org.apache.hadoop.fs.azurebfs.utils.EncryptionType; +import org.apache.hadoop.fs.impl.OpenFileParameters; import org.apache.hadoop.fs.permission.AclEntry; import org.apache.hadoop.fs.permission.FsPermission; import org.apache.hadoop.test.LambdaTestUtils; @@ -76,6 +79,9 @@ @RunWith(Parameterized.class) public class ITestAbfsCustomEncryption extends AbstractAbfsIntegrationTest { + + public static final String SERVER_FILE_CONTENT = "123"; + private final byte[] cpk = new byte[ENCRYPTION_KEY_LEN]; private final String cpkSHAEncoded; @@ -107,47 +113,51 @@ public class ITestAbfsCustomEncryption extends AbstractAbfsIntegrationTest { @Parameterized.Parameter(7) public Boolean isCpkResponseKeyExpected = false; + @Parameterized.Parameter(8) + public Boolean fileSystemListStatusResultToBeUsedForOpeningFile = false; + @Parameterized.Parameters(name = "{0} mode, {2}") public static Iterable params() { return Arrays.asList(new Object[][] { - {ENCRYPTION_CONTEXT, ENCRYPTION_CONTEXT, FSOperationType.READ, true, false, false, true, false}, - {ENCRYPTION_CONTEXT, ENCRYPTION_CONTEXT, FSOperationType.WRITE, false, true, false, true, false}, - {ENCRYPTION_CONTEXT, ENCRYPTION_CONTEXT, FSOperationType.APPEND, false, true, false, true, false}, - {ENCRYPTION_CONTEXT, ENCRYPTION_CONTEXT, FSOperationType.SET_ACL, false, false, false, false, false}, - {ENCRYPTION_CONTEXT, ENCRYPTION_CONTEXT, FSOperationType.GET_ATTR, true, false, false, true, false}, - {ENCRYPTION_CONTEXT, ENCRYPTION_CONTEXT, FSOperationType.SET_ATTR, false, true, false, true, false}, - {ENCRYPTION_CONTEXT, ENCRYPTION_CONTEXT, FSOperationType.LISTSTATUS, false, false, false, false, true}, - {ENCRYPTION_CONTEXT, ENCRYPTION_CONTEXT, FSOperationType.RENAME, false, false, false, false, false}, - {ENCRYPTION_CONTEXT, ENCRYPTION_CONTEXT, FSOperationType.DELETE, false, false, false, false, false}, - - {ENCRYPTION_CONTEXT, NONE, FSOperationType.WRITE, false, false, true, false, false}, - {ENCRYPTION_CONTEXT, NONE, FSOperationType.GET_ATTR, true, false, true, false, false}, - {ENCRYPTION_CONTEXT, NONE, FSOperationType.READ, false, false, true, false, false}, - {ENCRYPTION_CONTEXT, NONE, FSOperationType.SET_ATTR, false, true, true, false, false}, - {ENCRYPTION_CONTEXT, NONE, FSOperationType.RENAME, false, false, false, false, false}, - {ENCRYPTION_CONTEXT, NONE, FSOperationType.LISTSTATUS, false, false, false, false, false}, - {ENCRYPTION_CONTEXT, NONE, FSOperationType.DELETE, false, false, false, false, false}, - {ENCRYPTION_CONTEXT, NONE, FSOperationType.SET_ACL, false, false, false, false, false}, - {ENCRYPTION_CONTEXT, NONE, FSOperationType.SET_PERMISSION, false, false, false, false, false}, - - {GLOBAL_KEY, GLOBAL_KEY, FSOperationType.READ, true, false, false, true, false}, - {GLOBAL_KEY, GLOBAL_KEY, FSOperationType.WRITE, false, true, false, true, false}, - {GLOBAL_KEY, GLOBAL_KEY, FSOperationType.APPEND, false, true, false, true, false}, - {GLOBAL_KEY, GLOBAL_KEY, FSOperationType.SET_ACL, false, false, false, false, false}, - {GLOBAL_KEY, GLOBAL_KEY, FSOperationType.LISTSTATUS, false, false, false, false, false}, - {GLOBAL_KEY, GLOBAL_KEY, FSOperationType.RENAME, false, false, false, false, false}, - {GLOBAL_KEY, GLOBAL_KEY, FSOperationType.DELETE, false, false, false, false, false}, - {GLOBAL_KEY, GLOBAL_KEY, FSOperationType.GET_ATTR, true, false, false, true, false}, - {GLOBAL_KEY, GLOBAL_KEY, FSOperationType.SET_ATTR, false, true, false, true, false}, - - {GLOBAL_KEY, NONE, FSOperationType.READ, true, false, true, true, false}, - {GLOBAL_KEY, NONE, FSOperationType.WRITE, false, true, true, true, false}, - {GLOBAL_KEY, NONE, FSOperationType.SET_ATTR, false, false, true, true, false}, - {GLOBAL_KEY, NONE, FSOperationType.SET_ACL, false, false, false, false, false}, - {GLOBAL_KEY, NONE, FSOperationType.RENAME, false, false, false, false, false}, - {GLOBAL_KEY, NONE, FSOperationType.LISTSTATUS, false, false, false, false, false}, - {GLOBAL_KEY, NONE, FSOperationType.DELETE, false, false, false, false, false}, - {GLOBAL_KEY, NONE, FSOperationType.SET_PERMISSION, false, false, false, false, false}, + {ENCRYPTION_CONTEXT, ENCRYPTION_CONTEXT, FSOperationType.READ, true, false, false, true, false, false}, + {ENCRYPTION_CONTEXT, ENCRYPTION_CONTEXT, FSOperationType.READ, true, false, false, true, false, true}, + {ENCRYPTION_CONTEXT, ENCRYPTION_CONTEXT, FSOperationType.WRITE, false, true, false, true, false, false}, + {ENCRYPTION_CONTEXT, ENCRYPTION_CONTEXT, FSOperationType.APPEND, false, true, false, true, false, false}, + {ENCRYPTION_CONTEXT, ENCRYPTION_CONTEXT, FSOperationType.SET_ACL, false, false, false, false, false, false}, + {ENCRYPTION_CONTEXT, ENCRYPTION_CONTEXT, FSOperationType.GET_ATTR, true, false, false, true, false, false}, + {ENCRYPTION_CONTEXT, ENCRYPTION_CONTEXT, FSOperationType.SET_ATTR, false, true, false, true, false, false}, + {ENCRYPTION_CONTEXT, ENCRYPTION_CONTEXT, FSOperationType.LISTSTATUS, false, false, false, false, true, false}, + {ENCRYPTION_CONTEXT, ENCRYPTION_CONTEXT, FSOperationType.RENAME, false, false, false, false, false, false}, + {ENCRYPTION_CONTEXT, ENCRYPTION_CONTEXT, FSOperationType.DELETE, false, false, false, false, false, false}, + + {ENCRYPTION_CONTEXT, NONE, FSOperationType.WRITE, false, false, true, false, false, false}, + {ENCRYPTION_CONTEXT, NONE, FSOperationType.GET_ATTR, true, false, true, false, false, false}, + {ENCRYPTION_CONTEXT, NONE, FSOperationType.READ, false, false, true, false, false, false}, + {ENCRYPTION_CONTEXT, NONE, FSOperationType.SET_ATTR, false, true, true, false, false, false}, + {ENCRYPTION_CONTEXT, NONE, FSOperationType.RENAME, false, false, false, false, false, false}, + {ENCRYPTION_CONTEXT, NONE, FSOperationType.LISTSTATUS, false, false, false, false, false, false}, + {ENCRYPTION_CONTEXT, NONE, FSOperationType.DELETE, false, false, false, false, false, false}, + {ENCRYPTION_CONTEXT, NONE, FSOperationType.SET_ACL, false, false, false, false, false, false}, + {ENCRYPTION_CONTEXT, NONE, FSOperationType.SET_PERMISSION, false, false, false, false, false, false}, + + {GLOBAL_KEY, GLOBAL_KEY, FSOperationType.READ, true, false, false, true, false, false}, + {GLOBAL_KEY, GLOBAL_KEY, FSOperationType.WRITE, false, true, false, true, false, false}, + {GLOBAL_KEY, GLOBAL_KEY, FSOperationType.APPEND, false, true, false, true, false, false}, + {GLOBAL_KEY, GLOBAL_KEY, FSOperationType.SET_ACL, false, false, false, false, false, false}, + {GLOBAL_KEY, GLOBAL_KEY, FSOperationType.LISTSTATUS, false, false, false, false, false, false}, + {GLOBAL_KEY, GLOBAL_KEY, FSOperationType.RENAME, false, false, false, false, false, false}, + {GLOBAL_KEY, GLOBAL_KEY, FSOperationType.DELETE, false, false, false, false, false, false}, + {GLOBAL_KEY, GLOBAL_KEY, FSOperationType.GET_ATTR, true, false, false, true, false, false}, + {GLOBAL_KEY, GLOBAL_KEY, FSOperationType.SET_ATTR, false, true, false, true, false, false}, + + {GLOBAL_KEY, NONE, FSOperationType.READ, true, false, true, true, false, false}, + {GLOBAL_KEY, NONE, FSOperationType.WRITE, false, true, true, true, false, false}, + {GLOBAL_KEY, NONE, FSOperationType.SET_ATTR, false, false, true, true, false, false}, + {GLOBAL_KEY, NONE, FSOperationType.SET_ACL, false, false, false, false, false, false}, + {GLOBAL_KEY, NONE, FSOperationType.RENAME, false, false, false, false, false, false}, + {GLOBAL_KEY, NONE, FSOperationType.LISTSTATUS, false, false, false, false, false, false}, + {GLOBAL_KEY, NONE, FSOperationType.DELETE, false, false, false, false, false, false}, + {GLOBAL_KEY, NONE, FSOperationType.SET_PERMISSION, false, false, false, false, false, false}, }); } @@ -252,11 +262,39 @@ private AbfsRestOperation callOperation(AzureBlobFileSystem fs, String path = testPath.toString(); switch (operation) { case READ: - TracingContext tracingContext = getTestTracingContext(fs, true); - AbfsHttpOperation statusOp = client.getPathStatus(path, false, - tracingContext, null).getResult(); - return client.read(path, 0, new byte[5], 0, 5, statusOp.getResponseHeader(HttpHeaderConfigurations.ETAG), - null, encryptionAdapter, tracingContext); + if (!fileSystemListStatusResultToBeUsedForOpeningFile + || fileEncryptionType != ENCRYPTION_CONTEXT) { + TracingContext tracingContext = getTestTracingContext(fs, true); + AbfsHttpOperation statusOp = client.getPathStatus(path, false, + tracingContext, null).getResult(); + return client.read(path, 0, new byte[5], 0, 5, + statusOp.getResponseHeader(HttpHeaderConfigurations.ETAG), + null, encryptionAdapter, tracingContext); + } else { + /* + * In this block, its tested scenario is: + * 1.Create a file. + * 2.Fetch List of VersionFileStatus objects by listStatus API of the AzureBlobFileSystem. + * 3.Use the context value in the VersionFileStatus object for making read API call to backend. + * 4.Assert for no exception and get response. + * */ + FileStatus status = fs.listStatus(testPath)[0]; + Assertions.assertThat(status) + .isInstanceOf(AzureBlobFileSystemStore.VersionedFileStatus.class); + + Assertions.assertThat( + ((AzureBlobFileSystemStore.VersionedFileStatus) status).getEncryptionContext()) + .isNotNull(); + + try (FSDataInputStream in = fs.openFileWithOptions(testPath, + new OpenFileParameters().withMandatoryKeys(new HashSet<>()) + .withStatus(fs.listStatus(testPath)[0])).get()) { + byte[] readBuffer = new byte[3]; + Assertions.assertThat(in.read(readBuffer)).isGreaterThan(0); + Assertions.assertThat(readBuffer).isEqualTo(SERVER_FILE_CONTENT.getBytes()); + return null; + } + } case WRITE: return client.flush(path, 3, false, false, null, null, encryptionAdapter, getTestTracingContext(fs, false)); @@ -374,7 +412,7 @@ private EncryptionContextProvider createEncryptedFile(Path testPath) throws Exce } String relativePath = fs.getAbfsStore().getRelativePath(testPath); try (FSDataOutputStream out = fs.create(new Path(relativePath))) { - out.write("123".getBytes()); + out.write(SERVER_FILE_CONTENT.getBytes()); } // verify file is encrypted by calling getPathStatus (with properties) // without encryption headers in request From 241fb9bbb2aad1f5f7fe25fbe6b9e67aa23bcc9f Mon Sep 17 00:00:00 2001 From: Pranav Saxena Date: Sun, 11 Dec 2022 20:02:29 -0800 Subject: [PATCH 64/77] Remove public documentation for Encryption Options. --- .../hadoop-azure/src/site/markdown/abfs.md | 32 ------------------- 1 file changed, 32 deletions(-) diff --git a/hadoop-tools/hadoop-azure/src/site/markdown/abfs.md b/hadoop-tools/hadoop-azure/src/site/markdown/abfs.md index c3c2439cfd1fe..35d360556047e 100644 --- a/hadoop-tools/hadoop-azure/src/site/markdown/abfs.md +++ b/hadoop-tools/hadoop-azure/src/site/markdown/abfs.md @@ -888,38 +888,6 @@ specified SSL channel mode. Value should be of the enum DelegatingSSLSocketFactory.SSLChannelMode. The default value will be DelegatingSSLSocketFactory.SSLChannelMode.Default. -###
    Encryption Options -Only one of the following two options can be configured. If config values of -both types are set, ABFS driver will throw an exception. If using the global -key type, ensure both pre-computed values are provided. - -#### Customer-Provided Global Key -A global encryption key can be configured by providing the following -pre-computed values. The key will be applied to any new files created post -setting the configuration, and will be required in the requests to read ro -modify the contents of the files. - -`fs.azure.encryption.encoded.client-provided-key`: The Base64 encoded version -of the 256-bit encryption key. - -`fs.azure.encryption.encoded.client-provided-key-sha`: The Base64 encoded -version of the SHA256 has of the 256-bit encryption key. - -#### Encryption Context Provider - -ABFS driver supports an interface called `EncryptionContextProvider` that -can be used as a plugin for clients to provide custom implementations for -the encryption framework. This framework allows for an `encryptionContext` -and an `encryptionKey` to be generated by the EncryptionContextProvider for -a file to be created. The server keeps track of the encryptionContext for -each file. To perform subsequent operations such as read on the encrypted file, -ABFS driver will fetch the corresponding encryption key from the -EncryptionContextProvider implementation by providing the encryptionContext -string retrieved from a GetFileStatus request to the server. - -`fs.azure.encryption.context.provider.type`: The canonical name of the class -implementing EncryptionContextProvider. - ### Server Options When the config `fs.azure.io.read.tolerate.concurrent.append` is made true, the If-Match header sent to the server for read calls will be set as * otherwise the From 9daea681cc1f32aae238cbdc02737cd8db671c9f Mon Sep 17 00:00:00 2001 From: Pranav Saxena Date: Sun, 11 Dec 2022 21:59:14 -0800 Subject: [PATCH 65/77] remove non required import in abfsOutputStream; itestAbfsCustomEncryption: javadocs in createEncryptedFile, tearDown() impl. --- .../fs/azurebfs/services/AbfsOutputStream.java | 2 -- .../fs/azurebfs/ITestAbfsCustomEncryption.java | 13 ++++++++++++- 2 files changed, 12 insertions(+), 3 deletions(-) diff --git a/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/services/AbfsOutputStream.java b/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/services/AbfsOutputStream.java index 7625c254529b2..9dc0573979a1c 100644 --- a/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/services/AbfsOutputStream.java +++ b/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/services/AbfsOutputStream.java @@ -52,8 +52,6 @@ import org.apache.hadoop.fs.StreamCapabilities; import org.apache.hadoop.fs.Syncable; -import javax.security.auth.DestroyFailedException; - import static org.apache.hadoop.fs.azurebfs.constants.FileSystemConfigurations.STREAM_ID_LEN; import static org.apache.hadoop.fs.azurebfs.services.AbfsErrors.ERR_WRITE_WITHOUT_LEASE; import static org.apache.hadoop.fs.impl.StoreImplementationUtils.isProbeForSyncable; diff --git a/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/ITestAbfsCustomEncryption.java b/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/ITestAbfsCustomEncryption.java index 9422d27be87a9..bb92f6090883a 100644 --- a/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/ITestAbfsCustomEncryption.java +++ b/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/ITestAbfsCustomEncryption.java @@ -401,6 +401,16 @@ private AzureBlobFileSystem getOrCreateFS() throws Exception { } } + /** + * Creates a file in the server with values for the following keys: + *
      + *
    1. x-ms-encryption-key: for ENCRYPTION_CONTEXT, GLOBAL
    2. + *
    3. x-ms-encryption-key-sha256: for ENCRYPTION_CONTEXT, GLOBAL
    4. + *
    5. x-ms-encryption-context: for ENCRYPTION_CONTEXT
    6. + *
    + * Returns in case of ENCRYPTION_CONTEXT the encryptionProvider object which + * was used to create the x-ms-encryption-context value used for creating the file. + * */ private EncryptionContextProvider createEncryptedFile(Path testPath) throws Exception { AzureBlobFileSystem fs; if (getFileSystem().getAbfsClient().getEncryptionType() == fileEncryptionType) { @@ -431,7 +441,8 @@ private EncryptionContextProvider createEncryptedFile(Path testPath) throws Exce } @Override - protected void finalize() throws Throwable { + public void teardown() throws Exception { + super.teardown(); for(AzureBlobFileSystem azureBlobFileSystem : fileSystemsOpenedInTest) { azureBlobFileSystem.close(); } From 2313790ca10ea02a764f01a17b64314e84c27d05 Mon Sep 17 00:00:00 2001 From: Pranav Saxena Date: Sun, 11 Dec 2022 23:14:02 -0800 Subject: [PATCH 66/77] small refactors --- .../org/apache/hadoop/fs/azurebfs/services/AbfsClient.java | 7 +++---- .../org/apache/hadoop/fs/azurebfs/utils/NamespaceUtil.java | 1 - 2 files changed, 3 insertions(+), 5 deletions(-) diff --git a/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/services/AbfsClient.java b/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/services/AbfsClient.java index 79a0b919c6dbc..255558cf3f356 100644 --- a/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/services/AbfsClient.java +++ b/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/services/AbfsClient.java @@ -1284,11 +1284,10 @@ public synchronized String getAccessToken() throws IOException { private synchronized Boolean getIsNamespaceEnabled(TracingContext tracingContext) throws AzureBlobFileSystemException { - if(isNamespaceEnabled != null) { - return isNamespaceEnabled; + if (isNamespaceEnabled == null) { + isNamespaceEnabled = NamespaceUtil.isNamespaceEnabled(this, + tracingContext); } - - isNamespaceEnabled = NamespaceUtil.isNamespaceEnabled(this, tracingContext); return isNamespaceEnabled; } diff --git a/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/utils/NamespaceUtil.java b/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/utils/NamespaceUtil.java index 2a81867f1135a..8d5a4ea822e71 100644 --- a/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/utils/NamespaceUtil.java +++ b/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/utils/NamespaceUtil.java @@ -38,5 +38,4 @@ public static Boolean isNamespaceEnabled(final AbfsClient abfsClient, final Trac } return isNamespaceEnabled; } - } From aee1c7b2dda963b4945f56ebbf07382c5e9e7949 Mon Sep 17 00:00:00 2001 From: Pranav Saxena Date: Sun, 11 Dec 2022 23:31:48 -0800 Subject: [PATCH 67/77] refactor mock verification so that parameter can be null --- .../hadoop/fs/azurebfs/services/TestAbfsInputStream.java | 3 ++- 1 file changed, 2 insertions(+), 1 deletion(-) diff --git a/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/services/TestAbfsInputStream.java b/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/services/TestAbfsInputStream.java index 6d1ecc2936127..82a7e7045e6ee 100644 --- a/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/services/TestAbfsInputStream.java +++ b/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/services/TestAbfsInputStream.java @@ -49,6 +49,7 @@ import static org.mockito.ArgumentMatchers.any; import static org.mockito.ArgumentMatchers.anyBoolean; import static org.mockito.ArgumentMatchers.anyString; +import static org.mockito.ArgumentMatchers.nullable; import static org.mockito.Mockito.doReturn; import static org.mockito.Mockito.doThrow; import static org.mockito.Mockito.mock; @@ -244,7 +245,7 @@ private void checkGetPathStatusCalls(Path testFile, FileStatus fileStatus, tracingContext); verify(mockClient, times(1).description( "GetPathStatus should be invoked when FileStatus not provided")) - .getPathStatus(anyString(), anyBoolean(), any(TracingContext.class), any(EncryptionAdapter.class)); + .getPathStatus(anyString(), anyBoolean(), any(TracingContext.class), nullable(EncryptionAdapter.class)); Mockito.reset(mockClient); //clears invocation count for next test case } From bb45ae9c1c1b80d7a8c768679f2fa06bc4b84e55 Mon Sep 17 00:00:00 2001 From: Pranav Saxena Date: Mon, 12 Dec 2022 19:53:03 -0800 Subject: [PATCH 68/77] assertions changed for abfsclient namespaceEnabled check --- .../hadoop/fs/azurebfs/services/AbfsClient.java | 8 ++++---- .../fs/azurebfs/ITestAbfsNetworkStatistics.java | 6 ++++++ .../azurebfs/ITestAzureBlobFileSystemCreate.java | 15 +++++++++++---- 3 files changed, 21 insertions(+), 8 deletions(-) diff --git a/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/services/AbfsClient.java b/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/services/AbfsClient.java index 255558cf3f356..852dbc5511755 100644 --- a/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/services/AbfsClient.java +++ b/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/services/AbfsClient.java @@ -25,7 +25,6 @@ import java.net.MalformedURLException; import java.net.URL; import java.net.URLEncoder; -import java.nio.charset.StandardCharsets; import java.util.ArrayList; import java.util.List; import java.util.Locale; @@ -35,8 +34,6 @@ import java.util.concurrent.TimeUnit; import org.apache.hadoop.classification.VisibleForTesting; -import org.apache.hadoop.fs.PathIOException; -import org.apache.hadoop.fs.azurebfs.contracts.exceptions.AbfsRestOperationException; import org.apache.hadoop.fs.azurebfs.utils.NamespaceUtil; import org.apache.hadoop.fs.store.LogExactlyOnce; import org.apache.hadoop.fs.azurebfs.AzureBlobFileSystemStore.Permissions; @@ -69,7 +66,6 @@ import org.apache.hadoop.fs.azurebfs.contracts.services.AppendRequestParameters; import org.apache.hadoop.fs.azurebfs.oauth2.AccessTokenProvider; import org.apache.hadoop.fs.azurebfs.utils.TracingContext; -import org.apache.hadoop.fs.Path; import org.apache.hadoop.security.ssl.DelegatingSSLSocketFactory; import org.apache.hadoop.util.concurrent.HadoopExecutors; @@ -1299,6 +1295,10 @@ public EncryptionContextProvider getEncryptionContextProvider() { return encryptionContextProvider; } + public Boolean getIsNamespaceEnabled() { + return this.isNamespaceEnabled; + } + @VisibleForTesting String initializeUserAgent(final AbfsConfiguration abfsConfiguration, final String sslProviderName) { diff --git a/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/ITestAbfsNetworkStatistics.java b/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/ITestAbfsNetworkStatistics.java index 66b8da89572a1..e68833f3374e1 100644 --- a/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/ITestAbfsNetworkStatistics.java +++ b/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/ITestAbfsNetworkStatistics.java @@ -66,6 +66,12 @@ public void testAbfsHttpSendStatistics() throws IOException { long expectedRequestsSent = metricMap.get(SEND_REQUESTS.getStatName()); long expectedBytesSent = 0; + if(fs.getAbfsClient().getIsNamespaceEnabled() == null) { + //AbfsClient would initialise isNamespaceEnabled field for which it would + //make call getAclStatus. + expectedConnectionsMade++; + } + // -------------------------------------------------------------------- // Operation: Creating AbfsOutputStream try (AbfsOutputStream out = createAbfsOutputStreamWithFlushEnabled(fs, diff --git a/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/ITestAzureBlobFileSystemCreate.java b/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/ITestAzureBlobFileSystemCreate.java index b98d3cc9cd4b8..25dfa943ccffc 100644 --- a/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/ITestAzureBlobFileSystemCreate.java +++ b/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/ITestAzureBlobFileSystemCreate.java @@ -261,6 +261,13 @@ public void testCreateFileOverwrite(boolean enableConditionalCreateOverwrite) long totalConnectionMadeBeforeTest = fs.getInstrumentationMap() .get(CONNECTIONS_MADE.getStatName()); + int aclCall = 0; + if(fs.getAbfsClient().getIsNamespaceEnabled() == null) { + //AbfsClient would initialise isNamespaceEnabled field for which it would + //make call getAclStatus. + aclCall = 1; + } + int createRequestCount = 0; final Path nonOverwriteFile = new Path("/NonOverwriteTest_FileName_" + UUID.randomUUID().toString()); @@ -274,7 +281,7 @@ public void testCreateFileOverwrite(boolean enableConditionalCreateOverwrite) assertAbfsStatistics( CONNECTIONS_MADE, - totalConnectionMadeBeforeTest + createRequestCount, + totalConnectionMadeBeforeTest + createRequestCount + aclCall, fs.getInstrumentationMap()); // Case 2: Not Overwrite - File pre-exists @@ -290,7 +297,7 @@ public void testCreateFileOverwrite(boolean enableConditionalCreateOverwrite) assertAbfsStatistics( CONNECTIONS_MADE, - totalConnectionMadeBeforeTest + createRequestCount, + totalConnectionMadeBeforeTest + createRequestCount + aclCall, fs.getInstrumentationMap()); final Path overwriteFilePath = new Path("/OverwriteTest_FileName_" @@ -305,7 +312,7 @@ public void testCreateFileOverwrite(boolean enableConditionalCreateOverwrite) assertAbfsStatistics( CONNECTIONS_MADE, - totalConnectionMadeBeforeTest + createRequestCount, + totalConnectionMadeBeforeTest + createRequestCount + aclCall, fs.getInstrumentationMap()); // Case 4: Overwrite - File pre-exists @@ -327,7 +334,7 @@ public void testCreateFileOverwrite(boolean enableConditionalCreateOverwrite) assertAbfsStatistics( CONNECTIONS_MADE, - totalConnectionMadeBeforeTest + createRequestCount, + totalConnectionMadeBeforeTest + createRequestCount + aclCall, fs.getInstrumentationMap()); } From 3c6eddbde2ff1ebb25972f110f556b31c4681bc9 Mon Sep 17 00:00:00 2001 From: Pranav Saxena Date: Mon, 12 Dec 2022 20:17:20 -0800 Subject: [PATCH 69/77] Revert "assertions changed for abfsclient namespaceEnabled check" This reverts commit bb45ae9c1c1b80d7a8c768679f2fa06bc4b84e55. --- .../hadoop/fs/azurebfs/services/AbfsClient.java | 8 ++++---- .../fs/azurebfs/ITestAbfsNetworkStatistics.java | 6 ------ .../azurebfs/ITestAzureBlobFileSystemCreate.java | 15 ++++----------- 3 files changed, 8 insertions(+), 21 deletions(-) diff --git a/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/services/AbfsClient.java b/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/services/AbfsClient.java index 852dbc5511755..255558cf3f356 100644 --- a/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/services/AbfsClient.java +++ b/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/services/AbfsClient.java @@ -25,6 +25,7 @@ import java.net.MalformedURLException; import java.net.URL; import java.net.URLEncoder; +import java.nio.charset.StandardCharsets; import java.util.ArrayList; import java.util.List; import java.util.Locale; @@ -34,6 +35,8 @@ import java.util.concurrent.TimeUnit; import org.apache.hadoop.classification.VisibleForTesting; +import org.apache.hadoop.fs.PathIOException; +import org.apache.hadoop.fs.azurebfs.contracts.exceptions.AbfsRestOperationException; import org.apache.hadoop.fs.azurebfs.utils.NamespaceUtil; import org.apache.hadoop.fs.store.LogExactlyOnce; import org.apache.hadoop.fs.azurebfs.AzureBlobFileSystemStore.Permissions; @@ -66,6 +69,7 @@ import org.apache.hadoop.fs.azurebfs.contracts.services.AppendRequestParameters; import org.apache.hadoop.fs.azurebfs.oauth2.AccessTokenProvider; import org.apache.hadoop.fs.azurebfs.utils.TracingContext; +import org.apache.hadoop.fs.Path; import org.apache.hadoop.security.ssl.DelegatingSSLSocketFactory; import org.apache.hadoop.util.concurrent.HadoopExecutors; @@ -1295,10 +1299,6 @@ public EncryptionContextProvider getEncryptionContextProvider() { return encryptionContextProvider; } - public Boolean getIsNamespaceEnabled() { - return this.isNamespaceEnabled; - } - @VisibleForTesting String initializeUserAgent(final AbfsConfiguration abfsConfiguration, final String sslProviderName) { diff --git a/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/ITestAbfsNetworkStatistics.java b/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/ITestAbfsNetworkStatistics.java index e68833f3374e1..66b8da89572a1 100644 --- a/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/ITestAbfsNetworkStatistics.java +++ b/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/ITestAbfsNetworkStatistics.java @@ -66,12 +66,6 @@ public void testAbfsHttpSendStatistics() throws IOException { long expectedRequestsSent = metricMap.get(SEND_REQUESTS.getStatName()); long expectedBytesSent = 0; - if(fs.getAbfsClient().getIsNamespaceEnabled() == null) { - //AbfsClient would initialise isNamespaceEnabled field for which it would - //make call getAclStatus. - expectedConnectionsMade++; - } - // -------------------------------------------------------------------- // Operation: Creating AbfsOutputStream try (AbfsOutputStream out = createAbfsOutputStreamWithFlushEnabled(fs, diff --git a/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/ITestAzureBlobFileSystemCreate.java b/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/ITestAzureBlobFileSystemCreate.java index 25dfa943ccffc..b98d3cc9cd4b8 100644 --- a/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/ITestAzureBlobFileSystemCreate.java +++ b/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/ITestAzureBlobFileSystemCreate.java @@ -261,13 +261,6 @@ public void testCreateFileOverwrite(boolean enableConditionalCreateOverwrite) long totalConnectionMadeBeforeTest = fs.getInstrumentationMap() .get(CONNECTIONS_MADE.getStatName()); - int aclCall = 0; - if(fs.getAbfsClient().getIsNamespaceEnabled() == null) { - //AbfsClient would initialise isNamespaceEnabled field for which it would - //make call getAclStatus. - aclCall = 1; - } - int createRequestCount = 0; final Path nonOverwriteFile = new Path("/NonOverwriteTest_FileName_" + UUID.randomUUID().toString()); @@ -281,7 +274,7 @@ public void testCreateFileOverwrite(boolean enableConditionalCreateOverwrite) assertAbfsStatistics( CONNECTIONS_MADE, - totalConnectionMadeBeforeTest + createRequestCount + aclCall, + totalConnectionMadeBeforeTest + createRequestCount, fs.getInstrumentationMap()); // Case 2: Not Overwrite - File pre-exists @@ -297,7 +290,7 @@ public void testCreateFileOverwrite(boolean enableConditionalCreateOverwrite) assertAbfsStatistics( CONNECTIONS_MADE, - totalConnectionMadeBeforeTest + createRequestCount + aclCall, + totalConnectionMadeBeforeTest + createRequestCount, fs.getInstrumentationMap()); final Path overwriteFilePath = new Path("/OverwriteTest_FileName_" @@ -312,7 +305,7 @@ public void testCreateFileOverwrite(boolean enableConditionalCreateOverwrite) assertAbfsStatistics( CONNECTIONS_MADE, - totalConnectionMadeBeforeTest + createRequestCount + aclCall, + totalConnectionMadeBeforeTest + createRequestCount, fs.getInstrumentationMap()); // Case 4: Overwrite - File pre-exists @@ -334,7 +327,7 @@ public void testCreateFileOverwrite(boolean enableConditionalCreateOverwrite) assertAbfsStatistics( CONNECTIONS_MADE, - totalConnectionMadeBeforeTest + createRequestCount + aclCall, + totalConnectionMadeBeforeTest + createRequestCount, fs.getInstrumentationMap()); } From a2315924e15080482ac42d71fb586691df25957a Mon Sep 17 00:00:00 2001 From: Pranav Saxena Date: Mon, 12 Dec 2022 20:46:11 -0800 Subject: [PATCH 70/77] AbfsClientUtils to set certain fields for test. AbfsClient to have package-protected methods for setting these fields. --- .../fs/azurebfs/services/AbfsClient.java | 11 +++--- .../azurebfs/AbstractAbfsIntegrationTest.java | 2 ++ .../azurebfs/ITestAbfsCustomEncryption.java | 3 +- .../ITestAzureBlobFileSystemCreate.java | 2 ++ .../fs/azurebfs/services/AbfsClientUtils.java | 34 +++++++++++++++++++ 5 files changed, 46 insertions(+), 6 deletions(-) create mode 100644 hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/services/AbfsClientUtils.java diff --git a/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/services/AbfsClient.java b/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/services/AbfsClient.java index 255558cf3f356..e5b07f699347c 100644 --- a/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/services/AbfsClient.java +++ b/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/services/AbfsClient.java @@ -25,7 +25,6 @@ import java.net.MalformedURLException; import java.net.URL; import java.net.URLEncoder; -import java.nio.charset.StandardCharsets; import java.util.ArrayList; import java.util.List; import java.util.Locale; @@ -35,8 +34,6 @@ import java.util.concurrent.TimeUnit; import org.apache.hadoop.classification.VisibleForTesting; -import org.apache.hadoop.fs.PathIOException; -import org.apache.hadoop.fs.azurebfs.contracts.exceptions.AbfsRestOperationException; import org.apache.hadoop.fs.azurebfs.utils.NamespaceUtil; import org.apache.hadoop.fs.store.LogExactlyOnce; import org.apache.hadoop.fs.azurebfs.AzureBlobFileSystemStore.Permissions; @@ -69,7 +66,6 @@ import org.apache.hadoop.fs.azurebfs.contracts.services.AppendRequestParameters; import org.apache.hadoop.fs.azurebfs.oauth2.AccessTokenProvider; import org.apache.hadoop.fs.azurebfs.utils.TracingContext; -import org.apache.hadoop.fs.Path; import org.apache.hadoop.security.ssl.DelegatingSSLSocketFactory; import org.apache.hadoop.util.concurrent.HadoopExecutors; @@ -1368,10 +1364,15 @@ public SASTokenProvider getSasTokenProvider() { } @VisibleForTesting - public void setEncryptionContextProvider(EncryptionContextProvider provider) { + void setEncryptionContextProvider(EncryptionContextProvider provider) { encryptionContextProvider = provider; } + @VisibleForTesting + void setIsNamespaceEnabled(final Boolean isNamespaceEnabled) { + this.isNamespaceEnabled = isNamespaceEnabled; + } + /** * Getter for abfsCounters from AbfsClient. * @return AbfsCounters instance. diff --git a/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/AbstractAbfsIntegrationTest.java b/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/AbstractAbfsIntegrationTest.java index 4a5507526c3a1..a594ffb00c2df 100644 --- a/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/AbstractAbfsIntegrationTest.java +++ b/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/AbstractAbfsIntegrationTest.java @@ -39,6 +39,7 @@ import org.apache.hadoop.fs.azurebfs.contracts.exceptions.AzureBlobFileSystemException; import org.apache.hadoop.fs.azurebfs.security.AbfsDelegationTokenManager; import org.apache.hadoop.fs.azurebfs.services.AbfsClient; +import org.apache.hadoop.fs.azurebfs.services.AbfsClientUtils; import org.apache.hadoop.fs.azurebfs.services.AbfsOutputStream; import org.apache.hadoop.fs.azurebfs.services.AuthType; import org.apache.hadoop.fs.azure.AzureNativeFileSystemStore; @@ -199,6 +200,7 @@ public void setup() throws Exception { wasb = new NativeAzureFileSystem(azureNativeFileSystemStore); wasb.initialize(wasbUri, rawConfig); } + AbfsClientUtils.setIsNamespaceEnabled(abfs.getAbfsClient(), true); } @After diff --git a/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/ITestAbfsCustomEncryption.java b/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/ITestAbfsCustomEncryption.java index bb92f6090883a..3bab8face5d0a 100644 --- a/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/ITestAbfsCustomEncryption.java +++ b/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/ITestAbfsCustomEncryption.java @@ -32,6 +32,7 @@ import org.apache.hadoop.fs.PathIOException; import org.apache.hadoop.fs.azurebfs.constants.HttpHeaderConfigurations; import org.apache.hadoop.fs.azurebfs.security.EncodingHelper; +import org.apache.hadoop.fs.azurebfs.services.AbfsClientUtils; import org.apache.hadoop.fs.azurebfs.utils.TracingContext; import org.assertj.core.api.Assertions; import org.junit.Assume; @@ -230,7 +231,7 @@ private AbfsRestOperation callOperation(AzureBlobFileSystem fs, Path testPath, EncryptionContextProvider ecp) throws Exception { AbfsClient client = fs.getAbfsClient(); - client.setEncryptionContextProvider(ecp); + AbfsClientUtils.setEncryptionContextProvider(client, ecp); if (isExceptionCase) { LambdaTestUtils.intercept(IOException.class, () -> { switch (operation) { diff --git a/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/ITestAzureBlobFileSystemCreate.java b/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/ITestAzureBlobFileSystemCreate.java index b98d3cc9cd4b8..c50778d873e66 100644 --- a/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/ITestAzureBlobFileSystemCreate.java +++ b/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/ITestAzureBlobFileSystemCreate.java @@ -34,6 +34,7 @@ import org.apache.hadoop.fs.FSDataOutputStream; import org.apache.hadoop.fs.Path; import org.apache.hadoop.fs.azurebfs.security.EncryptionAdapter; +import org.apache.hadoop.fs.azurebfs.services.AbfsClientUtils; import org.apache.hadoop.fs.permission.FsAction; import org.apache.hadoop.fs.permission.FsPermission; import org.apache.hadoop.test.GenericTestUtils; @@ -257,6 +258,7 @@ public void testCreateFileOverwrite(boolean enableConditionalCreateOverwrite) final AzureBlobFileSystem fs = (AzureBlobFileSystem) FileSystem.newInstance(currentFs.getUri(), config); + AbfsClientUtils.setIsNamespaceEnabled(fs.getAbfsClient(), true); long totalConnectionMadeBeforeTest = fs.getInstrumentationMap() .get(CONNECTIONS_MADE.getStatName()); diff --git a/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/services/AbfsClientUtils.java b/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/services/AbfsClientUtils.java new file mode 100644 index 0000000000000..5c015267ef65e --- /dev/null +++ b/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/services/AbfsClientUtils.java @@ -0,0 +1,34 @@ +/** + * 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.fs.azurebfs.services; + +import org.apache.hadoop.fs.azurebfs.extensions.EncryptionContextProvider; + +public class AbfsClientUtils { + private AbfsClientUtils() { + + } + public static void setIsNamespaceEnabled(final AbfsClient abfsClient, final Boolean isNamespaceEnabled) { + abfsClient.setIsNamespaceEnabled(isNamespaceEnabled); + } + + public static void setEncryptionContextProvider(final AbfsClient abfsClient, final EncryptionContextProvider provider) { + abfsClient.setEncryptionContextProvider(provider); + } +} From 6ab375c8bb6c6412c47d8d31fe50920e4deba8d6 Mon Sep 17 00:00:00 2001 From: Pranav Saxena Date: Wed, 21 Dec 2022 20:31:37 -0800 Subject: [PATCH 71/77] added javadocs; indentation; comments on https://github.com/sumangala-patki/hadoop/pull/12 --- .../fs/azurebfs/AzureBlobFileSystemStore.java | 28 ++++++++++++++++--- .../services/ListResultEntrySchema.java | 2 +- .../fs/azurebfs/services/AbfsClient.java | 5 ++-- .../fs/azurebfs/utils/EncryptionType.java | 8 ++++-- .../fs/azurebfs/utils/NamespaceUtil.java | 23 ++++++++++++++- .../azurebfs/ITestAbfsCustomEncryption.java | 8 ++++++ 6 files changed, 63 insertions(+), 11 deletions(-) diff --git a/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/AzureBlobFileSystemStore.java b/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/AzureBlobFileSystemStore.java index 9554d72b916a5..8d3c8649329a4 100644 --- a/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/AzureBlobFileSystemStore.java +++ b/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/AzureBlobFileSystemStore.java @@ -54,11 +54,11 @@ import java.util.concurrent.ExecutorService; import java.util.concurrent.TimeUnit; +import org.apache.hadoop.classification.VisibleForTesting; import org.apache.hadoop.fs.PathIOException; import org.apache.hadoop.fs.azurebfs.extensions.EncryptionContextProvider; import org.apache.hadoop.fs.azurebfs.security.EncryptionAdapter; import org.apache.hadoop.fs.azurebfs.utils.EncryptionType; -import org.apache.hadoop.classification.VisibleForTesting; import org.apache.hadoop.fs.azurebfs.utils.NamespaceUtil; import org.apache.hadoop.util.Preconditions; import org.apache.hadoop.thirdparty.com.google.common.base.Strings; @@ -466,7 +466,7 @@ public Hashtable getPathStatus(final Path path, final AbfsRestOperation op = client .getPathStatus(relativePath, true, tracingContext, encryptionAdapter); perfInfo.registerResult(op.getResult()); - if(encryptionAdapter != null) { + if (encryptionAdapter != null) { encryptionAdapter.destroy(); } @@ -480,9 +480,29 @@ public Hashtable getPathStatus(final Path path, } } + /** + * Creates an object of {@link org.apache.hadoop.fs.azurebfs.security.EncryptionAdapter} + * from a file path. It calls {@link org.apache.hadoop.fs.azurebfs.services.AbfsClient + * #getPathStatus(String, boolean, TracingContext, EncryptionAdapter)} method to get + * contextValue (x-ms-encryption-context) from the server. The contextValue is passed + * to the constructor of EncryptionAdapter to create the required object of + * EncryptionAdapter. + * @param path Path of the file for which the object of EncryptionAdapter is required. + * @return
      + *
    • + * null: if encryptionType is not of type + * {@link org.apache.hadoop.fs.azurebfs.utils.EncryptionType#ENCRYPTION_CONTEXT}. + *
    • + *
    • + * new object of EncryptionAdapter containing required encryptionKeys for the give file: + * if encryptionType is of type {@link org.apache.hadoop.fs.azurebfs.utils.EncryptionType#ENCRYPTION_CONTEXT}. + *
    • + * + *
    + * */ private EncryptionAdapter createEncryptionAdapterFromServerStoreContext(final String path, final TracingContext tracingContext) throws IOException { - if(client.getEncryptionType() != EncryptionType.ENCRYPTION_CONTEXT) { + if (client.getEncryptionType() != EncryptionType.ENCRYPTION_CONTEXT) { return null; } final String responseHeaderEncryptionContext = client.getPathStatus(path, @@ -585,7 +605,7 @@ public OutputStream createFile(final Path path, } EncryptionAdapter encryptionAdapter = null; - if(client.getEncryptionType() == EncryptionType.ENCRYPTION_CONTEXT) { + if (client.getEncryptionType() == EncryptionType.ENCRYPTION_CONTEXT) { encryptionAdapter = new EncryptionAdapter( client.getEncryptionContextProvider(), getRelativePath(path)); } diff --git a/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/contracts/services/ListResultEntrySchema.java b/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/contracts/services/ListResultEntrySchema.java index d35b9cfea238c..77f52e4a2bbd1 100644 --- a/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/contracts/services/ListResultEntrySchema.java +++ b/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/contracts/services/ListResultEntrySchema.java @@ -265,4 +265,4 @@ public String getXMsEncryptionContext() { public String getCustomerProvidedKeySha256() { return customerProvidedKeySha256; } -} \ No newline at end of file +} diff --git a/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/services/AbfsClient.java b/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/services/AbfsClient.java index e5b07f699347c..782ef7dcb8bb5 100644 --- a/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/services/AbfsClient.java +++ b/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/services/AbfsClient.java @@ -251,7 +251,7 @@ private void addEncryptionKeyRequestHeaders(String path, List requestHeaders, boolean isCreateFileRequest, EncryptionAdapter encryptionAdapter, TracingContext tracingContext) throws IOException { - if(!getIsNamespaceEnabled(tracingContext)) { + if (!getIsNamespaceEnabled(tracingContext)) { return; } String encodedKey, encodedKeySHA256; @@ -281,7 +281,8 @@ private void addEncryptionKeyRequestHeaders(String path, } requestHeaders.add(new AbfsHttpHeader(X_MS_ENCRYPTION_KEY, encodedKey)); - requestHeaders.add(new AbfsHttpHeader(X_MS_ENCRYPTION_KEY_SHA256, encodedKeySHA256)); + requestHeaders.add( + new AbfsHttpHeader(X_MS_ENCRYPTION_KEY_SHA256, encodedKeySHA256)); requestHeaders.add(new AbfsHttpHeader(X_MS_ENCRYPTION_ALGORITHM, SERVER_SIDE_ENCRYPTION_ALGORITHM)); } diff --git a/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/utils/EncryptionType.java b/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/utils/EncryptionType.java index 880dc8fc5a279..3cd414a7666db 100644 --- a/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/utils/EncryptionType.java +++ b/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/utils/EncryptionType.java @@ -20,9 +20,11 @@ /** * Enum EncryptionType to represent the level of encryption applied. - * GLOBAL_KEY: encrypt all files with the same client-provided key - * ENCRYPTION_CONTEXT: uses client-provided implementation to generate keys - * NONE: encryption handled entirely at server + *
      + *
    1. GLOBAL_KEY: encrypt all files with the same client-provided key.
    2. + *
    3. ENCRYPTION_CONTEXT: uses client-provided implementation to generate keys.
    4. + *
    5. NONE: encryption handled entirely at server.
    6. + *
    */ public enum EncryptionType { GLOBAL_KEY, diff --git a/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/utils/NamespaceUtil.java b/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/utils/NamespaceUtil.java index 8d5a4ea822e71..2b5eac03c2360 100644 --- a/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/utils/NamespaceUtil.java +++ b/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/utils/NamespaceUtil.java @@ -10,6 +10,10 @@ import org.apache.hadoop.fs.azurebfs.contracts.exceptions.AzureBlobFileSystemException; import org.apache.hadoop.fs.azurebfs.services.AbfsClient; +/** + * Utility class to provide method which can return if the account is namespace + * enabled or not. + * */ public class NamespaceUtil { public static final Logger LOG = LoggerFactory.getLogger(NamespaceUtil.class); @@ -18,7 +22,24 @@ private NamespaceUtil() { } - public static Boolean isNamespaceEnabled(final AbfsClient abfsClient, final TracingContext tracingContext) + /** + * Return if the account used in the provided abfsClient object namespace enabled + * or not. + * It would call {@link org.apache.hadoop.fs.azurebfs.services.AbfsClient#getAclStatus(String, TracingContext)}. + *
      + *
    1. + * If the API call is successful, then the account is namespace enabled. + *
    2. + *
    3. + * If the server returns with {@link java.net.HttpURLConnection#HTTP_BAD_REQUEST}, the account is non-namespace enabled. + *
    4. + *
    5. + * If the server call gets some other exception, then the method would throw the exception. + *
    6. + *
    + * */ + public static Boolean isNamespaceEnabled(final AbfsClient abfsClient, + final TracingContext tracingContext) throws AzureBlobFileSystemException { Boolean isNamespaceEnabled; try { diff --git a/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/ITestAbfsCustomEncryption.java b/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/ITestAbfsCustomEncryption.java index 3bab8face5d0a..05bc62c72ccbe 100644 --- a/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/ITestAbfsCustomEncryption.java +++ b/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/ITestAbfsCustomEncryption.java @@ -108,9 +108,17 @@ public class ITestAbfsCustomEncryption extends AbstractAbfsIntegrationTest { @Parameterized.Parameter(5) public boolean isExceptionCase; + /** + * Boolean value to indicate that the server response would have header related + * to CPK and the test would need to assert its value. + * */ @Parameterized.Parameter(6) public boolean isCpkResponseHdrExpected; + /** + * Boolean value to indicate that the server response would have fields related + * to CPK and the test would need to assert its value. + * */ @Parameterized.Parameter(7) public Boolean isCpkResponseKeyExpected = false; From aaf158225cce63e70719a44b6f8fba153eda8ef2 Mon Sep 17 00:00:00 2001 From: Pranav Saxena Date: Wed, 21 Dec 2022 20:33:22 -0800 Subject: [PATCH 72/77] minor change --- .../org/apache/hadoop/fs/azurebfs/AzureBlobFileSystemStore.java | 1 - 1 file changed, 1 deletion(-) diff --git a/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/AzureBlobFileSystemStore.java b/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/AzureBlobFileSystemStore.java index 8d3c8649329a4..4b270c3b6de8d 100644 --- a/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/AzureBlobFileSystemStore.java +++ b/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/AzureBlobFileSystemStore.java @@ -497,7 +497,6 @@ public Hashtable getPathStatus(final Path path, * new object of EncryptionAdapter containing required encryptionKeys for the give file: * if encryptionType is of type {@link org.apache.hadoop.fs.azurebfs.utils.EncryptionType#ENCRYPTION_CONTEXT}. * - * * * */ private EncryptionAdapter createEncryptionAdapterFromServerStoreContext(final String path, From 4002da665850f02d446736398b4950743f2406ef Mon Sep 17 00:00:00 2001 From: Pranav Saxena Date: Wed, 28 Dec 2022 21:58:05 -0800 Subject: [PATCH 73/77] minor refactor --- hadoop-tools/hadoop-azure/.gitignore | 1 - .../java/org/apache/hadoop/fs/azurebfs/AbfsConfiguration.java | 1 - 2 files changed, 2 deletions(-) diff --git a/hadoop-tools/hadoop-azure/.gitignore b/hadoop-tools/hadoop-azure/.gitignore index f06a4aa88be1a..a7d3296d0fff3 100644 --- a/hadoop-tools/hadoop-azure/.gitignore +++ b/hadoop-tools/hadoop-azure/.gitignore @@ -1,7 +1,6 @@ .checkstyle bin/ src/test/resources/combinationConfigFiles -src/test/resources/accountSettings src/test/resources/abfs-combination-test-configs.xml dev-support/testlogs src/test/resources/accountSettings/* diff --git a/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/AbfsConfiguration.java b/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/AbfsConfiguration.java index efb55a55878fd..0942251d27a11 100644 --- a/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/AbfsConfiguration.java +++ b/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/AbfsConfiguration.java @@ -949,7 +949,6 @@ public SASTokenProvider getSASTokenProvider() throws AzureBlobFileSystemExceptio } public EncryptionContextProvider createEncryptionContextProvider() { - try { String configKey = FS_AZURE_ENCRYPTION_CONTEXT_PROVIDER_TYPE; if (get(configKey) == null) { From 1990a4824c9a38a69eb5cbcb6ef841e1f23f88f4 Mon Sep 17 00:00:00 2001 From: Pranav Saxena Date: Wed, 28 Dec 2022 22:12:30 -0800 Subject: [PATCH 74/77] testAbfsInputStream method signature change refactor --- .../hadoop/fs/azurebfs/services/TestAbfsInputStream.java | 3 ++- 1 file changed, 2 insertions(+), 1 deletion(-) diff --git a/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/services/TestAbfsInputStream.java b/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/services/TestAbfsInputStream.java index 27ac2ff485937..0da0629687c31 100644 --- a/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/services/TestAbfsInputStream.java +++ b/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/services/TestAbfsInputStream.java @@ -520,7 +520,8 @@ public void testStreamPurgeDuringReadAheadCallExecuting() throws Exception { .when(client) .read(any(String.class), any(Long.class), any(byte[].class), any(Integer.class), any(Integer.class), any(String.class), - any(String.class), any(TracingContext.class)); + any(String.class), nullable(EncryptionAdapter.class), + any(TracingContext.class)); final ReadBufferManager readBufferManager = ReadBufferManager.getBufferManager(); From a2472d56c4f9abea4023df01ddb2cd156fb85cf7 Mon Sep 17 00:00:00 2001 From: Pranav Saxena Date: Thu, 29 Dec 2022 01:54:24 -0800 Subject: [PATCH 75/77] spotbugs + javadocs --- .../azurebfs/security/EncryptionAdapter.java | 9 ++++++++ .../fs/azurebfs/services/AbfsClient.java | 8 +++++-- .../fs/azurebfs/utils/NamespaceUtil.java | 23 +++++++++++++++++++ 3 files changed, 38 insertions(+), 2 deletions(-) diff --git a/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/security/EncryptionAdapter.java b/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/security/EncryptionAdapter.java index 8020c138310f4..7180cfb0a8ca9 100644 --- a/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/security/EncryptionAdapter.java +++ b/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/security/EncryptionAdapter.java @@ -50,6 +50,11 @@ public class EncryptionAdapter { * convert the string into a byte-array. The converted byte-array would be used * by the implementation of EncryptionContextProvider to create byte-array of * encryptionKey. + * @param provider developer's implementation of {@link EncryptionContextProvider} + * @param path Path for which encryptionContext and encryptionKeys to be stored + * in the object + * @param encryptionContext encryptionContext for the path stored in the backend + * @throws IOException * */ public EncryptionAdapter(EncryptionContextProvider provider, String path, byte[] encryptionContext) throws IOException { @@ -67,6 +72,10 @@ public EncryptionAdapter(EncryptionContextProvider provider, String path, * on the server, encryptionContext is not there for the path. Implementation * of the EncryptionContextProvider would be used to create encryptionContext * from the path. + * @param provider developer's implementation of {@link EncryptionContextProvider} + * @param path file path for which encryptionContext and encryptionKeys to be + * created and stored + * @throws IOException * */ public EncryptionAdapter(EncryptionContextProvider provider, String path) throws IOException { diff --git a/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/services/AbfsClient.java b/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/services/AbfsClient.java index f3180c1f3ff97..d818525fba4a4 100644 --- a/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/services/AbfsClient.java +++ b/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/services/AbfsClient.java @@ -421,6 +421,10 @@ public AbfsRestOperation deleteFilesystem(TracingContext tracingContext) throws * {@link org.apache.hadoop.fs.azurebfs.extensions.EncryptionContextProvider} * @param tracingContext: Object of {@link org.apache.hadoop.fs.azurebfs.utils.TracingContext} * correlating to the current fs.create() request. + * @return object of {@link AbfsRestOperation} which contain all the information + * about the communication with the server. The information is in + * {@link AbfsRestOperation#getResult()} + * @throws IOException * */ public AbfsRestOperation createPath(final String path, final boolean isFile, final boolean overwrite, final Permissions permissions, @@ -1293,8 +1297,8 @@ public synchronized String getAccessToken() throws IOException { private synchronized Boolean getIsNamespaceEnabled(TracingContext tracingContext) throws AzureBlobFileSystemException { if (isNamespaceEnabled == null) { - isNamespaceEnabled = NamespaceUtil.isNamespaceEnabled(this, - tracingContext); + setIsNamespaceEnabled(NamespaceUtil.isNamespaceEnabled(this, + tracingContext)); } return isNamespaceEnabled; } diff --git a/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/utils/NamespaceUtil.java b/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/utils/NamespaceUtil.java index 2b5eac03c2360..8c6e03fa675e2 100644 --- a/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/utils/NamespaceUtil.java +++ b/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/utils/NamespaceUtil.java @@ -1,3 +1,21 @@ +/** + * 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.fs.azurebfs.utils; import java.net.HttpURLConnection; @@ -37,6 +55,11 @@ private NamespaceUtil() { * If the server call gets some other exception, then the method would throw the exception. * * + * @param abfsClient client for which namespace-enabled to be checked. + * @param tracingContext object to correlate Store requests. + * @return if the account corresponding to the given client is namespace-enabled + * or not. + * @throws AzureBlobFileSystemException * */ public static Boolean isNamespaceEnabled(final AbfsClient abfsClient, final TracingContext tracingContext) From 95b0f9d77fad1702c2748d8d0159dac584cc9c34 Mon Sep 17 00:00:00 2001 From: Pranav Saxena Date: Sun, 1 Jan 2023 19:36:58 -0800 Subject: [PATCH 76/77] javadocs refactor --- .../hadoop/fs/azurebfs/security/EncryptionAdapter.java | 6 ++++-- .../org/apache/hadoop/fs/azurebfs/services/AbfsClient.java | 3 ++- .../org/apache/hadoop/fs/azurebfs/utils/NamespaceUtil.java | 5 ++++- 3 files changed, 10 insertions(+), 4 deletions(-) diff --git a/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/security/EncryptionAdapter.java b/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/security/EncryptionAdapter.java index 7180cfb0a8ca9..138011791fedb 100644 --- a/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/security/EncryptionAdapter.java +++ b/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/security/EncryptionAdapter.java @@ -54,7 +54,8 @@ public class EncryptionAdapter { * @param path Path for which encryptionContext and encryptionKeys to be stored * in the object * @param encryptionContext encryptionContext for the path stored in the backend - * @throws IOException + * @throws IOException throws back the exception it receives from the + * {@link EncryptionAdapter#computeKeys()} method call. * */ public EncryptionAdapter(EncryptionContextProvider provider, String path, byte[] encryptionContext) throws IOException { @@ -75,7 +76,8 @@ public EncryptionAdapter(EncryptionContextProvider provider, String path, * @param provider developer's implementation of {@link EncryptionContextProvider} * @param path file path for which encryptionContext and encryptionKeys to be * created and stored - * @throws IOException + * @throws IOException throws back the exception it receives from the method call + * to {@link EncryptionContextProvider} object. * */ public EncryptionAdapter(EncryptionContextProvider provider, String path) throws IOException { diff --git a/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/services/AbfsClient.java b/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/services/AbfsClient.java index d818525fba4a4..7a94ddf516e3d 100644 --- a/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/services/AbfsClient.java +++ b/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/services/AbfsClient.java @@ -424,7 +424,8 @@ public AbfsRestOperation deleteFilesystem(TracingContext tracingContext) throws * @return object of {@link AbfsRestOperation} which contain all the information * about the communication with the server. The information is in * {@link AbfsRestOperation#getResult()} - * @throws IOException + * @throws IOException throws back the exception it receives from the + * {@link AbfsRestOperation#execute(TracingContext)} method call. * */ public AbfsRestOperation createPath(final String path, final boolean isFile, final boolean overwrite, final Permissions permissions, diff --git a/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/utils/NamespaceUtil.java b/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/utils/NamespaceUtil.java index 8c6e03fa675e2..40bf788c5a85f 100644 --- a/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/utils/NamespaceUtil.java +++ b/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/utils/NamespaceUtil.java @@ -59,7 +59,10 @@ private NamespaceUtil() { * @param tracingContext object to correlate Store requests. * @return if the account corresponding to the given client is namespace-enabled * or not. - * @throws AzureBlobFileSystemException + * @throws AzureBlobFileSystemException throws back the exception the method receives + * from the {@link AbfsClient#getAclStatus(String, TracingContext)}. In case it gets + * {@link AbfsRestOperationException}, it checks if the exception statusCode is + * BAD_REQUEST or not. If not, then it will pass the exception to the calling method. * */ public static Boolean isNamespaceEnabled(final AbfsClient abfsClient, final TracingContext tracingContext) From 0cd8c8c1cda3a82c742a91b54c9cb29c62b83f82 Mon Sep 17 00:00:00 2001 From: Pranav Saxena Date: Sun, 1 Jan 2023 19:48:32 -0800 Subject: [PATCH 77/77] checkstyle --- .../apache/hadoop/fs/azurebfs/AzureBlobFileSystemStore.java | 2 +- .../apache/hadoop/fs/azurebfs/security/EncodingHelper.java | 4 ++++ .../org/apache/hadoop/fs/azurebfs/utils/NamespaceUtil.java | 2 +- .../hadoop/fs/azurebfs/ITestAbfsCustomEncryption.java | 6 +++--- .../apache/hadoop/fs/azurebfs/services/AbfsClientUtils.java | 2 +- 5 files changed, 10 insertions(+), 6 deletions(-) diff --git a/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/AzureBlobFileSystemStore.java b/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/AzureBlobFileSystemStore.java index d8145a03f442a..19a738dcbc78a 100644 --- a/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/AzureBlobFileSystemStore.java +++ b/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/AzureBlobFileSystemStore.java @@ -545,7 +545,7 @@ public void setPathProperties(final Path path, final AbfsRestOperation op = client .setPathProperties(getRelativePath(path), commaSeparatedProperties, tracingContext, encryptionAdapter); - if(encryptionAdapter != null) { + if (encryptionAdapter != null) { encryptionAdapter.destroy(); } perfInfo.registerResult(op.getResult()).registerSuccess(true); diff --git a/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/security/EncodingHelper.java b/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/security/EncodingHelper.java index 9a90cf05bb927..6a0f2cdc5034b 100644 --- a/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/security/EncodingHelper.java +++ b/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/security/EncodingHelper.java @@ -28,6 +28,10 @@ * */ public final class EncodingHelper { + private EncodingHelper() { + + } + public static byte[] getSHA256Hash(byte[] key) { try { final MessageDigest digester = MessageDigest.getInstance("SHA-256"); diff --git a/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/utils/NamespaceUtil.java b/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/utils/NamespaceUtil.java index 40bf788c5a85f..f8055befe82c2 100644 --- a/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/utils/NamespaceUtil.java +++ b/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/utils/NamespaceUtil.java @@ -32,7 +32,7 @@ * Utility class to provide method which can return if the account is namespace * enabled or not. * */ -public class NamespaceUtil { +public final class NamespaceUtil { public static final Logger LOG = LoggerFactory.getLogger(NamespaceUtil.class); diff --git a/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/ITestAbfsCustomEncryption.java b/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/ITestAbfsCustomEncryption.java index 05bc62c72ccbe..130bf94b0f477 100644 --- a/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/ITestAbfsCustomEncryption.java +++ b/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/ITestAbfsCustomEncryption.java @@ -204,7 +204,7 @@ public void testCustomEncryptionCombinations() throws Exception { .isEqualTo(cpkSHAEncoded); } } else { - if(isCpkResponseKeyExpected) { + if (isCpkResponseKeyExpected) { if (requestEncryptionType == ENCRYPTION_CONTEXT) { String encryptionContext = ecp.getEncryptionContextForTest(relativePath); String expectedKeySHA = EncodingHelper.getBase64EncodedString( @@ -348,7 +348,7 @@ private AbfsRestOperation callOperation(AzureBlobFileSystem fs, private EncryptionAdapter createEncryptionAdapterFromServerStoreContext(final String path, final TracingContext tracingContext, final AbfsClient client) throws IOException { - if(client.getEncryptionType() != ENCRYPTION_CONTEXT) { + if (client.getEncryptionType() != ENCRYPTION_CONTEXT) { return null; } final String responseHeaderEncryptionContext = client.getPathStatus(path, @@ -452,7 +452,7 @@ private EncryptionContextProvider createEncryptedFile(Path testPath) throws Exce @Override public void teardown() throws Exception { super.teardown(); - for(AzureBlobFileSystem azureBlobFileSystem : fileSystemsOpenedInTest) { + for (AzureBlobFileSystem azureBlobFileSystem : fileSystemsOpenedInTest) { azureBlobFileSystem.close(); } } diff --git a/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/services/AbfsClientUtils.java b/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/services/AbfsClientUtils.java index 5c015267ef65e..e7dbf208c9b06 100644 --- a/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/services/AbfsClientUtils.java +++ b/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/services/AbfsClientUtils.java @@ -20,7 +20,7 @@ import org.apache.hadoop.fs.azurebfs.extensions.EncryptionContextProvider; -public class AbfsClientUtils { +public final class AbfsClientUtils { private AbfsClientUtils() { }