blob: cdcc4018de79902e9f25acd4f11fba8781d62f33 [file] [log] [blame]
/**
* 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.crunch.PCollection;
import org.apache.hadoop.conf.Configuration;
/**
* Helper functions and settings for determining the number of reducers to use in a pipeline
* job created by the Crunch planner.
*/
public class PartitionUtils {
public static final String BYTES_PER_REDUCE_TASK = "crunch.bytes.per.reduce.task";
public static final long DEFAULT_BYTES_PER_REDUCE_TASK = 1000L * 1000L * 1000L;
/**
* Set an upper limit on the number of reducers the Crunch planner will set for an MR
* job when it tries to determine how many reducers to use based on the input size.
*/
public static final String MAX_REDUCERS = "crunch.max.reducers";
public static final int DEFAULT_MAX_REDUCERS = 500;
public static <T> int getRecommendedPartitions(PCollection<T> pcollection) {
Configuration conf = pcollection.getPipeline().getConfiguration();
return getRecommendedPartitions(pcollection, conf);
}
public static <T> int getRecommendedPartitions(PCollection<T> pcollection, Configuration conf) {
long bytesPerTask = conf.getLong(BYTES_PER_REDUCE_TASK, DEFAULT_BYTES_PER_REDUCE_TASK);
int recommended = 1 + (int) (pcollection.getSize() / bytesPerTask);
int maxRecommended = conf.getInt(MAX_REDUCERS, DEFAULT_MAX_REDUCERS);
if (maxRecommended > 0 && recommended > maxRecommended) {
return maxRecommended;
} else {
return recommended;
}
}
}