|
25 | 25 | import com.facebook.presto.spi.ConnectorSession;
|
26 | 26 | import com.facebook.presto.spi.ConnectorTableMetadata;
|
27 | 27 | import com.facebook.presto.spi.FixedPageSource;
|
| 28 | +import com.facebook.presto.spi.PrestoException; |
28 | 29 | import com.facebook.presto.spi.SchemaTableName;
|
29 | 30 | import com.facebook.presto.spi.SystemTable;
|
30 | 31 | import com.facebook.presto.spi.connector.ConnectorTransactionHandle;
|
31 | 32 | import com.google.common.collect.ImmutableList;
|
32 | 33 | import com.google.common.collect.ImmutableMap;
|
33 | 34 | import io.airlift.slice.Slices;
|
34 | 35 | import org.apache.iceberg.DataFile;
|
| 36 | +import org.apache.iceberg.FileScanTask; |
35 | 37 | import org.apache.iceberg.Schema;
|
36 | 38 | import org.apache.iceberg.Table;
|
37 | 39 | import org.apache.iceberg.TableScan;
|
| 40 | +import org.apache.iceberg.io.CloseableIterable; |
38 | 41 | import org.apache.iceberg.transforms.Transforms;
|
39 | 42 | import org.apache.iceberg.types.Conversions;
|
40 | 43 | import org.apache.iceberg.types.Type;
|
41 | 44 | import org.apache.iceberg.types.Types;
|
42 | 45 |
|
| 46 | +import java.io.IOException; |
43 | 47 | import java.nio.ByteBuffer;
|
44 | 48 | import java.util.List;
|
45 | 49 | import java.util.Map;
|
|
49 | 53 | import static com.facebook.presto.common.type.IntegerType.INTEGER;
|
50 | 54 | import static com.facebook.presto.common.type.VarbinaryType.VARBINARY;
|
51 | 55 | import static com.facebook.presto.common.type.VarcharType.VARCHAR;
|
| 56 | +import static com.facebook.presto.iceberg.IcebergErrorCode.ICEBERG_FILESYSTEM_ERROR; |
52 | 57 | import static com.facebook.presto.iceberg.IcebergUtil.getTableScan;
|
53 | 58 | import static com.facebook.presto.iceberg.util.PageListBuilder.forTable;
|
54 | 59 | import static com.google.common.collect.ImmutableMap.toImmutableMap;
|
@@ -121,53 +126,58 @@ private static List<Page> buildPages(ConnectorTableMetadata tableMetadata, Table
|
121 | 126 | TableScan tableScan = getTableScan(TupleDomain.all(), snapshotId, icebergTable).includeColumnStats();
|
122 | 127 | Map<Integer, Type> idToTypeMap = getIdToTypeMap(icebergTable.schema());
|
123 | 128 |
|
124 |
| - tableScan.planFiles().forEach(fileScanTask -> { |
125 |
| - DataFile dataFile = fileScanTask.file(); |
126 |
| - pagesBuilder.beginRow(); |
127 |
| - pagesBuilder.appendInteger(dataFile.content().id()); |
128 |
| - pagesBuilder.appendVarchar(dataFile.path().toString()); |
129 |
| - pagesBuilder.appendVarchar(dataFile.format().name()); |
130 |
| - pagesBuilder.appendBigint(dataFile.recordCount()); |
131 |
| - pagesBuilder.appendBigint(dataFile.fileSizeInBytes()); |
132 |
| - if (checkNonNull(dataFile.columnSizes(), pagesBuilder)) { |
133 |
| - pagesBuilder.appendIntegerBigintMap(dataFile.columnSizes()); |
| 129 | + try (CloseableIterable<FileScanTask> fileScanTasks = tableScan.planFiles()) { |
| 130 | + for (FileScanTask fileScanTask : fileScanTasks) { |
| 131 | + DataFile dataFile = fileScanTask.file(); |
| 132 | + pagesBuilder.beginRow(); |
| 133 | + pagesBuilder.appendInteger(dataFile.content().id()); |
| 134 | + pagesBuilder.appendVarchar(dataFile.path().toString()); |
| 135 | + pagesBuilder.appendVarchar(dataFile.format().name()); |
| 136 | + pagesBuilder.appendBigint(dataFile.recordCount()); |
| 137 | + pagesBuilder.appendBigint(dataFile.fileSizeInBytes()); |
| 138 | + if (checkNonNull(dataFile.columnSizes(), pagesBuilder)) { |
| 139 | + pagesBuilder.appendIntegerBigintMap(dataFile.columnSizes()); |
| 140 | + } |
| 141 | + if (checkNonNull(dataFile.valueCounts(), pagesBuilder)) { |
| 142 | + pagesBuilder.appendIntegerBigintMap(dataFile.valueCounts()); |
| 143 | + } |
| 144 | + if (checkNonNull(dataFile.nullValueCounts(), pagesBuilder)) { |
| 145 | + pagesBuilder.appendIntegerBigintMap(dataFile.nullValueCounts()); |
| 146 | + } |
| 147 | + if (checkNonNull(dataFile.nanValueCounts(), pagesBuilder)) { |
| 148 | + pagesBuilder.appendIntegerBigintMap(dataFile.nanValueCounts()); |
| 149 | + } |
| 150 | + if (checkNonNull(dataFile.lowerBounds(), pagesBuilder)) { |
| 151 | + pagesBuilder.appendIntegerVarcharMap(dataFile.lowerBounds().entrySet().stream() |
| 152 | + .filter(entry -> idToTypeMap.containsKey(entry.getKey())) |
| 153 | + .collect(toImmutableMap( |
| 154 | + Map.Entry<Integer, ByteBuffer>::getKey, |
| 155 | + entry -> Transforms.identity().toHumanString(idToTypeMap.get(entry.getKey()), |
| 156 | + Conversions.fromByteBuffer(idToTypeMap.get(entry.getKey()), entry.getValue()))))); |
| 157 | + } |
| 158 | + if (checkNonNull(dataFile.upperBounds(), pagesBuilder)) { |
| 159 | + pagesBuilder.appendIntegerVarcharMap(dataFile.upperBounds().entrySet().stream() |
| 160 | + .filter(entry -> idToTypeMap.containsKey(entry.getKey())) |
| 161 | + .collect(toImmutableMap( |
| 162 | + Map.Entry<Integer, ByteBuffer>::getKey, |
| 163 | + entry -> Transforms.identity().toHumanString(idToTypeMap.get(entry.getKey()), |
| 164 | + Conversions.fromByteBuffer(idToTypeMap.get(entry.getKey()), entry.getValue()))))); |
| 165 | + } |
| 166 | + if (checkNonNull(dataFile.keyMetadata(), pagesBuilder)) { |
| 167 | + pagesBuilder.appendVarbinary(Slices.wrappedBuffer(dataFile.keyMetadata())); |
| 168 | + } |
| 169 | + if (checkNonNull(dataFile.splitOffsets(), pagesBuilder)) { |
| 170 | + pagesBuilder.appendBigintArray(dataFile.splitOffsets()); |
| 171 | + } |
| 172 | + if (checkNonNull(dataFile.equalityFieldIds(), pagesBuilder)) { |
| 173 | + pagesBuilder.appendIntegerArray(dataFile.equalityFieldIds()); |
| 174 | + } |
| 175 | + pagesBuilder.endRow(); |
134 | 176 | }
|
135 |
| - if (checkNonNull(dataFile.valueCounts(), pagesBuilder)) { |
136 |
| - pagesBuilder.appendIntegerBigintMap(dataFile.valueCounts()); |
137 |
| - } |
138 |
| - if (checkNonNull(dataFile.nullValueCounts(), pagesBuilder)) { |
139 |
| - pagesBuilder.appendIntegerBigintMap(dataFile.nullValueCounts()); |
140 |
| - } |
141 |
| - if (checkNonNull(dataFile.nanValueCounts(), pagesBuilder)) { |
142 |
| - pagesBuilder.appendIntegerBigintMap(dataFile.nanValueCounts()); |
143 |
| - } |
144 |
| - if (checkNonNull(dataFile.lowerBounds(), pagesBuilder)) { |
145 |
| - pagesBuilder.appendIntegerVarcharMap(dataFile.lowerBounds().entrySet().stream() |
146 |
| - .filter(entry -> idToTypeMap.containsKey(entry.getKey())) |
147 |
| - .collect(toImmutableMap( |
148 |
| - Map.Entry<Integer, ByteBuffer>::getKey, |
149 |
| - entry -> Transforms.identity().toHumanString(idToTypeMap.get(entry.getKey()), |
150 |
| - Conversions.fromByteBuffer(idToTypeMap.get(entry.getKey()), entry.getValue()))))); |
151 |
| - } |
152 |
| - if (checkNonNull(dataFile.upperBounds(), pagesBuilder)) { |
153 |
| - pagesBuilder.appendIntegerVarcharMap(dataFile.upperBounds().entrySet().stream() |
154 |
| - .filter(entry -> idToTypeMap.containsKey(entry.getKey())) |
155 |
| - .collect(toImmutableMap( |
156 |
| - Map.Entry<Integer, ByteBuffer>::getKey, |
157 |
| - entry -> Transforms.identity().toHumanString(idToTypeMap.get(entry.getKey()), |
158 |
| - Conversions.fromByteBuffer(idToTypeMap.get(entry.getKey()), entry.getValue()))))); |
159 |
| - } |
160 |
| - if (checkNonNull(dataFile.keyMetadata(), pagesBuilder)) { |
161 |
| - pagesBuilder.appendVarbinary(Slices.wrappedBuffer(dataFile.keyMetadata())); |
162 |
| - } |
163 |
| - if (checkNonNull(dataFile.splitOffsets(), pagesBuilder)) { |
164 |
| - pagesBuilder.appendBigintArray(dataFile.splitOffsets()); |
165 |
| - } |
166 |
| - if (checkNonNull(dataFile.equalityFieldIds(), pagesBuilder)) { |
167 |
| - pagesBuilder.appendIntegerArray(dataFile.equalityFieldIds()); |
168 |
| - } |
169 |
| - pagesBuilder.endRow(); |
170 |
| - }); |
| 177 | + } |
| 178 | + catch (IOException e) { |
| 179 | + throw new PrestoException(ICEBERG_FILESYSTEM_ERROR, "failed to read table files", e); |
| 180 | + } |
171 | 181 |
|
172 | 182 | return pagesBuilder.build();
|
173 | 183 | }
|
|
0 commit comments