diff --git a/CHANGELOG.md b/CHANGELOG.md index cb99e4f63..1f80c1058 100644 --- a/CHANGELOG.md +++ b/CHANGELOG.md @@ -4,6 +4,18 @@ All notable changes to this project will be documented in this file. The format is based on [Keep a Changelog](https://keepachangelog.com/en/1.0.0/), and this project adheres to [Semantic Versioning](https://semver.org/spec/v2.0.0.html). +## [UNRELEASED] + +### Changed + +- [BREAKING] All synchronous queued and streaming ingestion APIs now delegate to their asynchronous counterparts + internally and block for results. +- [BREAKING] Streaming client no longer check for blob size and if it exists. +- [BREAKING] Exceptions thrown the ingest API are now RuntimeExceptions: IngestionServiceException, IngestionClientException. +### Added +- The SDK now provides Reactor Core-based asynchronous APIs for all queued and streaming ingestion endpoints, + enabling non-blocking operations. + ## [6.0.2] - 2025-24-04 ### Fixed @@ -15,16 +27,19 @@ and this project adheres to [Semantic Versioning](https://semver.org/spec/v2.0.0 - The SDK now provides Reactor Core-based asynchronous APIs for all query, management, streaming query/ingestion (StreamingClient) endpoints, enabling non-blocking operations. You can read more about Reactor Core and [Mono type here](https://projectreactor.io/docs/core/release/api/). - `ConnectionStringBuilder` now supports keywords without regards to spaces or case. It now supports `toString()` that prints a canonical connection string, with censored secrets by default. + ### Changed - [BREAKING] All synchronous query/management, streaming query/ingestion (StreamingClient) APIs now delegate to their asynchronous counterparts internally and block for results. -- [BREAKING] * Make ManagedStreamingQueuingPolicy internal, expose just a factor -* Dont allow users to pass raw data size, provide it only if we have it +- [BREAKING] Make ManagedStreamingQueuingPolicy internal, expose just a factor +- [BREAKING] Don't allow users to pass raw data size, provide it only if we have it + - [BREAKING] Removing max keep alive from HttpClientPropertiesBuilder. ### Fixed - Fixed edge cases in query timeouts. - Long Queries would time out after 2 minutes. Remove keep alive timeout to fix. + ## [6.0.0-ALPHA-01] - 2024-11-27 ### Added - A new policy heuristic for choosing between queuing and streaming in Managed streaming client. A policy can be configured diff --git a/data/src/main/java/com/microsoft/azure/kusto/data/ExponentialRetry.java b/data/src/main/java/com/microsoft/azure/kusto/data/ExponentialRetry.java index 2e41f3ab9..f874d2444 100644 --- a/data/src/main/java/com/microsoft/azure/kusto/data/ExponentialRetry.java +++ b/data/src/main/java/com/microsoft/azure/kusto/data/ExponentialRetry.java @@ -3,6 +3,7 @@ import java.lang.invoke.MethodHandles; import java.time.Duration; import java.util.List; +import java.util.function.Predicate; import org.slf4j.Logger; import org.slf4j.LoggerFactory; @@ -12,7 +13,7 @@ import reactor.util.annotation.Nullable; import reactor.util.retry.Retry; -public class ExponentialRetry { +public class ExponentialRetry { private static final Logger log = LoggerFactory.getLogger(MethodHandles.lookup().lookupClass()); private final int maxAttempts; @@ -37,50 +38,25 @@ public ExponentialRetry(ExponentialRetry other) { this.maxJitterSecs = other.maxJitterSecs; } - // Caller should throw only permanent errors, returning null if a retry is needed - public T execute(KustoCheckedFunction function) throws E1, E2 { - for (int currentAttempt = 0; currentAttempt < maxAttempts; currentAttempt++) { - log.info("execute: Attempt {}", currentAttempt); - - try { - T result = function.apply(currentAttempt); - if (result != null) { - return result; - } - } catch (Exception e) { - log.error("execute: Error is permanent, stopping", e); - throw e; - } - - double currentSleepSecs = sleepBaseSecs * (float) Math.pow(2, currentAttempt); - double jitterSecs = (float) Math.random() * maxJitterSecs; - double sleepMs = (currentSleepSecs + jitterSecs) * 1000; - - log.info("execute: Attempt {} failed, trying again after sleep of {} seconds", currentAttempt, sleepMs / 1000); - - try { - Thread.sleep((long) sleepMs); - } catch (InterruptedException e) { - Thread.currentThread().interrupt(); - throw new RuntimeException("execute: Interrupted while sleeping", e); - } - } - - return null; - } - /** * Creates a retry mechanism with exponential backoff and jitter. * * @param retriableErrorClasses A list of error classes that are considered retriable. If null, * the method does not retry. + * @param filter A filter to use. Default is retrying retriable errors. * @return A configured {@link Retry} instance */ - public Retry retry(@Nullable List> retriableErrorClasses) { + public Retry retry(@Nullable List> retriableErrorClasses, + @Nullable Predicate filter) { + if (retriableErrorClasses != null && filter != null) { + throw new IllegalArgumentException("Cannot specify both retriableErrorClasses and filter"); + } + + Predicate filterToUse = filter == null ? throwable -> shouldRetry(throwable, retriableErrorClasses) : filter; return Retry.backoff(maxAttempts, Duration.ofSeconds((long) sleepBaseSecs)) .maxBackoff(Duration.ofSeconds(30)) .jitter(maxJitterSecs) - .filter(throwable -> shouldRetry(throwable, retriableErrorClasses)) + .filter(filterToUse) .doAfterRetry(retrySignal -> { long currentAttempt = retrySignal.totalRetries() + 1; log.info("Attempt {} failed.", currentAttempt); @@ -100,5 +76,4 @@ private static boolean shouldRetry(Throwable failure, List processGzipBody(Flux gzipBody) { // to occur in chunks, making it more memory-efficient for large payloads, as it prevents the entire // compressed stream from being loaded into memory at once (which for example is required by GZIPInputStream for decompression). - EmbeddedChannel channel = new EmbeddedChannel(ZlibCodecFactory.newZlibDecoder(ZlibWrapper.GZIP)); + EmbeddedChannel decoder = new EmbeddedChannel(ZlibCodecFactory.newZlibDecoder(ZlibWrapper.GZIP)); return gzipBody .reduce(new StringBuilder(), (stringBuilder, byteBuffer) -> { - channel.writeInbound(Unpooled.wrappedBuffer(byteBuffer)); // Write chunk to channel for decompression + decoder.writeInbound(Unpooled.wrappedBuffer(byteBuffer)); // Write chunk to channel for decompression - ByteBuf decompressedByteBuf = channel.readInbound(); + ByteBuf decompressedByteBuf = decoder.readInbound(); if (decompressedByteBuf == null) { return stringBuilder; } @@ -170,7 +170,7 @@ public static Mono processGzipBody(Flux gzipBody) { String string = decompressedByteBuf.toString(StandardCharsets.UTF_8); decompressedByteBuf.release(); - if (!channel.inboundMessages().isEmpty()) { + if (!decoder.inboundMessages().isEmpty()) { // TODO: remove this when we are sure that only one message exists in the channel throw new IllegalStateException("Expected exactly one message in the channel."); } @@ -178,7 +178,7 @@ public static Mono processGzipBody(Flux gzipBody) { return stringBuilder; }) .map(StringBuilder::toString) - .doFinally(ignore -> channel.finishAndReleaseAll()); + .doFinally(ignore -> decoder.finishAndReleaseAll()); } private static Mono processNonGzipBody(Flux gzipBody) { diff --git a/data/src/main/java/com/microsoft/azure/kusto/data/auth/CloudInfo.java b/data/src/main/java/com/microsoft/azure/kusto/data/auth/CloudInfo.java index f839d10b0..9e505a5c1 100644 --- a/data/src/main/java/com/microsoft/azure/kusto/data/auth/CloudInfo.java +++ b/data/src/main/java/com/microsoft/azure/kusto/data/auth/CloudInfo.java @@ -68,7 +68,7 @@ public class CloudInfo implements TraceableAttributes, Serializable { private final String kustoServiceResourceId; private final String firstPartyAuthorityUrl; private static final int ATTEMPT_COUNT = 3; - private static final Retry RETRY_CONFIG = new ExponentialRetry<>(ATTEMPT_COUNT).retry(null); + private static final Retry RETRY_CONFIG = new ExponentialRetry(ATTEMPT_COUNT).retry(null, null); public CloudInfo(boolean loginMfaRequired, String loginEndpoint, String kustoClientAppId, String kustoClientRedirectUri, String kustoServiceResourceId, String firstPartyAuthorityUrl) { diff --git a/data/src/main/java/com/microsoft/azure/kusto/data/http/HttpClientFactory.java b/data/src/main/java/com/microsoft/azure/kusto/data/http/HttpClientFactory.java index 93c42eeb2..d1126c4ed 100644 --- a/data/src/main/java/com/microsoft/azure/kusto/data/http/HttpClientFactory.java +++ b/data/src/main/java/com/microsoft/azure/kusto/data/http/HttpClientFactory.java @@ -60,8 +60,6 @@ public static HttpClient create(HttpClientProperties properties) { options.setProxyOptions(properties.getProxy()); } - // Todo: Is the per route connection maximum needed anymore? - return HttpClient.createDefault(options); } } diff --git a/data/src/test/java/com/microsoft/azure/kusto/data/exceptions/KustoServiceQueryErrorTest.java b/data/src/test/java/com/microsoft/azure/kusto/data/exceptions/KustoServiceQueryErrorTest.java index 42b9205b5..cfcee16dd 100644 --- a/data/src/test/java/com/microsoft/azure/kusto/data/exceptions/KustoServiceQueryErrorTest.java +++ b/data/src/test/java/com/microsoft/azure/kusto/data/exceptions/KustoServiceQueryErrorTest.java @@ -56,7 +56,6 @@ void testFromOneApiErrorArrayWithMultipleExceptionsOneApi() throws JsonProcessin ObjectMapper objectMapper = new ObjectMapper(); ArrayNode jsonExceptions = (ArrayNode) objectMapper.readTree(json).get("OneApiErrors"); - KustoServiceQueryError error = KustoServiceQueryError.fromOneApiErrorArray(jsonExceptions, true); assertEquals("Query execution failed with multiple inner exceptions:\n" + diff --git a/ingest/pom.xml b/ingest/pom.xml index dbe1df36e..ccc32dfb2 100644 --- a/ingest/pom.xml +++ b/ingest/pom.xml @@ -109,6 +109,9 @@ com.microsoft.azure:msal4j:jar io.projectreactor:reactor-core:jar com.fasterxml.jackson.core:jackson-core:jar + io.netty:netty-buffer:jar + io.netty:netty-codec:jar + io.netty:netty-transport:jar true @@ -156,7 +159,6 @@ com.microsoft.azure.kusto kusto-data ${project.parent.version} - compile @@ -276,5 +278,12 @@ vavr ${io.vavr.version} + + + io.projectreactor + reactor-test + ${reactor-test.version} + test + diff --git a/ingest/src/main/java/com/microsoft/azure/kusto/ingest/AzureStorageClient.java b/ingest/src/main/java/com/microsoft/azure/kusto/ingest/AzureStorageClient.java index 610bfa3c2..b50b346e1 100644 --- a/ingest/src/main/java/com/microsoft/azure/kusto/ingest/AzureStorageClient.java +++ b/ingest/src/main/java/com/microsoft/azure/kusto/ingest/AzureStorageClient.java @@ -4,138 +4,130 @@ package com.microsoft.azure.kusto.ingest; import com.azure.core.util.BinaryData; -import com.azure.data.tables.TableClient; +import com.azure.data.tables.TableAsyncClient; import com.azure.data.tables.implementation.models.TableServiceErrorException; import com.azure.data.tables.models.TableEntity; -import com.azure.storage.blob.BlobClient; -import com.azure.storage.blob.BlobContainerClient; -import com.azure.storage.blob.models.BlobStorageException; -import com.azure.storage.queue.QueueClient; -import com.azure.storage.queue.models.QueueStorageException; +import com.azure.storage.blob.BlobAsyncClient; +import com.azure.storage.blob.BlobContainerAsyncClient; +import com.azure.storage.queue.QueueAsyncClient; import com.microsoft.azure.kusto.data.Ensure; - +import com.microsoft.azure.kusto.ingest.utils.IngestionUtils; import org.apache.commons.codec.binary.Base64; import org.slf4j.Logger; import org.slf4j.LoggerFactory; +import reactor.core.Exceptions; +import reactor.core.publisher.Mono; import java.io.File; import java.io.IOException; import java.io.InputStream; -import java.io.OutputStream; import java.lang.invoke.MethodHandles; -import java.net.URISyntaxException; import java.nio.file.Files; -import java.util.zip.GZIPOutputStream; public class AzureStorageClient { private static final Logger log = LoggerFactory.getLogger(MethodHandles.lookup().lookupClass()); - private static final int GZIP_BUFFER_SIZE = 16384; - private static final int STREAM_BUFFER_SIZE = 16384; public AzureStorageClient() { } - void postMessageToQueue(QueueClient queueClient, String content) throws QueueStorageException { - // Ensure - Ensure.argIsNotNull(queueClient, "queueClient"); + Mono postMessageToQueue(QueueAsyncClient queueAsyncClient, String content) { + Ensure.argIsNotNull(queueAsyncClient, "queueAsyncClient"); Ensure.stringIsNotBlank(content, "content"); byte[] bytesEncoded = Base64.encodeBase64(content.getBytes()); - queueClient.sendMessage(BinaryData.fromBytes(bytesEncoded)); + return queueAsyncClient.sendMessage(BinaryData.fromBytes(bytesEncoded)).then(); } - public void azureTableInsertEntity(TableClient tableClient, TableEntity tableEntity) throws URISyntaxException, TableServiceErrorException { - Ensure.argIsNotNull(tableClient, "tableClient"); + public Mono azureTableInsertEntity(TableAsyncClient tableAsyncClient, TableEntity tableEntity) throws TableServiceErrorException { + Ensure.argIsNotNull(tableAsyncClient, "tableAsyncClient"); Ensure.argIsNotNull(tableEntity, "tableEntity"); - tableClient.createEntity(tableEntity); + return tableAsyncClient.createEntity(tableEntity); } - void uploadLocalFileToBlob(File file, String blobName, BlobContainerClient container, boolean shouldCompress) - throws IOException, BlobStorageException { - log.debug("uploadLocalFileToBlob: filePath: {}, blobName: {}, storageUri: {}", file.getPath(), blobName, container.getBlobContainerUrl()); + Mono uploadLocalFileToBlob(File file, String blobName, BlobContainerAsyncClient asyncContainer, boolean shouldCompress) throws IOException { + log.debug("uploadLocalFileToBlob: filePath: {}, blobName: {}, storageUri: {}", file.getPath(), blobName, asyncContainer.getBlobContainerUrl()); - // Ensure Ensure.fileExists(file, "sourceFile"); Ensure.stringIsNotBlank(blobName, "blobName"); - Ensure.argIsNotNull(container, "container"); + Ensure.argIsNotNull(asyncContainer, "asyncContainer"); - BlobClient blobClient = container.getBlobClient(blobName); + BlobAsyncClient blobAsyncClient = asyncContainer.getBlobAsyncClient(blobName); if (shouldCompress) { - compressAndUploadFileToBlob(file, blobClient); + return compressAndUploadFileToBlob(file, blobAsyncClient); } else { - uploadFileToBlob(file, blobClient); + return uploadFileToBlob(file, blobAsyncClient); } } - void compressAndUploadFileToBlob(File sourceFile, BlobClient blob) throws IOException, BlobStorageException { + Mono compressAndUploadFileToBlob(File sourceFile, BlobAsyncClient blobAsyncClient) throws IOException { Ensure.fileExists(sourceFile, "sourceFile"); - Ensure.argIsNotNull(blob, "blob"); - - try (InputStream fin = Files.newInputStream(sourceFile.toPath()); - GZIPOutputStream gzOut = new GZIPOutputStream(blob.getBlockBlobClient().getBlobOutputStream(true))) { - copyStream(fin, gzOut, GZIP_BUFFER_SIZE); - } + Ensure.argIsNotNull(blobAsyncClient, "blobAsyncClient"); + + return Mono.defer(() -> { + try { + InputStream inputStream = Files.newInputStream(sourceFile.toPath()); + return IngestionUtils.compressStream1(inputStream, false) + .flatMap(bytes -> blobAsyncClient.getBlockBlobAsyncClient() + .upload(BinaryData.fromStream(bytes, (long) bytes.available()), true)); + } catch (IOException e) { + throw Exceptions.propagate(e); + } + }).then(); } - void uploadFileToBlob(File sourceFile, BlobClient blobClient) throws IOException, BlobStorageException { - // Ensure - Ensure.argIsNotNull(blobClient, "blob"); + Mono uploadFileToBlob(File sourceFile, BlobAsyncClient blobAsyncClient) throws IOException { + Ensure.argIsNotNull(blobAsyncClient, "blob"); Ensure.fileExists(sourceFile, "sourceFile"); - blobClient.uploadFromFile(sourceFile.getPath()); + return blobAsyncClient.uploadFromFile(sourceFile.getPath()); } - int uploadStreamToBlob(InputStream inputStream, String blobName, BlobContainerClient container, boolean shouldCompress) - throws IOException, BlobStorageException { - log.debug("uploadStreamToBlob: blobName: {}, storageUri: {}", blobName, container); - - // Ensure + Mono uploadStreamToBlob(InputStream inputStream, + String blobName, + BlobContainerAsyncClient asyncContainer, + boolean shouldCompress) { Ensure.argIsNotNull(inputStream, "inputStream"); Ensure.stringIsNotBlank(blobName, "blobName"); - Ensure.argIsNotNull(container, "container"); + Ensure.argIsNotNull(asyncContainer, "asyncContainer"); + + log.debug("uploadStreamToBlob: blobName: {}, storageUri: {}", blobName, asyncContainer.getBlobContainerUrl()); - BlobClient blobClient = container.getBlobClient(blobName); + BlobAsyncClient blobAsyncClient = asyncContainer.getBlobAsyncClient(blobName); if (shouldCompress) { - return compressAndUploadStream(inputStream, blobClient); + return compressAndUploadStream(inputStream, blobAsyncClient); } else { - return uploadStream(inputStream, blobClient); + return uploadStream(inputStream, blobAsyncClient); } } // Returns original stream size - int uploadStream(InputStream inputStream, BlobClient blob) throws IOException, BlobStorageException { - // Ensure + Mono uploadStream(InputStream inputStream, BlobAsyncClient blobAsyncClient) { Ensure.argIsNotNull(inputStream, "inputStream"); - Ensure.argIsNotNull(blob, "blob"); + Ensure.argIsNotNull(blobAsyncClient, "blobAsyncClient"); + + IngestionUtils.IntegerHolder size = new IngestionUtils.IntegerHolder(); - OutputStream blobOutputStream = blob.getBlockBlobClient().getBlobOutputStream(true); - int originalSize = copyStream(inputStream, blobOutputStream, STREAM_BUFFER_SIZE); - blobOutputStream.close(); - return originalSize; + return IngestionUtils.toByteArray(inputStream) + .flatMap(bytes -> { + size.add(bytes.length); + return blobAsyncClient.getBlockBlobAsyncClient().upload(BinaryData.fromBytes(bytes), true); + }) + .thenReturn(size.getValue()); } // Returns original stream size - int compressAndUploadStream(InputStream inputStream, BlobClient blob) throws IOException, BlobStorageException { - // Ensure + Mono compressAndUploadStream(InputStream inputStream, BlobAsyncClient blobAsyncClient) { Ensure.argIsNotNull(inputStream, "inputStream"); - Ensure.argIsNotNull(blob, "blob"); - - try (GZIPOutputStream gzout = new GZIPOutputStream(blob.getBlockBlobClient().getBlobOutputStream(true))) { - return copyStream(inputStream, gzout, GZIP_BUFFER_SIZE); - } + Ensure.argIsNotNull(blobAsyncClient, "blobAsyncClient"); + + IngestionUtils.IntegerHolder size = new IngestionUtils.IntegerHolder(); + return IngestionUtils.toCompressedByteArray(inputStream, false) + .flatMap(bytes -> { + size.add(bytes.length); + return blobAsyncClient.getBlockBlobAsyncClient().upload(BinaryData.fromBytes(bytes), true); + }) + .map(x -> size.getValue()); } - // Returns original stream size - private int copyStream(InputStream inputStream, OutputStream outputStream, int bufferSize) throws IOException { - byte[] buffer = new byte[bufferSize]; - int length; - int size = 0; - while ((length = inputStream.read(buffer)) > 0) { - size += length; - outputStream.write(buffer, 0, length); - } - - return size; - } } diff --git a/ingest/src/main/java/com/microsoft/azure/kusto/ingest/IngestClient.java b/ingest/src/main/java/com/microsoft/azure/kusto/ingest/IngestClient.java index 96c45cb4f..f3f7508e3 100644 --- a/ingest/src/main/java/com/microsoft/azure/kusto/ingest/IngestClient.java +++ b/ingest/src/main/java/com/microsoft/azure/kusto/ingest/IngestClient.java @@ -10,6 +10,7 @@ import com.microsoft.azure.kusto.ingest.source.FileSourceInfo; import com.microsoft.azure.kusto.ingest.source.ResultSetSourceInfo; import com.microsoft.azure.kusto.ingest.source.StreamSourceInfo; +import reactor.core.publisher.Mono; import java.io.Closeable; @@ -31,6 +32,8 @@ public interface IngestClient extends Closeable { IngestionResult ingestFromFile(FileSourceInfo fileSourceInfo, IngestionProperties ingestionProperties) throws IngestionClientException, IngestionServiceException; + Mono ingestFromFileAsync(FileSourceInfo fileSourceInfo, IngestionProperties ingestionProperties); + /** *

Ingest data from a blob storage into Kusto database.

* This method ingests the data from a given blob, described in {@code blobSourceInfo}, into Kusto database, @@ -47,6 +50,8 @@ IngestionResult ingestFromFile(FileSourceInfo fileSourceInfo, IngestionPropertie IngestionResult ingestFromBlob(BlobSourceInfo blobSourceInfo, IngestionProperties ingestionProperties) throws IngestionClientException, IngestionServiceException; + Mono ingestFromBlobAsync(BlobSourceInfo blobSourceInfo, IngestionProperties ingestionProperties); + /** *

Ingest data from a Result Set into Kusto database.

* This method ingests the data from a given Result Set, described in {@code resultSetSourceInfo}, into Kusto database, @@ -66,6 +71,8 @@ IngestionResult ingestFromBlob(BlobSourceInfo blobSourceInfo, IngestionPropertie IngestionResult ingestFromResultSet(ResultSetSourceInfo resultSetSourceInfo, IngestionProperties ingestionProperties) throws IngestionClientException, IngestionServiceException; + Mono ingestFromResultSetAsync(ResultSetSourceInfo resultSetSourceInfo, IngestionProperties ingestionProperties); + /** *

Ingest data from an input stream, into Kusto database.

* This method ingests the data from a given input stream, described in {@code streamSourceInfo}, into Kusto database, @@ -81,4 +88,6 @@ IngestionResult ingestFromResultSet(ResultSetSourceInfo resultSetSourceInfo, Ing */ IngestionResult ingestFromStream(StreamSourceInfo streamSourceInfo, IngestionProperties ingestionProperties) throws IngestionClientException, IngestionServiceException; + + Mono ingestFromStreamAsync(StreamSourceInfo streamSourceInfo, IngestionProperties ingestionProperties); } diff --git a/ingest/src/main/java/com/microsoft/azure/kusto/ingest/IngestClientBase.java b/ingest/src/main/java/com/microsoft/azure/kusto/ingest/IngestClientBase.java index 9a2e430c0..49b992026 100644 --- a/ingest/src/main/java/com/microsoft/azure/kusto/ingest/IngestClientBase.java +++ b/ingest/src/main/java/com/microsoft/azure/kusto/ingest/IngestClientBase.java @@ -1,18 +1,20 @@ package com.microsoft.azure.kusto.ingest; import com.azure.core.util.CoreUtils; -import com.microsoft.azure.kusto.data.exceptions.ExceptionUtils; import com.microsoft.azure.kusto.data.instrumentation.MonitoredActivity; -import com.microsoft.azure.kusto.data.instrumentation.SupplierTwoExceptions; import com.microsoft.azure.kusto.data.instrumentation.TraceableAttributes; -import com.microsoft.azure.kusto.ingest.exceptions.IngestionClientException; -import com.microsoft.azure.kusto.ingest.exceptions.IngestionServiceException; import com.microsoft.azure.kusto.ingest.result.IngestionResult; -import com.microsoft.azure.kusto.ingest.source.*; import org.apache.http.conn.util.InetAddressUtils; -import java.io.IOException; import java.net.URI; + +import com.microsoft.azure.kusto.ingest.source.BlobSourceInfo; +import com.microsoft.azure.kusto.ingest.source.CompressionType; +import com.microsoft.azure.kusto.ingest.source.FileSourceInfo; +import com.microsoft.azure.kusto.ingest.source.ResultSetSourceInfo; +import com.microsoft.azure.kusto.ingest.source.StreamSourceInfo; +import reactor.core.publisher.Mono; + import java.util.HashMap; import java.util.Map; @@ -58,16 +60,18 @@ static boolean isReservedHostname(String rawUri) { return isLocalhost || isIpAddress || host.equalsIgnoreCase("onebox.dev.kusto.windows.net"); } - protected abstract IngestionResult ingestFromFileImpl(FileSourceInfo fileSourceInfo, IngestionProperties ingestionProperties) - throws IngestionClientException, IngestionServiceException; + public IngestionResult ingestFromFile(FileSourceInfo fileSourceInfo, IngestionProperties ingestionProperties) { + return ingestFromFileAsync(fileSourceInfo, ingestionProperties).block(); + } + + protected abstract Mono ingestFromFileAsyncImpl(FileSourceInfo fileSourceInfo, IngestionProperties ingestionProperties); - public IngestionResult ingestFromFile(FileSourceInfo fileSourceInfo, IngestionProperties ingestionProperties) - throws IngestionClientException, IngestionServiceException { - // trace ingestFromFile - return MonitoredActivity.invoke( - (SupplierTwoExceptions) () -> ingestFromFileImpl(fileSourceInfo, + public Mono ingestFromFileAsync(FileSourceInfo fileSourceInfo, IngestionProperties ingestionProperties) { + // trace ingestFromFileAsync + return Mono.defer(() -> MonitoredActivity.wrap( + ingestFromFileAsyncImpl(fileSourceInfo, ingestionProperties), - getClientType().concat(".ingestFromFile")); + getClientType().concat(".ingestFromFile"))); } /** @@ -78,21 +82,21 @@ public IngestionResult ingestFromFile(FileSourceInfo fileSourceInfo, IngestionPr * @param blobSourceInfo The specific SourceInfo to be ingested * @param ingestionProperties Settings used to customize the ingestion operation * @return {@link IngestionResult} object including the ingestion result - * @throws IngestionClientException An exception originating from a client activity - * @throws IngestionServiceException An exception returned from the service * @see BlobSourceInfo * @see IngestionProperties */ - protected abstract IngestionResult ingestFromBlobImpl(BlobSourceInfo blobSourceInfo, IngestionProperties ingestionProperties) - throws IngestionClientException, IngestionServiceException; + public IngestionResult ingestFromBlob(BlobSourceInfo blobSourceInfo, IngestionProperties ingestionProperties) { + return ingestFromBlobAsync(blobSourceInfo, ingestionProperties).block(); + } + + protected abstract Mono ingestFromBlobAsyncImpl(BlobSourceInfo blobSourceInfo, IngestionProperties ingestionProperties); - public IngestionResult ingestFromBlob(BlobSourceInfo blobSourceInfo, IngestionProperties ingestionProperties) - throws IngestionClientException, IngestionServiceException { + public Mono ingestFromBlobAsync(BlobSourceInfo blobSourceInfo, IngestionProperties ingestionProperties) { // trace ingestFromBlob - return MonitoredActivity.invoke( - (SupplierTwoExceptions) () -> ingestFromBlobImpl(blobSourceInfo, + return Mono.defer(() -> MonitoredActivity.wrap( + ingestFromBlobAsyncImpl(blobSourceInfo, ingestionProperties), - getClientType().concat(".ingestFromBlob")); + getClientType().concat(".ingestFromBlob"))); } /** @@ -106,21 +110,21 @@ public IngestionResult ingestFromBlob(BlobSourceInfo blobSourceInfo, IngestionPr * @param resultSetSourceInfo The specific SourceInfo to be ingested * @param ingestionProperties Settings used to customize the ingestion operation * @return {@link IngestionResult} object including the ingestion result - * @throws IngestionClientException An exception originating from a client activity - * @throws IngestionServiceException An exception returned from the service * @see ResultSetSourceInfo * @see IngestionProperties */ - protected abstract IngestionResult ingestFromResultSetImpl(ResultSetSourceInfo resultSetSourceInfo, IngestionProperties ingestionProperties) - throws IngestionClientException, IngestionServiceException; + public IngestionResult ingestFromResultSet(ResultSetSourceInfo resultSetSourceInfo, IngestionProperties ingestionProperties) { + return ingestFromResultSetAsync(resultSetSourceInfo, ingestionProperties).block(); + } + + protected abstract Mono ingestFromResultSetAsyncImpl(ResultSetSourceInfo resultSetSourceInfo, IngestionProperties ingestionProperties); - public IngestionResult ingestFromResultSet(ResultSetSourceInfo resultSetSourceInfo, IngestionProperties ingestionProperties) - throws IngestionClientException, IngestionServiceException { + public Mono ingestFromResultSetAsync(ResultSetSourceInfo resultSetSourceInfo, IngestionProperties ingestionProperties) { // trace ingestFromResultSet - return MonitoredActivity.invoke( - (SupplierTwoExceptions) () -> ingestFromResultSetImpl(resultSetSourceInfo, + return Mono.defer(() -> MonitoredActivity.wrap( + ingestFromResultSetAsyncImpl(resultSetSourceInfo, ingestionProperties), - getClientType().concat(".ingestFromResultSet")); + getClientType().concat(".ingestFromResultSet"))); } /** @@ -131,27 +135,21 @@ public IngestionResult ingestFromResultSet(ResultSetSourceInfo resultSetSourceIn * @param streamSourceInfo The specific SourceInfo to be ingested * @param ingestionProperties Settings used to customize the ingestion operation * @return {@link IngestionResult} object including the ingestion result - * @throws IngestionClientException An exception originating from a client activity - * @throws IngestionServiceException An exception returned from the service * @see StreamSourceInfo * @see IngestionProperties */ - protected abstract IngestionResult ingestFromStreamImpl(StreamSourceInfo streamSourceInfo, IngestionProperties ingestionProperties) - throws IngestionClientException, IngestionServiceException, IOException; + public IngestionResult ingestFromStream(StreamSourceInfo streamSourceInfo, IngestionProperties ingestionProperties) { + return ingestFromStreamAsync(streamSourceInfo, ingestionProperties).block(); + } - public IngestionResult ingestFromStream(StreamSourceInfo streamSourceInfo, IngestionProperties ingestionProperties) - throws IngestionClientException, IngestionServiceException { + protected abstract Mono ingestFromStreamAsyncImpl(StreamSourceInfo streamSourceInfo, IngestionProperties ingestionProperties); + + public Mono ingestFromStreamAsync(StreamSourceInfo streamSourceInfo, IngestionProperties ingestionProperties) { // trace ingestFromStream - return MonitoredActivity.invoke( - (SupplierTwoExceptions) () -> { - try { - return ingestFromStreamImpl(streamSourceInfo, - ingestionProperties); - } catch (IOException e) { - throw new IngestionServiceException(ExceptionUtils.getMessageEx(e), e); - } - }, - getClientType().concat(".ingestFromStream")); + return Mono.defer(() -> MonitoredActivity.wrap( + ingestFromStreamAsyncImpl(streamSourceInfo, + ingestionProperties), + getClientType().concat(".ingestFromStream"))); } protected Map getIngestionTraceAttributes(TraceableAttributes sourceInfo, TraceableAttributes ingestionProperties) { diff --git a/ingest/src/main/java/com/microsoft/azure/kusto/ingest/ManagedStreamingIngestClient.java b/ingest/src/main/java/com/microsoft/azure/kusto/ingest/ManagedStreamingIngestClient.java index 98bc4f25d..836b93f33 100644 --- a/ingest/src/main/java/com/microsoft/azure/kusto/ingest/ManagedStreamingIngestClient.java +++ b/ingest/src/main/java/com/microsoft/azure/kusto/ingest/ManagedStreamingIngestClient.java @@ -1,32 +1,46 @@ package com.microsoft.azure.kusto.ingest; import com.azure.core.http.HttpClient; -import com.azure.storage.blob.BlobClient; +import com.azure.core.http.HttpHeaders; +import com.azure.core.http.HttpResponse; +import com.azure.storage.blob.BlobAsyncClient; import com.azure.storage.blob.BlobClientBuilder; +import com.azure.storage.blob.models.BlobProperties; import com.azure.storage.blob.models.BlobStorageException; import com.azure.storage.common.policy.RequestRetryOptions; +import com.microsoft.azure.kusto.data.BaseClient; import com.microsoft.azure.kusto.data.Ensure; -import com.microsoft.azure.kusto.data.exceptions.ExceptionUtils; -import com.microsoft.azure.kusto.data.http.HttpClientProperties; +import com.microsoft.azure.kusto.data.ExponentialRetry; import com.microsoft.azure.kusto.data.StreamingClient; import com.microsoft.azure.kusto.data.auth.ConnectionStringBuilder; -import com.microsoft.azure.kusto.data.exceptions.DataServiceException; -import com.microsoft.azure.kusto.data.exceptions.DataWebException; -import com.microsoft.azure.kusto.data.exceptions.OneApiError; +import com.microsoft.azure.kusto.data.exceptions.*; +import com.microsoft.azure.kusto.data.http.HttpClientProperties; import com.microsoft.azure.kusto.ingest.exceptions.IngestionClientException; import com.microsoft.azure.kusto.ingest.exceptions.IngestionServiceException; import com.microsoft.azure.kusto.ingest.result.IngestionResult; -import com.microsoft.azure.kusto.ingest.source.*; -import com.microsoft.azure.kusto.data.ExponentialRetry; - +import com.microsoft.azure.kusto.ingest.source.BlobSourceInfo; +import com.microsoft.azure.kusto.ingest.source.FileSourceInfo; +import com.microsoft.azure.kusto.ingest.source.ResultSetSourceInfo; +import com.microsoft.azure.kusto.ingest.source.SourceInfo; +import com.microsoft.azure.kusto.ingest.source.StreamSourceInfo; import com.microsoft.azure.kusto.ingest.utils.IngestionUtils; import org.jetbrains.annotations.Nullable; import org.slf4j.Logger; import org.slf4j.LoggerFactory; - -import java.io.*; +import reactor.core.publisher.Flux; +import reactor.core.publisher.Mono; +import reactor.core.scheduler.Schedulers; +import reactor.util.retry.Retry; + +import java.io.ByteArrayInputStream; +import java.io.FileNotFoundException; +import java.io.IOException; +import java.io.InputStream; +import java.io.SequenceInputStream; import java.lang.invoke.MethodHandles; import java.net.URISyntaxException; +import java.nio.ByteBuffer; +import java.nio.charset.Charset; import java.util.UUID; /** @@ -35,7 +49,7 @@ * This class combines a managed streaming client with a queued streaming client, to create an optimized experience. * Since the streaming client communicates directly with the engine, it's more prone to failure, so this class * holds both a streaming client and a queued client. - * It tries {@value ATTEMPT_COUNT} times using the streaming client, after which it falls back to the queued streaming client in case of failure. + * It retries {@value RETRY_COUNT} times using the streaming client, after which it falls back to the queued streaming client in case of failure. * By default the policy for choosing a queued ingestion on the first try is the checking of weather the size of the estimated * raw stream size (a conversion to compressed CSV) is bigger than 4MB, it will fall back to the queued streaming client. * Use {@link #setQueuingPolicyFactor(double)} to override the predicate heuristics. @@ -44,14 +58,16 @@ public class ManagedStreamingIngestClient extends IngestClientBase implements QueuedIngestClient { private static final Logger log = LoggerFactory.getLogger(MethodHandles.lookup().lookupClass()); - public static final int ATTEMPT_COUNT = 3; + // 2 Retry count ends with total 3 streaming attempts + public static final int RETRY_COUNT = 2; public static final String CLASS_NAME = ManagedStreamingIngestClient.class.getSimpleName(); final QueuedIngestClient queuedIngestClient; final StreamingIngestClient streamingIngestClient; - private final ExponentialRetry exponentialRetryTemplate; + private ExponentialRetry exponentialRetryTemplate = new ExponentialRetry(RETRY_COUNT); + private Retry streamingRetry = new ExponentialRetry(exponentialRetryTemplate).retry(null, this::streamingIngestionErrorPredicate); private HttpClient httpClient = null; private ManagedStreamingQueuingPolicy queuingPolicy = ManagedStreamingQueuingPolicy.Default; - private static final String fallbackLogString = "Data size for source id '%s' is greater than max streaming size according to the policy. Falling back to queued."; + private static final String FALLBACK_LOG_STRING = "Data size for source id '%s' is greater than max streaming size according to the policy. Falling back to queued."; /** * @param dmConnectionString dm connection string @@ -155,7 +171,6 @@ public ManagedStreamingIngestClient(ConnectionStringBuilder ingestionEndpointCon log.info("Creating a new ManagedStreamingIngestClient from connection strings"); queuedIngestClient = new QueuedIngestClientImpl(ingestionEndpointConnectionStringBuilder, properties, autoCorrectEndpoint); streamingIngestClient = new StreamingIngestClient(queryEndpointConnectionStringBuilder, properties, autoCorrectEndpoint); - exponentialRetryTemplate = new ExponentialRetry(ATTEMPT_COUNT); } /** @@ -169,7 +184,6 @@ public ManagedStreamingIngestClient(ConnectionStringBuilder connectionStringBuil log.info("Creating a new ManagedStreamingIngestClient from connection strings"); queuedIngestClient = new QueuedIngestClientImpl(connectionStringBuilder, properties, autoCorrectEndpoint); streamingIngestClient = new StreamingIngestClient(connectionStringBuilder, properties, autoCorrectEndpoint); - exponentialRetryTemplate = new ExponentialRetry(ATTEMPT_COUNT); } /** @@ -184,7 +198,6 @@ public ManagedStreamingIngestClient(ConnectionStringBuilder connectionStringBuil queuedIngestClient = new QueuedIngestClientImpl(connectionStringBuilder, httpClient, autoCorrectEndpoint); streamingIngestClient = new StreamingIngestClient(connectionStringBuilder, httpClient, autoCorrectEndpoint); this.httpClient = httpClient; - exponentialRetryTemplate = new ExponentialRetry(ATTEMPT_COUNT); } /** @@ -199,7 +212,6 @@ public ManagedStreamingIngestClient(ConnectionStringBuilder ingestionEndpointCon log.info("Creating a new ManagedStreamingIngestClient from connection strings"); queuedIngestClient = new QueuedIngestClientImpl(ingestionEndpointConnectionStringBuilder, properties, true); streamingIngestClient = new StreamingIngestClient(queryEndpointConnectionStringBuilder, properties, true); - exponentialRetryTemplate = new ExponentialRetry(ATTEMPT_COUNT); } /*** @@ -212,7 +224,6 @@ public ManagedStreamingIngestClient(ConnectionStringBuilder connectionStringBuil log.info("Creating a new ManagedStreamingIngestClient from connection strings"); queuedIngestClient = new QueuedIngestClientImpl(connectionStringBuilder, properties, true); streamingIngestClient = new StreamingIngestClient(connectionStringBuilder, properties, true); - exponentialRetryTemplate = new ExponentialRetry(ATTEMPT_COUNT); } /*** @@ -226,7 +237,6 @@ public ManagedStreamingIngestClient(ConnectionStringBuilder connectionStringBuil queuedIngestClient = new QueuedIngestClientImpl(connectionStringBuilder, httpClient, true); streamingIngestClient = new StreamingIngestClient(connectionStringBuilder, httpClient, true); this.httpClient = httpClient; - exponentialRetryTemplate = new ExponentialRetry(ATTEMPT_COUNT); } /** @@ -242,7 +252,6 @@ public ManagedStreamingIngestClient(ResourceManager resourceManager, log.info("Creating a new ManagedStreamingIngestClient from raw parts"); queuedIngestClient = new QueuedIngestClientImpl(resourceManager, storageClient); streamingIngestClient = new StreamingIngestClient(streamingClient); - exponentialRetryTemplate = new ExponentialRetry(ATTEMPT_COUNT); } /** @@ -264,36 +273,29 @@ public ManagedStreamingIngestClient(ResourceManager resourceManager, ManagedStreamingIngestClient(StreamingIngestClient streamingIngestClient, QueuedIngestClient queuedIngestClient, ExponentialRetry exponentialRetry) { this.streamingIngestClient = streamingIngestClient; this.queuedIngestClient = queuedIngestClient; - exponentialRetryTemplate = exponentialRetry; + streamingRetry = new ExponentialRetry(exponentialRetry).retry(null, this::streamingIngestionErrorPredicate); } @Override - protected IngestionResult ingestFromFileImpl(FileSourceInfo fileSourceInfo, IngestionProperties ingestionProperties) - throws IngestionClientException, IngestionServiceException { + protected Mono ingestFromFileAsyncImpl(FileSourceInfo fileSourceInfo, IngestionProperties ingestionProperties) { Ensure.argIsNotNull(fileSourceInfo, "fileSourceInfo"); Ensure.argIsNotNull(ingestionProperties, "ingestionProperties"); - fileSourceInfo.validate(); ingestionProperties.validate(); + try { StreamSourceInfo streamSourceInfo = IngestionUtils.fileToStream(fileSourceInfo, true); - return ingestFromStream(streamSourceInfo, ingestionProperties); + return ingestFromStreamAsync(streamSourceInfo, ingestionProperties); } catch (FileNotFoundException e) { log.error("File not found when ingesting a file.", e); throw new IngestionClientException("IO exception - check file path.", e); } } - /** - * {@inheritDoc} - *

- */ @Override - protected IngestionResult ingestFromBlobImpl(BlobSourceInfo blobSourceInfo, IngestionProperties ingestionProperties) - throws IngestionClientException, IngestionServiceException { + protected Mono ingestFromBlobAsyncImpl(BlobSourceInfo blobSourceInfo, IngestionProperties ingestionProperties) { Ensure.argIsNotNull(blobSourceInfo, "blobSourceInfo"); Ensure.argIsNotNull(ingestionProperties, "ingestionProperties"); - blobSourceInfo.validate(); ingestionProperties.validate(); @@ -301,162 +303,186 @@ protected IngestionResult ingestFromBlobImpl(BlobSourceInfo blobSourceInfo, Inge if (httpClient != null) { blobClientBuilder.httpClient(httpClient); } + BlobAsyncClient blobAsyncClient = blobClientBuilder.buildAsyncClient(); + + return (blobSourceInfo.getBlobExactSize() != null + ? Mono.just(blobSourceInfo.getBlobExactSize()) + : blobAsyncClient.getProperties().map(BlobProperties::getBlobSize)) + .onErrorMap(BlobStorageException.class, e -> new IngestionServiceException( + blobSourceInfo.getBlobPath(), + "Failed getting blob properties: " + ExceptionUtils.getMessageEx(e), + e)) + .flatMap(blobSize -> handleBlobIngestion(blobSourceInfo, ingestionProperties, blobAsyncClient, blobSize)); + } - BlobClient blobClient = blobClientBuilder.buildClient(); - long blobSize = 0; - if (blobSourceInfo.getBlobExactSize() != null) { - try { - blobSize = blobClient.getProperties().getBlobSize(); - } catch (BlobStorageException e) { - throw new IngestionServiceException( - blobSourceInfo.getBlobPath(), - "Failed getting blob properties: " + ExceptionUtils.getMessageEx(e), - e); - } + private Mono handleBlobIngestion(BlobSourceInfo blobSourceInfo, + IngestionProperties ingestionProperties, + BlobAsyncClient blobAsyncClient, + long blobSize) { + if (queuingPolicy.shouldUseQueuedIngestion(blobSize, blobSourceInfo.getCompressionType() != null, ingestionProperties.getDataFormat())) { + log.info(String.format(FALLBACK_LOG_STRING, blobSourceInfo.getSourceId())); + return queuedIngestClient.ingestFromBlobAsync(blobSourceInfo, ingestionProperties); } + IngestionUtils.IntegerHolder i = new IngestionUtils.IntegerHolder(); + + // If an error occurs, each time the retryWhen subscribes to executeStream create a new instance + // instead of using the same executeStream Mono for all retries + return Mono.defer(() -> executeStream(blobSourceInfo, ingestionProperties, blobAsyncClient, i.increment())) + .retryWhen(streamingRetry) + .onErrorResume(e -> { + if (streamingIngestionErrorPredicate(e)) { + log.info("Streaming ingestion failed for sourceId: {}, falling back to queued ingestion.", blobSourceInfo.getSourceId()); + return queuedIngestClient.ingestFromBlobAsync(blobSourceInfo, ingestionProperties); + } + return Mono.error(e); + }); // Fall back to queued ingestion + } - if (queuingPolicy.shouldUseQueuedIngestion(blobSize, - blobSourceInfo.getCompressionType() != null, ingestionProperties.getDataFormat())) { - log.info(String.format(fallbackLogString, blobSourceInfo.getSourceId())); - return queuedIngestClient.ingestFromBlob(blobSourceInfo, ingestionProperties); + private void resetStream(StreamSourceInfo sourceInfo) { + try { + sourceInfo.getStream().reset(); + } catch (IOException ioException) { + throw new IngestionClientException("Failed to reset stream for retry", ioException); } + } - IngestionResult result = streamWithRetries(blobSourceInfo, ingestionProperties, blobClient); - if (result != null) { - return result; + private Mono executeStream(SourceInfo sourceInfo, IngestionProperties ingestionProperties, @Nullable BlobAsyncClient blobAsyncClient, + int currentAttempt) { + if (blobAsyncClient != null) { + String clientRequestId = String.format("KJC.executeManagedStreamingIngest.ingestFromBlob;%s;%d", sourceInfo.getSourceId(), currentAttempt); + return streamingIngestClient.ingestFromBlobAsync((BlobSourceInfo) sourceInfo, ingestionProperties, clientRequestId); } - return queuedIngestClient.ingestFromBlob(blobSourceInfo, ingestionProperties); + + String clientRequestId = String.format("KJC.executeManagedStreamingIngest.ingestFromStream;%s;%d", sourceInfo.getSourceId(), currentAttempt); + return streamingIngestClient.ingestFromStreamAsync((StreamSourceInfo) sourceInfo, ingestionProperties, clientRequestId); } - private IngestionResult streamWithRetries(SourceInfo sourceInfo, IngestionProperties ingestionProperties, @Nullable BlobClient blobClient) - throws IngestionClientException, IngestionServiceException { - ExponentialRetry retry = new ExponentialRetry<>( - exponentialRetryTemplate); - return retry.execute(currentAttempt -> { - try { - if (blobClient != null) { - String clientRequestId = String.format("KJC.executeManagedStreamingIngest.ingestFromBlob;%s;%d", sourceInfo.getSourceId(), currentAttempt); - return streamingIngestClient.ingestFromBlob((BlobSourceInfo) sourceInfo, ingestionProperties, clientRequestId); - } else { - String clientRequestId = String.format("KJC.executeManagedStreamingIngest.ingestFromStream;%s;%d", sourceInfo.getSourceId(), - currentAttempt); - return streamingIngestClient.ingestFromStream((StreamSourceInfo) sourceInfo, ingestionProperties, clientRequestId); - } - } catch (Exception e) { - if (e instanceof IngestionServiceException - && e.getCause() != null - && e.getCause() instanceof DataServiceException - && e.getCause().getCause() != null - && e.getCause().getCause() instanceof DataWebException) { - DataWebException webException = (DataWebException) e.getCause().getCause(); - OneApiError oneApiError = webException.getApiError(); - if (oneApiError.isPermanent()) { - throw e; - } - } - log.info(String.format("Streaming ingestion failed attempt %d", currentAttempt), e); + private boolean streamingIngestionErrorPredicate(Throwable e) { + if (e instanceof IngestionServiceException + && e.getCause() != null + && e.getCause() instanceof DataServiceException + && e.getCause().getCause() != null + && e.getCause().getCause() instanceof DataWebException) { + DataWebException webException = (DataWebException) e.getCause().getCause(); + OneApiError oneApiError = webException.getApiError(); + if (oneApiError.isPermanent()) { + return false; + } + } - if (sourceInfo instanceof StreamSourceInfo) { - try { - ((StreamSourceInfo) sourceInfo).getStream().reset(); - } catch (IOException ioException) { - throw new IngestionClientException("Failed to reset stream", ioException); - } - } + log.info("Streaming ingestion failed.", e); - } - return null; - }); + return true; } @Override - protected IngestionResult ingestFromResultSetImpl(ResultSetSourceInfo resultSetSourceInfo, IngestionProperties ingestionProperties) - throws IngestionClientException, IngestionServiceException { + protected Mono ingestFromResultSetAsyncImpl(ResultSetSourceInfo resultSetSourceInfo, IngestionProperties ingestionProperties) { Ensure.argIsNotNull(resultSetSourceInfo, "resultSetSourceInfo"); Ensure.argIsNotNull(ingestionProperties, "ingestionProperties"); - resultSetSourceInfo.validate(); ingestionProperties.validateResultSetProperties(); - try { - StreamSourceInfo streamSourceInfo = IngestionUtils.resultSetToStream(resultSetSourceInfo); - return ingestFromStream(streamSourceInfo, ingestionProperties); - } catch (IOException ex) { - String msg = "Failed to read from ResultSet."; - log.error(msg, ex); - throw new IngestionClientException(msg, ex); - } + return Mono.fromCallable(() -> IngestionUtils.resultSetToStream(resultSetSourceInfo)) + .subscribeOn(Schedulers.boundedElastic()) + .onErrorMap(IOException.class, e -> { + String msg = "Failed to read from ResultSet."; + log.error(msg, e); + return new IngestionClientException(msg, e); + }) + .flatMap(streamSourceInfo -> ingestFromStreamAsync(streamSourceInfo, ingestionProperties)); } @Override - protected IngestionResult ingestFromStreamImpl(StreamSourceInfo streamSourceInfo, IngestionProperties ingestionProperties) - throws IngestionClientException, IngestionServiceException, IOException { + protected Mono ingestFromStreamAsyncImpl(StreamSourceInfo streamSourceInfo, IngestionProperties ingestionProperties) { Ensure.argIsNotNull(streamSourceInfo, "streamSourceInfo"); Ensure.argIsNotNull(ingestionProperties, "ingestionProperties"); - streamSourceInfo.validate(); ingestionProperties.validate(); - UUID sourceId = streamSourceInfo.getSourceId(); - if (sourceId == null) { - sourceId = UUID.randomUUID(); + if (streamSourceInfo.getSourceId() == null) { + streamSourceInfo.setSourceId(UUID.randomUUID()); } - streamSourceInfo.setSourceId(sourceId); - byte[] streamingBytes; - InputStream byteArrayStream; - - if (queuingPolicy.shouldUseQueuedIngestion(streamSourceInfo.getStream().available(), - streamSourceInfo.getCompressionType() != null, ingestionProperties.getDataFormat())) { - log.info(String.format(fallbackLogString, sourceId)); - return queuedIngestClient.ingestFromStream(streamSourceInfo, ingestionProperties); + try { + int availableBytes = streamSourceInfo.getStream().available(); + boolean shouldUseQueuedIngestion = queuingPolicy.shouldUseQueuedIngestion( + availableBytes, + streamSourceInfo.getCompressionType() != null, + ingestionProperties.getDataFormat()); + return shouldUseQueuedIngestion + ? queuedIngestClient.ingestFromStreamAsync(streamSourceInfo, ingestionProperties) + : processStream(streamSourceInfo, ingestionProperties); + } catch (IOException e) { + throw new IngestionClientException("Failed to read from stream.", e); } + } + private Mono ingestStreamWithRetries(StreamSourceInfo streamSourceInfo, IngestionProperties ingestionProperties) { + IngestionUtils.IntegerHolder integerHolder = new IngestionUtils.IntegerHolder(); + return Mono.defer(() -> executeStream(streamSourceInfo, ingestionProperties, null, integerHolder.increment())) + .doOnError((e) -> resetStream(streamSourceInfo)) + .retryWhen(streamingRetry) + .onErrorResume(e -> { + // If the error is not recoverable, we should not fall back to queued ingestion + if (streamingIngestionErrorPredicate(e)) { + log.info("Streaming ingestion failed for sourceId: {}, falling back to queued ingestion.", streamSourceInfo.getSourceId()); + return queuedIngestClient.ingestFromStreamAsync(streamSourceInfo, ingestionProperties); + } + + return Mono.error(e); + }) + .doFinally(signal -> closeStreamSafely(streamSourceInfo)); + } + + private Mono processStream(StreamSourceInfo streamSourceInfo, IngestionProperties ingestionProperties) { + + if (streamSourceInfo.getStream() instanceof ByteArrayInputStream || streamSourceInfo.getStream() instanceof ResettableFileInputStream) { + StreamSourceInfo managedSourceInfo = new StreamSourceInfo(streamSourceInfo.getStream(), + true, streamSourceInfo.getSourceId(), streamSourceInfo.getCompressionType()); + return ingestStreamWithRetries(managedSourceInfo, ingestionProperties); + } try { - if (streamSourceInfo.getStream() instanceof ByteArrayInputStream || streamSourceInfo.getStream() instanceof ResettableFileInputStream) { - byteArrayStream = streamSourceInfo.getStream(); - } else { - // If its not a ByteArrayInputStream: - // Read 10mb (max streaming size), decide with that if we should stream - streamingBytes = IngestionUtils.readBytesFromInputStream(streamSourceInfo.getStream(), - ManagedStreamingQueuingPolicy.MAX_STREAMING_STREAM_SIZE_BYTES + 1); - byteArrayStream = new ByteArrayInputStream(streamingBytes); - int size = streamingBytes.length; - if (queuingPolicy.shouldUseQueuedIngestion(size, - streamSourceInfo.getCompressionType() != null, ingestionProperties.getDataFormat())) { - StreamSourceInfo managedSourceInfo = new StreamSourceInfo(new SequenceInputStream(byteArrayStream, streamSourceInfo.getStream()), - streamSourceInfo.isLeaveOpen(), sourceId, streamSourceInfo.getCompressionType()); - log.info(String.format(fallbackLogString, managedSourceInfo.getSourceId())); - - return queuedIngestClient.ingestFromStream(managedSourceInfo, ingestionProperties); - } + byte[] streamingBytes = IngestionUtils.readBytesFromInputStream(streamSourceInfo.getStream(), + ManagedStreamingQueuingPolicy.MAX_STREAMING_STREAM_SIZE_BYTES + 1); - if (!streamSourceInfo.isLeaveOpen()) { - // From this point we don't need the original stream anymore, we cached it - try { - streamSourceInfo.getStream().close(); - } catch (IOException e) { - log.warn("Failed to close stream", e); - } + InputStream byteArrayStream = new ByteArrayInputStream(streamingBytes); + int size = streamingBytes.length; + + boolean shouldUseQueuedIngestion = queuingPolicy.shouldUseQueuedIngestion( + size, + streamSourceInfo.getCompressionType() != null, + ingestionProperties.getDataFormat()); + + if (shouldUseQueuedIngestion) { + log.info(String.format(FALLBACK_LOG_STRING, streamSourceInfo.getSourceId())); + StreamSourceInfo managedSourceInfo = new StreamSourceInfo(new SequenceInputStream(byteArrayStream, streamSourceInfo.getStream()), + streamSourceInfo.isLeaveOpen(), streamSourceInfo.getSourceId(), streamSourceInfo.getCompressionType()); + + return queuedIngestClient.ingestFromStreamAsync(managedSourceInfo, ingestionProperties); + } + + if (!streamSourceInfo.isLeaveOpen()) { + + // From this point we don't need the original stream anymore, we cached it + try { + streamSourceInfo.getStream().close(); + } catch (IOException e) { + log.warn("Failed to close stream", e); } } + + StreamSourceInfo managedSourceInfo = new StreamSourceInfo(byteArrayStream, + true, streamSourceInfo.getSourceId(), streamSourceInfo.getCompressionType()); + return ingestStreamWithRetries(managedSourceInfo, ingestionProperties); } catch (IOException e) { throw new IngestionClientException("Failed to read from stream.", e); } + } - StreamSourceInfo managedSourceInfo = new StreamSourceInfo(byteArrayStream, true, sourceId, streamSourceInfo.getCompressionType()); + private void closeStreamSafely(StreamSourceInfo streamSourceInfo) { try { - IngestionResult result = streamWithRetries(managedSourceInfo, ingestionProperties, null); - if (result != null) { - return result; - } - - return queuedIngestClient.ingestFromStream(managedSourceInfo, ingestionProperties); - } finally { - try { - managedSourceInfo.getStream().close(); - } catch (IOException e) { - log.warn("Failed to close byte stream", e); - } + streamSourceInfo.getStream().close(); + } catch (IOException e) { + log.warn("Failed to close byte stream", e); } } @@ -466,6 +492,7 @@ protected IngestionResult ingestFromStreamImpl(StreamSourceInfo streamSourceInfo * use queued ingestion instead of trying streaming ingestion first. * Setting the factor will create a new {@link ManagedStreamingQueuingPolicy} with this factor, which will be used * in the future ingestion calls. + * * @param factor - Default is 1. **/ public void setQueuingPolicyFactor(double factor) { diff --git a/ingest/src/main/java/com/microsoft/azure/kusto/ingest/QueuedIngestClientImpl.java b/ingest/src/main/java/com/microsoft/azure/kusto/ingest/QueuedIngestClientImpl.java index 96312718a..30e3db3ea 100644 --- a/ingest/src/main/java/com/microsoft/azure/kusto/ingest/QueuedIngestClientImpl.java +++ b/ingest/src/main/java/com/microsoft/azure/kusto/ingest/QueuedIngestClientImpl.java @@ -9,14 +9,27 @@ import com.azure.storage.blob.models.BlobStorageException; import com.azure.storage.common.policy.RequestRetryOptions; import com.azure.storage.queue.models.QueueStorageException; -import com.microsoft.azure.kusto.data.*; +import com.microsoft.azure.kusto.data.Client; +import com.microsoft.azure.kusto.data.ClientDetails; +import com.microsoft.azure.kusto.data.ClientFactory; +import com.microsoft.azure.kusto.data.Ensure; +import com.microsoft.azure.kusto.data.UriUtils; import com.microsoft.azure.kusto.data.auth.ConnectionStringBuilder; import com.microsoft.azure.kusto.data.http.HttpClientFactory; import com.microsoft.azure.kusto.data.http.HttpClientProperties; import com.microsoft.azure.kusto.ingest.exceptions.IngestionClientException; import com.microsoft.azure.kusto.ingest.exceptions.IngestionServiceException; -import com.microsoft.azure.kusto.ingest.result.*; -import com.microsoft.azure.kusto.ingest.source.*; +import com.microsoft.azure.kusto.ingest.result.IngestionResult; +import com.microsoft.azure.kusto.ingest.result.IngestionStatus; +import com.microsoft.azure.kusto.ingest.result.IngestionStatusInTableDescription; +import com.microsoft.azure.kusto.ingest.result.IngestionStatusResult; +import com.microsoft.azure.kusto.ingest.result.OperationStatus; +import com.microsoft.azure.kusto.ingest.result.TableReportIngestionResult; +import com.microsoft.azure.kusto.ingest.source.BlobSourceInfo; +import com.microsoft.azure.kusto.ingest.source.CompressionType; +import com.microsoft.azure.kusto.ingest.source.FileSourceInfo; +import com.microsoft.azure.kusto.ingest.source.ResultSetSourceInfo; +import com.microsoft.azure.kusto.ingest.source.StreamSourceInfo; import com.microsoft.azure.kusto.ingest.utils.IngestionUtils; import com.microsoft.azure.kusto.ingest.utils.SecurityUtils; import com.microsoft.azure.kusto.ingest.utils.TableWithSas; @@ -25,6 +38,8 @@ import org.apache.commons.lang3.tuple.Pair; import org.slf4j.Logger; import org.slf4j.LoggerFactory; +import reactor.core.publisher.Mono; +import reactor.core.scheduler.Schedulers; import java.io.ByteArrayInputStream; import java.io.ByteArrayOutputStream; @@ -35,6 +50,7 @@ import java.time.Instant; import java.util.LinkedList; import java.util.List; +import java.util.Map; import java.util.UUID; public class QueuedIngestClientImpl extends IngestClientBase implements QueuedIngestClient { @@ -80,159 +96,172 @@ public IngestionResourceManager getResourceManager() { } @Override - protected IngestionResult ingestFromBlobImpl(BlobSourceInfo blobSourceInfo, IngestionProperties ingestionProperties) - throws IngestionClientException, IngestionServiceException { - // Argument validation: + protected Mono ingestFromBlobAsyncImpl(BlobSourceInfo blobSourceInfo, IngestionProperties ingestionProperties) { Ensure.argIsNotNull(blobSourceInfo, "blobSourceInfo"); Ensure.argIsNotNull(ingestionProperties, "ingestionProperties"); - blobSourceInfo.validate(); ingestionProperties.validate(); + ingestionProperties.setAuthorizationContextToken(resourceManager.getIdentityToken()); + + // Create the ingestion message + IngestionBlobInfo ingestionBlobInfo = new IngestionBlobInfo(blobSourceInfo.getBlobPath(), + ingestionProperties.getDatabaseName(), ingestionProperties.getTableName(), this.applicationForTracing, + this.clientVersionForTracing); + String urlWithoutSecrets = SecurityUtils.removeSecretsFromUrl(blobSourceInfo.getBlobPath()); + if (blobSourceInfo.getBlobExactSize() != null) { + ingestionBlobInfo.setRawDataSize(blobSourceInfo.getBlobExactSize()); + } else { + log.warn("Blob '{}' was sent for ingestion without specifying its raw data size", urlWithoutSecrets); + } + Map properties; try { - ingestionProperties.setAuthorizationContextToken(resourceManager.getIdentityToken()); - List tableStatuses = new LinkedList<>(); - - // Create the ingestion message - IngestionBlobInfo ingestionBlobInfo = new IngestionBlobInfo(blobSourceInfo.getBlobPath(), - ingestionProperties.getDatabaseName(), ingestionProperties.getTableName(), this.applicationForTracing, - this.clientVersionForTracing); - String urlWithoutSecrets = SecurityUtils.removeSecretsFromUrl(blobSourceInfo.getBlobPath()); - if (blobSourceInfo.getBlobExactSize() != null) { - ingestionBlobInfo.setRawDataSize(blobSourceInfo.getBlobExactSize()); - } else { - log.warn("Blob '{}' was sent for ingestion without specifying its raw data size", urlWithoutSecrets); - } - - ingestionBlobInfo.setReportLevel(ingestionProperties.getReportLevel().getKustoValue()); - ingestionBlobInfo.setReportMethod(ingestionProperties.getReportMethod().getKustoValue()); - ingestionBlobInfo.setFlushImmediately(ingestionProperties.getFlushImmediately()); - ingestionBlobInfo.setValidationPolicy(ingestionProperties.getValidationPolicy()); - ingestionBlobInfo.setAdditionalProperties(ingestionProperties.getIngestionProperties()); - if (blobSourceInfo.getSourceId() != null) { - ingestionBlobInfo.setId(blobSourceInfo.getSourceId()); - } - - String id = ingestionBlobInfo.getId().toString(); - IngestionStatus status = new IngestionStatus(); - status.setDatabase(ingestionProperties.getDatabaseName()); - status.setTable(ingestionProperties.getTableName()); - status.setStatus(OperationStatus.Queued); - status.setUpdatedOn(Instant.now()); - status.setIngestionSourceId(ingestionBlobInfo.getId()); - status.setIngestionSourcePath(urlWithoutSecrets); - boolean reportToTable = ingestionProperties.getReportLevel() != IngestionProperties.IngestionReportLevel.NONE && - ingestionProperties.getReportMethod() != IngestionProperties.IngestionReportMethod.QUEUE; - if (reportToTable) { - status.setStatus(OperationStatus.Pending); - TableWithSas statusTable = resourceManager - .getStatusTable(); - IngestionStatusInTableDescription ingestionStatusInTable = new IngestionStatusInTableDescription(); - ingestionStatusInTable.setTableClient(statusTable.getTable()); - ingestionStatusInTable.setTableConnectionString(statusTable.getUri()); - ingestionStatusInTable.setPartitionKey(ingestionBlobInfo.getId().toString()); - ingestionStatusInTable.setRowKey(ingestionBlobInfo.getId().toString()); - ingestionBlobInfo.setIngestionStatusInTable(ingestionStatusInTable); - azureStorageClient.azureTableInsertEntity(statusTable.getTable(), new TableEntity(id, id).setProperties(status.getEntityProperties())); - tableStatuses.add(ingestionBlobInfo.getIngestionStatusInTable()); - } + properties = ingestionProperties.getIngestionProperties(); + } catch (IOException e) { + throw new IngestionClientException("Failed to ingest from blob", e); + } - ResourceAlgorithms.postToQueueWithRetries(resourceManager, azureStorageClient, ingestionBlobInfo); + ingestionBlobInfo.setReportLevel(ingestionProperties.getReportLevel().getKustoValue()); + ingestionBlobInfo.setReportMethod(ingestionProperties.getReportMethod().getKustoValue()); + ingestionBlobInfo.setFlushImmediately(ingestionProperties.getFlushImmediately()); + ingestionBlobInfo.setValidationPolicy(ingestionProperties.getValidationPolicy()); + ingestionBlobInfo.setAdditionalProperties(properties); + if (blobSourceInfo.getSourceId() != null) { + ingestionBlobInfo.setId(blobSourceInfo.getSourceId()); + } - return reportToTable - ? new TableReportIngestionResult(tableStatuses) - : new IngestionStatusResult(status); - } catch (BlobStorageException | QueueStorageException | TableServiceException e) { - throw new IngestionServiceException("Failed to ingest from blob", e); - } catch (IOException | URISyntaxException e) { - throw new IngestionClientException("Failed to ingest from blob", e); + String id = ingestionBlobInfo.getId().toString(); + IngestionStatus status = new IngestionStatus(); + status.setDatabase(ingestionProperties.getDatabaseName()); + status.setTable(ingestionProperties.getTableName()); + status.setStatus(OperationStatus.Queued); + status.setUpdatedOn(Instant.now()); + status.setIngestionSourceId(ingestionBlobInfo.getId()); + status.setIngestionSourcePath(urlWithoutSecrets); + + boolean reportToTable = ingestionProperties.getReportLevel() != IngestionProperties.IngestionReportLevel.NONE && + ingestionProperties.getReportMethod() != IngestionProperties.IngestionReportMethod.QUEUE; + List tableStatuses = new LinkedList<>(); + + if (reportToTable) { + status.setStatus(OperationStatus.Pending); + TableWithSas statusTable = resourceManager.getStatusTable(); + IngestionStatusInTableDescription ingestionStatusInTable = new IngestionStatusInTableDescription(); + ingestionStatusInTable.setAsyncTableClient(statusTable.getTableAsyncClient()); + ingestionStatusInTable.setTableConnectionString(statusTable.getUri()); + ingestionStatusInTable.setPartitionKey(ingestionBlobInfo.getId().toString()); + ingestionStatusInTable.setRowKey(ingestionBlobInfo.getId().toString()); + ingestionBlobInfo.setIngestionStatusInTable(ingestionStatusInTable); + + return azureStorageClient + .azureTableInsertEntity(statusTable.getTableAsyncClient(), new TableEntity(id, id).setProperties(status.getEntityProperties())) + .doOnTerminate(() -> tableStatuses.add(ingestionBlobInfo.getIngestionStatusInTable())) + .then(ResourceAlgorithms.postToQueueWithRetriesAsync(resourceManager, azureStorageClient, ingestionBlobInfo) + .thenReturn((IngestionResult) new TableReportIngestionResult(tableStatuses))) + .onErrorMap(e -> { + if (e instanceof BlobStorageException || e instanceof QueueStorageException || e instanceof TableServiceException) { + return new IngestionServiceException("Failed to ingest from blob", (Exception) e); + } else if (e instanceof URISyntaxException) { + return new IngestionClientException("Failed to ingest from blob", e); + } else { + return e; + } + }); } + + return ResourceAlgorithms.postToQueueWithRetriesAsync(resourceManager, azureStorageClient, ingestionBlobInfo) + .thenReturn(new IngestionStatusResult(status)); } @Override - protected IngestionResult ingestFromFileImpl(FileSourceInfo fileSourceInfo, IngestionProperties ingestionProperties) - throws IngestionClientException, IngestionServiceException { - // Argument validation: + protected Mono ingestFromFileAsyncImpl(FileSourceInfo fileSourceInfo, IngestionProperties ingestionProperties) { Ensure.argIsNotNull(fileSourceInfo, "fileSourceInfo"); Ensure.argIsNotNull(ingestionProperties, "ingestionProperties"); - fileSourceInfo.validate(); ingestionProperties.validate(); + String filePath = fileSourceInfo.getFilePath(); try { - String filePath = fileSourceInfo.getFilePath(); Ensure.fileExists(filePath); - CompressionType sourceCompressionType = IngestionUtils.getCompression(filePath); - IngestionProperties.DataFormat dataFormat = ingestionProperties.getDataFormat(); - boolean shouldCompress = shouldCompress(sourceCompressionType, dataFormat); - - File file = new File(filePath); - String blobName = genBlobName( - file.getName(), - ingestionProperties.getDatabaseName(), - ingestionProperties.getTableName(), - dataFormat.getKustoValue(), // Used to use an empty string if the DataFormat was empty. Now it can't be empty, with a default of CSV. - shouldCompress ? CompressionType.gz : sourceCompressionType); - - String blobPath = ResourceAlgorithms.uploadLocalFileWithRetries(resourceManager, azureStorageClient, file, blobName, shouldCompress); - BlobSourceInfo blobSourceInfo = BlobSourceInfo.fromFile(blobPath, fileSourceInfo, sourceCompressionType, shouldCompress); - return ingestFromBlob(blobSourceInfo, ingestionProperties); - } catch (BlobStorageException e) { - throw new IngestionServiceException("Failed to ingest from file", e); } catch (IOException e) { throw new IngestionClientException("Failed to ingest from file", e); } + + CompressionType sourceCompressionType = IngestionUtils.getCompression(filePath); + IngestionProperties.DataFormat dataFormat = ingestionProperties.getDataFormat(); + boolean shouldCompress = shouldCompress(sourceCompressionType, dataFormat); + + File file = new File(filePath); + String blobName = genBlobName( + file.getName(), + ingestionProperties.getDatabaseName(), + ingestionProperties.getTableName(), + dataFormat.getKustoValue(), // Used to use an empty string if the DataFormat was empty. Now it can't be empty, with a default of + // CSV. + shouldCompress ? CompressionType.gz : sourceCompressionType); + return ResourceAlgorithms.uploadLocalFileWithRetriesAsync(resourceManager, azureStorageClient, file, blobName, shouldCompress) + .flatMap(blobPath -> { + BlobSourceInfo blobSourceInfo = BlobSourceInfo.fromFile(blobPath, fileSourceInfo, sourceCompressionType, shouldCompress); + return ingestFromBlobAsync(blobSourceInfo, ingestionProperties); + }) + .onErrorMap(BlobStorageException.class, e -> new IngestionServiceException("Failed to ingest from file", e)); } @Override - protected IngestionResult ingestFromStreamImpl(StreamSourceInfo streamSourceInfo, IngestionProperties ingestionProperties) - throws IngestionClientException, IngestionServiceException { - // Argument validation: + protected Mono ingestFromStreamAsyncImpl(StreamSourceInfo streamSourceInfo, IngestionProperties ingestionProperties) { Ensure.argIsNotNull(streamSourceInfo, "streamSourceInfo"); Ensure.argIsNotNull(ingestionProperties, "ingestionProperties"); - streamSourceInfo.validate(); ingestionProperties.validate(); + if (streamSourceInfo.getStream() == null) { + throw new IngestionClientException("The provided stream is null."); + } + try { - IngestionResult ingestionResult; - if (streamSourceInfo.getStream() == null) { - throw new IngestionClientException("The provided stream is null."); - } else if (streamSourceInfo.getStream().available() <= 0) { + if (streamSourceInfo.getStream().available() <= 0) { throw new IngestionClientException("The provided stream is empty."); } - IngestionProperties.DataFormat dataFormat = ingestionProperties.getDataFormat(); - boolean shouldCompress = shouldCompress(streamSourceInfo.getCompressionType(), dataFormat); - - String blobName = genBlobName( - "StreamUpload", - ingestionProperties.getDatabaseName(), - ingestionProperties.getTableName(), - dataFormat.getKustoValue(), // Used to use an empty string if the DataFormat was empty. Now it can't be empty, with a default of CSV. - shouldCompress ? CompressionType.gz : streamSourceInfo.getCompressionType()); - - ResourceAlgorithms.UploadResult blobUploadedDetails = ResourceAlgorithms.uploadStreamToBlobWithRetries(resourceManager, - azureStorageClient, - streamSourceInfo.getStream(), - blobName, - shouldCompress); - if (blobUploadedDetails.size == 0) { - String message = "Empty stream."; - log.error(message); - throw new IngestionClientException(message); - } - BlobSourceInfo blobSourceInfo = BlobSourceInfo.fromStream(blobUploadedDetails.blobPath, blobUploadedDetails.size, streamSourceInfo); - - ingestionResult = ingestFromBlob(blobSourceInfo, ingestionProperties); - if (!streamSourceInfo.isLeaveOpen()) { - streamSourceInfo.getStream().close(); - } - return ingestionResult; - } catch (BlobStorageException e) { - throw new IngestionServiceException("Failed to ingest from stream", e); } catch (IOException e) { - throw new IngestionClientException("Failed to ingest from stream", e); + throw new IngestionClientException("The provided stream is empty."); } + + IngestionProperties.DataFormat dataFormat = ingestionProperties.getDataFormat(); + boolean shouldCompress = shouldCompress(streamSourceInfo.getCompressionType(), dataFormat); + + String blobName = genBlobName( + "StreamUpload", + ingestionProperties.getDatabaseName(), + ingestionProperties.getTableName(), + dataFormat.getKustoValue(), // Used to use an empty string if the DataFormat was empty. Now it can't be empty, with a default of + // CSV. + shouldCompress ? CompressionType.gz : streamSourceInfo.getCompressionType()); + + return ResourceAlgorithms.uploadStreamToBlobWithRetriesAsync(resourceManager, + azureStorageClient, + streamSourceInfo.getStream(), + blobName, + shouldCompress) + .flatMap(blobUploadedDetails -> { + if (blobUploadedDetails.size == 0) { + String message = "Empty stream."; + log.error(message); + throw new IngestionClientException(message); + } + BlobSourceInfo blobSourceInfo = BlobSourceInfo.fromStream(blobUploadedDetails.blobPath, blobUploadedDetails.size, streamSourceInfo); + return ingestFromBlobAsync(blobSourceInfo, ingestionProperties); + }) + .onErrorMap(BlobStorageException.class, e -> new IngestionServiceException("Failed to ingest from stream", e)) + .doFinally(signalType -> { + if (!streamSourceInfo.isLeaveOpen()) { + try { + streamSourceInfo.getStream().close(); + } catch (IOException e) { + throw new IngestionClientException("Failed to close stream after ingestion", e); + } + } + }); + } @Override @@ -251,27 +280,25 @@ String genBlobName(String fileName, String databaseName, String tableName, Strin } @Override - protected IngestionResult ingestFromResultSetImpl(ResultSetSourceInfo resultSetSourceInfo, IngestionProperties ingestionProperties) - throws IngestionClientException, IngestionServiceException { - // Argument validation: + protected Mono ingestFromResultSetAsyncImpl(ResultSetSourceInfo resultSetSourceInfo, IngestionProperties ingestionProperties) { Ensure.argIsNotNull(resultSetSourceInfo, "resultSetSourceInfo"); Ensure.argIsNotNull(ingestionProperties, "ingestionProperties"); - resultSetSourceInfo.validate(); ingestionProperties.validateResultSetProperties(); - try { + + return Mono.fromCallable(() -> { ByteArrayOutputStream byteArrayOutputStream = new ByteArrayOutputStream(); - new CsvRoutines().write(resultSetSourceInfo.getResultSet(), byteArrayOutputStream); + new CsvRoutines().write(resultSetSourceInfo.getResultSet(), byteArrayOutputStream); // TODO: CsvRoutines is not maintained from 2021. replace? byteArrayOutputStream.flush(); ByteArrayInputStream byteArrayInputStream = new ByteArrayInputStream(byteArrayOutputStream.toByteArray()); - - StreamSourceInfo streamSourceInfo = new StreamSourceInfo(byteArrayInputStream, false, resultSetSourceInfo.getSourceId()); - return ingestFromStream(streamSourceInfo, ingestionProperties); - } catch (IOException ex) { - String msg = "Failed to read from ResultSet."; - log.error(msg, ex); - throw new IngestionClientException(msg, ex); - } + return new StreamSourceInfo(byteArrayInputStream, false, resultSetSourceInfo.getSourceId()); + }).subscribeOn(Schedulers.boundedElastic()) + .flatMap(streamSourceInfo -> ingestFromStreamAsync(streamSourceInfo, ingestionProperties)) + .onErrorMap(IOException.class, e -> { + String msg = "Failed to read from ResultSet."; + log.error(msg, e); + return new IngestionClientException(msg, e); + }); } protected void setConnectionDataSource(String connectionDataSource) { diff --git a/ingest/src/main/java/com/microsoft/azure/kusto/ingest/ResourceAlgorithms.java b/ingest/src/main/java/com/microsoft/azure/kusto/ingest/ResourceAlgorithms.java index 5894f985e..920cff8bf 100644 --- a/ingest/src/main/java/com/microsoft/azure/kusto/ingest/ResourceAlgorithms.java +++ b/ingest/src/main/java/com/microsoft/azure/kusto/ingest/ResourceAlgorithms.java @@ -1,109 +1,140 @@ package com.microsoft.azure.kusto.ingest; -import com.fasterxml.jackson.core.JsonProcessingException; import com.fasterxml.jackson.databind.ObjectMapper; import com.microsoft.azure.kusto.data.Utils; import com.microsoft.azure.kusto.data.exceptions.ExceptionUtils; import com.microsoft.azure.kusto.data.instrumentation.FunctionOneException; import com.microsoft.azure.kusto.data.instrumentation.MonitoredActivity; -import com.microsoft.azure.kusto.data.instrumentation.Tracer; import com.microsoft.azure.kusto.ingest.exceptions.IngestionClientException; -import com.microsoft.azure.kusto.ingest.exceptions.IngestionServiceException; import com.microsoft.azure.kusto.ingest.resources.RankedStorageAccount; import com.microsoft.azure.kusto.ingest.resources.ResourceWithSas; import com.microsoft.azure.kusto.ingest.utils.SecurityUtils; -import org.apache.commons.lang3.tuple.Pair; import org.jetbrains.annotations.NotNull; import org.slf4j.Logger; import org.slf4j.LoggerFactory; +import reactor.core.publisher.Mono; import java.io.File; import java.io.InputStream; import java.lang.invoke.MethodHandles; -import java.util.*; +import java.util.ArrayList; +import java.util.Collections; +import java.util.HashMap; +import java.util.List; +import java.util.Map; +import java.util.Objects; import java.util.stream.Collectors; import java.util.stream.IntStream; -public class ResourceAlgorithms { +class ResourceAlgorithms { private static final int RETRY_COUNT = 3; private static final Logger log = LoggerFactory.getLogger(MethodHandles.lookup().lookupClass()); private ResourceAlgorithms() { } - private static , TOut> TOut resourceActionWithRetries(ResourceManager resourceManager, - List resources, FunctionOneException action, String actionName, Map additionalAttributes) - throws IngestionClientException { + public static , TOut> Mono resourceActionWithRetriesAsync( + ResourceManager resourceManager, + List resources, + FunctionOneException, TWrapper, Exception> action, + String actionName, + Map additionalAttributes) { if (resources.isEmpty()) { throw new IngestionClientException(String.format("%s: No resources were provided.", actionName)); } List> totalAttributes = new ArrayList<>(); - Exception ex = null; - for (int i = 0; i < RETRY_COUNT; i++) { - TWrapper resource = resources.get(i % resources.size()); - try { - Map attributes = new HashMap<>(); - attributes.put("resource", resource.getEndpointWithoutSas()); - attributes.put("account", resource.getAccountName()); - attributes.put("type", resource.getClass().getName()); - attributes.put("retry", String.valueOf(i)); - attributes.putAll(additionalAttributes); - totalAttributes.add(attributes); - - return MonitoredActivity.invoke((FunctionOneException) (Tracer.Span span) -> { - try { - TOut result = action.apply(resource); - resourceManager.reportIngestionResult(resource, true); - return result; - } catch (Exception e) { - resourceManager.reportIngestionResult(resource, false); - span.addException(e); - throw e; - } - }, actionName, attributes); - } catch (Exception e) { - ex = e; - log.warn(String.format("Error during retry %d of %d for %s", i + 1, RETRY_COUNT, actionName), e); - } + + return attemptAction(1, resources, resourceManager, action, actionName, additionalAttributes, null, totalAttributes); + } + + private static , TOut> Mono attemptAction( + int attempt, + List resources, + ResourceManager resourceManager, + FunctionOneException, TWrapper, Exception> action, + String actionName, + Map additionalAttributes, + Exception ex, + List> totalAttributes) { + + if (attempt > RETRY_COUNT) { + String errorMessage = String.format("%s: All %d retries failed with last error: %s\n. Used resources: %s", + actionName, + RETRY_COUNT, + ex != null ? ExceptionUtils.getMessageEx(ex) : "", + totalAttributes.stream() + .map(x -> String.format("%s (%s)", x.get("resource"), x.get("account"))) + .collect(Collectors.joining(", "))); + throw new IngestionClientException(errorMessage); } - throw new IngestionClientException(String.format("%s: All %d retries failed with last error: %s\n. Used resources: %s", actionName, RETRY_COUNT, - totalAttributes.stream().map(x -> String.format("%s (%s)", x.get("resource"), x.get("account"))).collect(Collectors.joining(", ")), - ExceptionUtils.getMessageEx(ex))); + + TWrapper resource = resources.get((attempt - 1) % resources.size()); + Map attributes = new HashMap<>(); + attributes.put("resource", resource.getEndpointWithoutSas()); + attributes.put("account", resource.getAccountName()); + attributes.put("type", resource.getClass().getName()); + attributes.put("retry", String.valueOf(attempt)); + attributes.putAll(additionalAttributes); + totalAttributes.add(attributes); + + return MonitoredActivity.invokeAsync( + span -> action.apply(resource) + .doOnSuccess(ignored -> resourceManager.reportIngestionResult(resource, true)), + actionName, + attributes) + .onErrorResume(e -> { + log.warn(String.format("Error during attempt %d of %d for %s.", attempt, RETRY_COUNT, actionName), e); + resourceManager.reportIngestionResult(resource, false); + return attemptAction(attempt + 1, resources, resourceManager, action, actionName, additionalAttributes, (Exception) e, totalAttributes); + }); } - public static void postToQueueWithRetries(ResourceManager resourceManager, AzureStorageClient azureStorageClient, IngestionBlobInfo blob) - throws IngestionClientException, IngestionServiceException, JsonProcessingException { + public static Mono postToQueueWithRetriesAsync(ResourceManager resourceManager, AzureStorageClient azureStorageClient, IngestionBlobInfo blob) { ObjectMapper objectMapper = Utils.getObjectMapper(); - String message = objectMapper.writeValueAsString(blob); - resourceActionWithRetries(resourceManager, resourceManager.getShuffledQueues(), queue -> { - azureStorageClient.postMessageToQueue(queue.getQueue(), message); - return null; - }, "ResourceAlgorithms.postToQueueWithRetries", + String message; + try { + message = objectMapper.writeValueAsString(blob); + } catch (Exception e) { + throw new IngestionClientException("Failed to ingest from blob", e); + } + + return resourceActionWithRetriesAsync( + resourceManager, + resourceManager.getShuffledQueues(), + queue -> azureStorageClient.postMessageToQueue(queue.getAsyncQueue(), message), + "ResourceAlgorithms.postToQueueWithRetriesAsync", Collections.singletonMap("blob", SecurityUtils.removeSecretsFromUrl(blob.getBlobPath()))); } - public static UploadResult uploadStreamToBlobWithRetries(ResourceManager resourceManager, AzureStorageClient azureStorageClient, + public static Mono uploadStreamToBlobWithRetriesAsync(ResourceManager resourceManager, AzureStorageClient azureStorageClient, InputStream stream, - String blobName, boolean shouldCompress) - throws IngestionClientException, IngestionServiceException { - return resourceActionWithRetries(resourceManager, resourceManager.getShuffledContainers(), container -> { - int size = azureStorageClient.uploadStreamToBlob(stream, blobName, container.getContainer(), shouldCompress); - UploadResult uploadResult = new UploadResult(); - uploadResult.blobPath = container.getContainer().getBlobContainerUrl() + "/" + blobName + container.getSas(); - uploadResult.size = size; - return uploadResult; - }, "ResourceAlgorithms.uploadLocalFileWithRetries", Collections.emptyMap()); + String blobName, boolean shouldCompress) { + return resourceActionWithRetriesAsync( + resourceManager, + resourceManager.getShuffledContainers(), + container -> azureStorageClient.uploadStreamToBlob(stream, blobName, container.getAsyncContainer(), shouldCompress) + .map((size) -> { + UploadResult uploadResult = new UploadResult(); + uploadResult.blobPath = container.getAsyncContainer().getBlobContainerUrl() + "/" + blobName + container.getSas(); + uploadResult.size = size; + return uploadResult; + }), + "ResourceAlgorithms.uploadStreamToBlobWithRetriesAsync", + Collections.emptyMap()); } - public static String uploadLocalFileWithRetries(ResourceManager resourceManager, AzureStorageClient azureStorageClient, File file, String blobName, - boolean shouldCompress) - throws IngestionClientException, IngestionServiceException { - return resourceActionWithRetries(resourceManager, resourceManager.getShuffledContainers(), container -> { - azureStorageClient.uploadLocalFileToBlob(file, blobName, container.getContainer(), shouldCompress); - return (container.getContainer().getBlobContainerUrl() + "/" + blobName + container.getSas()); - }, "ResourceAlgorithms.uploadLocalFileWithRetries", Collections.emptyMap()); + public static Mono uploadLocalFileWithRetriesAsync(ResourceManager resourceManager, AzureStorageClient azureStorageClient, File file, + String blobName, + boolean shouldCompress) { + return resourceActionWithRetriesAsync( + resourceManager, + resourceManager.getShuffledContainers(), + container -> azureStorageClient.uploadLocalFileToBlob(file, blobName, container.getAsyncContainer(), shouldCompress) + .thenReturn(container.getAsyncContainer().getBlobContainerUrl() + "/" + blobName + container.getSas()), + "ResourceAlgorithms.uploadLocalFileWithRetriesAsync", + Collections.emptyMap()); } @NotNull diff --git a/ingest/src/main/java/com/microsoft/azure/kusto/ingest/ResourceManager.java b/ingest/src/main/java/com/microsoft/azure/kusto/ingest/ResourceManager.java index ca7c3edd3..1fa485a2c 100644 --- a/ingest/src/main/java/com/microsoft/azure/kusto/ingest/ResourceManager.java +++ b/ingest/src/main/java/com/microsoft/azure/kusto/ingest/ResourceManager.java @@ -6,6 +6,7 @@ import com.azure.core.http.HttpClient; import com.azure.storage.common.policy.RequestRetryOptions; import com.microsoft.azure.kusto.data.Client; +import com.microsoft.azure.kusto.data.ExponentialRetry; import com.microsoft.azure.kusto.data.KustoOperationResult; import com.microsoft.azure.kusto.data.KustoResultSetTable; import com.microsoft.azure.kusto.data.Utils; @@ -29,13 +30,14 @@ import io.vavr.CheckedFunction0; import org.slf4j.Logger; import org.slf4j.LoggerFactory; + import reactor.util.annotation.Nullable; import java.io.Closeable; import java.lang.invoke.MethodHandles; import java.net.URISyntaxException; -import java.time.Duration; import java.util.ArrayList; +import java.util.Collections; import java.util.List; import java.util.Timer; import java.util.TimerTask; @@ -48,9 +50,15 @@ import java.util.stream.Stream; class ResourceManager implements Closeable, IngestionResourceManager { + + private static final int MAX_RETRY_ATTEMPTS = 4; private static final long REFRESH_INGESTION_RESOURCES_PERIOD = TimeUnit.HOURS.toMillis(1); private static final long REFRESH_INGESTION_RESOURCES_PERIOD_ON_FAILURE = TimeUnit.MINUTES.toMillis(1); private static final long REFRESH_RESULT_POLL_TIMEOUT_MILLIS = TimeUnit.SECONDS.toMillis(15); + private static final long BASE_INTERVAL = TimeUnit.SECONDS.toMillis(2); + private static final double JITTER_FACTOR = 0.5; + private static final reactor.util.retry.Retry RETRY_CONFIG = new ExponentialRetry(MAX_RETRY_ATTEMPTS, BASE_INTERVAL, JITTER_FACTOR) + .retry(Collections.singletonList(ThrottleException.class), null); // TODO: fix import after removing vavr private final Client client; private final Logger log = LoggerFactory.getLogger(MethodHandles.lookup().lookupClass()); private Timer refreshTasksTimer; @@ -69,6 +77,12 @@ class ResourceManager implements Closeable, IngestionResourceManager { protected RefreshIngestionAuthTokenTask refreshIngestionAuthTokenTask; protected RefreshIngestionResourcesTask refreshIngestionResourcesTask; + ///

+ /// + /// Creates a new instance of the ResourceManager. + /// This class is not async at its nature therefore it was not implemented as one, a cache is only doing async work + /// once in a while and most requests should return immediately + /// public ResourceManager(Client client, long defaultRefreshTime, long refreshTimeOnFailure, @Nullable HttpClient httpClient) { this.client = client; // Using ctor with client so that the dependency is used @@ -121,6 +135,7 @@ public Boolean waitUntilRefreshedAtLeastOnce(long timeoutMillis) { } class RefreshIngestionResourcesTask extends RefreshResourceTask { + @Override public void run() { try { @@ -129,7 +144,7 @@ public void run() { return null; }, "ResourceManager.refreshIngestionResource"); } catch (Exception e) { - log.error("Error in refreshIngestionResources: " + e.getMessage(), e); + log.error("Error in refreshIngestionResources: " + e.getMessage(), e); // onerrorresume scheduleRefreshIngestionResourcesTask(refreshTimeOnFailure); } } diff --git a/ingest/src/main/java/com/microsoft/azure/kusto/ingest/StreamingIngestClient.java b/ingest/src/main/java/com/microsoft/azure/kusto/ingest/StreamingIngestClient.java index 8c630bbbc..c438d813a 100644 --- a/ingest/src/main/java/com/microsoft/azure/kusto/ingest/StreamingIngestClient.java +++ b/ingest/src/main/java/com/microsoft/azure/kusto/ingest/StreamingIngestClient.java @@ -4,15 +4,18 @@ package com.microsoft.azure.kusto.ingest; import com.azure.core.http.HttpClient; +import com.azure.storage.blob.BlobAsyncClient; import com.azure.storage.blob.models.BlobStorageException; -import com.microsoft.azure.kusto.data.*; +import com.microsoft.azure.kusto.data.ClientFactory; +import com.microsoft.azure.kusto.data.ClientRequestProperties; +import com.microsoft.azure.kusto.data.Ensure; +import com.microsoft.azure.kusto.data.StreamingClient; import com.microsoft.azure.kusto.data.auth.ConnectionStringBuilder; import com.microsoft.azure.kusto.data.exceptions.DataClientException; import com.microsoft.azure.kusto.data.exceptions.DataServiceException; import com.microsoft.azure.kusto.data.exceptions.ExceptionUtils; import com.microsoft.azure.kusto.data.http.HttpClientProperties; import com.microsoft.azure.kusto.data.instrumentation.MonitoredActivity; -import com.microsoft.azure.kusto.data.instrumentation.SupplierTwoExceptions; import com.microsoft.azure.kusto.ingest.exceptions.IngestionClientException; import com.microsoft.azure.kusto.ingest.exceptions.IngestionServiceException; import com.microsoft.azure.kusto.ingest.result.IngestionResult; @@ -28,13 +31,14 @@ import org.jetbrains.annotations.Nullable; import org.slf4j.Logger; import org.slf4j.LoggerFactory; +import reactor.core.publisher.Mono; +import reactor.core.scheduler.Schedulers; -import java.io.*; +import java.io.IOException; import java.lang.invoke.MethodHandles; import java.net.URI; import java.net.URISyntaxException; import java.util.Objects; -import java.util.zip.GZIPOutputStream; public class StreamingIngestClient extends IngestClientBase implements IngestClient { @@ -82,69 +86,63 @@ public static String generateEngineUriSuggestion(URI existingEndpoint) throws UR } @Override - protected IngestionResult ingestFromFileImpl(FileSourceInfo fileSourceInfo, IngestionProperties ingestionProperties) - throws IngestionClientException, IngestionServiceException { + protected Mono ingestFromFileAsyncImpl(FileSourceInfo fileSourceInfo, IngestionProperties ingestionProperties) { Ensure.argIsNotNull(fileSourceInfo, "fileSourceInfo"); Ensure.argIsNotNull(ingestionProperties, "ingestionProperties"); - fileSourceInfo.validate(); ingestionProperties.validate(); try { StreamSourceInfo streamSourceInfo = IngestionUtils.fileToStream(fileSourceInfo, false); - return ingestFromStream(streamSourceInfo, ingestionProperties); - } catch (FileNotFoundException e) { + return ingestFromStreamAsync(streamSourceInfo, ingestionProperties); + } catch (IOException e) { log.error("File not found when ingesting a file.", e); throw new IngestionClientException("IO exception - check file path.", e); } } @Override - protected IngestionResult ingestFromBlobImpl(BlobSourceInfo blobSourceInfo, IngestionProperties ingestionProperties) - throws IngestionClientException, IngestionServiceException { + protected Mono ingestFromBlobAsyncImpl(BlobSourceInfo blobSourceInfo, IngestionProperties ingestionProperties) { Ensure.argIsNotNull(blobSourceInfo, "blobSourceInfo"); Ensure.argIsNotNull(ingestionProperties, "ingestionProperties"); - blobSourceInfo.validate(); ingestionProperties.validate(); + BlobAsyncClient blobAsyncClient; try { - return ingestFromBlob(blobSourceInfo, ingestionProperties, null); + return ingestFromBlobAsync(blobSourceInfo, ingestionProperties, null) + .onErrorMap(BlobStorageException.class, e -> { + String msg = "Unexpected Storage error when ingesting a blob."; + log.error(msg, e); + return new IngestionClientException(msg, e); + }); } catch (IllegalArgumentException e) { String msg = "Unexpected error when ingesting a blob - Invalid blob path."; log.error(msg, e); throw new IngestionClientException(msg, e); - } catch (BlobStorageException e) { - String msg = "Unexpected Storage error when ingesting a blob."; - log.error(msg, e); - throw new IngestionClientException(msg, e); } } @Override - protected IngestionResult ingestFromResultSetImpl(ResultSetSourceInfo resultSetSourceInfo, IngestionProperties ingestionProperties) - throws IngestionClientException, IngestionServiceException { - // Argument validation: + protected Mono ingestFromResultSetAsyncImpl(ResultSetSourceInfo resultSetSourceInfo, IngestionProperties ingestionProperties) { Ensure.argIsNotNull(resultSetSourceInfo, "resultSetSourceInfo"); Ensure.argIsNotNull(ingestionProperties, "ingestionProperties"); - resultSetSourceInfo.validate(); ingestionProperties.validateResultSetProperties(); try { StreamSourceInfo streamSourceInfo = IngestionUtils.resultSetToStream(resultSetSourceInfo); - return ingestFromStream(streamSourceInfo, ingestionProperties); - } catch (IOException ex) { + return ingestFromStreamAsync(streamSourceInfo, ingestionProperties); + } catch (IOException e) { String msg = "Failed to read from ResultSet."; - log.error(msg, ex); - throw new IngestionClientException(msg, ex); + log.error(msg, e); + throw new IngestionClientException(msg, e); } } @Override - protected IngestionResult ingestFromStreamImpl(StreamSourceInfo streamSourceInfo, IngestionProperties ingestionProperties) - throws IngestionClientException, IngestionServiceException { - return ingestFromStreamImpl(streamSourceInfo, ingestionProperties, null); + protected Mono ingestFromStreamAsyncImpl(StreamSourceInfo streamSourceInfo, IngestionProperties ingestionProperties) { + return ingestFromStreamImplAsync(streamSourceInfo, ingestionProperties, null); } @Override @@ -152,102 +150,87 @@ protected String getClientType() { return CLASS_NAME; } - IngestionResult ingestFromStream(StreamSourceInfo streamSourceInfo, IngestionProperties ingestionProperties, @Nullable String clientRequestId) - throws IngestionClientException, IngestionServiceException { - // trace ingestFromStream - return MonitoredActivity.invoke( - (SupplierTwoExceptions) () -> ingestFromStreamImpl(streamSourceInfo, + Mono ingestFromStreamAsync(StreamSourceInfo streamSourceInfo, IngestionProperties ingestionProperties, @Nullable String clientRequestId) { + // trace ingestFromStreamAsync + return MonitoredActivity.wrap( + ingestFromStreamImplAsync(streamSourceInfo, ingestionProperties, clientRequestId), getClientType().concat(".ingestFromStream"), getIngestionTraceAttributes(streamSourceInfo, ingestionProperties)); } - private IngestionResult ingestFromStreamImpl(StreamSourceInfo streamSourceInfo, IngestionProperties ingestionProperties, @Nullable String clientRequestId) - throws IngestionClientException, IngestionServiceException { + private Mono ingestFromStreamImplAsync(StreamSourceInfo streamSourceInfo, + IngestionProperties ingestionProperties, + @Nullable String clientRequestId) { + Ensure.argIsNotNull(streamSourceInfo, "streamSourceInfo"); Ensure.argIsNotNull(ingestionProperties, "ingestionProperties"); - - IngestionProperties.DataFormat dataFormat = ingestionProperties.getDataFormat(); - - streamSourceInfo.validate(); ingestionProperties.validate(); + streamSourceInfo.validate(); - ClientRequestProperties clientRequestProperties = null; - if (StringUtils.isNotBlank(clientRequestId)) { - clientRequestProperties = new ClientRequestProperties(); - clientRequestProperties.setClientRequestId(clientRequestId); - } - - try { - InputStream stream = IngestClientBase.shouldCompress(streamSourceInfo.getCompressionType(), dataFormat) - ? compressStream(streamSourceInfo.getStream(), streamSourceInfo.isLeaveOpen()) - : streamSourceInfo.getStream(); - log.debug("Executing streaming ingest"); - this.streamingClient.executeStreamingIngest(ingestionProperties.getDatabaseName(), - ingestionProperties.getTableName(), - stream, - clientRequestProperties, - dataFormat.getKustoValue(), - ingestionProperties.getIngestionMapping().getIngestionMappingReference(), - !(streamSourceInfo.getCompressionType() == null || !streamSourceInfo.isLeaveOpen())); - } catch (DataClientException | IOException e) { - String msg = ExceptionUtils.getMessageEx(e); - log.error(msg, e); - throw new IngestionClientException(msg, e); - } catch (DataServiceException e) { - log.error(e.getMessage(), e); - throw new IngestionServiceException(e.getMessage(), e); - } - - log.debug("Stream was ingested successfully."); - IngestionStatus ingestionStatus = new IngestionStatus(); - ingestionStatus.status = OperationStatus.Succeeded; - ingestionStatus.table = ingestionProperties.getTableName(); - ingestionStatus.database = ingestionProperties.getDatabaseName(); - return new IngestionStatusResult(ingestionStatus); - } + IngestionProperties.DataFormat dataFormat = ingestionProperties.getDataFormat(); - private InputStream compressStream(InputStream uncompressedStream, boolean leaveOpen) throws IngestionClientException, IOException { - log.debug("Compressing the stream."); - ByteArrayOutputStream byteArrayOutputStream = new ByteArrayOutputStream(); - GZIPOutputStream gzipOutputStream = new GZIPOutputStream(byteArrayOutputStream); - byte[] b = new byte[STREAM_COMPRESS_BUFFER_SIZE]; - int read = uncompressedStream.read(b); - if (read == -1) { - String message = "Empty stream."; - log.error(message); - throw new IngestionClientException(message); - } - do { - gzipOutputStream.write(b, 0, read); - } while ((read = uncompressedStream.read(b)) != -1); - gzipOutputStream.flush(); - gzipOutputStream.close(); - InputStream inputStream = new ByteArrayInputStream(byteArrayOutputStream.toByteArray()); - byteArrayOutputStream.close(); - if (!leaveOpen) { - uncompressedStream.close(); - } - return inputStream; + return (IngestClientBase.shouldCompress(streamSourceInfo.getCompressionType(), dataFormat) + ? IngestionUtils.compressStream1(streamSourceInfo.getStream(), streamSourceInfo.isLeaveOpen()) + : Mono.just(streamSourceInfo.getStream())) + .subscribeOn(Schedulers.boundedElastic()) + .onErrorMap(IOException.class, e -> { + String msg = ExceptionUtils.getMessageEx(e); + log.error(msg, e); + return new IngestionClientException(msg, e); + }) + .flatMap(stream -> { + ClientRequestProperties clientRequestProperties = null; + if (StringUtils.isNotBlank(clientRequestId)) { + clientRequestProperties = new ClientRequestProperties(); + clientRequestProperties.setClientRequestId(clientRequestId); + } + + log.debug("Executing streaming ingest"); + return this.streamingClient.executeStreamingIngestAsync( + ingestionProperties.getDatabaseName(), + ingestionProperties.getTableName(), + stream, + clientRequestProperties, + dataFormat.getKustoValue(), + ingestionProperties.getIngestionMapping().getIngestionMappingReference(), + !(streamSourceInfo.getCompressionType() == null || !streamSourceInfo.isLeaveOpen())) + .doOnSuccess(ignored -> log.debug("Stream was ingested successfully.")); + }) + .onErrorMap(DataClientException.class, e -> { + String msg = ExceptionUtils.getMessageEx(e); + log.error(msg, e); + return new IngestionClientException(msg, e); + }) + .onErrorMap(DataServiceException.class, e -> { + log.error(e.getMessage(), e); + return new IngestionServiceException(e.getMessage(), e); + }) + .map(ignore -> { + log.debug("Stream was ingested successfully."); + IngestionStatus ingestionStatus = new IngestionStatus(); + ingestionStatus.status = OperationStatus.Succeeded; + ingestionStatus.table = ingestionProperties.getTableName(); + ingestionStatus.database = ingestionProperties.getDatabaseName(); + return new IngestionStatusResult(ingestionStatus); + }); } - IngestionResult ingestFromBlob(BlobSourceInfo blobSourceInfo, + Mono ingestFromBlobAsync(BlobSourceInfo blobSourceInfo, IngestionProperties ingestionProperties, - @Nullable String clientRequestId) - throws IngestionClientException, IngestionServiceException { - // trace ingestFromBlob - return MonitoredActivity.invoke( - (SupplierTwoExceptions) () -> ingestFromBlobImpl(blobSourceInfo, + @Nullable String clientRequestId) { + // trace ingestFromBlobAsync + return MonitoredActivity.wrap( + ingestFromBlobImplAsync(blobSourceInfo, ingestionProperties, clientRequestId), getClientType().concat(".ingestFromBlob"), getIngestionTraceAttributes(blobSourceInfo, ingestionProperties)); } - private IngestionResult ingestFromBlobImpl(BlobSourceInfo blobSourceInfo, IngestionProperties ingestionProperties, - @Nullable String clientRequestId) - throws IngestionClientException, IngestionServiceException { - String blobPath = blobSourceInfo.getBlobPath(); + private Mono ingestFromBlobImplAsync(BlobSourceInfo blobSourceInfo, IngestionProperties ingestionProperties, + @Nullable String clientRequestId) { + String blobPath = blobSourceInfo.getBlobPath(); ClientRequestProperties clientRequestProperties = null; if (StringUtils.isNotBlank(clientRequestId)) { clientRequestProperties = new ClientRequestProperties(); @@ -255,27 +238,28 @@ private IngestionResult ingestFromBlobImpl(BlobSourceInfo blobSourceInfo, Ingest } IngestionProperties.DataFormat dataFormat = ingestionProperties.getDataFormat(); - try { - this.streamingClient.executeStreamingIngestFromBlob(ingestionProperties.getDatabaseName(), - ingestionProperties.getTableName(), - blobPath, - clientRequestProperties, - dataFormat.getKustoValue(), - ingestionProperties.getIngestionMapping().getIngestionMappingReference()); - } catch (DataClientException e) { - log.error(e.getMessage(), e); - throw new IngestionClientException(e.getMessage(), e); - } catch (DataServiceException e) { - log.error(e.getMessage(), e); - throw new IngestionServiceException(e.getMessage(), e); - } - - log.debug("Blob was ingested successfully."); - IngestionStatus ingestionStatus = new IngestionStatus(); - ingestionStatus.status = OperationStatus.Succeeded; - ingestionStatus.table = ingestionProperties.getTableName(); - ingestionStatus.database = ingestionProperties.getDatabaseName(); - return new IngestionStatusResult(ingestionStatus); + return this.streamingClient.executeStreamingIngestFromBlobAsync(ingestionProperties.getDatabaseName(), + ingestionProperties.getTableName(), + blobPath, + clientRequestProperties, + dataFormat.getKustoValue(), + ingestionProperties.getIngestionMapping().getIngestionMappingReference()) + .onErrorMap(DataClientException.class, e -> { + log.error(e.getMessage(), e); + return new IngestionClientException(e.getMessage(), e); + }) + .onErrorMap(DataServiceException.class, e -> { + log.error(e.getMessage(), e); + return new IngestionServiceException(e.getMessage(), e); + }) + .doOnSuccess(ignored1 -> log.debug("Blob was ingested successfully.")) + .map(ignore -> { + IngestionStatus ingestionStatus = new IngestionStatus(); + ingestionStatus.status = OperationStatus.Succeeded; + ingestionStatus.table = ingestionProperties.getTableName(); + ingestionStatus.database = ingestionProperties.getDatabaseName(); + return new IngestionStatusResult(ingestionStatus); + }); } protected void setConnectionDataSource(String connectionDataSource) { diff --git a/ingest/src/main/java/com/microsoft/azure/kusto/ingest/exceptions/IngestionClientException.java b/ingest/src/main/java/com/microsoft/azure/kusto/ingest/exceptions/IngestionClientException.java index 177db6d5e..63b62602d 100644 --- a/ingest/src/main/java/com/microsoft/azure/kusto/ingest/exceptions/IngestionClientException.java +++ b/ingest/src/main/java/com/microsoft/azure/kusto/ingest/exceptions/IngestionClientException.java @@ -5,8 +5,7 @@ import com.azure.core.exception.AzureException; -public class IngestionClientException extends AzureException { // TODO: remove throws from internal method declarations (not from public Apis) on async ingest - // impl +public class IngestionClientException extends AzureException { private String ingestionSource; public String getIngestionSource() { diff --git a/ingest/src/main/java/com/microsoft/azure/kusto/ingest/exceptions/IngestionServiceException.java b/ingest/src/main/java/com/microsoft/azure/kusto/ingest/exceptions/IngestionServiceException.java index 7f56a0308..a8d94f732 100644 --- a/ingest/src/main/java/com/microsoft/azure/kusto/ingest/exceptions/IngestionServiceException.java +++ b/ingest/src/main/java/com/microsoft/azure/kusto/ingest/exceptions/IngestionServiceException.java @@ -5,8 +5,7 @@ import com.azure.core.exception.AzureException; -public class IngestionServiceException extends AzureException { // TODO: remove throws from internal method declarations (not from public Apis) on async ingest - // impl +public class IngestionServiceException extends AzureException { private String ingestionSource; public String getIngestionSource() { diff --git a/ingest/src/main/java/com/microsoft/azure/kusto/ingest/resources/ContainerWithSas.java b/ingest/src/main/java/com/microsoft/azure/kusto/ingest/resources/ContainerWithSas.java index 002d92d4b..7b293d838 100644 --- a/ingest/src/main/java/com/microsoft/azure/kusto/ingest/resources/ContainerWithSas.java +++ b/ingest/src/main/java/com/microsoft/azure/kusto/ingest/resources/ContainerWithSas.java @@ -1,15 +1,15 @@ package com.microsoft.azure.kusto.ingest.resources; import com.azure.core.http.HttpClient; -import com.azure.storage.blob.BlobContainerClient; +import com.azure.storage.blob.BlobContainerAsyncClient; import com.azure.storage.blob.BlobContainerClientBuilder; import com.microsoft.azure.kusto.data.UriUtils; import java.net.URISyntaxException; -public class ContainerWithSas implements ResourceWithSas { +public class ContainerWithSas implements ResourceWithSas { private final String sas; - private final BlobContainerClient container; + private final BlobContainerAsyncClient asyncContainer; public ContainerWithSas(String url, HttpClient httpClient) throws URISyntaxException { String[] parts = UriUtils.getSasAndEndpointFromResourceURL(url); @@ -17,33 +17,33 @@ public ContainerWithSas(String url, HttpClient httpClient) throws URISyntaxExcep String sas = parts[1]; this.sas = '?' + sas; - this.container = new BlobContainerClientBuilder() + this.asyncContainer = new BlobContainerClientBuilder() .endpoint(endpoint) .sasToken(sas) .httpClient(httpClient) - .buildClient(); + .buildAsyncClient(); } public String getSas() { return sas; } - public BlobContainerClient getContainer() { - return container; + public BlobContainerAsyncClient getAsyncContainer() { + return asyncContainer; } @Override public String getEndpointWithoutSas() { - return container.getBlobContainerUrl(); + return asyncContainer.getBlobContainerUrl(); } @Override public String getAccountName() { - return container.getAccountName(); + return asyncContainer.getAccountName(); } @Override - public BlobContainerClient getResource() { - return container; + public BlobContainerAsyncClient getResource() { + return asyncContainer; } } diff --git a/ingest/src/main/java/com/microsoft/azure/kusto/ingest/resources/QueueWithSas.java b/ingest/src/main/java/com/microsoft/azure/kusto/ingest/resources/QueueWithSas.java index baf52271d..f73ddae4e 100644 --- a/ingest/src/main/java/com/microsoft/azure/kusto/ingest/resources/QueueWithSas.java +++ b/ingest/src/main/java/com/microsoft/azure/kusto/ingest/resources/QueueWithSas.java @@ -2,53 +2,52 @@ import com.azure.core.http.HttpClient; import com.azure.storage.common.policy.RequestRetryOptions; -import com.azure.storage.queue.QueueClient; +import com.azure.storage.queue.QueueAsyncClient; import com.azure.storage.queue.QueueClientBuilder; -import com.azure.storage.queue.implementation.AzureQueueStorageImpl; import com.microsoft.azure.kusto.data.UriUtils; import reactor.util.annotation.Nullable; import java.net.URISyntaxException; -public class QueueWithSas implements ResourceWithSas { +public class QueueWithSas implements ResourceWithSas { private final String sas; - private final QueueClient queue; + private final QueueAsyncClient queueAsyncClient; public QueueWithSas(String url, HttpClient httpClient, @Nullable RequestRetryOptions retryOptions) throws URISyntaxException { String[] parts = UriUtils.getSasAndEndpointFromResourceURL(url); this.sas = '?' + parts[1]; - this.queue = new QueueClientBuilder() + this.queueAsyncClient = new QueueClientBuilder() .endpoint(parts[0]) .sasToken(parts[1]) .httpClient(httpClient) .retryOptions(retryOptions) - .buildClient(); + .buildAsyncClient(); } public String getSas() { return sas; } - public QueueClient getQueue() { - return queue; + public QueueAsyncClient getAsyncQueue() { + return queueAsyncClient; } public String getEndpoint() { - return queue.getQueueUrl() + sas; + return queueAsyncClient.getQueueUrl() + sas; } @Override public String getEndpointWithoutSas() { - return queue.getQueueUrl(); + return queueAsyncClient.getQueueUrl(); } @Override public String getAccountName() { - return queue.getAccountName(); + return queueAsyncClient.getAccountName(); } @Override - public QueueClient getResource() { - return queue; + public QueueAsyncClient getResource() { + return queueAsyncClient; } } diff --git a/ingest/src/main/java/com/microsoft/azure/kusto/ingest/result/IngestionResult.java b/ingest/src/main/java/com/microsoft/azure/kusto/ingest/result/IngestionResult.java index 898d395a4..777703b78 100644 --- a/ingest/src/main/java/com/microsoft/azure/kusto/ingest/result/IngestionResult.java +++ b/ingest/src/main/java/com/microsoft/azure/kusto/ingest/result/IngestionResult.java @@ -4,17 +4,23 @@ package com.microsoft.azure.kusto.ingest.result; import com.azure.data.tables.implementation.models.TableServiceErrorException; +import reactor.core.publisher.Mono; import java.io.Serializable; import java.net.URISyntaxException; import java.util.List; public interface IngestionResult extends Serializable { - /// - /// Retrieves the detailed ingestion status of - /// all data ingestion operations into Kusto associated with this com.microsoft.azure.kusto.ingest.IKustoIngestionResult instance. - /// - List getIngestionStatusCollection() throws URISyntaxException, TableServiceErrorException; - int getIngestionStatusesLength(); + /** + * Retrieves the detailed ingestion status of + * all data ingestion operations into Kusto associated with this com.microsoft.azure.kusto.ingest.IKustoIngestionResult instance. + */ + Mono> getIngestionStatusCollectionAsync() throws URISyntaxException, TableServiceErrorException; + + /** + * Blocking, retrieves the detailed ingestion status of + * all data ingestion operations into Kusto associated with this com.microsoft.azure.kusto.ingest.IKustoIngestionResult instance. + */ + List getIngestionStatusCollection() throws URISyntaxException, TableServiceErrorException; } diff --git a/ingest/src/main/java/com/microsoft/azure/kusto/ingest/result/IngestionStatusInTableDescription.java b/ingest/src/main/java/com/microsoft/azure/kusto/ingest/result/IngestionStatusInTableDescription.java index b7b9eb9e5..b171be89d 100644 --- a/ingest/src/main/java/com/microsoft/azure/kusto/ingest/result/IngestionStatusInTableDescription.java +++ b/ingest/src/main/java/com/microsoft/azure/kusto/ingest/result/IngestionStatusInTableDescription.java @@ -3,12 +3,13 @@ package com.microsoft.azure.kusto.ingest.result; -import com.azure.data.tables.TableClient; +import com.azure.data.tables.TableAsyncClient; import com.fasterxml.jackson.annotation.JsonIgnore; import com.fasterxml.jackson.annotation.JsonProperty; import com.microsoft.azure.kusto.ingest.utils.TableWithSas; import org.slf4j.Logger; import org.slf4j.LoggerFactory; +import reactor.util.annotation.Nullable; import java.io.Serializable; import java.lang.invoke.MethodHandles; @@ -26,7 +27,7 @@ public class IngestionStatusInTableDescription implements Serializable { private String rowKey; @JsonIgnore - private transient TableClient tableClient; + private transient TableAsyncClient tableAsyncClient; public String getTableConnectionString() { return this.tableConnectionString; @@ -52,20 +53,21 @@ public void setRowKey(String rowKey) { this.rowKey = rowKey; } - public TableClient getTableClient() { - if (tableClient == null) { + @Nullable + public TableAsyncClient getTableAsyncClient() { + if (tableAsyncClient == null) { try { - tableClient = TableWithSas.TableClientFromUrl(getTableConnectionString(), null); + tableAsyncClient = TableWithSas.createTableClientFromUrl(getTableConnectionString(), null); } catch (URISyntaxException uriSyntaxException) { log.error("TableConnectionString could not be parsed as URI reference.", uriSyntaxException); return null; } } - return tableClient; + return tableAsyncClient; } - public void setTableClient(TableClient tableClient) { - this.tableClient = tableClient; + public void setAsyncTableClient(TableAsyncClient tableAsyncClient) { + this.tableAsyncClient = tableAsyncClient; } } diff --git a/ingest/src/main/java/com/microsoft/azure/kusto/ingest/result/IngestionStatusResult.java b/ingest/src/main/java/com/microsoft/azure/kusto/ingest/result/IngestionStatusResult.java index 9a4a3d908..e05a2a838 100644 --- a/ingest/src/main/java/com/microsoft/azure/kusto/ingest/result/IngestionStatusResult.java +++ b/ingest/src/main/java/com/microsoft/azure/kusto/ingest/result/IngestionStatusResult.java @@ -3,6 +3,10 @@ package com.microsoft.azure.kusto.ingest.result; +import com.azure.data.tables.implementation.models.TableServiceErrorException; +import reactor.core.publisher.Mono; + +import java.net.URISyntaxException; import java.util.Collections; import java.util.List; @@ -15,12 +19,16 @@ public IngestionStatusResult(IngestionStatus ingestionStatus) { } @Override - public List getIngestionStatusCollection() { - return Collections.singletonList(this.ingestionStatus); + public Mono> getIngestionStatusCollectionAsync() { + if (ingestionStatus != null) { + return Mono.just(Collections.singletonList(ingestionStatus)); + } + + return Mono.empty(); } @Override - public int getIngestionStatusesLength() { - return 1; + public List getIngestionStatusCollection() throws URISyntaxException, TableServiceErrorException { + return getIngestionStatusCollectionAsync().block(); } } diff --git a/ingest/src/main/java/com/microsoft/azure/kusto/ingest/result/TableReportIngestionResult.java b/ingest/src/main/java/com/microsoft/azure/kusto/ingest/result/TableReportIngestionResult.java index 2dcbd9530..eb8dc31c2 100644 --- a/ingest/src/main/java/com/microsoft/azure/kusto/ingest/result/TableReportIngestionResult.java +++ b/ingest/src/main/java/com/microsoft/azure/kusto/ingest/result/TableReportIngestionResult.java @@ -3,12 +3,14 @@ package com.microsoft.azure.kusto.ingest.result; -import com.azure.data.tables.TableClient; +import com.azure.data.tables.TableAsyncClient; import com.azure.data.tables.implementation.models.TableServiceErrorException; -import com.azure.data.tables.models.TableEntity; +import reactor.core.publisher.Mono; +import java.net.URISyntaxException; import java.util.LinkedList; import java.util.List; +import java.util.stream.Collectors; public class TableReportIngestionResult implements IngestionResult { private final List descriptors; @@ -18,19 +20,30 @@ public TableReportIngestionResult(List descri } @Override - public List getIngestionStatusCollection() throws TableServiceErrorException { - List results = new LinkedList<>(); - for (IngestionStatusInTableDescription descriptor : descriptors) { - TableClient table = descriptor.getTableClient(); - TableEntity entity = table.getEntity(descriptor.getPartitionKey(), descriptor.getRowKey()); - results.add(IngestionStatus.fromEntity(entity)); - } - - return results; + public Mono> getIngestionStatusCollectionAsync() throws TableServiceErrorException { + List> ingestionStatusMonos = descriptors.stream() + .map(descriptor -> { + TableAsyncClient tableAsyncClient = descriptor.getTableAsyncClient(); + if (tableAsyncClient != null) { + return tableAsyncClient.getEntity(descriptor.getPartitionKey(), descriptor.getRowKey()) + .map(IngestionStatus::fromEntity); + } else { + return Mono.empty(); + } + }) + .collect(Collectors.toList()); + + return Mono.zip(ingestionStatusMonos, results -> { + List resultList = new LinkedList<>(); + for (Object result : results) { + resultList.add((IngestionStatus) result); + } + return resultList; + }); } @Override - public int getIngestionStatusesLength() { - return descriptors.size(); + public List getIngestionStatusCollection() throws URISyntaxException, TableServiceErrorException { + return getIngestionStatusCollectionAsync().block(); } } diff --git a/ingest/src/main/java/com/microsoft/azure/kusto/ingest/source/ResultSetSourceInfo.java b/ingest/src/main/java/com/microsoft/azure/kusto/ingest/source/ResultSetSourceInfo.java index 31f2cb9bf..ce2878c90 100644 --- a/ingest/src/main/java/com/microsoft/azure/kusto/ingest/source/ResultSetSourceInfo.java +++ b/ingest/src/main/java/com/microsoft/azure/kusto/ingest/source/ResultSetSourceInfo.java @@ -3,9 +3,6 @@ package com.microsoft.azure.kusto.ingest.source; -import com.microsoft.azure.kusto.data.instrumentation.TraceableAttributes; -import org.jetbrains.annotations.NotNull; - import java.sql.ResultSet; import java.util.Map; import java.util.Objects; diff --git a/ingest/src/main/java/com/microsoft/azure/kusto/ingest/utils/IngestionUtils.java b/ingest/src/main/java/com/microsoft/azure/kusto/ingest/utils/IngestionUtils.java index 5b314f0a4..30337bfce 100644 --- a/ingest/src/main/java/com/microsoft/azure/kusto/ingest/utils/IngestionUtils.java +++ b/ingest/src/main/java/com/microsoft/azure/kusto/ingest/utils/IngestionUtils.java @@ -1,5 +1,8 @@ package com.microsoft.azure.kusto.ingest.utils; +import com.azure.core.implementation.ByteBufferCollector; +import com.azure.core.util.FluxUtil; +import com.microsoft.azure.kusto.data.exceptions.ExceptionUtils; import com.microsoft.azure.kusto.ingest.ResettableFileInputStream; import com.microsoft.azure.kusto.ingest.exceptions.IngestionClientException; import com.microsoft.azure.kusto.ingest.source.CompressionType; @@ -7,12 +10,28 @@ import com.microsoft.azure.kusto.ingest.source.ResultSetSourceInfo; import com.microsoft.azure.kusto.ingest.source.StreamSourceInfo; import com.univocity.parsers.csv.CsvRoutines; +import io.netty.buffer.ByteBuf; +import io.netty.buffer.Unpooled; +import io.netty.channel.embedded.EmbeddedChannel; +import io.netty.handler.codec.compression.ZlibCodecFactory; +import io.netty.handler.codec.compression.ZlibWrapper; import org.jetbrains.annotations.NotNull; import org.slf4j.Logger; import org.slf4j.LoggerFactory; +import reactor.core.publisher.Flux; +import reactor.core.publisher.Mono; +import reactor.core.scheduler.Schedulers; -import java.io.*; +import java.io.ByteArrayInputStream; +import java.io.ByteArrayOutputStream; +import java.io.File; +import java.io.FileInputStream; +import java.io.FileNotFoundException; +import java.io.IOException; +import java.io.InputStream; import java.lang.invoke.MethodHandles; +import java.nio.ByteBuffer; +import java.util.zip.GZIPOutputStream; public class IngestionUtils { private IngestionUtils() { @@ -20,6 +39,7 @@ private IngestionUtils() { } private static final Logger log = LoggerFactory.getLogger(MethodHandles.lookup().lookupClass()); + private static final int STREAM_COMPRESS_BUFFER_SIZE = 16 * 1024; @NotNull public static StreamSourceInfo fileToStream(FileSourceInfo fileSourceInfo, boolean resettable) @@ -80,4 +100,114 @@ public static CompressionType getCompression(String fileName) { return null; } + + public static Mono compressStream1(InputStream uncompressedStream, boolean leaveOpen) { + log.debug("Compressing the stream."); + EmbeddedChannel encoder = new EmbeddedChannel(ZlibCodecFactory.newZlibEncoder(ZlibWrapper.GZIP)); + Flux byteBuffers = FluxUtil.toFluxByteBuffer(uncompressedStream); + + return byteBuffers + .switchIfEmpty(Mono.error(new IngestionClientException("Empty stream."))) + .reduce(new ByteBufferCollector(), (byteBufferCollector, byteBuffer) -> { + encoder.writeAndFlush(Unpooled.wrappedBuffer(byteBuffer)); + ByteBuf compressedByteBuf; + while ((compressedByteBuf = encoder.readOutbound()) != null) { + byteBufferCollector.write(compressedByteBuf.nioBuffer()); + compressedByteBuf.release(); + } + return byteBufferCollector; + }) + .map(byteBufferCollector -> { + encoder.finish(); + ByteBuf compressedByteBuf; + while ((compressedByteBuf = encoder.readOutbound()) != null) { + byteBufferCollector.write(compressedByteBuf.nioBuffer()); + compressedByteBuf.release(); + } + return byteBufferCollector; + }) + .map(ByteBufferCollector::toByteArray) + .doFinally(ignore -> { + encoder.finishAndReleaseAll(); + if (!leaveOpen) { + try { + uncompressedStream.close(); + } catch (IOException e) { + String msg = ExceptionUtils.getMessageEx(e); + log.error(msg, e); + throw new IngestionClientException(msg, e); + } + } + }).map(ByteArrayInputStream::new); + } + + public static Mono toCompressedByteArray(InputStream uncompressedStream, boolean leaveOpen) { + return Mono.fromCallable(() -> { + log.debug("Compressing the stream."); + ByteArrayOutputStream byteArrayOutputStream = new ByteArrayOutputStream(); + GZIPOutputStream gzipOutputStream = new GZIPOutputStream(byteArrayOutputStream); + byte[] b = new byte[STREAM_COMPRESS_BUFFER_SIZE]; + int read = uncompressedStream.read(b); + if (read == -1) { + String message = "Empty stream."; + log.error(message); + throw new IngestionClientException(message); + } + do { + gzipOutputStream.write(b, 0, read); + } while ((read = uncompressedStream.read(b)) != -1); + gzipOutputStream.flush(); + gzipOutputStream.close(); + byte[] content = byteArrayOutputStream.toByteArray(); + byteArrayOutputStream.close(); + if (!leaveOpen) { + uncompressedStream.close(); + } + return content; + }).subscribeOn(Schedulers.boundedElastic()); + } + + public static Mono compressStream(InputStream uncompressedStream, boolean leaveOpen) { + return toCompressedByteArray(uncompressedStream, leaveOpen) + .map(ByteArrayInputStream::new); + } + + public static Mono toByteArray(InputStream inputStream) { + return Mono.create(sink -> { + ByteArrayOutputStream byteArrayOutputStream = new ByteArrayOutputStream(); + byte[] buffer = new byte[8192]; + int bytesRead; + + try { + while ((bytesRead = inputStream.read(buffer)) != -1) { + byteArrayOutputStream.write(buffer, 0, bytesRead); + } + sink.success(byteArrayOutputStream.toByteArray()); + } catch (IOException e) { + sink.error(e); + } finally { + try { + inputStream.close(); + } catch (IOException e) { + sink.error(e); + } + } + }); + } + + public static class IntegerHolder { + int value; + + public int increment() { + return value++; + } + + public void add(int length) { + value += length; + } + + public int getValue() { + return value; + } + } } diff --git a/ingest/src/main/java/com/microsoft/azure/kusto/ingest/utils/TableWithSas.java b/ingest/src/main/java/com/microsoft/azure/kusto/ingest/utils/TableWithSas.java index 198b580eb..7a9936909 100644 --- a/ingest/src/main/java/com/microsoft/azure/kusto/ingest/utils/TableWithSas.java +++ b/ingest/src/main/java/com/microsoft/azure/kusto/ingest/utils/TableWithSas.java @@ -1,7 +1,7 @@ package com.microsoft.azure.kusto.ingest.utils; import com.azure.core.http.HttpClient; -import com.azure.data.tables.TableClient; +import com.azure.data.tables.TableAsyncClient; import com.azure.data.tables.TableClientBuilder; import com.microsoft.azure.kusto.data.UriUtils; import reactor.util.annotation.Nullable; @@ -10,22 +10,22 @@ public class TableWithSas { private final String uri; - private final TableClient table; + private final TableAsyncClient tableAsyncClient; public TableWithSas(String url, @Nullable HttpClient httpClient) throws URISyntaxException { this.uri = url; - this.table = TableClientFromUrl(url, httpClient); + this.tableAsyncClient = createTableClientFromUrl(url, httpClient); } public String getUri() { return uri; } - public TableClient getTable() { - return table; + public TableAsyncClient getTableAsyncClient() { + return tableAsyncClient; } - public static TableClient TableClientFromUrl(String url, @Nullable HttpClient httpClient) throws URISyntaxException { + public static TableAsyncClient createTableClientFromUrl(String url, @Nullable HttpClient httpClient) throws URISyntaxException { String[] parts = UriUtils.getSasAndEndpointFromResourceURL(url); int tableNameIndex = parts[0].lastIndexOf('/'); String tableName = parts[0].substring(tableNameIndex + 1); @@ -34,6 +34,6 @@ public static TableClient TableClientFromUrl(String url, @Nullable HttpClient ht .sasToken(parts[1]) .tableName(tableName) .httpClient(httpClient) - .buildClient(); + .buildAsyncClient(); } } diff --git a/ingest/src/test/java/com/microsoft/azure/kusto/ingest/AzureStorageClientTest.java b/ingest/src/test/java/com/microsoft/azure/kusto/ingest/AzureStorageClientTest.java index 33552a458..27a727fd9 100644 --- a/ingest/src/test/java/com/microsoft/azure/kusto/ingest/AzureStorageClientTest.java +++ b/ingest/src/test/java/com/microsoft/azure/kusto/ingest/AzureStorageClientTest.java @@ -3,23 +3,23 @@ package com.microsoft.azure.kusto.ingest; -import com.azure.data.tables.TableClient; +import com.azure.data.tables.TableAsyncClient; import com.azure.data.tables.models.TableEntity; -import com.azure.storage.blob.BlobClient; -import com.azure.storage.blob.BlobContainerClient; +import com.azure.storage.blob.BlobAsyncClient; +import com.azure.storage.blob.BlobContainerAsyncClient; import com.azure.storage.blob.BlobContainerClientBuilder; -import com.azure.storage.queue.QueueClient; +import com.azure.storage.queue.QueueAsyncClient; import com.microsoft.azure.kusto.data.Ensure; import com.microsoft.azure.kusto.ingest.source.CompressionType; import com.microsoft.azure.kusto.ingest.utils.IngestionUtils; import org.junit.jupiter.api.BeforeAll; import org.junit.jupiter.api.BeforeEach; import org.junit.jupiter.api.Test; +import reactor.core.publisher.Mono; import java.io.File; import java.io.IOException; import java.io.InputStream; -import java.net.URISyntaxException; import java.nio.file.Files; import java.nio.file.Paths; @@ -34,7 +34,7 @@ class AzureStorageClientTest { static private String testFilePath; static private File testFile; static private File testFileCompressed; - static private BlobClient blob; + static private BlobAsyncClient blob; @BeforeAll static void setUp() { @@ -42,7 +42,7 @@ static void setUp() { testFile = new File(testFilePath); String testFilePathCompressed = Paths.get("src", "test", "resources", "testdata.json.gz").toString(); testFileCompressed = new File(testFilePathCompressed); - blob = TestUtils.containerWithSasFromContainerName("storageUrl").getContainer().getBlobClient("bloby"); + blob = TestUtils.containerWithSasFromContainerName("storageUrl").getAsyncContainer().getBlobAsyncClient("bloby"); } @BeforeEach @@ -54,7 +54,7 @@ void uploadLocalFileToBlob(File file, String blobName, String storageUri, Ingest throws IOException { Ensure.fileExists(file, "file"); CompressionType sourceCompressionType = IngestionUtils.getCompression(file.getPath()); - azureStorageClientSpy.uploadLocalFileToBlob(file, blobName, new BlobContainerClientBuilder().endpoint(storageUri).buildClient(), + azureStorageClientSpy.uploadLocalFileToBlob(file, blobName, new BlobContainerClientBuilder().endpoint(storageUri).buildAsyncClient(), shouldCompress(sourceCompressionType, dataFormat)); } @@ -67,7 +67,7 @@ void postMessageToQueue_NullQueuePath_IllegalArgumentException() { @Test void postMessageToQueue_NullContent_IllegalArgumentException() { - QueueClient queue = TestUtils.queueWithSasFromQueueName("queue1").getQueue(); + QueueAsyncClient queue = TestUtils.queueWithSasFromQueueName("queue1").getAsyncQueue(); assertThrows( IllegalArgumentException.class, () -> azureStorageClient.postMessageToQueue(queue, null)); @@ -77,7 +77,7 @@ void postMessageToQueue_NullContent_IllegalArgumentException() { void postMessageToQueue_NullEntity_IllegalArgumentException() { assertThrows( IllegalArgumentException.class, - () -> azureStorageClient.azureTableInsertEntity(mock(TableClient.class), null)); + () -> azureStorageClient.azureTableInsertEntity(mock(TableAsyncClient.class), null)); } @Test @@ -89,23 +89,29 @@ void postMessageToQueue_NullTableUri_IllegalArgumentException() { } @Test - void uploadLocalFileToBlob_UncompressedFile_CompressAndUploadFileToBlobIsCalled() - throws IOException { - doNothing().when(azureStorageClientSpy).compressAndUploadFileToBlob(any(File.class), any(BlobClient.class)); - - azureStorageClientSpy.uploadLocalFileToBlob(testFile, "blobName", - new BlobContainerClientBuilder().endpoint("https://testcontosourl.com/blob").buildClient(), true); - verify(azureStorageClientSpy).compressAndUploadFileToBlob(any(), any(BlobClient.class)); + void uploadLocalFileToBlob_UncompressedFile_CompressAndUploadFileToBlobIsCalled() throws IOException { + doAnswer(answer -> Mono.empty()) + .when(azureStorageClientSpy) + .compressAndUploadFileToBlob(any(File.class), any(BlobAsyncClient.class)); + + azureStorageClientSpy.uploadLocalFileToBlob( + testFile, + "blobName", + new BlobContainerClientBuilder().endpoint("https://testcontosourl.com/blob").buildAsyncClient(), + true); + + verify(azureStorageClientSpy).compressAndUploadFileToBlob(any(), any(BlobAsyncClient.class)); } @Test - void uploadLocalFileToBlob_CompressedFile_UploadFileToBlobIsCalled() - throws IOException { - doNothing().when(azureStorageClientSpy).uploadFileToBlob(any(File.class), any(BlobClient.class)); + void uploadLocalFileToBlob_CompressedFile_UploadFileToBlobIsCalled() throws IOException { + doAnswer(answer -> Mono.empty()) + .when(azureStorageClientSpy) + .uploadFileToBlob(any(File.class), any(BlobAsyncClient.class)); uploadLocalFileToBlob(testFileCompressed, "blobName", "https://testcontosourl.com/blob", IngestionProperties.DataFormat.CSV); - verify(azureStorageClientSpy).uploadFileToBlob(any(File.class), any(BlobClient.class)); + verify(azureStorageClientSpy).uploadFileToBlob(any(File.class), any(BlobAsyncClient.class)); } @Test @@ -138,32 +144,34 @@ void uploadLocalFileToBlob_FileDoesNotExist_IOException() { } @Test - void uploadStreamToBlob_NotCompressMode_UploadStreamIsCalled() - throws IOException, URISyntaxException { + void uploadStreamToBlob_NotCompressMode_UploadStreamIsCalled() throws IOException { try (InputStream stream = Files.newInputStream(Paths.get(testFilePath))) { - doReturn(10).when(azureStorageClientSpy).uploadStream(any(InputStream.class), any(BlobClient.class)); + doAnswer(answer -> Mono.just(10)) + .when(azureStorageClientSpy) + .uploadStream(any(InputStream.class), any(BlobAsyncClient.class)); azureStorageClientSpy.uploadStreamToBlob(stream, "blobName", - new BlobContainerClientBuilder().endpoint("https://ms.com/storageUrl").buildClient(), false); - verify(azureStorageClientSpy).uploadStream(isA(InputStream.class), isA(BlobClient.class)); + new BlobContainerClientBuilder().endpoint("https://ms.com/storageUrl").buildAsyncClient(), false); + verify(azureStorageClientSpy).uploadStream(isA(InputStream.class), isA(BlobAsyncClient.class)); } } @Test - void uploadStreamToBlob_CompressMode_CompressAndUploadStreamIsCalled() - throws IOException, URISyntaxException { + void uploadStreamToBlob_CompressMode_CompressAndUploadStreamIsCalled() throws IOException { try (InputStream stream = Files.newInputStream(Paths.get(testFilePath))) { - doReturn(10).when(azureStorageClientSpy) - .compressAndUploadStream(any(InputStream.class), any(BlobClient.class)); + doAnswer(answer -> Mono.just(10)) + .when(azureStorageClientSpy) + .compressAndUploadStream(any(InputStream.class), any(BlobAsyncClient.class)); azureStorageClientSpy.uploadStreamToBlob(stream, "blobName", - new BlobContainerClientBuilder().endpoint("https://ms.com/storageUrl").buildClient(), true); - verify(azureStorageClientSpy).compressAndUploadStream(isA(InputStream.class), isA(BlobClient.class)); + new BlobContainerClientBuilder().endpoint("https://ms.com/storageUrl").buildAsyncClient(), true); + verify(azureStorageClientSpy).compressAndUploadStream(isA(InputStream.class), isA(BlobAsyncClient.class)); } } @Test void UploadStreamToBlob_NullInputStream_IllegalArgumentException() { - BlobContainerClient container = new BlobContainerClientBuilder().endpoint("https://blobPath.blob.core.windows.net/container/blob").buildClient(); + BlobContainerAsyncClient container = new BlobContainerClientBuilder().endpoint("https://blobPath.blob.core.windows.net/container/blob") + .buildAsyncClient(); assertThrows( IllegalArgumentException.class, () -> azureStorageClient.uploadStreamToBlob(null, "blobName", container, false)); @@ -172,7 +180,8 @@ void UploadStreamToBlob_NullInputStream_IllegalArgumentException() { @Test void uploadStreamToBlob_NullBlobName_IllegalArgumentException() throws IOException { try (InputStream stream = Files.newInputStream(Paths.get(testFilePath))) { - BlobContainerClient storageUrl = new BlobContainerClientBuilder().endpoint("https://blobPath.blob.core.windows.net/container/blob").buildClient(); + BlobContainerAsyncClient storageUrl = new BlobContainerClientBuilder().endpoint("https://blobPath.blob.core.windows.net/container/blob") + .buildAsyncClient(); assertThrows( IllegalArgumentException.class, () -> azureStorageClient.uploadStreamToBlob(stream, null, storageUrl, false)); diff --git a/ingest/src/test/java/com/microsoft/azure/kusto/ingest/E2ETest.java b/ingest/src/test/java/com/microsoft/azure/kusto/ingest/E2ETest.java index e240b0592..8c0ea0b86 100644 --- a/ingest/src/test/java/com/microsoft/azure/kusto/ingest/E2ETest.java +++ b/ingest/src/test/java/com/microsoft/azure/kusto/ingest/E2ETest.java @@ -780,12 +780,12 @@ void testStreamingIngestFromBlob() throws IngestionServiceException, IOException UUID.randomUUID(), item.ingestionProperties.getDataFormat()); - String blobPath = container.getContainer().getBlobContainerUrl() + "/" + blobName + container.getSas(); + String blobPath = container.getAsyncContainer().getBlobContainerUrl() + "/" + blobName + container.getSas(); azureStorageClient.uploadLocalFileToBlob(item.file, blobName, - container.getContainer(), !item.file.getName().endsWith(".gz")); + container.getAsyncContainer(), !item.file.getName().endsWith(".gz")).block(); try { - streamingIngestClient.ingestFromBlob(new BlobSourceInfo(blobPath), item.ingestionProperties); + managedStreamingIngestClient.ingestFromBlob(new BlobSourceInfo(blobPath), item.ingestionProperties); } catch (Exception ex) { Assertions.fail(ex); } diff --git a/ingest/src/test/java/com/microsoft/azure/kusto/ingest/ManagedStreamingTest.java b/ingest/src/test/java/com/microsoft/azure/kusto/ingest/ManagedStreamingTest.java index 80eca8d1f..82f04be65 100644 --- a/ingest/src/test/java/com/microsoft/azure/kusto/ingest/ManagedStreamingTest.java +++ b/ingest/src/test/java/com/microsoft/azure/kusto/ingest/ManagedStreamingTest.java @@ -2,6 +2,7 @@ import com.azure.data.tables.models.TableEntity; import com.microsoft.azure.kusto.data.ExponentialRetry; +import com.microsoft.azure.kusto.data.KustoOperationResult; import com.microsoft.azure.kusto.data.StreamingClient; import com.microsoft.azure.kusto.ingest.exceptions.IngestionClientException; import com.microsoft.azure.kusto.ingest.exceptions.IngestionServiceException; @@ -13,6 +14,7 @@ import org.junit.jupiter.api.BeforeAll; import org.junit.jupiter.api.Test; import org.mockito.ArgumentCaptor; +import reactor.core.publisher.Mono; import java.io.ByteArrayInputStream; import java.io.IOException; @@ -21,9 +23,16 @@ import java.util.Arrays; import java.util.Collections; -import static org.junit.jupiter.api.Assertions.*; +import static org.junit.jupiter.api.Assertions.assertEquals; +import static org.junit.jupiter.api.Assertions.assertFalse; +import static org.junit.jupiter.api.Assertions.assertTrue; import static org.mockito.ArgumentMatchers.any; -import static org.mockito.Mockito.*; +import static org.mockito.ArgumentMatchers.anyBoolean; +import static org.mockito.ArgumentMatchers.anyString; +import static org.mockito.Mockito.mock; +import static org.mockito.Mockito.spy; +import static org.mockito.Mockito.verify; +import static org.mockito.Mockito.when; public class ManagedStreamingTest { private static final ResourceManager resourceManagerMock = mock(ResourceManager.class); @@ -32,11 +41,13 @@ public class ManagedStreamingTest { private static QueuedIngestClient queuedIngestClientMock; private static IngestionProperties ingestionProperties; private static StreamingClient streamingClientMock; + private static KustoOperationResult kustoOperationResultMock; private static ManagedStreamingIngestClient managedStreamingIngestClient; private static ManagedStreamingIngestClient managedStreamingIngestClientSpy; @BeforeAll - static void setUp() throws Exception { + static void setUp() { + kustoOperationResultMock = mock(KustoOperationResult.class); when(resourceManagerMock.getShuffledContainers()) .thenReturn(Collections.singletonList(TestUtils.containerWithSasFromAccountNameAndContainerName(ACCOUNT_NAME, "someStorage"))); when(resourceManagerMock.getShuffledQueues()) @@ -47,12 +58,13 @@ static void setUp() throws Exception { when(resourceManagerMock.getIdentityToken()).thenReturn("identityToken"); - doNothing().when(azureStorageClientMock).azureTableInsertEntity(any(), any(TableEntity.class)); + when(azureStorageClientMock.azureTableInsertEntity(any(), any(TableEntity.class))).thenReturn(Mono.empty()); - doNothing().when(azureStorageClientMock).postMessageToQueue(any(), anyString()); + when(azureStorageClientMock.postMessageToQueue(any(), anyString())).thenReturn(Mono.empty()); + when(azureStorageClientMock.uploadStreamToBlob(any(), any(), any(), anyBoolean())).thenReturn(Mono.empty()); streamingClientMock = mock(StreamingClient.class); - when(streamingClientMock.executeStreamingIngest(any(String.class), any(String.class), any(InputStream.class), - isNull(), any(String.class), any(String.class), any(boolean.class))).thenReturn(null); + when(streamingClientMock.executeStreamingIngestAsync(any(), any(), any(), + any(), any(), any(), any(boolean.class))).thenReturn(Mono.just(kustoOperationResultMock)); ingestionProperties = new IngestionProperties("dbName", "tableName"); managedStreamingIngestClient = new ManagedStreamingIngestClient(resourceManagerMock, azureStorageClientMock, @@ -81,21 +93,21 @@ static int getStreamSize(InputStream inputStream) throws IOException { } @Test - void IngestFromStream_CsvStream() throws Exception { + void ingestFromStream_CsvStream() throws Exception { InputStream inputStream = createStreamOfSize(1); StreamSourceInfo streamSourceInfo = new StreamSourceInfo(inputStream); // Expect to work and also choose no queuing - OperationStatus status = managedStreamingIngestClient.ingestFromStream(streamSourceInfo, ingestionProperties).getIngestionStatusCollection() - .get(0).status; + OperationStatus status = managedStreamingIngestClient.ingestFromStream(streamSourceInfo, ingestionProperties).getIngestionStatusCollectionAsync() + .block().get(0).status; assertEquals(OperationStatus.Succeeded, status); BooleanConsumer assertPolicyWorked = (boolean wasExpectedToUseQueuing) -> { try { inputStream.reset(); IngestionStatus ingestionStatus = managedStreamingIngestClient.ingestFromStream(streamSourceInfo, ingestionProperties) - .getIngestionStatusCollection().get(0); + .getIngestionStatusCollectionAsync().block().get(0); if (wasExpectedToUseQueuing) { assertEquals(OperationStatus.Queued, ingestionStatus.status); } else { @@ -180,9 +192,9 @@ void ManagedStreaming_BigFile_ShouldQueueTheFullStream() throws IOException, Ing int size = inputStream.bb.available(); StreamSourceInfo streamSourceInfo = new StreamSourceInfo(inputStream); ArgumentCaptor streamSourceInfoCaptor = ArgumentCaptor.forClass(StreamSourceInfo.class); - + when(queuedIngestClientMock.ingestFromStreamAsync(any(), any())).thenReturn(Mono.empty()); managedStreamingIngestClientSpy.ingestFromStream(streamSourceInfo, ingestionProperties); - verify(queuedIngestClientMock, times(1)).ingestFromStream(streamSourceInfoCaptor.capture(), any()); + verify(queuedIngestClientMock).ingestFromStreamAsync(streamSourceInfoCaptor.capture(), any()); StreamSourceInfo value = streamSourceInfoCaptor.getValue(); int queuedStreamSize = getStreamSize(value.getStream()); diff --git a/ingest/src/test/java/com/microsoft/azure/kusto/ingest/QueuedIngestClientTest.java b/ingest/src/test/java/com/microsoft/azure/kusto/ingest/QueuedIngestClientTest.java index 2f1b92967..81e90593f 100644 --- a/ingest/src/test/java/com/microsoft/azure/kusto/ingest/QueuedIngestClientTest.java +++ b/ingest/src/test/java/com/microsoft/azure/kusto/ingest/QueuedIngestClientTest.java @@ -12,7 +12,11 @@ import com.microsoft.azure.kusto.ingest.result.IngestionStatus; import com.microsoft.azure.kusto.ingest.result.OperationStatus; import com.microsoft.azure.kusto.ingest.result.ValidationPolicy; -import com.microsoft.azure.kusto.ingest.source.*; +import com.microsoft.azure.kusto.ingest.source.BlobSourceInfo; +import com.microsoft.azure.kusto.ingest.source.CompressionType; +import com.microsoft.azure.kusto.ingest.source.FileSourceInfo; +import com.microsoft.azure.kusto.ingest.source.ResultSetSourceInfo; +import com.microsoft.azure.kusto.ingest.source.StreamSourceInfo; import com.microsoft.azure.kusto.ingest.utils.IngestionUtils; import org.junit.jupiter.api.AfterEach; import org.junit.jupiter.api.BeforeAll; @@ -22,19 +26,39 @@ import org.junit.jupiter.params.provider.Arguments; import org.junit.jupiter.params.provider.MethodSource; import org.mockito.ArgumentCaptor; +import reactor.core.publisher.Mono; +import reactor.test.StepVerifier; import java.io.IOException; import java.io.InputStream; import java.net.URISyntaxException; import java.nio.file.Files; import java.nio.file.Paths; -import java.sql.*; +import java.sql.Connection; +import java.sql.DriverManager; +import java.sql.ResultSet; +import java.sql.SQLException; +import java.sql.Statement; import java.util.Collections; import java.util.function.BiFunction; import java.util.stream.Stream; -import static org.junit.jupiter.api.Assertions.*; -import static org.mockito.Mockito.*; +import static org.junit.jupiter.api.Assertions.assertEquals; +import static org.junit.jupiter.api.Assertions.assertFalse; +import static org.junit.jupiter.api.Assertions.assertNotEquals; +import static org.junit.jupiter.api.Assertions.assertNotNull; +import static org.junit.jupiter.api.Assertions.assertThrows; +import static org.junit.jupiter.api.Assertions.assertTrue; +import static org.mockito.Mockito.any; +import static org.mockito.Mockito.anyBoolean; +import static org.mockito.Mockito.anyString; +import static org.mockito.Mockito.atLeast; +import static org.mockito.Mockito.atLeastOnce; +import static org.mockito.Mockito.doReturn; +import static org.mockito.Mockito.mock; +import static org.mockito.Mockito.spy; +import static org.mockito.Mockito.verify; +import static org.mockito.Mockito.when; class QueuedIngestClientTest { private static final ResourceManager resourceManagerMock = mock(ResourceManager.class); @@ -57,9 +81,11 @@ static void setUp() throws Exception { when(resourceManagerMock.getIdentityToken()).thenReturn("identityToken"); - doNothing().when(azureStorageClientMock).azureTableInsertEntity(any(), any(TableEntity.class)); + when(azureStorageClientMock.azureTableInsertEntity(any(), any(TableEntity.class))) + .thenReturn(Mono.empty()); - doNothing().when(azureStorageClientMock).postMessageToQueue(any(), anyString()); + when(azureStorageClientMock.postMessageToQueue(any(), anyString())).thenReturn(Mono.empty()); + when(azureStorageClientMock.uploadStreamToBlob(any(), any(), any(), anyBoolean())).thenReturn(Mono.empty()); } @BeforeEach @@ -83,7 +109,7 @@ void tareEach() throws IOException { void ingestFromBlob_IngestionReportMethodIsNotTable_EmptyIngestionStatus() throws Exception { BlobSourceInfo blobSourceInfo = new BlobSourceInfo("https://blobPath.blob.core.windows.net/container/blob"); IngestionResult result = queuedIngestClient.ingestFromBlob(blobSourceInfo, ingestionProperties); - assertEquals(OperationStatus.Queued, result.getIngestionStatusCollection().get(0).status); + assertEquals(OperationStatus.Queued, result.getIngestionStatusCollectionAsync().block().get(0).status); } @Test @@ -92,7 +118,6 @@ void ingestFromBlob_IngestionReportMethodIsTable_NotEmptyIngestionStatus() throw ingestionProperties.setReportMethod(IngestionProperties.IngestionReportMethod.TABLE); IngestionResult result = queuedIngestClient.ingestFromBlob(blobSourceInfo, ingestionProperties); - assertNotEquals(0, result.getIngestionStatusesLength()); } @Test @@ -187,11 +212,12 @@ void ingestFromFile_NullFileSourceInfo_IllegalArgumentException() { } @Test - void ingestFromFile_FileDoesNotExist_IngestionClientException() { + void ingestFromFileAsync_FileDoesNotExist_IngestionClientException() { FileSourceInfo fileSourceInfo = new FileSourceInfo("file.path"); - assertThrows( - IngestionClientException.class, - () -> queuedIngestClient.ingestFromFile(fileSourceInfo, ingestionProperties)); + Mono result = queuedIngestClient.ingestFromFileAsync(fileSourceInfo, ingestionProperties); + StepVerifier.create(result) + .expectError(IngestionClientException.class) + .verify(); } @Test @@ -239,40 +265,40 @@ void ingestFromResultSet_NullResultSetSourceInfo_IllegalArgumentException() { } @Test - void ingestFromResultSet_StreamIngest_IngestionClientException() throws Exception { + void ingestFromResultSetAsync_StreamIngest_IngestionClientException() throws Exception { try (IngestClient ingestClient = new QueuedIngestClientImpl(resourceManagerMock, azureStorageClientMock)) { // we need a spy to intercept the call to ingestFromStream so it wouldn't be called IngestClient ingestClientSpy = spy(ingestClient); IngestionClientException ingestionClientException = new IngestionClientException( "Client exception in ingestFromFile"); - doThrow(ingestionClientException).when(ingestClientSpy).ingestFromStream(any(), any()); + doReturn(Mono.error(ingestionClientException)).when(ingestClientSpy).ingestFromStreamAsync(any(), any()); ResultSet resultSet = getSampleResultSet(); ResultSetSourceInfo resultSetSourceInfo = new ResultSetSourceInfo(resultSet); - assertThrows( - IngestionClientException.class, - () -> ingestClientSpy.ingestFromResultSet(resultSetSourceInfo, ingestionProperties)); + StepVerifier.create(ingestClientSpy.ingestFromResultSetAsync(resultSetSourceInfo, ingestionProperties)) + .expectError(IngestionClientException.class) + .verify(); } } @Test - void ingestFromResultSet_StreamIngest_IngestionServiceException() throws Exception { + void ingestFromResultSetAsync_StreamIngest_IngestionServiceException() throws Exception { try (IngestClient ingestClient = new QueuedIngestClientImpl(resourceManagerMock, azureStorageClientMock)) { // we need a spy to intercept the call to ingestFromStream so it wouldn't be called IngestClient ingestClientSpy = spy(ingestClient); IngestionServiceException ingestionServiceException = new IngestionServiceException( "Service exception in ingestFromFile"); - doThrow(ingestionServiceException).when(ingestClientSpy).ingestFromStream(any(), any()); + doReturn(Mono.error(ingestionServiceException)).when(ingestClientSpy).ingestFromStreamAsync(any(), any()); ResultSet resultSet = getSampleResultSet(); ResultSetSourceInfo resultSetSourceInfo = new ResultSetSourceInfo(resultSet); - assertThrows( - IngestionServiceException.class, - () -> ingestClientSpy.ingestFromResultSet(resultSetSourceInfo, ingestionProperties)); + StepVerifier.create(ingestClientSpy.ingestFromResultSetAsync(resultSetSourceInfo, ingestionProperties)) + .expectError(IngestionServiceException.class) + .verify(); } } @@ -282,7 +308,7 @@ void ingestFromResultSet_StreamIngest_VerifyStreamContent() throws Exception { // we need a spy to intercept the call to ingestFromStream so it wouldn't be called IngestClient ingestClientSpy = spy(ingestClient); - doReturn(null).when(ingestClientSpy).ingestFromStream(any(), any()); + doReturn(Mono.empty()).when(ingestClientSpy).ingestFromStreamAsync(any(), any()); ResultSet resultSet = getSampleResultSet(); ResultSetSourceInfo resultSetSourceInfo = new ResultSetSourceInfo(resultSet); @@ -291,7 +317,7 @@ void ingestFromResultSet_StreamIngest_VerifyStreamContent() throws Exception { ArgumentCaptor argumentCaptor = ArgumentCaptor.forClass(StreamSourceInfo.class); - verify(ingestClientSpy, atLeastOnce()).ingestFromStream(argumentCaptor.capture(), any()); + verify(ingestClientSpy, atLeastOnce()).ingestFromStreamAsync(argumentCaptor.capture(), any()); InputStream ingestFromStreamReceivedStream = argumentCaptor.getValue().getStream(); int len = ingestFromStreamReceivedStream.available(); diff --git a/ingest/src/test/java/com/microsoft/azure/kusto/ingest/ResourceManagerTest.java b/ingest/src/test/java/com/microsoft/azure/kusto/ingest/ResourceManagerTest.java index d6b8ee424..7c631ad14 100644 --- a/ingest/src/test/java/com/microsoft/azure/kusto/ingest/ResourceManagerTest.java +++ b/ingest/src/test/java/com/microsoft/azure/kusto/ingest/ResourceManagerTest.java @@ -3,7 +3,7 @@ package com.microsoft.azure.kusto.ingest; -import com.azure.storage.blob.BlobContainerClient; +import com.azure.storage.blob.BlobContainerAsyncClient; import com.fasterxml.jackson.databind.ObjectMapper; import com.microsoft.azure.kusto.data.Client; import com.microsoft.azure.kusto.data.KustoOperationResult; @@ -21,12 +21,20 @@ import org.junit.jupiter.api.Test; import java.io.IOException; -import java.util.*; +import java.util.ArrayList; +import java.util.Arrays; +import java.util.Collections; +import java.util.HashSet; +import java.util.List; +import java.util.Map; +import java.util.Set; import java.util.regex.Matcher; import java.util.regex.Pattern; import java.util.stream.Collectors; -import static org.junit.jupiter.api.Assertions.*; +import static org.junit.jupiter.api.Assertions.assertEquals; +import static org.junit.jupiter.api.Assertions.assertTrue; +import static org.junit.jupiter.api.Assertions.fail; import static org.mockito.Mockito.mock; import static org.mockito.Mockito.when; @@ -77,13 +85,16 @@ static KustoOperationResult generateIngestionResourcesResult() throws KustoServi ObjectMapper objectMapper = Utils.getObjectMapper(); List> valuesList = new ArrayList<>(); for (int i = 0; i < STORAGES.size(); i++) { - valuesList.add(new ArrayList<>((Arrays.asList("TempStorage", STORAGES.get(i).getContainer().getBlobContainerUrl() + STORAGES.get(i).getSas())))); valuesList - .add(new ArrayList<>((Arrays.asList("SecuredReadyForAggregationQueue", QUEUES.get(i).getQueue().getQueueUrl() + QUEUES.get(i).getSas())))); + .add(new ArrayList<>((Arrays.asList("TempStorage", STORAGES.get(i).getAsyncContainer().getBlobContainerUrl() + STORAGES.get(i).getSas())))); + valuesList + .add(new ArrayList<>( + (Arrays.asList("SecuredReadyForAggregationQueue", QUEUES.get(i).getAsyncQueue().getQueueUrl() + QUEUES.get(i).getSas())))); } - valuesList.add(new ArrayList<>((Arrays.asList("FailedIngestionsQueue", FAILED_QUEUE_RES.getQueue().getQueueUrl() + FAILED_QUEUE_RES.getSas())))); - valuesList.add(new ArrayList<>((Arrays.asList("SuccessfulIngestionsQueue", SUCCESS_QUEUE_RES.getQueue().getQueueUrl() + SUCCESS_QUEUE_RES.getSas())))); - valuesList.add(new ArrayList<>((Arrays.asList("IngestionsStatusTable", STATUS_TABLE_RES.getTable().getTableEndpoint() + "?sas")))); + valuesList.add(new ArrayList<>((Arrays.asList("FailedIngestionsQueue", FAILED_QUEUE_RES.getAsyncQueue().getQueueUrl() + FAILED_QUEUE_RES.getSas())))); + valuesList.add( + new ArrayList<>((Arrays.asList("SuccessfulIngestionsQueue", SUCCESS_QUEUE_RES.getAsyncQueue().getQueueUrl() + SUCCESS_QUEUE_RES.getSas())))); + valuesList.add(new ArrayList<>((Arrays.asList("IngestionsStatusTable", STATUS_TABLE_RES.getTableAsyncClient().getTableEndpoint() + "?sas")))); String listAsJson = objectMapper.writeValueAsString(valuesList); String response = "{\"Tables\":[{\"TableName\":\"Table_0\",\"Columns\":[{\"ColumnName\":\"ResourceTypeName\"," + "\"DataType\":\"String\",\"ColumnType\":\"string\"},{\"ColumnName\":\"StorageRoot\",\"DataType\":" + @@ -211,8 +222,8 @@ void getIngestionResource_WhenNewStorageContainersArrive_ShouldReturnOnlyNewReso } void validateStorage(List storages) { - Map> storageByAccount = storages.stream().map(ContainerWithSas::getContainer) - .collect(Collectors.groupingBy(BlobContainerClient::getAccountName)); + Map> storageByAccount = storages.stream().map(ContainerWithSas::getAsyncContainer) + .collect(Collectors.groupingBy(BlobContainerAsyncClient::getAccountName)); assertEquals(ACCOUNTS_COUNT, storageByAccount.size()); } diff --git a/ingest/src/test/java/com/microsoft/azure/kusto/ingest/StreamingIngestClientTest.java b/ingest/src/test/java/com/microsoft/azure/kusto/ingest/StreamingIngestClientTest.java index 3dc3946c6..84ea09263 100644 --- a/ingest/src/test/java/com/microsoft/azure/kusto/ingest/StreamingIngestClientTest.java +++ b/ingest/src/test/java/com/microsoft/azure/kusto/ingest/StreamingIngestClientTest.java @@ -3,10 +3,11 @@ package com.microsoft.azure.kusto.ingest; -import com.azure.storage.blob.BlobClient; +import com.azure.storage.blob.BlobAsyncClient; import com.azure.storage.blob.models.BlobProperties; import com.azure.storage.blob.specialized.BlobInputStream; import com.microsoft.azure.kusto.data.ClientRequestProperties; +import com.microsoft.azure.kusto.data.KustoOperationResult; import com.microsoft.azure.kusto.data.StreamingClient; import com.microsoft.azure.kusto.data.auth.ConnectionStringBuilder; import com.microsoft.azure.kusto.data.exceptions.DataClientException; @@ -15,7 +16,11 @@ import com.microsoft.azure.kusto.ingest.exceptions.IngestionServiceException; import com.microsoft.azure.kusto.ingest.result.IngestionResult; import com.microsoft.azure.kusto.ingest.result.OperationStatus; -import com.microsoft.azure.kusto.ingest.source.*; +import com.microsoft.azure.kusto.ingest.source.BlobSourceInfo; +import com.microsoft.azure.kusto.ingest.source.CompressionType; +import com.microsoft.azure.kusto.ingest.source.FileSourceInfo; +import com.microsoft.azure.kusto.ingest.source.ResultSetSourceInfo; +import com.microsoft.azure.kusto.ingest.source.StreamSourceInfo; import org.junit.jupiter.api.BeforeAll; import org.junit.jupiter.api.BeforeEach; import org.junit.jupiter.api.Test; @@ -26,10 +31,11 @@ import org.mockito.ArgumentCaptor; import org.mockito.Captor; import org.mockito.Mock; +import reactor.core.publisher.Mono; +import reactor.test.StepVerifier; import java.io.ByteArrayInputStream; import java.io.ByteArrayOutputStream; -import java.io.File; import java.io.InputStream; import java.net.URISyntaxException; import java.nio.charset.StandardCharsets; @@ -41,8 +47,15 @@ import java.util.zip.GZIPInputStream; import java.util.zip.GZIPOutputStream; -import static org.junit.jupiter.api.Assertions.*; -import static org.mockito.Mockito.*; +import static org.junit.jupiter.api.Assertions.assertEquals; +import static org.junit.jupiter.api.Assertions.assertNotNull; +import static org.junit.jupiter.api.Assertions.assertThrows; +import static org.mockito.Mockito.any; +import static org.mockito.Mockito.atLeastOnce; +import static org.mockito.Mockito.isNull; +import static org.mockito.Mockito.mock; +import static org.mockito.Mockito.verify; +import static org.mockito.Mockito.when; class StreamingIngestClientTest { private static StreamingIngestClient streamingIngestClient; @@ -54,6 +67,8 @@ class StreamingIngestClientTest { @Captor private static ArgumentCaptor argumentCaptor; + private static KustoOperationResult kustoOperationResult; + private final String resourcesDirectory = System.getProperty("user.dir") + "/src/test/resources/"; @BeforeAll @@ -61,16 +76,15 @@ static void setUp() { streamingClientMock = mock(StreamingClient.class); streamingIngestClient = new StreamingIngestClient(streamingClientMock); argumentCaptor = ArgumentCaptor.forClass((InputStream.class)); + kustoOperationResult = mock(KustoOperationResult.class); } @BeforeEach void setUpEach() throws Exception { ingestionProperties = new IngestionProperties("dbName", "tableName"); - when(streamingClientMock.executeStreamingIngest(any(String.class), any(String.class), any(InputStream.class), - isNull(), any(String.class), any(String.class), any(boolean.class))).thenReturn(null); - - when(streamingClientMock.executeStreamingIngest(any(String.class), any(String.class), any(InputStream.class), - isNull(), any(String.class), isNull(), any(boolean.class))).thenReturn(null); + when(streamingClientMock.executeStreamingIngestAsync(any(), any(), any(), any(), any(), any(), any(boolean.class))) + .thenReturn(Mono.just(kustoOperationResult)); + when(streamingClientMock.executeStreamingIngestFromBlobAsync(any(), any(), any(), any(), any(), any())).thenReturn(Mono.just(kustoOperationResult)); } @Test @@ -78,9 +92,10 @@ void IngestFromStream_CsvStream() throws Exception { String data = "Name, Age, Weight, Height"; InputStream inputStream = new ByteArrayInputStream(StandardCharsets.UTF_8.encode(data).array()); StreamSourceInfo streamSourceInfo = new StreamSourceInfo(inputStream); - OperationStatus status = streamingIngestClient.ingestFromStream(streamSourceInfo, ingestionProperties).getIngestionStatusCollection().get(0).status; + OperationStatus status = streamingIngestClient.ingestFromStream(streamSourceInfo, ingestionProperties).getIngestionStatusCollectionAsync().block() + .get(0).status; assertEquals(OperationStatus.Succeeded, status); - verify(streamingClientMock, atLeastOnce()).executeStreamingIngest(any(String.class), any(String.class), argumentCaptor.capture(), + verify(streamingClientMock, atLeastOnce()).executeStreamingIngestAsync(any(String.class), any(String.class), argumentCaptor.capture(), isNull(), any(String.class), isNull(), any(boolean.class)); /* @@ -98,11 +113,15 @@ void ingestFromStream_CsvStream_WithClientRequestId() throws Exception { InputStream inputStream = new ByteArrayInputStream(StandardCharsets.UTF_8.encode(data).array()); StreamSourceInfo streamSourceInfo = new StreamSourceInfo(inputStream); String clientRequestId = "clientRequestId"; - OperationStatus status = streamingIngestClient.ingestFromStream(streamSourceInfo, ingestionProperties, clientRequestId).getIngestionStatusCollection() - .get(0).status; + + IngestionResult ingestionResult = streamingIngestClient.ingestFromStreamAsync(streamSourceInfo, ingestionProperties, clientRequestId).block(); + assertNotNull(ingestionResult); + + OperationStatus status = ingestionResult.getIngestionStatusCollectionAsync() + .block().get(0).status; assertEquals(OperationStatus.Succeeded, status); ArgumentCaptor clientRequestPropertiesArgumentCaptor = ArgumentCaptor.forClass(ClientRequestProperties.class); - verify(streamingClientMock, atLeastOnce()).executeStreamingIngest(any(String.class), any(String.class), argumentCaptor.capture(), + verify(streamingClientMock, atLeastOnce()).executeStreamingIngestAsync(any(String.class), any(String.class), argumentCaptor.capture(), clientRequestPropertiesArgumentCaptor.capture(), any(String.class), isNull(), any(boolean.class)); /* @@ -129,9 +148,10 @@ void ingestFromStream_CompressedCsvStream() throws Exception { StreamSourceInfo streamSourceInfo = new StreamSourceInfo(inputStream); // When ingesting compressed data, we should set this property true to avoid double compression. streamSourceInfo.setCompressionType(CompressionType.gz); - OperationStatus status = streamingIngestClient.ingestFromStream(streamSourceInfo, ingestionProperties).getIngestionStatusCollection().get(0).status; + OperationStatus status = streamingIngestClient.ingestFromStream(streamSourceInfo, ingestionProperties).getIngestionStatusCollectionAsync().block() + .get(0).status; assertEquals(OperationStatus.Succeeded, status); - verify(streamingClientMock, atLeastOnce()).executeStreamingIngest(any(String.class), any(String.class), argumentCaptor.capture(), + verify(streamingClientMock, atLeastOnce()).executeStreamingIngestAsync(any(String.class), any(String.class), argumentCaptor.capture(), isNull(), any(String.class), isNull(), any(boolean.class)); InputStream stream = argumentCaptor.getValue(); @@ -145,9 +165,10 @@ void ingestFromStream_JsonStream() throws Exception { StreamSourceInfo streamSourceInfo = new StreamSourceInfo(inputStream); ingestionProperties.setDataFormat(IngestionProperties.DataFormat.JSON); ingestionProperties.setIngestionMapping("JsonMapping", IngestionMapping.IngestionMappingKind.JSON); - OperationStatus status = streamingIngestClient.ingestFromStream(streamSourceInfo, ingestionProperties).getIngestionStatusCollection().get(0).status; + OperationStatus status = streamingIngestClient.ingestFromStream(streamSourceInfo, ingestionProperties).getIngestionStatusCollectionAsync().block() + .get(0).status; assertEquals(OperationStatus.Succeeded, status); - verify(streamingClientMock, atLeastOnce()).executeStreamingIngest(any(String.class), any(String.class), argumentCaptor.capture(), + verify(streamingClientMock, atLeastOnce()).executeStreamingIngestAsync(any(String.class), any(String.class), argumentCaptor.capture(), isNull(), any(String.class), any(String.class), any(boolean.class)); InputStream stream = argumentCaptor.getValue(); @@ -168,9 +189,10 @@ void ingestFromStream_CompressedJsonStream() throws Exception { streamSourceInfo.setCompressionType(CompressionType.gz); ingestionProperties.setDataFormat(IngestionProperties.DataFormat.JSON); ingestionProperties.setIngestionMapping("JsonMapping", IngestionMapping.IngestionMappingKind.JSON); - OperationStatus status = streamingIngestClient.ingestFromStream(streamSourceInfo, ingestionProperties).getIngestionStatusCollection().get(0).status; + OperationStatus status = streamingIngestClient.ingestFromStream(streamSourceInfo, ingestionProperties).getIngestionStatusCollectionAsync().block() + .get(0).status; assertEquals(OperationStatus.Succeeded, status); - verify(streamingClientMock, atLeastOnce()).executeStreamingIngest(any(String.class), any(String.class), argumentCaptor.capture(), + verify(streamingClientMock, atLeastOnce()).executeStreamingIngestAsync(any(String.class), any(String.class), argumentCaptor.capture(), isNull(), any(String.class), any(String.class), any(boolean.class)); InputStream stream = argumentCaptor.getValue(); @@ -215,21 +237,20 @@ void ingestFromStream_JsonNoMappingReference_IngestionSucceeds() ingestionProperties.setDataFormat(IngestionProperties.DataFormat.JSON); ingestionProperties.setIngestionMapping("JsonMapping", IngestionMapping.IngestionMappingKind.JSON); IngestionResult ingestionResult = streamingIngestClient.ingestFromStream(streamSourceInfo, ingestionProperties); - assertEquals("Succeeded", ingestionResult.getIngestionStatusCollection().get(0).status.name()); - assertEquals(1, ingestionResult.getIngestionStatusesLength()); + assertEquals("Succeeded", ingestionResult.getIngestionStatusCollectionAsync().block().get(0).status.name()); } @Test - void ingestFromStream_JsonWrongMappingKind_IngestionClientException() { + void ingestFromStreamAsync_JsonWrongMappingKind_IngestionClientException() { String data = "{\"Name\": \"name\", \"Age\": \"age\", \"Weight\": \"weight\", \"Height\": \"height\"}"; InputStream inputStream = new ByteArrayInputStream(StandardCharsets.UTF_8.encode(data).array()); StreamSourceInfo streamSourceInfo = new StreamSourceInfo(inputStream); ingestionProperties.setDataFormat(IngestionProperties.DataFormat.JSON); ingestionProperties.setIngestionMapping("CsvMapping", IngestionMapping.IngestionMappingKind.CSV); - IngestionClientException ingestionClientException = assertThrows(IngestionClientException.class, - () -> streamingIngestClient.ingestFromStream(streamSourceInfo, ingestionProperties), - "Expected IngestionClientException to be thrown, but it didn't"); - assertTrue(ingestionClientException.getMessage().contains("Wrong ingestion mapping for format 'json'; mapping kind should be 'Json', but was 'Csv'.")); + StepVerifier.create(streamingIngestClient.ingestFromStreamAsync(streamSourceInfo, ingestionProperties)) + .expectErrorMatches(e -> e instanceof IngestionClientException + && e.getMessage().contains("Wrong ingestion mapping for format 'json'; mapping kind should be 'Json', but was 'Csv'.")) + .verify(); } @Test @@ -240,65 +261,67 @@ void ingestFromStream_AvroNoMappingReference_IngestionSucceeds() ingestionProperties.setDataFormat(IngestionProperties.DataFormat.AVRO); ingestionProperties.setIngestionMapping("AvroMapping", IngestionMapping.IngestionMappingKind.AVRO); IngestionResult ingestionResult = streamingIngestClient.ingestFromStream(streamSourceInfo, ingestionProperties); - assertEquals("Succeeded", ingestionResult.getIngestionStatusCollection().get(0).status.name()); - assertEquals(1, ingestionResult.getIngestionStatusesLength()); + assertEquals("Succeeded", ingestionResult.getIngestionStatusCollectionAsync().block().get(0).status.name()); } @Test - void ingestFromStream_AvroWrongMappingKind_IngestionClientException() { + void ingestFromStreamAsync_AvroWrongMappingKind_IngestionClientException() { InputStream inputStream = new ByteArrayInputStream(new byte[10]); StreamSourceInfo streamSourceInfo = new StreamSourceInfo(inputStream); ingestionProperties.setDataFormat(IngestionProperties.DataFormat.AVRO); ingestionProperties.setIngestionMapping("CsvMapping", IngestionMapping.IngestionMappingKind.CSV); - IngestionClientException ingestionClientException = assertThrows(IngestionClientException.class, - () -> streamingIngestClient.ingestFromStream(streamSourceInfo, ingestionProperties), - "Expected IngestionClientException to be thrown, but it didn't"); - assertTrue(ingestionClientException.getMessage().contains("Wrong ingestion mapping for format 'avro'; mapping kind should be 'Avro', but was 'Csv'.")); + StepVerifier.create(streamingIngestClient.ingestFromStreamAsync(streamSourceInfo, ingestionProperties)) + .expectErrorMatches(e -> e instanceof IngestionClientException + && e.getMessage().contains("Wrong ingestion mapping for format 'avro'; mapping kind should be 'Avro', but was 'Csv'.")) + .verify(); } @Test - void ingestFromStream_EmptyStream_IngestionClientException() { + void ingestFromStreamAsync_EmptyStream_IngestionClientException() { InputStream inputStream = new ByteArrayInputStream(new byte[0]); StreamSourceInfo streamSourceInfo = new StreamSourceInfo(inputStream); - IngestionClientException ingestionClientException = assertThrows(IngestionClientException.class, - () -> streamingIngestClient.ingestFromStream(streamSourceInfo, ingestionProperties), - "Expected IngestionClientException to be thrown, but it didn't"); - assertTrue(ingestionClientException.getMessage().contains("Empty stream.")); + StepVerifier.create(streamingIngestClient.ingestFromStreamAsync(streamSourceInfo, ingestionProperties)) + .expectErrorMatches(e -> e instanceof IngestionClientException && e.getMessage().contains("Empty stream.")) + .verify(); } @Test - void ingestFromStream_CaughtDataClientException_IngestionClientException() throws Exception { - when(streamingClientMock.executeStreamingIngest(any(String.class), any(String.class), any(InputStream.class), - isNull(), any(String.class), isNull(), any(boolean.class))).thenThrow(DataClientException.class); + void ingestFromStreamAsync_CaughtDataClientException_IngestionClientException() { + when(streamingClientMock.executeStreamingIngestAsync(any(String.class), any(String.class), any(InputStream.class), + isNull(), any(String.class), isNull(), any(boolean.class))).thenReturn(Mono.error(new DataClientException("DataClientException"))); String data = "Name, Age, Weight, Height"; InputStream inputStream = new ByteArrayInputStream(StandardCharsets.UTF_8.encode(data).array()); StreamSourceInfo streamSourceInfo = new StreamSourceInfo(inputStream); - assertThrows(IngestionClientException.class, - () -> streamingIngestClient.ingestFromStream(streamSourceInfo, ingestionProperties), - "Expected IllegalArgumentException to be thrown, but it didn't"); + StepVerifier.create(streamingIngestClient.ingestFromStreamAsync(streamSourceInfo, ingestionProperties)) + .expectErrorMatches(e -> e instanceof IngestionClientException + && "DataClientException".equals(e.getMessage())) + .verify(); } @Test - void ingestFromStream_CaughtDataServiceException_IngestionServiceException() throws Exception { - when(streamingClientMock.executeStreamingIngest(any(String.class), any(String.class), any(InputStream.class), - isNull(), any(String.class), isNull(), any(boolean.class))).thenThrow(DataServiceException.class); + void ingestFromStream_CaughtDataServiceException_IngestionServiceException() { + when(streamingClientMock.executeStreamingIngestAsync(any(String.class), any(String.class), any(InputStream.class), + isNull(), any(String.class), isNull(), any(boolean.class))) + .thenReturn(Mono.error(new DataServiceException("ingestFromStream", "DataServiceException", true))); String data = "Name, Age, Weight, Height"; InputStream inputStream = new ByteArrayInputStream(StandardCharsets.UTF_8.encode(data).array()); StreamSourceInfo streamSourceInfo = new StreamSourceInfo(inputStream); - assertThrows(IngestionServiceException.class, - () -> streamingIngestClient.ingestFromStream(streamSourceInfo, ingestionProperties), - "Expected IllegalArgumentException to be thrown, but it didn't"); + StepVerifier.create(streamingIngestClient.ingestFromStreamAsync(streamSourceInfo, ingestionProperties)) + .expectErrorMatches(e -> e instanceof IngestionServiceException + && "DataServiceException".equals(e.getMessage())) + .verify(); } @Test void ingestFromFile_Csv() throws Exception { String path = resourcesDirectory + "testdata.csv"; FileSourceInfo fileSourceInfo = new FileSourceInfo(path); - OperationStatus status = streamingIngestClient.ingestFromFile(fileSourceInfo, ingestionProperties).getIngestionStatusCollection().get(0).status; + OperationStatus status = streamingIngestClient.ingestFromFile(fileSourceInfo, ingestionProperties).getIngestionStatusCollectionAsync().block() + .get(0).status; assertEquals(OperationStatus.Succeeded, status); - verify(streamingClientMock, atLeastOnce()).executeStreamingIngest(any(String.class), any(String.class), any(InputStream.class), + verify(streamingClientMock, atLeastOnce()).executeStreamingIngestAsync(any(String.class), any(String.class), any(InputStream.class), isNull(), any(String.class), isNull(), any(boolean.class)); } @@ -310,9 +333,10 @@ void ingestFromFile_Json() throws Exception { ingestionProperties.setDataFormat(IngestionProperties.DataFormat.JSON); ingestionProperties.setIngestionMapping("JsonMapping", IngestionMapping.IngestionMappingKind.JSON); - OperationStatus status = streamingIngestClient.ingestFromFile(fileSourceInfo, ingestionProperties).getIngestionStatusCollection().get(0).status; + OperationStatus status = streamingIngestClient.ingestFromFile(fileSourceInfo, ingestionProperties).getIngestionStatusCollectionAsync().block() + .get(0).status; assertEquals(OperationStatus.Succeeded, status); - verify(streamingClientMock, atLeastOnce()).executeStreamingIngest(any(String.class), any(String.class), argumentCaptor.capture(), + verify(streamingClientMock, atLeastOnce()).executeStreamingIngestAsync(any(String.class), any(String.class), argumentCaptor.capture(), isNull(), any(String.class), any(String.class), any(boolean.class)); verifyCompressedStreamContent(argumentCaptor.getValue(), contents); @@ -324,9 +348,10 @@ void ingestFromFile_CompressedJson() throws Exception { FileSourceInfo fileSourceInfo = new FileSourceInfo(path); ingestionProperties.setDataFormat(IngestionProperties.DataFormat.JSON); ingestionProperties.setIngestionMapping("JsonMapping", IngestionMapping.IngestionMappingKind.JSON); - OperationStatus status = streamingIngestClient.ingestFromFile(fileSourceInfo, ingestionProperties).getIngestionStatusCollection().get(0).status; + OperationStatus status = streamingIngestClient.ingestFromFile(fileSourceInfo, ingestionProperties).getIngestionStatusCollectionAsync().block() + .get(0).status; assertEquals(OperationStatus.Succeeded, status); - verify(streamingClientMock, atLeastOnce()).executeStreamingIngest(any(String.class), any(String.class), argumentCaptor.capture(), + verify(streamingClientMock, atLeastOnce()).executeStreamingIngestAsync(any(String.class), any(String.class), argumentCaptor.capture(), isNull(), any(String.class), any(String.class), any(boolean.class)); verifyCompressedStreamContent(argumentCaptor.getValue(), jsonDataUncompressed); @@ -383,20 +408,19 @@ void ingestFromFile_JsonNoMappingReference_IngestionSuccess() ingestionProperties.setDataFormat(IngestionProperties.DataFormat.JSON); ingestionProperties.setIngestionMapping("JsonMapping", IngestionMapping.IngestionMappingKind.JSON); IngestionResult ingestionResult = streamingIngestClient.ingestFromFile(fileSourceInfo, ingestionProperties); - assertEquals("Succeeded", ingestionResult.getIngestionStatusCollection().get(0).status.name()); - assertEquals(1, ingestionResult.getIngestionStatusesLength()); + assertEquals("Succeeded", ingestionResult.getIngestionStatusCollectionAsync().block().get(0).status.name()); } @Test - void ingestFromFile_JsonWrongMappingKind_IngestionClientException() { + void ingestFromFileAsync_JsonWrongMappingKind_IngestionClientException() { String path = resourcesDirectory + "testdata.json"; FileSourceInfo fileSourceInfo = new FileSourceInfo(path); ingestionProperties.setDataFormat(IngestionProperties.DataFormat.JSON); ingestionProperties.setIngestionMapping("CsvMapping", IngestionMapping.IngestionMappingKind.CSV); - IngestionClientException ingestionClientException = assertThrows(IngestionClientException.class, - () -> streamingIngestClient.ingestFromFile(fileSourceInfo, ingestionProperties), - "Expected IngestionClientException to be thrown, but it didn't"); - assertTrue(ingestionClientException.getMessage().contains("Wrong ingestion mapping for format 'json'; mapping kind should be 'Json', but was 'Csv'.")); + StepVerifier.create(streamingIngestClient.ingestFromFileAsync(fileSourceInfo, ingestionProperties)) + .expectErrorMatches(e -> e instanceof IngestionClientException + && e.getMessage().contains("Wrong ingestion mapping for format 'json'; mapping kind should be 'Json', but was 'Csv'.")) + .verify(); } @Test @@ -404,23 +428,22 @@ void ingestFromFile_JsonNoMappingKind_IngestionSuccess() throws IngestionClientE String path = resourcesDirectory + "testdata.json"; FileSourceInfo fileSourceInfo = new FileSourceInfo(path); IngestionResult ingestionResult = streamingIngestClient.ingestFromFile(fileSourceInfo, ingestionProperties); - assertEquals("Succeeded", ingestionResult.getIngestionStatusCollection().get(0).status.name()); - assertEquals(1, ingestionResult.getIngestionStatusesLength()); + assertEquals("Succeeded", ingestionResult.getIngestionStatusCollectionAsync().block().get(0).status.name()); } @Test - void ingestFromFile_EmptyFile_IngestionClientException() { + void ingestFromFileAsync_EmptyFile_IngestionClientException() { String path = resourcesDirectory + "empty.csv"; FileSourceInfo fileSourceInfo = new FileSourceInfo(path); - IngestionClientException ingestionClientException = assertThrows(IngestionClientException.class, - () -> streamingIngestClient.ingestFromFile(fileSourceInfo, ingestionProperties), - "Expected IngestionClientException to be thrown, but it didn't"); - assertTrue(ingestionClientException.getMessage().contains("Empty file:")); + StepVerifier.create(streamingIngestClient.ingestFromFileAsync(fileSourceInfo, ingestionProperties)) + .expectErrorMatches(e -> e instanceof IngestionClientException && e.getMessage().contains("Empty file:")) + .verify(); } @Test void ingestFromBlob() throws Exception { - BlobClient cloudBlockBlob = mock(BlobClient.class); + BlobAsyncClient cloudBlockBlob = mock(BlobAsyncClient.class); + // BlobClient cloudBlockBlob = mock(BlobAsyncClient.class); String blobPath = "https://kustotest.blob.core.windows.net/container/blob.csv"; BlobSourceInfo blobSourceInfo = new BlobSourceInfo(blobPath); BlobProperties blobProperties = mock(BlobProperties.class); @@ -429,57 +452,58 @@ void ingestFromBlob() throws Exception { BlobInputStream blobInputStream = mock(BlobInputStream.class); when(blobInputStream.read(any(byte[].class))).thenReturn(10).thenReturn(-1); - when(cloudBlockBlob.getProperties()).thenReturn(blobProperties); - when(cloudBlockBlob.openInputStream()).thenReturn(blobInputStream); + // when(cloudBlockBlob.getProperties()).thenReturn(Mono.just(blobProperties)); + // when(cloudBlockBlob.strea()).thenReturn(blobInputStream); - OperationStatus status = streamingIngestClient.ingestFromBlob(blobSourceInfo, ingestionProperties, null).getIngestionStatusCollection() - .get(0).status; + OperationStatus status = streamingIngestClient.ingestFromBlobAsync(blobSourceInfo, ingestionProperties, null).block() + .getIngestionStatusCollectionAsync() + .block().get(0).status; assertEquals(OperationStatus.Succeeded, status); - verify(streamingClientMock, atLeastOnce()).executeStreamingIngest(any(String.class), any(String.class), any(InputStream.class), - isNull(), any(String.class), isNull(), any(boolean.class)); + verify(streamingClientMock, atLeastOnce()).executeStreamingIngestFromBlobAsync(any(String.class), any(String.class), any(String.class), + isNull(), any(String.class), any()); } @Test - void ingestFromBlob_NullBlobSourceInfo_IllegalArgumentException() { - assertThrows(IllegalArgumentException.class, - () -> streamingIngestClient.ingestFromBlob(null, ingestionProperties), - "Expected IllegalArgumentException to be thrown, but it didn't"); + void ingestFromBlobAsync_NullBlobSourceInfo_IllegalArgumentException() { + StepVerifier.create(streamingIngestClient.ingestFromBlobAsync(null, ingestionProperties)) + .expectError(IllegalArgumentException.class) + .verify(); } @Test - void ingestFromBlob_BlobSourceInfoWithNullBlobPath_IllegalArgumentException() { + void ingestFromBlobAsync_BlobSourceInfoWithNullBlobPath_IllegalArgumentException() { BlobSourceInfo blobSourceInfo1 = new BlobSourceInfo(null); - assertThrows(IllegalArgumentException.class, - () -> streamingIngestClient.ingestFromBlob(blobSourceInfo1, ingestionProperties), - "Expected IllegalArgumentException to be thrown, but it didn't"); + StepVerifier.create(streamingIngestClient.ingestFromBlobAsync(blobSourceInfo1, ingestionProperties)) + .expectError(IllegalArgumentException.class) + .verify(); } @Test - void ingestFromBlob_BlobSourceInfoWithBlankBlobPath_IllegalArgumentException() { + void ingestFromBlobAsync_BlobSourceInfoWithBlankBlobPath_IllegalArgumentException() { BlobSourceInfo blobSourceInfo2 = new BlobSourceInfo(""); - assertThrows(IllegalArgumentException.class, - () -> streamingIngestClient.ingestFromBlob(blobSourceInfo2, ingestionProperties), - "Expected IllegalArgumentException to be thrown, but it didn't"); + StepVerifier.create(streamingIngestClient.ingestFromBlobAsync(blobSourceInfo2, ingestionProperties)) + .expectError(IllegalArgumentException.class) + .verify(); } @Test - void ingestFromBlob_NullIngestionProperties_IllegalArgumentException() { + void ingestFromBlobAsync_NullIngestionProperties_IllegalArgumentException() { String path = "blobPath"; BlobSourceInfo blobSourceInfo = new BlobSourceInfo(path); - assertThrows(IllegalArgumentException.class, - () -> streamingIngestClient.ingestFromBlob(blobSourceInfo, null), - "Expected IllegalArgumentException to be thrown, but it didn't"); + StepVerifier.create(streamingIngestClient.ingestFromBlobAsync(blobSourceInfo, null)) + .expectError(IllegalArgumentException.class) + .verify(); } @ParameterizedTest @CsvSource(value = {"null,table", "'',table", "database,null", "database,''"}, nullValues = {"null"}) - void ingestFromBlob_IngestionPropertiesWithIllegalDatabaseOrTableNames_IllegalArgumentException(String databaseName, String tableName) { + void ingestFromBlobAsync_IngestionPropertiesWithIllegalDatabaseOrTableNames_IllegalArgumentException(String databaseName, String tableName) { String path = "blobPath"; BlobSourceInfo blobSourceInfo = new BlobSourceInfo(path); ingestionProperties = new IngestionProperties(databaseName, tableName); - assertThrows(IllegalArgumentException.class, - () -> streamingIngestClient.ingestFromBlob(blobSourceInfo, ingestionProperties), - "Expected IllegalArgumentException to be thrown, but it didn't"); + StepVerifier.create(streamingIngestClient.ingestFromBlobAsync(blobSourceInfo, ingestionProperties)) + .expectError(IllegalArgumentException.class) + .verify(); } @Test @@ -498,10 +522,10 @@ void ingestFromResultSet() throws Exception { ArgumentCaptor argumentCaptor = ArgumentCaptor.forClass(InputStream.class); ResultSetSourceInfo resultSetSourceInfo = new ResultSetSourceInfo(resultSet); - OperationStatus status = streamingIngestClient.ingestFromResultSet(resultSetSourceInfo, ingestionProperties).getIngestionStatusCollection() - .get(0).status; + OperationStatus status = streamingIngestClient.ingestFromResultSet(resultSetSourceInfo, ingestionProperties).getIngestionStatusCollectionAsync() + .block().get(0).status; assertEquals(OperationStatus.Succeeded, status); - verify(streamingClientMock, atLeastOnce()).executeStreamingIngest(any(String.class), any(String.class), argumentCaptor.capture(), + verify(streamingClientMock, atLeastOnce()).executeStreamingIngestAsync(any(String.class), any(String.class), argumentCaptor.capture(), isNull(), any(String.class), isNull(), any(boolean.class)); InputStream stream = argumentCaptor.getValue(); @@ -509,30 +533,30 @@ void ingestFromResultSet() throws Exception { } @Test - void ingestFromResultSet_NullResultSetSourceInfo_IllegalArgumentException() { - assertThrows(IllegalArgumentException.class, - () -> streamingIngestClient.ingestFromResultSet(null, ingestionProperties), - "Expected IllegalArgumentException to be thrown, but it didn't"); + void ingestFromResultSetAsync_NullResultSetSourceInfo_IllegalArgumentException() { + StepVerifier.create(streamingIngestClient.ingestFromResultSetAsync(null, ingestionProperties)) + .expectError(IllegalArgumentException.class) + .verify(); } @Test - void ingestFromResultSet_NullIngestionProperties_IllegalArgumentException() { + void ingestFromResultSetAsync_NullIngestionProperties_IllegalArgumentException() { ResultSet resultSet = mock(ResultSet.class); ResultSetSourceInfo resultSetSourceInfo = new ResultSetSourceInfo(resultSet); - assertThrows(IllegalArgumentException.class, - () -> streamingIngestClient.ingestFromResultSet(resultSetSourceInfo, null), - "Expected IllegalArgumentException to be thrown, but it didn't"); + StepVerifier.create(streamingIngestClient.ingestFromResultSetAsync(resultSetSourceInfo, null)) + .expectError(IllegalArgumentException.class) + .verify(); } @ParameterizedTest @CsvSource(value = {"null,table", "'',table", "database,null", "database,''"}, nullValues = {"null"}) - void ingestFromResultSet_IngestionPropertiesWithIllegalDatabaseOrTableNames_IllegalArgumentException(String databaseName, String tableName) { + void ingestFromResultSetAsync_IngestionPropertiesWithIllegalDatabaseOrTableNames_IllegalArgumentException(String databaseName, String tableName) { ResultSet resultSet = mock(ResultSet.class); ResultSetSourceInfo resultSetSourceInfo = new ResultSetSourceInfo(resultSet); ingestionProperties = new IngestionProperties(databaseName, tableName); - assertThrows(IllegalArgumentException.class, - () -> streamingIngestClient.ingestFromResultSet(resultSetSourceInfo, ingestionProperties), - "Expected IllegalArgumentException to be thrown, but it didn't"); + StepVerifier.create(streamingIngestClient.ingestFromResultSetAsync(resultSetSourceInfo, ingestionProperties)) + .expectError(IllegalArgumentException.class) + .verify(); } @Test @@ -544,10 +568,10 @@ void ingestFromResultSet_EmptyResultSet_IngestionClientException() throws Except when(resultSetMetaData.getColumnCount()).thenReturn(0); ResultSetSourceInfo resultSetSourceInfo = new ResultSetSourceInfo(resultSet); - IngestionClientException ingestionClientException = assertThrows(IngestionClientException.class, - () -> streamingIngestClient.ingestFromResultSet(resultSetSourceInfo, ingestionProperties), - "Expected IngestionClientException to be thrown, but it didn't"); - assertTrue(ingestionClientException.getMessage().contains("Empty ResultSet.")); + StepVerifier.create(streamingIngestClient.ingestFromResultSetAsync(resultSetSourceInfo, ingestionProperties)) + .expectErrorMatches(e -> e instanceof IngestionClientException + && e.getMessage().contains("Empty ResultSet.")) + .verify(); } private static Stream provideStringsForAutoCorrectEndpointTruePass() { diff --git a/pom.xml b/pom.xml index 66e74078c..b93330765 100644 --- a/pom.xml +++ b/pom.xml @@ -45,6 +45,7 @@ 1.11.0 1.17.0 4.5.14 + 3.6.11 2.16.0 2.9.1 1.7.1 diff --git a/samples/README.md b/samples/README.md index 6fe4a3658..2df633847 100644 --- a/samples/README.md +++ b/samples/README.md @@ -247,12 +247,12 @@ FileSourceInfo fileSourceInfo = new FileSourceInfo(path, new File(path).length() From stream: ```java -OperationStatus status = streamingIngestClient.ingestFromStream(streamSourceInfo, ingestionProperties).getIngestionStatusCollection().get(0).status; +OperationStatus status = streamingIngestClient.ingestFromStream(streamSourceInfo, ingestionProperties).getIngestionStatusCollection().get(0).status; //TODO: this is async now? should we have a sync equivalent? ``` From File: ```java -OperationStatus status = streamingIngestClient.ingestFromFile(fileSourceInfo, ingestionProperties).getIngestionStatusCollection().get(0).status; +OperationStatus status = streamingIngestClient.ingestFromFile(fileSourceInfo, ingestionProperties).getIngestionStatusCollection().get(0).status; //TODO: this is async now? should we have a sync equivalent? ``` ### How to run this sample @@ -330,12 +330,12 @@ IngestionResult ingestionResult = client.ingestFromFile(fileSourceInfo, ingestio 5. Retrieve ingestion status and wait for result ```java -List statuses = ingestionResult.getIngestionStatusCollection(); +List statuses = ingestionResult.getIngestionStatusCollection(); //TODO: this is async now? should we have a sync equivalent? while (statuses.get(0).status == OperationStatus.Pending && timeoutInSec > 0) { Thread.sleep(1000); timeoutInSec -= 1; - statuses = ingestionResult.getIngestionStatusCollection(); + statuses = ingestionResult.getIngestionStatusCollection(); //TODO: this is async now? should we have a sync equivalent? } ``` diff --git a/samples/src/main/java/FileIngestion.java b/samples/src/main/java/FileIngestion.java index a836b7d92..5840a5987 100644 --- a/samples/src/main/java/FileIngestion.java +++ b/samples/src/main/java/FileIngestion.java @@ -1,6 +1,8 @@ // Copyright (c) Microsoft Corporation. // Licensed under the MIT License. +import com.azure.data.tables.TableAsyncClient; +import com.azure.data.tables.models.TableEntity; import com.microsoft.azure.kusto.data.auth.ConnectionStringBuilder; import com.microsoft.azure.kusto.ingest.ColumnMapping; import com.microsoft.azure.kusto.ingest.IngestClient; @@ -10,6 +12,7 @@ import com.microsoft.azure.kusto.ingest.result.IngestionResult; import com.microsoft.azure.kusto.ingest.source.FileSourceInfo; import com.microsoft.azure.kusto.ingest.source.StreamSourceInfo; +import com.microsoft.azure.kusto.ingest.utils.TableWithSas; import java.io.ByteArrayInputStream; import java.io.ByteArrayOutputStream; @@ -17,10 +20,10 @@ public class FileIngestion { public static void main(String[] args) { try { - // TableClient tableClient = - // TableWithSas.TableClientFromUrl("https://5s8kstrldruthruth01.blob.core.windows.net/20230313-ingestdata-e5c334ee145d4b4-0?sv=2018-03-28&sr=c&sig=QshIuU9ZZ1jvcgcPMnHcr0EvCwO9sxZbvAUaAtI%3D&st=2023-03-13T13%3A16%3A57Z&se=2023-03-17T14%3A16%3A57Z&sp=rw", + // TableAsyncClient tableAsyncClient = + // TableWithSas.createTableClientFromUrl("https://5s8kstrldruthruth01.blob.core.windows.net/20230313-ingestdata-e5c334ee145d4b4-0?sv=2018-03-28&sr=c&sig=QshIuU9ZZ1jvcgcPMnHcr0EvCwO9sxZbvAUaAtI%3D&st=2023-03-13T13%3A16%3A57Z&se=2023-03-17T14%3A16%3A57Z&sp=rw", // null); - // tableClient.createEntity(new TableEntity("123", "123")); + // tableAsyncClient.createEntity(new TableEntity("123", "123")).block(); ConnectionStringBuilder csb = ConnectionStringBuilder.createWithUserPrompt("https://ruthruth.eastus.kusto.windows.net"); try (IngestClient client = IngestClientFactory.createClient(csb)) { diff --git a/samples/src/main/java/FileIngestionCompletableFuture.java b/samples/src/main/java/FileIngestionCompletableFuture.java index f66ad6149..1e4c339d9 100644 --- a/samples/src/main/java/FileIngestionCompletableFuture.java +++ b/samples/src/main/java/FileIngestionCompletableFuture.java @@ -27,7 +27,7 @@ public static void main(String[] args) { System.getProperty("appKey"), System.getProperty("appTenant")); - CompletableFuture cf; + CompletableFuture cf; // TODO: adjust this to use the async API instead of using CompletableFuture or not? try (IngestClient client = IngestClientFactory.createClient(csb)) { // Creating the ingestion properties: IngestionProperties ingestionProperties = new IngestionProperties( diff --git a/samples/src/main/java/Jmeter/jmeterStressTest.jmx b/samples/src/main/java/Jmeter/jmeterStressTest.jmx index 788a6e688..aa986b2c4 100644 --- a/samples/src/main/java/Jmeter/jmeterStressTest.jmx +++ b/samples/src/main/java/Jmeter/jmeterStressTest.jmx @@ -407,7 +407,7 @@ try { } finally { client.close(); } - IngestionStatus ingestionStatus = ingestionResult.getIngestionStatusCollection().get(0); + IngestionStatus ingestionStatus = ingestionResult.getIngestionStatusCollection().get(0); //TODO: block here? if (ingestionStatus.status != OperationStatus.Queued){ throw new Exception("Failed upload" + ingestionStatus.errorCodeString); } diff --git a/samples/src/main/java/StreamingIngest.java b/samples/src/main/java/StreamingIngest.java index 0048624a1..9dec31526 100644 --- a/samples/src/main/java/StreamingIngest.java +++ b/samples/src/main/java/StreamingIngest.java @@ -14,7 +14,6 @@ import com.microsoft.azure.kusto.ingest.source.StreamSourceInfo; import java.io.ByteArrayInputStream; -import java.io.File; import java.io.FileInputStream; import java.io.FileNotFoundException; import java.io.InputStream; @@ -58,7 +57,8 @@ static void ingestFromStream() throws IngestionClientException, IngestionService InputStream inputStream = new ByteArrayInputStream(StandardCharsets.UTF_8.encode(data).array()); StreamSourceInfo streamSourceInfo = new StreamSourceInfo(inputStream); ingestionProperties.setDataFormat(IngestionProperties.DataFormat.CSV); - OperationStatus status = streamingIngestClient.ingestFromStream(streamSourceInfo, ingestionProperties).getIngestionStatusCollection().get(0).status; + OperationStatus status = streamingIngestClient.ingestFromStream(streamSourceInfo, ingestionProperties).getIngestionStatusCollectionAsync().block() + .get(0).status; System.out.println(status.toString()); String resourcesDirectory = System.getProperty("user.dir") + "/samples/src/main/resources/"; @@ -71,7 +71,7 @@ static void ingestFromStream() throws IngestionClientException, IngestionService * stream content is already compressed, we should set this property true to avoid double compression. */ streamSourceInfo.setCompressionType(CompressionType.gz); - status = streamingIngestClient.ingestFromStream(streamSourceInfo, ingestionProperties).getIngestionStatusCollection().get(0).status; + status = streamingIngestClient.ingestFromStream(streamSourceInfo, ingestionProperties).getIngestionStatusCollectionAsync().block().get(0).status; System.out.println(status.toString()); // Open JSON File Stream and Ingest @@ -79,7 +79,7 @@ static void ingestFromStream() throws IngestionClientException, IngestionService ingestionProperties.setIngestionMapping(mapping, IngestionMapping.IngestionMappingKind.JSON); fileInputStream = new FileInputStream(resourcesDirectory + "dataset.json"); streamSourceInfo.setStream(fileInputStream); - status = streamingIngestClient.ingestFromStream(streamSourceInfo, ingestionProperties).getIngestionStatusCollection().get(0).status; + status = streamingIngestClient.ingestFromStream(streamSourceInfo, ingestionProperties).getIngestionStatusCollectionAsync().block().get(0).status; System.out.println(status.toString()); } @@ -90,7 +90,8 @@ static void ingestFromFile() throws IngestionClientException, IngestionServiceEx String path = resourcesDirectory + "dataset.csv"; FileSourceInfo fileSourceInfo = new FileSourceInfo(path); ingestionProperties.setDataFormat(IngestionProperties.DataFormat.CSV); - OperationStatus status = streamingIngestClient.ingestFromFile(fileSourceInfo, ingestionProperties).getIngestionStatusCollection().get(0).status; + OperationStatus status = streamingIngestClient.ingestFromFile(fileSourceInfo, ingestionProperties).getIngestionStatusCollectionAsync().block() + .get(0).status; System.out.println(status.toString()); // Ingest compressed JSON file @@ -98,7 +99,7 @@ static void ingestFromFile() throws IngestionClientException, IngestionServiceEx fileSourceInfo = new FileSourceInfo(path); ingestionProperties.setDataFormat(IngestionProperties.DataFormat.JSON); ingestionProperties.setIngestionMapping(mapping, IngestionMapping.IngestionMappingKind.JSON); - status = streamingIngestClient.ingestFromFile(fileSourceInfo, ingestionProperties).getIngestionStatusCollection().get(0).status; + status = streamingIngestClient.ingestFromFile(fileSourceInfo, ingestionProperties).getIngestionStatusCollectionAsync().block().get(0).status; System.out.println(status.toString()); } } diff --git a/samples/src/main/java/TableStatus.java b/samples/src/main/java/TableStatus.java index f64dabd5e..201b47853 100644 --- a/samples/src/main/java/TableStatus.java +++ b/samples/src/main/java/TableStatus.java @@ -38,13 +38,13 @@ public static void main(String[] args) { FileSourceInfo fileSourceInfo = new FileSourceInfo(System.getProperty("filePath")); ingestionResult = client.ingestFromFile(fileSourceInfo, ingestionProperties); } - List statuses = ingestionResult.getIngestionStatusCollection(); + List statuses = ingestionResult.getIngestionStatusCollectionAsync().block(); // TODO: how to handle this // step 3: poll on the result. while (statuses.get(0).status == OperationStatus.Pending && timeoutInSec > 0) { Thread.sleep(1000); timeoutInSec -= 1; - statuses = ingestionResult.getIngestionStatusCollection(); + statuses = ingestionResult.getIngestionStatusCollectionAsync().block(); } ObjectMapper objectMapper = Utils.getObjectMapper();