001/* 002 * Licensed to the Apache Software Foundation (ASF) under one 003 * or more contributor license agreements. See the NOTICE file 004 * distributed with this work for additional information 005 * regarding copyright ownership. The ASF licenses this file 006 * to you under the Apache License, Version 2.0 (the 007 * "License"); you may not use this file except in compliance 008 * with the License. You may obtain a copy of the License at 009 * 010 * http://www.apache.org/licenses/LICENSE-2.0 011 * 012 * Unless required by applicable law or agreed to in writing, 013 * software distributed under the License is distributed on an 014 * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY 015 * KIND, either express or implied. See the License for the 016 * specific language governing permissions and limitations 017 * under the License. 018 */ 019package org.apache.reef.io.data.loading.impl; 020 021import org.apache.hadoop.mapred.InputSplit; 022import org.apache.reef.annotations.audience.DriverSide; 023import org.apache.reef.driver.catalog.NodeDescriptor; 024import org.apache.reef.tang.annotations.Parameter; 025 026import java.io.IOException; 027import java.util.Set; 028import java.util.concurrent.BlockingQueue; 029import java.util.concurrent.LinkedBlockingQueue; 030import java.util.logging.Logger; 031 032import javax.inject.Inject; 033 034/** 035 * This is an online version which satisfies 036 * requests in a greedy way, for single data center network topologies. 037 */ 038@DriverSide 039public final class SingleDataCenterEvaluatorToPartitionStrategy extends AbstractEvaluatorToPartitionStrategy { 040 private static final Logger LOG = Logger 041 .getLogger(SingleDataCenterEvaluatorToPartitionStrategy.class.getName()); 042 043 @Inject 044 SingleDataCenterEvaluatorToPartitionStrategy( 045 @Parameter(JobConfExternalConstructor.InputFormatClass.class) final String inputFormatClassName, 046 @Parameter(DistributedDataSetPartitionSerializer.DistributedDataSetPartitions.class) 047 final Set<String> serializedDataPartitions) { 048 super(inputFormatClassName, serializedDataPartitions); 049 } 050 051 @Override 052 protected void updateLocations(final NumberedSplit<InputSplit> numberedSplit) { 053 try { 054 final InputSplit split = numberedSplit.getEntry(); 055 final String[] locations = split.getLocations(); 056 for (final String location : locations) { 057 BlockingQueue<NumberedSplit<InputSplit>> newSplitQue = new LinkedBlockingQueue<>(); 058 final BlockingQueue<NumberedSplit<InputSplit>> splitQue = locationToSplits.putIfAbsent(location, newSplitQue); 059 if (splitQue != null) { 060 newSplitQue = splitQue; 061 } 062 newSplitQue.add(numberedSplit); 063 } 064 } catch (final IOException e) { 065 throw new RuntimeException("Unable to get InputSplits using the specified InputFormat", e); 066 } 067 } 068 069 @Override 070 protected NumberedSplit<InputSplit> tryAllocate(final NodeDescriptor nodeDescriptor, final String evaluatorId) { 071 LOG.fine("Picking a random split from the unallocated ones"); 072 return allocateSplit(evaluatorId, unallocatedSplits); 073 } 074 075}