@@ -1145,11 +1145,9 @@ public ColumnChunkPageReadStore readFilteredRowGroup(int blockIndex, RowRanges r
1145
1145
* @throws IOException any IOE.
1146
1146
*/
1147
1147
private void readAllPartsVectoredOrNormal (List <ConsecutivePartList > allParts , ChunkListBuilder builder )
1148
- throws IOException {
1149
- boolean isVectoredIO = options .useHadoopVectoredIO ()
1150
- && f .readVectoredAvailable ()
1151
- && partsLengthValidForVectoredIO (allParts );
1152
- if (isVectoredIO ) {
1148
+ throws IOException {
1149
+
1150
+ if (shouldUseVectoredIO (allParts )) {
1153
1151
readVectored (allParts , builder );
1154
1152
} else {
1155
1153
for (ConsecutivePartList consecutiveChunks : allParts ) {
@@ -1158,16 +1156,34 @@ private void readAllPartsVectoredOrNormal(List<ConsecutivePartList> allParts, Ch
1158
1156
}
1159
1157
}
1160
1158
1159
+ /**
1160
+ * Should the read use vectored IO?
1161
+ * <p>
1162
+ * This returns true if all necessary conditions are met:
1163
+ * <ol>
1164
+ * <li> The option is enabled</li>
1165
+ * <li> The Hadoop version supports vectored IO</li>
1166
+ * <li> Thfe part lengths are all valid for vectored IO</li>
1167
+ * </ol>
1168
+ * @param allParts all parts to read.
1169
+ * @return true or false.
1170
+ */
1171
+ private boolean shouldUseVectoredIO (final List <ConsecutivePartList > allParts ) {
1172
+ return options .useHadoopVectoredIO () && f .readVectoredAvailable () && arePartLengthsValidForVectoredIO (allParts );
1173
+ }
1174
+
1161
1175
/**
1162
1176
* Vectored IO doesn't support reading ranges of size greater than
1163
1177
* Integer.MAX_VALUE.
1164
1178
* @param allParts all parts to read.
1165
1179
* @return true or false.
1166
1180
*/
1167
- private boolean partsLengthValidForVectoredIO (List <ConsecutivePartList > allParts ) {
1168
- for (ConsecutivePartList consecutivePart : allParts ) {
1181
+ private boolean arePartLengthsValidForVectoredIO (List <ConsecutivePartList > allParts ) {
1182
+ for (ConsecutivePartList consecutivePart : allParts ) {
1169
1183
if (consecutivePart .length >= Integer .MAX_VALUE ) {
1170
- LOG .debug ("Part length {} greater than Integer.MAX_VALUE thus disabling vectored IO" , consecutivePart .length );
1184
+ LOG .debug (
1185
+ "Part length {} greater than Integer.MAX_VALUE thus disabling vectored IO" ,
1186
+ consecutivePart .length );
1171
1187
return false ;
1172
1188
}
1173
1189
}
@@ -1176,26 +1192,36 @@ private boolean partsLengthValidForVectoredIO(List<ConsecutivePartList> allParts
1176
1192
1177
1193
/**
1178
1194
* Read all parts through vectored IO.
1195
+ * <p>
1196
+ * The API is available in recent hadoop builds for all implementations of PositionedReadable;
1197
+ * the default implementation simply does a sequence of reads at different offsets.
1198
+ * <p>
1199
+ * If directly implemented by a Filesystem then it is likely to be a more efficient
1200
+ * operation such as a scatter-gather read (native IO) or set of parallel
1201
+ * GET requests against an object store.
1179
1202
* @param allParts all parts to be read.
1180
1203
* @param builder used to build chunk list to read the pages for the different columns.
1181
1204
* @throws IOException any IOE.
1182
1205
*/
1183
- private void readVectored (List <ConsecutivePartList > allParts ,
1184
- ChunkListBuilder builder ) throws IOException {
1206
+ private void readVectored (List <ConsecutivePartList > allParts , ChunkListBuilder builder ) throws IOException {
1185
1207
1186
1208
List <ParquetFileRange > ranges = new ArrayList <>(allParts .size ());
1209
+ long totalSize = 0 ;
1187
1210
for (ConsecutivePartList consecutiveChunks : allParts ) {
1188
- Preconditions .checkArgument (consecutiveChunks .length < Integer .MAX_VALUE ,
1189
- "Invalid length %s for vectored read operation. It must be less than max integer value." ,
1190
- consecutiveChunks .length );
1191
- ranges .add (new ParquetFileRange (consecutiveChunks .offset , (int ) consecutiveChunks .length ));
1192
- }
1193
- LOG .debug ("Doing vectored IO for ranges {}" , ranges );
1211
+ final long len = consecutiveChunks .length ;
1212
+ Preconditions .checkArgument (
1213
+ len < Integer .MAX_VALUE ,
1214
+ "Invalid length %s for vectored read operation. It must be less than max integer value." ,
1215
+ len );
1216
+ ranges .add (new ParquetFileRange (consecutiveChunks .offset , (int ) len ));
1217
+ totalSize += len ;
1218
+ }
1219
+ LOG .info ("Reading {} bytes of data with vectored IO in {} ranges" , totalSize , ranges .size ());
1194
1220
ByteBufferAllocator allocator = options .getAllocator ();
1195
- //blocking or asynchronous vectored read.
1221
+ // Request a vectored read;
1196
1222
f .readVectored (ranges , allocator ::allocate );
1197
1223
int k = 0 ;
1198
- for (ConsecutivePartList consecutivePart : allParts ) {
1224
+ for (ConsecutivePartList consecutivePart : allParts ) {
1199
1225
ParquetFileRange currRange = ranges .get (k ++);
1200
1226
consecutivePart .readFromVectoredRange (currRange , builder );
1201
1227
}
@@ -2093,22 +2119,26 @@ private void setReadMetrics(long startNs) {
2093
2119
}
2094
2120
2095
2121
/**
2096
- * Populate data in a parquet file range from vectored range.
2122
+ * Populate data in a parquet file range from a vectored range; will block for up
2123
+ * to {@link #HADOOP_VECTORED_READ_TIMEOUT_SECONDS} seconds.
2097
2124
* @param currRange range to populated.
2098
2125
* @param builder used to build chunk list to read the pages for the different columns.
2099
- * @throws IOException if there is an error while reading from the stream.
2126
+ * @throws IOException if there is an error while reading from the stream, including a timeout .
2100
2127
*/
2101
- public void readFromVectoredRange (ParquetFileRange currRange ,
2102
- ChunkListBuilder builder ) throws IOException {
2128
+ public void readFromVectoredRange (ParquetFileRange currRange , ChunkListBuilder builder ) throws IOException {
2103
2129
ByteBuffer buffer ;
2130
+ final long timeoutSeconds = HADOOP_VECTORED_READ_TIMEOUT_SECONDS ;
2104
2131
try {
2105
- LOG .debug ("Waiting for vectored read to finish for range {} " , currRange );
2106
- buffer = BindingUtils .awaitFuture (currRange .getDataReadFuture (),
2107
- HADOOP_VECTORED_READ_TIMEOUT_SECONDS , TimeUnit .SECONDS );
2132
+ LOG .debug (
2133
+ "Waiting for vectored read to finish for range {} with timeout {} seconds" ,
2134
+ currRange ,
2135
+ timeoutSeconds );
2136
+ buffer = BindingUtils .awaitFuture (currRange .getDataReadFuture (), timeoutSeconds , TimeUnit .SECONDS );
2108
2137
// report in a counter the data we just scanned
2109
2138
BenchmarkCounter .incrementBytesRead (currRange .getLength ());
2110
2139
} catch (TimeoutException e ) {
2111
- String error = String .format ("Timeout while fetching result for %s" , currRange );
2140
+ String error = String .format (
2141
+ "Timeout while fetching result for %s with time limit %d seconds" , currRange , timeoutSeconds );
2112
2142
LOG .error (error , e );
2113
2143
throw new IOException (error , e );
2114
2144
}
0 commit comments