public static ReadBuilder read(InputFile file) { return new ReadBuilder(file); }
@Override public boolean hasNext() { advance(); return batch.size > 0; }
@Override public boolean hasNext() { return (current != null && nextRow < current.size) || reader.hasNext(); }
public SparkOrcReader(InputFile location, FileScanTask task, Schema readSchema) { ColumnIdMap columnIds = new ColumnIdMap(); orcSchema = TypeConversion.toOrc(readSchema, columnIds); reader = ORC.read(location) .split(task.start(), task.length()) .schema(readSchema) .build(); int numFields = readSchema.columns().size(); row = new UnsafeRow(numFields); holder = new BufferHolder(row, INITIAL_SIZE); writer = new UnsafeRowWriter(holder, numFields); converter = new Converter[numFields]; for(int c=0; c < numFields; ++c) { converter[c] = buildConverter(holder, orcSchema.getChildren().get(c)); } }
public OrcIterator build() { Preconditions.checkNotNull(schema, "Schema is required"); try { Path path = new Path(file.location()); Reader reader = OrcFile.createReader(path, OrcFile.readerOptions(conf)); ColumnIdMap columnIds = new ColumnIdMap(); TypeDescription orcSchema = TypeConversion.toOrc(schema, columnIds); Reader.Options options = reader.options(); if (start != null) { options.range(start, length); } options.schema(orcSchema); return new OrcIterator(path, orcSchema, reader.rows(options)); } catch (IOException e) { throw new RuntimeException("Can't open " + file.location(), e); } } }
public static ColumnIdMap deserialize(TypeDescription schema, ByteBuffer serial) { ColumnIdMap result = new ColumnIdMap(); String[] parts = StandardCharsets.UTF_8.decode(serial).toString().split(","); for(int i = 0; i < parts.length; ++i) { String[] subparts = parts[i].split(":"); result.put(schema.findSubtype(Integer.parseInt(subparts[0])), Integer.parseInt(subparts[1])); } return result; } }
OrcFileAppender(Schema schema, OutputFile file, OrcFile.WriterOptions options, Map<String,byte[]> metadata) { orcSchema = TypeConversion.toOrc(schema, columnIds); options.setSchema(orcSchema); path = new Path(file.location()); try { writer = OrcFile.createWriter(path, options); } catch (IOException e) { throw new RuntimeException("Can't create file " + path, e); } writer.addUserMetadata(COLUMN_NUMBERS_ATTRIBUTE, columnIds.serialize()); metadata.forEach( (key,value) -> writer.addUserMetadata(key, ByteBuffer.wrap(value))); }
result = TypeDescription.createStruct(); for(Types.NestedField field: type.asStructType().fields()) { result.addField(field.name(), toOrc(field.fieldId(), field.type(), columnIds)); result = TypeDescription.createList(toOrc(list.elementId(), list.elementType(), columnIds)); break; TypeDescription key = toOrc(map.keyId(),map.keyType(), columnIds); result = TypeDescription.createMap(key, toOrc(map.valueId(), map.valueType(), columnIds)); break; columnIds.put(result, fieldId);
@Override public Metrics metrics() { try { long rows = writer.getNumberOfRows(); ColumnStatistics[] stats = writer.getStatistics(); // we don't currently have columnSizes or distinct counts. Map<Integer, Long> valueCounts = new HashMap<>(); Map<Integer, Long> nullCounts = new HashMap<>(); Integer[] icebergIds = new Integer[orcSchema.getMaximumId() + 1]; for(TypeDescription type: columnIds.keySet()) { icebergIds[type.getId()] = columnIds.get(type); } for(int c=1; c < stats.length; ++c) { if (icebergIds[c] != null) { valueCounts.put(icebergIds[c], stats[c].getNumberOfValues()); } } for(TypeDescription child: orcSchema.getChildren()) { int c = child.getId(); if (icebergIds[c] != null) { nullCounts.put(icebergIds[c], rows - stats[c].getNumberOfValues()); } } return new Metrics(rows, null, valueCounts, nullCounts); } catch (IOException e) { throw new RuntimeException("Can't get statistics " + path, e); } }
@Override public void close() throws IOException { reader.close(); }
@Override public Metrics metrics() { return writer.metrics(); }
public static WriteBuilder write(OutputFile file) { return new WriteBuilder(file); }
public OrcFileAppender build() { OrcFile.WriterOptions options = OrcFile.writerOptions(conf); return new OrcFileAppender(schema, file, options, metadata); } }
@Override public void add(InternalRow datum) { int row = batch.size++; for(int c=0; c < converters.length; ++c) { converters[c].addValue(row, c, datum, batch.cols[c]); } if (batch.size == BATCH_SIZE) { writer.add(batch); batch.reset(); } }
public SparkOrcWriter(OrcFileAppender writer) { TypeDescription schema = writer.getSchema(); batch = schema.createRowBatch(BATCH_SIZE); this.writer = writer; converters = buildConverters(schema); }
@Override public UnsafeRow next() { if (current == null || nextRow >= current.size) { current = reader.next(); nextRow = 0; } // Reset the holder to start the buffer over again. // BufferHolder.reset does the wrong thing... holder.cursor = Platform.BYTE_ARRAY_OFFSET; writer.reset(); for(int c=0; c < current.cols.length; ++c) { converter[c].convert(writer, c, current.cols[c], nextRow); } nextRow++; return row; }
/** * Convert an ORC schema to an Iceberg schema. * @param schema the ORC schema * @param columnIds the column ids * @return the Iceberg schema */ public Schema fromOrc(TypeDescription schema, ColumnIdMap columnIds) { return new Schema(convertOrcToType(schema, columnIds).asStructType().fields()); }
@Override public VectorizedRowBatch next() { // make sure we have the next batch advance(); // mark it as used advanced = false; return batch; } }