Java tutorial
/* * Copyright 2014 FullContact, Inc. * * 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. */ package com.fullcontact.sstable.hadoop.mapreduce; import com.fullcontact.sstable.hadoop.SSTableFunctions; import com.fullcontact.sstable.hadoop.SSTablePredicates; import com.fullcontact.sstable.index.SSTableIndexIndex; import com.google.common.collect.Lists; import com.google.common.collect.Maps; import org.apache.hadoop.conf.Configuration; import org.apache.hadoop.fs.BlockLocation; import org.apache.hadoop.fs.FileStatus; import org.apache.hadoop.fs.FileSystem; import org.apache.hadoop.fs.Path; import org.apache.hadoop.mapreduce.InputSplit; import org.apache.hadoop.mapreduce.JobContext; import org.apache.hadoop.mapreduce.lib.input.FileInputFormat; import org.slf4j.Logger; import org.slf4j.LoggerFactory; import java.io.IOException; import java.util.Collection; import java.util.Iterator; import java.util.List; import java.util.Map; /** * File input format for reading directly from compressed Cassandra sstables. Handles splitting the sstables based on * <sstable-file-name>-Index.db.index files. These index files must have previously been generated by running the * indexer across the Cassandra sstables. */ public abstract class SSTableInputFormat<K, V> extends FileInputFormat<K, V> { private static final Logger LOG = LoggerFactory.getLogger(SSTableInputFormat.class); // Assuming 40% compression ratio public static final int COMPRESSION_RATIO_ASSUMPTION = 40; private final Map<Path, SSTableIndexIndex> indexes = Maps.newHashMap(); @Override protected List<FileStatus> listStatus(final JobContext job) throws IOException { final List<FileStatus> files = Lists.newArrayList(); for (FileStatus file : super.listStatus(job)) { files.addAll(handleFile(file, job)); } LOG.debug("Initial file list: {} {}", files.size(), files); final Configuration configuration = job.getConfiguration(); for (Iterator<FileStatus> iterator = files.iterator(); iterator.hasNext();) { final FileStatus fileStatus = iterator.next(); final Path file = fileStatus.getPath(); final FileSystem fs = file.getFileSystem(configuration); if (!SSTablePredicates.IS_SSTABLE.apply(file.toString())) { // Ignore non-sstable date files, always (for now) LOG.debug("Removing non-sstable file: {}", file); iterator.remove(); } else { // read the index file LOG.debug("Reading index file for sstable file: {}", file); final Path indexFile = SSTableFunctions.INDEX_FILE.apply(file); LOG.debug("Reading index file: {}", indexFile); final SSTableIndexIndex index = SSTableIndexIndex.readIndex(fs, indexFile); indexes.put(file, index); } } LOG.debug("Final file list: {} {}", files.size(), files); return files; } /** * If we have a directory recursively gather the files we care about for this job. * * @param file Root file/directory. * @param job Job context. * @return All files we care about. * @throws IOException */ private Collection<FileStatus> handleFile(final FileStatus file, final JobContext job) throws IOException { final List<FileStatus> results = Lists.newArrayList(); if (file.isDir()) { final Path p = file.getPath(); LOG.debug("Expanding {}", p); final FileSystem fs = p.getFileSystem(job.getConfiguration()); final FileStatus[] children = fs.listStatus(p); for (FileStatus child : children) { results.addAll(handleFile(child, job)); } } else { results.add(file); } return results; } @Override protected boolean isSplitable(final JobContext context, final Path filename) { if (SSTablePredicates.IS_SSTABLE.apply(filename.toString())) { LOG.debug("{} is splittable.", filename); return indexes.containsKey(filename); } else { // Delegate non-sstable files to the FileInputFormat base class. LOG.debug("{} is not splittable.", filename); return super.isSplitable(context, filename); } } @Override public List<InputSplit> getSplits(final JobContext job) throws IOException { final Configuration configuration = job.getConfiguration(); final List<InputSplit> result = Lists.newArrayList(); final List<FileStatus> files = listStatus(job); LOG.debug("Initial file list: {} {}", files.size(), files); for (final FileStatus fileStatus : files) { final Path dataFile = fileStatus.getPath(); final FileSystem fileSystem = dataFile.getFileSystem(configuration); final BlockLocation[] blockLocations = fileSystem.getFileBlockLocations(fileStatus, 0, fileStatus.getLen()); // Data file, try to split if the .index file was found final SSTableIndexIndex index = indexes.get(dataFile); if (index == null) { throw new IOException("Index not found for " + dataFile); } for (final SSTableIndexIndex.Chunk chunk : index.getOffsets()) { // This isn't likely to work well because we are dealing with the index into uncompressed data... final int blockIndex = getBlockIndex(blockLocations, chunk.getStart() / COMPRESSION_RATIO_ASSUMPTION); final SSTableSplit split = new SSTableSplit(dataFile, chunk.getStart(), chunk.getEnd(), chunk.getEnd() - chunk.getStart(), blockLocations[blockIndex].getHosts()); result.add(split); } } LOG.debug("Splits calculated: {} {}", result.size(), result); return result; } protected int getBlockIndex(final BlockLocation[] blockLocations, final long offset) { for (int i = 0; i < blockLocations.length; i++) { // is the offset inside this block? if ((blockLocations[i].getOffset() <= offset) && (offset < blockLocations[i].getOffset() + blockLocations[i].getLength())) { return i; } } return blockLocations.length - 1; } }