forked from databricks/databricks-jdbc
-
Notifications
You must be signed in to change notification settings - Fork 0
Expand file tree
/
Copy pathAbstractArrowResultChunk.java
More file actions
362 lines (325 loc) · 12.5 KB
/
AbstractArrowResultChunk.java
File metadata and controls
362 lines (325 loc) · 12.5 KB
1
2
3
4
5
6
7
8
9
10
11
12
13
14
15
16
17
18
19
20
21
22
23
24
25
26
27
28
29
30
31
32
33
34
35
36
37
38
39
40
41
42
43
44
45
46
47
48
49
50
51
52
53
54
55
56
57
58
59
60
61
62
63
64
65
66
67
68
69
70
71
72
73
74
75
76
77
78
79
80
81
82
83
84
85
86
87
88
89
90
91
92
93
94
95
96
97
98
99
100
101
102
103
104
105
106
107
108
109
110
111
112
113
114
115
116
117
118
119
120
121
122
123
124
125
126
127
128
129
130
131
132
133
134
135
136
137
138
139
140
141
142
143
144
145
146
147
148
149
150
151
152
153
154
155
156
157
158
159
160
161
162
163
164
165
166
167
168
169
170
171
172
173
174
175
176
177
178
179
180
181
182
183
184
185
186
187
188
189
190
191
192
193
194
195
196
197
198
199
200
201
202
203
204
205
206
207
208
209
210
211
212
213
214
215
216
217
218
219
220
221
222
223
224
225
226
227
228
229
230
231
232
233
234
235
236
237
238
239
240
241
242
243
244
245
246
247
248
249
250
251
252
253
254
255
256
257
258
259
260
261
262
263
264
265
266
267
268
269
270
271
272
273
274
275
276
277
278
279
280
281
282
283
284
285
286
287
288
289
290
291
292
293
294
295
296
297
298
299
300
301
302
303
304
305
306
307
308
309
310
311
312
313
314
315
316
317
318
319
320
321
322
323
324
325
326
327
328
329
330
331
332
333
334
335
336
337
338
339
340
341
342
343
344
345
346
347
348
349
350
351
352
353
354
355
356
357
358
359
360
361
362
package com.databricks.jdbc.api.impl.arrow;
import static com.databricks.jdbc.common.DatabricksJdbcConstants.ARROW_METADATA_KEY;
import com.databricks.jdbc.common.CompressionCodec;
import com.databricks.jdbc.common.util.DriverUtil;
import com.databricks.jdbc.dbclient.IDatabricksHttpClient;
import com.databricks.jdbc.dbclient.impl.common.StatementId;
import com.databricks.jdbc.exception.DatabricksParsingException;
import com.databricks.jdbc.exception.DatabricksSQLException;
import com.databricks.jdbc.log.JdbcLogger;
import com.databricks.jdbc.log.JdbcLoggerFactory;
import com.databricks.jdbc.model.core.ExternalLink;
import java.io.IOException;
import java.io.InputStream;
import java.nio.channels.ClosedByInterruptException;
import java.time.Instant;
import java.util.ArrayList;
import java.util.List;
import java.util.concurrent.CompletableFuture;
import java.util.concurrent.ExecutionException;
import java.util.concurrent.TimeUnit;
import java.util.concurrent.TimeoutException;
import java.util.stream.Collectors;
import org.apache.arrow.memory.BufferAllocator;
import org.apache.arrow.memory.RootAllocator;
import org.apache.arrow.vector.ValueVector;
import org.apache.arrow.vector.VectorSchemaRoot;
import org.apache.arrow.vector.ipc.ArrowStreamReader;
import org.apache.arrow.vector.util.TransferPair;
import org.apache.commons.lang3.exception.ExceptionUtils;
/**
* An abstract class that represents a chunk of query result.
*
* <p>This class provides methods for downloading, processing, and releasing the data in the chunk.
* It also manages the state of the chunk and provides access to the data as Arrow record batches.
*/
public abstract class AbstractArrowResultChunk {
private static final JdbcLogger LOGGER =
JdbcLoggerFactory.getLogger(AbstractArrowResultChunk.class);
protected static final Integer SECONDS_BUFFER_FOR_EXPIRY = 60;
protected static final long CHUNK_READY_TIMEOUT_SECONDS = 30;
protected final long numRows;
protected final long rowOffset;
protected final long chunkIndex;
protected final StatementId statementId;
protected final BufferAllocator rootAllocator;
/**
* Future to track when the chunk becomes ready for consumption. This includes both the download
* and processing phases. The state of the Future is updated by the {@link ChunkDownloadTask} and
* indicates when the chunk's data is fully processed and available for use.
*/
protected final CompletableFuture<Void> chunkReadyFuture;
protected final ArrowResultChunkStateMachine stateMachine;
protected List<List<ValueVector>> recordBatchList;
protected ExternalLink chunkLink;
protected Instant expiryTime;
protected String errorMessage;
protected List<String> arrowMetadata;
static final class ArrowData {
private final List<List<ValueVector>> valueVectors;
private final List<String> metadata;
public ArrowData(List<List<ValueVector>> valueVectors, List<String> metadata) {
this.valueVectors = valueVectors;
this.metadata = metadata;
}
public List<List<ValueVector>> getValueVectors() {
return valueVectors;
}
public List<String> getMetadata() {
return metadata;
}
}
protected AbstractArrowResultChunk(
long numRows,
long rowOffset,
long chunkIndex,
StatementId statementId,
ChunkStatus initialStatus,
ExternalLink chunkLink,
Instant expiryTime) {
this.numRows = numRows;
this.rowOffset = rowOffset;
this.chunkIndex = chunkIndex;
this.statementId = statementId;
this.rootAllocator = new RootAllocator(Integer.MAX_VALUE);
this.chunkReadyFuture = new CompletableFuture<>();
this.chunkLink = chunkLink;
this.expiryTime = expiryTime;
this.stateMachine = new ArrowResultChunkStateMachine(initialStatus, chunkIndex, statementId);
}
/**
* Returns the index of this chunk.
*
* @return chunk index
*/
public Long getChunkIndex() {
return chunkIndex;
}
/**
* Checks if the chunk link is invalid or expired.
*
* @return true if link is invalid, false otherwise
*/
public boolean isChunkLinkInvalid() {
return getStatus() == ChunkStatus.PENDING
|| (!DriverUtil.isRunningAgainstFake()
&& expiryTime.minusSeconds(SECONDS_BUFFER_FOR_EXPIRY).isBefore(Instant.now()));
}
/**
* Releases all resources associated with this chunk.
*
* @return true if chunk was released, false if it was already released
*/
public boolean releaseChunk() {
if (getStatus() == ChunkStatus.CHUNK_RELEASED) {
return false;
}
if (getStatus() == ChunkStatus.PROCESSING_SUCCEEDED) {
logAllocatorStats("BeforeRelease");
purgeArrowData(this.recordBatchList);
rootAllocator.close();
}
setStatus(ChunkStatus.CHUNK_RELEASED);
return true;
}
/**
* Downloads and initializes data for this chunk using the provided HTTP client and compression
* codec.
*
* @param httpClient the HTTP client to use for downloading
* @param compressionCodec the compression codec to use for decompression
* @throws DatabricksParsingException if there is an error parsing the data
* @throws IOException if there is an error downloading or reading the data
*/
protected abstract void downloadData(
IDatabricksHttpClient httpClient, CompressionCodec compressionCodec)
throws DatabricksParsingException, IOException;
/** Handles a failure during the download or processing of this chunk. */
protected abstract void handleFailure(Exception exception, ChunkStatus failedStatus)
throws DatabricksParsingException;
/**
* Returns the number of record batches in the chunk.
*
* @return number of record batches
*/
protected int getRecordBatchCountInChunk() {
return getStatus() == ChunkStatus.PROCESSING_SUCCEEDED ? recordBatchList.size() : 0;
}
/**
* Returns the list of record batches, where each record batch is a list of value vectors.
*
* @return List of record batches
*/
protected List<List<ValueVector>> getRecordBatchList() {
return recordBatchList;
}
/**
* Returns the total number of rows in the chunk.
*
* @return number of rows
*/
protected long getNumRows() {
return numRows;
}
/**
* Returns the value vector for a specific record batch and column.
*
* @param recordBatchIndex index of the record batch
* @param columnIndex index of the column
* @return ValueVector for the specified position
*/
protected ValueVector getColumnVector(int recordBatchIndex, int columnIndex) {
return recordBatchList.get(recordBatchIndex).get(columnIndex);
}
/**
* Returns the current status of the chunk.
*
* @return current ChunkStatus
*/
protected ChunkStatus getStatus() {
return stateMachine.getCurrentStatus();
}
/**
* Updates the status of the chunk.
*
* @param targetStatus new status to set
*/
protected void setStatus(ChunkStatus targetStatus) {
try {
stateMachine.transition(targetStatus);
} catch (DatabricksParsingException e) {
LOGGER.warn(
"Failed to transition to state [%s] from state [%s] for chunk [%d] and statement [%s]. Stack trace: %s",
targetStatus, getStatus(), chunkIndex, statementId, ExceptionUtils.getStackTrace(e));
}
}
/**
* Returns an iterator for traversing the rows in this chunk.
*
* @return ArrowResultChunkIterator for this chunk
*/
protected ArrowResultChunkIterator getChunkIterator() {
return new ArrowResultChunkIterator(this);
}
/**
* Sets the external link details for this chunk.
*
* @param chunk the external link information
*/
protected void setChunkLink(ExternalLink chunk) {
chunkLink = chunk;
expiryTime = Instant.parse(chunk.getExpiration());
setStatus(ChunkStatus.URL_FETCHED);
}
protected CompletableFuture<Void> getChunkReadyFuture() {
return chunkReadyFuture;
}
/**
* Waits for the chunk to be ready for consumption.
*
* @throws ExecutionException if the chunk download or processing throws an exception
* @throws InterruptedException if the thread is interrupted while waiting
* @throws TimeoutException if the chunk is not ready within the timeout
*/
protected void waitForChunkReady()
throws ExecutionException, InterruptedException, TimeoutException {
try {
chunkReadyFuture.get(CHUNK_READY_TIMEOUT_SECONDS, TimeUnit.SECONDS);
} catch (InterruptedException e) {
Thread.currentThread().interrupt();
throw e;
}
}
/**
* Decompresses the given {@link InputStream} and initializes {@link #recordBatchList} from
* decompressed stream.
*
* @param inputStream the input stream to decompress
* @throws DatabricksSQLException if decompression fails
* @throws IOException if reading from the stream fails
*/
protected void initializeData(InputStream inputStream)
throws DatabricksSQLException, IOException {
LOGGER.debug("Parsing data for chunk index %s and statement %s", chunkIndex, statementId);
ArrowData arrowData = getRecordBatchList(inputStream, rootAllocator, statementId, chunkIndex);
recordBatchList = arrowData.getValueVectors();
arrowMetadata = arrowData.getMetadata();
LOGGER.debug("Data parsed for chunk index %s and statement %s", chunkIndex, statementId);
}
protected List<String> getArrowMetadata() {
return arrowMetadata;
}
/**
* Reads Arrow format data from an input stream and converts it into a list of record batches.
* Each record batch is represented as a list of {@link ValueVector}s.
*/
private ArrowData getRecordBatchList(
InputStream inputStream,
BufferAllocator rootAllocator,
StatementId statementId,
long chunkIndex)
throws IOException {
List<List<ValueVector>> recordBatchList = new ArrayList<>();
List<String> metadata = new ArrayList<>();
try (ArrowStreamReader arrowStreamReader = new ArrowStreamReader(inputStream, rootAllocator)) {
VectorSchemaRoot vectorSchemaRoot = arrowStreamReader.getVectorSchemaRoot();
boolean fetchedMetadata = false;
while (arrowStreamReader.loadNextBatch()) {
if (!fetchedMetadata) {
metadata = getMetadataInformationFromSchemaRoot(vectorSchemaRoot);
fetchedMetadata = true;
}
recordBatchList.add(getVectorsFromSchemaRoot(vectorSchemaRoot, rootAllocator));
vectorSchemaRoot.clear();
}
} catch (ClosedByInterruptException e) {
// release resources if thread is interrupted when reading arrow data
LOGGER.error(
e,
"Data parsing interrupted for chunk index [%s] and statement [%s]. Error [%s]",
chunkIndex,
statementId,
e.getMessage());
purgeArrowData(recordBatchList);
} catch (IOException e) {
LOGGER.error(
"Error while reading arrow data, purging the local list and rethrowing the exception.");
purgeArrowData(recordBatchList);
throw e;
}
return new ArrowData(recordBatchList, metadata);
}
private List<String> getMetadataInformationFromSchemaRoot(VectorSchemaRoot vectorSchemaRoot) {
return vectorSchemaRoot.getFieldVectors().stream()
.map(fieldVector -> fieldVector.getField().getMetadata().get(ARROW_METADATA_KEY))
.collect(Collectors.toList());
}
/**
* Transfers the data from the given {@link VectorSchemaRoot} to a list of {@link ValueVector}s.
*/
private List<ValueVector> getVectorsFromSchemaRoot(
VectorSchemaRoot vectorSchemaRoot, BufferAllocator rootAllocator) {
return vectorSchemaRoot.getFieldVectors().stream()
.map(
fieldVector -> {
TransferPair transferPair = fieldVector.getTransferPair(rootAllocator);
transferPair.transfer();
return transferPair.getTo();
})
.collect(Collectors.toList());
}
private void logAllocatorStats(String event) {
long allocatedMemory = rootAllocator.getAllocatedMemory();
long peakMemory = rootAllocator.getPeakMemoryAllocation();
long headRoom = rootAllocator.getHeadroom();
long initReservation = rootAllocator.getInitReservation();
LOGGER.debug(
"Chunk allocator stats Log - Event: %s, Chunk Index: %s, Allocated Memory: %s, Peak Memory: %s, Headroom: %s, Init Reservation: %s",
event, chunkIndex, allocatedMemory, peakMemory, headRoom, initReservation);
}
/** Releases all Arrow-related resources and clears the record batch list. */
private void purgeArrowData(List<List<ValueVector>> recordBatchList) {
recordBatchList.forEach(vectors -> vectors.forEach(ValueVector::close));
recordBatchList.clear();
}
}