@Override @SuppressWarnings("unchecked") public void initialize(InputSplit split, TaskAttemptContext ctx) throws IOException, InterruptedException { // set up columns that needs to read from the RCFile. tDesc = TStructDescriptor.getInstance(typeRef.getRawClass()); thriftWritable = ThriftWritable.newInstance((Class<TBase<?, ?>>) typeRef.getRawClass()); final List<Field> tFields = tDesc.getFields(); FileSplit fsplit = (FileSplit) split; Path file = fsplit.getPath(); LOG.info( String.format( "reading %s from %s:%d:%d", typeRef.getRawClass().getName(), file.toString(), fsplit.getStart(), fsplit.getStart() + fsplit.getLength())); ColumnarMetadata storedInfo = RCFileUtil.readMetadata(ctx.getConfiguration(), file); // list of field numbers List<Integer> tFieldIds = Lists.transform( tFields, new Function<Field, Integer>() { public Integer apply(Field fd) { return Integer.valueOf(fd.getFieldId()); } }); columnsBeingRead = RCFileUtil.findColumnsToRead(ctx.getConfiguration(), tFieldIds, storedInfo); for (int idx : columnsBeingRead) { int fid = storedInfo.getFieldId(idx); if (fid >= 0) { knownRequiredFields.add(tFields.get(tFieldIds.indexOf(fid))); } else { readUnknownsColumn = true; } } ColumnProjectionUtils.setReadColumnIDs(ctx.getConfiguration(), columnsBeingRead); // finally! super.initialize(split, ctx); }
/** * reads {@link ColumnarMetadata} stored in an RCFile. * * @throws IOException if metadata is not stored or in case of any other error. */ public static ColumnarMetadata readMetadata(Configuration conf, Path rcfile) throws IOException { Metadata metadata = null; Configuration confCopy = new Configuration(conf); // set up conf to read all the columns ColumnProjectionUtils.setFullyReadColumns(confCopy); RCFile.Reader reader = new RCFile.Reader(rcfile.getFileSystem(confCopy), rcfile, confCopy); // ugly hack to get metadata. RCFile has to provide access to metata try { Field f = RCFile.Reader.class.getDeclaredField("metadata"); f.setAccessible(true); metadata = (Metadata) f.get(reader); } catch (Throwable t) { throw new IOException("Could not access metadata field in RCFile reader", t); } reader.close(); Text metadataKey = new Text(COLUMN_METADATA_PROTOBUF_KEY); if (metadata == null || metadata.get(metadataKey) == null) { throw new IOException("could not find ColumnarMetadata in " + rcfile); } return Protobufs.mergeFromText(ColumnarMetadata.newBuilder(), metadata.get(metadataKey)) .build(); }
/** * Returns list of columns that need to be read from the RCFile. These columns are the * intersection of currently required columns and columns stored in the file. If any required * column does not exist in the file, we need to read the "unknown fields" column, which is * usually the last last one. */ public static ArrayList<Integer> findColumnsToRead( Configuration conf, List<Integer> currFieldIds, ColumnarMetadata storedInfo) throws IOException { ArrayList<Integer> columnsToRead = Lists.newArrayList(); // first find the required fields ArrayList<Integer> requiredFieldIds = Lists.newArrayList(); String reqFieldStr = conf.get(RCFileUtil.REQUIRED_FIELD_INDICES_CONF, ""); int numKnownFields = currFieldIds.size(); if (reqFieldStr == null || reqFieldStr.equals("")) { for (int i = 0; i < numKnownFields; i++) { requiredFieldIds.add(currFieldIds.get(i)); } } else { for (String str : reqFieldStr.split(",")) { int idx = Integer.valueOf(str); if (idx < 0 || idx >= numKnownFields) { throw new IOException("idx " + idx + " is out of range for known fields"); } requiredFieldIds.add(currFieldIds.get(idx)); } } List<Integer> storedFieldIds = storedInfo.getFieldIdList(); for (int i = 0; i < storedFieldIds.size(); i++) { int sid = storedFieldIds.get(i); if (sid > 0 && requiredFieldIds.contains(sid)) { columnsToRead.add(i); } } // unknown fields : the required fields that are not listed in storedFieldIds String unknownFields = ""; for (int rid : requiredFieldIds) { if (!storedFieldIds.contains(rid)) { unknownFields += " " + rid; } } if (unknownFields.length() > 0) { int last = storedFieldIds.size() - 1; LOG.info("unknown fields among required fileds :" + unknownFields); if (storedFieldIds.get(last) != -1) { // not expected throw new IOException("No unknowns column in in input"); } columnsToRead.add(last); } LOG.info( String.format( "reading %d%s out of %d stored columns for %d required columns", columnsToRead.size(), (unknownFields.length() > 0 ? " (including unknowns column)" : ""), storedInfo.getFieldIdCount(), requiredFieldIds.size())); return columnsToRead; }