|
45 | 45 | import java.util.Locale; |
46 | 46 | import java.util.Map; |
47 | 47 | import java.util.Objects; |
| 48 | +import java.util.Set; |
48 | 49 | import java.util.function.BiFunction; |
49 | 50 | import java.util.function.Function; |
50 | 51 | import java.util.stream.Collectors; |
51 | 52 | import java.util.stream.IntStream; |
52 | 53 | import org.apache.hadoop.conf.Configuration; |
53 | 54 | import org.apache.hadoop.fs.Path; |
54 | 55 | import org.apache.iceberg.FileFormat; |
| 56 | +import org.apache.iceberg.MetadataColumns; |
55 | 57 | import org.apache.iceberg.MetricsConfig; |
56 | 58 | import org.apache.iceberg.PartitionSpec; |
57 | 59 | import org.apache.iceberg.Schema; |
|
79 | 81 | import org.apache.iceberg.io.OutputFile; |
80 | 82 | import org.apache.iceberg.mapping.NameMapping; |
81 | 83 | import org.apache.iceberg.relocated.com.google.common.base.Preconditions; |
| 84 | +import org.apache.iceberg.relocated.com.google.common.collect.ImmutableSet; |
82 | 85 | import org.apache.iceberg.relocated.com.google.common.collect.Maps; |
| 86 | +import org.apache.iceberg.relocated.com.google.common.collect.Sets; |
| 87 | +import org.apache.iceberg.types.TypeUtil; |
83 | 88 | import org.apache.iceberg.util.ArrayUtil; |
84 | 89 | import org.apache.iceberg.util.PropertyUtil; |
85 | 90 | import org.apache.orc.CompressionKind; |
@@ -179,9 +184,8 @@ public WriteBuilder metricsConfig(MetricsConfig newMetricsConfig) { |
179 | 184 | return this; |
180 | 185 | } |
181 | 186 |
|
182 | | - // Supposed to always be a private method used strictly by data and delete write builders |
183 | | - private WriteBuilder createContextFunc( |
184 | | - Function<Map<String, String>, Context> newCreateContextFunc) { |
| 187 | + // supposed to always be a private method used strictly by data and delete write builders |
| 188 | + WriteBuilder createContextFunc(Function<Map<String, String>, Context> newCreateContextFunc) { |
185 | 189 | this.createContextFunc = newCreateContextFunc; |
186 | 190 | return this; |
187 | 191 | } |
@@ -219,7 +223,7 @@ public <D> FileAppender<D> build() { |
219 | 223 | metricsConfig); |
220 | 224 | } |
221 | 225 |
|
222 | | - private static class Context { |
| 226 | + static class Context { |
223 | 227 | private final long stripeSize; |
224 | 228 | private final long blockSize; |
225 | 229 | private final int vectorizedRowBatchSize; |
@@ -699,6 +703,7 @@ public static class ReadBuilder { |
699 | 703 | private Function<TypeDescription, OrcRowReader<?>> readerFunc; |
700 | 704 | private Function<TypeDescription, OrcBatchReader<?>> batchedReaderFunc; |
701 | 705 | private int recordsPerBatch = VectorizedRowBatch.DEFAULT_SIZE; |
| 706 | + private Set<Integer> constantFieldIds = ImmutableSet.of(); |
702 | 707 |
|
703 | 708 | private ReadBuilder(InputFile file) { |
704 | 709 | Preconditions.checkNotNull(file, "Input file cannot be null"); |
@@ -775,12 +780,18 @@ public ReadBuilder withNameMapping(NameMapping newNameMapping) { |
775 | 780 | return this; |
776 | 781 | } |
777 | 782 |
|
| 783 | + ReadBuilder constantFieldIds(Set<Integer> newConstantFieldIds) { |
| 784 | + this.constantFieldIds = newConstantFieldIds; |
| 785 | + return this; |
| 786 | + } |
| 787 | + |
778 | 788 | public <D> CloseableIterable<D> build() { |
779 | 789 | Preconditions.checkNotNull(schema, "Schema is required"); |
780 | 790 | return new OrcIterable<>( |
781 | 791 | file, |
782 | 792 | conf, |
783 | | - schema, |
| 793 | + TypeUtil.selectNot( |
| 794 | + schema, Sets.union(constantFieldIds, MetadataColumns.metadataFieldIds())), |
784 | 795 | nameMapping, |
785 | 796 | start, |
786 | 797 | length, |
|
0 commit comments