/core/src/main/java/com/mongodb/hadoop/mapred/BSONFileInputFormat.java

http://github.com/mongodb/mongo-hadoop · Java · 133 lines · 102 code · 15 blank · 16 comment · 9 complexity · 31f4a22fa8fac300ab1c384d169b21a8 MD5 · raw file

  1. /*
  2. * Copyright 2010-2013 10gen Inc.
  3. *
  4. * Licensed under the Apache License, Version 2.0 (the "License");
  5. * you may not use this file except in compliance with the License.
  6. * You may obtain a copy of the License at
  7. *
  8. * http://www.apache.org/licenses/LICENSE-2.0
  9. *
  10. * Unless required by applicable law or agreed to in writing, software
  11. * distributed under the License is distributed on an "AS IS" BASIS,
  12. * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
  13. * See the License for the specific language governing permissions and
  14. * limitations under the License.
  15. */
  16. package com.mongodb.hadoop.mapred;
  17. import com.mongodb.hadoop.io.BSONWritable;
  18. import com.mongodb.hadoop.mapred.input.BSONFileRecordReader;
  19. import com.mongodb.hadoop.mapred.input.BSONFileSplit;
  20. import com.mongodb.hadoop.splitter.BSONSplitter;
  21. import com.mongodb.hadoop.util.MongoConfigUtil;
  22. import org.apache.hadoop.fs.FileStatus;
  23. import org.apache.hadoop.fs.FileSystem;
  24. import org.apache.hadoop.fs.Path;
  25. import org.apache.hadoop.io.NullWritable;
  26. import org.apache.hadoop.io.compress.CompressionCodec;
  27. import org.apache.hadoop.io.compress.CompressionCodecFactory;
  28. import org.apache.hadoop.mapred.FileInputFormat;
  29. import org.apache.hadoop.mapred.FileSplit;
  30. import org.apache.hadoop.mapred.InputSplit;
  31. import org.apache.hadoop.mapred.JobConf;
  32. import org.apache.hadoop.mapred.RecordReader;
  33. import org.apache.hadoop.mapred.Reporter;
  34. import java.io.IOException;
  35. import java.util.ArrayList;
  36. import java.util.List;
  37. import static com.mongodb.hadoop.splitter.BSONSplitter.getSplitsFilePath;
  38. import static java.lang.String.format;
  39. public class BSONFileInputFormat extends FileInputFormat {
  40. @Override
  41. protected boolean isSplitable(final FileSystem fs, final Path filename) {
  42. CompressionCodec codec =
  43. new CompressionCodecFactory(fs.getConf()).getCodec(filename);
  44. return null == codec;
  45. }
  46. @Override
  47. public FileSplit[] getSplits(final JobConf job, final int numSplits) throws IOException {
  48. BSONSplitter splitter = new BSONSplitter();
  49. splitter.setConf(job);
  50. FileStatus[] inputFiles = listStatus(job);
  51. List<FileSplit> results = new ArrayList<FileSplit>();
  52. for (FileStatus file : inputFiles) {
  53. FileSystem fs = FileSystem.get(file.getPath().toUri(), job);
  54. if (!isSplitable(fs, file.getPath())) {
  55. LOG.info(
  56. "File " + file.getPath() + " is compressed so "
  57. + "cannot be split.");
  58. org.apache.hadoop.mapreduce.lib.input.FileSplit delegate =
  59. splitter.createFileSplit(file, fs, 0L, file.getLen());
  60. results.add(
  61. new BSONFileSplit(
  62. delegate.getPath(), delegate.getStart(),
  63. delegate.getLength(), delegate.getLocations()));
  64. continue;
  65. }
  66. splitter.setInputPath(file.getPath());
  67. Path splitFilePath = getSplitsFilePath(file.getPath(), job);
  68. try {
  69. splitter.loadSplitsFromSplitFile(file, splitFilePath);
  70. } catch (BSONSplitter.NoSplitFileException nsfe) {
  71. if (LOG.isDebugEnabled()) {
  72. LOG.debug(format("No split file for %s; building split file", file.getPath()));
  73. }
  74. splitter.readSplitsForFile(file);
  75. }
  76. if (LOG.isDebugEnabled()) {
  77. LOG.debug(format("BSONSplitter found %d splits.", splitter.getAllSplits().size()));
  78. }
  79. for (org.apache.hadoop.mapreduce.lib.input.FileSplit split : splitter.getAllSplits()) {
  80. BSONFileSplit fsplit = new BSONFileSplit(
  81. split.getPath(),
  82. split.getStart(),
  83. split.getLength(),
  84. split.getLocations());
  85. fsplit.setKeyField(MongoConfigUtil.getInputKey(job));
  86. results.add(fsplit);
  87. }
  88. }
  89. if (LOG.isDebugEnabled()) {
  90. LOG.debug(format("Total of %d found.", results.size()));
  91. }
  92. return results.toArray(new BSONFileSplit[results.size()]);
  93. }
  94. @Override
  95. public RecordReader<NullWritable, BSONWritable> getRecordReader(final InputSplit split, final JobConf job, final Reporter reporter)
  96. throws IOException {
  97. FileSplit fileSplit = (FileSplit) split;
  98. FileSystem fs = FileSystem.get(fileSplit.getPath().toUri(), job);
  99. if (split instanceof BSONFileSplit
  100. || !isSplitable(fs, fileSplit.getPath())) {
  101. BSONFileRecordReader reader = new BSONFileRecordReader();
  102. reader.initialize(split, job);
  103. return reader;
  104. }
  105. // Split was not created by BSONSplitter.
  106. BSONSplitter splitter = new BSONSplitter();
  107. splitter.setConf(job);
  108. splitter.setInputPath(fileSplit.getPath());
  109. org.apache.hadoop.mapreduce.lib.input.FileSplit newStyleFileSplit =
  110. new org.apache.hadoop.mapreduce.lib.input.FileSplit(
  111. fileSplit.getPath(), fileSplit.getStart(), fileSplit.getLength(),
  112. fileSplit.getLocations());
  113. long start = splitter.getStartingPositionForSplit(newStyleFileSplit);
  114. BSONFileRecordReader reader = new BSONFileRecordReader(start);
  115. reader.initialize(fileSplit, job);
  116. return reader;
  117. }
  118. }