Assumes the stream writing into this receiver has already been flushed.
- *
- * @return number of bytes
- */
public long getOutputSize() {
long result = 0;
for (ByteBuffer buffer : output) {
@@ -579,141 +395,4 @@ public class PhysicalWriterImpl implements PhysicalWriter {
return result;
}
}
-
- static class SizeCounters {
- long index = 0;
- long data = 0;
-
- long total() {
- return index + data;
- }
- }
-
- void buildStreamList(OrcProto.StripeFooter.Builder footerBuilder, SizeCounters sizes)
- throws IOException {
- footerBuilder.addAllStreams(unencrypted.placeStreams(StreamName.Area.INDEX, sizes));
- final long unencryptedIndexSize = sizes.index;
- int v = 0;
- for (VariantTracker variant : variants.values()) {
- OrcProto.StripeEncryptionVariant.Builder builder =
- footerBuilder.getEncryptionBuilder(v++);
- builder.addAllStreams(variant.placeStreams(StreamName.Area.INDEX, sizes));
- }
- if (sizes.index != unencryptedIndexSize) {
- // add a placeholder that covers the hole where the encrypted indexes are
- footerBuilder.addStreams(
- OrcProto.Stream.newBuilder()
- .setKind(OrcProto.Stream.Kind.ENCRYPTED_INDEX)
- .setLength(sizes.index - unencryptedIndexSize));
- }
- footerBuilder.addAllStreams(unencrypted.placeStreams(StreamName.Area.DATA, sizes));
- final long unencryptedDataSize = sizes.data;
- v = 0;
- for (VariantTracker variant : variants.values()) {
- OrcProto.StripeEncryptionVariant.Builder builder =
- footerBuilder.getEncryptionBuilder(v++);
- builder.addAllStreams(variant.placeStreams(StreamName.Area.DATA, sizes));
- }
- if (sizes.data != unencryptedDataSize) {
- // add a placeholder that covers the hole where the encrypted indexes are
- footerBuilder.addStreams(
- OrcProto.Stream.newBuilder()
- .setKind(OrcProto.Stream.Kind.ENCRYPTED_DATA)
- .setLength(sizes.data - unencryptedDataSize));
- }
- }
-
- @Override
- public void finalizeStripe(
- OrcProto.StripeFooter.Builder footerBuilder,
- OrcProto.StripeInformation.Builder dirEntry)
- throws IOException {
- SizeCounters sizes = new SizeCounters();
- buildStreamList(footerBuilder, sizes);
-
- OrcProto.StripeFooter footer = footerBuilder.build();
-
- // Do we need to pad the file so the stripe doesn't straddle a block boundary?
- padStripe(sizes.total() + footer.getSerializedSize());
-
- // write the unencrypted index streams
- unencrypted.writeStreams(StreamName.Area.INDEX, rawWriter);
- // write the encrypted index streams
- for (VariantTracker variant : variants.values()) {
- variant.writeStreams(StreamName.Area.INDEX, rawWriter);
- }
-
- // write the unencrypted data streams
- unencrypted.writeStreams(StreamName.Area.DATA, rawWriter);
- // write out the unencrypted data streams
- for (VariantTracker variant : variants.values()) {
- variant.writeStreams(StreamName.Area.DATA, rawWriter);
- }
-
- // Write out the footer.
- writeStripeFooter(footer, sizes, dirEntry);
-
- // fill in the data sizes
- dirEntry.setDataLength(sizes.data);
- dirEntry.setIndexLength(sizes.index);
-
- stripeNumber += 1;
- }
-
- @Override
- public void writeHeader() throws IOException {
- rawWriter.write(OrcFile.MAGIC.getBytes());
- headerLength = rawWriter.getPos();
- }
-
- @Override
- public BufferedStream createDataStream(StreamName name) {
- VariantTracker variant = getVariant(name.getEncryption());
- BufferedStream result = variant.streams.get(name);
- if (result == null) {
- result = new BufferedStream();
- variant.streams.put(name, result);
- }
- return result;
- }
-
- protected OutputStream createIndexStream(StreamName name) {
- BufferedStream buffer = createDataStream(name);
- VariantTracker tracker = getVariant(name.getEncryption());
- StreamOptions options =
- SerializationUtils.getCustomizedCodec(
- tracker.options, compressionStrategy, name.getKind());
- if (options.isEncrypted()) {
- if (options == tracker.options) {
- options = new StreamOptions(options);
- }
- options.modifyIv(CryptoUtils.modifyIvForStream(name, stripeNumber + 1));
- }
- return new OutStream(name.toString(), options, buffer);
- }
-
- @Override
- public void writeIndex(StreamName name, OrcProto.RowIndex.Builder index) throws IOException {
- OutputStream stream = createIndexStream(name);
- index.build().writeTo(stream);
- stream.flush();
- }
-
- @Override
- public void writeBloomFilter(StreamName name, OrcProto.BloomFilterIndex.Builder bloom)
- throws IOException {
- OutputStream stream = createIndexStream(name);
- bloom.build().writeTo(stream);
- stream.flush();
- }
-
- @Override
- public void writeStatistics(StreamName name, OrcProto.ColumnStatistics.Builder statistics) {
- VariantTracker tracker = getVariant(name.getEncryption());
- if (name.getKind() == OrcProto.Stream.Kind.FILE_STATISTICS) {
- tracker.fileStats[name.getColumn() - tracker.rootColumn] = statistics.build();
- } else {
- tracker.stripeStats[name.getColumn() - tracker.rootColumn].add(statistics.build());
- }
- }
}
diff --git a/flink-formats/flink-orc/src/test/java/org/apache/flink/orc/OrcColumnarRowInputFormatTest.java b/flink-formats/flink-orc/src/test/java/org/apache/flink/orc/OrcColumnarRowInputFormatTest.java
index 04d6bd74ab5..2b1458c8179 100644
--- a/flink-formats/flink-orc/src/test/java/org/apache/flink/orc/OrcColumnarRowInputFormatTest.java
+++ b/flink-formats/flink-orc/src/test/java/org/apache/flink/orc/OrcColumnarRowInputFormatTest.java
@@ -204,26 +204,20 @@ class OrcColumnarRowInputFormatTest {
RowType tableType =
RowType.of(
- new LogicalType[] {
- /* 0 */ DataTypes.INT().getLogicalType(),
- /* 1 */ DataTypes.INT().getLogicalType(), // part-1
- /* 2 */ DataTypes.STRING().getLogicalType(),
- /* 3 */ DataTypes.BIGINT().getLogicalType(), // part-2
- /* 4 */ DataTypes.STRING().getLogicalType(),
- /* 5 */ DataTypes.STRING().getLogicalType(), // part-3
- /* 6 */ DataTypes.STRING().getLogicalType(),
- /* 7 */ DataTypes.INT().getLogicalType(),
- /* 8 */ DataTypes.DECIMAL(10, 5).getLogicalType(), // part-4
- /* 9 */ DataTypes.STRING().getLogicalType(),
- /* 10*/ DataTypes.INT().getLogicalType(),
- /* 11*/ DataTypes.INT().getLogicalType(),
- /* 12*/ DataTypes.STRING().getLogicalType(), // part-5
- /* 13*/ DataTypes.INT().getLogicalType()
- },
- new String[] {
- "_col0", "f1", "_col1", "f3", "_col2", "f5", "_col3", "_col4", "f8",
- "_col5", "_col6", "_col7", "f13", "_col8"
- });
+ /* 0 */ DataTypes.INT().getLogicalType(),
+ /* 1 */ DataTypes.INT().getLogicalType(), // part-1
+ /* 2 */ DataTypes.STRING().getLogicalType(),
+ /* 3 */ DataTypes.BIGINT().getLogicalType(), // part-2
+ /* 4 */ DataTypes.STRING().getLogicalType(),
+ /* 5 */ DataTypes.STRING().getLogicalType(), // part-3
+ /* 6 */ DataTypes.STRING().getLogicalType(),
+ /* 7 */ DataTypes.INT().getLogicalType(),
+ /* 8 */ DataTypes.DECIMAL(10, 5).getLogicalType(), // part-4
+ /* 9 */ DataTypes.STRING().getLogicalType(),
+ /* 11*/ DataTypes.INT().getLogicalType(),
+ /* 12*/ DataTypes.INT().getLogicalType(),
+ /* 13*/ DataTypes.STRING().getLogicalType(), // part-5
+ /* 14*/ DataTypes.INT().getLogicalType());
int[] projectedFields = {8, 1, 3, 0, 5, 2};
diff --git a/flink-formats/flink-orc/src/test/java/org/apache/flink/orc/OrcColumnarRowSplitReaderTest.java b/flink-formats/flink-orc/src/test/java/org/apache/flink/orc/OrcColumnarRowSplitReaderTest.java
index a7fd08c343d..bf0418c657f 100644
--- a/flink-formats/flink-orc/src/test/java/org/apache/flink/orc/OrcColumnarRowSplitReaderTest.java
+++ b/flink-formats/flink-orc/src/test/java/org/apache/flink/orc/OrcColumnarRowSplitReaderTest.java
@@ -73,11 +73,6 @@ public class OrcColumnarRowSplitReaderTest {
DataTypes.INT()
};
- private final String[] testSchemaNameFlat =
- new String[] {
- "_col0", "_col1", "_col2", "_col3", "_col4", "_col5", "_col6", "_col7", "_col8"
- };
-
private final DataType[] testSchemaDecimal = new DataType[] {DataTypes.DECIMAL(10, 5)};
private static Path testFileFlat;
@@ -102,12 +97,7 @@ public class OrcColumnarRowSplitReaderTest {
for (FileInputSplit split : splits) {
try (OrcColumnarRowSplitReader reader =
- createReader(
- new int[] {0, 1},
- testSchemaFlat,
- testSchemaNameFlat,
- new HashMap<>(),
- split)) {
+ createReader(new int[] {0, 1}, testSchemaFlat, new HashMap<>(), split)) {
// read and count all rows
while (!reader.reachedEnd()) {
RowData row = reader.nextRecord(null);
@@ -129,12 +119,7 @@ public class OrcColumnarRowSplitReaderTest {
FileInputSplit[] splits = createSplits(testFileDecimal, 1);
try (OrcColumnarRowSplitReader reader =
- createReader(
- new int[] {0},
- testSchemaDecimal,
- new String[] {"_col0"},
- new HashMap<>(),
- splits[0])) {
+ createReader(new int[] {0}, testSchemaDecimal, new HashMap<>(), splits[0])) {
assertThat(reader.reachedEnd()).isFalse();
RowData row = reader.nextRecord(null);
@@ -191,14 +176,10 @@ public class OrcColumnarRowSplitReaderTest {
/* 7 */ DataTypes.INT(),
/* 8 */ DataTypes.DECIMAL(10, 5), // part-4
/* 9 */ DataTypes.STRING(),
- /* 10*/ DataTypes.INT(),
/* 11*/ DataTypes.INT(),
- /* 12*/ DataTypes.STRING(), // part-5
- /* 13*/ DataTypes.INT()
- },
- new String[] {
- "_col0", "f1", "_col1", "f3", "_col2", "f5", "_col3", "_col4", "f8",
- "_col5", "_col6", "_col7", "f13", "_col8"
+ /* 12*/ DataTypes.INT(),
+ /* 13*/ DataTypes.STRING(), // part-5
+ /* 14*/ DataTypes.INT()
},
partSpec,
split)) {
@@ -241,12 +222,7 @@ public class OrcColumnarRowSplitReaderTest {
for (FileInputSplit split : splits) {
try (OrcColumnarRowSplitReader reader =
- createReader(
- new int[] {2, 0, 1},
- testSchemaFlat,
- testSchemaNameFlat,
- new HashMap<>(),
- split)) {
+ createReader(new int[] {2, 0, 1}, testSchemaFlat, new HashMap<>(), split)) {
// read and count all rows
while (!reader.reachedEnd()) {
RowData row = reader.nextRecord(null);
@@ -427,25 +403,10 @@ public class OrcColumnarRowSplitReaderTest {
Map