[GitHub] noslowerdna commented on a change in pull request #20: CRUNCH-679: Improvements for usage of DistCp

2019-03-01 Thread GitBox
noslowerdna commented on a change in pull request #20: CRUNCH-679: Improvements 
for usage of DistCp
URL: https://github.com/apache/crunch/pull/20#discussion_r261687267
 
 

 ##
 File path: 
crunch-core/src/main/java/org/apache/crunch/util/CrunchRenameCopyListing.java
 ##
 @@ -0,0 +1,261 @@
+/**
+ * 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.crunch.util;
+
+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.io.SequenceFile;
+import org.apache.hadoop.io.Text;
+import org.apache.hadoop.security.Credentials;
+import org.apache.hadoop.tools.CopyListing;
+import org.apache.hadoop.tools.CopyListingFileStatus;
+import org.apache.hadoop.tools.DistCpOptions;
+import org.apache.hadoop.tools.DistCpOptions.FileAttribute;
+import org.apache.hadoop.tools.SimpleCopyListing;
+import org.apache.hadoop.tools.util.DistCpUtils;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import java.io.IOException;
+import java.util.HashMap;
+import java.util.Map;
+import java.util.Stack;
+
+/**
+ * A custom {@link CopyListing} implementation capable of dynamically renaming
+ * the target paths according to a configured set of values.
+ * 
+ * Once https://issues.apache.org/jira/browse/HADOOP-16147 is available, this
+ * class can be significantly simplified.
+ * 
+ */
+public class CrunchRenameCopyListing extends SimpleCopyListing {
+  /**
+   * Comma-separated list of original-file:renamed-file path rename pairs.
+   */
+  public static final String DISTCP_PATH_RENAMES = 
"crunch.distcp.path.renames";
+
+  private static final Logger LOG = 
LoggerFactory.getLogger(CrunchRenameCopyListing.class);
+  private final Map pathRenames;
+
+  private long totalPaths = 0;
+  private long totalBytesToCopy = 0;
+
+  /**
+   * Protected constructor, to initialize configuration.
+   *
+   * @param configuration The input configuration, with which the 
source/target FileSystems may be accessed.
+   * @param credentials - Credentials object on which the FS delegation tokens 
are cached. If null
+   * delegation token caching is skipped
+   */
+  protected CrunchRenameCopyListing(Configuration configuration, Credentials 
credentials) {
+super(configuration, credentials);
+
+pathRenames = new HashMap<>();
+
+String[] pathRenameConf = configuration.getStrings(DISTCP_PATH_RENAMES);
+if (pathRenameConf == null) {
+  throw new IllegalArgumentException("Missing required configuration: " + 
DISTCP_PATH_RENAMES);
+}
+for (String pathRename : pathRenameConf) {
+  String[] pathRenameParts = pathRename.split(":");
+  if (pathRenameParts.length != 2) {
+throw new IllegalArgumentException("Invalid path rename format: " + 
pathRename);
+  }
+  if (pathRenames.put(pathRenameParts[0], pathRenameParts[1]) != null) {
+throw new IllegalArgumentException("Invalid duplicate path rename: " + 
pathRenameParts[0]);
+  }
+}
+LOG.info("Loaded {} path rename entries", pathRenames.size());
+  }
+
+  @Override
+  public void doBuildListing(SequenceFile.Writer fileListWriter, DistCpOptions 
options) throws IOException {
+try {
+  for (Path path : options.getSourcePaths()) {
+FileSystem sourceFS = path.getFileSystem(getConf());
+final boolean preserveAcls = options.shouldPreserve(FileAttribute.ACL);
+final boolean preserveXAttrs = 
options.shouldPreserve(FileAttribute.XATTR);
+final boolean preserveRawXAttrs = options.shouldPreserveRawXattrs();
+path = makeQualified(path);
+
+FileStatus rootStatus = sourceFS.getFileStatus(path);
+Path sourcePathRoot = computeSourceRootPath(rootStatus, options);
+
+FileStatus[] sourceFiles = sourceFS.listStatus(path);
+boolean explore = (sourceFiles != null && sourceFiles.length > 0);
+if (!explore || rootStatus.isDirectory()) {
+  CopyListingFileStatus rootCopyListingStatus = 
DistCpUtils.toCopyListingFileStatus(sourceFS, rootStatus, preserveAcls,
+  preserveXAttrs, preserveRawXAttrs);
+  writeToFileListingRoot(fileListWriter, 

[GitHub] mkwhitacre commented on a change in pull request #20: CRUNCH-679: Improvements for usage of DistCp

2019-03-01 Thread GitBox
mkwhitacre commented on a change in pull request #20: CRUNCH-679: Improvements 
for usage of DistCp
URL: https://github.com/apache/crunch/pull/20#discussion_r261682765
 
 

 ##
 File path: 
crunch-core/src/main/java/org/apache/crunch/util/CrunchRenameCopyListing.java
 ##
 @@ -0,0 +1,261 @@
+/**
+ * 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.crunch.util;
+
+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.io.SequenceFile;
+import org.apache.hadoop.io.Text;
+import org.apache.hadoop.security.Credentials;
+import org.apache.hadoop.tools.CopyListing;
+import org.apache.hadoop.tools.CopyListingFileStatus;
+import org.apache.hadoop.tools.DistCpOptions;
+import org.apache.hadoop.tools.DistCpOptions.FileAttribute;
+import org.apache.hadoop.tools.SimpleCopyListing;
+import org.apache.hadoop.tools.util.DistCpUtils;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import java.io.IOException;
+import java.util.HashMap;
+import java.util.Map;
+import java.util.Stack;
+
+/**
+ * A custom {@link CopyListing} implementation capable of dynamically renaming
+ * the target paths according to a configured set of values.
+ * 
+ * Once https://issues.apache.org/jira/browse/HADOOP-16147 is available, this
+ * class can be significantly simplified.
+ * 
+ */
+public class CrunchRenameCopyListing extends SimpleCopyListing {
+  /**
+   * Comma-separated list of original-file:renamed-file path rename pairs.
+   */
+  public static final String DISTCP_PATH_RENAMES = 
"crunch.distcp.path.renames";
+
+  private static final Logger LOG = 
LoggerFactory.getLogger(CrunchRenameCopyListing.class);
+  private final Map pathRenames;
+
+  private long totalPaths = 0;
+  private long totalBytesToCopy = 0;
+
+  /**
+   * Protected constructor, to initialize configuration.
+   *
+   * @param configuration The input configuration, with which the 
source/target FileSystems may be accessed.
+   * @param credentials - Credentials object on which the FS delegation tokens 
are cached. If null
+   * delegation token caching is skipped
+   */
+  protected CrunchRenameCopyListing(Configuration configuration, Credentials 
credentials) {
+super(configuration, credentials);
+
+pathRenames = new HashMap<>();
+
+String[] pathRenameConf = configuration.getStrings(DISTCP_PATH_RENAMES);
+if (pathRenameConf == null) {
+  throw new IllegalArgumentException("Missing required configuration: " + 
DISTCP_PATH_RENAMES);
+}
+for (String pathRename : pathRenameConf) {
+  String[] pathRenameParts = pathRename.split(":");
+  if (pathRenameParts.length != 2) {
+throw new IllegalArgumentException("Invalid path rename format: " + 
pathRename);
+  }
+  if (pathRenames.put(pathRenameParts[0], pathRenameParts[1]) != null) {
+throw new IllegalArgumentException("Invalid duplicate path rename: " + 
pathRenameParts[0]);
+  }
+}
+LOG.info("Loaded {} path rename entries", pathRenames.size());
+  }
+
+  @Override
+  public void doBuildListing(SequenceFile.Writer fileListWriter, DistCpOptions 
options) throws IOException {
+try {
+  for (Path path : options.getSourcePaths()) {
+FileSystem sourceFS = path.getFileSystem(getConf());
+final boolean preserveAcls = options.shouldPreserve(FileAttribute.ACL);
+final boolean preserveXAttrs = 
options.shouldPreserve(FileAttribute.XATTR);
+final boolean preserveRawXAttrs = options.shouldPreserveRawXattrs();
+path = makeQualified(path);
+
+FileStatus rootStatus = sourceFS.getFileStatus(path);
+Path sourcePathRoot = computeSourceRootPath(rootStatus, options);
+
+FileStatus[] sourceFiles = sourceFS.listStatus(path);
+boolean explore = (sourceFiles != null && sourceFiles.length > 0);
+if (!explore || rootStatus.isDirectory()) {
+  CopyListingFileStatus rootCopyListingStatus = 
DistCpUtils.toCopyListingFileStatus(sourceFS, rootStatus, preserveAcls,
+  preserveXAttrs, preserveRawXAttrs);
+  writeToFileListingRoot(fileListWriter, 

[jira] [Work logged] (CRUNCH-679) Improvements for usage of DistCp

2019-03-01 Thread ASF GitHub Bot (JIRA)


 [ 
https://issues.apache.org/jira/browse/CRUNCH-679?focusedWorklogId=206528=com.atlassian.jira.plugin.system.issuetabpanels:worklog-tabpanel#worklog-206528
 ]

ASF GitHub Bot logged work on CRUNCH-679:
-

Author: ASF GitHub Bot
Created on: 01/Mar/19 17:09
Start Date: 01/Mar/19 17:09
Worklog Time Spent: 10m 
  Work Description: mkwhitacre commented on pull request #20: CRUNCH-679: 
Improvements for usage of DistCp
URL: https://github.com/apache/crunch/pull/20#discussion_r261682765
 
 

 ##
 File path: 
crunch-core/src/main/java/org/apache/crunch/util/CrunchRenameCopyListing.java
 ##
 @@ -0,0 +1,261 @@
+/**
+ * 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.crunch.util;
+
+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.io.SequenceFile;
+import org.apache.hadoop.io.Text;
+import org.apache.hadoop.security.Credentials;
+import org.apache.hadoop.tools.CopyListing;
+import org.apache.hadoop.tools.CopyListingFileStatus;
+import org.apache.hadoop.tools.DistCpOptions;
+import org.apache.hadoop.tools.DistCpOptions.FileAttribute;
+import org.apache.hadoop.tools.SimpleCopyListing;
+import org.apache.hadoop.tools.util.DistCpUtils;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import java.io.IOException;
+import java.util.HashMap;
+import java.util.Map;
+import java.util.Stack;
+
+/**
+ * A custom {@link CopyListing} implementation capable of dynamically renaming
+ * the target paths according to a configured set of values.
+ * 
+ * Once https://issues.apache.org/jira/browse/HADOOP-16147 is available, this
+ * class can be significantly simplified.
+ * 
+ */
+public class CrunchRenameCopyListing extends SimpleCopyListing {
+  /**
+   * Comma-separated list of original-file:renamed-file path rename pairs.
+   */
+  public static final String DISTCP_PATH_RENAMES = 
"crunch.distcp.path.renames";
+
+  private static final Logger LOG = 
LoggerFactory.getLogger(CrunchRenameCopyListing.class);
+  private final Map pathRenames;
+
+  private long totalPaths = 0;
+  private long totalBytesToCopy = 0;
+
+  /**
+   * Protected constructor, to initialize configuration.
+   *
+   * @param configuration The input configuration, with which the 
source/target FileSystems may be accessed.
+   * @param credentials - Credentials object on which the FS delegation tokens 
are cached. If null
+   * delegation token caching is skipped
+   */
+  protected CrunchRenameCopyListing(Configuration configuration, Credentials 
credentials) {
+super(configuration, credentials);
+
+pathRenames = new HashMap<>();
+
+String[] pathRenameConf = configuration.getStrings(DISTCP_PATH_RENAMES);
+if (pathRenameConf == null) {
+  throw new IllegalArgumentException("Missing required configuration: " + 
DISTCP_PATH_RENAMES);
+}
+for (String pathRename : pathRenameConf) {
+  String[] pathRenameParts = pathRename.split(":");
+  if (pathRenameParts.length != 2) {
+throw new IllegalArgumentException("Invalid path rename format: " + 
pathRename);
+  }
+  if (pathRenames.put(pathRenameParts[0], pathRenameParts[1]) != null) {
+throw new IllegalArgumentException("Invalid duplicate path rename: " + 
pathRenameParts[0]);
+  }
+}
+LOG.info("Loaded {} path rename entries", pathRenames.size());
+  }
+
+  @Override
+  public void doBuildListing(SequenceFile.Writer fileListWriter, DistCpOptions 
options) throws IOException {
+try {
+  for (Path path : options.getSourcePaths()) {
+FileSystem sourceFS = path.getFileSystem(getConf());
+final boolean preserveAcls = options.shouldPreserve(FileAttribute.ACL);
+final boolean preserveXAttrs = 
options.shouldPreserve(FileAttribute.XATTR);
+final boolean preserveRawXAttrs = options.shouldPreserveRawXattrs();
+path = makeQualified(path);
+
+FileStatus rootStatus = sourceFS.getFileStatus(path);
+Path sourcePathRoot = computeSourceRootPath(rootStatus, options);
+
+FileStatus[] 

[jira] [Work logged] (CRUNCH-680) Kafka Source should split very large partitions

2019-03-01 Thread ASF GitHub Bot (JIRA)


 [ 
https://issues.apache.org/jira/browse/CRUNCH-680?focusedWorklogId=206520=com.atlassian.jira.plugin.system.issuetabpanels:worklog-tabpanel#worklog-206520
 ]

ASF GitHub Bot logged work on CRUNCH-680:
-

Author: ASF GitHub Bot
Created on: 01/Mar/19 16:40
Start Date: 01/Mar/19 16:40
Worklog Time Spent: 10m 
  Work Description: mkwhitacre commented on pull request #21: CRUNCH-680: 
Kafka Source should split very large partitions
URL: https://github.com/apache/crunch/pull/21
 
 
   
 

This is an automated message from the Apache Git Service.
To respond to the message, please log on 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


Issue Time Tracking
---

Worklog Id: (was: 206520)
Time Spent: 20m  (was: 10m)

> Kafka Source should split very large partitions
> ---
>
> Key: CRUNCH-680
> URL: https://issues.apache.org/jira/browse/CRUNCH-680
> Project: Crunch
>  Issue Type: Improvement
>  Components: IO
>Reporter: Andrew Olson
>Assignee: Micah Whitacre
>Priority: Minor
>  Time Spent: 20m
>  Remaining Estimate: 0h
>
> If a single Kafka partition has a very large number of messages, the map task 
> for that partition can take a long time to run leading to potential timeout 
> problems. We should limit the number of messages assigned to each split so 
> that the workload is more evenly balanced.
> Based on our testing we have determined that 5 million messages should be a 
> generally reasonable default for the maximum split size, with a configuration 
> property (org.apache.crunch.kafka.split.max) provided to optionally override 
> that value.



--
This message was sent by Atlassian JIRA
(v7.6.3#76005)


[GitHub] mkwhitacre closed pull request #10: CRUNCH-519: Plan dot file can display more information

2019-03-01 Thread GitBox
mkwhitacre closed pull request #10: CRUNCH-519: Plan dot file can display more 
information
URL: https://github.com/apache/crunch/pull/10
 
 
   


This is an automated message from the Apache Git Service.
To respond to the message, please log on 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


[jira] [Assigned] (CRUNCH-680) Kafka Source should split very large partitions

2019-03-01 Thread Micah Whitacre (JIRA)


 [ 
https://issues.apache.org/jira/browse/CRUNCH-680?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel
 ]

Micah Whitacre reassigned CRUNCH-680:
-

Assignee: Micah Whitacre

> Kafka Source should split very large partitions
> ---
>
> Key: CRUNCH-680
> URL: https://issues.apache.org/jira/browse/CRUNCH-680
> Project: Crunch
>  Issue Type: Improvement
>  Components: IO
>Reporter: Andrew Olson
>Assignee: Micah Whitacre
>Priority: Minor
>  Time Spent: 10m
>  Remaining Estimate: 0h
>
> If a single Kafka partition has a very large number of messages, the map task 
> for that partition can take a long time to run leading to potential timeout 
> problems. We should limit the number of messages assigned to each split so 
> that the workload is more evenly balanced.
> Based on our testing we have determined that 5 million messages should be a 
> generally reasonable default for the maximum split size, with a configuration 
> property (org.apache.crunch.kafka.split.max) provided to optionally override 
> that value.



--
This message was sent by Atlassian JIRA
(v7.6.3#76005)


[jira] [Commented] (CRUNCH-680) Kafka Source should split very large partitions

2019-03-01 Thread Micah Whitacre (JIRA)


[ 
https://issues.apache.org/jira/browse/CRUNCH-680?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel=16781860#comment-16781860
 ] 

Micah Whitacre commented on CRUNCH-680:
---

Changes look good and the PR was merged.

> Kafka Source should split very large partitions
> ---
>
> Key: CRUNCH-680
> URL: https://issues.apache.org/jira/browse/CRUNCH-680
> Project: Crunch
>  Issue Type: Improvement
>  Components: IO
>Reporter: Andrew Olson
>Assignee: Micah Whitacre
>Priority: Minor
>  Time Spent: 20m
>  Remaining Estimate: 0h
>
> If a single Kafka partition has a very large number of messages, the map task 
> for that partition can take a long time to run leading to potential timeout 
> problems. We should limit the number of messages assigned to each split so 
> that the workload is more evenly balanced.
> Based on our testing we have determined that 5 million messages should be a 
> generally reasonable default for the maximum split size, with a configuration 
> property (org.apache.crunch.kafka.split.max) provided to optionally override 
> that value.



--
This message was sent by Atlassian JIRA
(v7.6.3#76005)


[GitHub] mkwhitacre merged pull request #21: CRUNCH-680: Kafka Source should split very large partitions

2019-03-01 Thread GitBox
mkwhitacre merged pull request #21: CRUNCH-680: Kafka Source should split very 
large partitions
URL: https://github.com/apache/crunch/pull/21
 
 
   


This is an automated message from the Apache Git Service.
To respond to the message, please log on 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