[GitHub] [hadoop] ajfabbri commented on a change in pull request #843: HADOOP-15183 S3Guard store becomes inconsistent after partial failure of rename

2019-05-29 Thread GitBox
ajfabbri commented on a change in pull request #843: HADOOP-15183 S3Guard store 
becomes inconsistent after partial failure of rename
URL: https://github.com/apache/hadoop/pull/843#discussion_r288854348
 
 

 ##
 File path: 
hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/S3AFileSystem.java
 ##
 @@ -1225,130 +1301,292 @@ private boolean innerRename(Path source, Path dest)
   }
 }
 
-// If we have a MetadataStore, track deletions/creations.
-Collection srcPaths = null;
-List dstMetas = null;
-if (hasMetadataStore()) {
-  srcPaths = new HashSet<>(); // srcPaths need fast look up before put
-  dstMetas = new ArrayList<>();
-}
-// TODO S3Guard HADOOP-13761: retries when source paths are not visible yet
+// Validation completed: time to begin the operation.
+// The store-specific rename operation is used to keep the store
+// to date with the in-progress operation.
+// for the null store, these are all no-ops.
+final RenameTracker renameTracker =
+metadataStore.initiateRenameOperation(
+createStoreContext(),
+src, srcStatus, dest);
+final AtomicLong bytesCopied = new AtomicLong();
+int renameParallelLimit = RENAME_PARALLEL_LIMIT;
+final List> activeCopies =
+new ArrayList<>(renameParallelLimit);
+// aggregate operation to wait for the copies to complete then reset
+// the list.
+final FunctionsRaisingIOE.FunctionRaisingIOE
+completeActiveCopies = (String reason) -> {
+  LOG.debug("Waiting for {} active copies to complete: {}",
+  activeCopies.size(), reason);
+  waitForCompletion(activeCopies);
+  activeCopies.clear();
+  return null;
+};
+
 // TODO S3Guard: performance: mark destination dirs as authoritative
 
 // Ok! Time to start
