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))); }
/** * 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()); }
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); } } }
TypeDescription type = fieldTypes.get(c); fields.add(Types.NestedField.optional(columnIds.get(type), name, convertOrcToType(type, columnIds))); TypeDescription child = schema.getChildren().get(0); return Types.ListType.ofOptional(columnIds.get(child), convertOrcToType(child, columnIds)); TypeDescription value = schema.getChildren().get(1); return Types.MapType.ofOptional(columnIds.get(key), columnIds.get(value), convertOrcToType(key, columnIds), convertOrcToType(value, columnIds));
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)); } }
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;