From 57ac936c143226bce7c30f8f478293dc423fc83a Mon Sep 17 00:00:00 2001 From: Jayant Singh Date: Thu, 27 Nov 2025 10:39:39 +0000 Subject: [PATCH 1/6] Add StreamingChunkProvider for result fetching Implement streaming chunk provider that fetches results without dependency on total chunk count. Key components: - StreamingChunkProvider: Memory-bounded parallel downloads with proactive link prefetching - ChunkLinkFetcher interface with SeaChunkLinkFetcher for SEA API - StreamingChunkDownloadTask: Simplified download task with retry logic - Support for initial links from ResultData to avoid extra fetch calls Enable via URL parameter: EnableStreamingChunkProvider=1 Next: Implement ThriftChunkLinkFetcher to unify SEA and Thrift code paths --- .../api/impl/DatabricksConnectionContext.java | 5 + .../jdbc/api/impl/arrow/ArrowResultChunk.java | 17 + .../api/impl/arrow/ArrowStreamResult.java | 89 ++- .../api/impl/arrow/ChunkLinkFetchResult.java | 111 ++++ .../jdbc/api/impl/arrow/ChunkLinkFetcher.java | 41 ++ .../api/impl/arrow/SeaChunkLinkFetcher.java | 105 ++++ .../arrow/StreamingChunkDownloadTask.java | 112 ++++ .../impl/arrow/StreamingChunkProvider.java | 570 ++++++++++++++++++ .../IDatabricksConnectionContext.java | 3 + .../jdbc/common/DatabricksJdbcUrlParams.java | 6 +- 10 files changed, 1051 insertions(+), 8 deletions(-) create mode 100644 src/main/java/com/databricks/jdbc/api/impl/arrow/ChunkLinkFetchResult.java create mode 100644 src/main/java/com/databricks/jdbc/api/impl/arrow/ChunkLinkFetcher.java create mode 100644 src/main/java/com/databricks/jdbc/api/impl/arrow/SeaChunkLinkFetcher.java create mode 100644 src/main/java/com/databricks/jdbc/api/impl/arrow/StreamingChunkDownloadTask.java create mode 100644 src/main/java/com/databricks/jdbc/api/impl/arrow/StreamingChunkProvider.java diff --git a/src/main/java/com/databricks/jdbc/api/impl/DatabricksConnectionContext.java b/src/main/java/com/databricks/jdbc/api/impl/DatabricksConnectionContext.java index 41d3f379b..cd6bffc83 100644 --- a/src/main/java/com/databricks/jdbc/api/impl/DatabricksConnectionContext.java +++ b/src/main/java/com/databricks/jdbc/api/impl/DatabricksConnectionContext.java @@ -1150,4 +1150,9 @@ public boolean getDisableOauthRefreshToken() { public boolean isTokenFederationEnabled() { return getParameter(DatabricksJdbcUrlParams.ENABLE_TOKEN_FEDERATION, "1").equals("1"); } + + @Override + public boolean isStreamingChunkProviderEnabled() { + return getParameter(DatabricksJdbcUrlParams.ENABLE_STREAMING_CHUNK_PROVIDER).equals("1"); + } } diff --git a/src/main/java/com/databricks/jdbc/api/impl/arrow/ArrowResultChunk.java b/src/main/java/com/databricks/jdbc/api/impl/arrow/ArrowResultChunk.java index 9852dc8af..17380c592 100644 --- a/src/main/java/com/databricks/jdbc/api/impl/arrow/ArrowResultChunk.java +++ b/src/main/java/com/databricks/jdbc/api/impl/arrow/ArrowResultChunk.java @@ -197,6 +197,23 @@ public Builder withChunkInfo(BaseChunkInfo baseChunkInfo) { return this; } + /** + * Sets chunk metadata directly without requiring a BaseChunkInfo object. Useful for streaming + * chunk creation where metadata comes from ExternalLink. + * + * @param chunkIndex The index of this chunk + * @param rowCount The number of rows in this chunk + * @param rowOffset The starting row offset for this chunk + * @return this builder + */ + public Builder withChunkMetadata(long chunkIndex, long rowCount, long rowOffset) { + this.chunkIndex = chunkIndex; + this.numRows = rowCount; + this.rowOffset = rowOffset; + this.status = status == null ? ChunkStatus.PENDING : status; + return this; + } + public Builder withInputStream(InputStream stream, long rowCount) { this.numRows = rowCount; this.inputStream = stream; diff --git a/src/main/java/com/databricks/jdbc/api/impl/arrow/ArrowStreamResult.java b/src/main/java/com/databricks/jdbc/api/impl/arrow/ArrowStreamResult.java index c86c27447..54f45c097 100644 --- a/src/main/java/com/databricks/jdbc/api/impl/arrow/ArrowStreamResult.java +++ b/src/main/java/com/databricks/jdbc/api/impl/arrow/ArrowStreamResult.java @@ -4,6 +4,7 @@ import com.databricks.jdbc.api.impl.ComplexDataTypeParser; import com.databricks.jdbc.api.impl.IExecutionResult; +import com.databricks.jdbc.api.internal.IDatabricksConnectionContext; import com.databricks.jdbc.api.internal.IDatabricksSession; import com.databricks.jdbc.api.internal.IDatabricksStatementInternal; import com.databricks.jdbc.common.CompressionCodec; @@ -18,11 +19,14 @@ import com.databricks.jdbc.model.client.thrift.generated.TGetResultSetMetadataResp; import com.databricks.jdbc.model.core.ColumnInfo; import com.databricks.jdbc.model.core.ColumnInfoTypeName; +import com.databricks.jdbc.model.core.ExternalLink; import com.databricks.jdbc.model.core.ResultData; import com.databricks.jdbc.model.core.ResultManifest; import com.google.common.annotations.VisibleForTesting; import java.util.ArrayList; +import java.util.Collection; import java.util.List; +import java.util.stream.Collectors; /** Result container for Arrow-based query results. */ public class ArrowStreamResult implements IExecutionResult { @@ -69,13 +73,7 @@ public ArrowStreamResult( "Creating ArrowStreamResult with remote links for statementId: {}", statementId.toSQLExecStatementId()); this.chunkProvider = - new RemoteChunkProvider( - statementId, - resultManifest, - resultData, - session, - httpClient, - session.getConnectionContext().getCloudFetchThreadPoolSize()); + createRemoteChunkProvider(statementId, resultManifest, resultData, session, httpClient); } this.columnInfos = resultManifest.getSchema().getColumnCount() == 0 @@ -83,6 +81,61 @@ public ArrowStreamResult( : new ArrayList<>(resultManifest.getSchema().getColumns()); } + /** + * Creates the appropriate remote chunk provider based on configuration. + * + * @param statementId The statement ID + * @param resultManifest The result manifest containing chunk metadata + * @param resultData The result data containing initial external links + * @param session The session for fetching additional chunks + * @param httpClient The HTTP client for downloading chunk data + * @return A ChunkProvider instance + */ + private static ChunkProvider createRemoteChunkProvider( + StatementId statementId, + ResultManifest resultManifest, + ResultData resultData, + IDatabricksSession session, + IDatabricksHttpClient httpClient) + throws DatabricksSQLException { + + IDatabricksConnectionContext connectionContext = session.getConnectionContext(); + + if (connectionContext.isStreamingChunkProviderEnabled()) { + LOGGER.info( + "Using StreamingChunkProvider for statementId: {}", statementId.toSQLExecStatementId()); + + ChunkLinkFetcher linkFetcher = new SeaChunkLinkFetcher(session, statementId); + CompressionCodec compressionCodec = resultManifest.getResultCompression(); + int maxChunksInMemory = connectionContext.getCloudFetchThreadPoolSize(); + int chunkReadyTimeoutSeconds = connectionContext.getChunkReadyTimeoutSeconds(); + double cloudFetchSpeedThreshold = connectionContext.getCloudFetchSpeedThreshold(); + + // Convert ExternalLinks to ChunkLinkInfo for the provider + Collection initialLinks = + convertToChunkLinkInfos(resultData.getExternalLinks()); + + return new StreamingChunkProvider( + linkFetcher, + httpClient, + compressionCodec, + statementId, + maxChunksInMemory, + chunkReadyTimeoutSeconds, + cloudFetchSpeedThreshold, + initialLinks); + } else { + // Use the original RemoteChunkProvider + return new RemoteChunkProvider( + statementId, + resultManifest, + resultData, + session, + httpClient, + connectionContext.getCloudFetchThreadPoolSize()); + } + } + public ArrowStreamResult( TFetchResultsResp resultsResp, boolean isInlineArrow, @@ -268,4 +321,26 @@ private void setColumnInfo(TGetResultSetMetadataResp resultManifest) { columnInfos.add(getColumnInfoFromTColumnDesc(tColumnDesc)); } } + + /** + * Converts a collection of ExternalLinks to ChunkLinkInfo objects. + * + * @param externalLinks The external links to convert, may be null + * @return A collection of ChunkLinkInfo objects, or null if input is null + */ + private static Collection convertToChunkLinkInfos( + Collection externalLinks) { + if (externalLinks == null) { + return null; + } + return externalLinks.stream() + .map( + link -> + new ChunkLinkFetchResult.ChunkLinkInfo( + link.getChunkIndex() != null ? link.getChunkIndex() : 0, + link, + link.getRowCount() != null ? link.getRowCount() : 0, + link.getRowOffset() != null ? link.getRowOffset() : 0)) + .collect(Collectors.toList()); + } } diff --git a/src/main/java/com/databricks/jdbc/api/impl/arrow/ChunkLinkFetchResult.java b/src/main/java/com/databricks/jdbc/api/impl/arrow/ChunkLinkFetchResult.java new file mode 100644 index 000000000..f2d21cafc --- /dev/null +++ b/src/main/java/com/databricks/jdbc/api/impl/arrow/ChunkLinkFetchResult.java @@ -0,0 +1,111 @@ +package com.databricks.jdbc.api.impl.arrow; + +import com.databricks.jdbc.model.core.ExternalLink; +import java.util.Collections; +import java.util.List; + +/** + * Result of a chunk link fetch operation from {@link ChunkLinkFetcher}. + * + *

Contains the fetched chunk links and information about whether more chunks are available. + */ +public class ChunkLinkFetchResult { + + private final List chunkLinks; + private final boolean hasMore; + private final long nextFetchIndex; + + private ChunkLinkFetchResult(List chunkLinks, boolean hasMore, long nextFetchIndex) { + this.chunkLinks = chunkLinks; + this.hasMore = hasMore; + this.nextFetchIndex = nextFetchIndex; + } + + /** + * Creates a result with the given links and continuation info. + * + * @param links The fetched chunk links + * @param hasMore Whether more chunks are available + * @param nextFetchIndex The next chunk index to fetch from, or -1 if no more + * @return A new ChunkLinkFetchResult + */ + public static ChunkLinkFetchResult of(List links, boolean hasMore, long nextFetchIndex) { + return new ChunkLinkFetchResult(links, hasMore, nextFetchIndex); + } + + /** + * Creates a result indicating the end of the stream (no more chunks). + * + * @return A ChunkLinkFetchResult representing end of stream + */ + public static ChunkLinkFetchResult endOfStream() { + return new ChunkLinkFetchResult(Collections.emptyList(), false, -1); + } + + /** + * Returns the list of chunk links fetched in this batch. + * + * @return List of ChunkLinkInfo, may be empty + */ + public List getChunkLinks() { + return chunkLinks; + } + + /** + * Returns whether more chunks are available after this batch. + * + * @return true if more chunks can be fetched, false otherwise + */ + public boolean hasMore() { + return hasMore; + } + + /** + * Returns the next chunk index to fetch from. + * + * @return The next fetch index, or -1 if no more chunks + */ + public long getNextFetchIndex() { + return nextFetchIndex; + } + + /** + * Checks if this result represents the end of the chunk stream. + * + * @return true if no more chunks are available + */ + public boolean isEndOfStream() { + return !hasMore && chunkLinks.isEmpty(); + } + + /** Information about a single chunk link. */ + public static class ChunkLinkInfo { + private final long chunkIndex; + private final ExternalLink link; + private final long rowCount; + private final long rowOffset; + + public ChunkLinkInfo(long chunkIndex, ExternalLink link, long rowCount, long rowOffset) { + this.chunkIndex = chunkIndex; + this.link = link; + this.rowCount = rowCount; + this.rowOffset = rowOffset; + } + + public long getChunkIndex() { + return chunkIndex; + } + + public ExternalLink getLink() { + return link; + } + + public long getRowCount() { + return rowCount; + } + + public long getRowOffset() { + return rowOffset; + } + } +} diff --git a/src/main/java/com/databricks/jdbc/api/impl/arrow/ChunkLinkFetcher.java b/src/main/java/com/databricks/jdbc/api/impl/arrow/ChunkLinkFetcher.java new file mode 100644 index 000000000..6c2e68e90 --- /dev/null +++ b/src/main/java/com/databricks/jdbc/api/impl/arrow/ChunkLinkFetcher.java @@ -0,0 +1,41 @@ +package com.databricks.jdbc.api.impl.arrow; + +import com.databricks.jdbc.exception.DatabricksSQLException; +import com.databricks.jdbc.model.core.ExternalLink; + +/** + * Abstraction for fetching chunk links from either SEA or Thrift backend. Implementations handle + * the protocol-specific details of how links are retrieved. + * + *

This interface enables a unified streaming approach for chunk downloads regardless of the + * underlying client type (SEA or Thrift). + */ +public interface ChunkLinkFetcher { + + /** + * Fetches the next batch of chunk links starting from the given index. + * + *

The implementation may return one or more links in a single call. The returned {@link + * ChunkLinkFetchResult} indicates whether more chunks are available. + * + * @param startChunkIndex The chunk index to start fetching from + * @return ChunkLinkFetchResult containing the fetched links and continuation information + * @throws DatabricksSQLException if the fetch operation fails + */ + ChunkLinkFetchResult fetchLinks(long startChunkIndex) throws DatabricksSQLException; + + /** + * Refetches a specific chunk link that may have expired. + * + *

This is used when a previously fetched link has expired before the chunk could be + * downloaded. Both SEA and Thrift clients support this via the getResultChunks API. + * + * @param chunkIndex The specific chunk index to refetch + * @return The refreshed ExternalLink with a new expiration time + * @throws DatabricksSQLException if the refetch operation fails + */ + ExternalLink refetchLink(long chunkIndex) throws DatabricksSQLException; + + /** Closes any resources held by the fetcher. */ + void close(); +} diff --git a/src/main/java/com/databricks/jdbc/api/impl/arrow/SeaChunkLinkFetcher.java b/src/main/java/com/databricks/jdbc/api/impl/arrow/SeaChunkLinkFetcher.java new file mode 100644 index 000000000..e8c9cb811 --- /dev/null +++ b/src/main/java/com/databricks/jdbc/api/impl/arrow/SeaChunkLinkFetcher.java @@ -0,0 +1,105 @@ +package com.databricks.jdbc.api.impl.arrow; + +import com.databricks.jdbc.api.internal.IDatabricksSession; +import com.databricks.jdbc.dbclient.impl.common.StatementId; +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 com.databricks.jdbc.model.telemetry.enums.DatabricksDriverErrorCode; + +import java.util.ArrayList; +import java.util.Collection; +import java.util.List; + +/** + * ChunkLinkFetcher implementation for the SQL Execution API (SEA) client. + * + *

SEA provides chunk links via the getResultChunks API, which returns links with nextChunkIndex + * to indicate continuation. When nextChunkIndex is null, it indicates no more chunks. + */ +public class SeaChunkLinkFetcher implements ChunkLinkFetcher { + + private static final JdbcLogger LOGGER = JdbcLoggerFactory.getLogger(SeaChunkLinkFetcher.class); + + private final IDatabricksSession session; + private final StatementId statementId; + + public SeaChunkLinkFetcher(IDatabricksSession session, StatementId statementId) { + this.session = session; + this.statementId = statementId; + LOGGER.debug("Created SeaChunkLinkFetcher for statement {}", statementId); + } + + @Override + public ChunkLinkFetchResult fetchLinks(long startChunkIndex) throws DatabricksSQLException { + LOGGER.debug("Fetching links starting from chunk index {} for statement {}", + startChunkIndex, statementId); + + Collection links = session.getDatabricksClient() + .getResultChunks(statementId, startChunkIndex); + + if (links == null || links.isEmpty()) { + LOGGER.debug("No links returned, end of stream reached for statement {}", statementId); + return ChunkLinkFetchResult.endOfStream(); + } + + List chunkLinks = new ArrayList<>(); + Long nextIndex = null; + + for (ExternalLink link : links) { + chunkLinks.add(new ChunkLinkFetchResult.ChunkLinkInfo( + link.getChunkIndex(), + link, + link.getRowCount() != null ? link.getRowCount() : 0, + link.getRowOffset() != null ? link.getRowOffset() : 0 + )); + + // SEA uses nextChunkIndex to indicate continuation. + // The LAST link's nextChunkIndex determines if there are more chunks. + // null means no more chunks after this batch. + nextIndex = link.getNextChunkIndex(); + } + + boolean hasMore = (nextIndex != null); + + LOGGER.debug("Fetched {} links for statement {}, hasMore={}, nextIndex={}", + chunkLinks.size(), statementId, hasMore, nextIndex); + + return ChunkLinkFetchResult.of(chunkLinks, hasMore, hasMore ? nextIndex : -1); + } + + @Override + public ExternalLink refetchLink(long chunkIndex) throws DatabricksSQLException { + LOGGER.info("Refetching expired link for chunk {} of statement {}", chunkIndex, statementId); + + Collection links = session.getDatabricksClient() + .getResultChunks(statementId, chunkIndex); + + if (links == null || links.isEmpty()) { + throw new DatabricksSQLException( + String.format("Failed to refetch link for chunk %d: no links returned", chunkIndex), DatabricksDriverErrorCode.CHUNK_READY_ERROR); + } + + // Find the link for the requested chunk index + for (ExternalLink link : links) { + if (link.getChunkIndex() != null && link.getChunkIndex() == chunkIndex) { + LOGGER.debug("Successfully refetched link for chunk {} of statement {}", + chunkIndex, statementId); + return link; + } + } + + // If exact match not found, return the first link (server should return the requested chunk) + ExternalLink firstLink = links.iterator().next(); + LOGGER.warn("Exact chunk index {} not found in response, using first link with index {} for statement {}", + chunkIndex, firstLink.getChunkIndex(), statementId); + return firstLink; + } + + @Override + public void close() { + LOGGER.debug("Closing SeaChunkLinkFetcher for statement {}", statementId); + // No resources to clean up for SEA fetcher + } +} diff --git a/src/main/java/com/databricks/jdbc/api/impl/arrow/StreamingChunkDownloadTask.java b/src/main/java/com/databricks/jdbc/api/impl/arrow/StreamingChunkDownloadTask.java new file mode 100644 index 000000000..49d136c39 --- /dev/null +++ b/src/main/java/com/databricks/jdbc/api/impl/arrow/StreamingChunkDownloadTask.java @@ -0,0 +1,112 @@ +package com.databricks.jdbc.api.impl.arrow; + +import com.databricks.jdbc.common.CompressionCodec; +import com.databricks.jdbc.dbclient.IDatabricksHttpClient; +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 com.databricks.jdbc.model.telemetry.enums.DatabricksDriverErrorCode; +import java.io.IOException; +import java.util.concurrent.Callable; + +/** + * A download task for streaming chunk provider. Simpler than ChunkDownloadTask - uses + * ChunkLinkFetcher directly for link refresh instead of ChunkLinkDownloadService. + */ +public class StreamingChunkDownloadTask implements Callable { + + private static final JdbcLogger LOGGER = + JdbcLoggerFactory.getLogger(StreamingChunkDownloadTask.class); + + private static final int MAX_RETRIES = 5; + private static final long RETRY_DELAY_MS = 1500; + + private final ArrowResultChunk chunk; + private final IDatabricksHttpClient httpClient; + private final CompressionCodec compressionCodec; + private final ChunkLinkFetcher linkFetcher; + private final double cloudFetchSpeedThreshold; + + public StreamingChunkDownloadTask( + ArrowResultChunk chunk, + IDatabricksHttpClient httpClient, + CompressionCodec compressionCodec, + ChunkLinkFetcher linkFetcher, + double cloudFetchSpeedThreshold) { + this.chunk = chunk; + this.httpClient = httpClient; + this.compressionCodec = compressionCodec; + this.linkFetcher = linkFetcher; + this.cloudFetchSpeedThreshold = cloudFetchSpeedThreshold; + } + + @Override + public Void call() throws DatabricksSQLException { + int retries = 0; + boolean downloadSuccessful = false; + + try { + while (!downloadSuccessful) { + try { + // Check if link is expired and refresh if needed + if (chunk.isChunkLinkInvalid()) { + LOGGER.debug("Link invalid for chunk {}, refetching", chunk.getChunkIndex()); + ExternalLink freshLink = linkFetcher.refetchLink(chunk.getChunkIndex()); + chunk.setChunkLink(freshLink); + } + + // Perform the download + chunk.downloadData(httpClient, compressionCodec, cloudFetchSpeedThreshold); + downloadSuccessful = true; + + LOGGER.debug("Successfully downloaded chunk {}", chunk.getChunkIndex()); + + } catch (IOException | DatabricksSQLException e) { + retries++; + if (retries >= MAX_RETRIES) { + LOGGER.error( + "Failed to download chunk {} after {} attempts: {}", + chunk.getChunkIndex(), + MAX_RETRIES, + e.getMessage()); + chunk.setStatus(ChunkStatus.DOWNLOAD_FAILED); + throw new DatabricksSQLException( + String.format( + "Failed to download chunk %d after %d attempts", + chunk.getChunkIndex(), MAX_RETRIES), + e, + DatabricksDriverErrorCode.CHUNK_DOWNLOAD_ERROR); + } else { + LOGGER.warn( + "Retry {} for chunk {}: {}", retries, chunk.getChunkIndex(), e.getMessage()); + chunk.setStatus(ChunkStatus.DOWNLOAD_RETRY); + try { + Thread.sleep(RETRY_DELAY_MS); + } catch (InterruptedException ie) { + Thread.currentThread().interrupt(); + throw new DatabricksSQLException( + "Chunk download interrupted", + ie, + DatabricksDriverErrorCode.THREAD_INTERRUPTED_ERROR); + } + } + } + } + } finally { + if (downloadSuccessful) { + chunk.getChunkReadyFuture().complete(null); + } else { + chunk.setStatus(ChunkStatus.DOWNLOAD_FAILED); + chunk + .getChunkReadyFuture() + .completeExceptionally( + new DatabricksSQLException( + "Download failed for chunk " + chunk.getChunkIndex(), + DatabricksDriverErrorCode.CHUNK_DOWNLOAD_ERROR)); + } + } + + return null; + } +} diff --git a/src/main/java/com/databricks/jdbc/api/impl/arrow/StreamingChunkProvider.java b/src/main/java/com/databricks/jdbc/api/impl/arrow/StreamingChunkProvider.java new file mode 100644 index 000000000..c0a0ca43d --- /dev/null +++ b/src/main/java/com/databricks/jdbc/api/impl/arrow/StreamingChunkProvider.java @@ -0,0 +1,570 @@ +package com.databricks.jdbc.api.impl.arrow; + +import com.databricks.jdbc.common.CompressionCodec; +import com.databricks.jdbc.dbclient.IDatabricksHttpClient; +import com.databricks.jdbc.dbclient.impl.common.StatementId; +import com.databricks.jdbc.exception.DatabricksSQLException; +import com.databricks.jdbc.log.JdbcLogger; +import com.databricks.jdbc.log.JdbcLoggerFactory; +import com.databricks.jdbc.model.telemetry.enums.DatabricksDriverErrorCode; + +import javax.annotation.Nonnull; +import java.util.Collection; +import java.util.concurrent.ConcurrentHashMap; +import java.util.concurrent.ConcurrentMap; +import java.util.concurrent.ExecutionException; +import java.util.concurrent.ExecutorService; +import java.util.concurrent.Executors; +import java.util.concurrent.ThreadFactory; +import java.util.concurrent.TimeoutException; +import java.util.concurrent.atomic.AtomicInteger; +import java.util.concurrent.atomic.AtomicLong; +import java.util.concurrent.locks.Condition; +import java.util.concurrent.locks.ReentrantLock; + +/** + * A streaming chunk provider that fetches chunk links proactively and downloads chunks in parallel. + * + *

Key features: + *

    + *
  • No dependency on total chunk count - streams until end of data
  • + *
  • Proactive link prefetching with configurable window
  • + *
  • Memory-bounded parallel downloads
  • + *
  • Automatic link refresh on expiration
  • + *
+ * + *

This provider uses two key windows: + *

    + *
  • Link prefetch window: How many links to fetch ahead of consumption
  • + *
  • Download window: How many chunks to keep in memory (downloading or ready)
  • + *
+ */ +public class StreamingChunkProvider implements ChunkProvider { + + private static final JdbcLogger LOGGER = JdbcLoggerFactory.getLogger(StreamingChunkProvider.class); + private static final String DOWNLOAD_THREAD_PREFIX = "databricks-jdbc-streaming-downloader-"; + private static final String PREFETCH_THREAD_NAME = "databricks-jdbc-link-prefetcher"; + + // Configuration + private final int linkPrefetchWindow; + private final int maxChunksInMemory; + private final int chunkReadyTimeoutSeconds; + + // Dependencies + private final ChunkLinkFetcher linkFetcher; + private final IDatabricksHttpClient httpClient; + private final CompressionCodec compressionCodec; + private final StatementId statementId; + private final double cloudFetchSpeedThreshold; + + // Chunk storage + private final ConcurrentMap chunks = new ConcurrentHashMap<>(); + + // Position tracking + private volatile long currentChunkIndex = -1; + private volatile long highestKnownChunkIndex = -1; + private volatile long nextLinkFetchIndex = 0; + private volatile long nextDownloadIndex = 0; + + // State flags + private volatile boolean endOfStreamReached = false; + private volatile boolean closed = false; + + // Row tracking + private final AtomicLong totalRowCount = new AtomicLong(0); + + // Synchronization for prefetch thread + private final ReentrantLock prefetchLock = new ReentrantLock(); + private final Condition consumerAdvanced = prefetchLock.newCondition(); + + // Synchronization for download coordination + private final ReentrantLock downloadLock = new ReentrantLock(); + private final Condition downloadSlotAvailable = downloadLock.newCondition(); + + // Executors + private final ExecutorService downloadExecutor; + private final Thread linkPrefetchThread; + + // Track chunks currently in memory (for sliding window) + private final AtomicInteger chunksInMemory = new AtomicInteger(0); + + /** + * Creates a new StreamingChunkProvider. + * + * @param linkFetcher Fetcher for chunk links + * @param httpClient HTTP client for downloads + * @param compressionCodec Codec for decompressing chunk data + * @param statementId Statement ID for logging and chunk creation + * @param maxChunksInMemory Maximum chunks to keep in memory (download window) + * @param linkPrefetchWindow How many links to fetch ahead + * @param chunkReadyTimeoutSeconds Timeout waiting for chunk to be ready + * @param cloudFetchSpeedThreshold Speed threshold for logging warnings + * @param initialLinks Initial links provided with result data (avoids extra fetch), may be null + */ + public StreamingChunkProvider( + ChunkLinkFetcher linkFetcher, + IDatabricksHttpClient httpClient, + CompressionCodec compressionCodec, + StatementId statementId, + int maxChunksInMemory, + int linkPrefetchWindow, + int chunkReadyTimeoutSeconds, + double cloudFetchSpeedThreshold, + Collection initialLinks) { + + this.linkFetcher = linkFetcher; + this.httpClient = httpClient; + this.compressionCodec = compressionCodec; + this.statementId = statementId; + this.maxChunksInMemory = maxChunksInMemory; + this.linkPrefetchWindow = linkPrefetchWindow; + this.chunkReadyTimeoutSeconds = chunkReadyTimeoutSeconds; + this.cloudFetchSpeedThreshold = cloudFetchSpeedThreshold; + + LOGGER.info( + "Creating StreamingChunkProvider for statement {}: maxChunksInMemory={}, linkPrefetchWindow={}", + statementId, + maxChunksInMemory, + linkPrefetchWindow); + + // Process initial links if provided + processInitialLinks(initialLinks); + + // Create download executor + this.downloadExecutor = createDownloadExecutor(maxChunksInMemory); + + // Start link prefetch thread + this.linkPrefetchThread = new Thread(this::linkPrefetchLoop, PREFETCH_THREAD_NAME); + this.linkPrefetchThread.setDaemon(true); + this.linkPrefetchThread.start(); + + // Trigger initial downloads and prefetch + triggerDownloads(); + notifyConsumerAdvanced(); + } + + /** + * Convenience constructor with default prefetch window. + */ + public StreamingChunkProvider( + ChunkLinkFetcher linkFetcher, + IDatabricksHttpClient httpClient, + CompressionCodec compressionCodec, + StatementId statementId, + int maxChunksInMemory, + int chunkReadyTimeoutSeconds, + double cloudFetchSpeedThreshold, + Collection initialLinks) { + this( + linkFetcher, + httpClient, + compressionCodec, + statementId, + maxChunksInMemory, + 64, // Default prefetch window + chunkReadyTimeoutSeconds, + cloudFetchSpeedThreshold, + initialLinks); + } + + // ==================== ChunkProvider Interface ==================== + + @Override + public boolean hasNextChunk() { + if (closed) { + return false; + } + + // If we haven't reached end of stream, there might be more + if (!endOfStreamReached) { + return true; + } + + // We've reached end of stream - check if there are unconsumed chunks + return currentChunkIndex < highestKnownChunkIndex; + } + + @Override + public boolean next() throws DatabricksSQLException { + if (closed) { + return false; + } + + // Release previous chunk if any + if (currentChunkIndex >= 0) { + releaseChunk(currentChunkIndex); + } + + if (!hasNextChunk()) { + return false; + } + + currentChunkIndex++; + + // Notify prefetch thread that consumer advanced + notifyConsumerAdvanced(); + + // Trigger downloads for newly available slots + triggerDownloads(); + + return true; + } + + @Override + public AbstractArrowResultChunk getChunk() throws DatabricksSQLException { + if (currentChunkIndex < 0) { + return null; + } + + ArrowResultChunk chunk = chunks.get(currentChunkIndex); + + if (chunk == null) { + // Chunk not yet created - wait for it + LOGGER.debug("Chunk {} not yet available, waiting for prefetch", currentChunkIndex); + waitForChunkCreation(currentChunkIndex); + chunk = chunks.get(currentChunkIndex); + } + + if (chunk == null) { + throw new DatabricksSQLException( + "Chunk " + currentChunkIndex + " not found after waiting", DatabricksDriverErrorCode.CHUNK_READY_ERROR); + } + + // Wait for chunk to be ready (downloaded and processed) + try { + chunk.waitForChunkReady(); + } catch (InterruptedException e) { + Thread.currentThread().interrupt(); + throw new DatabricksSQLException( + "Interrupted waiting for chunk " + currentChunkIndex, e, DatabricksDriverErrorCode.THREAD_INTERRUPTED_ERROR); + } catch (ExecutionException e) { + throw new DatabricksSQLException( + "Failed to prepare chunk " + currentChunkIndex, e.getCause(), DatabricksDriverErrorCode.CHUNK_READY_ERROR); + } catch (TimeoutException e) { + throw new DatabricksSQLException( + "Timeout waiting for chunk " + currentChunkIndex + " (timeout: " + chunkReadyTimeoutSeconds + "s)", DatabricksDriverErrorCode.CHUNK_READY_ERROR); + } + + return chunk; + } + + @Override + public void close() { + if (closed) { + return; + } + + LOGGER.info("Closing StreamingChunkProvider for statement {}", statementId); + closed = true; + + // Wake up any waiting threads + notifyConsumerAdvanced(); + notifyDownloadSlotAvailable(); + + // Interrupt prefetch thread + if (linkPrefetchThread != null) { + linkPrefetchThread.interrupt(); + } + + // Shutdown download executor + if (downloadExecutor != null) { + downloadExecutor.shutdownNow(); + } + + // Release all chunks + for (ArrowResultChunk chunk : chunks.values()) { + try { + chunk.releaseChunk(); + } catch (Exception e) { + LOGGER.warn("Error releasing chunk: {}", e.getMessage()); + } + } + chunks.clear(); + + // Close link fetcher + if (linkFetcher != null) { + linkFetcher.close(); + } + } + + @Override + public long getRowCount() { + return totalRowCount.get(); + } + + @Override + public long getChunkCount() { + // In streaming mode, we don't know total chunks until end of stream + if (endOfStreamReached) { + return highestKnownChunkIndex + 1; + } + return -1; // Unknown + } + + @Override + public boolean isClosed() { + return closed; + } + + // ==================== Link Prefetch Logic ==================== + + private void linkPrefetchLoop() { + LOGGER.debug("Link prefetch thread started for statement {}", statementId); + + while (!closed && !Thread.currentThread().isInterrupted()) { + try { + prefetchLock.lock(); + try { + // Calculate target prefetch index + long targetIndex = currentChunkIndex + linkPrefetchWindow; + + // Wait if we're caught up + while (!closed && !endOfStreamReached && nextLinkFetchIndex > targetIndex) { + LOGGER.debug("Prefetch caught up, waiting for consumer. next={}, target={}", + nextLinkFetchIndex, targetIndex); + consumerAdvanced.await(); + targetIndex = currentChunkIndex + linkPrefetchWindow; + } + } finally { + prefetchLock.unlock(); + } + + if (closed || endOfStreamReached) { + break; + } + + // Fetch next batch of links + fetchNextLinkBatch(); + + } catch (InterruptedException e) { + Thread.currentThread().interrupt(); + LOGGER.debug("Link prefetch thread interrupted"); + break; + } catch (DatabricksSQLException e) { + LOGGER.error("Error fetching links: {}", e.getMessage()); + // Continue trying - next iteration may succeed + } + } + + LOGGER.debug("Link prefetch thread exiting for statement {}", statementId); + } + + private void fetchNextLinkBatch() throws DatabricksSQLException { + if (endOfStreamReached || closed) { + return; + } + + LOGGER.debug("Fetching links starting from index {} for statement {}", + nextLinkFetchIndex, statementId); + + ChunkLinkFetchResult result = linkFetcher.fetchLinks(nextLinkFetchIndex); + + if (result.isEndOfStream()) { + LOGGER.info("End of stream reached for statement {}", statementId); + endOfStreamReached = true; + return; + } + + // Process received links - create chunks + for (ChunkLinkFetchResult.ChunkLinkInfo linkInfo : result.getChunkLinks()) { + createChunkFromLink(linkInfo); + } + + // Update next fetch position + if (result.hasMore()) { + nextLinkFetchIndex = result.getNextFetchIndex(); + } else { + endOfStreamReached = true; + LOGGER.info("End of stream reached for statement {} (hasMore=false)", statementId); + } + + // Trigger downloads for new chunks + triggerDownloads(); + } + + /** + * Processes initial links provided with the result data. + * This avoids an extra fetch call for links the server already provided. + * + * @param initialLinks The initial links from ResultData, may be null or empty + */ + private void processInitialLinks(Collection initialLinks) { + if (initialLinks == null || initialLinks.isEmpty()) { + LOGGER.debug("No initial links provided for statement {}", statementId); + return; + } + + LOGGER.info("Processing {} initial links for statement {}", initialLinks.size(), statementId); + + Long lastNextChunkIndex = null; + + for (ChunkLinkFetchResult.ChunkLinkInfo linkInfo : initialLinks) { + createChunkFromLink(linkInfo); + if (linkInfo.getLink() != null) { + lastNextChunkIndex = linkInfo.getLink().getNextChunkIndex(); + } + } + + // Set next fetch index based on the last link's nextChunkIndex + if (lastNextChunkIndex != null) { + nextLinkFetchIndex = lastNextChunkIndex; + LOGGER.debug("Next link fetch index set to {} from initial links", nextLinkFetchIndex); + } else { + // Last link has null nextChunkIndex - no more chunks + endOfStreamReached = true; + LOGGER.info("End of stream reached from initial links for statement {}", statementId); + } + } + + /** + * Creates a chunk from link info and registers it for download. + * + * @param linkInfo The chunk link info containing index, row count, offset, and link + */ + private void createChunkFromLink(ChunkLinkFetchResult.ChunkLinkInfo linkInfo) { + long chunkIndex = linkInfo.getChunkIndex(); + if (chunks.containsKey(chunkIndex)) { + LOGGER.debug("Chunk {} already exists, skipping creation", chunkIndex); + return; + } + + try { + ArrowResultChunk chunk = ArrowResultChunk.builder() + .withStatementId(statementId) + .withChunkMetadata(chunkIndex, linkInfo.getRowCount(), linkInfo.getRowOffset()) + .withChunkReadyTimeoutSeconds(chunkReadyTimeoutSeconds) + .build(); + + chunk.setChunkLink(linkInfo.getLink()); + chunks.put(chunkIndex, chunk); + highestKnownChunkIndex = Math.max(highestKnownChunkIndex, chunkIndex); + totalRowCount.addAndGet(linkInfo.getRowCount()); + + LOGGER.debug("Created chunk {} with {} rows for statement {}", + chunkIndex, linkInfo.getRowCount(), statementId); + + } catch (Exception e) { + LOGGER.error("Failed to create chunk {}: {}", chunkIndex, e.getMessage()); + } + } + + // ==================== Download Coordination ==================== + + private void triggerDownloads() { + downloadLock.lock(); + try { + while (!closed && chunksInMemory.get() < maxChunksInMemory && nextDownloadIndex <= highestKnownChunkIndex) { + ArrowResultChunk chunk = chunks.get(nextDownloadIndex); + + if (chunk == null) { + // Chunk not yet created, wait for prefetch + break; + } + + // Only submit if not already downloading/downloaded + ChunkStatus status = chunk.getStatus(); + if (status == ChunkStatus.PENDING || status == ChunkStatus.URL_FETCHED) { + submitDownloadTask(chunk); + chunksInMemory.incrementAndGet(); + } + + nextDownloadIndex++; + } + } finally { + downloadLock.unlock(); + } + } + + private void submitDownloadTask(ArrowResultChunk chunk) { + LOGGER.debug("Submitting download task for chunk {}", chunk.getChunkIndex()); + + StreamingChunkDownloadTask task = new StreamingChunkDownloadTask( + chunk, + httpClient, + compressionCodec, + linkFetcher, + cloudFetchSpeedThreshold); + + downloadExecutor.submit(task); + } + + // ==================== Resource Management ==================== + + private void releaseChunk(long chunkIndex) { + ArrowResultChunk chunk = chunks.get(chunkIndex); + if (chunk != null && chunk.releaseChunk()) { + chunks.remove(chunkIndex); + chunksInMemory.decrementAndGet(); + + LOGGER.debug("Released chunk {}, chunksInMemory={}", chunkIndex, chunksInMemory.get()); + + // Notify download coordinator + notifyDownloadSlotAvailable(); + + // Trigger more downloads + triggerDownloads(); + } + } + + private void waitForChunkCreation(long chunkIndex) throws DatabricksSQLException { + long startTime = System.currentTimeMillis(); + long timeoutMs = chunkReadyTimeoutSeconds * 1000L; + + while (!closed && !chunks.containsKey(chunkIndex)) { + if (endOfStreamReached && chunkIndex > highestKnownChunkIndex) { + throw new DatabricksSQLException( + "Chunk " + chunkIndex + " does not exist (highest known: " + highestKnownChunkIndex + ")", DatabricksDriverErrorCode.CHUNK_READY_ERROR); + } + + if (timeoutMs > 0 && System.currentTimeMillis() - startTime > timeoutMs) { + throw new DatabricksSQLException( + "Timeout waiting for chunk " + chunkIndex + " to be created", DatabricksDriverErrorCode.CHUNK_READY_ERROR); + } + + try { + Thread.sleep(10); // Brief wait before checking again + } catch (InterruptedException e) { + Thread.currentThread().interrupt(); + throw new DatabricksSQLException("Interrupted waiting for chunk creation", e, DatabricksDriverErrorCode.THREAD_INTERRUPTED_ERROR); + } + } + } + + // ==================== Synchronization Helpers ==================== + + private void notifyConsumerAdvanced() { + prefetchLock.lock(); + try { + consumerAdvanced.signalAll(); + } finally { + prefetchLock.unlock(); + } + } + + private void notifyDownloadSlotAvailable() { + downloadLock.lock(); + try { + downloadSlotAvailable.signalAll(); + } finally { + downloadLock.unlock(); + } + } + + // ==================== Executor Creation ==================== + + private ExecutorService createDownloadExecutor(int poolSize) { + ThreadFactory threadFactory = new ThreadFactory() { + private final AtomicInteger threadCount = new AtomicInteger(1); + + @Override + public Thread newThread(@Nonnull Runnable r) { + Thread thread = new Thread(r); + thread.setName(DOWNLOAD_THREAD_PREFIX + threadCount.getAndIncrement()); + thread.setDaemon(true); + return thread; + } + }; + + return Executors.newFixedThreadPool(poolSize, threadFactory); + } +} diff --git a/src/main/java/com/databricks/jdbc/api/internal/IDatabricksConnectionContext.java b/src/main/java/com/databricks/jdbc/api/internal/IDatabricksConnectionContext.java index c66ee39e0..204425289 100644 --- a/src/main/java/com/databricks/jdbc/api/internal/IDatabricksConnectionContext.java +++ b/src/main/java/com/databricks/jdbc/api/internal/IDatabricksConnectionContext.java @@ -416,4 +416,7 @@ public interface IDatabricksConnectionContext { /** Returns whether token federation is enabled for authentication. */ boolean isTokenFederationEnabled(); + + /** Returns whether streaming chunk provider is enabled for result fetching. */ + boolean isStreamingChunkProviderEnabled(); } diff --git a/src/main/java/com/databricks/jdbc/common/DatabricksJdbcUrlParams.java b/src/main/java/com/databricks/jdbc/common/DatabricksJdbcUrlParams.java index 41c0e6532..c1d22e218 100644 --- a/src/main/java/com/databricks/jdbc/common/DatabricksJdbcUrlParams.java +++ b/src/main/java/com/databricks/jdbc/common/DatabricksJdbcUrlParams.java @@ -187,7 +187,11 @@ public enum DatabricksJdbcUrlParams { "Disable requesting OAuth refresh tokens (omit offline_access unless explicitly provided)", "1"), ENABLE_TOKEN_FEDERATION( - "EnableTokenFederation", "Enable token federation for authentication", "1"); + "EnableTokenFederation", "Enable token federation for authentication", "1"), + ENABLE_STREAMING_CHUNK_PROVIDER( + "EnableStreamingChunkProvider", + "Enable streaming chunk provider for result fetching (experimental)", + "0"); private final String paramName; private final String defaultValue; From 51b0dee8a75603eab45cf4f3c36f674b2726f8d5 Mon Sep 17 00:00:00 2001 From: Jayant Singh Date: Thu, 27 Nov 2025 11:25:30 +0000 Subject: [PATCH 2/6] fmt --- .../api/impl/arrow/ChunkLinkFetchResult.java | 6 +- .../api/impl/arrow/SeaChunkLinkFetcher.java | 49 ++-- .../arrow/StreamingChunkDownloadTask.java | 2 +- .../impl/arrow/StreamingChunkProvider.java | 210 ++++++++++-------- 4 files changed, 152 insertions(+), 115 deletions(-) diff --git a/src/main/java/com/databricks/jdbc/api/impl/arrow/ChunkLinkFetchResult.java b/src/main/java/com/databricks/jdbc/api/impl/arrow/ChunkLinkFetchResult.java index f2d21cafc..044a81187 100644 --- a/src/main/java/com/databricks/jdbc/api/impl/arrow/ChunkLinkFetchResult.java +++ b/src/main/java/com/databricks/jdbc/api/impl/arrow/ChunkLinkFetchResult.java @@ -15,7 +15,8 @@ public class ChunkLinkFetchResult { private final boolean hasMore; private final long nextFetchIndex; - private ChunkLinkFetchResult(List chunkLinks, boolean hasMore, long nextFetchIndex) { + private ChunkLinkFetchResult( + List chunkLinks, boolean hasMore, long nextFetchIndex) { this.chunkLinks = chunkLinks; this.hasMore = hasMore; this.nextFetchIndex = nextFetchIndex; @@ -29,7 +30,8 @@ private ChunkLinkFetchResult(List chunkLinks, boolean hasMore, lo * @param nextFetchIndex The next chunk index to fetch from, or -1 if no more * @return A new ChunkLinkFetchResult */ - public static ChunkLinkFetchResult of(List links, boolean hasMore, long nextFetchIndex) { + public static ChunkLinkFetchResult of( + List links, boolean hasMore, long nextFetchIndex) { return new ChunkLinkFetchResult(links, hasMore, nextFetchIndex); } diff --git a/src/main/java/com/databricks/jdbc/api/impl/arrow/SeaChunkLinkFetcher.java b/src/main/java/com/databricks/jdbc/api/impl/arrow/SeaChunkLinkFetcher.java index e8c9cb811..895698a51 100644 --- a/src/main/java/com/databricks/jdbc/api/impl/arrow/SeaChunkLinkFetcher.java +++ b/src/main/java/com/databricks/jdbc/api/impl/arrow/SeaChunkLinkFetcher.java @@ -7,7 +7,6 @@ import com.databricks.jdbc.log.JdbcLoggerFactory; import com.databricks.jdbc.model.core.ExternalLink; import com.databricks.jdbc.model.telemetry.enums.DatabricksDriverErrorCode; - import java.util.ArrayList; import java.util.Collection; import java.util.List; @@ -33,11 +32,13 @@ public SeaChunkLinkFetcher(IDatabricksSession session, StatementId statementId) @Override public ChunkLinkFetchResult fetchLinks(long startChunkIndex) throws DatabricksSQLException { - LOGGER.debug("Fetching links starting from chunk index {} for statement {}", - startChunkIndex, statementId); + LOGGER.debug( + "Fetching links starting from chunk index {} for statement {}", + startChunkIndex, + statementId); - Collection links = session.getDatabricksClient() - .getResultChunks(statementId, startChunkIndex); + Collection links = + session.getDatabricksClient().getResultChunks(statementId, startChunkIndex); if (links == null || links.isEmpty()) { LOGGER.debug("No links returned, end of stream reached for statement {}", statementId); @@ -48,12 +49,12 @@ public ChunkLinkFetchResult fetchLinks(long startChunkIndex) throws DatabricksSQ Long nextIndex = null; for (ExternalLink link : links) { - chunkLinks.add(new ChunkLinkFetchResult.ChunkLinkInfo( - link.getChunkIndex(), - link, - link.getRowCount() != null ? link.getRowCount() : 0, - link.getRowOffset() != null ? link.getRowOffset() : 0 - )); + chunkLinks.add( + new ChunkLinkFetchResult.ChunkLinkInfo( + link.getChunkIndex(), + link, + link.getRowCount() != null ? link.getRowCount() : 0, + link.getRowOffset() != null ? link.getRowOffset() : 0)); // SEA uses nextChunkIndex to indicate continuation. // The LAST link's nextChunkIndex determines if there are more chunks. @@ -63,8 +64,12 @@ public ChunkLinkFetchResult fetchLinks(long startChunkIndex) throws DatabricksSQ boolean hasMore = (nextIndex != null); - LOGGER.debug("Fetched {} links for statement {}, hasMore={}, nextIndex={}", - chunkLinks.size(), statementId, hasMore, nextIndex); + LOGGER.debug( + "Fetched {} links for statement {}, hasMore={}, nextIndex={}", + chunkLinks.size(), + statementId, + hasMore, + nextIndex); return ChunkLinkFetchResult.of(chunkLinks, hasMore, hasMore ? nextIndex : -1); } @@ -73,27 +78,31 @@ public ChunkLinkFetchResult fetchLinks(long startChunkIndex) throws DatabricksSQ public ExternalLink refetchLink(long chunkIndex) throws DatabricksSQLException { LOGGER.info("Refetching expired link for chunk {} of statement {}", chunkIndex, statementId); - Collection links = session.getDatabricksClient() - .getResultChunks(statementId, chunkIndex); + Collection links = + session.getDatabricksClient().getResultChunks(statementId, chunkIndex); if (links == null || links.isEmpty()) { throw new DatabricksSQLException( - String.format("Failed to refetch link for chunk %d: no links returned", chunkIndex), DatabricksDriverErrorCode.CHUNK_READY_ERROR); + String.format("Failed to refetch link for chunk %d: no links returned", chunkIndex), + DatabricksDriverErrorCode.CHUNK_READY_ERROR); } // Find the link for the requested chunk index for (ExternalLink link : links) { if (link.getChunkIndex() != null && link.getChunkIndex() == chunkIndex) { - LOGGER.debug("Successfully refetched link for chunk {} of statement {}", - chunkIndex, statementId); + LOGGER.debug( + "Successfully refetched link for chunk {} of statement {}", chunkIndex, statementId); return link; } } // If exact match not found, return the first link (server should return the requested chunk) ExternalLink firstLink = links.iterator().next(); - LOGGER.warn("Exact chunk index {} not found in response, using first link with index {} for statement {}", - chunkIndex, firstLink.getChunkIndex(), statementId); + LOGGER.warn( + "Exact chunk index {} not found in response, using first link with index {} for statement {}", + chunkIndex, + firstLink.getChunkIndex(), + statementId); return firstLink; } diff --git a/src/main/java/com/databricks/jdbc/api/impl/arrow/StreamingChunkDownloadTask.java b/src/main/java/com/databricks/jdbc/api/impl/arrow/StreamingChunkDownloadTask.java index 49d136c39..d36a9477a 100644 --- a/src/main/java/com/databricks/jdbc/api/impl/arrow/StreamingChunkDownloadTask.java +++ b/src/main/java/com/databricks/jdbc/api/impl/arrow/StreamingChunkDownloadTask.java @@ -70,7 +70,7 @@ public Void call() throws DatabricksSQLException { chunk.getChunkIndex(), MAX_RETRIES, e.getMessage()); - chunk.setStatus(ChunkStatus.DOWNLOAD_FAILED); + // Status will be set to DOWNLOAD_FAILED in the finally block throw new DatabricksSQLException( String.format( "Failed to download chunk %d after %d attempts", diff --git a/src/main/java/com/databricks/jdbc/api/impl/arrow/StreamingChunkProvider.java b/src/main/java/com/databricks/jdbc/api/impl/arrow/StreamingChunkProvider.java index c0a0ca43d..71790fd79 100644 --- a/src/main/java/com/databricks/jdbc/api/impl/arrow/StreamingChunkProvider.java +++ b/src/main/java/com/databricks/jdbc/api/impl/arrow/StreamingChunkProvider.java @@ -3,12 +3,11 @@ import com.databricks.jdbc.common.CompressionCodec; 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.telemetry.enums.DatabricksDriverErrorCode; - -import javax.annotation.Nonnull; import java.util.Collection; import java.util.concurrent.ConcurrentHashMap; import java.util.concurrent.ConcurrentMap; @@ -21,27 +20,31 @@ import java.util.concurrent.atomic.AtomicLong; import java.util.concurrent.locks.Condition; import java.util.concurrent.locks.ReentrantLock; +import javax.annotation.Nonnull; /** * A streaming chunk provider that fetches chunk links proactively and downloads chunks in parallel. * *

Key features: + * *

    - *
  • No dependency on total chunk count - streams until end of data
  • - *
  • Proactive link prefetching with configurable window
  • - *
  • Memory-bounded parallel downloads
  • - *
  • Automatic link refresh on expiration
  • + *
  • No dependency on total chunk count - streams until end of data + *
  • Proactive link prefetching with configurable window + *
  • Memory-bounded parallel downloads + *
  • Automatic link refresh on expiration *
* *

This provider uses two key windows: + * *

    - *
  • Link prefetch window: How many links to fetch ahead of consumption
  • - *
  • Download window: How many chunks to keep in memory (downloading or ready)
  • + *
  • Link prefetch window: How many links to fetch ahead of consumption + *
  • Download window: How many chunks to keep in memory (downloading or ready) *
*/ public class StreamingChunkProvider implements ChunkProvider { - private static final JdbcLogger LOGGER = JdbcLoggerFactory.getLogger(StreamingChunkProvider.class); + private static final JdbcLogger LOGGER = + JdbcLoggerFactory.getLogger(StreamingChunkProvider.class); private static final String DOWNLOAD_THREAD_PREFIX = "databricks-jdbc-streaming-downloader-"; private static final String PREFETCH_THREAD_NAME = "databricks-jdbc-link-prefetcher"; @@ -76,10 +79,10 @@ public class StreamingChunkProvider implements ChunkProvider { // Synchronization for prefetch thread private final ReentrantLock prefetchLock = new ReentrantLock(); private final Condition consumerAdvanced = prefetchLock.newCondition(); + private final Condition chunkCreated = prefetchLock.newCondition(); // Synchronization for download coordination private final ReentrantLock downloadLock = new ReentrantLock(); - private final Condition downloadSlotAvailable = downloadLock.newCondition(); // Executors private final ExecutorService downloadExecutor; @@ -110,7 +113,8 @@ public StreamingChunkProvider( int linkPrefetchWindow, int chunkReadyTimeoutSeconds, double cloudFetchSpeedThreshold, - Collection initialLinks) { + Collection initialLinks) + throws DatabricksParsingException { this.linkFetcher = linkFetcher; this.httpClient = httpClient; @@ -143,9 +147,7 @@ public StreamingChunkProvider( notifyConsumerAdvanced(); } - /** - * Convenience constructor with default prefetch window. - */ + /** Convenience constructor with default prefetch window. */ public StreamingChunkProvider( ChunkLinkFetcher linkFetcher, IDatabricksHttpClient httpClient, @@ -154,7 +156,8 @@ public StreamingChunkProvider( int maxChunksInMemory, int chunkReadyTimeoutSeconds, double cloudFetchSpeedThreshold, - Collection initialLinks) { + Collection initialLinks) + throws DatabricksParsingException { this( linkFetcher, httpClient, @@ -204,9 +207,6 @@ public boolean next() throws DatabricksSQLException { // Notify prefetch thread that consumer advanced notifyConsumerAdvanced(); - // Trigger downloads for newly available slots - triggerDownloads(); - return true; } @@ -227,7 +227,8 @@ public AbstractArrowResultChunk getChunk() throws DatabricksSQLException { if (chunk == null) { throw new DatabricksSQLException( - "Chunk " + currentChunkIndex + " not found after waiting", DatabricksDriverErrorCode.CHUNK_READY_ERROR); + "Chunk " + currentChunkIndex + " not found after waiting", + DatabricksDriverErrorCode.CHUNK_READY_ERROR); } // Wait for chunk to be ready (downloaded and processed) @@ -236,13 +237,22 @@ public AbstractArrowResultChunk getChunk() throws DatabricksSQLException { } catch (InterruptedException e) { Thread.currentThread().interrupt(); throw new DatabricksSQLException( - "Interrupted waiting for chunk " + currentChunkIndex, e, DatabricksDriverErrorCode.THREAD_INTERRUPTED_ERROR); + "Interrupted waiting for chunk " + currentChunkIndex, + e, + DatabricksDriverErrorCode.THREAD_INTERRUPTED_ERROR); } catch (ExecutionException e) { throw new DatabricksSQLException( - "Failed to prepare chunk " + currentChunkIndex, e.getCause(), DatabricksDriverErrorCode.CHUNK_READY_ERROR); + "Failed to prepare chunk " + currentChunkIndex, + e.getCause(), + DatabricksDriverErrorCode.CHUNK_READY_ERROR); } catch (TimeoutException e) { throw new DatabricksSQLException( - "Timeout waiting for chunk " + currentChunkIndex + " (timeout: " + chunkReadyTimeoutSeconds + "s)", DatabricksDriverErrorCode.CHUNK_READY_ERROR); + "Timeout waiting for chunk " + + currentChunkIndex + + " (timeout: " + + chunkReadyTimeoutSeconds + + "s)", + DatabricksDriverErrorCode.CHUNK_READY_ERROR); } return chunk; @@ -257,9 +267,9 @@ public void close() { LOGGER.info("Closing StreamingChunkProvider for statement {}", statementId); closed = true; - // Wake up any waiting threads + // Wake up any waiting threads so they can exit notifyConsumerAdvanced(); - notifyDownloadSlotAvailable(); + notifyChunkCreated(); // Interrupt prefetch thread if (linkPrefetchThread != null) { @@ -320,8 +330,10 @@ private void linkPrefetchLoop() { // Wait if we're caught up while (!closed && !endOfStreamReached && nextLinkFetchIndex > targetIndex) { - LOGGER.debug("Prefetch caught up, waiting for consumer. next={}, target={}", - nextLinkFetchIndex, targetIndex); + LOGGER.debug( + "Prefetch caught up, waiting for consumer. next={}, target={}", + nextLinkFetchIndex, + targetIndex); consumerAdvanced.await(); targetIndex = currentChunkIndex + linkPrefetchWindow; } @@ -354,8 +366,8 @@ private void fetchNextLinkBatch() throws DatabricksSQLException { return; } - LOGGER.debug("Fetching links starting from index {} for statement {}", - nextLinkFetchIndex, statementId); + LOGGER.debug( + "Fetching links starting from index {} for statement {}", nextLinkFetchIndex, statementId); ChunkLinkFetchResult result = linkFetcher.fetchLinks(nextLinkFetchIndex); @@ -383,12 +395,13 @@ private void fetchNextLinkBatch() throws DatabricksSQLException { } /** - * Processes initial links provided with the result data. - * This avoids an extra fetch call for links the server already provided. + * Processes initial links provided with the result data. This avoids an extra fetch call for + * links the server already provided. * * @param initialLinks The initial links from ResultData, may be null or empty */ - private void processInitialLinks(Collection initialLinks) { + private void processInitialLinks(Collection initialLinks) + throws DatabricksParsingException { if (initialLinks == null || initialLinks.isEmpty()) { LOGGER.debug("No initial links provided for statement {}", statementId); return; @@ -421,31 +434,34 @@ private void processInitialLinks(Collection * * @param linkInfo The chunk link info containing index, row count, offset, and link */ - private void createChunkFromLink(ChunkLinkFetchResult.ChunkLinkInfo linkInfo) { + private void createChunkFromLink(ChunkLinkFetchResult.ChunkLinkInfo linkInfo) + throws DatabricksParsingException { long chunkIndex = linkInfo.getChunkIndex(); if (chunks.containsKey(chunkIndex)) { LOGGER.debug("Chunk {} already exists, skipping creation", chunkIndex); return; } - try { - ArrowResultChunk chunk = ArrowResultChunk.builder() - .withStatementId(statementId) - .withChunkMetadata(chunkIndex, linkInfo.getRowCount(), linkInfo.getRowOffset()) - .withChunkReadyTimeoutSeconds(chunkReadyTimeoutSeconds) - .build(); - - chunk.setChunkLink(linkInfo.getLink()); - chunks.put(chunkIndex, chunk); - highestKnownChunkIndex = Math.max(highestKnownChunkIndex, chunkIndex); - totalRowCount.addAndGet(linkInfo.getRowCount()); - - LOGGER.debug("Created chunk {} with {} rows for statement {}", - chunkIndex, linkInfo.getRowCount(), statementId); - - } catch (Exception e) { - LOGGER.error("Failed to create chunk {}: {}", chunkIndex, e.getMessage()); - } + ArrowResultChunk chunk = + ArrowResultChunk.builder() + .withStatementId(statementId) + .withChunkMetadata(chunkIndex, linkInfo.getRowCount(), linkInfo.getRowOffset()) + .withChunkReadyTimeoutSeconds(chunkReadyTimeoutSeconds) + .build(); + + chunk.setChunkLink(linkInfo.getLink()); + chunks.put(chunkIndex, chunk); + highestKnownChunkIndex = Math.max(highestKnownChunkIndex, chunkIndex); + totalRowCount.addAndGet(linkInfo.getRowCount()); + + // Notify any waiting consumers that a chunk is available + notifyChunkCreated(); + + LOGGER.debug( + "Created chunk {} with {} rows for statement {}", + chunkIndex, + linkInfo.getRowCount(), + statementId); } // ==================== Download Coordination ==================== @@ -453,7 +469,9 @@ private void createChunkFromLink(ChunkLinkFetchResult.ChunkLinkInfo linkInfo) { private void triggerDownloads() { downloadLock.lock(); try { - while (!closed && chunksInMemory.get() < maxChunksInMemory && nextDownloadIndex <= highestKnownChunkIndex) { + while (!closed + && chunksInMemory.get() < maxChunksInMemory + && nextDownloadIndex <= highestKnownChunkIndex) { ArrowResultChunk chunk = chunks.get(nextDownloadIndex); if (chunk == null) { @@ -478,12 +496,9 @@ private void triggerDownloads() { private void submitDownloadTask(ArrowResultChunk chunk) { LOGGER.debug("Submitting download task for chunk {}", chunk.getChunkIndex()); - StreamingChunkDownloadTask task = new StreamingChunkDownloadTask( - chunk, - httpClient, - compressionCodec, - linkFetcher, - cloudFetchSpeedThreshold); + StreamingChunkDownloadTask task = + new StreamingChunkDownloadTask( + chunk, httpClient, compressionCodec, linkFetcher, cloudFetchSpeedThreshold); downloadExecutor.submit(task); } @@ -498,35 +513,45 @@ private void releaseChunk(long chunkIndex) { LOGGER.debug("Released chunk {}, chunksInMemory={}", chunkIndex, chunksInMemory.get()); - // Notify download coordinator - notifyDownloadSlotAvailable(); - - // Trigger more downloads + // Trigger more downloads to fill the freed slot triggerDownloads(); } } private void waitForChunkCreation(long chunkIndex) throws DatabricksSQLException { - long startTime = System.currentTimeMillis(); - long timeoutMs = chunkReadyTimeoutSeconds * 1000L; + long remainingNanos = 5 * 1_000_000_000L; - while (!closed && !chunks.containsKey(chunkIndex)) { - if (endOfStreamReached && chunkIndex > highestKnownChunkIndex) { - throw new DatabricksSQLException( - "Chunk " + chunkIndex + " does not exist (highest known: " + highestKnownChunkIndex + ")", DatabricksDriverErrorCode.CHUNK_READY_ERROR); - } + prefetchLock.lock(); + try { + while (!closed && !chunks.containsKey(chunkIndex)) { + if (endOfStreamReached && chunkIndex > highestKnownChunkIndex) { + throw new DatabricksSQLException( + "Chunk " + + chunkIndex + + " does not exist (highest known: " + + highestKnownChunkIndex + + ")", + DatabricksDriverErrorCode.CHUNK_READY_ERROR); + } - if (timeoutMs > 0 && System.currentTimeMillis() - startTime > timeoutMs) { - throw new DatabricksSQLException( - "Timeout waiting for chunk " + chunkIndex + " to be created", DatabricksDriverErrorCode.CHUNK_READY_ERROR); - } + if (remainingNanos <= 0) { + throw new DatabricksSQLException( + "Timeout waiting for chunk " + chunkIndex + " to be created", + DatabricksDriverErrorCode.CHUNK_READY_ERROR); + } - try { - Thread.sleep(10); // Brief wait before checking again - } catch (InterruptedException e) { - Thread.currentThread().interrupt(); - throw new DatabricksSQLException("Interrupted waiting for chunk creation", e, DatabricksDriverErrorCode.THREAD_INTERRUPTED_ERROR); + try { + remainingNanos = chunkCreated.awaitNanos(remainingNanos); + } catch (InterruptedException e) { + Thread.currentThread().interrupt(); + throw new DatabricksSQLException( + "Interrupted waiting for chunk creation", + e, + DatabricksDriverErrorCode.THREAD_INTERRUPTED_ERROR); + } } + } finally { + prefetchLock.unlock(); } } @@ -541,29 +566,30 @@ private void notifyConsumerAdvanced() { } } - private void notifyDownloadSlotAvailable() { - downloadLock.lock(); + private void notifyChunkCreated() { + prefetchLock.lock(); try { - downloadSlotAvailable.signalAll(); + chunkCreated.signalAll(); } finally { - downloadLock.unlock(); + prefetchLock.unlock(); } } // ==================== Executor Creation ==================== private ExecutorService createDownloadExecutor(int poolSize) { - ThreadFactory threadFactory = new ThreadFactory() { - private final AtomicInteger threadCount = new AtomicInteger(1); - - @Override - public Thread newThread(@Nonnull Runnable r) { - Thread thread = new Thread(r); - thread.setName(DOWNLOAD_THREAD_PREFIX + threadCount.getAndIncrement()); - thread.setDaemon(true); - return thread; - } - }; + ThreadFactory threadFactory = + new ThreadFactory() { + private final AtomicInteger threadCount = new AtomicInteger(1); + + @Override + public Thread newThread(@Nonnull Runnable r) { + Thread thread = new Thread(r); + thread.setName(DOWNLOAD_THREAD_PREFIX + threadCount.getAndIncrement()); + thread.setDaemon(true); + return thread; + } + }; return Executors.newFixedThreadPool(poolSize, threadFactory); } From cd1452931ac8f2cf146528da1046db6c0c275f4a Mon Sep 17 00:00:00 2001 From: Jayant Singh Date: Thu, 27 Nov 2025 21:49:54 +0000 Subject: [PATCH 3/6] Add Thrift implementation --- .../impl/arrow/AbstractArrowResultChunk.java | 9 + .../jdbc/api/impl/arrow/ArrowResultChunk.java | 2 +- .../api/impl/arrow/ArrowStreamResult.java | 160 ++++++++++++++--- .../impl/arrow/ChunkLinkDownloadService.java | 8 +- .../api/impl/arrow/ChunkLinkFetchResult.java | 33 +++- .../jdbc/api/impl/arrow/ChunkLinkFetcher.java | 19 ++- .../api/impl/arrow/SeaChunkLinkFetcher.java | 54 +++--- .../arrow/StreamingChunkDownloadTask.java | 3 +- .../impl/arrow/StreamingChunkProvider.java | 47 ++--- .../impl/arrow/ThriftChunkLinkFetcher.java | 161 ++++++++++++++++++ .../common/util/DatabricksThriftUtil.java | 6 +- .../jdbc/dbclient/IDatabricksClient.java | 20 ++- .../impl/sqlexec/DatabricksSdkClient.java | 14 +- .../impl/thrift/DatabricksThriftAccessor.java | 52 ++++++ .../thrift/DatabricksThriftServiceClient.java | 82 ++++++--- .../jdbc/model/core/GetChunksResult.java | 109 ++++++++++++ .../api/impl/arrow/ArrowStreamResultTest.java | 8 +- .../arrow/ChunkLinkDownloadServiceTest.java | 83 ++++----- .../DatabricksThriftServiceClientTest.java | 20 ++- 19 files changed, 718 insertions(+), 172 deletions(-) create mode 100644 src/main/java/com/databricks/jdbc/api/impl/arrow/ThriftChunkLinkFetcher.java create mode 100644 src/main/java/com/databricks/jdbc/model/core/GetChunksResult.java diff --git a/src/main/java/com/databricks/jdbc/api/impl/arrow/AbstractArrowResultChunk.java b/src/main/java/com/databricks/jdbc/api/impl/arrow/AbstractArrowResultChunk.java index dcc9edff5..06d711170 100644 --- a/src/main/java/com/databricks/jdbc/api/impl/arrow/AbstractArrowResultChunk.java +++ b/src/main/java/com/databricks/jdbc/api/impl/arrow/AbstractArrowResultChunk.java @@ -110,6 +110,15 @@ public Long getChunkIndex() { return chunkIndex; } + /** + * Returns the starting row offset for this chunk. + * + * @return the row offset + */ + public long getRowOffset() { + return rowOffset; + } + /** * Checks if the chunk link is invalid or expired. * diff --git a/src/main/java/com/databricks/jdbc/api/impl/arrow/ArrowResultChunk.java b/src/main/java/com/databricks/jdbc/api/impl/arrow/ArrowResultChunk.java index 17380c592..9aba1304d 100644 --- a/src/main/java/com/databricks/jdbc/api/impl/arrow/ArrowResultChunk.java +++ b/src/main/java/com/databricks/jdbc/api/impl/arrow/ArrowResultChunk.java @@ -158,7 +158,7 @@ private void logDownloadMetrics( double speedMBps = (contentLength / 1024.0 / 1024.0) / (downloadTimeMs / 1000.0); String baseUrl = url.split("\\?")[0]; - LOGGER.info( + LOGGER.debug( String.format( "CloudFetch download: %.4f MB/s, %d bytes in %dms from %s", speedMBps, contentLength, downloadTimeMs, baseUrl)); diff --git a/src/main/java/com/databricks/jdbc/api/impl/arrow/ArrowStreamResult.java b/src/main/java/com/databricks/jdbc/api/impl/arrow/ArrowStreamResult.java index 54f45c097..06b5d7ec5 100644 --- a/src/main/java/com/databricks/jdbc/api/impl/arrow/ArrowStreamResult.java +++ b/src/main/java/com/databricks/jdbc/api/impl/arrow/ArrowStreamResult.java @@ -1,5 +1,6 @@ package com.databricks.jdbc.api.impl.arrow; +import static com.databricks.jdbc.common.util.DatabricksThriftUtil.createExternalLink; import static com.databricks.jdbc.common.util.DatabricksThriftUtil.getColumnInfoFromTColumnDesc; import com.databricks.jdbc.api.impl.ComplexDataTypeParser; @@ -17,6 +18,7 @@ import com.databricks.jdbc.model.client.thrift.generated.TColumnDesc; import com.databricks.jdbc.model.client.thrift.generated.TFetchResultsResp; import com.databricks.jdbc.model.client.thrift.generated.TGetResultSetMetadataResp; +import com.databricks.jdbc.model.client.thrift.generated.TSparkArrowResultLink; import com.databricks.jdbc.model.core.ColumnInfo; import com.databricks.jdbc.model.core.ColumnInfoTypeName; import com.databricks.jdbc.model.core.ExternalLink; @@ -111,9 +113,9 @@ private static ChunkProvider createRemoteChunkProvider( int chunkReadyTimeoutSeconds = connectionContext.getChunkReadyTimeoutSeconds(); double cloudFetchSpeedThreshold = connectionContext.getCloudFetchSpeedThreshold(); - // Convert ExternalLinks to ChunkLinkInfo for the provider - Collection initialLinks = - convertToChunkLinkInfos(resultData.getExternalLinks()); + // Convert ExternalLinks to ChunkLinkFetchResult for the provider + ChunkLinkFetchResult initialLinks = + convertToChunkLinkFetchResult(resultData.getExternalLinks()); return new StreamingChunkProvider( linkFetcher, @@ -163,16 +165,61 @@ public ArrowStreamResult( if (isInlineArrow) { this.chunkProvider = new InlineChunkProvider(resultsResp, parentStatement, session); } else { - CompressionCodec compressionCodec = - CompressionCodec.getCompressionMapping(resultsResp.getResultSetMetadata()); this.chunkProvider = - new RemoteChunkProvider( - parentStatement, - resultsResp, - session, - httpClient, - session.getConnectionContext().getCloudFetchThreadPoolSize(), - compressionCodec); + createThriftRemoteChunkProvider(resultsResp, parentStatement, session, httpClient); + } + } + + /** + * Creates the appropriate remote chunk provider for Thrift based on configuration. + * + * @param resultsResp The Thrift fetch results response + * @param parentStatement The parent statement for fetching additional chunks + * @param session The session for fetching additional chunks + * @param httpClient The HTTP client for downloading chunk data + * @return A ChunkProvider instance + */ + private static ChunkProvider createThriftRemoteChunkProvider( + TFetchResultsResp resultsResp, + IDatabricksStatementInternal parentStatement, + IDatabricksSession session, + IDatabricksHttpClient httpClient) + throws DatabricksSQLException { + + IDatabricksConnectionContext connectionContext = session.getConnectionContext(); + CompressionCodec compressionCodec = + CompressionCodec.getCompressionMapping(resultsResp.getResultSetMetadata()); + + if (connectionContext.isStreamingChunkProviderEnabled()) { + StatementId statementId = parentStatement.getStatementId(); + LOGGER.info("Using StreamingChunkProvider for Thrift statementId: {}", statementId); + + ChunkLinkFetcher linkFetcher = new ThriftChunkLinkFetcher(session, statementId); + int maxChunksInMemory = connectionContext.getCloudFetchThreadPoolSize(); + int chunkReadyTimeoutSeconds = connectionContext.getChunkReadyTimeoutSeconds(); + double cloudFetchSpeedThreshold = connectionContext.getCloudFetchSpeedThreshold(); + + // Convert initial Thrift links to ChunkLinkFetchResult + ChunkLinkFetchResult initialLinks = convertThriftLinksToChunkLinkFetchResult(resultsResp); + + return new StreamingChunkProvider( + linkFetcher, + httpClient, + compressionCodec, + statementId, + maxChunksInMemory, + chunkReadyTimeoutSeconds, + cloudFetchSpeedThreshold, + initialLinks); + } else { + // Use the original RemoteChunkProvider + return new RemoteChunkProvider( + parentStatement, + resultsResp, + session, + httpClient, + connectionContext.getCloudFetchThreadPoolSize(), + compressionCodec); } } @@ -323,24 +370,87 @@ private void setColumnInfo(TGetResultSetMetadataResp resultManifest) { } /** - * Converts a collection of ExternalLinks to ChunkLinkInfo objects. + * Converts a collection of ExternalLinks to a ChunkLinkFetchResult. * * @param externalLinks The external links to convert, may be null - * @return A collection of ChunkLinkInfo objects, or null if input is null + * @return A ChunkLinkFetchResult, or null if input is null or empty */ - private static Collection convertToChunkLinkInfos( + private static ChunkLinkFetchResult convertToChunkLinkFetchResult( Collection externalLinks) { - if (externalLinks == null) { + if (externalLinks == null || externalLinks.isEmpty()) { + return null; + } + + List linkList = + externalLinks instanceof List + ? (List) externalLinks + : new ArrayList<>(externalLinks); + + List chunkLinks = + linkList.stream() + .map( + link -> + new ChunkLinkFetchResult.ChunkLinkInfo( + link.getChunkIndex(), link, link.getRowCount(), link.getRowOffset())) + .collect(Collectors.toList()); + + // Derive hasMore and nextRowOffset from last link (SEA style) + ExternalLink lastLink = linkList.get(linkList.size() - 1); + boolean hasMore = lastLink.getNextChunkIndex() != null; + long nextFetchIndex = hasMore ? lastLink.getNextChunkIndex() : -1; + long nextRowOffset = lastLink.getRowOffset() + lastLink.getRowCount(); + + return ChunkLinkFetchResult.of(chunkLinks, hasMore, nextFetchIndex, nextRowOffset); + } + + /** + * Converts Thrift result links to a ChunkLinkFetchResult. + * + *

This method converts TSparkArrowResultLink from the Thrift response to the unified + * ChunkLinkFetchResult format used by StreamingChunkProvider. + * + * @param resultsResp The Thrift fetch results response containing initial links + * @return A ChunkLinkFetchResult, or null if no links + */ + private static ChunkLinkFetchResult convertThriftLinksToChunkLinkFetchResult( + TFetchResultsResp resultsResp) { + List resultLinks = resultsResp.getResults().getResultLinks(); + if (resultLinks == null || resultLinks.isEmpty()) { return null; } - return externalLinks.stream() - .map( - link -> - new ChunkLinkFetchResult.ChunkLinkInfo( - link.getChunkIndex() != null ? link.getChunkIndex() : 0, - link, - link.getRowCount() != null ? link.getRowCount() : 0, - link.getRowOffset() != null ? link.getRowOffset() : 0)) - .collect(Collectors.toList()); + + List chunkLinks = new ArrayList<>(); + int lastIndex = resultLinks.size() - 1; + boolean hasMoreRows = resultsResp.hasMoreRows; + + for (int i = 0; i < resultLinks.size(); i++) { + TSparkArrowResultLink thriftLink = resultLinks.get(i); + + // Convert Thrift link to ExternalLink + ExternalLink externalLink = createExternalLink(thriftLink, i); + + // For the last link, set nextChunkIndex based on hasMoreRows + if (i == lastIndex) { + if (hasMoreRows) { + // More chunks available - next fetch should start from lastIndex + 1 + externalLink.setNextChunkIndex((long) i + 1); + } + // If hasMoreRows is false, nextChunkIndex remains null (end of stream) + } else { + // Not the last link - next chunk follows immediately + externalLink.setNextChunkIndex((long) i + 1); + } + + chunkLinks.add( + new ChunkLinkFetchResult.ChunkLinkInfo( + i, externalLink, thriftLink.getRowCount(), thriftLink.getStartRowOffset())); + } + + // Calculate next fetch positions from last link + TSparkArrowResultLink lastThriftLink = resultLinks.get(lastIndex); + long nextFetchIndex = hasMoreRows ? lastIndex + 1 : -1; + long nextRowOffset = lastThriftLink.getStartRowOffset() + lastThriftLink.getRowCount(); + + return ChunkLinkFetchResult.of(chunkLinks, hasMoreRows, nextFetchIndex, nextRowOffset); } } diff --git a/src/main/java/com/databricks/jdbc/api/impl/arrow/ChunkLinkDownloadService.java b/src/main/java/com/databricks/jdbc/api/impl/arrow/ChunkLinkDownloadService.java index 7d59ea13c..bd824152e 100644 --- a/src/main/java/com/databricks/jdbc/api/impl/arrow/ChunkLinkDownloadService.java +++ b/src/main/java/com/databricks/jdbc/api/impl/arrow/ChunkLinkDownloadService.java @@ -10,6 +10,7 @@ import com.databricks.jdbc.log.JdbcLogger; import com.databricks.jdbc.log.JdbcLoggerFactory; import com.databricks.jdbc.model.core.ExternalLink; +import com.databricks.jdbc.model.core.GetChunksResult; import java.time.Instant; import java.util.Collection; import java.util.Map; @@ -219,8 +220,11 @@ private void triggerNextBatchDownload() { CompletableFuture.runAsync( () -> { try { - Collection links = - session.getDatabricksClient().getResultChunks(statementId, batchStartIndex); + // rowOffset is 0 here as this service is used by RemoteChunkProvider (SEA-only) + // which fetches by chunkIndex, not rowOffset + GetChunksResult result = + session.getDatabricksClient().getResultChunks(statementId, batchStartIndex, 0); + Collection links = result.getExternalLinks(); LOGGER.info( "Retrieved {} links for batch starting at {} for statement id {}", links.size(), diff --git a/src/main/java/com/databricks/jdbc/api/impl/arrow/ChunkLinkFetchResult.java b/src/main/java/com/databricks/jdbc/api/impl/arrow/ChunkLinkFetchResult.java index 044a81187..0eb133a44 100644 --- a/src/main/java/com/databricks/jdbc/api/impl/arrow/ChunkLinkFetchResult.java +++ b/src/main/java/com/databricks/jdbc/api/impl/arrow/ChunkLinkFetchResult.java @@ -14,16 +14,18 @@ public class ChunkLinkFetchResult { private final List chunkLinks; private final boolean hasMore; private final long nextFetchIndex; + private final long nextRowOffset; private ChunkLinkFetchResult( - List chunkLinks, boolean hasMore, long nextFetchIndex) { + List chunkLinks, boolean hasMore, long nextFetchIndex, long nextRowOffset) { this.chunkLinks = chunkLinks; this.hasMore = hasMore; this.nextFetchIndex = nextFetchIndex; + this.nextRowOffset = nextRowOffset; } /** - * Creates a result with the given links and continuation info. + * Creates a result with the given links and continuation info (for SEA). * * @param links The fetched chunk links * @param hasMore Whether more chunks are available @@ -32,7 +34,21 @@ private ChunkLinkFetchResult( */ public static ChunkLinkFetchResult of( List links, boolean hasMore, long nextFetchIndex) { - return new ChunkLinkFetchResult(links, hasMore, nextFetchIndex); + return new ChunkLinkFetchResult(links, hasMore, nextFetchIndex, 0); + } + + /** + * Creates a result with full continuation info (for Thrift). + * + * @param links The fetched chunk links + * @param hasMore Whether more chunks are available + * @param nextFetchIndex The next chunk index to fetch from, or -1 if no more + * @param nextRowOffset The next row offset for Thrift FETCH_ABSOLUTE + * @return A new ChunkLinkFetchResult + */ + public static ChunkLinkFetchResult of( + List links, boolean hasMore, long nextFetchIndex, long nextRowOffset) { + return new ChunkLinkFetchResult(links, hasMore, nextFetchIndex, nextRowOffset); } /** @@ -41,7 +57,7 @@ public static ChunkLinkFetchResult of( * @return A ChunkLinkFetchResult representing end of stream */ public static ChunkLinkFetchResult endOfStream() { - return new ChunkLinkFetchResult(Collections.emptyList(), false, -1); + return new ChunkLinkFetchResult(Collections.emptyList(), false, -1, 0); } /** @@ -71,6 +87,15 @@ public long getNextFetchIndex() { return nextFetchIndex; } + /** + * Returns the next row offset for Thrift FETCH_ABSOLUTE continuation. + * + * @return The next row offset, or 0 if not applicable + */ + public long getNextRowOffset() { + return nextRowOffset; + } + /** * Checks if this result represents the end of the chunk stream. * diff --git a/src/main/java/com/databricks/jdbc/api/impl/arrow/ChunkLinkFetcher.java b/src/main/java/com/databricks/jdbc/api/impl/arrow/ChunkLinkFetcher.java index 6c2e68e90..f5892e867 100644 --- a/src/main/java/com/databricks/jdbc/api/impl/arrow/ChunkLinkFetcher.java +++ b/src/main/java/com/databricks/jdbc/api/impl/arrow/ChunkLinkFetcher.java @@ -13,16 +13,22 @@ public interface ChunkLinkFetcher { /** - * Fetches the next batch of chunk links starting from the given index. + * Fetches the next batch of chunk links starting from the given position. * *

The implementation may return one or more links in a single call. The returned {@link * ChunkLinkFetchResult} indicates whether more chunks are available. * - * @param startChunkIndex The chunk index to start fetching from + *

SEA implementations use startChunkIndex while Thrift implementations use startRowOffset. + * Each implementation uses the parameter relevant to its protocol and ignores the other. + * + * @param startChunkIndex The chunk index to start fetching from (used by SEA) + * @param startRowOffset The row offset to start fetching from (used by Thrift with + * FETCH_ABSOLUTE) * @return ChunkLinkFetchResult containing the fetched links and continuation information * @throws DatabricksSQLException if the fetch operation fails */ - ChunkLinkFetchResult fetchLinks(long startChunkIndex) throws DatabricksSQLException; + ChunkLinkFetchResult fetchLinks(long startChunkIndex, long startRowOffset) + throws DatabricksSQLException; /** * Refetches a specific chunk link that may have expired. @@ -30,11 +36,14 @@ public interface ChunkLinkFetcher { *

This is used when a previously fetched link has expired before the chunk could be * downloaded. Both SEA and Thrift clients support this via the getResultChunks API. * - * @param chunkIndex The specific chunk index to refetch + *

SEA uses chunkIndex while Thrift uses rowOffset to identify the chunk to refetch. + * + * @param chunkIndex The specific chunk index to refetch (used by SEA) + * @param rowOffset The row offset of the chunk to refetch (used by Thrift) * @return The refreshed ExternalLink with a new expiration time * @throws DatabricksSQLException if the refetch operation fails */ - ExternalLink refetchLink(long chunkIndex) throws DatabricksSQLException; + ExternalLink refetchLink(long chunkIndex, long rowOffset) throws DatabricksSQLException; /** Closes any resources held by the fetcher. */ void close(); diff --git a/src/main/java/com/databricks/jdbc/api/impl/arrow/SeaChunkLinkFetcher.java b/src/main/java/com/databricks/jdbc/api/impl/arrow/SeaChunkLinkFetcher.java index 895698a51..4b0843454 100644 --- a/src/main/java/com/databricks/jdbc/api/impl/arrow/SeaChunkLinkFetcher.java +++ b/src/main/java/com/databricks/jdbc/api/impl/arrow/SeaChunkLinkFetcher.java @@ -6,6 +6,7 @@ import com.databricks.jdbc.log.JdbcLogger; import com.databricks.jdbc.log.JdbcLoggerFactory; import com.databricks.jdbc.model.core.ExternalLink; +import com.databricks.jdbc.model.core.GetChunksResult; import com.databricks.jdbc.model.telemetry.enums.DatabricksDriverErrorCode; import java.util.ArrayList; import java.util.Collection; @@ -31,14 +32,17 @@ public SeaChunkLinkFetcher(IDatabricksSession session, StatementId statementId) } @Override - public ChunkLinkFetchResult fetchLinks(long startChunkIndex) throws DatabricksSQLException { + public ChunkLinkFetchResult fetchLinks(long startChunkIndex, long startRowOffset) + throws DatabricksSQLException { + // SEA uses startChunkIndex; startRowOffset is ignored LOGGER.debug( "Fetching links starting from chunk index {} for statement {}", startChunkIndex, statementId); - Collection links = - session.getDatabricksClient().getResultChunks(statementId, startChunkIndex); + GetChunksResult result = + session.getDatabricksClient().getResultChunks(statementId, startChunkIndex, startRowOffset); + Collection links = result.getExternalLinks(); if (links == null || links.isEmpty()) { LOGGER.debug("No links returned, end of stream reached for statement {}", statementId); @@ -51,35 +55,35 @@ public ChunkLinkFetchResult fetchLinks(long startChunkIndex) throws DatabricksSQ for (ExternalLink link : links) { chunkLinks.add( new ChunkLinkFetchResult.ChunkLinkInfo( - link.getChunkIndex(), - link, - link.getRowCount() != null ? link.getRowCount() : 0, - link.getRowOffset() != null ? link.getRowOffset() : 0)); - - // SEA uses nextChunkIndex to indicate continuation. - // The LAST link's nextChunkIndex determines if there are more chunks. - // null means no more chunks after this batch. + link.getChunkIndex(), link, link.getRowCount(), link.getRowOffset())); + + // Track last link's nextChunkIndex for logging nextIndex = link.getNextChunkIndex(); } - boolean hasMore = (nextIndex != null); + // Use hasMoreData and nextRowOffset from GetChunksResult (unified with Thrift) + boolean hasMore = result.hasMoreData(); + long nextRowOffset = result.getNextRowOffset(); LOGGER.debug( - "Fetched {} links for statement {}, hasMore={}, nextIndex={}", + "Fetched {} links for statement {}, hasMore={}, nextIndex={}, nextRowOffset={}", chunkLinks.size(), statementId, hasMore, - nextIndex); + nextIndex, + nextRowOffset); - return ChunkLinkFetchResult.of(chunkLinks, hasMore, hasMore ? nextIndex : -1); + return ChunkLinkFetchResult.of(chunkLinks, hasMore, hasMore ? nextIndex : -1, nextRowOffset); } @Override - public ExternalLink refetchLink(long chunkIndex) throws DatabricksSQLException { + public ExternalLink refetchLink(long chunkIndex, long rowOffset) throws DatabricksSQLException { + // SEA uses chunkIndex; rowOffset is ignored LOGGER.info("Refetching expired link for chunk {} of statement {}", chunkIndex, statementId); - Collection links = - session.getDatabricksClient().getResultChunks(statementId, chunkIndex); + GetChunksResult result = + session.getDatabricksClient().getResultChunks(statementId, chunkIndex, rowOffset); + Collection links = result.getExternalLinks(); if (links == null || links.isEmpty()) { throw new DatabricksSQLException( @@ -96,14 +100,12 @@ public ExternalLink refetchLink(long chunkIndex) throws DatabricksSQLException { } } - // If exact match not found, return the first link (server should return the requested chunk) - ExternalLink firstLink = links.iterator().next(); - LOGGER.warn( - "Exact chunk index {} not found in response, using first link with index {} for statement {}", - chunkIndex, - firstLink.getChunkIndex(), - statementId); - return firstLink; + // Exact match not found - this indicates a server bug + throw new DatabricksSQLException( + String.format( + "Failed to refetch link for chunk %d: server returned links but none matched requested index", + chunkIndex), + DatabricksDriverErrorCode.CHUNK_READY_ERROR); } @Override diff --git a/src/main/java/com/databricks/jdbc/api/impl/arrow/StreamingChunkDownloadTask.java b/src/main/java/com/databricks/jdbc/api/impl/arrow/StreamingChunkDownloadTask.java index d36a9477a..2bd8ed2eb 100644 --- a/src/main/java/com/databricks/jdbc/api/impl/arrow/StreamingChunkDownloadTask.java +++ b/src/main/java/com/databricks/jdbc/api/impl/arrow/StreamingChunkDownloadTask.java @@ -52,7 +52,8 @@ public Void call() throws DatabricksSQLException { // Check if link is expired and refresh if needed if (chunk.isChunkLinkInvalid()) { LOGGER.debug("Link invalid for chunk {}, refetching", chunk.getChunkIndex()); - ExternalLink freshLink = linkFetcher.refetchLink(chunk.getChunkIndex()); + ExternalLink freshLink = + linkFetcher.refetchLink(chunk.getChunkIndex(), chunk.getRowOffset()); chunk.setChunkLink(freshLink); } diff --git a/src/main/java/com/databricks/jdbc/api/impl/arrow/StreamingChunkProvider.java b/src/main/java/com/databricks/jdbc/api/impl/arrow/StreamingChunkProvider.java index 71790fd79..719f3f104 100644 --- a/src/main/java/com/databricks/jdbc/api/impl/arrow/StreamingChunkProvider.java +++ b/src/main/java/com/databricks/jdbc/api/impl/arrow/StreamingChunkProvider.java @@ -8,7 +8,6 @@ import com.databricks.jdbc.log.JdbcLogger; import com.databricks.jdbc.log.JdbcLoggerFactory; import com.databricks.jdbc.model.telemetry.enums.DatabricksDriverErrorCode; -import java.util.Collection; import java.util.concurrent.ConcurrentHashMap; import java.util.concurrent.ConcurrentMap; import java.util.concurrent.ExecutionException; @@ -67,6 +66,7 @@ public class StreamingChunkProvider implements ChunkProvider { private volatile long currentChunkIndex = -1; private volatile long highestKnownChunkIndex = -1; private volatile long nextLinkFetchIndex = 0; + private volatile long nextRowOffsetToFetch = 0; private volatile long nextDownloadIndex = 0; // State flags @@ -113,7 +113,7 @@ public StreamingChunkProvider( int linkPrefetchWindow, int chunkReadyTimeoutSeconds, double cloudFetchSpeedThreshold, - Collection initialLinks) + ChunkLinkFetchResult initialLinks) throws DatabricksParsingException { this.linkFetcher = linkFetcher; @@ -156,7 +156,7 @@ public StreamingChunkProvider( int maxChunksInMemory, int chunkReadyTimeoutSeconds, double cloudFetchSpeedThreshold, - Collection initialLinks) + ChunkLinkFetchResult initialLinks) throws DatabricksParsingException { this( linkFetcher, @@ -367,9 +367,12 @@ private void fetchNextLinkBatch() throws DatabricksSQLException { } LOGGER.debug( - "Fetching links starting from index {} for statement {}", nextLinkFetchIndex, statementId); + "Fetching links starting from index {}, row offset {} for statement {}", + nextLinkFetchIndex, + nextRowOffsetToFetch, + statementId); - ChunkLinkFetchResult result = linkFetcher.fetchLinks(nextLinkFetchIndex); + ChunkLinkFetchResult result = linkFetcher.fetchLinks(nextLinkFetchIndex, nextRowOffsetToFetch); if (result.isEndOfStream()) { LOGGER.info("End of stream reached for statement {}", statementId); @@ -382,9 +385,10 @@ private void fetchNextLinkBatch() throws DatabricksSQLException { createChunkFromLink(linkInfo); } - // Update next fetch position + // Update next fetch positions if (result.hasMore()) { nextLinkFetchIndex = result.getNextFetchIndex(); + nextRowOffsetToFetch = result.getNextRowOffset(); } else { endOfStreamReached = true; LOGGER.info("End of stream reached for statement {} (hasMore=false)", statementId); @@ -398,32 +402,33 @@ private void fetchNextLinkBatch() throws DatabricksSQLException { * Processes initial links provided with the result data. This avoids an extra fetch call for * links the server already provided. * - * @param initialLinks The initial links from ResultData, may be null or empty + * @param initialLinks The initial links from ResultData, may be null */ - private void processInitialLinks(Collection initialLinks) + private void processInitialLinks(ChunkLinkFetchResult initialLinks) throws DatabricksParsingException { - if (initialLinks == null || initialLinks.isEmpty()) { + if (initialLinks == null || initialLinks.isEndOfStream()) { LOGGER.debug("No initial links provided for statement {}", statementId); return; } - LOGGER.info("Processing {} initial links for statement {}", initialLinks.size(), statementId); - - Long lastNextChunkIndex = null; + LOGGER.info( + "Processing {} initial links for statement {}", + initialLinks.getChunkLinks().size(), + statementId); - for (ChunkLinkFetchResult.ChunkLinkInfo linkInfo : initialLinks) { + for (ChunkLinkFetchResult.ChunkLinkInfo linkInfo : initialLinks.getChunkLinks()) { createChunkFromLink(linkInfo); - if (linkInfo.getLink() != null) { - lastNextChunkIndex = linkInfo.getLink().getNextChunkIndex(); - } } - // Set next fetch index based on the last link's nextChunkIndex - if (lastNextChunkIndex != null) { - nextLinkFetchIndex = lastNextChunkIndex; - LOGGER.debug("Next link fetch index set to {} from initial links", nextLinkFetchIndex); + // Set next fetch positions using unified API + if (initialLinks.hasMore()) { + nextLinkFetchIndex = initialLinks.getNextFetchIndex(); + nextRowOffsetToFetch = initialLinks.getNextRowOffset(); + LOGGER.debug( + "Next fetch position set to chunk index {}, row offset {} from initial links", + nextLinkFetchIndex, + nextRowOffsetToFetch); } else { - // Last link has null nextChunkIndex - no more chunks endOfStreamReached = true; LOGGER.info("End of stream reached from initial links for statement {}", statementId); } diff --git a/src/main/java/com/databricks/jdbc/api/impl/arrow/ThriftChunkLinkFetcher.java b/src/main/java/com/databricks/jdbc/api/impl/arrow/ThriftChunkLinkFetcher.java new file mode 100644 index 000000000..bd126b6b2 --- /dev/null +++ b/src/main/java/com/databricks/jdbc/api/impl/arrow/ThriftChunkLinkFetcher.java @@ -0,0 +1,161 @@ +package com.databricks.jdbc.api.impl.arrow; + +import com.databricks.jdbc.api.internal.IDatabricksSession; +import com.databricks.jdbc.dbclient.impl.common.StatementId; +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 com.databricks.jdbc.model.core.GetChunksResult; +import com.databricks.jdbc.model.telemetry.enums.DatabricksDriverErrorCode; +import java.util.ArrayList; +import java.util.Collection; +import java.util.List; + +/** + * ChunkLinkFetcher implementation for the Thrift client. + * + *

Thrift provides chunk links via the getResultChunks API, which returns links with + * nextChunkIndex to indicate continuation. When nextChunkIndex is null, it indicates no more + * chunks. + */ +public class ThriftChunkLinkFetcher implements ChunkLinkFetcher { + + private static final JdbcLogger LOGGER = + JdbcLoggerFactory.getLogger(ThriftChunkLinkFetcher.class); + + private final IDatabricksSession session; + private final StatementId statementId; + + public ThriftChunkLinkFetcher(IDatabricksSession session, StatementId statementId) { + this.session = session; + this.statementId = statementId; + LOGGER.debug("Created ThriftChunkLinkFetcher for statement {}", statementId); + } + + @Override + public ChunkLinkFetchResult fetchLinks(long startChunkIndex, long startRowOffset) + throws DatabricksSQLException { + // Thrift uses startRowOffset with FETCH_ABSOLUTE; startChunkIndex is used for metadata + LOGGER.debug( + "Fetching links starting from chunk index {}, row offset {} for statement {}", + startChunkIndex, + startRowOffset, + statementId); + + GetChunksResult result = + session.getDatabricksClient().getResultChunks(statementId, startChunkIndex, startRowOffset); + + Collection links = result.getExternalLinks(); + boolean hasMore = result.hasMoreData(); + long nextRowOffset = result.getNextRowOffset(); + + if (links == null || links.isEmpty()) { + // For Thrift, hasMoreData() is the source of truth. Even with no links, + // if hasMore is true, we should indicate continuation with the same offset. + if (hasMore) { + LOGGER.debug( + "No links returned but hasMoreData=true for statement {}. " + + "Returning empty result with hasMore=true for retry with offset {}", + statementId, + nextRowOffset); + return ChunkLinkFetchResult.of(new ArrayList<>(), true, startChunkIndex, nextRowOffset); + } + LOGGER.debug("No links returned, end of stream reached for statement {}", statementId); + return ChunkLinkFetchResult.endOfStream(); + } + + List chunkLinks = new ArrayList<>(); + Long nextIndex = null; + + for (ExternalLink link : links) { + chunkLinks.add( + new ChunkLinkFetchResult.ChunkLinkInfo( + link.getChunkIndex(), link, link.getRowCount(), link.getRowOffset())); + + // Track the last link's nextChunkIndex for logging/metadata + nextIndex = link.getNextChunkIndex(); + } + + LOGGER.debug( + "Fetched {} links for statement {}, hasMore={}, nextIndex={}, nextRowOffset={}", + chunkLinks.size(), + statementId, + hasMore, + nextIndex, + nextRowOffset); + + // For Thrift, hasMore comes from GetChunksResult.hasMoreData() (the server's hasMoreRows flag) + return ChunkLinkFetchResult.of(chunkLinks, hasMore, hasMore ? nextIndex : -1, nextRowOffset); + } + + @Override + public ExternalLink refetchLink(long chunkIndex, long rowOffset) throws DatabricksSQLException { + // Thrift uses rowOffset with FETCH_ABSOLUTE + LOGGER.info( + "Refetching expired link for chunk {}, row offset {} of statement {}", + chunkIndex, + rowOffset, + statementId); + + // For Thrift, we may need to retry if hasMoreData=true but no links returned yet + int maxRetries = 100; // Reasonable limit to prevent infinite loops + int retryCount = 0; + + while (retryCount < maxRetries) { + GetChunksResult result = + session.getDatabricksClient().getResultChunks(statementId, chunkIndex, rowOffset); + Collection links = result.getExternalLinks(); + + if (links != null && !links.isEmpty()) { + // Find the link for the requested chunk index + for (ExternalLink link : links) { + if (link.getChunkIndex() != null && link.getChunkIndex() == chunkIndex) { + LOGGER.debug( + "Successfully refetched link for chunk {} of statement {}", + chunkIndex, + statementId); + return link; + } + } + + // Exact match not found - this indicates a server bug + throw new DatabricksSQLException( + String.format( + "Failed to refetch link for chunk %d: server returned links but none matched requested index", + chunkIndex), + DatabricksDriverErrorCode.CHUNK_READY_ERROR); + } + + // No links returned - check if we should retry + if (!result.hasMoreData()) { + // No more data and no links - this is unexpected for a refetch + throw new DatabricksSQLException( + String.format( + "Failed to refetch link for chunk %d: no links returned and hasMoreData=false", + chunkIndex), + DatabricksDriverErrorCode.CHUNK_READY_ERROR); + } + + // hasMoreData=true but no links yet - retry + retryCount++; + LOGGER.debug( + "No links returned for chunk {} but hasMoreData=true, retrying ({}/{})", + chunkIndex, + retryCount, + maxRetries); + } + + throw new DatabricksSQLException( + String.format( + "Failed to refetch link for chunk %d: max retries (%d) exceeded", + chunkIndex, maxRetries), + DatabricksDriverErrorCode.CHUNK_READY_ERROR); + } + + @Override + public void close() { + LOGGER.debug("Closing ThriftChunkLinkFetcher for statement {}", statementId); + // No resources to clean up for Thrift fetcher + } +} diff --git a/src/main/java/com/databricks/jdbc/common/util/DatabricksThriftUtil.java b/src/main/java/com/databricks/jdbc/common/util/DatabricksThriftUtil.java index b5df82129..308dfafe6 100644 --- a/src/main/java/com/databricks/jdbc/common/util/DatabricksThriftUtil.java +++ b/src/main/java/com/databricks/jdbc/common/util/DatabricksThriftUtil.java @@ -21,6 +21,7 @@ import com.databricks.jdbc.model.telemetry.enums.DatabricksDriverErrorCode; import com.databricks.sdk.service.sql.StatementState; import java.nio.ByteBuffer; +import java.time.Instant; import java.util.*; public class DatabricksThriftUtil { @@ -73,7 +74,10 @@ public static ExternalLink createExternalLink(TSparkArrowResultLink chunkInfo, l return new ExternalLink() .setExternalLink(chunkInfo.getFileLink()) .setChunkIndex(chunkIndex) - .setExpiration(Long.toString(chunkInfo.getExpiryTime())); + .setExpiration(Instant.ofEpochMilli(chunkInfo.getExpiryTime()).toString()) + .setRowOffset(chunkInfo.getStartRowOffset()) + .setByteCount(chunkInfo.getBytesNum()) + .setRowCount(chunkInfo.getRowCount()); } public static void verifySuccessStatus(TStatus status, String errorContext) diff --git a/src/main/java/com/databricks/jdbc/dbclient/IDatabricksClient.java b/src/main/java/com/databricks/jdbc/dbclient/IDatabricksClient.java index e1c6398f1..1acb54c39 100644 --- a/src/main/java/com/databricks/jdbc/dbclient/IDatabricksClient.java +++ b/src/main/java/com/databricks/jdbc/dbclient/IDatabricksClient.java @@ -9,12 +9,11 @@ import com.databricks.jdbc.dbclient.impl.common.StatementId; import com.databricks.jdbc.exception.DatabricksSQLException; import com.databricks.jdbc.model.client.thrift.generated.TFetchResultsResp; -import com.databricks.jdbc.model.core.ExternalLink; +import com.databricks.jdbc.model.core.GetChunksResult; import com.databricks.jdbc.model.core.ResultData; import com.databricks.jdbc.telemetry.latency.DatabricksMetricsTimed; import com.databricks.sdk.core.DatabricksConfig; import java.sql.SQLException; -import java.util.Collection; import java.util.Map; /** Interface for Databricks client which abstracts the integration with Databricks server. */ @@ -117,10 +116,23 @@ DatabricksResultSet getStatementResult( /** * Fetches the chunk details for given chunk index and statement-Id. * + *

For SEA clients, the chunkIndex is used to identify which chunk to fetch. For Thrift + * clients, the rowOffset is used with FETCH_ABSOLUTE orientation to seek to the correct position. + * + *

The returned {@link GetChunksResult} contains the external links and continuation + * information: + * + *

    + *
  • SEA: hasMoreData derived from last link's nextChunkIndex + *
  • Thrift: hasMoreData from server's hasMoreRows flag, nextRowOffset for continuation + *
+ * * @param statementId statement-Id for which chunk should be fetched - * @param chunkIndex chunkIndex for which chunk should be fetched + * @param chunkIndex chunkIndex for which chunk should be fetched (used by SEA) + * @param rowOffset row offset for fetching results (used by Thrift with FETCH_ABSOLUTE) + * @return GetChunksResult containing links and continuation information */ - Collection getResultChunks(StatementId statementId, long chunkIndex) + GetChunksResult getResultChunks(StatementId statementId, long chunkIndex, long rowOffset) throws DatabricksSQLException; /** diff --git a/src/main/java/com/databricks/jdbc/dbclient/impl/sqlexec/DatabricksSdkClient.java b/src/main/java/com/databricks/jdbc/dbclient/impl/sqlexec/DatabricksSdkClient.java index b8dfed049..cc09f93f0 100644 --- a/src/main/java/com/databricks/jdbc/dbclient/impl/sqlexec/DatabricksSdkClient.java +++ b/src/main/java/com/databricks/jdbc/dbclient/impl/sqlexec/DatabricksSdkClient.java @@ -30,7 +30,7 @@ import com.databricks.jdbc.model.client.sqlexec.GetStatementResponse; import com.databricks.jdbc.model.client.thrift.generated.TFetchResultsResp; import com.databricks.jdbc.model.core.Disposition; -import com.databricks.jdbc.model.core.ExternalLink; +import com.databricks.jdbc.model.core.GetChunksResult; import com.databricks.jdbc.model.core.ResultData; import com.databricks.jdbc.model.telemetry.enums.DatabricksDriverErrorCode; import com.databricks.sdk.WorkspaceClient; @@ -409,14 +409,12 @@ public void cancelStatement(StatementId typedStatementId) throws DatabricksSQLEx } @Override - public Collection getResultChunks(StatementId typedStatementId, long chunkIndex) - throws DatabricksSQLException { - DatabricksThreadContextHolder.setStatementId(typedStatementId); + public GetChunksResult getResultChunks( + StatementId typedStatementId, long chunkIndex, long rowOffset) throws DatabricksSQLException { + // SEA uses chunkIndex; rowOffset is ignored String statementId = typedStatementId.toSQLExecStatementId(); LOGGER.debug( - "public Optional getResultChunk(String statementId = {}, long chunkIndex = {})", - statementId, - chunkIndex); + "getResultChunks(statementId={}, chunkIndex={}) using SEA client", statementId, chunkIndex); GetStatementResultChunkNRequest request = new GetStatementResultChunkNRequest().setStatementId(statementId).setChunkIndex(chunkIndex); String path = String.format(RESULT_CHUNK_PATH, statementId, chunkIndex); @@ -424,7 +422,7 @@ public Collection getResultChunks(StatementId typedStatementId, lo Request req = new Request(Request.GET, path, apiClient.serialize(request)); req.withHeaders(getHeaders("getStatementResultN")); ResultData resultData = apiClient.execute(req, ResultData.class); - return resultData.getExternalLinks(); + return GetChunksResult.forSea(resultData.getExternalLinks()); } catch (IOException e) { String errorMessage = "Error while processing the get result chunk request"; LOGGER.error(errorMessage, e); diff --git a/src/main/java/com/databricks/jdbc/dbclient/impl/thrift/DatabricksThriftAccessor.java b/src/main/java/com/databricks/jdbc/dbclient/impl/thrift/DatabricksThriftAccessor.java index 5897ec5ac..502c4a727 100644 --- a/src/main/java/com/databricks/jdbc/dbclient/impl/thrift/DatabricksThriftAccessor.java +++ b/src/main/java/com/databricks/jdbc/dbclient/impl/thrift/DatabricksThriftAccessor.java @@ -521,6 +521,58 @@ TFetchResultsResp getResultSetResp( return response; } + /** + * Fetches results using FETCH_ABSOLUTE orientation starting from the given row offset. + * + *

This method is used by the streaming chunk provider to seek to a specific row position and + * fetch a batch of results. + * + * @param operationHandle The operation handle for the statement + * @param startRowOffset The row offset to start fetching from (0-indexed) + * @param context Context string for logging + * @return The fetch results response + * @throws DatabricksHttpException if the fetch fails + */ + TFetchResultsResp fetchResultsWithAbsoluteOffset( + TOperationHandle operationHandle, long startRowOffset, String context) + throws DatabricksHttpException { + String statementId = StatementId.loggableStatementId(operationHandle); + LOGGER.debug( + "Fetching results with FETCH_ABSOLUTE at offset {} for statement {}", + startRowOffset, + statementId); + + TFetchResultsReq request = + new TFetchResultsReq() + .setOperationHandle(operationHandle) + .setStartRowOffset(startRowOffset) + .setFetchType((short) 0) // 0 represents Query output + .setMaxRows(maxRowsPerBlock) + .setMaxBytes(DEFAULT_BYTE_LIMIT); + + TFetchResultsResp response; + try { + response = getThriftClient().FetchResults(request); + } catch (TException e) { + String errorMessage = + String.format( + "Error while fetching results from Thrift server with FETCH_ABSOLUTE. " + + "startRowOffset=%d, maxRows=%d, Error {%s}", + startRowOffset, request.getMaxRows(), e.getMessage()); + LOGGER.error(e, errorMessage); + throw new DatabricksHttpException(errorMessage, e, DatabricksDriverErrorCode.INVALID_STATE); + } + + verifySuccessStatus( + response.getStatus(), + String.format( + "Error while fetching results with FETCH_ABSOLUTE. startRowOffset=%d, hasMoreRows=%s", + startRowOffset, response.hasMoreRows), + statementId); + + return response; + } + private TFetchResultsResp listFunctions(TGetFunctionsReq request) throws TException, DatabricksSQLException { if (enableDirectResults) request.setGetDirectResults(DEFAULT_DIRECT_RESULTS); diff --git a/src/main/java/com/databricks/jdbc/dbclient/impl/thrift/DatabricksThriftServiceClient.java b/src/main/java/com/databricks/jdbc/dbclient/impl/thrift/DatabricksThriftServiceClient.java index 1ebeab8ec..f0580171b 100644 --- a/src/main/java/com/databricks/jdbc/dbclient/impl/thrift/DatabricksThriftServiceClient.java +++ b/src/main/java/com/databricks/jdbc/dbclient/impl/thrift/DatabricksThriftServiceClient.java @@ -29,6 +29,7 @@ import com.databricks.jdbc.log.JdbcLoggerFactory; import com.databricks.jdbc.model.client.thrift.generated.*; import com.databricks.jdbc.model.core.ExternalLink; +import com.databricks.jdbc.model.core.GetChunksResult; import com.databricks.jdbc.model.core.ResultData; import com.databricks.jdbc.model.telemetry.enums.DatabricksDriverErrorCode; import com.databricks.sdk.core.DatabricksConfig; @@ -36,7 +37,6 @@ import java.math.BigDecimal; import java.sql.SQLException; import java.util.*; -import java.util.concurrent.atomic.AtomicInteger; import java.util.stream.Collectors; public class DatabricksThriftServiceClient implements IDatabricksClient, IDatabricksMetadataClient { @@ -300,32 +300,66 @@ public DatabricksResultSet getStatementResult( } @Override - public Collection getResultChunks(StatementId statementId, long chunkIndex) + public GetChunksResult getResultChunks(StatementId statementId, long chunkIndex, long rowOffset) throws DatabricksSQLException { - String context = - String.format( - "public Optional getResultChunk(String statementId = {%s}, long chunkIndex = {%s}) using Thrift client", - statementId, chunkIndex); - LOGGER.debug(context); - DatabricksThreadContextHolder.setStatementId(statementId); - TFetchResultsResp fetchResultsResp; + // Thrift uses rowOffset with FETCH_ABSOLUTE; chunkIndex is used for link metadata + LOGGER.debug( + "getResultChunks(statementId={}, chunkIndex={}, rowOffset={}) using Thrift client", + statementId, + chunkIndex, + rowOffset); + + TFetchResultsResp fetchResultsResp = + thriftAccessor.fetchResultsWithAbsoluteOffset( + getOperationHandle(statementId), rowOffset, "getResultChunks"); + + boolean hasMoreRows = fetchResultsResp.hasMoreRows; + List resultLinks = fetchResultsResp.getResults().getResultLinks(); + + if (resultLinks == null || resultLinks.isEmpty()) { + LOGGER.debug( + "No result links returned for statement {}, hasMoreRows={}", statementId, hasMoreRows); + // For Thrift, hasMoreRows is the source of truth. Even with no links, + // if hasMoreRows is true, we should retry with the same offset. + return GetChunksResult.forThrift(Collections.emptyList(), hasMoreRows, rowOffset); + } + List externalLinks = new ArrayList<>(); - AtomicInteger index = new AtomicInteger(0); - do { - fetchResultsResp = thriftAccessor.getResultSetResp(getOperationHandle(statementId), context); - fetchResultsResp - .getResults() - .getResultLinks() - .forEach( - resultLink -> - externalLinks.add(createExternalLink(resultLink, index.getAndIncrement()))); - } while (fetchResultsResp.hasMoreRows); - if (chunkIndex < 0 || externalLinks.size() <= chunkIndex) { - String error = String.format("Out of bounds error for chunkIndex. Context: %s", context); - LOGGER.error(error); - throw new DatabricksSQLException(error, DatabricksDriverErrorCode.INVALID_STATE); + int lastIndex = resultLinks.size() - 1; + long nextRowOffset = rowOffset; // Will be updated based on links + + for (int i = 0; i < resultLinks.size(); i++) { + TSparkArrowResultLink thriftLink = resultLinks.get(i); + long linkChunkIndex = chunkIndex + i; + + ExternalLink externalLink = createExternalLink(thriftLink, linkChunkIndex); + + // Set nextChunkIndex based on position and hasMoreRows + if (i == lastIndex) { + // For the last link, only set nextChunkIndex if there are more rows + if (hasMoreRows) { + externalLink.setNextChunkIndex(linkChunkIndex + 1); + } + // If hasMoreRows is false, nextChunkIndex remains null (end of stream) + + // Calculate nextRowOffset from last link + nextRowOffset = thriftLink.getStartRowOffset() + thriftLink.getRowCount(); + } else { + // Not the last link - next chunk follows immediately + externalLink.setNextChunkIndex(linkChunkIndex + 1); + } + + externalLinks.add(externalLink); } - return externalLinks; + + LOGGER.debug( + "Fetched {} links for statement {}, hasMoreRows={}, nextRowOffset={}", + externalLinks.size(), + statementId, + hasMoreRows, + nextRowOffset); + + return GetChunksResult.forThrift(externalLinks, hasMoreRows, nextRowOffset); } @Override diff --git a/src/main/java/com/databricks/jdbc/model/core/GetChunksResult.java b/src/main/java/com/databricks/jdbc/model/core/GetChunksResult.java new file mode 100644 index 000000000..3c0557243 --- /dev/null +++ b/src/main/java/com/databricks/jdbc/model/core/GetChunksResult.java @@ -0,0 +1,109 @@ +package com.databricks.jdbc.model.core; + +import java.util.ArrayList; +import java.util.Collection; +import java.util.Collections; +import java.util.List; + +/** + * Result of fetching chunk links from the server. + * + *

This class encapsulates both the external links and continuation information needed to support + * both SEA and Thrift protocols: + * + *

    + *
  • SEA: Uses chunkIndex for continuation, hasMoreData derived from nextChunkIndex on last link + *
  • Thrift: Uses rowOffset for continuation, hasMoreData from server's hasMoreRows flag + *
+ */ +public class GetChunksResult { + + private final Collection externalLinks; + private final boolean hasMoreData; + private final long nextRowOffset; + + private GetChunksResult( + Collection externalLinks, boolean hasMoreData, long nextRowOffset) { + this.externalLinks = externalLinks != null ? externalLinks : Collections.emptyList(); + this.hasMoreData = hasMoreData; + this.nextRowOffset = nextRowOffset; + } + + /** + * Creates a result for SEA client responses. + * + *

For SEA, hasMoreData is derived from the last link's nextChunkIndex (null means no more + * data). nextRowOffset is calculated from the last link's rowOffset + rowCount for unified + * handling with Thrift. + * + * @param links The external links from the response + * @return GetChunksResult with hasMoreData and nextRowOffset derived from links + */ + public static GetChunksResult forSea(Collection links) { + if (links == null || links.isEmpty()) { + return new GetChunksResult(Collections.emptyList(), false, 0); + } + + // Get last link efficiently + List linkList = + links instanceof List ? (List) links : new ArrayList<>(links); + ExternalLink lastLink = linkList.get(linkList.size() - 1); + + boolean hasMore = lastLink.getNextChunkIndex() != null; + long nextRowOffset = lastLink.getRowOffset() + lastLink.getRowCount(); + + return new GetChunksResult(links, hasMore, nextRowOffset); + } + + /** + * Creates a result for Thrift client responses. + * + *

For Thrift, hasMoreData comes directly from the server's hasMoreRows flag. + * + * @param links The external links from the response (may be empty) + * @param hasMoreRows The hasMoreRows flag from the Thrift response + * @param nextRowOffset The row offset to use for the next fetch + * @return GetChunksResult with Thrift continuation info + */ + public static GetChunksResult forThrift( + Collection links, boolean hasMoreRows, long nextRowOffset) { + return new GetChunksResult(links, hasMoreRows, nextRowOffset); + } + + /** + * Creates an end-of-stream result (no more data available). + * + * @return GetChunksResult indicating end of stream + */ + public static GetChunksResult endOfStream() { + return new GetChunksResult(Collections.emptyList(), false, 0); + } + + /** + * @return The external links from the response, never null + */ + public Collection getExternalLinks() { + return externalLinks; + } + + /** + * @return true if there is more data available to fetch + */ + public boolean hasMoreData() { + return hasMoreData; + } + + /** + * @return The row offset to use for the next fetch. Populated for both SEA and Thrift. + */ + public long getNextRowOffset() { + return nextRowOffset; + } + + /** + * @return true if the result contains no links + */ + public boolean isEmpty() { + return externalLinks.isEmpty(); + } +} diff --git a/src/test/java/com/databricks/jdbc/api/impl/arrow/ArrowStreamResultTest.java b/src/test/java/com/databricks/jdbc/api/impl/arrow/ArrowStreamResultTest.java index 7bd55237c..cb9f4532e 100644 --- a/src/test/java/com/databricks/jdbc/api/impl/arrow/ArrowStreamResultTest.java +++ b/src/test/java/com/databricks/jdbc/api/impl/arrow/ArrowStreamResultTest.java @@ -3,8 +3,7 @@ import static com.databricks.jdbc.TestConstants.*; import static java.lang.Math.min; import static org.junit.jupiter.api.Assertions.*; -import static org.mockito.ArgumentMatchers.eq; -import static org.mockito.ArgumentMatchers.isA; +import static org.mockito.ArgumentMatchers.*; import static org.mockito.Mockito.when; import com.databricks.jdbc.api.impl.DatabricksConnectionContextFactory; @@ -24,6 +23,7 @@ import com.databricks.jdbc.model.core.ColumnInfo; import com.databricks.jdbc.model.core.ColumnInfoTypeName; import com.databricks.jdbc.model.core.ExternalLink; +import com.databricks.jdbc.model.core.GetChunksResult; import com.databricks.jdbc.model.core.ResultData; import com.databricks.jdbc.model.core.ResultManifest; import com.databricks.jdbc.model.core.ResultSchema; @@ -283,8 +283,8 @@ private void setupMockResponse() throws Exception { private void setupResultChunkMocks() throws DatabricksSQLException { for (int chunkIndex = 1; chunkIndex < numberOfChunks; chunkIndex++) { boolean isLastChunk = (chunkIndex == (numberOfChunks - 1)); - when(mockedSdkClient.getResultChunks(STATEMENT_ID, chunkIndex)) - .thenReturn(getChunkLinks(chunkIndex, isLastChunk)); + when(mockedSdkClient.getResultChunks(eq(STATEMENT_ID), eq((long) chunkIndex), anyLong())) + .thenReturn(GetChunksResult.forSea(getChunkLinks(chunkIndex, isLastChunk))); } } diff --git a/src/test/java/com/databricks/jdbc/api/impl/arrow/ChunkLinkDownloadServiceTest.java b/src/test/java/com/databricks/jdbc/api/impl/arrow/ChunkLinkDownloadServiceTest.java index 52981ced0..3ec64c866 100644 --- a/src/test/java/com/databricks/jdbc/api/impl/arrow/ChunkLinkDownloadServiceTest.java +++ b/src/test/java/com/databricks/jdbc/api/impl/arrow/ChunkLinkDownloadServiceTest.java @@ -12,6 +12,7 @@ import com.databricks.jdbc.exception.DatabricksSQLException; import com.databricks.jdbc.exception.DatabricksValidationException; import com.databricks.jdbc.model.core.ExternalLink; +import com.databricks.jdbc.model.core.GetChunksResult; import com.databricks.jdbc.model.telemetry.enums.DatabricksDriverErrorCode; import java.time.Instant; import java.time.temporal.ChronoUnit; @@ -55,14 +56,14 @@ void testGetLinkForChunk_Success() when(mockSession.getDatabricksClient()).thenReturn(mockClient); // Mock the response to link requests - when(mockClient.getResultChunks(eq(mockStatementId), eq(1L))) - .thenReturn(Collections.singletonList(linkForChunkIndex_1)); - when(mockClient.getResultChunks(eq(mockStatementId), eq(2L))) - .thenReturn(Collections.singletonList(linkForChunkIndex_2)); - when(mockClient.getResultChunks(eq(mockStatementId), eq(3L))) - .thenReturn(Collections.singletonList(linkForChunkIndex_3)); - when(mockClient.getResultChunks(eq(mockStatementId), eq(4L))) - .thenReturn(Collections.singletonList(linkForChunkIndex_4)); + when(mockClient.getResultChunks(eq(mockStatementId), eq(1L), eq(0L))) + .thenReturn(GetChunksResult.forSea(Collections.singletonList(linkForChunkIndex_1))); + when(mockClient.getResultChunks(eq(mockStatementId), eq(2L), eq(0L))) + .thenReturn(GetChunksResult.forSea(Collections.singletonList(linkForChunkIndex_2))); + when(mockClient.getResultChunks(eq(mockStatementId), eq(3L), eq(0L))) + .thenReturn(GetChunksResult.forSea(Collections.singletonList(linkForChunkIndex_3))); + when(mockClient.getResultChunks(eq(mockStatementId), eq(4L), eq(0L))) + .thenReturn(GetChunksResult.forSea(Collections.singletonList(linkForChunkIndex_4))); long chunkIndex = 1L; when(mockChunkMap.get(chunkIndex)).thenReturn(mock(ArrowResultChunk.class)); @@ -78,7 +79,7 @@ void testGetLinkForChunk_Success() TimeUnit.MILLISECONDS.sleep(500); assertEquals(linkForChunkIndex_1, result); - verify(mockClient).getResultChunks(mockStatementId, NEXT_BATCH_START_INDEX); + verify(mockClient).getResultChunks(mockStatementId, NEXT_BATCH_START_INDEX, 0L); } @Test @@ -115,7 +116,8 @@ void testGetLinkForChunk_ClientError() new DatabricksSQLException("Test error", DatabricksDriverErrorCode.INVALID_STATE); when(mockSession.getDatabricksClient()).thenReturn(mockClient); // Mock an error in response to the link request - when(mockClient.getResultChunks(eq(mockStatementId), anyLong())).thenThrow(expectedError); + when(mockClient.getResultChunks(eq(mockStatementId), anyLong(), anyLong())) + .thenThrow(expectedError); when(mockChunkMap.get(chunkIndex)).thenReturn(mock(ArrowResultChunk.class)); ChunkLinkDownloadService service = @@ -133,14 +135,14 @@ void testGetLinkForChunk_ClientError() void testAutoTriggerForSEAClient() throws DatabricksSQLException, InterruptedException { when(mockSession.getDatabricksClient()).thenReturn(mockClient); // Mock the response to link requests - when(mockClient.getResultChunks(eq(mockStatementId), eq(1L))) - .thenReturn(Collections.singletonList(linkForChunkIndex_1)); - when(mockClient.getResultChunks(eq(mockStatementId), eq(2L))) - .thenReturn(Collections.singletonList(linkForChunkIndex_2)); - when(mockClient.getResultChunks(eq(mockStatementId), eq(3L))) - .thenReturn(Collections.singletonList(linkForChunkIndex_3)); - when(mockClient.getResultChunks(eq(mockStatementId), eq(4L))) - .thenReturn(Collections.singletonList(linkForChunkIndex_4)); + when(mockClient.getResultChunks(eq(mockStatementId), eq(1L), eq(0L))) + .thenReturn(GetChunksResult.forSea(Collections.singletonList(linkForChunkIndex_1))); + when(mockClient.getResultChunks(eq(mockStatementId), eq(2L), eq(0L))) + .thenReturn(GetChunksResult.forSea(Collections.singletonList(linkForChunkIndex_2))); + when(mockClient.getResultChunks(eq(mockStatementId), eq(3L), eq(0L))) + .thenReturn(GetChunksResult.forSea(Collections.singletonList(linkForChunkIndex_3))); + when(mockClient.getResultChunks(eq(mockStatementId), eq(4L), eq(0L))) + .thenReturn(GetChunksResult.forSea(Collections.singletonList(linkForChunkIndex_4))); // Download chain will be triggered immediately in the constructor when(mockSession.getConnectionContext().getClientType()).thenReturn(DatabricksClientType.SEA); @@ -150,7 +152,7 @@ void testAutoTriggerForSEAClient() throws DatabricksSQLException, InterruptedExc // Sleep to allow the service to complete the download pipeline TimeUnit.MILLISECONDS.sleep(500); - verify(mockClient).getResultChunks(mockStatementId, NEXT_BATCH_START_INDEX); + verify(mockClient).getResultChunks(mockStatementId, NEXT_BATCH_START_INDEX, 0L); } @Test @@ -163,14 +165,14 @@ void testHandleExpiredLinks() when(mockSession.getDatabricksClient()).thenReturn(mockClient); // Mock the response to link requests. Return the expired link for chunk index 1 - when(mockClient.getResultChunks(eq(mockStatementId), eq(1L))) - .thenReturn(Collections.singletonList(expiredLinkForChunkIndex_1)); - when(mockClient.getResultChunks(eq(mockStatementId), eq(2L))) - .thenReturn(Collections.singletonList(linkForChunkIndex_2)); - when(mockClient.getResultChunks(eq(mockStatementId), eq(3L))) - .thenReturn(Collections.singletonList(linkForChunkIndex_3)); - when(mockClient.getResultChunks(eq(mockStatementId), eq(4L))) - .thenReturn(Collections.singletonList(linkForChunkIndex_4)); + when(mockClient.getResultChunks(eq(mockStatementId), eq(1L), anyLong())) + .thenReturn(GetChunksResult.forSea(Collections.singletonList(expiredLinkForChunkIndex_1))); + when(mockClient.getResultChunks(eq(mockStatementId), eq(2L), anyLong())) + .thenReturn(GetChunksResult.forSea(Collections.singletonList(linkForChunkIndex_2))); + when(mockClient.getResultChunks(eq(mockStatementId), eq(3L), anyLong())) + .thenReturn(GetChunksResult.forSea(Collections.singletonList(linkForChunkIndex_3))); + when(mockClient.getResultChunks(eq(mockStatementId), eq(4L), anyLong())) + .thenReturn(GetChunksResult.forSea(Collections.singletonList(linkForChunkIndex_4))); long chunkIndex = 1L; ArrowResultChunk mockChunk = mock(ArrowResultChunk.class); @@ -185,8 +187,8 @@ void testHandleExpiredLinks() TimeUnit.MILLISECONDS.sleep(500); // Mock a new valid link for chunk index 1 - when(mockClient.getResultChunks(eq(mockStatementId), eq(1L))) - .thenReturn(Collections.singletonList(linkForChunkIndex_1)); + when(mockClient.getResultChunks(eq(mockStatementId), eq(1L), eq(0L))) + .thenReturn(GetChunksResult.forSea(Collections.singletonList(linkForChunkIndex_1))); // Try to get the link for chunk index 1. Download chain will be re-triggered because the link // is expired CompletableFuture future = service.getLinkForChunk(chunkIndex); @@ -195,7 +197,7 @@ void testHandleExpiredLinks() TimeUnit.MILLISECONDS.sleep(500); assertEquals(linkForChunkIndex_1, result); - verify(mockClient, times(2)).getResultChunks(mockStatementId, chunkIndex); + verify(mockClient, times(2)).getResultChunks(mockStatementId, chunkIndex, 0L); } @Test @@ -222,14 +224,17 @@ void testBatchDownloadChaining() when(mockSession.getDatabricksClient()).thenReturn(mockClient); // Mock the links for the first batch. The link futures for both chunks will be completed at the // same time - when(mockClient.getResultChunks(eq(mockStatementId), eq(1L))) - .thenReturn(Arrays.asList(linkForChunkIndex_1, linkForChunkIndex_2)); + when(mockClient.getResultChunks(eq(mockStatementId), eq(1L), eq(0L))) + .thenReturn( + GetChunksResult.forSea(Arrays.asList(linkForChunkIndex_1, linkForChunkIndex_2))); // Mock the links for the second batch. - when(mockClient.getResultChunks(eq(mockStatementId), eq(3L))) - .thenReturn(Arrays.asList(linkForChunkIndex_3, linkForChunkIndex_4)); + when(mockClient.getResultChunks(eq(mockStatementId), eq(3L), eq(0L))) + .thenReturn( + GetChunksResult.forSea(Arrays.asList(linkForChunkIndex_3, linkForChunkIndex_4))); // Mock the links for the third batch. - when(mockClient.getResultChunks(eq(mockStatementId), eq(5L))) - .thenReturn(Arrays.asList(linkForChunkIndex_5, linkForChunkIndex_6)); + when(mockClient.getResultChunks(eq(mockStatementId), eq(5L), eq(0L))) + .thenReturn( + GetChunksResult.forSea(Arrays.asList(linkForChunkIndex_5, linkForChunkIndex_6))); ChunkLinkDownloadService service = new ChunkLinkDownloadService<>( @@ -260,11 +265,11 @@ void testBatchDownloadChaining() assertEquals(linkForChunkIndex_5, result5); assertEquals(linkForChunkIndex_6, result6); // Verify the request for first batch - verify(mockClient, times(1)).getResultChunks(mockStatementId, 1L); + verify(mockClient, times(1)).getResultChunks(mockStatementId, 1L, 0L); // Verify the request for second batch - verify(mockClient, times(1)).getResultChunks(mockStatementId, 3L); + verify(mockClient, times(1)).getResultChunks(mockStatementId, 3L, 0L); // Verify the request for third batch - verify(mockClient, times(1)).getResultChunks(mockStatementId, 5L); + verify(mockClient, times(1)).getResultChunks(mockStatementId, 5L, 0L); } private ExternalLink createExternalLink( diff --git a/src/test/java/com/databricks/jdbc/dbclient/impl/thrift/DatabricksThriftServiceClientTest.java b/src/test/java/com/databricks/jdbc/dbclient/impl/thrift/DatabricksThriftServiceClientTest.java index 27f4c04e5..09bd0c738 100644 --- a/src/test/java/com/databricks/jdbc/dbclient/impl/thrift/DatabricksThriftServiceClientTest.java +++ b/src/test/java/com/databricks/jdbc/dbclient/impl/thrift/DatabricksThriftServiceClientTest.java @@ -10,6 +10,7 @@ import static com.databricks.jdbc.model.core.ColumnInfoTypeName.*; import static org.junit.jupiter.api.Assertions.*; import static org.mockito.ArgumentMatchers.any; +import static org.mockito.ArgumentMatchers.anyLong; import static org.mockito.ArgumentMatchers.eq; import static org.mockito.Mockito.verify; import static org.mockito.Mockito.when; @@ -369,17 +370,20 @@ void testGetResultChunks() throws SQLException { .setStatus(new TStatus().setStatusCode(TStatusCode.SUCCESS_STATUS)) .setResults(resultData) .setResultSetMetadata(resultMetadataData); - when(thriftAccessor.getResultSetResp(any(), any())).thenReturn(response); + when(thriftAccessor.fetchResultsWithAbsoluteOffset(any(), anyLong(), any())) + .thenReturn(response); when(resultData.getResultLinks()) .thenReturn( Collections.singletonList(new TSparkArrowResultLink().setFileLink(TEST_STRING))); - Collection resultChunks = client.getResultChunks(TEST_STMT_ID, 0); + // Pass chunkIndex=0 and rowOffset=0 for the first chunk + Collection resultChunks = + client.getResultChunks(TEST_STMT_ID, 0, 0).getExternalLinks(); assertEquals(resultChunks.size(), 1); assertEquals(resultChunks.stream().findFirst().get().getExternalLink(), TEST_STRING); } @Test - void testGetResultChunksThrowsError() throws SQLException { + void testGetResultChunksReturnsEmptyWhenNoLinks() throws SQLException { DatabricksThriftServiceClient client = new DatabricksThriftServiceClient(thriftAccessor, connectionContext); TFetchResultsResp response = @@ -387,10 +391,12 @@ void testGetResultChunksThrowsError() throws SQLException { .setStatus(new TStatus().setStatusCode(TStatusCode.SUCCESS_STATUS)) .setResults(resultData) .setResultSetMetadata(resultMetadataData); - when(thriftAccessor.getResultSetResp(any(), any())).thenReturn(response); - assertThrows(DatabricksSQLException.class, () -> client.getResultChunks(TEST_STMT_ID, -1)); - assertThrows(DatabricksSQLException.class, () -> client.getResultChunks(TEST_STMT_ID, 2)); - assertThrows(DatabricksSQLException.class, () -> client.getResultChunks(TEST_STMT_ID, 1)); + when(thriftAccessor.fetchResultsWithAbsoluteOffset(any(), anyLong(), any())) + .thenReturn(response); + when(resultData.getResultLinks()).thenReturn(null); + Collection resultChunks = + client.getResultChunks(TEST_STMT_ID, 0, 0).getExternalLinks(); + assertEquals(0, resultChunks.size()); } @Test From d23eed4c610f2c173c8186b9147c09dd3485df1d Mon Sep 17 00:00:00 2001 From: Jayant Singh Date: Thu, 27 Nov 2025 22:12:03 +0000 Subject: [PATCH 4/6] fmt --- .../api/impl/arrow/ArrowStreamResult.java | 1 + .../impl/arrow/ChunkLinkDownloadService.java | 23 ++-- .../jdbc/api/impl/arrow/ChunkLinkFetcher.java | 1 + .../api/impl/arrow/SeaChunkLinkFetcher.java | 53 ++------- .../impl/arrow/StreamingChunkProvider.java | 1 + .../impl/arrow/ThriftChunkLinkFetcher.java | 73 +++--------- .../jdbc/dbclient/IDatabricksClient.java | 14 +-- .../impl/sqlexec/DatabricksSdkClient.java | 44 ++++++- .../thrift/DatabricksThriftServiceClient.java | 36 +++--- .../core}/ChunkLinkFetchResult.java | 28 ++--- .../jdbc/model/core/GetChunksResult.java | 109 ------------------ .../api/impl/arrow/ArrowStreamResultTest.java | 30 ++++- .../arrow/ChunkLinkDownloadServiceTest.java | 65 ++++++++--- .../DatabricksThriftServiceClientTest.java | 16 +-- 14 files changed, 200 insertions(+), 294 deletions(-) rename src/main/java/com/databricks/jdbc/{api/impl/arrow => model/core}/ChunkLinkFetchResult.java (80%) delete mode 100644 src/main/java/com/databricks/jdbc/model/core/GetChunksResult.java diff --git a/src/main/java/com/databricks/jdbc/api/impl/arrow/ArrowStreamResult.java b/src/main/java/com/databricks/jdbc/api/impl/arrow/ArrowStreamResult.java index 06b5d7ec5..21464923c 100644 --- a/src/main/java/com/databricks/jdbc/api/impl/arrow/ArrowStreamResult.java +++ b/src/main/java/com/databricks/jdbc/api/impl/arrow/ArrowStreamResult.java @@ -19,6 +19,7 @@ import com.databricks.jdbc.model.client.thrift.generated.TFetchResultsResp; import com.databricks.jdbc.model.client.thrift.generated.TGetResultSetMetadataResp; import com.databricks.jdbc.model.client.thrift.generated.TSparkArrowResultLink; +import com.databricks.jdbc.model.core.ChunkLinkFetchResult; import com.databricks.jdbc.model.core.ColumnInfo; import com.databricks.jdbc.model.core.ColumnInfoTypeName; import com.databricks.jdbc.model.core.ExternalLink; diff --git a/src/main/java/com/databricks/jdbc/api/impl/arrow/ChunkLinkDownloadService.java b/src/main/java/com/databricks/jdbc/api/impl/arrow/ChunkLinkDownloadService.java index bd824152e..d521e4d46 100644 --- a/src/main/java/com/databricks/jdbc/api/impl/arrow/ChunkLinkDownloadService.java +++ b/src/main/java/com/databricks/jdbc/api/impl/arrow/ChunkLinkDownloadService.java @@ -9,10 +9,9 @@ import com.databricks.jdbc.exception.DatabricksValidationException; import com.databricks.jdbc.log.JdbcLogger; import com.databricks.jdbc.log.JdbcLoggerFactory; +import com.databricks.jdbc.model.core.ChunkLinkFetchResult; import com.databricks.jdbc.model.core.ExternalLink; -import com.databricks.jdbc.model.core.GetChunksResult; import java.time.Instant; -import java.util.Collection; import java.util.Map; import java.util.concurrent.CompletableFuture; import java.util.concurrent.ConcurrentHashMap; @@ -222,32 +221,34 @@ private void triggerNextBatchDownload() { try { // rowOffset is 0 here as this service is used by RemoteChunkProvider (SEA-only) // which fetches by chunkIndex, not rowOffset - GetChunksResult result = + ChunkLinkFetchResult result = session.getDatabricksClient().getResultChunks(statementId, batchStartIndex, 0); - Collection links = result.getExternalLinks(); LOGGER.info( "Retrieved {} links for batch starting at {} for statement id {}", - links.size(), + result.getChunkLinks().size(), batchStartIndex, statementId); // Complete futures for all chunks in this batch - for (ExternalLink link : links) { + for (ChunkLinkFetchResult.ChunkLinkInfo linkInfo : result.getChunkLinks()) { CompletableFuture future = - chunkIndexToLinkFuture.get(link.getChunkIndex()); + chunkIndexToLinkFuture.get(linkInfo.getChunkIndex()); if (future != null) { LOGGER.debug( "Completing future for chunk {} for statement id {}", - link.getChunkIndex(), + linkInfo.getChunkIndex(), statementId); - future.complete(link); + future.complete(linkInfo.getLink()); } } // Update next batch start index and trigger next batch - if (!links.isEmpty()) { + if (!result.getChunkLinks().isEmpty()) { long maxChunkIndex = - links.stream().mapToLong(ExternalLink::getChunkIndex).max().getAsLong(); + result.getChunkLinks().stream() + .mapToLong(ChunkLinkFetchResult.ChunkLinkInfo::getChunkIndex) + .max() + .getAsLong(); nextBatchStartIndex.set(maxChunkIndex + 1); LOGGER.debug("Updated next batch start index to {}", maxChunkIndex + 1); diff --git a/src/main/java/com/databricks/jdbc/api/impl/arrow/ChunkLinkFetcher.java b/src/main/java/com/databricks/jdbc/api/impl/arrow/ChunkLinkFetcher.java index f5892e867..6d6a88e56 100644 --- a/src/main/java/com/databricks/jdbc/api/impl/arrow/ChunkLinkFetcher.java +++ b/src/main/java/com/databricks/jdbc/api/impl/arrow/ChunkLinkFetcher.java @@ -1,6 +1,7 @@ package com.databricks.jdbc.api.impl.arrow; import com.databricks.jdbc.exception.DatabricksSQLException; +import com.databricks.jdbc.model.core.ChunkLinkFetchResult; import com.databricks.jdbc.model.core.ExternalLink; /** diff --git a/src/main/java/com/databricks/jdbc/api/impl/arrow/SeaChunkLinkFetcher.java b/src/main/java/com/databricks/jdbc/api/impl/arrow/SeaChunkLinkFetcher.java index 4b0843454..28482118f 100644 --- a/src/main/java/com/databricks/jdbc/api/impl/arrow/SeaChunkLinkFetcher.java +++ b/src/main/java/com/databricks/jdbc/api/impl/arrow/SeaChunkLinkFetcher.java @@ -5,12 +5,9 @@ import com.databricks.jdbc.exception.DatabricksSQLException; import com.databricks.jdbc.log.JdbcLogger; import com.databricks.jdbc.log.JdbcLoggerFactory; +import com.databricks.jdbc.model.core.ChunkLinkFetchResult; import com.databricks.jdbc.model.core.ExternalLink; -import com.databricks.jdbc.model.core.GetChunksResult; import com.databricks.jdbc.model.telemetry.enums.DatabricksDriverErrorCode; -import java.util.ArrayList; -import java.util.Collection; -import java.util.List; /** * ChunkLinkFetcher implementation for the SQL Execution API (SEA) client. @@ -40,40 +37,9 @@ public ChunkLinkFetchResult fetchLinks(long startChunkIndex, long startRowOffset startChunkIndex, statementId); - GetChunksResult result = - session.getDatabricksClient().getResultChunks(statementId, startChunkIndex, startRowOffset); - Collection links = result.getExternalLinks(); - - if (links == null || links.isEmpty()) { - LOGGER.debug("No links returned, end of stream reached for statement {}", statementId); - return ChunkLinkFetchResult.endOfStream(); - } - - List chunkLinks = new ArrayList<>(); - Long nextIndex = null; - - for (ExternalLink link : links) { - chunkLinks.add( - new ChunkLinkFetchResult.ChunkLinkInfo( - link.getChunkIndex(), link, link.getRowCount(), link.getRowOffset())); - - // Track last link's nextChunkIndex for logging - nextIndex = link.getNextChunkIndex(); - } - - // Use hasMoreData and nextRowOffset from GetChunksResult (unified with Thrift) - boolean hasMore = result.hasMoreData(); - long nextRowOffset = result.getNextRowOffset(); - - LOGGER.debug( - "Fetched {} links for statement {}, hasMore={}, nextIndex={}, nextRowOffset={}", - chunkLinks.size(), - statementId, - hasMore, - nextIndex, - nextRowOffset); - - return ChunkLinkFetchResult.of(chunkLinks, hasMore, hasMore ? nextIndex : -1, nextRowOffset); + return session + .getDatabricksClient() + .getResultChunks(statementId, startChunkIndex, startRowOffset); } @Override @@ -81,22 +47,21 @@ public ExternalLink refetchLink(long chunkIndex, long rowOffset) throws Databric // SEA uses chunkIndex; rowOffset is ignored LOGGER.info("Refetching expired link for chunk {} of statement {}", chunkIndex, statementId); - GetChunksResult result = + ChunkLinkFetchResult result = session.getDatabricksClient().getResultChunks(statementId, chunkIndex, rowOffset); - Collection links = result.getExternalLinks(); - if (links == null || links.isEmpty()) { + if (result.isEndOfStream()) { throw new DatabricksSQLException( String.format("Failed to refetch link for chunk %d: no links returned", chunkIndex), DatabricksDriverErrorCode.CHUNK_READY_ERROR); } // Find the link for the requested chunk index - for (ExternalLink link : links) { - if (link.getChunkIndex() != null && link.getChunkIndex() == chunkIndex) { + for (ChunkLinkFetchResult.ChunkLinkInfo linkInfo : result.getChunkLinks()) { + if (linkInfo.getChunkIndex() == chunkIndex) { LOGGER.debug( "Successfully refetched link for chunk {} of statement {}", chunkIndex, statementId); - return link; + return linkInfo.getLink(); } } diff --git a/src/main/java/com/databricks/jdbc/api/impl/arrow/StreamingChunkProvider.java b/src/main/java/com/databricks/jdbc/api/impl/arrow/StreamingChunkProvider.java index 719f3f104..d0f87db5a 100644 --- a/src/main/java/com/databricks/jdbc/api/impl/arrow/StreamingChunkProvider.java +++ b/src/main/java/com/databricks/jdbc/api/impl/arrow/StreamingChunkProvider.java @@ -7,6 +7,7 @@ import com.databricks.jdbc.exception.DatabricksSQLException; import com.databricks.jdbc.log.JdbcLogger; import com.databricks.jdbc.log.JdbcLoggerFactory; +import com.databricks.jdbc.model.core.ChunkLinkFetchResult; import com.databricks.jdbc.model.telemetry.enums.DatabricksDriverErrorCode; import java.util.concurrent.ConcurrentHashMap; import java.util.concurrent.ConcurrentMap; diff --git a/src/main/java/com/databricks/jdbc/api/impl/arrow/ThriftChunkLinkFetcher.java b/src/main/java/com/databricks/jdbc/api/impl/arrow/ThriftChunkLinkFetcher.java index bd126b6b2..109e31b86 100644 --- a/src/main/java/com/databricks/jdbc/api/impl/arrow/ThriftChunkLinkFetcher.java +++ b/src/main/java/com/databricks/jdbc/api/impl/arrow/ThriftChunkLinkFetcher.java @@ -5,12 +5,9 @@ import com.databricks.jdbc.exception.DatabricksSQLException; import com.databricks.jdbc.log.JdbcLogger; import com.databricks.jdbc.log.JdbcLoggerFactory; +import com.databricks.jdbc.model.core.ChunkLinkFetchResult; import com.databricks.jdbc.model.core.ExternalLink; -import com.databricks.jdbc.model.core.GetChunksResult; import com.databricks.jdbc.model.telemetry.enums.DatabricksDriverErrorCode; -import java.util.ArrayList; -import java.util.Collection; -import java.util.List; /** * ChunkLinkFetcher implementation for the Thrift client. @@ -43,50 +40,9 @@ public ChunkLinkFetchResult fetchLinks(long startChunkIndex, long startRowOffset startRowOffset, statementId); - GetChunksResult result = - session.getDatabricksClient().getResultChunks(statementId, startChunkIndex, startRowOffset); - - Collection links = result.getExternalLinks(); - boolean hasMore = result.hasMoreData(); - long nextRowOffset = result.getNextRowOffset(); - - if (links == null || links.isEmpty()) { - // For Thrift, hasMoreData() is the source of truth. Even with no links, - // if hasMore is true, we should indicate continuation with the same offset. - if (hasMore) { - LOGGER.debug( - "No links returned but hasMoreData=true for statement {}. " - + "Returning empty result with hasMore=true for retry with offset {}", - statementId, - nextRowOffset); - return ChunkLinkFetchResult.of(new ArrayList<>(), true, startChunkIndex, nextRowOffset); - } - LOGGER.debug("No links returned, end of stream reached for statement {}", statementId); - return ChunkLinkFetchResult.endOfStream(); - } - - List chunkLinks = new ArrayList<>(); - Long nextIndex = null; - - for (ExternalLink link : links) { - chunkLinks.add( - new ChunkLinkFetchResult.ChunkLinkInfo( - link.getChunkIndex(), link, link.getRowCount(), link.getRowOffset())); - - // Track the last link's nextChunkIndex for logging/metadata - nextIndex = link.getNextChunkIndex(); - } - - LOGGER.debug( - "Fetched {} links for statement {}, hasMore={}, nextIndex={}, nextRowOffset={}", - chunkLinks.size(), - statementId, - hasMore, - nextIndex, - nextRowOffset); - - // For Thrift, hasMore comes from GetChunksResult.hasMoreData() (the server's hasMoreRows flag) - return ChunkLinkFetchResult.of(chunkLinks, hasMore, hasMore ? nextIndex : -1, nextRowOffset); + return session + .getDatabricksClient() + .getResultChunks(statementId, startChunkIndex, startRowOffset); } @Override @@ -98,24 +54,23 @@ public ExternalLink refetchLink(long chunkIndex, long rowOffset) throws Databric rowOffset, statementId); - // For Thrift, we may need to retry if hasMoreData=true but no links returned yet + // For Thrift, we may need to retry if hasMore=true but no links returned yet int maxRetries = 100; // Reasonable limit to prevent infinite loops int retryCount = 0; while (retryCount < maxRetries) { - GetChunksResult result = + ChunkLinkFetchResult result = session.getDatabricksClient().getResultChunks(statementId, chunkIndex, rowOffset); - Collection links = result.getExternalLinks(); - if (links != null && !links.isEmpty()) { + if (!result.getChunkLinks().isEmpty()) { // Find the link for the requested chunk index - for (ExternalLink link : links) { - if (link.getChunkIndex() != null && link.getChunkIndex() == chunkIndex) { + for (ChunkLinkFetchResult.ChunkLinkInfo linkInfo : result.getChunkLinks()) { + if (linkInfo.getChunkIndex() == chunkIndex) { LOGGER.debug( "Successfully refetched link for chunk {} of statement {}", chunkIndex, statementId); - return link; + return linkInfo.getLink(); } } @@ -128,19 +83,19 @@ public ExternalLink refetchLink(long chunkIndex, long rowOffset) throws Databric } // No links returned - check if we should retry - if (!result.hasMoreData()) { + if (!result.hasMore()) { // No more data and no links - this is unexpected for a refetch throw new DatabricksSQLException( String.format( - "Failed to refetch link for chunk %d: no links returned and hasMoreData=false", + "Failed to refetch link for chunk %d: no links returned and hasMore=false", chunkIndex), DatabricksDriverErrorCode.CHUNK_READY_ERROR); } - // hasMoreData=true but no links yet - retry + // hasMore=true but no links yet - retry retryCount++; LOGGER.debug( - "No links returned for chunk {} but hasMoreData=true, retrying ({}/{})", + "No links returned for chunk {} but hasMore=true, retrying ({}/{})", chunkIndex, retryCount, maxRetries); diff --git a/src/main/java/com/databricks/jdbc/dbclient/IDatabricksClient.java b/src/main/java/com/databricks/jdbc/dbclient/IDatabricksClient.java index 1acb54c39..612b10475 100644 --- a/src/main/java/com/databricks/jdbc/dbclient/IDatabricksClient.java +++ b/src/main/java/com/databricks/jdbc/dbclient/IDatabricksClient.java @@ -9,7 +9,7 @@ import com.databricks.jdbc.dbclient.impl.common.StatementId; import com.databricks.jdbc.exception.DatabricksSQLException; import com.databricks.jdbc.model.client.thrift.generated.TFetchResultsResp; -import com.databricks.jdbc.model.core.GetChunksResult; +import com.databricks.jdbc.model.core.ChunkLinkFetchResult; import com.databricks.jdbc.model.core.ResultData; import com.databricks.jdbc.telemetry.latency.DatabricksMetricsTimed; import com.databricks.sdk.core.DatabricksConfig; @@ -114,25 +114,25 @@ DatabricksResultSet getStatementResult( throws SQLException; /** - * Fetches the chunk details for given chunk index and statement-Id. + * Fetches the chunk links for given chunk index and statement-Id. * *

For SEA clients, the chunkIndex is used to identify which chunk to fetch. For Thrift * clients, the rowOffset is used with FETCH_ABSOLUTE orientation to seek to the correct position. * - *

The returned {@link GetChunksResult} contains the external links and continuation + *

The returned {@link ChunkLinkFetchResult} contains the chunk links and continuation * information: * *

    - *
  • SEA: hasMoreData derived from last link's nextChunkIndex - *
  • Thrift: hasMoreData from server's hasMoreRows flag, nextRowOffset for continuation + *
  • SEA: hasMore derived from last link's nextChunkIndex + *
  • Thrift: hasMore from server's hasMoreRows flag, nextRowOffset for continuation *
* * @param statementId statement-Id for which chunk should be fetched * @param chunkIndex chunkIndex for which chunk should be fetched (used by SEA) * @param rowOffset row offset for fetching results (used by Thrift with FETCH_ABSOLUTE) - * @return GetChunksResult containing links and continuation information + * @return ChunkLinkFetchResult containing links and continuation information */ - GetChunksResult getResultChunks(StatementId statementId, long chunkIndex, long rowOffset) + ChunkLinkFetchResult getResultChunks(StatementId statementId, long chunkIndex, long rowOffset) throws DatabricksSQLException; /** diff --git a/src/main/java/com/databricks/jdbc/dbclient/impl/sqlexec/DatabricksSdkClient.java b/src/main/java/com/databricks/jdbc/dbclient/impl/sqlexec/DatabricksSdkClient.java index cc09f93f0..7c627dacb 100644 --- a/src/main/java/com/databricks/jdbc/dbclient/impl/sqlexec/DatabricksSdkClient.java +++ b/src/main/java/com/databricks/jdbc/dbclient/impl/sqlexec/DatabricksSdkClient.java @@ -29,8 +29,9 @@ import com.databricks.jdbc.model.client.sqlexec.ExecuteStatementResponse; import com.databricks.jdbc.model.client.sqlexec.GetStatementResponse; import com.databricks.jdbc.model.client.thrift.generated.TFetchResultsResp; +import com.databricks.jdbc.model.core.ChunkLinkFetchResult; import com.databricks.jdbc.model.core.Disposition; -import com.databricks.jdbc.model.core.GetChunksResult; +import com.databricks.jdbc.model.core.ExternalLink; import com.databricks.jdbc.model.core.ResultData; import com.databricks.jdbc.model.telemetry.enums.DatabricksDriverErrorCode; import com.databricks.sdk.WorkspaceClient; @@ -409,7 +410,7 @@ public void cancelStatement(StatementId typedStatementId) throws DatabricksSQLEx } @Override - public GetChunksResult getResultChunks( + public ChunkLinkFetchResult getResultChunks( StatementId typedStatementId, long chunkIndex, long rowOffset) throws DatabricksSQLException { // SEA uses chunkIndex; rowOffset is ignored String statementId = typedStatementId.toSQLExecStatementId(); @@ -422,7 +423,7 @@ public GetChunksResult getResultChunks( Request req = new Request(Request.GET, path, apiClient.serialize(request)); req.withHeaders(getHeaders("getStatementResultN")); ResultData resultData = apiClient.execute(req, ResultData.class); - return GetChunksResult.forSea(resultData.getExternalLinks()); + return buildChunkLinkFetchResult(resultData.getExternalLinks()); } catch (IOException e) { String errorMessage = "Error while processing the get result chunk request"; LOGGER.error(errorMessage, e); @@ -430,6 +431,43 @@ public GetChunksResult getResultChunks( } } + /** + * Builds a ChunkLinkFetchResult from SEA external links. + * + * @param links The external links from the SEA response + * @return ChunkLinkFetchResult with links and continuation info + */ + private ChunkLinkFetchResult buildChunkLinkFetchResult(Collection links) { + if (links == null || links.isEmpty()) { + return ChunkLinkFetchResult.endOfStream(); + } + + List linkList = + links instanceof List ? (List) links : new ArrayList<>(links); + + List chunkLinks = new ArrayList<>(); + for (ExternalLink link : linkList) { + chunkLinks.add( + new ChunkLinkFetchResult.ChunkLinkInfo( + link.getChunkIndex(), link, link.getRowCount(), link.getRowOffset())); + } + + // Derive continuation info from last link + ExternalLink lastLink = linkList.get(linkList.size() - 1); + boolean hasMore = lastLink.getNextChunkIndex() != null; + long nextFetchIndex = hasMore ? lastLink.getNextChunkIndex() : -1; + long nextRowOffset = lastLink.getRowOffset() + lastLink.getRowCount(); + + LOGGER.debug( + "Built ChunkLinkFetchResult with {} links, hasMore={}, nextFetchIndex={}, nextRowOffset={}", + chunkLinks.size(), + hasMore, + nextFetchIndex, + nextRowOffset); + + return ChunkLinkFetchResult.of(chunkLinks, hasMore, nextFetchIndex, nextRowOffset); + } + @Override public ResultData getResultChunksData(StatementId typedStatementId, long chunkIndex) throws DatabricksSQLException { diff --git a/src/main/java/com/databricks/jdbc/dbclient/impl/thrift/DatabricksThriftServiceClient.java b/src/main/java/com/databricks/jdbc/dbclient/impl/thrift/DatabricksThriftServiceClient.java index f0580171b..f53a1087f 100644 --- a/src/main/java/com/databricks/jdbc/dbclient/impl/thrift/DatabricksThriftServiceClient.java +++ b/src/main/java/com/databricks/jdbc/dbclient/impl/thrift/DatabricksThriftServiceClient.java @@ -28,8 +28,8 @@ import com.databricks.jdbc.log.JdbcLogger; import com.databricks.jdbc.log.JdbcLoggerFactory; import com.databricks.jdbc.model.client.thrift.generated.*; +import com.databricks.jdbc.model.core.ChunkLinkFetchResult; import com.databricks.jdbc.model.core.ExternalLink; -import com.databricks.jdbc.model.core.GetChunksResult; import com.databricks.jdbc.model.core.ResultData; import com.databricks.jdbc.model.telemetry.enums.DatabricksDriverErrorCode; import com.databricks.sdk.core.DatabricksConfig; @@ -300,8 +300,8 @@ public DatabricksResultSet getStatementResult( } @Override - public GetChunksResult getResultChunks(StatementId statementId, long chunkIndex, long rowOffset) - throws DatabricksSQLException { + public ChunkLinkFetchResult getResultChunks( + StatementId statementId, long chunkIndex, long rowOffset) throws DatabricksSQLException { // Thrift uses rowOffset with FETCH_ABSOLUTE; chunkIndex is used for link metadata LOGGER.debug( "getResultChunks(statementId={}, chunkIndex={}, rowOffset={}) using Thrift client", @@ -320,13 +320,14 @@ public GetChunksResult getResultChunks(StatementId statementId, long chunkIndex, LOGGER.debug( "No result links returned for statement {}, hasMoreRows={}", statementId, hasMoreRows); // For Thrift, hasMoreRows is the source of truth. Even with no links, - // if hasMoreRows is true, we should retry with the same offset. - return GetChunksResult.forThrift(Collections.emptyList(), hasMoreRows, rowOffset); + // if hasMoreRows is true, we should indicate continuation with the same offset. + return ChunkLinkFetchResult.of(new ArrayList<>(), hasMoreRows, chunkIndex, rowOffset); } - List externalLinks = new ArrayList<>(); + List chunkLinks = new ArrayList<>(); int lastIndex = resultLinks.size() - 1; - long nextRowOffset = rowOffset; // Will be updated based on links + long nextRowOffset = rowOffset; + long nextFetchIndex = chunkIndex; for (int i = 0; i < resultLinks.size(); i++) { TSparkArrowResultLink thriftLink = resultLinks.get(i); @@ -336,30 +337,33 @@ public GetChunksResult getResultChunks(StatementId statementId, long chunkIndex, // Set nextChunkIndex based on position and hasMoreRows if (i == lastIndex) { - // For the last link, only set nextChunkIndex if there are more rows if (hasMoreRows) { externalLink.setNextChunkIndex(linkChunkIndex + 1); + nextFetchIndex = linkChunkIndex + 1; } - // If hasMoreRows is false, nextChunkIndex remains null (end of stream) - - // Calculate nextRowOffset from last link nextRowOffset = thriftLink.getStartRowOffset() + thriftLink.getRowCount(); } else { - // Not the last link - next chunk follows immediately externalLink.setNextChunkIndex(linkChunkIndex + 1); } - externalLinks.add(externalLink); + chunkLinks.add( + new ChunkLinkFetchResult.ChunkLinkInfo( + linkChunkIndex, + externalLink, + thriftLink.getRowCount(), + thriftLink.getStartRowOffset())); } LOGGER.debug( - "Fetched {} links for statement {}, hasMoreRows={}, nextRowOffset={}", - externalLinks.size(), + "Built ChunkLinkFetchResult with {} links for statement {}, hasMore={}, nextFetchIndex={}, nextRowOffset={}", + chunkLinks.size(), statementId, hasMoreRows, + nextFetchIndex, nextRowOffset); - return GetChunksResult.forThrift(externalLinks, hasMoreRows, nextRowOffset); + return ChunkLinkFetchResult.of( + chunkLinks, hasMoreRows, hasMoreRows ? nextFetchIndex : -1, nextRowOffset); } @Override diff --git a/src/main/java/com/databricks/jdbc/api/impl/arrow/ChunkLinkFetchResult.java b/src/main/java/com/databricks/jdbc/model/core/ChunkLinkFetchResult.java similarity index 80% rename from src/main/java/com/databricks/jdbc/api/impl/arrow/ChunkLinkFetchResult.java rename to src/main/java/com/databricks/jdbc/model/core/ChunkLinkFetchResult.java index 0eb133a44..16739b768 100644 --- a/src/main/java/com/databricks/jdbc/api/impl/arrow/ChunkLinkFetchResult.java +++ b/src/main/java/com/databricks/jdbc/model/core/ChunkLinkFetchResult.java @@ -1,13 +1,18 @@ -package com.databricks.jdbc.api.impl.arrow; +package com.databricks.jdbc.model.core; -import com.databricks.jdbc.model.core.ExternalLink; import java.util.Collections; import java.util.List; /** - * Result of a chunk link fetch operation from {@link ChunkLinkFetcher}. + * Result of fetching chunk links from the server. * - *

Contains the fetched chunk links and information about whether more chunks are available. + *

Contains the fetched chunk links and continuation information for both SEA and Thrift + * protocols: + * + *

    + *
  • SEA: Uses chunkIndex for continuation, hasMore derived from nextChunkIndex on last link + *
  • Thrift: Uses rowOffset for continuation, hasMore from server's hasMoreRows flag + *
*/ public class ChunkLinkFetchResult { @@ -25,20 +30,7 @@ private ChunkLinkFetchResult( } /** - * Creates a result with the given links and continuation info (for SEA). - * - * @param links The fetched chunk links - * @param hasMore Whether more chunks are available - * @param nextFetchIndex The next chunk index to fetch from, or -1 if no more - * @return A new ChunkLinkFetchResult - */ - public static ChunkLinkFetchResult of( - List links, boolean hasMore, long nextFetchIndex) { - return new ChunkLinkFetchResult(links, hasMore, nextFetchIndex, 0); - } - - /** - * Creates a result with full continuation info (for Thrift). + * Creates a result with the given links and continuation info. * * @param links The fetched chunk links * @param hasMore Whether more chunks are available diff --git a/src/main/java/com/databricks/jdbc/model/core/GetChunksResult.java b/src/main/java/com/databricks/jdbc/model/core/GetChunksResult.java deleted file mode 100644 index 3c0557243..000000000 --- a/src/main/java/com/databricks/jdbc/model/core/GetChunksResult.java +++ /dev/null @@ -1,109 +0,0 @@ -package com.databricks.jdbc.model.core; - -import java.util.ArrayList; -import java.util.Collection; -import java.util.Collections; -import java.util.List; - -/** - * Result of fetching chunk links from the server. - * - *

This class encapsulates both the external links and continuation information needed to support - * both SEA and Thrift protocols: - * - *

    - *
  • SEA: Uses chunkIndex for continuation, hasMoreData derived from nextChunkIndex on last link - *
  • Thrift: Uses rowOffset for continuation, hasMoreData from server's hasMoreRows flag - *
- */ -public class GetChunksResult { - - private final Collection externalLinks; - private final boolean hasMoreData; - private final long nextRowOffset; - - private GetChunksResult( - Collection externalLinks, boolean hasMoreData, long nextRowOffset) { - this.externalLinks = externalLinks != null ? externalLinks : Collections.emptyList(); - this.hasMoreData = hasMoreData; - this.nextRowOffset = nextRowOffset; - } - - /** - * Creates a result for SEA client responses. - * - *

For SEA, hasMoreData is derived from the last link's nextChunkIndex (null means no more - * data). nextRowOffset is calculated from the last link's rowOffset + rowCount for unified - * handling with Thrift. - * - * @param links The external links from the response - * @return GetChunksResult with hasMoreData and nextRowOffset derived from links - */ - public static GetChunksResult forSea(Collection links) { - if (links == null || links.isEmpty()) { - return new GetChunksResult(Collections.emptyList(), false, 0); - } - - // Get last link efficiently - List linkList = - links instanceof List ? (List) links : new ArrayList<>(links); - ExternalLink lastLink = linkList.get(linkList.size() - 1); - - boolean hasMore = lastLink.getNextChunkIndex() != null; - long nextRowOffset = lastLink.getRowOffset() + lastLink.getRowCount(); - - return new GetChunksResult(links, hasMore, nextRowOffset); - } - - /** - * Creates a result for Thrift client responses. - * - *

For Thrift, hasMoreData comes directly from the server's hasMoreRows flag. - * - * @param links The external links from the response (may be empty) - * @param hasMoreRows The hasMoreRows flag from the Thrift response - * @param nextRowOffset The row offset to use for the next fetch - * @return GetChunksResult with Thrift continuation info - */ - public static GetChunksResult forThrift( - Collection links, boolean hasMoreRows, long nextRowOffset) { - return new GetChunksResult(links, hasMoreRows, nextRowOffset); - } - - /** - * Creates an end-of-stream result (no more data available). - * - * @return GetChunksResult indicating end of stream - */ - public static GetChunksResult endOfStream() { - return new GetChunksResult(Collections.emptyList(), false, 0); - } - - /** - * @return The external links from the response, never null - */ - public Collection getExternalLinks() { - return externalLinks; - } - - /** - * @return true if there is more data available to fetch - */ - public boolean hasMoreData() { - return hasMoreData; - } - - /** - * @return The row offset to use for the next fetch. Populated for both SEA and Thrift. - */ - public long getNextRowOffset() { - return nextRowOffset; - } - - /** - * @return true if the result contains no links - */ - public boolean isEmpty() { - return externalLinks.isEmpty(); - } -} diff --git a/src/test/java/com/databricks/jdbc/api/impl/arrow/ArrowStreamResultTest.java b/src/test/java/com/databricks/jdbc/api/impl/arrow/ArrowStreamResultTest.java index cb9f4532e..6d3accf72 100644 --- a/src/test/java/com/databricks/jdbc/api/impl/arrow/ArrowStreamResultTest.java +++ b/src/test/java/com/databricks/jdbc/api/impl/arrow/ArrowStreamResultTest.java @@ -20,10 +20,10 @@ import com.databricks.jdbc.model.client.thrift.generated.TGetResultSetMetadataResp; import com.databricks.jdbc.model.client.thrift.generated.TRowSet; import com.databricks.jdbc.model.client.thrift.generated.TSparkArrowResultLink; +import com.databricks.jdbc.model.core.ChunkLinkFetchResult; import com.databricks.jdbc.model.core.ColumnInfo; import com.databricks.jdbc.model.core.ColumnInfoTypeName; import com.databricks.jdbc.model.core.ExternalLink; -import com.databricks.jdbc.model.core.GetChunksResult; import com.databricks.jdbc.model.core.ResultData; import com.databricks.jdbc.model.core.ResultManifest; import com.databricks.jdbc.model.core.ResultSchema; @@ -284,10 +284,36 @@ private void setupResultChunkMocks() throws DatabricksSQLException { for (int chunkIndex = 1; chunkIndex < numberOfChunks; chunkIndex++) { boolean isLastChunk = (chunkIndex == (numberOfChunks - 1)); when(mockedSdkClient.getResultChunks(eq(STATEMENT_ID), eq((long) chunkIndex), anyLong())) - .thenReturn(GetChunksResult.forSea(getChunkLinks(chunkIndex, isLastChunk))); + .thenReturn(buildChunkLinkFetchResult(getChunkLinks(chunkIndex, isLastChunk))); } } + private ChunkLinkFetchResult buildChunkLinkFetchResult(List links) { + if (links == null || links.isEmpty()) { + return ChunkLinkFetchResult.endOfStream(); + } + + List chunkLinks = new ArrayList<>(); + for (ExternalLink link : links) { + chunkLinks.add( + new ChunkLinkFetchResult.ChunkLinkInfo( + link.getChunkIndex(), + link, + link.getRowCount() != null ? link.getRowCount() : 0, + link.getRowOffset() != null ? link.getRowOffset() : 0)); + } + + ExternalLink lastLink = links.get(links.size() - 1); + boolean hasMore = lastLink.getNextChunkIndex() != null; + long nextFetchIndex = hasMore ? lastLink.getNextChunkIndex() : -1; + long nextRowOffset = 0; + if (lastLink.getRowOffset() != null && lastLink.getRowCount() != null) { + nextRowOffset = lastLink.getRowOffset() + lastLink.getRowCount(); + } + + return ChunkLinkFetchResult.of(chunkLinks, hasMore, nextFetchIndex, nextRowOffset); + } + private File createTestArrowFile( String fileName, Schema schema, Object[][] testData, RootAllocator allocator) throws IOException { diff --git a/src/test/java/com/databricks/jdbc/api/impl/arrow/ChunkLinkDownloadServiceTest.java b/src/test/java/com/databricks/jdbc/api/impl/arrow/ChunkLinkDownloadServiceTest.java index 3ec64c866..85d4e0517 100644 --- a/src/test/java/com/databricks/jdbc/api/impl/arrow/ChunkLinkDownloadServiceTest.java +++ b/src/test/java/com/databricks/jdbc/api/impl/arrow/ChunkLinkDownloadServiceTest.java @@ -11,8 +11,8 @@ import com.databricks.jdbc.dbclient.impl.common.StatementId; import com.databricks.jdbc.exception.DatabricksSQLException; import com.databricks.jdbc.exception.DatabricksValidationException; +import com.databricks.jdbc.model.core.ChunkLinkFetchResult; import com.databricks.jdbc.model.core.ExternalLink; -import com.databricks.jdbc.model.core.GetChunksResult; import com.databricks.jdbc.model.telemetry.enums.DatabricksDriverErrorCode; import java.time.Instant; import java.time.temporal.ChronoUnit; @@ -57,13 +57,13 @@ void testGetLinkForChunk_Success() // Mock the response to link requests when(mockClient.getResultChunks(eq(mockStatementId), eq(1L), eq(0L))) - .thenReturn(GetChunksResult.forSea(Collections.singletonList(linkForChunkIndex_1))); + .thenReturn(buildChunkLinkFetchResult(Collections.singletonList(linkForChunkIndex_1))); when(mockClient.getResultChunks(eq(mockStatementId), eq(2L), eq(0L))) - .thenReturn(GetChunksResult.forSea(Collections.singletonList(linkForChunkIndex_2))); + .thenReturn(buildChunkLinkFetchResult(Collections.singletonList(linkForChunkIndex_2))); when(mockClient.getResultChunks(eq(mockStatementId), eq(3L), eq(0L))) - .thenReturn(GetChunksResult.forSea(Collections.singletonList(linkForChunkIndex_3))); + .thenReturn(buildChunkLinkFetchResult(Collections.singletonList(linkForChunkIndex_3))); when(mockClient.getResultChunks(eq(mockStatementId), eq(4L), eq(0L))) - .thenReturn(GetChunksResult.forSea(Collections.singletonList(linkForChunkIndex_4))); + .thenReturn(buildChunkLinkFetchResult(Collections.singletonList(linkForChunkIndex_4))); long chunkIndex = 1L; when(mockChunkMap.get(chunkIndex)).thenReturn(mock(ArrowResultChunk.class)); @@ -136,13 +136,13 @@ void testAutoTriggerForSEAClient() throws DatabricksSQLException, InterruptedExc when(mockSession.getDatabricksClient()).thenReturn(mockClient); // Mock the response to link requests when(mockClient.getResultChunks(eq(mockStatementId), eq(1L), eq(0L))) - .thenReturn(GetChunksResult.forSea(Collections.singletonList(linkForChunkIndex_1))); + .thenReturn(buildChunkLinkFetchResult(Collections.singletonList(linkForChunkIndex_1))); when(mockClient.getResultChunks(eq(mockStatementId), eq(2L), eq(0L))) - .thenReturn(GetChunksResult.forSea(Collections.singletonList(linkForChunkIndex_2))); + .thenReturn(buildChunkLinkFetchResult(Collections.singletonList(linkForChunkIndex_2))); when(mockClient.getResultChunks(eq(mockStatementId), eq(3L), eq(0L))) - .thenReturn(GetChunksResult.forSea(Collections.singletonList(linkForChunkIndex_3))); + .thenReturn(buildChunkLinkFetchResult(Collections.singletonList(linkForChunkIndex_3))); when(mockClient.getResultChunks(eq(mockStatementId), eq(4L), eq(0L))) - .thenReturn(GetChunksResult.forSea(Collections.singletonList(linkForChunkIndex_4))); + .thenReturn(buildChunkLinkFetchResult(Collections.singletonList(linkForChunkIndex_4))); // Download chain will be triggered immediately in the constructor when(mockSession.getConnectionContext().getClientType()).thenReturn(DatabricksClientType.SEA); @@ -166,13 +166,14 @@ void testHandleExpiredLinks() // Mock the response to link requests. Return the expired link for chunk index 1 when(mockClient.getResultChunks(eq(mockStatementId), eq(1L), anyLong())) - .thenReturn(GetChunksResult.forSea(Collections.singletonList(expiredLinkForChunkIndex_1))); + .thenReturn( + buildChunkLinkFetchResult(Collections.singletonList(expiredLinkForChunkIndex_1))); when(mockClient.getResultChunks(eq(mockStatementId), eq(2L), anyLong())) - .thenReturn(GetChunksResult.forSea(Collections.singletonList(linkForChunkIndex_2))); + .thenReturn(buildChunkLinkFetchResult(Collections.singletonList(linkForChunkIndex_2))); when(mockClient.getResultChunks(eq(mockStatementId), eq(3L), anyLong())) - .thenReturn(GetChunksResult.forSea(Collections.singletonList(linkForChunkIndex_3))); + .thenReturn(buildChunkLinkFetchResult(Collections.singletonList(linkForChunkIndex_3))); when(mockClient.getResultChunks(eq(mockStatementId), eq(4L), anyLong())) - .thenReturn(GetChunksResult.forSea(Collections.singletonList(linkForChunkIndex_4))); + .thenReturn(buildChunkLinkFetchResult(Collections.singletonList(linkForChunkIndex_4))); long chunkIndex = 1L; ArrowResultChunk mockChunk = mock(ArrowResultChunk.class); @@ -188,7 +189,7 @@ void testHandleExpiredLinks() // Mock a new valid link for chunk index 1 when(mockClient.getResultChunks(eq(mockStatementId), eq(1L), eq(0L))) - .thenReturn(GetChunksResult.forSea(Collections.singletonList(linkForChunkIndex_1))); + .thenReturn(buildChunkLinkFetchResult(Collections.singletonList(linkForChunkIndex_1))); // Try to get the link for chunk index 1. Download chain will be re-triggered because the link // is expired CompletableFuture future = service.getLinkForChunk(chunkIndex); @@ -226,15 +227,15 @@ void testBatchDownloadChaining() // same time when(mockClient.getResultChunks(eq(mockStatementId), eq(1L), eq(0L))) .thenReturn( - GetChunksResult.forSea(Arrays.asList(linkForChunkIndex_1, linkForChunkIndex_2))); + buildChunkLinkFetchResult(Arrays.asList(linkForChunkIndex_1, linkForChunkIndex_2))); // Mock the links for the second batch. when(mockClient.getResultChunks(eq(mockStatementId), eq(3L), eq(0L))) .thenReturn( - GetChunksResult.forSea(Arrays.asList(linkForChunkIndex_3, linkForChunkIndex_4))); + buildChunkLinkFetchResult(Arrays.asList(linkForChunkIndex_3, linkForChunkIndex_4))); // Mock the links for the third batch. when(mockClient.getResultChunks(eq(mockStatementId), eq(5L), eq(0L))) .thenReturn( - GetChunksResult.forSea(Arrays.asList(linkForChunkIndex_5, linkForChunkIndex_6))); + buildChunkLinkFetchResult(Arrays.asList(linkForChunkIndex_5, linkForChunkIndex_6))); ChunkLinkDownloadService service = new ChunkLinkDownloadService<>( @@ -282,4 +283,34 @@ private ExternalLink createExternalLink( return link; } + + /** + * Helper method to build ChunkLinkFetchResult from a list of ExternalLinks. This mimics the + * behavior of the SEA client's buildChunkLinkFetchResult method. + */ + private ChunkLinkFetchResult buildChunkLinkFetchResult(List links) { + if (links == null || links.isEmpty()) { + return ChunkLinkFetchResult.endOfStream(); + } + + List chunkLinks = new ArrayList<>(); + for (ExternalLink link : links) { + chunkLinks.add( + new ChunkLinkFetchResult.ChunkLinkInfo( + link.getChunkIndex(), + link, + link.getRowCount() != null ? link.getRowCount() : 0, + link.getRowOffset() != null ? link.getRowOffset() : 0)); + } + + ExternalLink lastLink = links.get(links.size() - 1); + boolean hasMore = lastLink.getNextChunkIndex() != null; + long nextFetchIndex = hasMore ? lastLink.getNextChunkIndex() : -1; + long nextRowOffset = 0; + if (lastLink.getRowOffset() != null && lastLink.getRowCount() != null) { + nextRowOffset = lastLink.getRowOffset() + lastLink.getRowCount(); + } + + return ChunkLinkFetchResult.of(chunkLinks, hasMore, nextFetchIndex, nextRowOffset); + } } diff --git a/src/test/java/com/databricks/jdbc/dbclient/impl/thrift/DatabricksThriftServiceClientTest.java b/src/test/java/com/databricks/jdbc/dbclient/impl/thrift/DatabricksThriftServiceClientTest.java index 09bd0c738..62c6e022d 100644 --- a/src/test/java/com/databricks/jdbc/dbclient/impl/thrift/DatabricksThriftServiceClientTest.java +++ b/src/test/java/com/databricks/jdbc/dbclient/impl/thrift/DatabricksThriftServiceClientTest.java @@ -24,7 +24,7 @@ import com.databricks.jdbc.exception.DatabricksParsingException; import com.databricks.jdbc.exception.DatabricksSQLException; import com.databricks.jdbc.model.client.thrift.generated.*; -import com.databricks.jdbc.model.core.ExternalLink; +import com.databricks.jdbc.model.core.ChunkLinkFetchResult; import com.databricks.jdbc.model.core.ResultColumn; import com.databricks.sdk.core.DatabricksConfig; import com.databricks.sdk.service.sql.StatementState; @@ -376,10 +376,10 @@ void testGetResultChunks() throws SQLException { .thenReturn( Collections.singletonList(new TSparkArrowResultLink().setFileLink(TEST_STRING))); // Pass chunkIndex=0 and rowOffset=0 for the first chunk - Collection resultChunks = - client.getResultChunks(TEST_STMT_ID, 0, 0).getExternalLinks(); - assertEquals(resultChunks.size(), 1); - assertEquals(resultChunks.stream().findFirst().get().getExternalLink(), TEST_STRING); + ChunkLinkFetchResult result = client.getResultChunks(TEST_STMT_ID, 0, 0); + List chunkLinks = result.getChunkLinks(); + assertEquals(1, chunkLinks.size()); + assertEquals(TEST_STRING, chunkLinks.get(0).getLink().getExternalLink()); } @Test @@ -394,9 +394,9 @@ void testGetResultChunksReturnsEmptyWhenNoLinks() throws SQLException { when(thriftAccessor.fetchResultsWithAbsoluteOffset(any(), anyLong(), any())) .thenReturn(response); when(resultData.getResultLinks()).thenReturn(null); - Collection resultChunks = - client.getResultChunks(TEST_STMT_ID, 0, 0).getExternalLinks(); - assertEquals(0, resultChunks.size()); + ChunkLinkFetchResult result = client.getResultChunks(TEST_STMT_ID, 0, 0); + assertTrue(result.isEndOfStream()); + assertEquals(0, result.getChunkLinks().size()); } @Test From 5bffdb5f329c2962fef6811b53ef80756a2a5e7b Mon Sep 17 00:00:00 2001 From: Jayant Singh Date: Sat, 6 Dec 2025 14:04:33 +0530 Subject: [PATCH 5/6] Address review comments --- .../api/impl/DatabricksConnectionContext.java | 5 + .../api/impl/arrow/ArrowStreamResult.java | 23 +-- .../impl/arrow/ChunkLinkDownloadService.java | 10 +- .../api/impl/arrow/SeaChunkLinkFetcher.java | 6 +- .../impl/arrow/StreamingChunkProvider.java | 166 +++++++++--------- .../impl/arrow/ThriftChunkLinkFetcher.java | 6 +- .../IDatabricksConnectionContext.java | 12 ++ .../jdbc/common/DatabricksJdbcUrlParams.java | 8 +- .../jdbc/common/util/DriverUtil.java | 2 +- .../impl/sqlexec/DatabricksSdkClient.java | 11 +- .../thrift/DatabricksThriftServiceClient.java | 11 +- .../jdbc/model/core/ChunkLinkFetchResult.java | 47 +---- .../api/impl/arrow/ArrowStreamResultTest.java | 21 +-- .../arrow/ChunkLinkDownloadServiceTest.java | 19 +- .../DatabricksThriftServiceClientTest.java | 5 +- 15 files changed, 155 insertions(+), 197 deletions(-) diff --git a/src/main/java/com/databricks/jdbc/api/impl/DatabricksConnectionContext.java b/src/main/java/com/databricks/jdbc/api/impl/DatabricksConnectionContext.java index cd6bffc83..fdd82d425 100644 --- a/src/main/java/com/databricks/jdbc/api/impl/DatabricksConnectionContext.java +++ b/src/main/java/com/databricks/jdbc/api/impl/DatabricksConnectionContext.java @@ -1155,4 +1155,9 @@ public boolean isTokenFederationEnabled() { public boolean isStreamingChunkProviderEnabled() { return getParameter(DatabricksJdbcUrlParams.ENABLE_STREAMING_CHUNK_PROVIDER).equals("1"); } + + @Override + public int getLinkPrefetchWindow() { + return Integer.parseInt(getParameter(DatabricksJdbcUrlParams.LINK_PREFETCH_WINDOW)); + } } diff --git a/src/main/java/com/databricks/jdbc/api/impl/arrow/ArrowStreamResult.java b/src/main/java/com/databricks/jdbc/api/impl/arrow/ArrowStreamResult.java index 21464923c..98a021554 100644 --- a/src/main/java/com/databricks/jdbc/api/impl/arrow/ArrowStreamResult.java +++ b/src/main/java/com/databricks/jdbc/api/impl/arrow/ArrowStreamResult.java @@ -29,7 +29,6 @@ import java.util.ArrayList; import java.util.Collection; import java.util.List; -import java.util.stream.Collectors; /** Result container for Arrow-based query results. */ public class ArrowStreamResult implements IExecutionResult { @@ -111,6 +110,7 @@ private static ChunkProvider createRemoteChunkProvider( ChunkLinkFetcher linkFetcher = new SeaChunkLinkFetcher(session, statementId); CompressionCodec compressionCodec = resultManifest.getResultCompression(); int maxChunksInMemory = connectionContext.getCloudFetchThreadPoolSize(); + int linkPrefetchWindow = connectionContext.getLinkPrefetchWindow(); int chunkReadyTimeoutSeconds = connectionContext.getChunkReadyTimeoutSeconds(); double cloudFetchSpeedThreshold = connectionContext.getCloudFetchSpeedThreshold(); @@ -124,6 +124,7 @@ private static ChunkProvider createRemoteChunkProvider( compressionCodec, statementId, maxChunksInMemory, + linkPrefetchWindow, chunkReadyTimeoutSeconds, cloudFetchSpeedThreshold, initialLinks); @@ -197,6 +198,7 @@ private static ChunkProvider createThriftRemoteChunkProvider( ChunkLinkFetcher linkFetcher = new ThriftChunkLinkFetcher(session, statementId); int maxChunksInMemory = connectionContext.getCloudFetchThreadPoolSize(); + int linkPrefetchWindow = connectionContext.getLinkPrefetchWindow(); int chunkReadyTimeoutSeconds = connectionContext.getChunkReadyTimeoutSeconds(); double cloudFetchSpeedThreshold = connectionContext.getCloudFetchSpeedThreshold(); @@ -209,6 +211,7 @@ private static ChunkProvider createThriftRemoteChunkProvider( compressionCodec, statementId, maxChunksInMemory, + linkPrefetchWindow, chunkReadyTimeoutSeconds, cloudFetchSpeedThreshold, initialLinks); @@ -387,21 +390,13 @@ private static ChunkLinkFetchResult convertToChunkLinkFetchResult( ? (List) externalLinks : new ArrayList<>(externalLinks); - List chunkLinks = - linkList.stream() - .map( - link -> - new ChunkLinkFetchResult.ChunkLinkInfo( - link.getChunkIndex(), link, link.getRowCount(), link.getRowOffset())) - .collect(Collectors.toList()); - // Derive hasMore and nextRowOffset from last link (SEA style) ExternalLink lastLink = linkList.get(linkList.size() - 1); boolean hasMore = lastLink.getNextChunkIndex() != null; long nextFetchIndex = hasMore ? lastLink.getNextChunkIndex() : -1; long nextRowOffset = lastLink.getRowOffset() + lastLink.getRowCount(); - return ChunkLinkFetchResult.of(chunkLinks, hasMore, nextFetchIndex, nextRowOffset); + return ChunkLinkFetchResult.of(linkList, hasMore, nextFetchIndex, nextRowOffset); } /** @@ -420,14 +415,14 @@ private static ChunkLinkFetchResult convertThriftLinksToChunkLinkFetchResult( return null; } - List chunkLinks = new ArrayList<>(); + List chunkLinks = new ArrayList<>(); int lastIndex = resultLinks.size() - 1; boolean hasMoreRows = resultsResp.hasMoreRows; for (int i = 0; i < resultLinks.size(); i++) { TSparkArrowResultLink thriftLink = resultLinks.get(i); - // Convert Thrift link to ExternalLink + // Convert Thrift link to ExternalLink (sets chunkIndex, rowOffset, rowCount, etc.) ExternalLink externalLink = createExternalLink(thriftLink, i); // For the last link, set nextChunkIndex based on hasMoreRows @@ -442,9 +437,7 @@ private static ChunkLinkFetchResult convertThriftLinksToChunkLinkFetchResult( externalLink.setNextChunkIndex((long) i + 1); } - chunkLinks.add( - new ChunkLinkFetchResult.ChunkLinkInfo( - i, externalLink, thriftLink.getRowCount(), thriftLink.getStartRowOffset())); + chunkLinks.add(externalLink); } // Calculate next fetch positions from last link diff --git a/src/main/java/com/databricks/jdbc/api/impl/arrow/ChunkLinkDownloadService.java b/src/main/java/com/databricks/jdbc/api/impl/arrow/ChunkLinkDownloadService.java index d521e4d46..b04fdfd10 100644 --- a/src/main/java/com/databricks/jdbc/api/impl/arrow/ChunkLinkDownloadService.java +++ b/src/main/java/com/databricks/jdbc/api/impl/arrow/ChunkLinkDownloadService.java @@ -230,15 +230,15 @@ private void triggerNextBatchDownload() { statementId); // Complete futures for all chunks in this batch - for (ChunkLinkFetchResult.ChunkLinkInfo linkInfo : result.getChunkLinks()) { + for (ExternalLink link : result.getChunkLinks()) { CompletableFuture future = - chunkIndexToLinkFuture.get(linkInfo.getChunkIndex()); + chunkIndexToLinkFuture.get(link.getChunkIndex()); if (future != null) { LOGGER.debug( "Completing future for chunk {} for statement id {}", - linkInfo.getChunkIndex(), + link.getChunkIndex(), statementId); - future.complete(linkInfo.getLink()); + future.complete(link); } } @@ -246,7 +246,7 @@ private void triggerNextBatchDownload() { if (!result.getChunkLinks().isEmpty()) { long maxChunkIndex = result.getChunkLinks().stream() - .mapToLong(ChunkLinkFetchResult.ChunkLinkInfo::getChunkIndex) + .mapToLong(ExternalLink::getChunkIndex) .max() .getAsLong(); nextBatchStartIndex.set(maxChunkIndex + 1); diff --git a/src/main/java/com/databricks/jdbc/api/impl/arrow/SeaChunkLinkFetcher.java b/src/main/java/com/databricks/jdbc/api/impl/arrow/SeaChunkLinkFetcher.java index 28482118f..0f6930631 100644 --- a/src/main/java/com/databricks/jdbc/api/impl/arrow/SeaChunkLinkFetcher.java +++ b/src/main/java/com/databricks/jdbc/api/impl/arrow/SeaChunkLinkFetcher.java @@ -57,11 +57,11 @@ public ExternalLink refetchLink(long chunkIndex, long rowOffset) throws Databric } // Find the link for the requested chunk index - for (ChunkLinkFetchResult.ChunkLinkInfo linkInfo : result.getChunkLinks()) { - if (linkInfo.getChunkIndex() == chunkIndex) { + for (ExternalLink link : result.getChunkLinks()) { + if (link.getChunkIndex() == chunkIndex) { LOGGER.debug( "Successfully refetched link for chunk {} of statement {}", chunkIndex, statementId); - return linkInfo.getLink(); + return link; } } diff --git a/src/main/java/com/databricks/jdbc/api/impl/arrow/StreamingChunkProvider.java b/src/main/java/com/databricks/jdbc/api/impl/arrow/StreamingChunkProvider.java index d0f87db5a..c50081eb5 100644 --- a/src/main/java/com/databricks/jdbc/api/impl/arrow/StreamingChunkProvider.java +++ b/src/main/java/com/databricks/jdbc/api/impl/arrow/StreamingChunkProvider.java @@ -8,6 +8,7 @@ import com.databricks.jdbc.log.JdbcLogger; import com.databricks.jdbc.log.JdbcLoggerFactory; import com.databricks.jdbc.model.core.ChunkLinkFetchResult; +import com.databricks.jdbc.model.core.ExternalLink; import com.databricks.jdbc.model.telemetry.enums.DatabricksDriverErrorCode; import java.util.concurrent.ConcurrentHashMap; import java.util.concurrent.ConcurrentMap; @@ -64,15 +65,20 @@ public class StreamingChunkProvider implements ChunkProvider { private final ConcurrentMap chunks = new ConcurrentHashMap<>(); // Position tracking - private volatile long currentChunkIndex = -1; - private volatile long highestKnownChunkIndex = -1; + // Using AtomicLong for single-writer variables to make thread-safety explicit: + // - currentChunkIndex: written only by consumer thread + // - highestKnownChunkIndex: written only by prefetch thread (after construction) + // - nextDownloadIndex: written only under downloadLock, but AtomicLong for consistency + private final AtomicLong currentChunkIndex = new AtomicLong(-1); + private final AtomicLong highestKnownChunkIndex = new AtomicLong(-1); private volatile long nextLinkFetchIndex = 0; private volatile long nextRowOffsetToFetch = 0; - private volatile long nextDownloadIndex = 0; + private final AtomicLong nextDownloadIndex = new AtomicLong(0); // State flags private volatile boolean endOfStreamReached = false; private volatile boolean closed = false; + private volatile DatabricksSQLException prefetchError = null; // Row tracking private final AtomicLong totalRowCount = new AtomicLong(0); @@ -82,7 +88,12 @@ public class StreamingChunkProvider implements ChunkProvider { private final Condition consumerAdvanced = prefetchLock.newCondition(); private final Condition chunkCreated = prefetchLock.newCondition(); - // Synchronization for download coordination + // Synchronization for download coordination. + // This lock is needed because triggerDownloads() is called from both the prefetch thread + // (via fetchNextLinkBatch) and the consumer thread (via releaseChunk), and the download + // logic reads multiple shared variables (chunksInMemory, nextDownloadIndex, + // highestKnownChunkIndex) + // that must be consistent within the loop. private final ReentrantLock downloadLock = new ReentrantLock(); // Executors @@ -148,29 +159,6 @@ public StreamingChunkProvider( notifyConsumerAdvanced(); } - /** Convenience constructor with default prefetch window. */ - public StreamingChunkProvider( - ChunkLinkFetcher linkFetcher, - IDatabricksHttpClient httpClient, - CompressionCodec compressionCodec, - StatementId statementId, - int maxChunksInMemory, - int chunkReadyTimeoutSeconds, - double cloudFetchSpeedThreshold, - ChunkLinkFetchResult initialLinks) - throws DatabricksParsingException { - this( - linkFetcher, - httpClient, - compressionCodec, - statementId, - maxChunksInMemory, - 64, // Default prefetch window - chunkReadyTimeoutSeconds, - cloudFetchSpeedThreshold, - initialLinks); - } - // ==================== ChunkProvider Interface ==================== @Override @@ -185,7 +173,7 @@ public boolean hasNextChunk() { } // We've reached end of stream - check if there are unconsumed chunks - return currentChunkIndex < highestKnownChunkIndex; + return currentChunkIndex.get() < highestKnownChunkIndex.get(); } @Override @@ -195,15 +183,16 @@ public boolean next() throws DatabricksSQLException { } // Release previous chunk if any - if (currentChunkIndex >= 0) { - releaseChunk(currentChunkIndex); + long prevIndex = currentChunkIndex.get(); + if (prevIndex >= 0) { + releaseChunk(prevIndex); } if (!hasNextChunk()) { return false; } - currentChunkIndex++; + currentChunkIndex.incrementAndGet(); // Notify prefetch thread that consumer advanced notifyConsumerAdvanced(); @@ -213,22 +202,23 @@ public boolean next() throws DatabricksSQLException { @Override public AbstractArrowResultChunk getChunk() throws DatabricksSQLException { - if (currentChunkIndex < 0) { + long chunkIdx = currentChunkIndex.get(); + if (chunkIdx < 0) { return null; } - ArrowResultChunk chunk = chunks.get(currentChunkIndex); + ArrowResultChunk chunk = chunks.get(chunkIdx); if (chunk == null) { // Chunk not yet created - wait for it - LOGGER.debug("Chunk {} not yet available, waiting for prefetch", currentChunkIndex); - waitForChunkCreation(currentChunkIndex); - chunk = chunks.get(currentChunkIndex); + LOGGER.debug("Chunk {} not yet available, waiting for prefetch", chunkIdx); + waitForChunkCreation(chunkIdx); + chunk = chunks.get(chunkIdx); } if (chunk == null) { throw new DatabricksSQLException( - "Chunk " + currentChunkIndex + " not found after waiting", + "Chunk " + chunkIdx + " not found after waiting", DatabricksDriverErrorCode.CHUNK_READY_ERROR); } @@ -238,21 +228,17 @@ public AbstractArrowResultChunk getChunk() throws DatabricksSQLException { } catch (InterruptedException e) { Thread.currentThread().interrupt(); throw new DatabricksSQLException( - "Interrupted waiting for chunk " + currentChunkIndex, + "Interrupted waiting for chunk " + chunkIdx, e, DatabricksDriverErrorCode.THREAD_INTERRUPTED_ERROR); } catch (ExecutionException e) { throw new DatabricksSQLException( - "Failed to prepare chunk " + currentChunkIndex, + "Failed to prepare chunk " + chunkIdx, e.getCause(), DatabricksDriverErrorCode.CHUNK_READY_ERROR); } catch (TimeoutException e) { throw new DatabricksSQLException( - "Timeout waiting for chunk " - + currentChunkIndex - + " (timeout: " - + chunkReadyTimeoutSeconds - + "s)", + "Timeout waiting for chunk " + chunkIdx + " (timeout: " + chunkReadyTimeoutSeconds + "s)", DatabricksDriverErrorCode.CHUNK_READY_ERROR); } @@ -307,7 +293,7 @@ public long getRowCount() { public long getChunkCount() { // In streaming mode, we don't know total chunks until end of stream if (endOfStreamReached) { - return highestKnownChunkIndex + 1; + return highestKnownChunkIndex.get() + 1; } return -1; // Unknown } @@ -326,17 +312,17 @@ private void linkPrefetchLoop() { try { prefetchLock.lock(); try { - // Calculate target prefetch index - long targetIndex = currentChunkIndex + linkPrefetchWindow; + long targetIndex = currentChunkIndex.get() + linkPrefetchWindow; // Wait if we're caught up - while (!closed && !endOfStreamReached && nextLinkFetchIndex > targetIndex) { + while (!endOfStreamReached && nextLinkFetchIndex > targetIndex) { + if (closed) break; LOGGER.debug( "Prefetch caught up, waiting for consumer. next={}, target={}", nextLinkFetchIndex, targetIndex); consumerAdvanced.await(); - targetIndex = currentChunkIndex + linkPrefetchWindow; + targetIndex = currentChunkIndex.get() + linkPrefetchWindow; } } finally { prefetchLock.unlock(); @@ -355,7 +341,9 @@ private void linkPrefetchLoop() { break; } catch (DatabricksSQLException e) { LOGGER.error("Error fetching links: {}", e.getMessage()); - // Continue trying - next iteration may succeed + prefetchError = e; + notifyChunkCreated(); // Wake up any waiting consumer to check the error + break; } } @@ -382,8 +370,8 @@ private void fetchNextLinkBatch() throws DatabricksSQLException { } // Process received links - create chunks - for (ChunkLinkFetchResult.ChunkLinkInfo linkInfo : result.getChunkLinks()) { - createChunkFromLink(linkInfo); + for (ExternalLink link : result.getChunkLinks()) { + createChunkFromLink(link); } // Update next fetch positions @@ -407,7 +395,7 @@ private void fetchNextLinkBatch() throws DatabricksSQLException { */ private void processInitialLinks(ChunkLinkFetchResult initialLinks) throws DatabricksParsingException { - if (initialLinks == null || initialLinks.isEndOfStream()) { + if (initialLinks == null) { LOGGER.debug("No initial links provided for statement {}", statementId); return; } @@ -417,8 +405,8 @@ private void processInitialLinks(ChunkLinkFetchResult initialLinks) initialLinks.getChunkLinks().size(), statementId); - for (ChunkLinkFetchResult.ChunkLinkInfo linkInfo : initialLinks.getChunkLinks()) { - createChunkFromLink(linkInfo); + for (ExternalLink link : initialLinks.getChunkLinks()) { + createChunkFromLink(link); } // Set next fetch positions using unified API @@ -436,38 +424,37 @@ private void processInitialLinks(ChunkLinkFetchResult initialLinks) } /** - * Creates a chunk from link info and registers it for download. + * Creates a chunk from an external link and registers it for download. * - * @param linkInfo The chunk link info containing index, row count, offset, and link + * @param link The external link containing chunkIndex, rowCount, rowOffset, and download URL */ - private void createChunkFromLink(ChunkLinkFetchResult.ChunkLinkInfo linkInfo) - throws DatabricksParsingException { - long chunkIndex = linkInfo.getChunkIndex(); + private void createChunkFromLink(ExternalLink link) throws DatabricksParsingException { + long chunkIndex = link.getChunkIndex(); if (chunks.containsKey(chunkIndex)) { LOGGER.debug("Chunk {} already exists, skipping creation", chunkIndex); return; } + long rowCount = link.getRowCount(); + long rowOffset = link.getRowOffset(); + ArrowResultChunk chunk = ArrowResultChunk.builder() .withStatementId(statementId) - .withChunkMetadata(chunkIndex, linkInfo.getRowCount(), linkInfo.getRowOffset()) + .withChunkMetadata(chunkIndex, rowCount, rowOffset) .withChunkReadyTimeoutSeconds(chunkReadyTimeoutSeconds) .build(); - chunk.setChunkLink(linkInfo.getLink()); + chunk.setChunkLink(link); chunks.put(chunkIndex, chunk); - highestKnownChunkIndex = Math.max(highestKnownChunkIndex, chunkIndex); - totalRowCount.addAndGet(linkInfo.getRowCount()); + highestKnownChunkIndex.updateAndGet(current -> Math.max(current, chunkIndex)); + totalRowCount.addAndGet(rowCount); // Notify any waiting consumers that a chunk is available notifyChunkCreated(); LOGGER.debug( - "Created chunk {} with {} rows for statement {}", - chunkIndex, - linkInfo.getRowCount(), - statementId); + "Created chunk {} with {} rows for statement {}", chunkIndex, rowCount, statementId); } // ==================== Download Coordination ==================== @@ -475,10 +462,11 @@ private void createChunkFromLink(ChunkLinkFetchResult.ChunkLinkInfo linkInfo) private void triggerDownloads() { downloadLock.lock(); try { + long downloadIdx = nextDownloadIndex.get(); while (!closed && chunksInMemory.get() < maxChunksInMemory - && nextDownloadIndex <= highestKnownChunkIndex) { - ArrowResultChunk chunk = chunks.get(nextDownloadIndex); + && downloadIdx <= highestKnownChunkIndex.get()) { + ArrowResultChunk chunk = chunks.get(downloadIdx); if (chunk == null) { // Chunk not yet created, wait for prefetch @@ -492,7 +480,7 @@ private void triggerDownloads() { chunksInMemory.incrementAndGet(); } - nextDownloadIndex++; + downloadIdx = nextDownloadIndex.incrementAndGet(); } } finally { downloadLock.unlock(); @@ -524,30 +512,44 @@ private void releaseChunk(long chunkIndex) { } } + /** + * Waits for a chunk to be created by the prefetch thread. + * + *

This method waits indefinitely for the chunk to be created, relying on the following exit + * conditions: + * + *

    + *
  • Chunk is created (success) + *
  • Provider is closed + *
  • Prefetch thread encountered an error + *
  • End of stream reached and chunk doesn't exist + *
  • Thread is interrupted + *
+ * + *

The overall timeout for chunk retrieval is enforced by {@link + * ArrowResultChunk#waitForChunkReady()} which has a configurable timeout. + */ private void waitForChunkCreation(long chunkIndex) throws DatabricksSQLException { - long remainingNanos = 5 * 1_000_000_000L; - prefetchLock.lock(); try { while (!closed && !chunks.containsKey(chunkIndex)) { - if (endOfStreamReached && chunkIndex > highestKnownChunkIndex) { + // Check if prefetch thread encountered an error + if (prefetchError != null) { throw new DatabricksSQLException( - "Chunk " - + chunkIndex - + " does not exist (highest known: " - + highestKnownChunkIndex - + ")", + "Link prefetch failed: " + prefetchError.getMessage(), + prefetchError, DatabricksDriverErrorCode.CHUNK_READY_ERROR); } - if (remainingNanos <= 0) { + long highestKnown = highestKnownChunkIndex.get(); + if (endOfStreamReached && chunkIndex > highestKnown) { throw new DatabricksSQLException( - "Timeout waiting for chunk " + chunkIndex + " to be created", + "Chunk " + chunkIndex + " does not exist (highest known: " + highestKnown + ")", DatabricksDriverErrorCode.CHUNK_READY_ERROR); } try { - remainingNanos = chunkCreated.awaitNanos(remainingNanos); + chunkCreated.await(); } catch (InterruptedException e) { Thread.currentThread().interrupt(); throw new DatabricksSQLException( diff --git a/src/main/java/com/databricks/jdbc/api/impl/arrow/ThriftChunkLinkFetcher.java b/src/main/java/com/databricks/jdbc/api/impl/arrow/ThriftChunkLinkFetcher.java index 109e31b86..d38653271 100644 --- a/src/main/java/com/databricks/jdbc/api/impl/arrow/ThriftChunkLinkFetcher.java +++ b/src/main/java/com/databricks/jdbc/api/impl/arrow/ThriftChunkLinkFetcher.java @@ -64,13 +64,13 @@ public ExternalLink refetchLink(long chunkIndex, long rowOffset) throws Databric if (!result.getChunkLinks().isEmpty()) { // Find the link for the requested chunk index - for (ChunkLinkFetchResult.ChunkLinkInfo linkInfo : result.getChunkLinks()) { - if (linkInfo.getChunkIndex() == chunkIndex) { + for (ExternalLink link : result.getChunkLinks()) { + if (link.getChunkIndex() == chunkIndex) { LOGGER.debug( "Successfully refetched link for chunk {} of statement {}", chunkIndex, statementId); - return linkInfo.getLink(); + return link; } } diff --git a/src/main/java/com/databricks/jdbc/api/internal/IDatabricksConnectionContext.java b/src/main/java/com/databricks/jdbc/api/internal/IDatabricksConnectionContext.java index 204425289..4854a4b43 100644 --- a/src/main/java/com/databricks/jdbc/api/internal/IDatabricksConnectionContext.java +++ b/src/main/java/com/databricks/jdbc/api/internal/IDatabricksConnectionContext.java @@ -419,4 +419,16 @@ public interface IDatabricksConnectionContext { /** Returns whether streaming chunk provider is enabled for result fetching. */ boolean isStreamingChunkProviderEnabled(); + + /** + * Returns the number of chunk links to prefetch ahead of consumption. + * + *

This controls how far ahead the streaming chunk provider fetches links before they are + * needed. Higher values reduce latency by ensuring links are ready when needed. Lower values + * reduce the risk of link expiry for workloads that process data slowly (e.g., heavy computation + * per row), since prefetched links may expire before being used. + * + * @return the link prefetch window size (default: 128) + */ + int getLinkPrefetchWindow(); } diff --git a/src/main/java/com/databricks/jdbc/common/DatabricksJdbcUrlParams.java b/src/main/java/com/databricks/jdbc/common/DatabricksJdbcUrlParams.java index c1d22e218..c725c1940 100644 --- a/src/main/java/com/databricks/jdbc/common/DatabricksJdbcUrlParams.java +++ b/src/main/java/com/databricks/jdbc/common/DatabricksJdbcUrlParams.java @@ -191,7 +191,13 @@ public enum DatabricksJdbcUrlParams { ENABLE_STREAMING_CHUNK_PROVIDER( "EnableStreamingChunkProvider", "Enable streaming chunk provider for result fetching (experimental)", - "0"); + "0"), + LINK_PREFETCH_WINDOW( + "LinkPrefetchWindow", + "Number of chunk links to prefetch ahead of consumption. " + + "Higher values reduce latency by having links ready sooner. " + + "Lower values reduce risk of link expiry for slow processing workloads", + "128"); private final String paramName; private final String defaultValue; diff --git a/src/main/java/com/databricks/jdbc/common/util/DriverUtil.java b/src/main/java/com/databricks/jdbc/common/util/DriverUtil.java index b6418f394..16187e05f 100644 --- a/src/main/java/com/databricks/jdbc/common/util/DriverUtil.java +++ b/src/main/java/com/databricks/jdbc/common/util/DriverUtil.java @@ -20,7 +20,7 @@ public class DriverUtil { private static final JdbcLogger LOGGER = JdbcLoggerFactory.getLogger(DriverUtil.class); - private static final String DRIVER_VERSION = "3.0.5"; + private static final String DRIVER_VERSION = "3.0.6"; private static final String DRIVER_NAME = "oss-jdbc"; private static final String JDBC_VERSION = "4.3"; diff --git a/src/main/java/com/databricks/jdbc/dbclient/impl/sqlexec/DatabricksSdkClient.java b/src/main/java/com/databricks/jdbc/dbclient/impl/sqlexec/DatabricksSdkClient.java index 7c627dacb..c0bfcaabc 100644 --- a/src/main/java/com/databricks/jdbc/dbclient/impl/sqlexec/DatabricksSdkClient.java +++ b/src/main/java/com/databricks/jdbc/dbclient/impl/sqlexec/DatabricksSdkClient.java @@ -445,13 +445,6 @@ private ChunkLinkFetchResult buildChunkLinkFetchResult(Collection List linkList = links instanceof List ? (List) links : new ArrayList<>(links); - List chunkLinks = new ArrayList<>(); - for (ExternalLink link : linkList) { - chunkLinks.add( - new ChunkLinkFetchResult.ChunkLinkInfo( - link.getChunkIndex(), link, link.getRowCount(), link.getRowOffset())); - } - // Derive continuation info from last link ExternalLink lastLink = linkList.get(linkList.size() - 1); boolean hasMore = lastLink.getNextChunkIndex() != null; @@ -460,12 +453,12 @@ private ChunkLinkFetchResult buildChunkLinkFetchResult(Collection LOGGER.debug( "Built ChunkLinkFetchResult with {} links, hasMore={}, nextFetchIndex={}, nextRowOffset={}", - chunkLinks.size(), + linkList.size(), hasMore, nextFetchIndex, nextRowOffset); - return ChunkLinkFetchResult.of(chunkLinks, hasMore, nextFetchIndex, nextRowOffset); + return ChunkLinkFetchResult.of(linkList, hasMore, nextFetchIndex, nextRowOffset); } @Override diff --git a/src/main/java/com/databricks/jdbc/dbclient/impl/thrift/DatabricksThriftServiceClient.java b/src/main/java/com/databricks/jdbc/dbclient/impl/thrift/DatabricksThriftServiceClient.java index f53a1087f..8f3a10fd6 100644 --- a/src/main/java/com/databricks/jdbc/dbclient/impl/thrift/DatabricksThriftServiceClient.java +++ b/src/main/java/com/databricks/jdbc/dbclient/impl/thrift/DatabricksThriftServiceClient.java @@ -324,7 +324,7 @@ public ChunkLinkFetchResult getResultChunks( return ChunkLinkFetchResult.of(new ArrayList<>(), hasMoreRows, chunkIndex, rowOffset); } - List chunkLinks = new ArrayList<>(); + List chunkLinks = new ArrayList<>(); int lastIndex = resultLinks.size() - 1; long nextRowOffset = rowOffset; long nextFetchIndex = chunkIndex; @@ -333,6 +333,8 @@ public ChunkLinkFetchResult getResultChunks( TSparkArrowResultLink thriftLink = resultLinks.get(i); long linkChunkIndex = chunkIndex + i; + // createExternalLink sets chunkIndex, rowOffset, rowCount, byteCount, expiration, + // externalLink ExternalLink externalLink = createExternalLink(thriftLink, linkChunkIndex); // Set nextChunkIndex based on position and hasMoreRows @@ -346,12 +348,7 @@ public ChunkLinkFetchResult getResultChunks( externalLink.setNextChunkIndex(linkChunkIndex + 1); } - chunkLinks.add( - new ChunkLinkFetchResult.ChunkLinkInfo( - linkChunkIndex, - externalLink, - thriftLink.getRowCount(), - thriftLink.getStartRowOffset())); + chunkLinks.add(externalLink); } LOGGER.debug( diff --git a/src/main/java/com/databricks/jdbc/model/core/ChunkLinkFetchResult.java b/src/main/java/com/databricks/jdbc/model/core/ChunkLinkFetchResult.java index 16739b768..51c2ad071 100644 --- a/src/main/java/com/databricks/jdbc/model/core/ChunkLinkFetchResult.java +++ b/src/main/java/com/databricks/jdbc/model/core/ChunkLinkFetchResult.java @@ -13,16 +13,18 @@ *

  • SEA: Uses chunkIndex for continuation, hasMore derived from nextChunkIndex on last link *
  • Thrift: Uses rowOffset for continuation, hasMore from server's hasMoreRows flag * + * + *

    Each {@link ExternalLink} contains chunkIndex, rowCount, rowOffset, and the download URL. */ public class ChunkLinkFetchResult { - private final List chunkLinks; + private final List chunkLinks; private final boolean hasMore; private final long nextFetchIndex; private final long nextRowOffset; private ChunkLinkFetchResult( - List chunkLinks, boolean hasMore, long nextFetchIndex, long nextRowOffset) { + List chunkLinks, boolean hasMore, long nextFetchIndex, long nextRowOffset) { this.chunkLinks = chunkLinks; this.hasMore = hasMore; this.nextFetchIndex = nextFetchIndex; @@ -32,14 +34,14 @@ private ChunkLinkFetchResult( /** * Creates a result with the given links and continuation info. * - * @param links The fetched chunk links + * @param links The fetched external links (each contains chunkIndex, rowCount, rowOffset, URL) * @param hasMore Whether more chunks are available * @param nextFetchIndex The next chunk index to fetch from, or -1 if no more * @param nextRowOffset The next row offset for Thrift FETCH_ABSOLUTE * @return A new ChunkLinkFetchResult */ public static ChunkLinkFetchResult of( - List links, boolean hasMore, long nextFetchIndex, long nextRowOffset) { + List links, boolean hasMore, long nextFetchIndex, long nextRowOffset) { return new ChunkLinkFetchResult(links, hasMore, nextFetchIndex, nextRowOffset); } @@ -53,11 +55,11 @@ public static ChunkLinkFetchResult endOfStream() { } /** - * Returns the list of chunk links fetched in this batch. + * Returns the list of external links fetched in this batch. * - * @return List of ChunkLinkInfo, may be empty + * @return List of ExternalLink, may be empty */ - public List getChunkLinks() { + public List getChunkLinks() { return chunkLinks; } @@ -96,35 +98,4 @@ public long getNextRowOffset() { public boolean isEndOfStream() { return !hasMore && chunkLinks.isEmpty(); } - - /** Information about a single chunk link. */ - public static class ChunkLinkInfo { - private final long chunkIndex; - private final ExternalLink link; - private final long rowCount; - private final long rowOffset; - - public ChunkLinkInfo(long chunkIndex, ExternalLink link, long rowCount, long rowOffset) { - this.chunkIndex = chunkIndex; - this.link = link; - this.rowCount = rowCount; - this.rowOffset = rowOffset; - } - - public long getChunkIndex() { - return chunkIndex; - } - - public ExternalLink getLink() { - return link; - } - - public long getRowCount() { - return rowCount; - } - - public long getRowOffset() { - return rowOffset; - } - } } diff --git a/src/test/java/com/databricks/jdbc/api/impl/arrow/ArrowStreamResultTest.java b/src/test/java/com/databricks/jdbc/api/impl/arrow/ArrowStreamResultTest.java index 6d3accf72..989e4cb4d 100644 --- a/src/test/java/com/databricks/jdbc/api/impl/arrow/ArrowStreamResultTest.java +++ b/src/test/java/com/databricks/jdbc/api/impl/arrow/ArrowStreamResultTest.java @@ -248,7 +248,9 @@ private List getChunkLinks(long chunkIndex, boolean isLast) { new ExternalLink() .setChunkIndex(chunkIndex) .setExternalLink(CHUNK_URL_PREFIX + chunkIndex) - .setExpiration(Instant.now().plusSeconds(3600L).toString()); + .setExpiration(Instant.now().plusSeconds(3600L).toString()) + .setRowOffset(chunkIndex * this.rowsInChunk) + .setRowCount(this.rowsInChunk); if (!isLast) { chunkLink.setNextChunkIndex(chunkIndex + 1); } @@ -293,25 +295,12 @@ private ChunkLinkFetchResult buildChunkLinkFetchResult(List links) return ChunkLinkFetchResult.endOfStream(); } - List chunkLinks = new ArrayList<>(); - for (ExternalLink link : links) { - chunkLinks.add( - new ChunkLinkFetchResult.ChunkLinkInfo( - link.getChunkIndex(), - link, - link.getRowCount() != null ? link.getRowCount() : 0, - link.getRowOffset() != null ? link.getRowOffset() : 0)); - } - ExternalLink lastLink = links.get(links.size() - 1); boolean hasMore = lastLink.getNextChunkIndex() != null; long nextFetchIndex = hasMore ? lastLink.getNextChunkIndex() : -1; - long nextRowOffset = 0; - if (lastLink.getRowOffset() != null && lastLink.getRowCount() != null) { - nextRowOffset = lastLink.getRowOffset() + lastLink.getRowCount(); - } + long nextRowOffset = lastLink.getRowOffset() + lastLink.getRowCount(); - return ChunkLinkFetchResult.of(chunkLinks, hasMore, nextFetchIndex, nextRowOffset); + return ChunkLinkFetchResult.of(links, hasMore, nextFetchIndex, nextRowOffset); } private File createTestArrowFile( diff --git a/src/test/java/com/databricks/jdbc/api/impl/arrow/ChunkLinkDownloadServiceTest.java b/src/test/java/com/databricks/jdbc/api/impl/arrow/ChunkLinkDownloadServiceTest.java index 85d4e0517..13207280a 100644 --- a/src/test/java/com/databricks/jdbc/api/impl/arrow/ChunkLinkDownloadServiceTest.java +++ b/src/test/java/com/databricks/jdbc/api/impl/arrow/ChunkLinkDownloadServiceTest.java @@ -280,6 +280,8 @@ private ExternalLink createExternalLink( link.setChunkIndex(chunkIndex); link.setHttpHeaders(headers); link.setExpiration(expiration); + link.setRowOffset(chunkIndex * 100L); + link.setRowCount(100L); return link; } @@ -293,24 +295,11 @@ private ChunkLinkFetchResult buildChunkLinkFetchResult(List links) return ChunkLinkFetchResult.endOfStream(); } - List chunkLinks = new ArrayList<>(); - for (ExternalLink link : links) { - chunkLinks.add( - new ChunkLinkFetchResult.ChunkLinkInfo( - link.getChunkIndex(), - link, - link.getRowCount() != null ? link.getRowCount() : 0, - link.getRowOffset() != null ? link.getRowOffset() : 0)); - } - ExternalLink lastLink = links.get(links.size() - 1); boolean hasMore = lastLink.getNextChunkIndex() != null; long nextFetchIndex = hasMore ? lastLink.getNextChunkIndex() : -1; - long nextRowOffset = 0; - if (lastLink.getRowOffset() != null && lastLink.getRowCount() != null) { - nextRowOffset = lastLink.getRowOffset() + lastLink.getRowCount(); - } + long nextRowOffset = lastLink.getRowOffset() + lastLink.getRowCount(); - return ChunkLinkFetchResult.of(chunkLinks, hasMore, nextFetchIndex, nextRowOffset); + return ChunkLinkFetchResult.of(links, hasMore, nextFetchIndex, nextRowOffset); } } diff --git a/src/test/java/com/databricks/jdbc/dbclient/impl/thrift/DatabricksThriftServiceClientTest.java b/src/test/java/com/databricks/jdbc/dbclient/impl/thrift/DatabricksThriftServiceClientTest.java index 62c6e022d..98ff3d274 100644 --- a/src/test/java/com/databricks/jdbc/dbclient/impl/thrift/DatabricksThriftServiceClientTest.java +++ b/src/test/java/com/databricks/jdbc/dbclient/impl/thrift/DatabricksThriftServiceClientTest.java @@ -25,6 +25,7 @@ import com.databricks.jdbc.exception.DatabricksSQLException; import com.databricks.jdbc.model.client.thrift.generated.*; import com.databricks.jdbc.model.core.ChunkLinkFetchResult; +import com.databricks.jdbc.model.core.ExternalLink; import com.databricks.jdbc.model.core.ResultColumn; import com.databricks.sdk.core.DatabricksConfig; import com.databricks.sdk.service.sql.StatementState; @@ -377,9 +378,9 @@ void testGetResultChunks() throws SQLException { Collections.singletonList(new TSparkArrowResultLink().setFileLink(TEST_STRING))); // Pass chunkIndex=0 and rowOffset=0 for the first chunk ChunkLinkFetchResult result = client.getResultChunks(TEST_STMT_ID, 0, 0); - List chunkLinks = result.getChunkLinks(); + List chunkLinks = result.getChunkLinks(); assertEquals(1, chunkLinks.size()); - assertEquals(TEST_STRING, chunkLinks.get(0).getLink().getExternalLink()); + assertEquals(TEST_STRING, chunkLinks.get(0).getExternalLink()); } @Test From 22fb391ab4bf35a3cf639d0739a7775929190338 Mon Sep 17 00:00:00 2001 From: Jayant Singh Date: Tue, 9 Dec 2025 09:15:58 +0000 Subject: [PATCH 6/6] fmt --- src/main/java/com/databricks/jdbc/common/util/DriverUtil.java | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/src/main/java/com/databricks/jdbc/common/util/DriverUtil.java b/src/main/java/com/databricks/jdbc/common/util/DriverUtil.java index 16187e05f..b6418f394 100644 --- a/src/main/java/com/databricks/jdbc/common/util/DriverUtil.java +++ b/src/main/java/com/databricks/jdbc/common/util/DriverUtil.java @@ -20,7 +20,7 @@ public class DriverUtil { private static final JdbcLogger LOGGER = JdbcLoggerFactory.getLogger(DriverUtil.class); - private static final String DRIVER_VERSION = "3.0.6"; + private static final String DRIVER_VERSION = "3.0.5"; private static final String DRIVER_NAME = "oss-jdbc"; private static final String JDBC_VERSION = "4.3";