-if (srcStatus.isFile()) {
-  LOG.debug("rename: renaming file {} to {}", src, dst);
-  long length = srcStatus.getLen();
-  S3ObjectAttributes objectAttributes =
-  createObjectAttributes(srcStatus.getPath(),
-  srcStatus.getETag(), srcStatus.getVersionId());
-  S3AReadOpContext readContext = createReadContext(srcStatus, inputPolicy,
-  changeDetectionPolicy, readAhead);
-  if (dstStatus != null && dstStatus.isDirectory()) {
-String newDstKey = maybeAddTrailingSlash(dstKey);
-String filename =
-srcKey.substring(pathToKey(src.getParent()).length()+1);
-newDstKey = newDstKey + filename;
-CopyResult copyResult = copyFile(srcKey, newDstKey, length,
-objectAttributes, readContext);
-S3Guard.addMoveFile(metadataStore, srcPaths, dstMetas, src,
-keyToQualifiedPath(newDstKey), length, getDefaultBlockSize(dst),
-username, copyResult.getETag(), copyResult.getVersionId());
+try {
+  if (srcStatus.isFile()) {
+// the source is a file.
+Path copyDestinationPath = dst;
+String copyDestinationKey = dstKey;
+S3ObjectAttributes sourceAttributes =
+createObjectAttributes(srcStatus);
+S3AReadOpContext readContext = createReadContext(srcStatus, 
inputPolicy,
+changeDetectionPolicy, readAhead);
+if (dstStatus != null && dstStatus.isDirectory()) {
+  // destination is a directory: build the final destination underneath
+  String newDstKey = maybeAddTrailingSlash(dstKey);
+  String filename =
+  srcKey.substring(pathToKey(src.getParent()).length() + 1);
+  newDstKey = newDstKey + filename;
+  copyDestinationKey = newDstKey;
+  copyDestinationPath = keyToQualifiedPath(newDstKey);
+}
+// destination either does not exist or is a file to overwrite.
+LOG.debug("rename: renaming file {} to {}", src, copyDestinationPath);
+copySourceAndUpdateTracker(renameTracker,
+src,
+srcKey,
+sourceAttributes,
+readContext,
+copyDestinationPath,
+copyDestinationKey,
+false);
+bytesCopied.addAndGet(srcStatus.getLen());
+ // delete the source
+deleteObjectAtPath(src, srcKey, true);
+// and update the tracker
+renameTracker.sourceObjectsDeleted(Lists.newArrayList(src));
   } else {
-CopyResult copyResult = copyFile(srcKey, dstKey, srcStatus.getLen(),
-objectAttributes, readContext);
-S3Guard.addMoveFile(metadataStore, srcPaths, dstMetas, src, dst,
-length, getDefaultBlockSize(dst), username,
-copyResult.getETag(), copyResult.getVersionId());
-  }
-  innerDelete(srcStatus, false);
-} else {
-  LOG.debug("rename: renaming directory {} to {}", src, dst);
-
-  // This is a directory to directory copy
-  dstKey = maybeAddTrailingSlash(dstKey);
-  srcKey = maybeAddTrailingSlash(srcKey);
+

[GitHub] [hadoop] ajfabbri commented on a change in pull request #843: HADOOP-15183 S3Guard store becomes inconsistent after partial failure of rename

2019-05-29 Thread GitBox
ajfabbri commented on a change in pull request #843: HADOOP-15183 S3Guard store 
becomes inconsistent after partial failure of rename
URL: https://github.com/apache/hadoop/pull/843#discussion_r288790499
 
 

 ##
 File path: 
hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/S3AFileSystem.java
 ##
 @@ -730,21 +779,29 @@ public S3AEncryptionMethods 
getServerSideEncryptionAlgorithm() {
 
   /**
* Demand create the directory allocator, then create a temporary file.
+   * This does not mark the file for deletion when a process is exits.
 
 Review comment:
   nit: /process is exits/process exits/


This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.
 
For queries about this service, please contact Infrastructure at:
us...@infra.apache.org


With regards,
Apache Git Services

-
To unsubscribe, e-mail: common-issues-unsubscr...@hadoop.apache.org
For additional commands, e-mail: common-issues-h...@hadoop.apache.org



[GitHub] [hadoop] ajfabbri commented on a change in pull request #843: HADOOP-15183 S3Guard store becomes inconsistent after partial failure of rename

2019-05-29 Thread GitBox
ajfabbri commented on a change in pull request #843: HADOOP-15183 S3Guard store 
becomes inconsistent after partial failure of rename
URL: https://github.com/apache/hadoop/pull/843#discussion_r288790196
 
 

 ##
 File path: 
hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/S3AFileSystem.java
 ##
 @@ -232,8 +248,33 @@
   /** Principal who created the FS; recorded during initialization. */
   private UserGroupInformation owner;
 
-  // The maximum number of entries that can be deleted in any call to s3
+  /**
+   * The maximum number of entries that can be deleted in any bulk delete
+   * call to S3 {@value}.
+   */
   private static final int MAX_ENTRIES_TO_DELETE = 1000;
+
+  /**
+   * This is an arbitrary value: {@value}.
+   * It declares how many parallel copy operations
+   * in a single rename can be queued before the operation pauses
+   * and awaits completion.
+   * A very large value wouldn't just starve other threads from
+   * performing work, there's a risk that the S3 store itself would
+   * throttle operations (which all go to the same shard).
+   * It is not currently configurable just to avoid people choosing values
+   * which work on a microbenchmark (single rename, no other work, ...)
+   * but don't scale well to execution in a large process against a common
+   * store, all while separate processes are working with the same shard
+   * of storage.
+   *
+   * It should be a factor of {@link #MAX_ENTRIES_TO_DELETE} so that
+   * all copies will have finished before deletion is contemplated.
+   * (There's always a block for that, it just makes more sense to
+   * perform the bulk delete after another block of copies have completed).
+   */
+  public static final int RENAME_PARALLEL_LIMIT = 10;
 
 Review comment:
   Yep, we can always make this tunable later if we care.


This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.
 
For queries about this service, please contact Infrastructure at:
us...@infra.apache.org


With regards,
Apache Git Services

-
To unsubscribe, e-mail: common-issues-unsubscr...@hadoop.apache.org
For additional commands, e-mail: common-issues-h...@hadoop.apache.org



[GitHub] [hadoop] ajfabbri commented on a change in pull request #843: HADOOP-15183 S3Guard store becomes inconsistent after partial failure of rename

2019-05-29 Thread GitBox
ajfabbri commented on a change in pull request #843: HADOOP-15183 S3Guard store 
becomes inconsistent after partial failure of rename
URL: https://github.com/apache/hadoop/pull/843#discussion_r288847579
 
 

 ##
 File path: 
hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/S3AFileSystem.java
 ##
 @@ -1225,130 +1301,292 @@ private boolean innerRename(Path source, Path dest)
   }
 }
 
-// If we have a MetadataStore, track deletions/creations.
-Collection srcPaths = null;
-List dstMetas = null;
-if (hasMetadataStore()) {
-  srcPaths = new HashSet<>(); // srcPaths need fast look up before put
-  dstMetas = new ArrayList<>();
-}
-// TODO S3Guard HADOOP-13761: retries when source paths are not visible yet
+// Validation completed: time to begin the operation.
+// The store-specific rename operation is used to keep the store
+// to date with the in-progress operation.
+// for the null store, these are all no-ops.
+final RenameTracker renameTracker =
+metadataStore.initiateRenameOperation(
+createStoreContext(),
+src, srcStatus, dest);
+final AtomicLong bytesCopied = new AtomicLong();
+int renameParallelLimit = RENAME_PARALLEL_LIMIT;
+final List> activeCopies =
+new ArrayList<>(renameParallelLimit);
+// aggregate operation to wait for the copies to complete then reset
+// the list.
+final FunctionsRaisingIOE.FunctionRaisingIOE
+completeActiveCopies = (String reason) -> {
+  LOG.debug("Waiting for {} active copies to complete: {}",
+  activeCopies.size(), reason);
+  waitForCompletion(activeCopies);
+  activeCopies.clear();
+  return null;
+};
+
 // TODO S3Guard: performance: mark destination dirs as authoritative
 
 // Ok! Time to start
-if (srcStatus.isFile()) {
-  LOG.debug("rename: renaming file {} to {}", src, dst);
-  long length = srcStatus.getLen();
-  S3ObjectAttributes objectAttributes =
-  createObjectAttributes(srcStatus.getPath(),
-  srcStatus.getETag(), srcStatus.getVersionId());
-  S3AReadOpContext readContext = createReadContext(srcStatus, inputPolicy,
-  changeDetectionPolicy, readAhead);
-  if (dstStatus != null && dstStatus.isDirectory()) {
-String newDstKey = maybeAddTrailingSlash(dstKey);
-String filename =
-srcKey.substring(pathToKey(src.getParent()).length()+1);
-newDstKey = newDstKey + filename;
-CopyResult copyResult = copyFile(srcKey, newDstKey, length,
-objectAttributes, readContext);
-S3Guard.addMoveFile(metadataStore, srcPaths, dstMetas, src,
-keyToQualifiedPath(newDstKey), length, getDefaultBlockSize(dst),
-username, copyResult.getETag(), copyResult.getVersionId());
+try {
+  if (srcStatus.isFile()) {
+// the source is a file.
+Path copyDestinationPath = dst;
+String copyDestinationKey = dstKey;
+S3ObjectAttributes sourceAttributes =
+createObjectAttributes(srcStatus);
+S3AReadOpContext readContext = createReadContext(srcStatus, 
inputPolicy,
+changeDetectionPolicy, readAhead);
+if (dstStatus != null && dstStatus.isDirectory()) {
+  // destination is a directory: build the final destination underneath
+  String newDstKey = maybeAddTrailingSlash(dstKey);
+  String filename =
+  srcKey.substring(pathToKey(src.getParent()).length() + 1);
+  newDstKey = newDstKey + filename;
+  copyDestinationKey = newDstKey;
+  copyDestinationPath = keyToQualifiedPath(newDstKey);
+}
+// destination either does not exist or is a file to overwrite.
+LOG.debug("rename: renaming file {} to {}", src, copyDestinationPath);
+copySourceAndUpdateTracker(renameTracker,
+src,
+srcKey,
+sourceAttributes,
+readContext,
+copyDestinationPath,
+copyDestinationKey,
+false);
+bytesCopied.addAndGet(srcStatus.getLen());
+ // delete the source
+deleteObjectAtPath(src, srcKey, true);
+// and update the tracker
+renameTracker.sourceObjectsDeleted(Lists.newArrayList(src));
   } else {
-CopyResult copyResult = copyFile(srcKey, dstKey, srcStatus.getLen(),
-objectAttributes, readContext);
-S3Guard.addMoveFile(metadataStore, srcPaths, dstMetas, src, dst,
-length, getDefaultBlockSize(dst), username,
-copyResult.getETag(), copyResult.getVersionId());
-  }
-  innerDelete(srcStatus, false);
-} else {
-  LOG.debug("rename: renaming directory {} to {}", src, dst);
-
-  // This is a directory to directory copy
-  dstKey = maybeAddTrailingSlash(dstKey);
-  srcKey = maybeAddTrailingSlash(srcKey);
+

[GitHub] [hadoop] ajfabbri commented on a change in pull request #843: HADOOP-15183 S3Guard store becomes inconsistent after partial failure of rename

2019-05-29 Thread GitBox
ajfabbri commented on a change in pull request #843: HADOOP-15183 S3Guard store 
becomes inconsistent after partial failure of rename
URL: https://github.com/apache/hadoop/pull/843#discussion_r288854508
 
 

 ##
 File path: 
hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/S3AFileSystem.java
 ##
 @@ -1225,130 +1301,292 @@ private boolean innerRename(Path source, Path dest)
   }
 }
 
-// If we have a MetadataStore, track deletions/creations.
-Collection srcPaths = null;
-List dstMetas = null;
-if (hasMetadataStore()) {
-  srcPaths = new HashSet<>(); // srcPaths need fast look up before put
-  dstMetas = new ArrayList<>();
-}
-// TODO S3Guard HADOOP-13761: retries when source paths are not visible yet
+// Validation completed: time to begin the operation.
+// The store-specific rename operation is used to keep the store
+// to date with the in-progress operation.
+// for the null store, these are all no-ops.
+final RenameTracker renameTracker =
+metadataStore.initiateRenameOperation(
+createStoreContext(),
+src, srcStatus, dest);
+final AtomicLong bytesCopied = new AtomicLong();
+int renameParallelLimit = RENAME_PARALLEL_LIMIT;
+final List> activeCopies =
+new ArrayList<>(renameParallelLimit);
+// aggregate operation to wait for the copies to complete then reset
+// the list.
+final FunctionsRaisingIOE.FunctionRaisingIOE
+completeActiveCopies = (String reason) -> {
+  LOG.debug("Waiting for {} active copies to complete: {}",
+  activeCopies.size(), reason);
+  waitForCompletion(activeCopies);
+  activeCopies.clear();
+  return null;
+};
+
 // TODO S3Guard: performance: mark destination dirs as authoritative
 
 // Ok! Time to start
-if (srcStatus.isFile()) {
-  LOG.debug("rename: renaming file {} to {}", src, dst);
-  long length = srcStatus.getLen();
-  S3ObjectAttributes objectAttributes =
-  createObjectAttributes(srcStatus.getPath(),
-  srcStatus.getETag(), srcStatus.getVersionId());
-  S3AReadOpContext readContext = createReadContext(srcStatus, inputPolicy,
-  changeDetectionPolicy, readAhead);
-  if (dstStatus != null && dstStatus.isDirectory()) {
-String newDstKey = maybeAddTrailingSlash(dstKey);
-String filename =
-srcKey.substring(pathToKey(src.getParent()).length()+1);
-newDstKey = newDstKey + filename;
-CopyResult copyResult = copyFile(srcKey, newDstKey, length,
-objectAttributes, readContext);
-S3Guard.addMoveFile(metadataStore, srcPaths, dstMetas, src,
-keyToQualifiedPath(newDstKey), length, getDefaultBlockSize(dst),
-username, copyResult.getETag(), copyResult.getVersionId());
+try {
+  if (srcStatus.isFile()) {
+// the source is a file.
+Path copyDestinationPath = dst;
+String copyDestinationKey = dstKey;
+S3ObjectAttributes sourceAttributes =
+createObjectAttributes(srcStatus);
+S3AReadOpContext readContext = createReadContext(srcStatus, 
inputPolicy,
+changeDetectionPolicy, readAhead);
+if (dstStatus != null && dstStatus.isDirectory()) {
+  // destination is a directory: build the final destination underneath
+  String newDstKey = maybeAddTrailingSlash(dstKey);
+  String filename =
+  srcKey.substring(pathToKey(src.getParent()).length() + 1);
+  newDstKey = newDstKey + filename;
+  copyDestinationKey = newDstKey;
+  copyDestinationPath = keyToQualifiedPath(newDstKey);
+}
+// destination either does not exist or is a file to overwrite.
+LOG.debug("rename: renaming file {} to {}", src, copyDestinationPath);
+copySourceAndUpdateTracker(renameTracker,
+src,
+srcKey,
+sourceAttributes,
+readContext,
+copyDestinationPath,
+copyDestinationKey,
+false);
+bytesCopied.addAndGet(srcStatus.getLen());
+ // delete the source
+deleteObjectAtPath(src, srcKey, true);
+// and update the tracker
+renameTracker.sourceObjectsDeleted(Lists.newArrayList(src));
   } else {
-CopyResult copyResult = copyFile(srcKey, dstKey, srcStatus.getLen(),
-objectAttributes, readContext);
-S3Guard.addMoveFile(metadataStore, srcPaths, dstMetas, src, dst,
-length, getDefaultBlockSize(dst), username,
-copyResult.getETag(), copyResult.getVersionId());
-  }
-  innerDelete(srcStatus, false);
-} else {
-  LOG.debug("rename: renaming directory {} to {}", src, dst);
-
-  // This is a directory to directory copy
-  dstKey = maybeAddTrailingSlash(dstKey);
-  srcKey = maybeAddTrailingSlash(srcKey);
+

[GitHub] [hadoop] ajfabbri commented on a change in pull request #843: HADOOP-15183 S3Guard store becomes inconsistent after partial failure of rename

2019-05-29 Thread GitBox
ajfabbri commented on a change in pull request #843: HADOOP-15183 S3Guard store 
becomes inconsistent after partial failure of rename
URL: https://github.com/apache/hadoop/pull/843#discussion_r288854779
 
 

 ##
 File path: 
hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/S3AFileSystem.java
 ##
 @@ -2016,22 +2264,116 @@ void removeKeys(List 
keysToDelete,
 for (DeleteObjectsRequest.KeyVersion keyVersion : keysToDelete) {
   blockRootDelete(keyVersion.getKey());
 }
-if (enableMultiObjectsDelete) {
-  deleteObjects(new DeleteObjectsRequest(bucket)
-  .withKeys(keysToDelete)
-  .withQuiet(true));
-} else {
-  for (DeleteObjectsRequest.KeyVersion keyVersion : keysToDelete) {
-deleteObject(keyVersion.getKey());
+try {
+  if (enableMultiObjectsDelete) {
+deleteObjects(new DeleteObjectsRequest(bucket)
+.withKeys(keysToDelete)
+.withQuiet(true));
+  } else {
+for (DeleteObjectsRequest.KeyVersion keyVersion : keysToDelete) {
+  deleteObject(keyVersion.getKey());
+}
   }
+} catch (MultiObjectDeleteException ex) {
+  // partial delete.
+  // Update the stats with the count of the actual number of successful
+  // deletions.
+  int rejected = ex.getErrors().size();
+  noteDeleted(keysToDelete.size() - rejected, deleteFakeDir);
+  incrementStatistic(FILES_DELETE_REJECTED, rejected);
+  throw ex;
 }
+noteDeleted(keysToDelete.size(), deleteFakeDir);
+  }
+
+  /**
+   * Note the deletion of files or fake directories deleted.
+   * @param count count of keys deleted.
+   * @param deleteFakeDir are the deletions fake directories?
+   */
+  private void noteDeleted(final int count, final boolean deleteFakeDir) {
 if (!deleteFakeDir) {
-  instrumentation.fileDeleted(keysToDelete.size());
+  instrumentation.fileDeleted(count);
 } else {
-  instrumentation.fakeDirsDeleted(keysToDelete.size());
+  instrumentation.fakeDirsDeleted(count);
 }
-if (clearKeys) {
-  keysToDelete.clear();
+  }
+
+  /**
+   * Invoke {@link #removeKeysS3(List, boolean)} with handling of
+   * {@code MultiObjectDeleteException} in which S3Guard is updated with all
+   * deleted entries, before the exception is rethrown.
+   *
+   * If an exception is not raised. the metastore is not updated.
+   * @param keysToDelete collection of keys to delete on the s3-backend.
+   *if empty, no request is made of the object store.
+   * @param deleteFakeDir indicates whether this is for deleting fake dirs
+   * @throws InvalidRequestException if the request was rejected due to
+   * a mistaken attempt to delete the root directory.
+   * @throws MultiObjectDeleteException one or more of the keys could not
+   * be deleted in a multiple object delete operation.
+   * @throws AmazonClientException amazon-layer failure.
+   * @throws IOException other IO Exception.
+   */
+  @VisibleForTesting
+  @Retries.RetryMixed
+  void removeKeys(
+  final List keysToDelete,
+  final boolean deleteFakeDir)
+  throws MultiObjectDeleteException, AmazonClientException,
+  IOException {
+removeKeys(keysToDelete, deleteFakeDir, new ArrayList<>());
+  }
+
+  /**
+   * Invoke {@link #removeKeysS3(List, boolean)} with handling of
+   * {@code MultiObjectDeleteException} in which S3Guard is updated with all
+   * deleted entries, before the exception is rethrown.
+   *
+   * @param keysToDelete collection of keys to delete on the s3-backend.
+   *if empty, no request is made of the object store.
+   * @param deleteFakeDir indicates whether this is for deleting fake dirs
+   * @param undeletedObjectsOnFailure List which will be built up of all
+   * files that were not deleted. This happens even as an exception
+   * is raised.
+   * @throws InvalidRequestException if the request was rejected due to
+   * a mistaken attempt to delete the root directory.
+   * @throws MultiObjectDeleteException one or more of the keys could not
+   * be deleted in a multiple object delete operation.
+   * @throws AmazonClientException amazon-layer failure.
+   * @throws IOException other IO Exception.
+   */
+  @VisibleForTesting
+  @Retries.RetryMixed
+  void removeKeys(
 
 Review comment:
   Is there a test that exercises the partial failure logic here? (I may answer 
this as I get further in the review)


This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.
 
For queries about this service, please contact Infrastructure at:
us...@infra.apache.org


With regards,
Apache Git Services

-
To unsubscribe, e-mail: common-issues-unsubscr...@hadoop.apache.org
For additional commands, e-mail: 

[GitHub] [hadoop] ajfabbri commented on a change in pull request #843: HADOOP-15183 S3Guard store becomes inconsistent after partial failure of rename

2019-05-29 Thread GitBox
ajfabbri commented on a change in pull request #843: HADOOP-15183 S3Guard store 
becomes inconsistent after partial failure of rename
URL: https://github.com/apache/hadoop/pull/843#discussion_r288855378
 
 

 ##
 File path: 
hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/Statistic.java
 ##
 @@ -207,6 +209,12 @@
   "S3Guard metadata store put one metadata path latency"),
   S3GUARD_METADATASTORE_INITIALIZATION("s3guard_metadatastore_initialization",
   "S3Guard metadata store initialization times"),
+  S3GUARD_METADATASTORE_RECORD_READS(
+  "s3guard_metadatastore_record_reads",
+  "S3Guard metadata store records read"),
+  S3GUARD_METADATASTORE_RECORD_WRITES(
+  "s3guard_metadatastore_record_writes",
+  "S3Guard metadata store records written"),
 
 Review comment:
   Seems like this could be interesting for comparing different rename 
trackers? If so I'm curious what sort of stats / deltas you get with the new  
logic.


This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.
 
For queries about this service, please contact Infrastructure at:
us...@infra.apache.org


With regards,
Apache Git Services

-
To unsubscribe, e-mail: common-issues-unsubscr...@hadoop.apache.org
For additional commands, e-mail: common-issues-h...@hadoop.apache.org