From 36f269c375336b047251ad2e64a3cac00c4a0c81 Mon Sep 17 00:00:00 2001 From: Manasa Date: Wed, 18 Dec 2019 11:39:49 -0800 Subject: [PATCH 1/9] SAMZA-2421: Add SystemProducer for Azure Blob Storage --- build.gradle | 2 + gradle/dependency-versions.gradle | 2 +- .../azureblob/AzureBlobBasicMetrics.java | 98 +++ .../system/azureblob/AzureBlobConfig.java | 189 ++++++ .../azureblob/AzureBlobSystemAdmin.java | 45 ++ .../azureblob/AzureBlobSystemFactory.java | 56 ++ .../azureblob/avro/AzureBlobAvroWriter.java | 358 +++++++++++ .../avro/AzureBlobAvroWriterFactory.java | 43 ++ .../azureblob/avro/AzureBlobOutputStream.java | 314 +++++++++ .../azureblob/compression/Compression.java | 41 ++ .../compression/CompressionFactory.java | 36 ++ .../compression/CompressionType.java | 31 + .../compression/GzipCompression.java | 75 +++ .../compression/NoneCompression.java | 37 ++ .../producer/AzureBlobSystemProducer.java | 505 +++++++++++++++ .../AzureBlobSystemProducerMetrics.java | 134 ++++ .../azureblob/producer/AzureBlobWriter.java | 52 ++ .../producer/AzureBlobWriterFactory.java | 44 ++ .../producer/AzureBlobWriterMetrics.java | 81 +++ .../avro/TestAzureBlobAvroWriter.java | 452 +++++++++++++ .../avro/TestAzureBlobOutputStream.java | 347 ++++++++++ .../compression/TestGzipCompression.java | 76 +++ .../producer/TestAzureBlobSystemProducer.java | 600 ++++++++++++++++++ 23 files changed, 3617 insertions(+), 1 deletion(-) create mode 100644 samza-azure/src/main/java/org/apache/samza/system/azureblob/AzureBlobBasicMetrics.java create mode 100644 samza-azure/src/main/java/org/apache/samza/system/azureblob/AzureBlobConfig.java create mode 100644 samza-azure/src/main/java/org/apache/samza/system/azureblob/AzureBlobSystemAdmin.java create mode 100644 samza-azure/src/main/java/org/apache/samza/system/azureblob/AzureBlobSystemFactory.java create mode 100644 samza-azure/src/main/java/org/apache/samza/system/azureblob/avro/AzureBlobAvroWriter.java create mode 100644 samza-azure/src/main/java/org/apache/samza/system/azureblob/avro/AzureBlobAvroWriterFactory.java create mode 100644 samza-azure/src/main/java/org/apache/samza/system/azureblob/avro/AzureBlobOutputStream.java create mode 100644 samza-azure/src/main/java/org/apache/samza/system/azureblob/compression/Compression.java create mode 100644 samza-azure/src/main/java/org/apache/samza/system/azureblob/compression/CompressionFactory.java create mode 100644 samza-azure/src/main/java/org/apache/samza/system/azureblob/compression/CompressionType.java create mode 100644 samza-azure/src/main/java/org/apache/samza/system/azureblob/compression/GzipCompression.java create mode 100644 samza-azure/src/main/java/org/apache/samza/system/azureblob/compression/NoneCompression.java create mode 100644 samza-azure/src/main/java/org/apache/samza/system/azureblob/producer/AzureBlobSystemProducer.java create mode 100644 samza-azure/src/main/java/org/apache/samza/system/azureblob/producer/AzureBlobSystemProducerMetrics.java create mode 100644 samza-azure/src/main/java/org/apache/samza/system/azureblob/producer/AzureBlobWriter.java create mode 100644 samza-azure/src/main/java/org/apache/samza/system/azureblob/producer/AzureBlobWriterFactory.java create mode 100644 samza-azure/src/main/java/org/apache/samza/system/azureblob/producer/AzureBlobWriterMetrics.java create mode 100644 samza-azure/src/test/java/org/apache/samza/system/azureblob/avro/TestAzureBlobAvroWriter.java create mode 100644 samza-azure/src/test/java/org/apache/samza/system/azureblob/avro/TestAzureBlobOutputStream.java create mode 100644 samza-azure/src/test/java/org/apache/samza/system/azureblob/compression/TestGzipCompression.java create mode 100644 samza-azure/src/test/java/org/apache/samza/system/azureblob/producer/TestAzureBlobSystemProducer.java diff --git a/build.gradle b/build.gradle index c4ca3491c6..8e4ad424f7 100644 --- a/build.gradle +++ b/build.gradle @@ -211,10 +211,12 @@ project(":samza-azure_$scalaSuffix") { apply plugin: 'checkstyle' dependencies { + compile "com.azure:azure-storage-blob:12.0.1" compile "com.microsoft.azure:azure-storage:5.3.1" compile "com.microsoft.azure:azure-eventhubs:1.0.1" compile "com.fasterxml.jackson.core:jackson-core:2.8.8" compile "io.dropwizard.metrics:metrics-core:3.1.2" + compile "org.apache.avro:avro:$avroVersion" compile project(':samza-api') compile project(":samza-core_$scalaSuffix") compile "org.slf4j:slf4j-api:$slf4jVersion" diff --git a/gradle/dependency-versions.gradle b/gradle/dependency-versions.gradle index f70e879d37..56cfb76233 100644 --- a/gradle/dependency-versions.gradle +++ b/gradle/dependency-versions.gradle @@ -18,7 +18,7 @@ */ ext { apacheCommonsCollections4Version = "4.0" - avroVersion = "1.7.1" + avroVersion = "1.7.7" calciteVersion = "1.19.0" commonsCliVersion = "1.2" commonsCodecVersion = "1.9" diff --git a/samza-azure/src/main/java/org/apache/samza/system/azureblob/AzureBlobBasicMetrics.java b/samza-azure/src/main/java/org/apache/samza/system/azureblob/AzureBlobBasicMetrics.java new file mode 100644 index 0000000000..388135ee04 --- /dev/null +++ b/samza-azure/src/main/java/org/apache/samza/system/azureblob/AzureBlobBasicMetrics.java @@ -0,0 +1,98 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, + * software distributed under the License is distributed on an + * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY + * KIND, either express or implied. See the License for the + * specific language governing permissions and limitations + * under the License. + */ + +package org.apache.samza.system.azureblob; + +import org.apache.samza.metrics.Counter; +import org.apache.samza.metrics.MetricsRegistry; + +/** + * This class holds all the metrics to be measured (like write, write byte, error) + * for a single group (like aggregate, system or source). + */ +public class AzureBlobBasicMetrics { + public static final String EVENT_WRITE_RATE = "eventWriteRate"; + public static final String EVENT_PRODUCE_ERROR = "eventProduceError"; + public static final String EVENT_WRITE_BYTE_RATE = "eventWriteByteRate"; + public static final String EVENT_COMPRESS_BYTE_RATE = "eventCompressByteRate"; + public static final String AZURE_BLOCK_UPLOAD_RATE = "azureBlockUploadRate"; + public static final String AZURE_BLOB_COMMIT_RATE = "azureBlobCommitRate"; + + private final Counter writeMetrics; + private final Counter writeByteMetrics; + private final Counter errorMetrics; + private final Counter compressByteMetrics; + private final Counter azureUploadMetrics; + private final Counter azureCommitMetrics; + + public AzureBlobBasicMetrics(String group, MetricsRegistry metricsRegistry) { + writeMetrics = metricsRegistry.newCounter(group, EVENT_WRITE_RATE); + errorMetrics = metricsRegistry.newCounter(group, EVENT_PRODUCE_ERROR); + writeByteMetrics = metricsRegistry.newCounter(group, EVENT_WRITE_BYTE_RATE); + compressByteMetrics = metricsRegistry.newCounter(group, EVENT_COMPRESS_BYTE_RATE); + azureUploadMetrics = metricsRegistry.newCounter(group, AZURE_BLOCK_UPLOAD_RATE); + azureCommitMetrics = metricsRegistry.newCounter(group, AZURE_BLOB_COMMIT_RATE); + } + + /** + * Increments the write metrics counter by 1. + */ + public void updateWriteMetrics() { + writeMetrics.inc(); + } + + /** + * Increments the write byte metrics counter by the number of bytes written. + * @param dataLength number of bytes written. + */ + public void updateWriteByteMetrics(long dataLength) { + writeByteMetrics.inc(dataLength); + } + + /** + * Increments the compress byte metrics counter by the number of compressed bytes written. + * @param dataLength number of bytes written. + */ + public void updateCompressByteMetrics(long dataLength) { + compressByteMetrics.inc(dataLength); + } + + /** + * Increments the error metrics counter by 1. + */ + public void updateErrorMetrics() { + errorMetrics.inc(); + } + + + /** + * Increments the azure block upload metrics counter by 1. + */ + public void updateAzureUploadMetrics() { + azureUploadMetrics.inc(); + } + + + /** + * Increments the azure blob commit metrics counter by 1. + */ + public void updateAzureCommitMetrics() { + azureCommitMetrics.inc(); + } +} diff --git a/samza-azure/src/main/java/org/apache/samza/system/azureblob/AzureBlobConfig.java b/samza-azure/src/main/java/org/apache/samza/system/azureblob/AzureBlobConfig.java new file mode 100644 index 0000000000..facd83ac7f --- /dev/null +++ b/samza-azure/src/main/java/org/apache/samza/system/azureblob/AzureBlobConfig.java @@ -0,0 +1,189 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, + * software distributed under the License is distributed on an + * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY + * KIND, either express or implied. See the License for the + * specific language governing permissions and limitations + * under the License. + */ + +package org.apache.samza.system.azureblob; + +import org.apache.samza.system.azureblob.compression.CompressionType; +import java.time.Duration; +import org.apache.samza.config.Config; +import org.apache.samza.config.ConfigException; +import org.apache.samza.config.MapConfig; + +public class AzureBlobConfig extends MapConfig { + private static final String SYSTEM_AZUREBLOB_PREFIX = "systems.%s.azureblob."; + //Server Instance Level Property + + // The duration after which an Azure request will be logged as a warning. + public static final String AZURE_BLOB_LOG_SLOW_REQUESTS_MS = "samza.azureblob.log.slowRequestMs"; + private static final long AZURE_BLOB_LOG_SLOW_REQUESTS_MS_DEFAULT = Duration.ofSeconds(30).toMillis(); + + // system Level Properties. + // fully qualified class name of the AzureBlobWriter impl for the producer system + public static final String SYSTEM_WRITER_FACTORY_CLASS_NAME = "systems.%s.producer.azureblob.writer.factory.class"; + public static final String SYSTEM_WRITER_FACTORY_CLASS_NAME_DEFAULT = "org.apache.samza.system.azureblob.avro.AzureBlobAvroWriterFactory"; + + // Azure Storage Account name under which the Azure container representing this system is. + // System name = Azure container name (https://docs.microsoft.com/en-us/rest/api/storageservices/naming-and-referencing-containers--blobs--and-metadata#container-names) + public static final String SYSTEM_AZURE_ACCOUNT_NAME = SYSTEM_AZUREBLOB_PREFIX + "account.name"; + + // Azure Storage Account key associated with the Azure Storage Account + public static final String SYSTEM_AZURE_ACCOUNT_KEY = SYSTEM_AZUREBLOB_PREFIX + "account.key"; + + // Whether to use proxy while connecting to Azure Storage + public static final String SYSTEM_AZURE_USE_PROXY = SYSTEM_AZUREBLOB_PREFIX + "proxy.use"; + public static final boolean SYSTEM_AZURE_USE_PROXY_DEFAULT = false; + + // name of the host to be used as proxy + public static final String SYSTEM_AZURE_PROXY_HOSTNAME = SYSTEM_AZUREBLOB_PREFIX + "proxy.hostname"; + + // port in the proxy host to be used + public static final String SYSTEM_AZURE_PROXY_PORT = SYSTEM_AZUREBLOB_PREFIX + "proxy.port"; + + // type of compression to be used before uploading blocks : “none” or “gzip” + public static final String SYSTEM_COMPRESSION_TYPE = SYSTEM_AZUREBLOB_PREFIX + "compression.type"; + private static final CompressionType SYSTEM_COMPRESSION_TYPE_DEFAULT = CompressionType.NONE; + + // maximum size of uncompressed block in bytes + public static final String SYSTEM_MAX_FLUSH_THRESHOLD_SIZE = SYSTEM_AZUREBLOB_PREFIX + "maxFlushThresholdSize"; + private static final int SYSTEM_MAX_FLUSH_THRESHOLD_SIZE_DEFAULT = 10485760; + + // maximum size of uncompressed blob in bytes + public static final String SYSTEM_MAX_BLOB_SIZE = SYSTEM_AZUREBLOB_PREFIX + "maxBlobSize"; + private static final long SYSTEM_MAX_BLOB_SIZE_DEFAULT = Long.MAX_VALUE; // unlimited + + // maximum number of messages in a blob + public static final String SYSTEM_MAX_MESSAGES_PER_BLOB = SYSTEM_AZUREBLOB_PREFIX + "maxMessagesPerBlob"; + private static final long SYSTEM_MAX_MESSAGES_PER_BLOB_DEFAULT = Long.MAX_VALUE; // unlimited + + // number of threads to asynchronously upload blocks + public static final String SYSTEM_THREAD_POOL_COUNT = SYSTEM_AZUREBLOB_PREFIX + "threadPoolCount"; + private static final int SYSTEM_THREAD_POOL_COUNT_DEFAULT = 2; + + // size of the queue to hold blocks ready to be uploaded by asynchronous threads. + // If all threads are busy uploading then blocks are queued and if queue is full then main thread will start uploading + // which will block processing of incoming messages + // Default - Thread Pool Count * 2 + public static final String SYSTEM_BLOCKING_QUEUE_SIZE = SYSTEM_AZUREBLOB_PREFIX + "blockingQueueSize"; + + // timeout to finish uploading all blocks before committing a blob + public static final String SYSTEM_FLUSH_TIMEOUT_MS = SYSTEM_AZUREBLOB_PREFIX + "flushTimeoutMs"; + private static final long SYSTEM_FLUSH_TIMEOUT_MS_DEFAULT = Duration.ofMinutes(3).toMillis(); + + // timeout to finish committing all the blobs currently being written to. This does not include the flush timeout per blob + public static final String SYSTEM_CLOSE_TIMEOUT_MS = SYSTEM_AZUREBLOB_PREFIX + "closeTimeoutMs"; + private static final long SYSTEM_CLOSE_TIMEOUT_MS_DEFAULT = Duration.ofMinutes(5).toMillis(); + + // if true, a random string of 8 chars is suffixed to the blob name to prevent name collision + // when more than one Samza tasks are writing to the same SSP. + // It is advisable to set this to true + public static final String SYSTEM_SUFFIX_RANDOM_STRING_TO_BLOB_NAME = SYSTEM_AZUREBLOB_PREFIX + "suffixRandomStringToBlobName"; + private static final boolean SYSTEM_SUFFIX_RANDOM_STRING_TO_BLOB_NAME_DEFAULT = false; + + public AzureBlobConfig(Config config) { + super(config); + } + + public String getAzureAccountKey(String systemName) { + String accountKey = get(String.format(SYSTEM_AZURE_ACCOUNT_KEY, systemName)); + if (accountKey == null) { + throw new ConfigException("Azure account key is required."); + } + return accountKey; + } + + public String getAzureAccountName(String systemName) { + String accountName = get(String.format(SYSTEM_AZURE_ACCOUNT_NAME, systemName)); + if (accountName == null) { + throw new ConfigException("Azure account name is required."); + } + return accountName; + } + + public boolean getUseProxy(String systemName) { + return getBoolean(String.format(SYSTEM_AZURE_USE_PROXY, systemName), SYSTEM_AZURE_USE_PROXY_DEFAULT); + } + + public String getAzureProxyHostname(String systemName) { + String hostname = get(String.format(SYSTEM_AZURE_PROXY_HOSTNAME, systemName)); + if (hostname == null) { + throw new ConfigException("Azure proxy host name is required."); + } + return hostname; + } + + public int getAzureProxyPort(String systemName) { + return getInt(String.format(SYSTEM_AZURE_PROXY_PORT, systemName)); + } + + public CompressionType getCompressionType(String systemName) { + return CompressionType.valueOf(get(String.format(SYSTEM_COMPRESSION_TYPE, systemName), SYSTEM_COMPRESSION_TYPE_DEFAULT.name()).toUpperCase()); + } + + public String getAzureBlobWriterFactoryClassName(String systemName) { + return get(String.format(SYSTEM_WRITER_FACTORY_CLASS_NAME, systemName), SYSTEM_WRITER_FACTORY_CLASS_NAME_DEFAULT); + } + + public int getMaxFlushThresholdSize(String systemName) { + return getInt(String.format(SYSTEM_MAX_FLUSH_THRESHOLD_SIZE, systemName), SYSTEM_MAX_FLUSH_THRESHOLD_SIZE_DEFAULT); + } + + public int getAzureBlobThreadPoolCount(String systemName) { + return getInt(String.format(SYSTEM_THREAD_POOL_COUNT, systemName), SYSTEM_THREAD_POOL_COUNT_DEFAULT); + } + + public int getBlockingQueueSizeOrDefault(String systemName, int defaultQueueSize) { + return getInt(String.format(SYSTEM_BLOCKING_QUEUE_SIZE, systemName), defaultQueueSize); + } + + public long getFlushTimeoutMs(String systemName) { + long timeout = getLong(String.format(SYSTEM_FLUSH_TIMEOUT_MS, systemName), SYSTEM_FLUSH_TIMEOUT_MS_DEFAULT); + if (timeout <= 0) { + throw new ConfigException("Azure Blob flush timeout can not be <= 0"); + } + return timeout; + } + + public long getCloseTimeoutMs(String systemName) { + long timeout = getLong(String.format(SYSTEM_CLOSE_TIMEOUT_MS, systemName), SYSTEM_CLOSE_TIMEOUT_MS_DEFAULT); + if (timeout <= 0) { + throw new ConfigException("Azure Blob close timeout can not be <= 0"); + } + return timeout; + } + + public long getLogSlowRequestsMs() { + long duration = getLong(AZURE_BLOB_LOG_SLOW_REQUESTS_MS, AZURE_BLOB_LOG_SLOW_REQUESTS_MS_DEFAULT); + if (duration <= 0) { + throw new ConfigException("Azure blob duration to log slow requests can not be <=0."); + } + return duration; + } + + public boolean getSuffixRandomStringToBlobName(String systemName) { + return getBoolean(String.format(SYSTEM_SUFFIX_RANDOM_STRING_TO_BLOB_NAME, systemName), SYSTEM_SUFFIX_RANDOM_STRING_TO_BLOB_NAME_DEFAULT); + } + + public long getMaxBlobSize(String systemName) { + return getLong(String.format(SYSTEM_MAX_BLOB_SIZE, systemName), SYSTEM_MAX_BLOB_SIZE_DEFAULT); + } + + public long getMaxMessagesPerBlob(String systemName) { + return getLong(String.format(SYSTEM_MAX_MESSAGES_PER_BLOB, systemName), SYSTEM_MAX_MESSAGES_PER_BLOB_DEFAULT); + } +} diff --git a/samza-azure/src/main/java/org/apache/samza/system/azureblob/AzureBlobSystemAdmin.java b/samza-azure/src/main/java/org/apache/samza/system/azureblob/AzureBlobSystemAdmin.java new file mode 100644 index 0000000000..112123da36 --- /dev/null +++ b/samza-azure/src/main/java/org/apache/samza/system/azureblob/AzureBlobSystemAdmin.java @@ -0,0 +1,45 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, + * software distributed under the License is distributed on an + * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY + * KIND, either express or implied. See the License for the + * specific language governing permissions and limitations + * under the License. + */ + +package org.apache.samza.system.azureblob; + +import java.util.HashMap; +import java.util.Map; +import java.util.Set; +import org.apache.samza.system.SystemAdmin; +import org.apache.samza.system.SystemStreamMetadata; +import org.apache.samza.system.SystemStreamPartition; + + +/** + * {@inheritDoc} + */ +public class AzureBlobSystemAdmin implements SystemAdmin { + public Map getOffsetsAfter(Map offsets) { + return new HashMap<>(); + } + + public Map getSystemStreamMetadata(Set streamNames) { + return new HashMap<>(); + } + + public Integer offsetComparator(String offset1, String offset2) { + return null; + } +} \ No newline at end of file diff --git a/samza-azure/src/main/java/org/apache/samza/system/azureblob/AzureBlobSystemFactory.java b/samza-azure/src/main/java/org/apache/samza/system/azureblob/AzureBlobSystemFactory.java new file mode 100644 index 0000000000..512fe46181 --- /dev/null +++ b/samza-azure/src/main/java/org/apache/samza/system/azureblob/AzureBlobSystemFactory.java @@ -0,0 +1,56 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, + * software distributed under the License is distributed on an + * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY + * KIND, either express or implied. See the License for the + * specific language governing permissions and limitations + * under the License. + */ + +package org.apache.samza.system.azureblob; + +import org.apache.samza.system.azureblob.producer.AzureBlobSystemProducer; +import org.apache.samza.config.Config; +import org.apache.samza.metrics.MetricsRegistry; +import org.apache.samza.system.SystemAdmin; +import org.apache.samza.system.SystemConsumer; +import org.apache.samza.system.SystemFactory; +import org.apache.samza.system.SystemProducer; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + +/** + * Do not use this SystemProducer for Coordinator stream store/producer and KafkaCheckpointManager + * as their usage of SystemProducer is a bit inconsistent with this implementation and they also couple + * a SystemProducer with a SystemConsumer which is out of scope for this Factory. + * {@inheritDoc} + */ +public class AzureBlobSystemFactory implements SystemFactory { + private static final Logger LOG = LoggerFactory.getLogger(AzureBlobSystemFactory.class.getName()); + + @Override + public SystemConsumer getConsumer(String systemName, Config config, MetricsRegistry registry) { + throw new UnsupportedOperationException("SystemConsumer not supported for AzureBlob!"); + } + + @Override + public SystemProducer getProducer(String systemName, Config config, MetricsRegistry registry) { + AzureBlobConfig azureBlobConfig = new AzureBlobConfig(config); + return new AzureBlobSystemProducer(systemName, azureBlobConfig, registry); + } + + @Override + public SystemAdmin getAdmin(String systemName, Config config) { + return new AzureBlobSystemAdmin(); + } +} diff --git a/samza-azure/src/main/java/org/apache/samza/system/azureblob/avro/AzureBlobAvroWriter.java b/samza-azure/src/main/java/org/apache/samza/system/azureblob/avro/AzureBlobAvroWriter.java new file mode 100644 index 0000000000..f0cae9200f --- /dev/null +++ b/samza-azure/src/main/java/org/apache/samza/system/azureblob/avro/AzureBlobAvroWriter.java @@ -0,0 +1,358 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, + * software distributed under the License is distributed on an + * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY + * KIND, either express or implied. See the License for the + * specific language governing permissions and limitations + * under the License. + */ + +package org.apache.samza.system.azureblob.avro; + +import com.azure.storage.blob.BlobContainerAsyncClient; +import com.azure.storage.blob.specialized.BlockBlobAsyncClient; +import com.google.common.annotations.VisibleForTesting; +import com.google.common.base.Preconditions; +import org.apache.samza.system.azureblob.compression.Compression; +import org.apache.samza.system.azureblob.compression.GzipCompression; +import org.apache.samza.system.azureblob.producer.AzureBlobWriter; +import org.apache.samza.system.azureblob.producer.AzureBlobWriterMetrics; +import java.io.ByteArrayOutputStream; +import java.io.IOException; +import java.nio.ByteBuffer; +import java.text.SimpleDateFormat; +import java.util.ArrayList; +import java.util.List; +import java.util.Optional; +import java.util.TimeZone; +import java.util.UUID; +import java.util.concurrent.Executor; +import org.apache.avro.Schema; +import org.apache.avro.file.DataFileWriter; +import org.apache.avro.generic.GenericDatumWriter; +import org.apache.avro.generic.IndexedRecord; +import org.apache.avro.io.BinaryEncoder; +import org.apache.avro.io.DatumWriter; +import org.apache.avro.io.EncoderFactory; +import org.apache.avro.specific.SpecificDatumWriter; +import org.apache.avro.specific.SpecificRecord; +import org.apache.samza.SamzaException; +import org.apache.samza.system.OutgoingMessageEnvelope; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + +/** + * This class implements {@link org.apache.samza.system.azureblob.producer.AzureBlobWriter} + * for writing avro records to Azure Blob Storage. + * + * It uses {@link org.apache.avro.file.DataFileWriter} to convert avro records it receives to byte[]. + * This byte[] is passed on to {@link org.apache.samza.system.azureblob.avro.AzureBlobOutputStream}. + * AzureBlobOutputStream in turn uploads data to Storage as a blob. + * + * It also accepts encoded records as byte[] as long as the first OutgoingMessageEnvelope this writer receives + * is a decoded record from which to get the schema and record type (GenericRecord vs SpecificRecord). + * The subsequent encoded records are written directly to AzureBlobOutputStream without checking if they conform + * to the schema. It is the responsibility of the user to ensure this. Failing to do so may result in an + * unreadable avro blob. + * + * It expects all OutgoingMessageEnvelopes to be of the same schema. + * To handle schema evolution (sending envelopes of different schema), this writer has to be closed and a new writer + * has to be created. The first envelope of the new writer should contain a valid record to get schema from. + * If used by AzureBlobSystemProducer, this is done through systemProducer.flush(source). + * + * Once closed this object can not be used. + * This is a thread safe class. + * + * If the number of records or size of the current blob exceeds the specified limits then a new blob is created. + */ +public class AzureBlobAvroWriter implements AzureBlobWriter { + private static final Logger LOG = LoggerFactory.getLogger(AzureBlobAvroWriter.class); + private static final String PUBLISHED_FILE_NAME_DATE_FORMAT = "yyyy/MM/dd/HH/mm-ss"; + private static final String BLOB_NAME_AVRO = "%s/%s-%s.avro%s"; + private static final String BLOB_NAME_RANDOM_STRING_AVRO = "%s/%s-%s-%s.avro%s"; + private static final SimpleDateFormat UTC_FORMATTER = buildUTCFormatter(); + + // Avro's DataFileWriter has internal buffers and also adds metadata. + // Based on the current default sizes of these buffers and metadata, the data overhead is a little less than 100KB + // However, taking the overhead to be capped at 1MB to ensure enough room if the default values are increased. + static final long DATAFILEWRITER_OVERHEAD = 1000000; // 1MB + + // currentBlobWriterComponents == null only for the first blob immediately after this AzureBlobAvroWriter object has been created. + // rest of this object's lifecycle, currentBlobWriterComponents is not null. + private BlobWriterComponents currentBlobWriterComponents = null; + private final List allBlobWriterComponents = new ArrayList<>(); + private Schema schema = null; + // datumWriter == null only for the first blob immediately after this AzureBlobAvroWriter object has been created. + // It is created from the schema taken from the first OutgoingMessageEnvelope. Hence the first OME has to be a decoded avro record. + // For rest of this object's lifecycle, datumWriter is not null. + private DatumWriter datumWriter = null; + private volatile boolean isClosed = false; + + private final Executor blobThreadPool; + private final AzureBlobWriterMetrics metrics; + private final int maxBlockFlushThresholdSize; + private final long flushTimeoutMs; + private final Compression compression; + private final BlobContainerAsyncClient containerAsyncClient; + private final String blobURLPrefix; + private final long maxBlobSize; + private final long maxRecordsPerBlob; + private final boolean useRandomStringInBlobName; + private final Object currentDataFileWriterLock = new Object(); + private volatile long blobNumber = 0; + private volatile long recordsInCurrentBlob = 0; + + public AzureBlobAvroWriter(BlobContainerAsyncClient containerAsyncClient, String blobURLPrefix, + Executor blobThreadPool, AzureBlobWriterMetrics metrics, + int maxBlockFlushThresholdSize, long flushTimeoutMs, Compression compression, boolean useRandomStringInBlobName, + long maxBlobSize, long maxRecordsPerBlob) { + + this.blobThreadPool = blobThreadPool; + this.metrics = metrics; + this.maxBlockFlushThresholdSize = maxBlockFlushThresholdSize; + this.flushTimeoutMs = flushTimeoutMs; + this.compression = compression; + this.containerAsyncClient = containerAsyncClient; + this.blobURLPrefix = blobURLPrefix; + this.useRandomStringInBlobName = useRandomStringInBlobName; + this.maxBlobSize = maxBlobSize; + this.maxRecordsPerBlob = maxRecordsPerBlob; + } + + /** + * This method expects the {@link org.apache.samza.system.OutgoingMessageEnvelope} + * to contain a message which is a {@link org.apache.avro.generic.IndexedRecord} or an encoded record aka byte[]. + * If the record is already encoded, it will directly write the byte[] to the output stream without checking if it conforms to schema. + * Else, it encodes the record and writes to output stream. + * However, the first envelope should always be a record and not a byte[]. + * If the blocksize threshold crosses, it will upload the output stream contents as a block. + * If the number of records in current blob or size of current blob exceed limits then a new blob is created. + * Multi-threading and thread-safety: + * The underlying {@link org.apache.avro.file.DataFileWriter} is not thread-safe. + * For this reason, it is essential to wrap accesses to this object in a synchronized block. + * Method write(OutgoingMessageEnvelope) allows multiple threads to encode records as that operation is stateless but + * restricts access to the shared objects through the synchronized block. + * Concurrent access to shared objects is controlled through a common lock and synchronized block and hence ensures + * thread safety. + * @param ome - OutgoingMessageEnvelope that contains the IndexedRecord (GenericRecord or SpecificRecord) or an encoded record as byte[] + * @throws IOException when + * - OutgoingMessageEnvelope's message is not an IndexedRecord or + * - underlying dataFileWriter.append fails + * @throws IllegalStateException when the first OutgoingMessageEnvelope's message is not a record. + */ + @Override + public void write(OutgoingMessageEnvelope ome) throws IOException { + Optional optionalIndexedRecord; + byte[] encodedRecord; + if (ome.getMessage() instanceof IndexedRecord) { + optionalIndexedRecord = Optional.of((IndexedRecord) ome.getMessage()); + encodedRecord = encodeRecord((IndexedRecord) ome.getMessage()); + } else { + optionalIndexedRecord = Optional.empty(); + encodedRecord = (byte[]) ome.getMessage(); + } + + synchronized (currentDataFileWriterLock) { + // if currentBlobWriterComponents is null, then it is the first blob of this AzureBlobAvroWriter object + if (currentBlobWriterComponents == null || willCurrentBlobExceedSize(encodedRecord) || willCurrentBlobExceedRecordLimit()) { + startNextBlob(optionalIndexedRecord); + } + currentBlobWriterComponents.dataFileWriter.appendEncoded(ByteBuffer.wrap(encodedRecord)); + recordsInCurrentBlob++; + } + } + /** + * This method flushes all records written in dataFileWriter to the underlying AzureBlobOutputStream. + * dataFileWriter.flush then explicitly invokes flush of the AzureBlobOutputStream. + * This in turn async uploads content of the output stream as a block and reinits the output stream. + * AzureBlobOutputStream.flush is not ensured if dataFileWriter.flush fails. + * In such a scenario, the current block is not uploaded and blocks uploaded so far are lost. + * {@inheritDoc} + * @throws IOException if underlying dataFileWriter.flush fails + */ + @Override + public void flush() throws IOException { + synchronized (currentDataFileWriterLock) { + currentBlobWriterComponents.dataFileWriter.flush(); + } + } + + /** + * This method closes all DataFileWriters and output streams associated with all the blobs created. + * flush should be explicitly called before close. + * {@inheritDoc} + * @throws IllegalStateException when closing a closed writer + * @throws SamzaException if underlying DataFileWriter.close fails + */ + @Override + public void close() { + synchronized (currentDataFileWriterLock) { + if (isClosed) { + throw new IllegalStateException("Attempting to close an already closed AzureBlobAvroWriter"); + } + allBlobWriterComponents.forEach(blobWriterComponents -> { + try { + closeDataFileWriter(blobWriterComponents.dataFileWriter, blobWriterComponents.azureBlobOutputStream, + blobWriterComponents.blockBlobAsyncClient); + } catch (IOException e) { + throw new SamzaException(e); + } + }); + isClosed = true; + } + } + + @VisibleForTesting + AzureBlobAvroWriter(BlobContainerAsyncClient containerAsyncClient, AzureBlobWriterMetrics metrics, + Executor blobThreadPool, int maxBlockFlushThresholdSize, int flushTimeoutMs, String blobURLPrefix, + DataFileWriter dataFileWriter, + AzureBlobOutputStream azureBlobOutputStream, BlockBlobAsyncClient blockBlobAsyncClient, + long maxBlobSize, long maxRecordsPerBlob) { + if (dataFileWriter == null || azureBlobOutputStream == null || blockBlobAsyncClient == null) { + this.currentBlobWriterComponents = null; + } else { + this.currentBlobWriterComponents = + new BlobWriterComponents(dataFileWriter, azureBlobOutputStream, blockBlobAsyncClient); + } + this.allBlobWriterComponents.add(this.currentBlobWriterComponents); + this.blobThreadPool = blobThreadPool; + this.blobURLPrefix = blobURLPrefix; + this.metrics = metrics; + this.maxBlockFlushThresholdSize = maxBlockFlushThresholdSize; + this.flushTimeoutMs = flushTimeoutMs; + this.compression = new GzipCompression(); + this.containerAsyncClient = containerAsyncClient; + this.useRandomStringInBlobName = false; + this.maxBlobSize = maxBlobSize; + this.maxRecordsPerBlob = maxRecordsPerBlob; + } + + @VisibleForTesting + byte[] encodeRecord(IndexedRecord record) { + ByteArrayOutputStream out = new ByteArrayOutputStream(); + Schema schema = record.getSchema(); + try { + EncoderFactory encoderfactory = new EncoderFactory(); + BinaryEncoder encoder = encoderfactory.binaryEncoder(out, null); + DatumWriter writer; + if (record instanceof SpecificRecord) { + writer = new SpecificDatumWriter<>(schema); + } else { + writer = new GenericDatumWriter<>(schema); + } + writer.write(record, encoder); + encoder.flush(); //encoder may buffer + } catch (Exception e) { + throw new SamzaException("Unable to serialize Avro record using schema within the record: " + schema.toString(), e); + } + return out.toByteArray(); + } + + private static SimpleDateFormat buildUTCFormatter() { + SimpleDateFormat formatter = new SimpleDateFormat(PUBLISHED_FILE_NAME_DATE_FORMAT); + formatter.setTimeZone(TimeZone.getTimeZone("UTC")); + return formatter; + } + + private void closeDataFileWriter(DataFileWriter dataFileWriter, AzureBlobOutputStream azureBlobOutputStream, + BlockBlobAsyncClient blockBlobAsyncClient) throws IOException { + try { + LOG.info("Closing the blob: {}", blockBlobAsyncClient.getBlobUrl().toString()); + // dataFileWriter.close calls close of the azureBlobOutputStream associated with it. + dataFileWriter.close(); + } finally { + // ensure that close is called even if dataFileWriter.close fails. + // This is to avoid loss of all the blocks uploaded for the blob + // as commitBlockList happens in close of azureBlobOutputStream. + azureBlobOutputStream.close(); + } + } + + private void startNextBlob(Optional optionalIndexedRecord) throws IOException { + if (currentBlobWriterComponents != null) { + LOG.info("Starting new blob as current blob size is " + + currentBlobWriterComponents.azureBlobOutputStream.getSize() + + " and max blob size is " + maxBlobSize + + " or number of records is " + recordsInCurrentBlob + + " and max records in blob is " + maxRecordsPerBlob); + currentBlobWriterComponents.dataFileWriter.flush(); + currentBlobWriterComponents.azureBlobOutputStream.releaseBuffer(); + recordsInCurrentBlob = 0; + } + // datumWriter is null when AzureBlobAvroWriter is created but has not yet received a message. + // optionalIndexedRecord is the first message in this case. + if (datumWriter == null) { + if (optionalIndexedRecord.isPresent()) { + IndexedRecord record = optionalIndexedRecord.get(); + schema = record.getSchema(); + if (record instanceof SpecificRecord) { + datumWriter = new SpecificDatumWriter<>(schema); + } else { + datumWriter = new GenericDatumWriter<>(schema); + } + } else { + throw new IllegalStateException("Writing without schema setup."); + } + } + String blobURL; + if (useRandomStringInBlobName) { + blobURL = String.format(BLOB_NAME_RANDOM_STRING_AVRO, blobURLPrefix, + UTC_FORMATTER.format(System.currentTimeMillis()), blobNumber, UUID.randomUUID().toString().substring(0, 8), + compression.getFileExtension()); + } else { + blobURL = String.format(BLOB_NAME_AVRO, blobURLPrefix, + UTC_FORMATTER.format(System.currentTimeMillis()), blobNumber, compression.getFileExtension()); + } + LOG.info("Creating new blob: {}", blobURL); + BlockBlobAsyncClient blockBlobAsyncClient = containerAsyncClient.getBlobAsyncClient(blobURL).getBlockBlobAsyncClient(); + blobNumber++; + + DataFileWriter dataFileWriter = new DataFileWriter<>(datumWriter); + AzureBlobOutputStream azureBlobOutputStream = new AzureBlobOutputStream(blockBlobAsyncClient, blobThreadPool, metrics, + flushTimeoutMs, maxBlockFlushThresholdSize, compression); + dataFileWriter.create(schema, azureBlobOutputStream); + dataFileWriter.setFlushOnEveryBlock(false); + this.currentBlobWriterComponents = new BlobWriterComponents(dataFileWriter, azureBlobOutputStream, blockBlobAsyncClient); + allBlobWriterComponents.add(this.currentBlobWriterComponents); + LOG.info("Created new blob: {}", blobURL); + } + + private boolean willCurrentBlobExceedSize(byte[] encodedRecord) { + AzureBlobOutputStream azureBlobOutputStream = currentBlobWriterComponents.azureBlobOutputStream; + return (azureBlobOutputStream.getSize() + encodedRecord.length + DATAFILEWRITER_OVERHEAD) > maxBlobSize; + } + + private boolean willCurrentBlobExceedRecordLimit() { + return (recordsInCurrentBlob + 1) > maxRecordsPerBlob; + } + + /** + * Holds the components needed to write to an Azure Blob + * - including Avro's DataFileWriter, AzureBlobOutputStream and Azure's BlockBlobAsyncClient + */ + private class BlobWriterComponents { + final DataFileWriter dataFileWriter; + final AzureBlobOutputStream azureBlobOutputStream; + final BlockBlobAsyncClient blockBlobAsyncClient; + + public BlobWriterComponents(DataFileWriter dataFileWriter, AzureBlobOutputStream azureBlobOutputStream, + BlockBlobAsyncClient blockBlobAsyncClient) { + Preconditions.checkNotNull(dataFileWriter, "DataFileWriter can not be null when creating WriterComponents for an Azure Blob."); + Preconditions.checkNotNull(azureBlobOutputStream, "AzureBlobOutputStream can not be null when creating WriterComponents for an Azure Blob."); + Preconditions.checkNotNull(blockBlobAsyncClient, "BlockBlobAsyncClient can not be null when creating WriterComponents for an Azure Blob."); + this.dataFileWriter = dataFileWriter; + this.azureBlobOutputStream = azureBlobOutputStream; + this.blockBlobAsyncClient = blockBlobAsyncClient; + } + } +} diff --git a/samza-azure/src/main/java/org/apache/samza/system/azureblob/avro/AzureBlobAvroWriterFactory.java b/samza-azure/src/main/java/org/apache/samza/system/azureblob/avro/AzureBlobAvroWriterFactory.java new file mode 100644 index 0000000000..2510766312 --- /dev/null +++ b/samza-azure/src/main/java/org/apache/samza/system/azureblob/avro/AzureBlobAvroWriterFactory.java @@ -0,0 +1,43 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, + * software distributed under the License is distributed on an + * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY + * KIND, either express or implied. See the License for the + * specific language governing permissions and limitations + * under the License. + */ + +package org.apache.samza.system.azureblob.avro; + +import org.apache.samza.system.azureblob.compression.Compression; +import org.apache.samza.system.azureblob.producer.AzureBlobWriter; +import org.apache.samza.system.azureblob.producer.AzureBlobWriterFactory; +import org.apache.samza.system.azureblob.producer.AzureBlobWriterMetrics; +import com.azure.storage.blob.BlobContainerAsyncClient; +import java.io.IOException; +import java.util.concurrent.Executor; + + +public class AzureBlobAvroWriterFactory implements AzureBlobWriterFactory { + + /** + * {@inheritDoc} + */ + public AzureBlobWriter getWriterInstance(BlobContainerAsyncClient containerAsyncClient, String blobURL, + Executor blobUploadThreadPool, AzureBlobWriterMetrics metrics, + int maxBlockFlushThresholdSize, long flushTimeoutMs, Compression compression, boolean useRandomStringInBlobName, + long maxBlobSize, long maxMessagesPerBlob) throws IOException { + return new AzureBlobAvroWriter(containerAsyncClient, blobURL, blobUploadThreadPool, metrics, + maxBlockFlushThresholdSize, flushTimeoutMs, compression, useRandomStringInBlobName, maxBlobSize, maxMessagesPerBlob); + } +} diff --git a/samza-azure/src/main/java/org/apache/samza/system/azureblob/avro/AzureBlobOutputStream.java b/samza-azure/src/main/java/org/apache/samza/system/azureblob/avro/AzureBlobOutputStream.java new file mode 100644 index 0000000000..e03d005fa8 --- /dev/null +++ b/samza-azure/src/main/java/org/apache/samza/system/azureblob/avro/AzureBlobOutputStream.java @@ -0,0 +1,314 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, + * software distributed under the License is distributed on an + * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY + * KIND, either express or implied. See the License for the + * specific language governing permissions and limitations + * under the License. + */ + +package org.apache.samza.system.azureblob.avro; + +import com.azure.storage.blob.specialized.BlockBlobAsyncClient; +import com.google.common.annotations.VisibleForTesting; +import org.apache.samza.system.azureblob.compression.Compression; +import org.apache.samza.system.azureblob.producer.AzureBlobWriterMetrics; +import java.io.ByteArrayOutputStream; +import java.io.IOException; +import java.io.OutputStream; +import java.nio.ByteBuffer; +import java.util.ArrayList; +import java.util.Base64; +import java.util.Collections; +import java.util.Map; +import java.util.Optional; +import java.util.Set; +import java.util.concurrent.CompletableFuture; +import java.util.concurrent.ConcurrentHashMap; +import java.util.concurrent.ExecutionException; +import java.util.concurrent.Executor; +import java.util.concurrent.TimeUnit; +import java.util.concurrent.TimeoutException; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; +import reactor.core.publisher.Flux; + +/** + * This class extends {@link java.io.OutputStream} and uses {@link java.io.ByteArrayOutputStream} + * for caching the write calls till upload is not called. + * + * It asynchronously uploads the blocks and waits on them to finish at close. + * The blob is persisted at close. + * + * flush must be explicitly called before close. + * Any writes after a flush and before a close will be lost if no flush is called just before close. + * Once closed this object can not be used. + * + * releaseBuffer releases the underlying buffer i.e ByteArrayOutputStream which holds the data written until it is flushed. + * flush must be explicitly called prior to releaseBuffer else all data written + * since the beginning/previous flush will be lost. + * No data can be written after releaseBuffer, flush after releaseBuffer is a no-op + * and close must still be invoked to wait for all pending uploads to finish and persist the blob. + * releaseBuffer is optional and maybe called after its last flush and before close (which might happen much later), + * so as to reduce the overall memory footprint. close can not replace releaseBuffer as it is a blocking call. + * + * This library is thread safe. + */ +public class AzureBlobOutputStream extends OutputStream { + + private static final Logger LOG = LoggerFactory.getLogger(AzureBlobOutputStream.class); + private static final int MAX_ATTEMPT = 3; + private static final int MAX_BLOCKS_IN_AZURE_BLOB = 50000; + public static final String BLOB_RAW_SIZE_BYTES_METADATA = "rawSizeBytes"; + private final long flushTimeoutMs; + private final BlockBlobAsyncClient blobAsyncClient; + private final Executor blobThreadPool; + private Optional byteArrayOutputStream; + // All the block Names should be explicitly present in the blockList during CommitBlockList, + // even if stageBlock is a blocking call. + private final ArrayList blockList; + private final Set> pendingUpload = ConcurrentHashMap.newKeySet(); + private final int maxBlockFlushThresholdSize; + private final AzureBlobWriterMetrics metrics; + private final Compression compression; + + private volatile boolean isClosed = false; + private long totalUploadedBlockSize = 0; + private int blockNum; + + public AzureBlobOutputStream(BlockBlobAsyncClient blobAsyncClient, Executor blobThreadPool, AzureBlobWriterMetrics metrics, + long flushTimeoutMs, int maxBlockFlushThresholdSize, Compression compression) { + byteArrayOutputStream = Optional.of(new ByteArrayOutputStream(maxBlockFlushThresholdSize)); + this.blobAsyncClient = blobAsyncClient; + blockList = new ArrayList<>(); + blockNum = 0; + this.blobThreadPool = blobThreadPool; + this.flushTimeoutMs = flushTimeoutMs; + this.maxBlockFlushThresholdSize = maxBlockFlushThresholdSize; + this.metrics = metrics; + this.compression = compression; + } + + /** + * {@inheritDoc} + */ + @Override + public synchronized void write(int b) { + if (!byteArrayOutputStream.isPresent()) { + throw new IllegalStateException("Internal Buffer must have been released earlier for blob " + blobAsyncClient.getBlobUrl().toString()); + } + + if (byteArrayOutputStream.get().size() + 1 > maxBlockFlushThresholdSize) { + uploadBlockAsync(); + } + byteArrayOutputStream.get().write(b); + metrics.updateWriteByteMetrics(1); + } + + /** + * {@inheritDoc} + */ + @Override + public synchronized void write(byte[] b, int off, int len) { + if (!byteArrayOutputStream.isPresent()) { + throw new IllegalStateException("Internal Buffer must have been released earlier for blob " + blobAsyncClient.getBlobUrl().toString()); + } + + int remainingBytes = len; + int offset = off; + while (remainingBytes > 0) { + int bytesToWrite = Math.min(maxBlockFlushThresholdSize - byteArrayOutputStream.get().size(), remainingBytes); + byteArrayOutputStream.get().write(b, offset, bytesToWrite); + offset += bytesToWrite; + remainingBytes -= bytesToWrite; + if (byteArrayOutputStream.get().size() >= maxBlockFlushThresholdSize) { + uploadBlockAsync(); + } + } + metrics.updateWriteByteMetrics(len); + } + /** + * {@inheritDoc} + */ + @Override + public synchronized void flush() { + if (byteArrayOutputStream.isPresent()) { + uploadBlockAsync(); + } + } + + /** + * This api waits for all pending upload (stageBlock task) futures to finish. + * It then synchronously commits the list of blocks to persist the actual blob on storage. + * @throws IllegalStateException when + * - when closing an already closed stream + * @throws RuntimeException when + * - byteArrayOutputStream.close fails or + * - any of the pending uploads fails or + * - blob's commitBlockList fails + */ + @Override + public synchronized void close() { + + if (isClosed) { + return; + } + + LOG.info("{}: Close", blobAsyncClient.getBlobUrl().toString()); + try { + if (byteArrayOutputStream.isPresent()) { + byteArrayOutputStream.get().close(); + } + if (blockList.size() == 0) { + return; + } + CompletableFuture future = + CompletableFuture.allOf(pendingUpload.toArray(new CompletableFuture[0])); + + LOG.info("Closing blob: {} PendingUpload:{} ", blobAsyncClient.getBlobUrl().toString(), pendingUpload.size()); + + future.get((long) flushTimeoutMs, TimeUnit.MILLISECONDS); + LOG.info("For blob: {} committing blockList size:{}", blobAsyncClient.getBlobUrl().toString(), blockList.size()); + metrics.updateAzureCommitMetrics(); + Map blobMetadata = Collections.singletonMap(BLOB_RAW_SIZE_BYTES_METADATA, Long.toString(totalUploadedBlockSize)); + blobAsyncClient.commitBlockListWithResponse(blockList, null, blobMetadata, null, null).block(); + } catch (InterruptedException | ExecutionException | TimeoutException e) { + String msg = String.format("Close blob %s failed with exception. Total pending sends %d", + blobAsyncClient.getBlobUrl().toString(), pendingUpload.size()); + throw new RuntimeException(msg, e); + } catch (Exception e) { + String msg = String.format("Close blob %s failed with exception. Resetting the system producer. %s", + blobAsyncClient.getBlobUrl().toString(), e.getLocalizedMessage()); + throw new RuntimeException(msg, e); + } finally { + blockList.clear(); + pendingUpload.stream().forEach(future -> future.cancel(true)); + pendingUpload.clear(); + isClosed = true; + } + } + + /** + * Returns the size of the blob so far including data in the uploaded blocks and data currently in buffer. + * @return data written since the beginning + */ + public synchronized long getSize() { + return byteArrayOutputStream.isPresent() ? byteArrayOutputStream.get().size() + totalUploadedBlockSize : totalUploadedBlockSize; + } + + /** + * Releases the underlying buffer i.e; ByteArrayOutputStream. + * flush must be explicitly called prior to releaseBuffer else all data written + * since the beginning/previous flush will be lost. + * No data can be written after releaseBuffer, flush after releaseBuffer is a no-op + * and close must still be invoked to wait for all pending uploads to finish and persist the blob. + * This is optional and can be used to release memory. + * @throws IOException if ByteArrayOutputStream.close fails + */ + public synchronized void releaseBuffer() throws IOException { + if (byteArrayOutputStream.isPresent()) { + byteArrayOutputStream.get().close(); + byteArrayOutputStream = Optional.empty(); + LOG.info("Internal buffer has been released for blob " + blobAsyncClient.getBlobUrl().toString() + + ". Writes are no longer entertained."); + } + } + + @VisibleForTesting + AzureBlobOutputStream(BlockBlobAsyncClient blobAsyncClient, Executor blobThreadPool, AzureBlobWriterMetrics metrics, + int flushTimeoutMs, int maxBlockFlushThresholdSize, + ByteArrayOutputStream byteArrayOutputStream, Compression compression) { + this.byteArrayOutputStream = Optional.of(byteArrayOutputStream); + this.blobAsyncClient = blobAsyncClient; + blockList = new ArrayList<>(); + blockNum = 0; + this.blobThreadPool = blobThreadPool; + this.flushTimeoutMs = flushTimeoutMs; + this.maxBlockFlushThresholdSize = maxBlockFlushThresholdSize; + this.metrics = metrics; + this.compression = compression; + } + + /** + * This api will async upload the outputstream into block using stageBlocks, + * reint outputstream + * and add the operation to future. + * @throws RuntimeException when + * - blob's stageBlock fails after MAX_ATTEMPTs + * - number of blocks exceeds MAX_BLOCKS_IN_AZURE_BLOB + */ + private synchronized void uploadBlockAsync() { + if (!byteArrayOutputStream.isPresent()) { + return; + } + long size = byteArrayOutputStream.get().size(); + if (size == 0) { + return; + } + LOG.info("Blob: {} uploadBlock. Size:{}", blobAsyncClient.getBlobUrl().toString(), size); + + String blockId = String.format("%05d", blockNum); + String blockIdEncoded = Base64.getEncoder().encodeToString(blockId.getBytes()); + blockList.add(blockIdEncoded); + byte[] localByte = byteArrayOutputStream.get().toByteArray(); + byteArrayOutputStream.get().reset(); + totalUploadedBlockSize += localByte.length; + + CompletableFuture future = CompletableFuture.runAsync(new Runnable() { + // call async stageblock and add to future + @Override + public void run() { + int retryCount = 0; + byte[] compressedLocalByte = compression.compress(localByte); + int blockSize = compressedLocalByte.length; + + while (retryCount < MAX_ATTEMPT) { + try { + ByteBuffer outputStream = ByteBuffer.wrap(compressedLocalByte, 0, blockSize); + metrics.updateCompressByteMetrics(blockSize); + LOG.info("{} Upload block start for blob: {} for block size:{}.", blobAsyncClient.getBlobUrl().toString(), blockId, blockSize); + // StageBlock generates exception on Failure. + metrics.updateAzureUploadMetrics(); + blobAsyncClient.stageBlock(blockIdEncoded, Flux.just(outputStream), blockSize).block(); + break; + } catch (Exception e) { + retryCount += 1; + String msg = "Upload block for blob: " + blobAsyncClient.getBlobUrl().toString() + + " failed for blockid: " + blockId + " due to exception. RetryCount: " + retryCount; + LOG.error(msg, e); + if (retryCount == MAX_ATTEMPT) { + throw new RuntimeException("Exceeded number of retries. Max attempts is: " + MAX_ATTEMPT, e); + } + } + } + } + }, blobThreadPool); + + pendingUpload.add(future); + future.handle((aVoid, throwable) -> { + if (throwable == null) { + LOG.info("Upload block for blob: {} with blockid: {} finished.", blobAsyncClient.getBlobUrl().toString(), blockId); + pendingUpload.remove(future); + return aVoid; + } else { + throw new RuntimeException("Blob upload failed for blob " + blobAsyncClient.getBlobUrl().toString() + + " and block with id: " + blockId, throwable); + } + }); + + blockNum += 1; + if (blockNum >= MAX_BLOCKS_IN_AZURE_BLOB) { + throw new RuntimeException("Azure blob only supports 50000 blocks in a blob. Current number of blocks is " + blockNum); + } + } +} \ No newline at end of file diff --git a/samza-azure/src/main/java/org/apache/samza/system/azureblob/compression/Compression.java b/samza-azure/src/main/java/org/apache/samza/system/azureblob/compression/Compression.java new file mode 100644 index 0000000000..caaab178d2 --- /dev/null +++ b/samza-azure/src/main/java/org/apache/samza/system/azureblob/compression/Compression.java @@ -0,0 +1,41 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, + * software distributed under the License is distributed on an + * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY + * KIND, either express or implied. See the License for the + * specific language governing permissions and limitations + * under the License. + */ + +package org.apache.samza.system.azureblob.compression; + +/** + * Interface for compression. + */ +public interface Compression { + /** + * Compresses a byte[]. + * Depending on the type of compression, the compressed byte[] might be larger than input for small inputs. + * For example: Gzip compression of an 0 length byte array yields byte[24] due to CRC and other metadata it adds. + * @param input byte[] to be compressed + * @return compressed byte[]. + */ + byte[] compress(byte[] input); + + /** + * The file extension to be used for the particular compression. + * For example: .gz for GZIP compression and "" for no compression. + * @return + */ + String getFileExtension(); +} diff --git a/samza-azure/src/main/java/org/apache/samza/system/azureblob/compression/CompressionFactory.java b/samza-azure/src/main/java/org/apache/samza/system/azureblob/compression/CompressionFactory.java new file mode 100644 index 0000000000..4a45636531 --- /dev/null +++ b/samza-azure/src/main/java/org/apache/samza/system/azureblob/compression/CompressionFactory.java @@ -0,0 +1,36 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, + * software distributed under the License is distributed on an + * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY + * KIND, either express or implied. See the License for the + * specific language governing permissions and limitations + * under the License. + */ + +package org.apache.samza.system.azureblob.compression; + +/** + * This factory instantiates the appropriate implementation of + * {@link org.apache.samza.system.azureblob.compression.Compression} + * based on the {@link org.apache.samza.system.azureblob.compression.CompressionType}. + */ +public class CompressionFactory { + public Compression getCompression(CompressionType compressionType) { + switch (compressionType) { + case NONE: return new NoneCompression(); + case GZIP: return new GzipCompression(); + default: throw new IllegalArgumentException("Unknown compression name: " + compressionType.name()); + } + + } +} diff --git a/samza-azure/src/main/java/org/apache/samza/system/azureblob/compression/CompressionType.java b/samza-azure/src/main/java/org/apache/samza/system/azureblob/compression/CompressionType.java new file mode 100644 index 0000000000..6b8c176702 --- /dev/null +++ b/samza-azure/src/main/java/org/apache/samza/system/azureblob/compression/CompressionType.java @@ -0,0 +1,31 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, + * software distributed under the License is distributed on an + * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY + * KIND, either express or implied. See the License for the + * specific language governing permissions and limitations + * under the License. + */ + +package org.apache.samza.system.azureblob.compression; + +/** + * The compression type to use. + * Supported compression types: + * - None: no compression is applied + * - GZIP: gzip compression is applied + */ +public enum CompressionType { + NONE, GZIP +} + diff --git a/samza-azure/src/main/java/org/apache/samza/system/azureblob/compression/GzipCompression.java b/samza-azure/src/main/java/org/apache/samza/system/azureblob/compression/GzipCompression.java new file mode 100644 index 0000000000..d2d2f2997d --- /dev/null +++ b/samza-azure/src/main/java/org/apache/samza/system/azureblob/compression/GzipCompression.java @@ -0,0 +1,75 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, + * software distributed under the License is distributed on an + * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY + * KIND, either express or implied. See the License for the + * specific language governing permissions and limitations + * under the License. + */ + +package org.apache.samza.system.azureblob.compression; + +import java.io.ByteArrayOutputStream; +import java.io.IOException; +import java.util.zip.GZIPOutputStream; +import org.apache.commons.lang3.Validate; +import org.apache.samza.SamzaException; + + +/** + * This class implements the {@link org.apache.samza.system.azureblob.compression.Compression}. + * It uses GZIPOutputStream to compress the given byte[]. + * The file extension for to be used for this compressed data is ".gz" + */ +public class GzipCompression implements Compression { + /** + * {@inheritDoc} + * @throws SamzaException if compression fails + */ + @Override + public byte[] compress(byte[] input) { + Validate.notNull(input, "Input for compression is null"); + + ByteArrayOutputStream byteArrayOutputStream = null; + GZIPOutputStream gzipOutputStream = null; + try { + byteArrayOutputStream = new ByteArrayOutputStream(input.length); + gzipOutputStream = new GZIPOutputStream(byteArrayOutputStream); + gzipOutputStream.write(input); + gzipOutputStream.close(); + return byteArrayOutputStream.toByteArray(); + + } catch (IOException e) { + throw new SamzaException("Failed to compress.", e); + } finally { + try { + if (gzipOutputStream != null) { + gzipOutputStream.close(); + } + if (byteArrayOutputStream != null) { + byteArrayOutputStream.close(); + } + } catch (Exception e) { + throw new SamzaException("Failed to close output streams during compression.", e); + } + } + } + + /** + * {@inheritDoc} + */ + @Override + public String getFileExtension() { + return ".gz"; + } +} diff --git a/samza-azure/src/main/java/org/apache/samza/system/azureblob/compression/NoneCompression.java b/samza-azure/src/main/java/org/apache/samza/system/azureblob/compression/NoneCompression.java new file mode 100644 index 0000000000..c76322cc52 --- /dev/null +++ b/samza-azure/src/main/java/org/apache/samza/system/azureblob/compression/NoneCompression.java @@ -0,0 +1,37 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, + * software distributed under the License is distributed on an + * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY + * KIND, either express or implied. See the License for the + * specific language governing permissions and limitations + * under the License. + */ + +package org.apache.samza.system.azureblob.compression; + +/** + * This class implements the {@link org.apache.samza.system.azureblob.compression.Compression}. + * It does not compress the input and returns it as is. + * There is no file extension. + */ +public class NoneCompression implements Compression { + @Override + public byte[] compress(byte[] input) { + return input; + } + + @Override + public String getFileExtension() { + return ""; + } +} diff --git a/samza-azure/src/main/java/org/apache/samza/system/azureblob/producer/AzureBlobSystemProducer.java b/samza-azure/src/main/java/org/apache/samza/system/azureblob/producer/AzureBlobSystemProducer.java new file mode 100644 index 0000000000..0a4600b833 --- /dev/null +++ b/samza-azure/src/main/java/org/apache/samza/system/azureblob/producer/AzureBlobSystemProducer.java @@ -0,0 +1,505 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, + * software distributed under the License is distributed on an + * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY + * KIND, either express or implied. See the License for the + * specific language governing permissions and limitations + * under the License. + */ + +package org.apache.samza.system.azureblob.producer; + +import com.azure.core.http.HttpClient; +import com.azure.core.http.HttpResponse; +import com.azure.core.http.ProxyOptions; +import com.azure.core.http.netty.NettyAsyncHttpClientBuilder; +import com.azure.core.http.policy.HttpLogDetailLevel; +import com.azure.core.http.policy.HttpLogOptions; +import com.azure.core.util.Configuration; +import com.azure.storage.blob.BlobContainerAsyncClient; +import com.azure.storage.blob.BlobServiceAsyncClient; +import com.azure.storage.blob.BlobServiceClientBuilder; +import com.azure.storage.blob.models.BlobErrorCode; +import com.azure.storage.blob.models.BlobStorageException; +import com.azure.storage.blob.models.SkuName; +import com.azure.storage.common.StorageSharedKeyCredential; +import com.google.common.annotations.VisibleForTesting; +import com.google.common.base.Preconditions; +import org.apache.samza.system.azureblob.AzureBlobConfig; +import org.apache.samza.system.azureblob.compression.CompressionFactory; +import java.io.IOException; +import java.net.InetSocketAddress; +import java.util.Locale; +import java.util.Map; +import java.util.Set; +import java.util.concurrent.BlockingQueue; +import java.util.concurrent.CompletableFuture; +import java.util.concurrent.ConcurrentHashMap; +import java.util.concurrent.LinkedBlockingDeque; +import java.util.concurrent.ThreadPoolExecutor; +import java.util.concurrent.TimeUnit; +import java.util.concurrent.locks.ReadWriteLock; +import java.util.concurrent.locks.ReentrantReadWriteLock; +import org.apache.samza.metrics.MetricsRegistry; +import org.apache.samza.system.OutgoingMessageEnvelope; +import org.apache.samza.system.SystemProducer; +import org.apache.samza.system.SystemProducerException; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + + +/** + * AzureBlob system producer to send messages to Azure Blob Storage. + * This system producer is thread safe. + * For different sources: sends/flushes can happen in parallel. + * For same source: It supports sends in parallel. flushes are exclusive. + * + * + * Azure Blob Storage has a 3 level hierarchy: an Azure account contains multiple containers (akin to directories + * in a file system) and each container has multiple blobs (akin to files). + * + * Azure Container: System name maps to the name of Azure container. + * An instance of a system producer writes to a single Azure container considering the container as a system. + * + * Azure Blob: For a given stream-partition pair, a blob is created with name stream/partition/timestamp-randomString. + * The stream and partition are extracted from the SSP of OutgoingMessageEnvelope in send(). + * Blob is started when the first message for that stream-partition is sent by a source + * and closed during flush for that source. + * Subsequent sends by the source to the same stream-partition will create a new blob with a different timestamp. + * Thus, timestamp corresponds to writer creation time i.e; the first send for source-SSP + * or first send after a flush for the source. + * If max blob size or record limit are configured, then a new blob is started when limits exceed. + * + * A random string is used as a suffix in the blob name to prevent collisions: + * - if two system producers are writing to the same SSP. + * - if two sources send to the same SSP. + * + * Lifecycle of the system producer is shown below. All sources have to be registered before starting the producer. + * Several messages can be sent by a source via send(source, envelope). This can be followed by a flush(source) or stop() + * After flush(source), more messages can be sent for that source and other sources as well. stop() internally calls + * flush(source) for all the sources registered. After stop(), no calls to send and flush are allowed. + * + * + * ┌──────────────────────────────┐ + * │ │ + * ▼ │ + * Lifecycle: register(source) ────────▶ start() ──────▶ send(source, envelope) ──────▶ flush(source) ──────▶ stop() + * [multiple times │ ▲ │ ▲ + * for └────┘ └─────────────────────────────────────┘ + * multiple sources] + * + * This SystemProducer does not open up the envelopes sent through it. It is the responsibility of the user of this + * SystemProducer to ensure the envelopes are valid and a correct writer has been chosen by wiring up the + * writer factory config. + * + */ +public class AzureBlobSystemProducer implements SystemProducer { + + private static final Logger LOG = LoggerFactory.getLogger(AzureBlobSystemProducer.class.getName()); + + private static final String BLOB_NAME_PREFIX = "%s"; + private static final String BLOB_NAME_PARTITION_PREFIX = "%s/%s"; + + private static final String AZURE_URL = "https://%s.blob.core.windows.net"; + + private static final int PREMIUM_MAX_BLOCK_SIZE = 100 * 1024 * 1024; // 100MB + private static final int STANDARD_MAX_BLOCK_SIZE = 4 * 1024 * 1024; // 4MB + + private BlobContainerAsyncClient containerAsyncClient; + private final String systemName; + private final AzureBlobConfig config; + + // Map of writers indexed first by sourceName and then by (streamName, partitionName) or just streamName if partition key does not exist. + private Map> writerMap; + private final AzureBlobWriterFactory writerFactory; + + private final int blockFlushThresholdSize; + private final long flushTimeoutMs; + private final long closeTimeout; + private final ThreadPoolExecutor asyncBlobThreadPool; + + private volatile boolean isStarted = false; + private volatile boolean isStopped = false; + + private final AzureBlobSystemProducerMetrics metrics; + + private final Map sourceWriterCreationLockMap = new ConcurrentHashMap<>(); + private final Map sourceSendFlushLockMap = new ConcurrentHashMap<>(); + + public AzureBlobSystemProducer(String systemName, AzureBlobConfig config, MetricsRegistry metricsRegistry) { + Preconditions.checkNotNull(systemName, "System name can not be null when creating AzureBlobSystemProducer"); + Preconditions.checkNotNull(config, "Config can not be null when creating AzureBlobSystemProducer"); + Preconditions.checkNotNull(metricsRegistry, "Metrics registry can not be null when creating AzureBlobSystemProducer"); + + // Azure logs do not show without this property set + System.setProperty(Configuration.PROPERTY_AZURE_LOG_LEVEL, "1"); + this.systemName = systemName; + this.config = config; + + String writerFactoryClassName = this.config.getAzureBlobWriterFactoryClassName(this.systemName); + try { + this.writerFactory = (AzureBlobWriterFactory) Class.forName(writerFactoryClassName).newInstance(); + } catch (Exception e) { + throw new SystemProducerException("Could not create writer factory with name " + writerFactoryClassName, e); + } + this.flushTimeoutMs = this.config.getFlushTimeoutMs(this.systemName); + this.closeTimeout = this.config.getCloseTimeoutMs(this.systemName); + this.blockFlushThresholdSize = this.config.getMaxFlushThresholdSize(this.systemName); + int asyncBlobThreadPoolCount = this.config.getAzureBlobThreadPoolCount(this.systemName); + int blockingQueueSize = this.config.getBlockingQueueSizeOrDefault(this.systemName, asyncBlobThreadPoolCount * 2); + + LOG.info("SystemName: {} block flush size:{}", systemName, this.blockFlushThresholdSize); + LOG.info("SystemName: {} thread count:{}", systemName, asyncBlobThreadPoolCount); + + BlockingQueue + linkedBlockingDeque = new LinkedBlockingDeque<>(blockingQueueSize); + + this.asyncBlobThreadPool = + new ThreadPoolExecutor(asyncBlobThreadPoolCount, asyncBlobThreadPoolCount, 60, + TimeUnit.SECONDS, linkedBlockingDeque, new ThreadPoolExecutor.CallerRunsPolicy()); + + this.writerMap = new ConcurrentHashMap<>(); + + this.metrics = new AzureBlobSystemProducerMetrics(systemName, config.getAzureAccountName(systemName), metricsRegistry); + } + + /** + * {@inheritDoc} + * @throws SystemProducerException + */ + @Override + public synchronized void start() { + if (isStarted) { + throw new SystemProducerException("Attempting to start an already started producer."); + } + + String accountName = config.getAzureAccountName(systemName); + String accountKey = config.getAzureAccountKey(systemName); + + setupAzureContainer(accountName, accountKey); + + LOG.info("Starting producer."); + isStarted = true; + } + + /** + * {@inheritDoc} + * @throws SystemProducerException + */ + @Override + public synchronized void stop() { + if (!isStarted) { + throw new SystemProducerException("Attempting to stop a producer that was not started."); + } + + if (isStopped) { + throw new SystemProducerException("Attempting to stop an already stopped producer."); + } + + try { + writerMap.forEach((source, sourceWriterMap) -> flush(source)); + asyncBlobThreadPool.shutdown(); + isStarted = false; + } catch (Exception e) { + throw new SystemProducerException("Stop failed with exception.", e); + } finally { + writerMap.clear(); + isStopped = true; + } + } + + /** + * {@inheritDoc} + * @throws SystemProducerException + */ + @Override + public synchronized void register(String source) { + LOG.info("Registering source {}", source); + if (isStarted) { + throw new SystemProducerException("Cannot register once the producer is started."); + } + if (writerMap.containsKey(source)) { + // source already registered => writerMap and metrics have entries for the source + LOG.warn("Source: {} already registered", source); + return; + } + writerMap.put(source, new ConcurrentHashMap<>()); + sourceWriterCreationLockMap.put(source, new Object()); + sourceSendFlushLockMap.put(source, new ReentrantReadWriteLock()); + metrics.register(source); + } + + /** + * Multi-threading and thread-safety: + * + * From Samza usage of SystemProducer: + * The lifecycle of SystemProducer shown above is consistent with most use cases within Samza (with the exception of + * Coordinator stream store/producer and KafkaCheckpointManager). + * A single parent thread creates the SystemProducer, registers all sources and starts it before handing it + * to multiple threads for use (send and flush). Finally, the single parent thread stops the producer. + * The most frequent operations on a SystemProducer are send and flush while register, start and stop are one-time operations. + * + * Based on this usage pattern: to provide multi-threaded support and improve throughput of this SystemProducer, + * multiple sends and flushes need to happen in parallel. However, the following rules are needed to ensure + * o data loss and data consistency. + * 1. sends can happen in parallel for same source or different sources. + * 2. send and flush for the same source can not happen in parallel. Although, the AzureBlobWriter is thread safe, + * interleaving write and flush and close operations of a writer can lead to data loss if a write happens between flush and close. + * There are other scenarios such as issuing a write to the writer after close and so on. + * 3. writer creation for the same writer key (SSP) can not happen in parallel - for the reason that multiple + * writers could get created with only one being retained but all being used and GCed after a send, leading to data loss. + * + * These 3 rules are achieved by using a per source ReadWriteLock to allow sends in parallel but guarantee exclusivity for flush. + * Additionally, a per source lock is used to ensure writer creation is in a critical section. + * + * Concurrent access to shared objects as follows: + * 1. AzureBlobWriters is permitted as long as there are no interleaving of operations for a writer. + * If multiple operations of writer (as in flush) then make it synchronized. + * 2. ConcurrentHashMaps (esp writerMap per source) get and put - disallow interleaving by doing put and clear under locks. + * 3. WriterFactory and Metrics are thread-safe. WriterFactory is stateless while Metrics' operations interleaving + * is thread-safe too as they work on different counters. + * The above locking mechanisms ensure thread-safety. + * {@inheritDoc} + * @throws SystemProducerException + */ + @Override + public void send(String source, OutgoingMessageEnvelope messageEnvelope) { + if (!isStarted) { + throw new SystemProducerException("Trying to send before producer has started."); + } + ReadWriteLock lock = sourceSendFlushLockMap.get(source); + if (lock == null) { + throw new SystemProducerException("Attempting to send to source: " + source + " but it was not registered"); + } + lock.readLock().lock(); + try { + AzureBlobWriter writer = getOrCreateWriter(source, messageEnvelope); + writer.write(messageEnvelope); + metrics.updateWriteMetrics(source); + } catch (Exception e) { + metrics.updateErrorMetrics(source); + Object partitionKey = getPartitionKey(messageEnvelope); + String msg = "Send failed for source: " + source + ", system: " + systemName + + ", stream: " + messageEnvelope.getSystemStream().getStream() + + ", partitionKey: " + ((partitionKey != null) ? partitionKey : "null"); + throw new SystemProducerException(msg, e); + } finally { + lock.readLock().unlock(); + } + } + + /** + * {@inheritDoc} + * @throws SystemProducerException + */ + @Override + public void flush(String source) { + if (!isStarted) { + throw new SystemProducerException("Trying to flush before producer has started."); + } + ReadWriteLock lock = sourceSendFlushLockMap.get(source); + if (lock == null) { + throw new SystemProducerException("Attempting to flush source: " + source + " but it was not registered"); + } + lock.writeLock().lock(); + Map sourceWriterMap = writerMap.get(source); + try { + // first flush all the writers + // then close and remove all the writers + flushWriters(sourceWriterMap); + closeWriters(source, sourceWriterMap); + } catch (Exception e) { + metrics.updateErrorMetrics(source); + throw new SystemProducerException("Flush failed for system:" + systemName + " and source: " + source, e); + } finally { + sourceWriterMap.clear(); + lock.writeLock().unlock(); + } + } + + @VisibleForTesting + void setupAzureContainer(String accountName, String accountKey) { + try { + // Use your Azure Blob Storage account's name and key to create a credential object to access your account. + StorageSharedKeyCredential credential = new StorageSharedKeyCredential(accountName, accountKey); + + HttpClient httpClient; + if (config.getUseProxy(systemName)) { + LOG.info("HTTP Proxy setup for AzureBlob pipeline"); + httpClient = new NettyAsyncHttpClientBuilder() + .proxy(new ProxyOptions(ProxyOptions.Type.HTTP, + new InetSocketAddress(config.getAzureProxyHostname(systemName), config.getAzureProxyPort(systemName)))).build(); + } else { + httpClient = HttpClient.createDefault(); + } + + // From the Azure portal, get your Storage account blob service AsyncClient endpoint. + String endpoint = String.format(Locale.ROOT, AZURE_URL, accountName); + + HttpLogOptions httpLogOptions = new HttpLogOptions(); + httpLogOptions.setLogLevel(HttpLogDetailLevel.BASIC); + BlobServiceAsyncClient storageClient = + new BlobServiceClientBuilder() + .httpLogOptions(httpLogOptions) + .endpoint(endpoint) + .credential(credential) + .httpClient(httpClient) + .buildAsyncClient(); + + + SkuName accountType = storageClient.getAccountInfo().block().getSkuName(); + long flushThresholdSize = config.getMaxFlushThresholdSize(systemName); + boolean isPremiumAccount = SkuName.PREMIUM_LRS == accountType; + if (isPremiumAccount && flushThresholdSize > PREMIUM_MAX_BLOCK_SIZE) { // 100 MB + throw new SystemProducerException("Azure storage account with name: " + accountName + + " is a premium account and can only handle upto 100MB threshold size. Given flush threshold size is " + + flushThresholdSize); + } else if (!isPremiumAccount && flushThresholdSize > STANDARD_MAX_BLOCK_SIZE) { // STANDARD account + throw new SystemProducerException("Azure storage account with name: " + accountName + + " is a standard account and can only handle upto 4MB threshold size. Given flush threshold size is " + + flushThresholdSize); + } + + containerAsyncClient = storageClient.getBlobContainerAsyncClient(systemName); + + // Only way to check if container exists or not is by creating it and look for failure/success. + createContainerIfNotExists(containerAsyncClient); + } catch (Exception e) { + metrics.updateAzureContainerMetrics(); + throw new SystemProducerException("Failed to set up Azure container for SystemName: " + systemName, e); + } + } + + /** + * // find the writer in the writerMap else create one + * @param source for which to find/create the writer + * @param messageEnvelope to fetch the schema from if writer needs to be created + * @return an AzureBlobWriter object + */ + @VisibleForTesting + AzureBlobWriter getOrCreateWriter(String source, OutgoingMessageEnvelope messageEnvelope) { + String writerMapKey; + String blobURLPrefix; + String partitionKey = getPartitionKey(messageEnvelope); + // using most significant bits in UUID (8 digits) to avoid collision in blob names + if (partitionKey == null) { + writerMapKey = messageEnvelope.getSystemStream().getStream(); + blobURLPrefix = String.format(BLOB_NAME_PREFIX, messageEnvelope.getSystemStream().getStream()); + } else { + writerMapKey = messageEnvelope.getSystemStream().getStream() + "/" + partitionKey; + blobURLPrefix = String.format(BLOB_NAME_PARTITION_PREFIX, messageEnvelope.getSystemStream().getStream(), partitionKey); + } + Map sourceWriterMap = writerMap.get(source); + if (sourceWriterMap == null) { + throw new SystemProducerException("Attempting to send to source: " + source + " but it is not registered"); + } + AzureBlobWriter writer = sourceWriterMap.get(writerMapKey); + if (writer == null) { + synchronized (sourceWriterCreationLockMap.get(source)) { + writer = sourceWriterMap.get(writerMapKey); + if (writer == null) { + AzureBlobWriterMetrics writerMetrics = + new AzureBlobWriterMetrics(metrics.getAggregateMetrics(), metrics.getSystemMetrics(), metrics.getSourceMetrics(source)); + writer = createNewWriter(blobURLPrefix, writerMetrics); + sourceWriterMap.put(writerMapKey, writer); + } + } + } + return writer; + } + + private void createContainerIfNotExists(BlobContainerAsyncClient containerClient) { + try { + containerClient.create().block(); + } catch (BlobStorageException e) { + //StorageErrorCode defines constants corresponding to all error codes returned by the service. + if (e.getErrorCode() == BlobErrorCode.RESOURCE_NOT_FOUND) { + HttpResponse response = e.getResponse(); + LOG.error("Error creating the container url " + containerClient.getBlobContainerUrl().toString() + " with status code: " + response.getStatusCode(), e); + } else if (e.getErrorCode() == BlobErrorCode.CONTAINER_BEING_DELETED) { + LOG.error("Container is being deleted. Container URL is: " + containerClient.getBlobContainerUrl().toString(), e); + } else if (e.getErrorCode() == BlobErrorCode.CONTAINER_ALREADY_EXISTS) { + return; + } + throw e; + } + } + + private String getPartitionKey(OutgoingMessageEnvelope messageEnvelope) { + Object partitionKey = messageEnvelope.getPartitionKey(); + if (partitionKey == null || !(partitionKey instanceof String)) { + return null; + } + return (String) partitionKey; + } + + private void flushWriters(Map sourceWriterMap) { + sourceWriterMap.forEach((stream, writer) -> { + try { + LOG.info("Flushing topic:{}", stream); + writer.flush(); + } catch (IOException e) { + throw new SystemProducerException("Close failed for topic " + stream, e); + } + }); + } + + private void closeWriters(String source, Map sourceWriterMap) throws Exception { + Set> pendingClose = ConcurrentHashMap.newKeySet(); + try { + sourceWriterMap.forEach((stream, writer) -> { + LOG.info("Closing topic:{}", stream); + CompletableFuture future = CompletableFuture.runAsync(new Runnable() { + @Override + public void run() { + try { + writer.close(); + } catch (IOException e) { + throw new SystemProducerException("Close failed for topic " + stream, e); + } + } + }, asyncBlobThreadPool); + pendingClose.add(future); + future.handle((aVoid, throwable) -> { + sourceWriterMap.remove(writer); + if (throwable != null) { + throw new SystemProducerException("Close failed for topic " + stream, throwable); + } else { + LOG.info("Blob close finished for stream " + stream); + return aVoid; + } + }); + }); + CompletableFuture future = CompletableFuture.allOf(pendingClose.toArray(new CompletableFuture[0])); + LOG.info("Flush source: {} has pending closes: {} ", source, pendingClose.size()); + future.get((long) closeTimeout, TimeUnit.MILLISECONDS); + } finally { + pendingClose.clear(); + } + } + + @VisibleForTesting + AzureBlobWriter createNewWriter(String blobURL, AzureBlobWriterMetrics writerMetrics) { + try { + return writerFactory.getWriterInstance(containerAsyncClient, blobURL, asyncBlobThreadPool, writerMetrics, + blockFlushThresholdSize, flushTimeoutMs, + new CompressionFactory().getCompression(config.getCompressionType(systemName)), + config.getSuffixRandomStringToBlobName(systemName), + config.getMaxBlobSize(systemName), + config.getMaxMessagesPerBlob(systemName)); + } catch (Exception e) { + throw new RuntimeException("Failed to create a writer for the producer.", e); + } + } + +} \ No newline at end of file diff --git a/samza-azure/src/main/java/org/apache/samza/system/azureblob/producer/AzureBlobSystemProducerMetrics.java b/samza-azure/src/main/java/org/apache/samza/system/azureblob/producer/AzureBlobSystemProducerMetrics.java new file mode 100644 index 0000000000..6629e68be4 --- /dev/null +++ b/samza-azure/src/main/java/org/apache/samza/system/azureblob/producer/AzureBlobSystemProducerMetrics.java @@ -0,0 +1,134 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, + * software distributed under the License is distributed on an + * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY + * KIND, either express or implied. See the License for the + * specific language governing permissions and limitations + * under the License. + */ + +package org.apache.samza.system.azureblob.producer; + +import org.apache.samza.system.azureblob.AzureBlobBasicMetrics; +import java.util.HashMap; +import java.util.Map; +import org.apache.samza.metrics.Counter; +import org.apache.samza.metrics.MetricsRegistry; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + + +/** + * This class holds all the metrics for a {@link org.apache.samza.system.azureblob.producer.AzureBlobSystemProducer}. + * It maintains: aggregate metrics, system level metrics and source metrics for all the sources that register and + * send through the SystemProducer. It has a map for holding the metrics of all sources. + * + * Apart from the basic metrics for each group, this class also holds metrics for Azure container creation errors. + */ +public class AzureBlobSystemProducerMetrics { + private static final Logger LOG = LoggerFactory.getLogger(AzureBlobSystemProducerMetrics.class.getName()); + + protected static final String AZURE_CONTAINER_ERROR = "azureContainerError"; + protected static final String AGGREGATE = "aggregate"; + protected static final String SYSTEM_METRIC_FORMAT = "%s_%s"; + + private final MetricsRegistry metricsRegistry; + private final Map sourceMetricsMap; + private final AzureBlobBasicMetrics aggregateMetrics; + private final AzureBlobBasicMetrics systemMetrics; + private final Counter aggregateAzureContainerErrorMetrics; + private final Counter systemAzureContainerErrorMetrics; + + private final String systemName; + private final String accountName; + + public AzureBlobSystemProducerMetrics(String systemName, String accountName, MetricsRegistry metricsRegistry) { + this.metricsRegistry = metricsRegistry; + this.systemName = systemName; + this.accountName = accountName; + + sourceMetricsMap = new HashMap<>(); + aggregateMetrics = new AzureBlobBasicMetrics(AGGREGATE, metricsRegistry); + systemMetrics = new AzureBlobBasicMetrics(String.format(SYSTEM_METRIC_FORMAT, accountName, systemName), metricsRegistry); + aggregateAzureContainerErrorMetrics = metricsRegistry.newCounter(AGGREGATE, AZURE_CONTAINER_ERROR); + systemAzureContainerErrorMetrics = metricsRegistry.newCounter(String.format(SYSTEM_METRIC_FORMAT, accountName, systemName), AZURE_CONTAINER_ERROR); + } + + /** + * Adds a AzureBlobBasicMetrics object for the source being registered with the SystemProducer. + * @param source to be registered. + */ + public void register(String source) { + if (systemName.equals(source)) { + // source is the same as the system name. creating counters for source name will double count metrics + LOG.warn("Source:{} is the same as the system name.", source); + return; + } + sourceMetricsMap.putIfAbsent(source, new AzureBlobBasicMetrics(source, metricsRegistry)); + } + + /** + * Increments the error metrics counters of aggregate, system and the source by 1. + * @param source for which the error occurred. + */ + public void updateErrorMetrics(String source) { + AzureBlobBasicMetrics sourceMetrics = sourceMetricsMap.get(source); + if (sourceMetrics != null) { + sourceMetrics.updateErrorMetrics(); + } + incrementErrorMetrics(); + } + + /** + * Increments the write metrics counters of aggregate, system and the source by 1. + * Write metrics is for number of messages successfully written to the source. + * @param source for which the message was sent. + */ + public void updateWriteMetrics(String source) { + AzureBlobBasicMetrics sourceMetrics = sourceMetricsMap.get(source); + if (sourceMetrics != null) { + sourceMetrics.updateWriteMetrics(); + } + incrementWriteMetrics(); + } + + /** + * Increments the Azure container creation error metrics by 1. + */ + public void updateAzureContainerMetrics() { + aggregateAzureContainerErrorMetrics.inc(); + systemAzureContainerErrorMetrics.inc(); + } + + public AzureBlobBasicMetrics getAggregateMetrics() { + return aggregateMetrics; + } + + public AzureBlobBasicMetrics getSystemMetrics() { + return systemMetrics; + } + + public AzureBlobBasicMetrics getSourceMetrics(String source) { + return sourceMetricsMap.get(source); + } + + private void incrementWriteMetrics() { + aggregateMetrics.updateWriteMetrics(); + systemMetrics.updateWriteMetrics(); + } + + private void incrementErrorMetrics() { + aggregateMetrics.updateErrorMetrics(); + systemMetrics.updateErrorMetrics(); + } +} diff --git a/samza-azure/src/main/java/org/apache/samza/system/azureblob/producer/AzureBlobWriter.java b/samza-azure/src/main/java/org/apache/samza/system/azureblob/producer/AzureBlobWriter.java new file mode 100644 index 0000000000..f7fd4220bd --- /dev/null +++ b/samza-azure/src/main/java/org/apache/samza/system/azureblob/producer/AzureBlobWriter.java @@ -0,0 +1,52 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, + * software distributed under the License is distributed on an + * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY + * KIND, either express or implied. See the License for the + * specific language governing permissions and limitations + * under the License. + */ + +package org.apache.samza.system.azureblob.producer; + +import java.io.IOException; +import org.apache.samza.system.OutgoingMessageEnvelope; + +/** + * {@link org.apache.samza.system.azureblob.producer.AzureBlobSystemProducer} + * uses an AzureBlobWriter to write messages to Azure Blob Storage. + * + * Implementation is expected to be thread-safe. + */ +public interface AzureBlobWriter { + /** + * Write the given {@link org.apache.samza.system.OutgoingMessageEnvelope} to the blob opened. + * @param ome message to be written + */ + void write(OutgoingMessageEnvelope ome) throws IOException; + + /** + * Asynchronously upload the messages written as a block. + * After this the messages written will go as a new block. + */ + void flush() throws IOException; + + /** + * Close the writer and all of its underlying components. + * At the end of close, all the messages sent to the writer should be persisted in a blob. + * flush should be called explicitly before close. + * It is not the responsibility of close to upload blocks. + * After close, no other operations can be performed. + */ + void close() throws IOException; +} diff --git a/samza-azure/src/main/java/org/apache/samza/system/azureblob/producer/AzureBlobWriterFactory.java b/samza-azure/src/main/java/org/apache/samza/system/azureblob/producer/AzureBlobWriterFactory.java new file mode 100644 index 0000000000..87ca5a6a48 --- /dev/null +++ b/samza-azure/src/main/java/org/apache/samza/system/azureblob/producer/AzureBlobWriterFactory.java @@ -0,0 +1,44 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, + * software distributed under the License is distributed on an + * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY + * KIND, either express or implied. See the License for the + * specific language governing permissions and limitations + * under the License. + */ + +package org.apache.samza.system.azureblob.producer; + +import org.apache.samza.system.azureblob.compression.Compression; +import com.azure.storage.blob.BlobContainerAsyncClient; +import java.io.IOException; +import java.util.concurrent.Executor; + + +public interface AzureBlobWriterFactory { + /** + * creates an instance of AzureBlobWriter. + * @param containerAsyncClient Azure container client + * @param blobURL Azure blob url + * @param blobUploadThreadPool thread pool to be used by writer for uploading + * @param metrics metrics to measure the number of bytes written by writer + * @param maxBlockFlushThresholdSize threshold at which to upload + * @param flushTimeoutMs timeout after which the flush is abandoned + * @return AzureBlobWriter instance + * @throws IOException if writer creation fails + */ + AzureBlobWriter getWriterInstance(BlobContainerAsyncClient containerAsyncClient, String blobURL, + Executor blobUploadThreadPool, AzureBlobWriterMetrics metrics, + int maxBlockFlushThresholdSize, long flushTimeoutMs, Compression compression, boolean useRandomStringInBlobName, + long maxBlobSize, long maxMessagesPerBlob) throws IOException; +} diff --git a/samza-azure/src/main/java/org/apache/samza/system/azureblob/producer/AzureBlobWriterMetrics.java b/samza-azure/src/main/java/org/apache/samza/system/azureblob/producer/AzureBlobWriterMetrics.java new file mode 100644 index 0000000000..5605fc6408 --- /dev/null +++ b/samza-azure/src/main/java/org/apache/samza/system/azureblob/producer/AzureBlobWriterMetrics.java @@ -0,0 +1,81 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, + * software distributed under the License is distributed on an + * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY + * KIND, either express or implied. See the License for the + * specific language governing permissions and limitations + * under the License. + */ + +package org.apache.samza.system.azureblob.producer; + +import org.apache.samza.system.azureblob.AzureBlobBasicMetrics; + +/** + * This class holds the basic metrics object of type {@link org.apache.samza.system.azureblob.AzureBlobBasicMetrics} + * for each group to be tracked. + * It maintains 3 groups - aggregate metrics, system-level metrics and source metrics. + * + * This metrics is used by the AzureBlobWriter to measure the number of bytes written by the writer. + */ +public class AzureBlobWriterMetrics { + private final AzureBlobBasicMetrics systemMetrics; + private final AzureBlobBasicMetrics aggregateMetrics; + private final AzureBlobBasicMetrics sourceMetrics; + + public AzureBlobWriterMetrics(AzureBlobBasicMetrics systemMetrics, AzureBlobBasicMetrics aggregateMetrics, + AzureBlobBasicMetrics sourceMetrics) { + this.systemMetrics = systemMetrics; + this.aggregateMetrics = aggregateMetrics; + this.sourceMetrics = sourceMetrics; + } + + /** + * Increments the write byte metrics counters of all the groups by the number of bytes written. + * @param dataLength number of bytes written. + */ + public void updateWriteByteMetrics(long dataLength) { + systemMetrics.updateWriteByteMetrics(dataLength); + aggregateMetrics.updateWriteByteMetrics(dataLength); + sourceMetrics.updateWriteByteMetrics(dataLength); + } + + /** + * Increments the compress byte metrics counters of all the groups by the number of compressed bytes written. + * @param dataLength number of bytes written. + */ + public void updateCompressByteMetrics(long dataLength) { + systemMetrics.updateCompressByteMetrics(dataLength); + aggregateMetrics.updateCompressByteMetrics(dataLength); + sourceMetrics.updateCompressByteMetrics(dataLength); + } + + /** + * Increments the azure block upload metrics counters of all the groups + */ + public void updateAzureUploadMetrics() { + systemMetrics.updateAzureUploadMetrics(); + aggregateMetrics.updateAzureUploadMetrics(); + sourceMetrics.updateAzureUploadMetrics(); + } + + /** + * Increments the azure blob commit metrics counters of all the groups + */ + public void updateAzureCommitMetrics() { + systemMetrics.updateAzureCommitMetrics(); + aggregateMetrics.updateAzureCommitMetrics(); + sourceMetrics.updateAzureCommitMetrics(); + } + +} diff --git a/samza-azure/src/test/java/org/apache/samza/system/azureblob/avro/TestAzureBlobAvroWriter.java b/samza-azure/src/test/java/org/apache/samza/system/azureblob/avro/TestAzureBlobAvroWriter.java new file mode 100644 index 0000000000..c9220cf8d2 --- /dev/null +++ b/samza-azure/src/test/java/org/apache/samza/system/azureblob/avro/TestAzureBlobAvroWriter.java @@ -0,0 +1,452 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, + * software distributed under the License is distributed on an + * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY + * KIND, either express or implied. See the License for the + * specific language governing permissions and limitations + * under the License. + */ + +package org.apache.samza.system.azureblob.avro; + +import com.azure.storage.blob.BlobAsyncClient; +import com.azure.storage.blob.BlobContainerAsyncClient; +import com.azure.storage.blob.specialized.BlockBlobAsyncClient; +import org.apache.samza.system.azureblob.producer.AzureBlobWriterMetrics; +import java.io.ByteArrayOutputStream; +import java.io.IOException; +import java.nio.ByteBuffer; +import java.util.Arrays; +import java.util.List; +import java.util.concurrent.LinkedBlockingDeque; +import java.util.concurrent.ThreadPoolExecutor; +import java.util.concurrent.TimeUnit; +import org.apache.avro.Schema; +import org.apache.avro.file.DataFileWriter; +import org.apache.avro.generic.GenericDatumWriter; +import org.apache.avro.generic.GenericRecord; +import org.apache.avro.generic.IndexedRecord; +import org.apache.avro.io.BinaryEncoder; +import org.apache.avro.io.DatumWriter; +import org.apache.avro.io.EncoderFactory; +import org.apache.avro.specific.SpecificDatumWriter; +import org.apache.avro.specific.SpecificRecord; +import org.apache.samza.SamzaException; +import org.apache.samza.system.OutgoingMessageEnvelope; +import org.apache.samza.system.SystemStream; +import org.junit.Assert; +import org.junit.Before; +import org.junit.Test; +import org.junit.runner.RunWith; +import org.mockito.ArgumentCaptor; +import org.powermock.api.mockito.PowerMockito; +import org.powermock.core.classloader.annotations.PrepareForTest; +import org.powermock.modules.junit4.PowerMockRunner; + +import static org.mockito.Matchers.*; +import static org.mockito.Mockito.*; + + +@RunWith(PowerMockRunner.class) +@PrepareForTest({BlobContainerAsyncClient.class, BlockBlobAsyncClient.class, AzureBlobAvroWriter.class, AzureBlobOutputStream.class}) +public class TestAzureBlobAvroWriter { + private ThreadPoolExecutor threadPool; + private OutgoingMessageEnvelope ome; + private byte[] encodedRecord; + private AzureBlobAvroWriter azureBlobAvroWriter; + private DataFileWriter mockDataFileWriter; + private AzureBlobOutputStream mockAzureBlobOutputStream; + private BlockBlobAsyncClient mockBlockBlobAsyncClient; + + private static final String VALUE = "FAKE_VALUE"; + private static final String SYSTEM_NAME = "FAKE_SYSTEM"; + private static final int THRESHOLD = 100; + + private class SpecificRecordEvent extends org.apache.avro.specific.SpecificRecordBase + implements org.apache.avro.specific.SpecificRecord { + public final org.apache.avro.Schema schema = org.apache.avro.Schema.parse( + "{\"type\":\"record\",\"name\":\"SpecificRecordEvent\",\"namespace\":\"org.apache.samza.events\",\"fields\":[]}"); + + public org.apache.avro.Schema getSchema() { + return schema; + } + + public java.lang.Object get(int field) { + return null; + } + + public void put(int field, Object value) {} + } + + private class GenericRecordEvent implements org.apache.avro.generic.GenericRecord { + public final org.apache.avro.Schema schema = org.apache.avro.Schema.parse( + "{\"type\":\"record\",\"name\":\"GenericRecordEvent\",\"namespace\":\"org.apache.samza.events\",\"fields\":[]}"); + + public org.apache.avro.Schema getSchema() { + return schema; + } + + public java.lang.Object get(String key) { + return null; + } + + public java.lang.Object get(int field) { + return null; + } + + public void put(int field, Object value) {} + public void put(String key, Object value) {} + } + + private OutgoingMessageEnvelope createOME(String streamName) { + SystemStream systemStream = new SystemStream(SYSTEM_NAME, streamName); + SpecificRecord record = new SpecificRecordEvent(); + return new OutgoingMessageEnvelope(systemStream, record); + } + + private OutgoingMessageEnvelope createOMEGenericRecord(String streamName) { + SystemStream systemStream = new SystemStream(SYSTEM_NAME, streamName); + GenericRecord record = new GenericRecordEvent(); + return new OutgoingMessageEnvelope(systemStream, record); + } + + @Before + public void setup() throws Exception { + threadPool = new ThreadPoolExecutor(1, 1, 60, TimeUnit.SECONDS, new LinkedBlockingDeque<>()); + ome = createOME("Topic1"); + + encodedRecord = new byte[100]; + BlobContainerAsyncClient mockContainerAsyncClient = PowerMockito.mock(BlobContainerAsyncClient.class); + mockDataFileWriter = mock(DataFileWriter.class); + mockAzureBlobOutputStream = mock(AzureBlobOutputStream.class); + mockBlockBlobAsyncClient = PowerMockito.mock(BlockBlobAsyncClient.class); + when(mockBlockBlobAsyncClient.getBlobUrl()).thenReturn("https://samza.blob.core.windows.net/fake-blob-url"); + + azureBlobAvroWriter = + spy(new AzureBlobAvroWriter(mockContainerAsyncClient, mock(AzureBlobWriterMetrics.class), threadPool, THRESHOLD, + 60000, "test", mockDataFileWriter, mockAzureBlobOutputStream, mockBlockBlobAsyncClient, Long.MAX_VALUE, + Long.MAX_VALUE)); // keeping blob size and number of records unlimited + doReturn(encodedRecord).when(azureBlobAvroWriter).encodeRecord((IndexedRecord) ome.getMessage()); + } + @Test + public void testWrite() throws Exception { + int numberOfMessages = 10; + for (int i = 0; i < numberOfMessages; ++i) { + azureBlobAvroWriter.write(ome); + } + verify(mockDataFileWriter, times(numberOfMessages)).appendEncoded(ByteBuffer.wrap(encodedRecord)); + } + + @Test + public void testWriteGenericRecord() throws Exception { + OutgoingMessageEnvelope omeGenericRecord = createOMEGenericRecord("Topic1"); + doReturn(encodedRecord).when(azureBlobAvroWriter).encodeRecord((IndexedRecord) omeGenericRecord.getMessage()); + int numberOfMessages = 10; + for (int i = 0; i < numberOfMessages; ++i) { + azureBlobAvroWriter.write(omeGenericRecord); + } + verify(mockDataFileWriter, times(numberOfMessages)).appendEncoded(ByteBuffer.wrap(encodedRecord)); + } + + @Test + public void testWriteByteArray() throws Exception { + OutgoingMessageEnvelope omeEncoded = new OutgoingMessageEnvelope(new SystemStream(SYSTEM_NAME, "Topic1"), "randomString".getBytes()); + int numberOfMessages = 10; + azureBlobAvroWriter.write(ome); + for (int i = 0; i < numberOfMessages; ++i) { + azureBlobAvroWriter.write(omeEncoded); + } + verify(mockDataFileWriter).appendEncoded(ByteBuffer.wrap(encodedRecord)); + verify(mockDataFileWriter, times(numberOfMessages)).appendEncoded(ByteBuffer.wrap((byte[]) omeEncoded.getMessage())); + } + + @Test(expected = IllegalStateException.class) + public void testWriteByteArrayWithoutSchema() throws Exception { + azureBlobAvroWriter = + spy(new AzureBlobAvroWriter(PowerMockito.mock(BlobContainerAsyncClient.class), mock(AzureBlobWriterMetrics.class), + threadPool, THRESHOLD, 60000, "test", + null, null, null, 1000, 100)); + OutgoingMessageEnvelope omeEncoded = new OutgoingMessageEnvelope(new SystemStream(SYSTEM_NAME, "Topic1"), new byte[100]); + azureBlobAvroWriter.write(omeEncoded); + } + + @Test(expected = IOException.class) + public void testWriteWhenDataFileWriterFails() throws Exception { + doThrow(new IOException("Failed")).when(mockDataFileWriter).appendEncoded(ByteBuffer.wrap(encodedRecord)); + azureBlobAvroWriter.write(ome); + } + + @Test + public void testClose() throws Exception { + azureBlobAvroWriter.close(); + verify(mockDataFileWriter).close(); + verify(mockAzureBlobOutputStream).close(); + } + + @Test(expected = SamzaException.class) + public void testCloseWhenDataFileWriterFails() throws Exception { + doThrow(new IOException("Failed")).when(mockDataFileWriter).close(); + + azureBlobAvroWriter.flush(); + azureBlobAvroWriter.close(); + } + + @Test(expected = RuntimeException.class) + public void testCloseWhenOutputStreamFails() throws Exception { + doThrow(new RuntimeException("failed")).when(mockAzureBlobOutputStream).close(); + + azureBlobAvroWriter.close(); + } + + @Test + public void testFlush() throws Exception { + azureBlobAvroWriter.flush(); + verify(mockDataFileWriter).flush(); + } + + @Test(expected = IOException.class) + public void testFlushWhenDataFileWriterFails() throws Exception { + doThrow(new IOException("Failed")).when(mockDataFileWriter).flush(); + azureBlobAvroWriter.flush(); + } + + @Test + public void testMaxBlobSizeExceeded() throws Exception { + String blobUrlPrefix = "test"; + long maxBlobSize = 1000; + BlobContainerAsyncClient mockContainerClient = PowerMockito.mock(BlobContainerAsyncClient.class); + azureBlobAvroWriter = spy(new AzureBlobAvroWriter(mockContainerClient, + mock(AzureBlobWriterMetrics.class), threadPool, THRESHOLD, 60000, blobUrlPrefix, + null, null, null, maxBlobSize, 10)); + + when(mockAzureBlobOutputStream.getSize()).thenReturn((long) maxBlobSize - 1); + PowerMockito.whenNew(AzureBlobOutputStream.class).withAnyArguments().thenReturn(mockAzureBlobOutputStream); + PowerMockito.whenNew(DataFileWriter.class).withAnyArguments().thenReturn(mockDataFileWriter); + BlobAsyncClient mockBlobAsyncClient = mock(BlobAsyncClient.class); + doReturn(mockBlobAsyncClient).when(mockContainerClient).getBlobAsyncClient(anyString()); + doReturn(mockBlockBlobAsyncClient).when(mockBlobAsyncClient).getBlockBlobAsyncClient(); + // first OME creates the first blob + azureBlobAvroWriter.write(ome); + // Second OME creates the second blob because maxBlobSize is 1000 and mockAzureBlobOutputStream.getSize is 999. + azureBlobAvroWriter.write(ome); + + ArgumentCaptor argument = ArgumentCaptor.forClass(String.class); + verify(mockContainerClient, times(2)).getBlobAsyncClient(argument.capture()); + argument.getAllValues().forEach(blobName -> { + Assert.assertTrue(blobName.contains(blobUrlPrefix)); + }); + List allBlobNames = argument.getAllValues(); + Assert.assertNotEquals(allBlobNames.get(0), allBlobNames.get(1)); + } + + @Test + public void testRecordLimitExceeded() throws Exception { + String blobUrlPrefix = "test"; + long maxBlobSize = AzureBlobAvroWriter.DATAFILEWRITER_OVERHEAD + 1000; + long maxRecordsPerBlob = 10; + BlobContainerAsyncClient mockContainerClient = PowerMockito.mock(BlobContainerAsyncClient.class); + azureBlobAvroWriter = spy(new AzureBlobAvroWriter(mockContainerClient, + mock(AzureBlobWriterMetrics.class), threadPool, THRESHOLD, 60000, blobUrlPrefix, + null, null, null, maxBlobSize, maxRecordsPerBlob)); + + when(mockAzureBlobOutputStream.getSize()).thenReturn((long) 1); + PowerMockito.whenNew(AzureBlobOutputStream.class).withAnyArguments().thenReturn(mockAzureBlobOutputStream); + PowerMockito.whenNew(DataFileWriter.class).withAnyArguments().thenReturn(mockDataFileWriter); + BlobAsyncClient mockBlobAsyncClient = mock(BlobAsyncClient.class); + doReturn(mockBlobAsyncClient).when(mockContainerClient).getBlobAsyncClient(anyString()); + doReturn(mockBlockBlobAsyncClient).when(mockBlobAsyncClient).getBlockBlobAsyncClient(); + for (int i = 0; i <= maxRecordsPerBlob; i++) { + azureBlobAvroWriter.write(ome); + } + // first OME creates the first blob and 11th OME creates the second blob. + + ArgumentCaptor argument = ArgumentCaptor.forClass(String.class); + verify(mockContainerClient, times(2)).getBlobAsyncClient(argument.capture()); + argument.getAllValues().forEach(blobName -> { + Assert.assertTrue(blobName.contains(blobUrlPrefix)); + }); + List allBlobNames = argument.getAllValues(); + Assert.assertNotEquals(allBlobNames.get(0), allBlobNames.get(1)); + } + + @Test + public void testMultipleBlobClose() throws Exception { + String blobUrlPrefix = "test"; + long maxBlobSize = AzureBlobAvroWriter.DATAFILEWRITER_OVERHEAD + 1000; + long maxRecordsPerBlob = 10; + BlobContainerAsyncClient mockContainerClient = PowerMockito.mock(BlobContainerAsyncClient.class); + azureBlobAvroWriter = spy(new AzureBlobAvroWriter(mockContainerClient, + mock(AzureBlobWriterMetrics.class), threadPool, THRESHOLD, 60000, blobUrlPrefix, + mockDataFileWriter, mockAzureBlobOutputStream, mockBlockBlobAsyncClient, maxBlobSize, maxRecordsPerBlob)); + + DataFileWriter mockDataFileWriter2 = mock(DataFileWriter.class); + AzureBlobOutputStream mockAzureBlobOutputStream2 = mock(AzureBlobOutputStream.class); + + when(mockAzureBlobOutputStream.getSize()).thenReturn((long) 1); + when(mockAzureBlobOutputStream.getSize()).thenReturn((long) 1); + BlobAsyncClient mockBlobAsyncClient = mock(BlobAsyncClient.class); + doReturn(mockBlobAsyncClient).when(mockContainerClient).getBlobAsyncClient(anyString()); + doReturn(mockBlockBlobAsyncClient).when(mockBlobAsyncClient).getBlockBlobAsyncClient(); + PowerMockito.whenNew(AzureBlobOutputStream.class).withAnyArguments().thenReturn(mockAzureBlobOutputStream2); + PowerMockito.whenNew(DataFileWriter.class).withAnyArguments().thenReturn(mockDataFileWriter2); + for (int i = 0; i <= maxRecordsPerBlob; i++) { + azureBlobAvroWriter.write(ome); + } + // first OME creates the first blob and 11th OME creates the second blob. + + azureBlobAvroWriter.close(); + verify(mockAzureBlobOutputStream).close(); + verify(mockAzureBlobOutputStream2).close(); + verify(mockDataFileWriter).close(); + verify(mockDataFileWriter2).close(); + } + + @Test + public void testEncodeRecord() throws Exception { + azureBlobAvroWriter = spy(new AzureBlobAvroWriter(PowerMockito.mock(BlobContainerAsyncClient.class), + mock(AzureBlobWriterMetrics.class), threadPool, THRESHOLD, + 60000, "test", mockDataFileWriter, mockAzureBlobOutputStream, mockBlockBlobAsyncClient, + Long.MAX_VALUE, Long.MAX_VALUE)); + IndexedRecord record = new GenericRecordEvent(); + Assert.assertTrue(Arrays.equals(encodeRecord(record), azureBlobAvroWriter.encodeRecord(record))); + } + + @Test + public void testMultipleThreadWrites() throws Exception { + Thread t1 = writeInThread(ome, azureBlobAvroWriter, 10); + OutgoingMessageEnvelope ome2 = createOMEGenericRecord("TOPIC2"); + Thread t2 = writeInThread(ome2, azureBlobAvroWriter, 10); + + t1.start(); + t2.start(); + t1.join(60000); + t2.join(60000); + + verify(mockDataFileWriter, times(10)).appendEncoded(ByteBuffer.wrap(encodedRecord)); + verify(mockDataFileWriter, times(10)).appendEncoded(ByteBuffer.wrap(encodeRecord((IndexedRecord) ome2.getMessage()))); + } + + @Test + public void testMultipleThreadWriteFlush() throws Exception { + Thread t1 = writeInThread(ome, azureBlobAvroWriter, 10); + Thread t2 = flushInThread(azureBlobAvroWriter); + + t1.start(); + t2.start(); + t1.join(60000); + t2.join(60000); + + verify(mockDataFileWriter, times(10)).appendEncoded(ByteBuffer.wrap(encodedRecord)); + verify(mockDataFileWriter).flush(); + } + + @Test + public void testMultipleThreadWriteFlushInBoth() throws Exception { + Thread t1 = writeFlushInThread(ome, azureBlobAvroWriter, 10); + OutgoingMessageEnvelope ome2 = createOMEGenericRecord("TOPIC2"); + Thread t2 = writeFlushInThread(ome2, azureBlobAvroWriter, 10); + + t1.start(); + t2.start(); + t1.join(60000); + t2.join(60000); + + verify(mockDataFileWriter, times(10)).appendEncoded(ByteBuffer.wrap(encodedRecord)); + verify(mockDataFileWriter, times(10)).appendEncoded(ByteBuffer.wrap(encodeRecord((IndexedRecord) ome2.getMessage()))); + verify(mockDataFileWriter, times(2)).flush(); + } + + @Test + public void testMultipleThreadWriteFlushFinallyClose() throws Exception { + Thread t1 = writeFlushInThread(ome, azureBlobAvroWriter, 10); + OutgoingMessageEnvelope ome2 = createOMEGenericRecord("TOPIC2"); + Thread t2 = writeFlushInThread(ome2, azureBlobAvroWriter, 10); + + t1.start(); + t2.start(); + t1.join(60000); + t2.join(60000); + azureBlobAvroWriter.close(); + + verify(mockDataFileWriter, times(10)).appendEncoded(ByteBuffer.wrap(encodedRecord)); + verify(mockDataFileWriter, times(10)).appendEncoded(ByteBuffer.wrap(encodeRecord((IndexedRecord) ome2.getMessage()))); + verify(mockDataFileWriter, times(2)).flush(); + verify(mockDataFileWriter).close(); + verify(mockAzureBlobOutputStream).close(); + } + + private byte[] encodeRecord(IndexedRecord record) throws Exception { + ByteArrayOutputStream out = new ByteArrayOutputStream(); + Schema schema = record.getSchema(); + EncoderFactory encoderfactory = new EncoderFactory(); + BinaryEncoder encoder = encoderfactory.binaryEncoder(out, null); + DatumWriter writer; + if (record instanceof SpecificRecord) { + writer = new SpecificDatumWriter<>(schema); + } else { + writer = new GenericDatumWriter<>(schema); + } + writer.write(record, encoder); + encoder.flush(); //encoder may buffer + return out.toByteArray(); + } + + private Thread writeInThread(OutgoingMessageEnvelope ome, AzureBlobAvroWriter azureBlobAvroWriter, + int numberOfSends) { + Thread t = new Thread() { + @Override + public void run() { + try { + for (int i = 0; i < numberOfSends; i++) { + azureBlobAvroWriter.write(ome); + } + } catch (IOException e) { + throw new SamzaException(e); + } + } + }; + return t; + } + + private Thread flushInThread(AzureBlobAvroWriter azureBlobAvroWriter) { + Thread t = new Thread() { + @Override + public void run() { + try { + azureBlobAvroWriter.flush(); + } catch (IOException e) { + throw new SamzaException(e); + } + } + }; + return t; + } + + private Thread writeFlushInThread(OutgoingMessageEnvelope ome, AzureBlobAvroWriter azureBlobAvroWriter, + int numberOfSends) { + Thread t = new Thread() { + @Override + public void run() { + try { + for (int i = 0; i < numberOfSends; i++) { + azureBlobAvroWriter.write(ome); + } + azureBlobAvroWriter.flush(); + } catch (IOException e) { + throw new SamzaException(e); + } + } + }; + return t; + } +} diff --git a/samza-azure/src/test/java/org/apache/samza/system/azureblob/avro/TestAzureBlobOutputStream.java b/samza-azure/src/test/java/org/apache/samza/system/azureblob/avro/TestAzureBlobOutputStream.java new file mode 100644 index 0000000000..20d4aeff9e --- /dev/null +++ b/samza-azure/src/test/java/org/apache/samza/system/azureblob/avro/TestAzureBlobOutputStream.java @@ -0,0 +1,347 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, + * software distributed under the License is distributed on an + * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY + * KIND, either express or implied. See the License for the + * specific language governing permissions and limitations + * under the License. + */ + +package org.apache.samza.system.azureblob.avro; + +import com.azure.core.http.rest.SimpleResponse; +import com.azure.core.implementation.util.FluxUtil; +import org.apache.samza.system.azureblob.compression.Compression; +import org.apache.samza.system.azureblob.producer.AzureBlobWriterMetrics; +import com.azure.storage.blob.specialized.BlockBlobAsyncClient; +import java.io.ByteArrayOutputStream; +import java.io.IOException; +import java.nio.ByteBuffer; +import java.util.ArrayList; +import java.util.Base64; +import java.util.Map; +import java.util.concurrent.LinkedBlockingDeque; +import java.util.concurrent.ThreadPoolExecutor; +import java.util.concurrent.TimeUnit; +import org.junit.Assert; +import org.junit.Before; +import org.junit.Test; +import org.junit.runner.RunWith; +import org.mockito.ArgumentCaptor; +import org.powermock.api.mockito.PowerMockito; +import org.powermock.core.classloader.annotations.PrepareForTest; +import org.powermock.modules.junit4.PowerMockRunner; +import reactor.core.publisher.Flux; +import reactor.core.publisher.Mono; + +import static org.mockito.Mockito.any; +import static org.mockito.Mockito.anyList; +import static org.mockito.Mockito.anyLong; +import static org.mockito.Mockito.anyString; +import static org.mockito.Mockito.doReturn; +import static org.mockito.Mockito.eq; +import static org.mockito.Mockito.mock; +import static org.mockito.Mockito.never; +import static org.mockito.Mockito.spy; +import static org.mockito.Mockito.times; +import static org.mockito.Mockito.verify; +import static org.mockito.Mockito.when; + +@RunWith(PowerMockRunner.class) +@PrepareForTest({BlockBlobAsyncClient.class}) +public class TestAzureBlobOutputStream { + private ThreadPoolExecutor threadPool; + private ByteArrayOutputStream mockByteArrayOutputStream; + private static final int THRESHOLD = 100; + private BlockBlobAsyncClient mockBlobAsyncClient; + private AzureBlobOutputStream azureBlobOutputStream; + private static String randomString = "roZzozzLiR7GCEjcB0UsRUNgBAip8cSLGXQSo3RQvbIDoxOaaRs4hrec2s5rMPWgTPRY4UnE959worEtyhRjwUFnRnVuNFZ554yuPQCbI69qFkQX7MmrB4blmpSnFeGjWKjFjIRLFNVSsQBYMkr5jT4T83uVtuGumsjACVrpcilihdd194H8Y71rQcrXZoTQtw5OvmPicbwptawpHoRNzHihyaDVYgAs0dQbvVEu1gitKpamzYdMLFtc5h8PFZSVEB"; + private static byte[] bytes = randomString.substring(0, THRESHOLD).getBytes(); + private static byte[] compressedBytes = randomString.substring(0, THRESHOLD / 2).getBytes(); + private AzureBlobWriterMetrics mockMetrics; + private Compression mockCompression; + + @Before + public void setup() throws Exception { + threadPool = new ThreadPoolExecutor(1, 1, 60, + TimeUnit.SECONDS, new LinkedBlockingDeque()); + + + mockByteArrayOutputStream = spy(new ByteArrayOutputStream(THRESHOLD)); + + mockBlobAsyncClient = PowerMockito.mock(BlockBlobAsyncClient.class); + when(mockBlobAsyncClient.stageBlock(anyString(), any(), anyLong())).thenReturn( + Mono.just(new SimpleResponse(null, 200, null, null)).flatMap(FluxUtil::toMono)); + when(mockBlobAsyncClient.commitBlockListWithResponse(any(), any(), any(), any(), any())).thenReturn( + Mono.just(new SimpleResponse(null, 200, null, null))); + + when(mockBlobAsyncClient.getBlobUrl()).thenReturn("https://samza.blob.core.windows.net/fake-blob-url"); + + mockMetrics = mock(AzureBlobWriterMetrics.class); + + mockCompression = mock(Compression.class); + doReturn(compressedBytes).when(mockCompression).compress(bytes); + + azureBlobOutputStream = spy(new AzureBlobOutputStream(mockBlobAsyncClient, threadPool, mockMetrics, + 60000, THRESHOLD, mockByteArrayOutputStream, mockCompression)); + } + + @Test + public void testWrite() { + byte[] b = new byte[THRESHOLD - 10]; + azureBlobOutputStream.write(b, 0, THRESHOLD - 10); + verify(mockBlobAsyncClient, never()).stageBlock(any(), any(), anyLong()); // since size of byte[] written is less than threshold + verify(mockMetrics).updateWriteByteMetrics(THRESHOLD - 10); + verify(mockMetrics, never()).updateAzureUploadMetrics(); + } + + @Test + public void testWriteLargerThanThreshold() { + byte[] largeRecord = randomString.substring(0, 2 * THRESHOLD).getBytes(); + byte[] largeRecordFirstHalf = randomString.substring(0, THRESHOLD).getBytes(); + byte[] largeRecordSecondHalf = randomString.substring(THRESHOLD, 2 * THRESHOLD).getBytes(); + + byte[] compressB = randomString.substring(0, THRESHOLD / 2).getBytes(); + + doReturn(compressB).when(mockCompression).compress(largeRecordFirstHalf); + doReturn(compressB).when(mockCompression).compress(largeRecordSecondHalf); + + azureBlobOutputStream.write(largeRecord, 0, 2 * THRESHOLD); + // azureBlobOutputStream.close waits on the CompletableFuture which does the actual stageBlock in uploadBlockAsync + azureBlobOutputStream.close(); + + // invoked 2 times for the data which is 2*threshold + verify(mockCompression).compress(largeRecordFirstHalf); + verify(mockCompression).compress(largeRecordSecondHalf); + ArgumentCaptor argument = ArgumentCaptor.forClass(Flux.class); + verify(mockBlobAsyncClient).stageBlock(eq(blockIdEncoded(0)), argument.capture(), eq((long) compressB.length)); + verify(mockBlobAsyncClient).stageBlock(eq(blockIdEncoded(1)), argument.capture(), eq((long) compressB.length)); + argument.getAllValues().forEach(flux -> { + Assert.assertEquals(ByteBuffer.wrap(compressB), flux.blockFirst()); + }); + verify(mockMetrics).updateWriteByteMetrics(2 * THRESHOLD); + verify(mockMetrics, times(2)).updateAzureUploadMetrics(); + } + + @Test + public void testWriteLargeRecordWithSmallRecordInBuffer() { + byte[] halfBlock = new byte[THRESHOLD / 2]; + byte[] fullBlock = new byte[THRESHOLD]; + byte[] largeRecord = new byte[2 * THRESHOLD]; + byte[] fullBlockCompressedByte = new byte[50]; + byte[] halfBlockCompressedByte = new byte[25]; + doReturn(fullBlockCompressedByte).when(mockCompression).compress(fullBlock); + doReturn(halfBlockCompressedByte).when(mockCompression).compress(halfBlock); + + // FIRST write a small record = same as half block + azureBlobOutputStream.write(halfBlock, 0, THRESHOLD / 2); + verify(mockMetrics).updateWriteByteMetrics(THRESHOLD / 2); + + // SECOND write the large record + azureBlobOutputStream.write(largeRecord, 0, 2 * THRESHOLD); + verify(mockMetrics).updateWriteByteMetrics(2 * THRESHOLD); + + azureBlobOutputStream.flush(); // to flush out buffered data + + // azureBlobOutputStream.close waits on the CompletableFuture which does the actual stageBlock in uploadBlockAsync + azureBlobOutputStream.close(); + + verify(mockCompression, times(2)).compress(fullBlock); + verify(mockCompression).compress(halfBlock); + + ArgumentCaptor argument = ArgumentCaptor.forClass(Flux.class); + ArgumentCaptor argument2 = ArgumentCaptor.forClass(Flux.class); + verify(mockBlobAsyncClient).stageBlock(eq(blockIdEncoded(0)), argument.capture(), eq((long) fullBlockCompressedByte.length)); + verify(mockBlobAsyncClient).stageBlock(eq(blockIdEncoded(1)), argument.capture(), eq((long) fullBlockCompressedByte.length)); + verify(mockBlobAsyncClient).stageBlock(eq(blockIdEncoded(2)), argument2.capture(), eq((long) halfBlockCompressedByte.length)); + argument.getAllValues().forEach(flux -> { + Assert.assertEquals(ByteBuffer.wrap(fullBlockCompressedByte), flux.blockFirst()); + }); + Assert.assertEquals(ByteBuffer.wrap(halfBlockCompressedByte), ((Flux) argument2.getValue()).blockFirst()); + verify(mockMetrics, times(3)).updateAzureUploadMetrics(); + } + + + @Test + public void testWriteThresholdCrossed() throws Exception { + azureBlobOutputStream.write(bytes, 0, THRESHOLD / 2); + azureBlobOutputStream.write(bytes, THRESHOLD / 2, THRESHOLD / 2); + // azureBlobOutputStream.close waits on the CompletableFuture which does the actual stageBlock in uploadBlockAsync + azureBlobOutputStream.close(); + + verify(mockCompression).compress(bytes); + ArgumentCaptor argument = ArgumentCaptor.forClass(Flux.class); + verify(mockBlobAsyncClient).stageBlock(eq(blockIdEncoded(0)), argument.capture(), eq((long) compressedBytes.length)); // since size of byte[] written is less than threshold + Assert.assertEquals(ByteBuffer.wrap(compressedBytes), ((Flux) argument.getValue()).blockFirst()); + verify(mockMetrics, times(2)).updateWriteByteMetrics(THRESHOLD / 2); + verify(mockMetrics, times(1)).updateAzureUploadMetrics(); + } + + @Test(expected = RuntimeException.class) + public void testWriteFailed() { + when(mockBlobAsyncClient.stageBlock(anyString(), any(), anyLong())).thenThrow(new Exception("Test Failed")); + + byte[] b = new byte[100]; + azureBlobOutputStream.write(b, 0, THRESHOLD); // threshold crossed so stageBlock is scheduled. + // azureBlobOutputStream.close waits on the CompletableFuture which does the actual stageBlock in uploadBlockAsync + azureBlobOutputStream.close(); + } + + @Test + public void testClose() { + azureBlobOutputStream.write(bytes, 0, THRESHOLD); + int blockNum = 0; + String blockId = String.format("%05d", blockNum); + String blockIdEncoded = Base64.getEncoder().encodeToString(blockId.getBytes()); + + PowerMockito.doAnswer(invocation -> { + ArrayList blockListArg = (ArrayList) invocation.getArguments()[0]; + String blockIdArg = (String) blockListArg.toArray()[0]; + Assert.assertEquals(blockIdEncoded, blockIdArg); + Map blobMetadata = (Map) invocation.getArguments()[2]; + Assert.assertEquals(blobMetadata.get(AzureBlobOutputStream.BLOB_RAW_SIZE_BYTES_METADATA), Long.toString(THRESHOLD)); + return Mono.just(new SimpleResponse(null, 200, null, null)); + }).when(mockBlobAsyncClient).commitBlockListWithResponse(anyList(), any(), any(), any(), any()); + + azureBlobOutputStream.close(); + verify(mockMetrics).updateAzureCommitMetrics(); + } + + @Test + public void testCloseMultipleBlocks() { + azureBlobOutputStream.write(bytes, 0, THRESHOLD); + azureBlobOutputStream.write(bytes, 0, THRESHOLD); + + int blockNum = 0; + String blockId = String.format("%05d", blockNum); + String blockIdEncoded = Base64.getEncoder().encodeToString(blockId.getBytes()); + + int blockNum1 = 1; + String blockId1 = String.format("%05d", blockNum1); + String blockIdEncoded1 = Base64.getEncoder().encodeToString(blockId1.getBytes()); + + PowerMockito.doAnswer(invocation -> { + ArrayList blockListArg = (ArrayList) invocation.getArguments()[0]; + String blockIdArg = (String) blockListArg.toArray()[0]; + String blockIdArg1 = (String) blockListArg.toArray()[1]; + Assert.assertEquals(blockIdEncoded, blockIdArg); + Assert.assertEquals(blockIdEncoded1, blockIdArg1); + Map blobMetadata = (Map) invocation.getArguments()[2]; + Assert.assertEquals(blobMetadata.get(AzureBlobOutputStream.BLOB_RAW_SIZE_BYTES_METADATA), Long.toString(2 * THRESHOLD)); + return Mono.just(new SimpleResponse(null, 200, null, null)); + }).when(mockBlobAsyncClient).commitBlockListWithResponse(anyList(), any(), any(), any(), any()); + + azureBlobOutputStream.close(); + verify(mockMetrics).updateAzureCommitMetrics(); + } + + @Test(expected = RuntimeException.class) + public void testCloseFailed() { + when(mockBlobAsyncClient.commitBlockListWithResponse(anyList(), any(), any(), any(), any())) + .thenReturn(Mono.error(new Exception("Test Failed"))); + + byte[] b = new byte[100]; + azureBlobOutputStream.write(b, 0, THRESHOLD); + azureBlobOutputStream.close(); + } + + @Test + public void testMultipleClose() { + azureBlobOutputStream.write(bytes, 0, THRESHOLD); + azureBlobOutputStream.close(); + azureBlobOutputStream.close(); + } + + @Test + public void testFlush() throws Exception { + azureBlobOutputStream.write(bytes); + azureBlobOutputStream.flush(); + // azureBlobOutputStream.close waits on the CompletableFuture which does the actual stageBlock in uploadBlockAsync + azureBlobOutputStream.close(); + + int blockNum = 0; // as there is only one block and its id will be 0 + String blockId = String.format("%05d", blockNum); + String blockIdEncoded = Base64.getEncoder().encodeToString(blockId.getBytes()); + + verify(mockCompression).compress(bytes); + ArgumentCaptor argument = ArgumentCaptor.forClass(Flux.class); + verify(mockBlobAsyncClient).stageBlock(eq(blockIdEncoded), argument.capture(), eq((long) compressedBytes.length)); // since size of byte[] written is less than threshold + Assert.assertEquals(ByteBuffer.wrap(compressedBytes), ((Flux) argument.getValue()).blockFirst()); + verify(mockMetrics).updateAzureUploadMetrics(); + } + + @Test (expected = RuntimeException.class) + public void testFlushFailed() throws IOException { + azureBlobOutputStream.write(bytes); + when(mockBlobAsyncClient.stageBlock(anyString(), any(), anyLong())) + .thenReturn(Mono.error(new Exception("Test Failed"))); + + azureBlobOutputStream.flush(); + // azureBlobOutputStream.close waits on the CompletableFuture which does the actual stageBlock in uploadBlockAsync + azureBlobOutputStream.close(); + verify(mockMetrics).updateAzureUploadMetrics(); + } + + + @Test + public void testReleaseBuffer() throws Exception { + azureBlobOutputStream.releaseBuffer(); + verify(mockByteArrayOutputStream).close(); + } + + @Test(expected = IllegalStateException.class) + public void testWriteAfterReleaseBuffer() throws Exception { + azureBlobOutputStream.releaseBuffer(); + azureBlobOutputStream.write(new byte[10], 0, 10); + } + + @Test + public void testCloseAfterReleaseBuffer() throws Exception { + azureBlobOutputStream.write(bytes, 0, 100); + azureBlobOutputStream.releaseBuffer(); + azureBlobOutputStream.close(); + // mockByteArrayOutputStream.close called only once during releaseBuffer and not during azureBlobOutputStream.close + verify(mockByteArrayOutputStream).close(); + // azureBlobOutputStream.close still commits the list of blocks. + verify(mockBlobAsyncClient).commitBlockListWithResponse(any(), any(), any(), any(), any()); + } + + @Test + public void testFlushAfterReleaseBuffer() throws Exception { + azureBlobOutputStream.releaseBuffer(); + azureBlobOutputStream.flush(); // becomes no-op after release buffer + verify(mockBlobAsyncClient, never()).stageBlock(anyString(), any(), anyLong()); + } + + @Test + public void testGetSize() throws Exception { + Assert.assertEquals(0, azureBlobOutputStream.getSize()); + azureBlobOutputStream.write(bytes, 0, bytes.length); + Assert.assertEquals(bytes.length, azureBlobOutputStream.getSize()); + } + + @Test + public void testGetSizeAfterFlush() throws Exception { + azureBlobOutputStream.write(bytes, 0, bytes.length); + Assert.assertEquals(bytes.length, azureBlobOutputStream.getSize()); + azureBlobOutputStream.flush(); + Assert.assertEquals(bytes.length, azureBlobOutputStream.getSize()); + azureBlobOutputStream.write(bytes, 0, bytes.length - 10); + Assert.assertEquals(bytes.length + bytes.length - 10, azureBlobOutputStream.getSize()); + } + + private String blockIdEncoded(int blockNum) { + String blockId = String.format("%05d", blockNum); + return Base64.getEncoder().encodeToString(blockId.getBytes()); + } +} diff --git a/samza-azure/src/test/java/org/apache/samza/system/azureblob/compression/TestGzipCompression.java b/samza-azure/src/test/java/org/apache/samza/system/azureblob/compression/TestGzipCompression.java new file mode 100644 index 0000000000..e0cfc5320b --- /dev/null +++ b/samza-azure/src/test/java/org/apache/samza/system/azureblob/compression/TestGzipCompression.java @@ -0,0 +1,76 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, + * software distributed under the License is distributed on an + * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY + * KIND, either express or implied. See the License for the + * specific language governing permissions and limitations + * under the License. + */ + +package org.apache.samza.system.azureblob.compression; + +import java.io.ByteArrayOutputStream; +import java.io.IOException; +import java.util.zip.GZIPOutputStream; +import org.junit.Before; +import org.junit.Test; +import org.junit.Assert; + +public class TestGzipCompression { + private GzipCompression gzipCompression; + + @Before + public void setup() { + gzipCompression = new GzipCompression(); + } + + @Test + public void testCompression() throws IOException { + byte[] input = "This is fake input data".getBytes(); + byte[] result = compress(input); + + Assert.assertArrayEquals(gzipCompression.compress(input), result); + } + + @Test + public void testCompressionEmpty() throws IOException { + byte[] input = "".getBytes(); + byte[] result = compress(input); + + Assert.assertArrayEquals(gzipCompression.compress(input), result); + } + + @Test(expected = RuntimeException.class) + public void testCompressionNull() { + byte[] input = null; + gzipCompression.compress(input); + } + + @Test + public void testCompressionZero() throws IOException { + byte[] input = new byte[100]; + byte[] result = compress(input); + + Assert.assertArrayEquals(gzipCompression.compress(input), result); + } + + private byte[] compress(byte[] input) throws IOException { + ByteArrayOutputStream bos = new ByteArrayOutputStream(input.length); + GZIPOutputStream gzipOS = new GZIPOutputStream(bos); + gzipOS.write(input); + gzipOS.close(); + gzipOS.close(); + bos.close(); + return bos.toByteArray(); + } +} diff --git a/samza-azure/src/test/java/org/apache/samza/system/azureblob/producer/TestAzureBlobSystemProducer.java b/samza-azure/src/test/java/org/apache/samza/system/azureblob/producer/TestAzureBlobSystemProducer.java new file mode 100644 index 0000000000..e3e8d542ea --- /dev/null +++ b/samza-azure/src/test/java/org/apache/samza/system/azureblob/producer/TestAzureBlobSystemProducer.java @@ -0,0 +1,600 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, + * software distributed under the License is distributed on an + * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY + * KIND, either express or implied. See the License for the + * specific language governing permissions and limitations + * under the License. + */ + +package org.apache.samza.system.azureblob.producer; + +import org.apache.samza.system.azureblob.AzureBlobBasicMetrics; +import org.apache.samza.system.azureblob.AzureBlobConfig; +import org.apache.samza.system.azureblob.avro.AzureBlobAvroWriter; +import java.io.IOException; +import java.util.HashMap; +import java.util.Map; +import java.util.concurrent.LinkedBlockingDeque; +import java.util.concurrent.ThreadPoolExecutor; +import java.util.concurrent.TimeUnit; +import org.apache.samza.config.Config; +import org.apache.samza.config.MapConfig; +import org.apache.samza.metrics.Counter; +import org.apache.samza.metrics.MetricsRegistry; +import org.apache.samza.system.OutgoingMessageEnvelope; +import org.apache.samza.system.SystemProducerException; +import org.apache.samza.system.SystemStream; +import org.junit.Assert; +import org.junit.Before; +import org.junit.Test; +import org.junit.runner.RunWith; +import org.powermock.api.mockito.PowerMockito; +import org.powermock.core.classloader.annotations.PrepareForTest; +import org.powermock.modules.junit4.PowerMockRunner; + +import static org.mockito.Mockito.any; +import static org.mockito.Mockito.anyString; +import static org.mockito.Mockito.doNothing; +import static org.mockito.Mockito.doReturn; +import static org.mockito.Mockito.doThrow; +import static org.mockito.Mockito.mock; +import static org.mockito.Mockito.spy; +import static org.mockito.Mockito.times; +import static org.mockito.Mockito.verify; +import static org.mockito.Mockito.when; + +@RunWith(PowerMockRunner.class) +@PrepareForTest({AzureBlobSystemProducer.class, ThreadPoolExecutor.class}) +public class TestAzureBlobSystemProducer { + + private final static String SYSTEM_NAME = "FAKE_SYSTEM"; + private final static String SOURCE = "FAKE_SOURCE"; + private final static String STREAM = "FAKE_STREAM"; + private final static String ACCOUNT_NAME = "FAKE_ACCOUNT_NAME"; + private final static String ACCOUNT_KEY = "FAKE_ACCOUNT_KEY"; + + private OutgoingMessageEnvelope ome; + private AzureBlobSystemProducer systemProducer; + private MetricsRegistry mockMetricsRegistry; + private AzureBlobWriter mockAzureWriter; + private ThreadPoolExecutor mockThreadPoolExecutor; + private Counter mockErrorCounter; + private boolean exceptionOccured = false; + + @Before + public void setup() throws Exception { + + AzureBlobConfig azureBlobConfig = new AzureBlobConfig(getBasicConfigs()); + + mockMetricsRegistry = mock(MetricsRegistry.class); + mockErrorCounter = mock(Counter.class); + + when(mockMetricsRegistry.newCounter(anyString(), anyString())).thenReturn(mock(Counter.class)); + when(mockMetricsRegistry.newCounter(SOURCE, AzureBlobBasicMetrics.EVENT_PRODUCE_ERROR)).thenReturn(mockErrorCounter); + + ome = createOME(STREAM); + + mockThreadPoolExecutor = spy(new ThreadPoolExecutor(1, 1, 60, TimeUnit.SECONDS, + new LinkedBlockingDeque())); + + PowerMockito.whenNew(ThreadPoolExecutor.class).withAnyArguments().thenReturn(mockThreadPoolExecutor); + + mockAzureWriter = mock(AzureBlobWriter.class); + doNothing().when(mockAzureWriter).close(); + + systemProducer = spy(new AzureBlobSystemProducer(SYSTEM_NAME, azureBlobConfig, mockMetricsRegistry)); + // use mock writer impl + doReturn(mockAzureWriter).when(systemProducer).createNewWriter(anyString(), any()); + // bypass Azure connection setup + doNothing().when(systemProducer).setupAzureContainer(anyString(), anyString()); + } + + @Test + public void testStart() { + + systemProducer.start(); + verify(systemProducer).setupAzureContainer(ACCOUNT_NAME, ACCOUNT_KEY); + } + + @Test(expected = SystemProducerException.class) + public void testMultipleStart() { + systemProducer.start(); + systemProducer.start(); + } + + @Test + public void testStop() throws Exception { + doNothing().when(mockAzureWriter).close(); + + systemProducer.register(SOURCE); + systemProducer.start(); + systemProducer.send(SOURCE, ome); + systemProducer.flush(SOURCE); + systemProducer.stop(); + + verify(mockAzureWriter).flush(); // called during flush IN STOP + verify(mockAzureWriter).close(); // called during flush in STOP + } + + @Test + public void testStopBeforeFlush() throws Exception { + + systemProducer.register(SOURCE); + systemProducer.start(); + systemProducer.send(SOURCE, ome); + systemProducer.stop(); + + verify(mockAzureWriter).flush(); // called during flush IN STOP + verify(mockAzureWriter).close(); // called during flush in STOP + } + + @Test(expected = SystemProducerException.class) + public void testStopBeforeStart() { + systemProducer.stop(); + } + + @Test(expected = SystemProducerException.class) + public void testStopWhenThreadpoolShutdownFails() throws Exception { + doThrow(new SecurityException("failed")).when(mockThreadPoolExecutor).shutdown(); + systemProducer.start(); + systemProducer.stop(); + } + + @Test (expected = SystemProducerException.class) + public void testStopWhenWriterFails() throws IOException { + doThrow(new SystemProducerException("Failed")).when(mockAzureWriter).flush(); + systemProducer.register(SOURCE); + systemProducer.start(); + systemProducer.send(SOURCE, ome); + systemProducer.stop(); + } + + @Test(expected = SystemProducerException.class) + public void testRegisterAfterStart() throws Exception { + systemProducer.start(); + systemProducer.register(SOURCE); + } + + @Test + public void testRegisterMetrics() throws Exception { + systemProducer.register(SOURCE); + + // verify that new counter for system was created during constructor of producer + verify(mockMetricsRegistry).newCounter( + String.format(AzureBlobSystemProducerMetrics.SYSTEM_METRIC_FORMAT, ACCOUNT_NAME, SYSTEM_NAME), + AzureBlobBasicMetrics.EVENT_WRITE_RATE); + // verify that new counter for source was created during register + verify(mockMetricsRegistry).newCounter(SOURCE, AzureBlobBasicMetrics.EVENT_WRITE_RATE); + verify(mockMetricsRegistry).newCounter(SOURCE, AzureBlobBasicMetrics.EVENT_WRITE_BYTE_RATE); + verify(mockMetricsRegistry).newCounter(SOURCE, AzureBlobBasicMetrics.EVENT_PRODUCE_ERROR); + } + + @Test + public void testRegisterWithSystemName() throws Exception { + systemProducer.register(SYSTEM_NAME); + + // verify that new counter for system was created during constructor of producer but not during register + verify(mockMetricsRegistry).newCounter( + String.format(AzureBlobSystemProducerMetrics.SYSTEM_METRIC_FORMAT, ACCOUNT_NAME, SYSTEM_NAME), + AzureBlobBasicMetrics.EVENT_WRITE_RATE); + } + + @Test + public void testFlush() throws IOException { + + systemProducer.register(SOURCE); + systemProducer.start(); + systemProducer.send(SOURCE, ome); + systemProducer.flush(SOURCE); + + verify(mockAzureWriter).flush(); // called during flush + verify(mockAzureWriter).close(); // called during flush + } + + @Test(expected = SystemProducerException.class) + public void testFlushBeforeStart() throws Exception { + systemProducer.flush(SOURCE); + } + + @Test(expected = SystemProducerException.class) + public void testFlushBeforeRegister() throws Exception { + systemProducer.start(); + systemProducer.flush(SOURCE); + } + + @Test + public void testFlushWhenWriterUploadFails() throws Exception { + doThrow(new SystemProducerException("failed")).when(mockAzureWriter).flush(); + + systemProducer.register(SOURCE); + systemProducer.start(); + systemProducer.send(SOURCE, ome); + try { + systemProducer.flush(SOURCE); + Assert.fail("Expected exception not thrown."); + } catch (SystemProducerException e) { + } + + verify(mockErrorCounter).inc(); + } + + @Test + public void testFlushWhenWriterCloseFails() throws Exception { + doThrow(new SystemProducerException("failed")).when(mockAzureWriter).close(); + + systemProducer.register(SOURCE); + systemProducer.start(); + systemProducer.send(SOURCE, ome); + try { + systemProducer.flush(SOURCE); + Assert.fail("Expected exception not thrown."); + } catch (SystemProducerException e) { + } + verify(mockErrorCounter).inc(); + } + + @Test + public void testSend() throws IOException { + int numberOfMessages = 10; + Counter mockWriteCounter = mock(Counter.class); + when(mockMetricsRegistry.newCounter(SOURCE, AzureBlobBasicMetrics.EVENT_WRITE_RATE)).thenReturn(mockWriteCounter); + + systemProducer.register(SOURCE); + systemProducer.start(); + for (int i = 0; i < numberOfMessages; i++) { + systemProducer.send(SOURCE, ome); + } + verify(mockAzureWriter, times(numberOfMessages)).write(ome); + + // verify metrics + verify(mockWriteCounter, times(numberOfMessages)).inc(); + } + + @Test + public void testSendWhenWriterCreateFails() throws Exception { + AzureBlobConfig azureBlobConfig = new AzureBlobConfig(getBasicConfigs()); + + AzureBlobSystemProducer systemProducer = spy(new AzureBlobSystemProducer(SYSTEM_NAME, azureBlobConfig, + mockMetricsRegistry)); + PowerMockito.whenNew(AzureBlobAvroWriter.class).withAnyArguments().thenThrow(new SystemProducerException("Failed")); + // bypass Azure connection setup + doNothing().when(systemProducer).setupAzureContainer(anyString(), anyString()); + + systemProducer.register(SOURCE); + systemProducer.start(); + try { + systemProducer.send(SOURCE, ome); + Assert.fail("Expected exception not thrown."); + } catch (SystemProducerException e) { + } + verify(mockErrorCounter).inc(); + } + + @Test + public void testSendWhenWriterFails() throws Exception { + + doThrow(new SystemProducerException("failed")).when(mockAzureWriter).write(ome); + + systemProducer.register(SOURCE); + systemProducer.start(); + try { + systemProducer.send(SOURCE, ome); + Assert.fail("Expected exception not thrown."); + } catch (SystemProducerException e) { + } + verify(mockErrorCounter).inc(); + } + + @Test + public void testMutipleThread() throws Exception { + String source1 = "FAKE_SOURCE_1"; + String source2 = "FAKE_SOURCE_2"; + + String stream1 = "FAKE_STREAM_1"; + String stream2 = "FAKE_STREAM_2"; + + int sendsInFirstThread = 10; + int sendsInSecondThread = 20; + + OutgoingMessageEnvelope ome1 = createOME(stream1); + OutgoingMessageEnvelope ome2 = createAnotherOME(stream2); + + AzureBlobWriter mockAzureWriter1 = mock(AzureBlobWriter.class); + doNothing().when(mockAzureWriter1).close(); + + AzureBlobWriter mockAzureWriter2 = mock(AzureBlobWriter.class); + doNothing().when(mockAzureWriter2).close(); + + AzureBlobConfig azureBlobConfig = new AzureBlobConfig(getBasicConfigs()); + + AzureBlobSystemProducer systemProducer = spy(new AzureBlobSystemProducer(SYSTEM_NAME, azureBlobConfig, mockMetricsRegistry)); + // bypass Azure connection setup + doNothing().when(systemProducer).setupAzureContainer(anyString(), anyString()); + + doReturn(mockAzureWriter1).when(systemProducer).getOrCreateWriter(source1, ome1); + doReturn(mockAzureWriter2).when(systemProducer).getOrCreateWriter(source2, ome2); + + systemProducer.register(source1); + systemProducer.register(source2); + systemProducer.start(); + Thread t1 = sendFlushInThread(source1, ome1, systemProducer, sendsInFirstThread); + Thread t2 = sendFlushInThread(source2, ome2, systemProducer, sendsInSecondThread); + t1.start(); + t2.start(); + t1.join(60000); + t2.join(60000); + systemProducer.stop(); + verify(mockAzureWriter1, times(sendsInFirstThread)).write(ome1); + verify(mockAzureWriter2, times(sendsInSecondThread)).write(ome2); + } + + @Test + public void testMutipleThreadOneWriterFails() throws Exception { + String source1 = "FAKE_SOURCE_1"; + String source2 = "FAKE_SOURCE_2"; + + String stream1 = "FAKE_STREAM_1"; + String stream2 = "FAKE_STREAM_2"; + + int sendsInFirstThread = 10; + int sendsInSecondThread = 20; + + OutgoingMessageEnvelope ome1 = createOME(stream1); + OutgoingMessageEnvelope ome2 = createAnotherOME(stream2); + + AzureBlobWriter mockAzureWriter1 = mock(AzureBlobWriter.class); + doThrow(new SystemProducerException("failed")).when(mockAzureWriter1).write(ome1); + doNothing().when(mockAzureWriter1).close(); + + AzureBlobWriter mockAzureWriter2 = mock(AzureBlobWriter.class); + doNothing().when(mockAzureWriter2).close(); + + AzureBlobConfig azureBlobConfig = new AzureBlobConfig(getBasicConfigs()); + + AzureBlobSystemProducer systemProducer = spy(new AzureBlobSystemProducer(SYSTEM_NAME, azureBlobConfig, mockMetricsRegistry)); + // bypass Azure connection setup + doNothing().when(systemProducer).setupAzureContainer(anyString(), anyString()); + + doReturn(mockAzureWriter1).when(systemProducer).getOrCreateWriter(source1, ome1); + doReturn(mockAzureWriter2).when(systemProducer).getOrCreateWriter(source2, ome2); + + + systemProducer.register(source1); + systemProducer.register(source2); + systemProducer.start(); + Thread t1 = sendFlushInThread(source1, ome1, systemProducer, sendsInFirstThread); + Thread t2 = sendFlushInThread(source2, ome2, systemProducer, sendsInSecondThread); + + Thread.UncaughtExceptionHandler handler = new Thread.UncaughtExceptionHandler() { + public void uncaughtException(Thread th, Throwable ex) { + if (ex instanceof SystemProducerException) { + exceptionOccured = true; + } + } + }; + t1.setUncaughtExceptionHandler(handler); + t1.start(); + t2.start(); + t1.join(60000); + t2.join(60000); + + systemProducer.stop(); + + if (!exceptionOccured) { + Assert.fail("Expected SystemProducerException but did not occur."); + } + verify(mockAzureWriter1).write(ome1); + verify(mockAzureWriter2, times(sendsInSecondThread)).write(ome2); + } + + @Test + public void testMutipleThreadSendFlushToSingleWriter() throws Exception { + String source1 = "FAKE_SOURCE_1"; + + String stream1 = "FAKE_STREAM_1"; + + int sendsInFirstThread = 10; + int sendsInSecondThread = 20; + + OutgoingMessageEnvelope ome1 = createOME(stream1); + + AzureBlobWriter mockAzureWriter1 = mock(AzureBlobWriter.class); + doNothing().when(mockAzureWriter1).close(); + + AzureBlobConfig azureBlobConfig = new AzureBlobConfig(getBasicConfigs()); + + AzureBlobSystemProducer systemProducer = spy(new AzureBlobSystemProducer(SYSTEM_NAME, azureBlobConfig, mockMetricsRegistry)); + // bypass Azure connection setup + doNothing().when(systemProducer).setupAzureContainer(anyString(), anyString()); + + systemProducer.register(source1); + systemProducer.start(); + + doReturn(mockAzureWriter1).when(systemProducer).createNewWriter(anyString(), any()); + + Thread t1 = sendFlushInThread(source1, ome1, systemProducer, sendsInFirstThread); + Thread t2 = sendFlushInThread(source1, ome1, systemProducer, sendsInSecondThread); + + t1.start(); + t2.start(); + t1.join(60000); + t2.join(60000); + systemProducer.stop(); + verify(mockAzureWriter1, times(sendsInFirstThread + sendsInSecondThread)).write(ome1); + verify(mockAzureWriter1, times(2)).flush(); + verify(mockAzureWriter1, times(2)).close(); + } + + @Test + public void testMutipleThreadSendToSingleWriter() throws Exception { + String source1 = "FAKE_SOURCE_1"; + + String stream1 = "FAKE_STREAM_1"; + + int sendsInFirstThread = 10; + int sendsInSecondThread = 20; + + OutgoingMessageEnvelope ome1 = createOME(stream1); + OutgoingMessageEnvelope ome2 = createAnotherOME(stream1); + + AzureBlobWriter mockAzureWriter1 = mock(AzureBlobWriter.class); + doNothing().when(mockAzureWriter1).close(); + + AzureBlobConfig azureBlobConfig = new AzureBlobConfig(getBasicConfigs()); + + AzureBlobSystemProducer systemProducer = spy(new AzureBlobSystemProducer(SYSTEM_NAME, azureBlobConfig, mockMetricsRegistry)); + // bypass Azure connection setup + doNothing().when(systemProducer).setupAzureContainer(anyString(), anyString()); + + doReturn(mockAzureWriter1).when(systemProducer).createNewWriter(anyString(), any()); + + systemProducer.register(source1); + systemProducer.start(); + Thread t1 = new Thread() { + @Override + public void run() { + for (int i = 0; i < sendsInFirstThread; i++) { + systemProducer.send(source1, ome1); + } + } + }; + Thread t2 = new Thread() { + @Override + public void run() { + for (int i = 0; i < sendsInSecondThread; i++) { + systemProducer.send(source1, ome2); + } + } + }; + t1.start(); + t2.start(); + t1.join(60000); + t2.join(60000); + systemProducer.stop(); + verify(mockAzureWriter1, times(sendsInFirstThread)).write(ome1); + verify(mockAzureWriter1, times(sendsInSecondThread)).write(ome2); + } + + @Test + public void testMutipleThreadFlushToSingleWriter() throws Exception { + String source1 = "FAKE_SOURCE_1"; + + AzureBlobWriter mockAzureWriter1 = mock(AzureBlobWriter.class); + doNothing().when(mockAzureWriter1).close(); + + AzureBlobConfig azureBlobConfig = new AzureBlobConfig(getBasicConfigs()); + + AzureBlobSystemProducer systemProducer = spy(new AzureBlobSystemProducer(SYSTEM_NAME, azureBlobConfig, mockMetricsRegistry)); + // bypass Azure connection setup + doNothing().when(systemProducer).setupAzureContainer(anyString(), anyString()); + + doReturn(mockAzureWriter1).when(systemProducer).createNewWriter(anyString(), any()); + + systemProducer.register(source1); + systemProducer.start(); + systemProducer.send(source1, ome); //to create writer + Thread t1 = new Thread() { + @Override + public void run() { + systemProducer.flush(source1); + } + }; + Thread t2 = new Thread() { + @Override + public void run() { + systemProducer.flush(source1); + } + }; + t1.start(); + t2.start(); + t1.join(60000); + t2.join(60000); + systemProducer.stop(); + // systemProducer.flush called twice but first flush clears the writer map of the source. + // hence, writer.flush and close called only once. + verify(mockAzureWriter1).flush(); + verify(mockAzureWriter1).close(); + } + + private Thread sendFlushInThread(String source, OutgoingMessageEnvelope ome, AzureBlobSystemProducer systemProducer, + int numberOfSends) { + Thread t = new Thread() { + @Override + public void run() { + try { + for (int i = 0; i < numberOfSends; i++) { + systemProducer.send(source, ome); + } + systemProducer.flush(source); + } catch (Exception e) { + throw e; + } + } + }; + return t; + } + + private OutgoingMessageEnvelope createOME(String streamName) { + SystemStream systemStream = new SystemStream(SYSTEM_NAME, streamName); + DummyPageViewEvent record = new DummyPageViewEvent(); + return new OutgoingMessageEnvelope(systemStream, record); + } + + private OutgoingMessageEnvelope createAnotherOME(String streamName) { + SystemStream systemStream = new SystemStream(SYSTEM_NAME, streamName); + AnotherDummyPageViewEvent record = new AnotherDummyPageViewEvent(); + return new OutgoingMessageEnvelope(systemStream, record); + } + + private class DummyPageViewEvent extends org.apache.avro.specific.SpecificRecordBase + implements org.apache.avro.specific.SpecificRecord { + public final org.apache.avro.Schema schema = org.apache.avro.Schema.parse( + "{\"type\":\"record\",\"name\":\"DummyPageViewEvent\",\"namespace\":\"org.apache.samza.events\",\"fields\":[]}"); + + public org.apache.avro.Schema getSchema() { + return schema; + } + + public java.lang.Object get(int field) { + return null; + } + + public void put(int field, Object value) {} + } + + private class AnotherDummyPageViewEvent extends org.apache.avro.specific.SpecificRecordBase + implements org.apache.avro.specific.SpecificRecord { + public final org.apache.avro.Schema schema = org.apache.avro.Schema.parse( + "{\"type\":\"record\",\"name\":\"AnotherDummyPageViewEvent\",\"namespace\":\"org.apache.samza.events\",\"fields\":[]}"); + + public org.apache.avro.Schema getSchema() { + return schema; + } + + public java.lang.Object get(int field) { + return null; + } + + public void put(int field, Object value) {} + } + + private Config getBasicConfigs() { + Map bareConfigs = new HashMap<>(); + bareConfigs.put(String.format(AzureBlobConfig.SYSTEM_AZURE_ACCOUNT_NAME, SYSTEM_NAME), ACCOUNT_NAME); + bareConfigs.put(String.format(AzureBlobConfig.SYSTEM_AZURE_ACCOUNT_KEY, SYSTEM_NAME), ACCOUNT_KEY); + bareConfigs.put(String.format(AzureBlobConfig.SYSTEM_CLOSE_TIMEOUT_MS, SYSTEM_NAME), "1000"); + Config config = new MapConfig(bareConfigs); + return config; + } +} \ No newline at end of file From 11e8087f29357050b4ca67a61267c341b976019c Mon Sep 17 00:00:00 2001 From: Manasa Date: Tue, 24 Dec 2019 10:58:44 -0800 Subject: [PATCH 2/9] fix comments --- .../system/azureblob/AzureBlobConfig.java | 11 ++--- .../azureblob/AzureBlobSystemAdmin.java | 7 ++- .../azureblob/avro/AzureBlobAvroWriter.java | 7 ++- .../azureblob/avro/AzureBlobOutputStream.java | 48 +++++++++---------- .../compression/CompressionFactory.java | 10 ++++ .../producer/AzureBlobSystemProducer.java | 10 ++-- .../avro/TestAzureBlobAvroWriter.java | 17 ++++--- .../avro/TestAzureBlobOutputStream.java | 6 +-- 8 files changed, 63 insertions(+), 53 deletions(-) diff --git a/samza-azure/src/main/java/org/apache/samza/system/azureblob/AzureBlobConfig.java b/samza-azure/src/main/java/org/apache/samza/system/azureblob/AzureBlobConfig.java index facd83ac7f..7c468c6e24 100644 --- a/samza-azure/src/main/java/org/apache/samza/system/azureblob/AzureBlobConfig.java +++ b/samza-azure/src/main/java/org/apache/samza/system/azureblob/AzureBlobConfig.java @@ -35,7 +35,7 @@ public class AzureBlobConfig extends MapConfig { // system Level Properties. // fully qualified class name of the AzureBlobWriter impl for the producer system - public static final String SYSTEM_WRITER_FACTORY_CLASS_NAME = "systems.%s.producer.azureblob.writer.factory.class"; + public static final String SYSTEM_WRITER_FACTORY_CLASS_NAME = SYSTEM_AZUREBLOB_PREFIX + "writer.factory.class"; public static final String SYSTEM_WRITER_FACTORY_CLASS_NAME_DEFAULT = "org.apache.samza.system.azureblob.avro.AzureBlobAvroWriterFactory"; // Azure Storage Account name under which the Azure container representing this system is. @@ -73,7 +73,7 @@ public class AzureBlobConfig extends MapConfig { // number of threads to asynchronously upload blocks public static final String SYSTEM_THREAD_POOL_COUNT = SYSTEM_AZUREBLOB_PREFIX + "threadPoolCount"; - private static final int SYSTEM_THREAD_POOL_COUNT_DEFAULT = 2; + private static final int SYSTEM_THREAD_POOL_COUNT_DEFAULT = 1; // size of the queue to hold blocks ready to be uploaded by asynchronous threads. // If all threads are busy uploading then blocks are queued and if queue is full then main thread will start uploading @@ -91,9 +91,8 @@ public class AzureBlobConfig extends MapConfig { // if true, a random string of 8 chars is suffixed to the blob name to prevent name collision // when more than one Samza tasks are writing to the same SSP. - // It is advisable to set this to true public static final String SYSTEM_SUFFIX_RANDOM_STRING_TO_BLOB_NAME = SYSTEM_AZUREBLOB_PREFIX + "suffixRandomStringToBlobName"; - private static final boolean SYSTEM_SUFFIX_RANDOM_STRING_TO_BLOB_NAME_DEFAULT = false; + private static final boolean SYSTEM_SUFFIX_RANDOM_STRING_TO_BLOB_NAME_DEFAULT = true; public AzureBlobConfig(Config config) { super(config); @@ -147,8 +146,8 @@ public int getAzureBlobThreadPoolCount(String systemName) { return getInt(String.format(SYSTEM_THREAD_POOL_COUNT, systemName), SYSTEM_THREAD_POOL_COUNT_DEFAULT); } - public int getBlockingQueueSizeOrDefault(String systemName, int defaultQueueSize) { - return getInt(String.format(SYSTEM_BLOCKING_QUEUE_SIZE, systemName), defaultQueueSize); + public int getBlockingQueueSize(String systemName) { + return getInt(String.format(SYSTEM_BLOCKING_QUEUE_SIZE, systemName), 2 * getAzureBlobThreadPoolCount(systemName)); } public long getFlushTimeoutMs(String systemName) { diff --git a/samza-azure/src/main/java/org/apache/samza/system/azureblob/AzureBlobSystemAdmin.java b/samza-azure/src/main/java/org/apache/samza/system/azureblob/AzureBlobSystemAdmin.java index 112123da36..ee3f5a223c 100644 --- a/samza-azure/src/main/java/org/apache/samza/system/azureblob/AzureBlobSystemAdmin.java +++ b/samza-azure/src/main/java/org/apache/samza/system/azureblob/AzureBlobSystemAdmin.java @@ -19,7 +19,6 @@ package org.apache.samza.system.azureblob; -import java.util.HashMap; import java.util.Map; import java.util.Set; import org.apache.samza.system.SystemAdmin; @@ -32,14 +31,14 @@ */ public class AzureBlobSystemAdmin implements SystemAdmin { public Map getOffsetsAfter(Map offsets) { - return new HashMap<>(); + throw new UnsupportedOperationException("getOffsetsAfter not supported for AzureBlobSystemAdmin"); } public Map getSystemStreamMetadata(Set streamNames) { - return new HashMap<>(); + throw new UnsupportedOperationException("getSystemStreamMetadata not supported for AzureBlobSystemAdmin"); } public Integer offsetComparator(String offset1, String offset2) { - return null; + throw new UnsupportedOperationException("offsetComparator not supported for AzureBlobSystemAdmin"); } } \ No newline at end of file diff --git a/samza-azure/src/main/java/org/apache/samza/system/azureblob/avro/AzureBlobAvroWriter.java b/samza-azure/src/main/java/org/apache/samza/system/azureblob/avro/AzureBlobAvroWriter.java index f0cae9200f..f5e37aa644 100644 --- a/samza-azure/src/main/java/org/apache/samza/system/azureblob/avro/AzureBlobAvroWriter.java +++ b/samza-azure/src/main/java/org/apache/samza/system/azureblob/avro/AzureBlobAvroWriter.java @@ -157,9 +157,11 @@ public void write(OutgoingMessageEnvelope ome) throws IOException { if (ome.getMessage() instanceof IndexedRecord) { optionalIndexedRecord = Optional.of((IndexedRecord) ome.getMessage()); encodedRecord = encodeRecord((IndexedRecord) ome.getMessage()); - } else { + } else if (ome.getMessage() instanceof byte[]) { optionalIndexedRecord = Optional.empty(); encodedRecord = (byte[]) ome.getMessage(); + } else { + throw new IllegalArgumentException("AzureBlobAvroWriter only supports IndexedRecord and byte[]."); } synchronized (currentDataFileWriterLock) { @@ -270,11 +272,12 @@ private void closeDataFileWriter(DataFileWriter dataFileWriter, AzureBlobOutputS LOG.info("Closing the blob: {}", blockBlobAsyncClient.getBlobUrl().toString()); // dataFileWriter.close calls close of the azureBlobOutputStream associated with it. dataFileWriter.close(); - } finally { + } catch (Exception e) { // ensure that close is called even if dataFileWriter.close fails. // This is to avoid loss of all the blocks uploaded for the blob // as commitBlockList happens in close of azureBlobOutputStream. azureBlobOutputStream.close(); + throw e; } } diff --git a/samza-azure/src/main/java/org/apache/samza/system/azureblob/avro/AzureBlobOutputStream.java b/samza-azure/src/main/java/org/apache/samza/system/azureblob/avro/AzureBlobOutputStream.java index e03d005fa8..3f1fd46d6a 100644 --- a/samza-azure/src/main/java/org/apache/samza/system/azureblob/avro/AzureBlobOutputStream.java +++ b/samza-azure/src/main/java/org/apache/samza/system/azureblob/avro/AzureBlobOutputStream.java @@ -35,10 +35,8 @@ import java.util.Set; import java.util.concurrent.CompletableFuture; import java.util.concurrent.ConcurrentHashMap; -import java.util.concurrent.ExecutionException; import java.util.concurrent.Executor; import java.util.concurrent.TimeUnit; -import java.util.concurrent.TimeoutException; import org.slf4j.Logger; import org.slf4j.LoggerFactory; import reactor.core.publisher.Flux; @@ -88,15 +86,8 @@ public class AzureBlobOutputStream extends OutputStream { public AzureBlobOutputStream(BlockBlobAsyncClient blobAsyncClient, Executor blobThreadPool, AzureBlobWriterMetrics metrics, long flushTimeoutMs, int maxBlockFlushThresholdSize, Compression compression) { - byteArrayOutputStream = Optional.of(new ByteArrayOutputStream(maxBlockFlushThresholdSize)); - this.blobAsyncClient = blobAsyncClient; - blockList = new ArrayList<>(); - blockNum = 0; - this.blobThreadPool = blobThreadPool; - this.flushTimeoutMs = flushTimeoutMs; - this.maxBlockFlushThresholdSize = maxBlockFlushThresholdSize; - this.metrics = metrics; - this.compression = compression; + this(blobAsyncClient, blobThreadPool, metrics, flushTimeoutMs, maxBlockFlushThresholdSize, + new ByteArrayOutputStream(maxBlockFlushThresholdSize), compression); } /** @@ -159,8 +150,8 @@ public synchronized void flush() { */ @Override public synchronized void close() { - if (isClosed) { + LOG.info("{}: already closed", blobAsyncClient.getBlobUrl().toString()); return; } @@ -182,14 +173,10 @@ public synchronized void close() { metrics.updateAzureCommitMetrics(); Map blobMetadata = Collections.singletonMap(BLOB_RAW_SIZE_BYTES_METADATA, Long.toString(totalUploadedBlockSize)); blobAsyncClient.commitBlockListWithResponse(blockList, null, blobMetadata, null, null).block(); - } catch (InterruptedException | ExecutionException | TimeoutException e) { + } catch (Exception e) { String msg = String.format("Close blob %s failed with exception. Total pending sends %d", blobAsyncClient.getBlobUrl().toString(), pendingUpload.size()); throw new RuntimeException(msg, e); - } catch (Exception e) { - String msg = String.format("Close blob %s failed with exception. Resetting the system producer. %s", - blobAsyncClient.getBlobUrl().toString(), e.getLocalizedMessage()); - throw new RuntimeException(msg, e); } finally { blockList.clear(); pendingUpload.stream().forEach(future -> future.cancel(true)); @@ -226,7 +213,7 @@ public synchronized void releaseBuffer() throws IOException { @VisibleForTesting AzureBlobOutputStream(BlockBlobAsyncClient blobAsyncClient, Executor blobThreadPool, AzureBlobWriterMetrics metrics, - int flushTimeoutMs, int maxBlockFlushThresholdSize, + long flushTimeoutMs, int maxBlockFlushThresholdSize, ByteArrayOutputStream byteArrayOutputStream, Compression compression) { this.byteArrayOutputStream = Optional.of(byteArrayOutputStream); this.blobAsyncClient = blobAsyncClient; @@ -257,6 +244,8 @@ private synchronized void uploadBlockAsync() { } LOG.info("Blob: {} uploadBlock. Size:{}", blobAsyncClient.getBlobUrl().toString(), size); + // Azure sdk requires block Id to be encoded and all blockIds of a blob to be of the same length + // also, a block blob can have upto 50,000 blocks, hence using a 5 digit block id. String blockId = String.format("%05d", blockNum); String blockIdEncoded = Base64.getEncoder().encodeToString(blockId.getBytes()); blockList.add(blockIdEncoded); @@ -264,29 +253,34 @@ private synchronized void uploadBlockAsync() { byteArrayOutputStream.get().reset(); totalUploadedBlockSize += localByte.length; + + CompletableFuture futureNew = new CompletableFuture<>(); + + futureNew + CompletableFuture future = CompletableFuture.runAsync(new Runnable() { // call async stageblock and add to future @Override public void run() { - int retryCount = 0; + int attemptCount = 0; byte[] compressedLocalByte = compression.compress(localByte); int blockSize = compressedLocalByte.length; - while (retryCount < MAX_ATTEMPT) { + while (attemptCount < MAX_ATTEMPT) { try { ByteBuffer outputStream = ByteBuffer.wrap(compressedLocalByte, 0, blockSize); metrics.updateCompressByteMetrics(blockSize); LOG.info("{} Upload block start for blob: {} for block size:{}.", blobAsyncClient.getBlobUrl().toString(), blockId, blockSize); - // StageBlock generates exception on Failure. metrics.updateAzureUploadMetrics(); + // StageBlock generates exception on Failure. blobAsyncClient.stageBlock(blockIdEncoded, Flux.just(outputStream), blockSize).block(); break; } catch (Exception e) { - retryCount += 1; + attemptCount += 1; String msg = "Upload block for blob: " + blobAsyncClient.getBlobUrl().toString() - + " failed for blockid: " + blockId + " due to exception. RetryCount: " + retryCount; + + " failed for blockid: " + blockId + " due to exception. RetryCount: " + attemptCount; LOG.error(msg, e); - if (retryCount == MAX_ATTEMPT) { + if (attemptCount == MAX_ATTEMPT) { throw new RuntimeException("Exceeded number of retries. Max attempts is: " + MAX_ATTEMPT, e); } } @@ -294,7 +288,11 @@ public void run() { } }, blobThreadPool); - pendingUpload.add(future); + if (future.isDone()) { + LOG.info("Upload block for blob: {} with blockid: {} finished.", blobAsyncClient.getBlobUrl().toString(), blockId); + } else { + pendingUpload.add(future); + } future.handle((aVoid, throwable) -> { if (throwable == null) { LOG.info("Upload block for blob: {} with blockid: {} finished.", blobAsyncClient.getBlobUrl().toString(), blockId); diff --git a/samza-azure/src/main/java/org/apache/samza/system/azureblob/compression/CompressionFactory.java b/samza-azure/src/main/java/org/apache/samza/system/azureblob/compression/CompressionFactory.java index 4a45636531..bcba6c7c5f 100644 --- a/samza-azure/src/main/java/org/apache/samza/system/azureblob/compression/CompressionFactory.java +++ b/samza-azure/src/main/java/org/apache/samza/system/azureblob/compression/CompressionFactory.java @@ -25,6 +25,16 @@ * based on the {@link org.apache.samza.system.azureblob.compression.CompressionType}. */ public class CompressionFactory { + private static CompressionFactory compressionFactoryInstance = null; + private CompressionFactory() {} + + public static CompressionFactory getInstance() { + if (compressionFactoryInstance == null) { + compressionFactoryInstance = new CompressionFactory(); + } + return compressionFactoryInstance; + } + public Compression getCompression(CompressionType compressionType) { switch (compressionType) { case NONE: return new NoneCompression(); diff --git a/samza-azure/src/main/java/org/apache/samza/system/azureblob/producer/AzureBlobSystemProducer.java b/samza-azure/src/main/java/org/apache/samza/system/azureblob/producer/AzureBlobSystemProducer.java index 0a4600b833..7a55635df5 100644 --- a/samza-azure/src/main/java/org/apache/samza/system/azureblob/producer/AzureBlobSystemProducer.java +++ b/samza-azure/src/main/java/org/apache/samza/system/azureblob/producer/AzureBlobSystemProducer.java @@ -120,7 +120,7 @@ public class AzureBlobSystemProducer implements SystemProducer { private final AzureBlobConfig config; // Map of writers indexed first by sourceName and then by (streamName, partitionName) or just streamName if partition key does not exist. - private Map> writerMap; + private final Map> writerMap; private final AzureBlobWriterFactory writerFactory; private final int blockFlushThresholdSize; @@ -156,7 +156,7 @@ public AzureBlobSystemProducer(String systemName, AzureBlobConfig config, Metric this.closeTimeout = this.config.getCloseTimeoutMs(this.systemName); this.blockFlushThresholdSize = this.config.getMaxFlushThresholdSize(this.systemName); int asyncBlobThreadPoolCount = this.config.getAzureBlobThreadPoolCount(this.systemName); - int blockingQueueSize = this.config.getBlockingQueueSizeOrDefault(this.systemName, asyncBlobThreadPoolCount * 2); + int blockingQueueSize = this.config.getBlockingQueueSize(this.systemName); LOG.info("SystemName: {} block flush size:{}", systemName, this.blockFlushThresholdSize); LOG.info("SystemName: {} thread count:{}", systemName, asyncBlobThreadPoolCount); @@ -362,11 +362,11 @@ void setupAzureContainer(String accountName, String accountKey) { boolean isPremiumAccount = SkuName.PREMIUM_LRS == accountType; if (isPremiumAccount && flushThresholdSize > PREMIUM_MAX_BLOCK_SIZE) { // 100 MB throw new SystemProducerException("Azure storage account with name: " + accountName - + " is a premium account and can only handle upto 100MB threshold size. Given flush threshold size is " + + " is a premium account and can only handle upto " + PREMIUM_MAX_BLOCK_SIZE + " threshold size. Given flush threshold size is " + flushThresholdSize); } else if (!isPremiumAccount && flushThresholdSize > STANDARD_MAX_BLOCK_SIZE) { // STANDARD account throw new SystemProducerException("Azure storage account with name: " + accountName - + " is a standard account and can only handle upto 4MB threshold size. Given flush threshold size is " + + " is a standard account and can only handle upto " + STANDARD_MAX_BLOCK_SIZE + " threshold size. Given flush threshold size is " + flushThresholdSize); } @@ -493,7 +493,7 @@ AzureBlobWriter createNewWriter(String blobURL, AzureBlobWriterMetrics writerMet try { return writerFactory.getWriterInstance(containerAsyncClient, blobURL, asyncBlobThreadPool, writerMetrics, blockFlushThresholdSize, flushTimeoutMs, - new CompressionFactory().getCompression(config.getCompressionType(systemName)), + CompressionFactory.getInstance().getCompression(config.getCompressionType(systemName)), config.getSuffixRandomStringToBlobName(systemName), config.getMaxBlobSize(systemName), config.getMaxMessagesPerBlob(systemName)); diff --git a/samza-azure/src/test/java/org/apache/samza/system/azureblob/avro/TestAzureBlobAvroWriter.java b/samza-azure/src/test/java/org/apache/samza/system/azureblob/avro/TestAzureBlobAvroWriter.java index c9220cf8d2..3f3887d838 100644 --- a/samza-azure/src/test/java/org/apache/samza/system/azureblob/avro/TestAzureBlobAvroWriter.java +++ b/samza-azure/src/test/java/org/apache/samza/system/azureblob/avro/TestAzureBlobAvroWriter.java @@ -53,8 +53,14 @@ import org.powermock.core.classloader.annotations.PrepareForTest; import org.powermock.modules.junit4.PowerMockRunner; -import static org.mockito.Matchers.*; -import static org.mockito.Mockito.*; +import static org.mockito.Mockito.anyString; +import static org.mockito.Mockito.doReturn; +import static org.mockito.Mockito.doThrow; +import static org.mockito.Mockito.mock; +import static org.mockito.Mockito.spy; +import static org.mockito.Mockito.times; +import static org.mockito.Mockito.verify; +import static org.mockito.Mockito.when; @RunWith(PowerMockRunner.class) @@ -190,7 +196,6 @@ public void testWriteWhenDataFileWriterFails() throws Exception { public void testClose() throws Exception { azureBlobAvroWriter.close(); verify(mockDataFileWriter).close(); - verify(mockAzureBlobOutputStream).close(); } @Test(expected = SamzaException.class) @@ -199,10 +204,12 @@ public void testCloseWhenDataFileWriterFails() throws Exception { azureBlobAvroWriter.flush(); azureBlobAvroWriter.close(); + verify(mockAzureBlobOutputStream).close(); } @Test(expected = RuntimeException.class) public void testCloseWhenOutputStreamFails() throws Exception { + doThrow(new IOException("DataFileWriter failed")).when(mockDataFileWriter).close(); doThrow(new RuntimeException("failed")).when(mockAzureBlobOutputStream).close(); azureBlobAvroWriter.close(); @@ -292,7 +299,6 @@ public void testMultipleBlobClose() throws Exception { DataFileWriter mockDataFileWriter2 = mock(DataFileWriter.class); AzureBlobOutputStream mockAzureBlobOutputStream2 = mock(AzureBlobOutputStream.class); - when(mockAzureBlobOutputStream.getSize()).thenReturn((long) 1); when(mockAzureBlobOutputStream.getSize()).thenReturn((long) 1); BlobAsyncClient mockBlobAsyncClient = mock(BlobAsyncClient.class); doReturn(mockBlobAsyncClient).when(mockContainerClient).getBlobAsyncClient(anyString()); @@ -305,8 +311,6 @@ public void testMultipleBlobClose() throws Exception { // first OME creates the first blob and 11th OME creates the second blob. azureBlobAvroWriter.close(); - verify(mockAzureBlobOutputStream).close(); - verify(mockAzureBlobOutputStream2).close(); verify(mockDataFileWriter).close(); verify(mockDataFileWriter2).close(); } @@ -382,7 +386,6 @@ public void testMultipleThreadWriteFlushFinallyClose() throws Exception { verify(mockDataFileWriter, times(10)).appendEncoded(ByteBuffer.wrap(encodeRecord((IndexedRecord) ome2.getMessage()))); verify(mockDataFileWriter, times(2)).flush(); verify(mockDataFileWriter).close(); - verify(mockAzureBlobOutputStream).close(); } private byte[] encodeRecord(IndexedRecord record) throws Exception { diff --git a/samza-azure/src/test/java/org/apache/samza/system/azureblob/avro/TestAzureBlobOutputStream.java b/samza-azure/src/test/java/org/apache/samza/system/azureblob/avro/TestAzureBlobOutputStream.java index 20d4aeff9e..9091db7225 100644 --- a/samza-azure/src/test/java/org/apache/samza/system/azureblob/avro/TestAzureBlobOutputStream.java +++ b/samza-azure/src/test/java/org/apache/samza/system/azureblob/avro/TestAzureBlobOutputStream.java @@ -21,6 +21,7 @@ import com.azure.core.http.rest.SimpleResponse; import com.azure.core.implementation.util.FluxUtil; +import java.util.Arrays; import org.apache.samza.system.azureblob.compression.Compression; import org.apache.samza.system.azureblob.producer.AzureBlobWriterMetrics; import com.azure.storage.blob.specialized.BlockBlobAsyncClient; @@ -232,10 +233,7 @@ public void testCloseMultipleBlocks() { PowerMockito.doAnswer(invocation -> { ArrayList blockListArg = (ArrayList) invocation.getArguments()[0]; - String blockIdArg = (String) blockListArg.toArray()[0]; - String blockIdArg1 = (String) blockListArg.toArray()[1]; - Assert.assertEquals(blockIdEncoded, blockIdArg); - Assert.assertEquals(blockIdEncoded1, blockIdArg1); + Assert.assertEquals(Arrays.asList(blockIdEncoded, blockIdEncoded1), blockListArg); Map blobMetadata = (Map) invocation.getArguments()[2]; Assert.assertEquals(blobMetadata.get(AzureBlobOutputStream.BLOB_RAW_SIZE_BYTES_METADATA), Long.toString(2 * THRESHOLD)); return Mono.just(new SimpleResponse(null, 200, null, null)); From 60951a43fd413942233ea526ee384d6d0836551e Mon Sep 17 00:00:00 2001 From: Manasa Date: Tue, 24 Dec 2019 11:04:03 -0800 Subject: [PATCH 3/9] fix typo --- .../samza/system/azureblob/avro/AzureBlobOutputStream.java | 5 ----- 1 file changed, 5 deletions(-) diff --git a/samza-azure/src/main/java/org/apache/samza/system/azureblob/avro/AzureBlobOutputStream.java b/samza-azure/src/main/java/org/apache/samza/system/azureblob/avro/AzureBlobOutputStream.java index 3f1fd46d6a..25d6ef0107 100644 --- a/samza-azure/src/main/java/org/apache/samza/system/azureblob/avro/AzureBlobOutputStream.java +++ b/samza-azure/src/main/java/org/apache/samza/system/azureblob/avro/AzureBlobOutputStream.java @@ -253,11 +253,6 @@ private synchronized void uploadBlockAsync() { byteArrayOutputStream.get().reset(); totalUploadedBlockSize += localByte.length; - - CompletableFuture futureNew = new CompletableFuture<>(); - - futureNew - CompletableFuture future = CompletableFuture.runAsync(new Runnable() { // call async stageblock and add to future @Override From a2dafecfb02882e5502ac7c6f90439d6aa34e229 Mon Sep 17 00:00:00 2001 From: Manasa Date: Tue, 24 Dec 2019 15:07:26 -0800 Subject: [PATCH 4/9] fix comments in tests --- .../azureblob/avro/AzureBlobAvroWriter.java | 4 +- .../azureblob/avro/AzureBlobOutputStream.java | 4 +- .../avro/TestAzureBlobAvroWriter.java | 116 ++++++++++++++---- 3 files changed, 98 insertions(+), 26 deletions(-) diff --git a/samza-azure/src/main/java/org/apache/samza/system/azureblob/avro/AzureBlobAvroWriter.java b/samza-azure/src/main/java/org/apache/samza/system/azureblob/avro/AzureBlobAvroWriter.java index f5e37aa644..234a87c97c 100644 --- a/samza-azure/src/main/java/org/apache/samza/system/azureblob/avro/AzureBlobAvroWriter.java +++ b/samza-azure/src/main/java/org/apache/samza/system/azureblob/avro/AzureBlobAvroWriter.java @@ -219,7 +219,7 @@ public void close() { Executor blobThreadPool, int maxBlockFlushThresholdSize, int flushTimeoutMs, String blobURLPrefix, DataFileWriter dataFileWriter, AzureBlobOutputStream azureBlobOutputStream, BlockBlobAsyncClient blockBlobAsyncClient, - long maxBlobSize, long maxRecordsPerBlob) { + long maxBlobSize, long maxRecordsPerBlob, Compression compression) { if (dataFileWriter == null || azureBlobOutputStream == null || blockBlobAsyncClient == null) { this.currentBlobWriterComponents = null; } else { @@ -232,7 +232,7 @@ public void close() { this.metrics = metrics; this.maxBlockFlushThresholdSize = maxBlockFlushThresholdSize; this.flushTimeoutMs = flushTimeoutMs; - this.compression = new GzipCompression(); + this.compression = compression; this.containerAsyncClient = containerAsyncClient; this.useRandomStringInBlobName = false; this.maxBlobSize = maxBlobSize; diff --git a/samza-azure/src/main/java/org/apache/samza/system/azureblob/avro/AzureBlobOutputStream.java b/samza-azure/src/main/java/org/apache/samza/system/azureblob/avro/AzureBlobOutputStream.java index 25d6ef0107..974b7977cc 100644 --- a/samza-azure/src/main/java/org/apache/samza/system/azureblob/avro/AzureBlobOutputStream.java +++ b/samza-azure/src/main/java/org/apache/samza/system/azureblob/avro/AzureBlobOutputStream.java @@ -291,7 +291,9 @@ public void run() { future.handle((aVoid, throwable) -> { if (throwable == null) { LOG.info("Upload block for blob: {} with blockid: {} finished.", blobAsyncClient.getBlobUrl().toString(), blockId); - pendingUpload.remove(future); + if (pendingUpload.contains(future)) { + pendingUpload.remove(future); + } return aVoid; } else { throw new RuntimeException("Blob upload failed for blob " + blobAsyncClient.getBlobUrl().toString() diff --git a/samza-azure/src/test/java/org/apache/samza/system/azureblob/avro/TestAzureBlobAvroWriter.java b/samza-azure/src/test/java/org/apache/samza/system/azureblob/avro/TestAzureBlobAvroWriter.java index 3f3887d838..265847e954 100644 --- a/samza-azure/src/test/java/org/apache/samza/system/azureblob/avro/TestAzureBlobAvroWriter.java +++ b/samza-azure/src/test/java/org/apache/samza/system/azureblob/avro/TestAzureBlobAvroWriter.java @@ -22,6 +22,9 @@ import com.azure.storage.blob.BlobAsyncClient; import com.azure.storage.blob.BlobContainerAsyncClient; import com.azure.storage.blob.specialized.BlockBlobAsyncClient; +import org.apache.samza.system.azureblob.compression.Compression; +import org.apache.samza.system.azureblob.compression.CompressionFactory; +import org.apache.samza.system.azureblob.compression.CompressionType; import org.apache.samza.system.azureblob.producer.AzureBlobWriterMetrics; import java.io.ByteArrayOutputStream; import java.io.IOException; @@ -49,6 +52,7 @@ import org.junit.Test; import org.junit.runner.RunWith; import org.mockito.ArgumentCaptor; +import org.mockito.Matchers; import org.powermock.api.mockito.PowerMockito; import org.powermock.core.classloader.annotations.PrepareForTest; import org.powermock.modules.junit4.PowerMockRunner; @@ -73,6 +77,7 @@ public class TestAzureBlobAvroWriter { private DataFileWriter mockDataFileWriter; private AzureBlobOutputStream mockAzureBlobOutputStream; private BlockBlobAsyncClient mockBlockBlobAsyncClient; + private Compression mockCompression; private static final String VALUE = "FAKE_VALUE"; private static final String SYSTEM_NAME = "FAKE_SYSTEM"; @@ -138,10 +143,11 @@ public void setup() throws Exception { mockBlockBlobAsyncClient = PowerMockito.mock(BlockBlobAsyncClient.class); when(mockBlockBlobAsyncClient.getBlobUrl()).thenReturn("https://samza.blob.core.windows.net/fake-blob-url"); + mockCompression = CompressionFactory.getInstance().getCompression(CompressionType.GZIP); azureBlobAvroWriter = spy(new AzureBlobAvroWriter(mockContainerAsyncClient, mock(AzureBlobWriterMetrics.class), threadPool, THRESHOLD, 60000, "test", mockDataFileWriter, mockAzureBlobOutputStream, mockBlockBlobAsyncClient, Long.MAX_VALUE, - Long.MAX_VALUE)); // keeping blob size and number of records unlimited + Long.MAX_VALUE, mockCompression)); // keeping blob size and number of records unlimited doReturn(encodedRecord).when(azureBlobAvroWriter).encodeRecord((IndexedRecord) ome.getMessage()); } @Test @@ -181,7 +187,7 @@ public void testWriteByteArrayWithoutSchema() throws Exception { azureBlobAvroWriter = spy(new AzureBlobAvroWriter(PowerMockito.mock(BlobContainerAsyncClient.class), mock(AzureBlobWriterMetrics.class), threadPool, THRESHOLD, 60000, "test", - null, null, null, 1000, 100)); + null, null, null, 1000, 100, mockCompression)); OutgoingMessageEnvelope omeEncoded = new OutgoingMessageEnvelope(new SystemStream(SYSTEM_NAME, "Topic1"), new byte[100]); azureBlobAvroWriter.write(omeEncoded); } @@ -231,21 +237,46 @@ public void testFlushWhenDataFileWriterFails() throws Exception { public void testMaxBlobSizeExceeded() throws Exception { String blobUrlPrefix = "test"; long maxBlobSize = 1000; + AzureBlobWriterMetrics mockMetrics = mock(AzureBlobWriterMetrics.class); BlobContainerAsyncClient mockContainerClient = PowerMockito.mock(BlobContainerAsyncClient.class); azureBlobAvroWriter = spy(new AzureBlobAvroWriter(mockContainerClient, - mock(AzureBlobWriterMetrics.class), threadPool, THRESHOLD, 60000, blobUrlPrefix, - null, null, null, maxBlobSize, 10)); + mockMetrics, threadPool, THRESHOLD, 60000, blobUrlPrefix, + null, null, null, maxBlobSize, 10, mockCompression)); + + DataFileWriter mockDataFileWriter1 = mock(DataFileWriter.class); + PowerMockito.whenNew(DataFileWriter.class).withAnyArguments().thenReturn(mockDataFileWriter1); + + BlobAsyncClient mockBlobAsyncClient1 = mock(BlobAsyncClient.class); + doReturn(mockBlobAsyncClient1).when(mockContainerClient).getBlobAsyncClient(Matchers.endsWith("0.avro.gz")); + BlockBlobAsyncClient mockBlockBlobAsyncClient1 = mock(BlockBlobAsyncClient.class); + doReturn(mockBlockBlobAsyncClient1).when(mockBlobAsyncClient1).getBlockBlobAsyncClient(); + + AzureBlobOutputStream mockAzureBlobOutputStream1 = mock(AzureBlobOutputStream.class); + PowerMockito.whenNew(AzureBlobOutputStream.class).withArguments(mockBlockBlobAsyncClient1, threadPool, + mockMetrics, + (long) 60000, THRESHOLD, mockCompression).thenReturn(mockAzureBlobOutputStream1); + when(mockAzureBlobOutputStream1.getSize()).thenReturn((long) maxBlobSize - 1); - when(mockAzureBlobOutputStream.getSize()).thenReturn((long) maxBlobSize - 1); - PowerMockito.whenNew(AzureBlobOutputStream.class).withAnyArguments().thenReturn(mockAzureBlobOutputStream); - PowerMockito.whenNew(DataFileWriter.class).withAnyArguments().thenReturn(mockDataFileWriter); - BlobAsyncClient mockBlobAsyncClient = mock(BlobAsyncClient.class); - doReturn(mockBlobAsyncClient).when(mockContainerClient).getBlobAsyncClient(anyString()); - doReturn(mockBlockBlobAsyncClient).when(mockBlobAsyncClient).getBlockBlobAsyncClient(); // first OME creates the first blob azureBlobAvroWriter.write(ome); + + OutgoingMessageEnvelope ome2 = createOME("Topic2"); + DataFileWriter mockDataFileWriter2 = mock(DataFileWriter.class); + PowerMockito.whenNew(DataFileWriter.class).withAnyArguments().thenReturn(mockDataFileWriter2); + + BlobAsyncClient mockBlobAsyncClient2 = mock(BlobAsyncClient.class); + doReturn(mockBlobAsyncClient2).when(mockContainerClient).getBlobAsyncClient(Matchers.endsWith("1.avro.gz")); + BlockBlobAsyncClient mockBlockBlobAsyncClient2 = mock(BlockBlobAsyncClient.class); + doReturn(mockBlockBlobAsyncClient2).when(mockBlobAsyncClient2).getBlockBlobAsyncClient(); + + AzureBlobOutputStream mockAzureBlobOutputStream2 = mock(AzureBlobOutputStream.class); + PowerMockito.whenNew(AzureBlobOutputStream.class).withArguments(mockBlockBlobAsyncClient2, threadPool, + mockMetrics, + (long) 60000, THRESHOLD, mockCompression).thenReturn(mockAzureBlobOutputStream2); + when(mockAzureBlobOutputStream2.getSize()).thenReturn((long) maxBlobSize - 1); + // Second OME creates the second blob because maxBlobSize is 1000 and mockAzureBlobOutputStream.getSize is 999. - azureBlobAvroWriter.write(ome); + azureBlobAvroWriter.write(ome2); ArgumentCaptor argument = ArgumentCaptor.forClass(String.class); verify(mockContainerClient, times(2)).getBlobAsyncClient(argument.capture()); @@ -254,28 +285,61 @@ public void testMaxBlobSizeExceeded() throws Exception { }); List allBlobNames = argument.getAllValues(); Assert.assertNotEquals(allBlobNames.get(0), allBlobNames.get(1)); + + verify(mockDataFileWriter1).appendEncoded(ByteBuffer.wrap(encodeRecord((IndexedRecord) ome.getMessage()))); + verify(mockDataFileWriter2).appendEncoded(ByteBuffer.wrap(encodeRecord((IndexedRecord) ome2.getMessage()))); + + verify(mockDataFileWriter1).create(((IndexedRecord) ome.getMessage()).getSchema(), mockAzureBlobOutputStream1); + verify(mockDataFileWriter2).create(((IndexedRecord) ome2.getMessage()).getSchema(), mockAzureBlobOutputStream2); } @Test public void testRecordLimitExceeded() throws Exception { String blobUrlPrefix = "test"; + AzureBlobWriterMetrics mockMetrics = mock(AzureBlobWriterMetrics.class); long maxBlobSize = AzureBlobAvroWriter.DATAFILEWRITER_OVERHEAD + 1000; long maxRecordsPerBlob = 10; BlobContainerAsyncClient mockContainerClient = PowerMockito.mock(BlobContainerAsyncClient.class); azureBlobAvroWriter = spy(new AzureBlobAvroWriter(mockContainerClient, - mock(AzureBlobWriterMetrics.class), threadPool, THRESHOLD, 60000, blobUrlPrefix, - null, null, null, maxBlobSize, maxRecordsPerBlob)); + mockMetrics, threadPool, THRESHOLD, 60000, blobUrlPrefix, + null, null, null, maxBlobSize, maxRecordsPerBlob, mockCompression)); - when(mockAzureBlobOutputStream.getSize()).thenReturn((long) 1); - PowerMockito.whenNew(AzureBlobOutputStream.class).withAnyArguments().thenReturn(mockAzureBlobOutputStream); - PowerMockito.whenNew(DataFileWriter.class).withAnyArguments().thenReturn(mockDataFileWriter); - BlobAsyncClient mockBlobAsyncClient = mock(BlobAsyncClient.class); - doReturn(mockBlobAsyncClient).when(mockContainerClient).getBlobAsyncClient(anyString()); - doReturn(mockBlockBlobAsyncClient).when(mockBlobAsyncClient).getBlockBlobAsyncClient(); - for (int i = 0; i <= maxRecordsPerBlob; i++) { + DataFileWriter mockDataFileWriter1 = mock(DataFileWriter.class); + PowerMockito.whenNew(DataFileWriter.class).withAnyArguments().thenReturn(mockDataFileWriter1); + + BlobAsyncClient mockBlobAsyncClient1 = mock(BlobAsyncClient.class); + doReturn(mockBlobAsyncClient1).when(mockContainerClient).getBlobAsyncClient(Matchers.endsWith("0.avro.gz")); + BlockBlobAsyncClient mockBlockBlobAsyncClient1 = mock(BlockBlobAsyncClient.class); + doReturn(mockBlockBlobAsyncClient1).when(mockBlobAsyncClient1).getBlockBlobAsyncClient(); + + AzureBlobOutputStream mockAzureBlobOutputStream1 = mock(AzureBlobOutputStream.class); + PowerMockito.whenNew(AzureBlobOutputStream.class).withArguments(mockBlockBlobAsyncClient1, threadPool, + mockMetrics, + (long) 60000, THRESHOLD, mockCompression).thenReturn(mockAzureBlobOutputStream1); + when(mockAzureBlobOutputStream1.getSize()).thenReturn((long) 1); + + // first OME creates the first blob and 11th OME (ome2) creates the second blob. + + for (int i = 0; i < maxRecordsPerBlob; i++) { azureBlobAvroWriter.write(ome); } - // first OME creates the first blob and 11th OME creates the second blob. + + OutgoingMessageEnvelope ome2 = createOME("Topic2"); + DataFileWriter mockDataFileWriter2 = mock(DataFileWriter.class); + PowerMockito.whenNew(DataFileWriter.class).withAnyArguments().thenReturn(mockDataFileWriter2); + + BlobAsyncClient mockBlobAsyncClient2 = mock(BlobAsyncClient.class); + doReturn(mockBlobAsyncClient2).when(mockContainerClient).getBlobAsyncClient(Matchers.endsWith("1.avro.gz")); + BlockBlobAsyncClient mockBlockBlobAsyncClient2 = mock(BlockBlobAsyncClient.class); + doReturn(mockBlockBlobAsyncClient2).when(mockBlobAsyncClient2).getBlockBlobAsyncClient(); + + AzureBlobOutputStream mockAzureBlobOutputStream2 = mock(AzureBlobOutputStream.class); + PowerMockito.whenNew(AzureBlobOutputStream.class).withArguments(mockBlockBlobAsyncClient2, threadPool, + mockMetrics, + (long) 60000, THRESHOLD, mockCompression).thenReturn(mockAzureBlobOutputStream2); + when(mockAzureBlobOutputStream2.getSize()).thenReturn((long) 1); + + azureBlobAvroWriter.write(ome2); ArgumentCaptor argument = ArgumentCaptor.forClass(String.class); verify(mockContainerClient, times(2)).getBlobAsyncClient(argument.capture()); @@ -284,6 +348,12 @@ public void testRecordLimitExceeded() throws Exception { }); List allBlobNames = argument.getAllValues(); Assert.assertNotEquals(allBlobNames.get(0), allBlobNames.get(1)); + + verify(mockDataFileWriter1, times((int) maxRecordsPerBlob)).appendEncoded(ByteBuffer.wrap(encodeRecord((IndexedRecord) ome.getMessage()))); + verify(mockDataFileWriter2).appendEncoded(ByteBuffer.wrap(encodeRecord((IndexedRecord) ome2.getMessage()))); + + verify(mockDataFileWriter1).create(((IndexedRecord) ome.getMessage()).getSchema(), mockAzureBlobOutputStream1); + verify(mockDataFileWriter2).create(((IndexedRecord) ome2.getMessage()).getSchema(), mockAzureBlobOutputStream2); } @Test @@ -294,7 +364,7 @@ public void testMultipleBlobClose() throws Exception { BlobContainerAsyncClient mockContainerClient = PowerMockito.mock(BlobContainerAsyncClient.class); azureBlobAvroWriter = spy(new AzureBlobAvroWriter(mockContainerClient, mock(AzureBlobWriterMetrics.class), threadPool, THRESHOLD, 60000, blobUrlPrefix, - mockDataFileWriter, mockAzureBlobOutputStream, mockBlockBlobAsyncClient, maxBlobSize, maxRecordsPerBlob)); + mockDataFileWriter, mockAzureBlobOutputStream, mockBlockBlobAsyncClient, maxBlobSize, maxRecordsPerBlob, mockCompression)); DataFileWriter mockDataFileWriter2 = mock(DataFileWriter.class); AzureBlobOutputStream mockAzureBlobOutputStream2 = mock(AzureBlobOutputStream.class); @@ -320,7 +390,7 @@ public void testEncodeRecord() throws Exception { azureBlobAvroWriter = spy(new AzureBlobAvroWriter(PowerMockito.mock(BlobContainerAsyncClient.class), mock(AzureBlobWriterMetrics.class), threadPool, THRESHOLD, 60000, "test", mockDataFileWriter, mockAzureBlobOutputStream, mockBlockBlobAsyncClient, - Long.MAX_VALUE, Long.MAX_VALUE)); + Long.MAX_VALUE, Long.MAX_VALUE, mockCompression)); IndexedRecord record = new GenericRecordEvent(); Assert.assertTrue(Arrays.equals(encodeRecord(record), azureBlobAvroWriter.encodeRecord(record))); } From 806d8a8fd31525e6b173a8348b4d3175196febd9 Mon Sep 17 00:00:00 2001 From: Manasa Date: Thu, 2 Jan 2020 15:14:47 -0800 Subject: [PATCH 5/9] fix comments --- .../azureblob/avro/AzureBlobAvroWriter.java | 1 - .../azureblob/avro/AzureBlobOutputStream.java | 9 ++- .../producer/AzureBlobSystemProducer.java | 19 ++++- .../avro/TestAzureBlobOutputStream.java | 74 +++++++++---------- .../producer/TestAzureBlobSystemProducer.java | 6 -- 5 files changed, 58 insertions(+), 51 deletions(-) diff --git a/samza-azure/src/main/java/org/apache/samza/system/azureblob/avro/AzureBlobAvroWriter.java b/samza-azure/src/main/java/org/apache/samza/system/azureblob/avro/AzureBlobAvroWriter.java index 234a87c97c..680e404de9 100644 --- a/samza-azure/src/main/java/org/apache/samza/system/azureblob/avro/AzureBlobAvroWriter.java +++ b/samza-azure/src/main/java/org/apache/samza/system/azureblob/avro/AzureBlobAvroWriter.java @@ -24,7 +24,6 @@ import com.google.common.annotations.VisibleForTesting; import com.google.common.base.Preconditions; import org.apache.samza.system.azureblob.compression.Compression; -import org.apache.samza.system.azureblob.compression.GzipCompression; import org.apache.samza.system.azureblob.producer.AzureBlobWriter; import org.apache.samza.system.azureblob.producer.AzureBlobWriterMetrics; import java.io.ByteArrayOutputStream; diff --git a/samza-azure/src/main/java/org/apache/samza/system/azureblob/avro/AzureBlobOutputStream.java b/samza-azure/src/main/java/org/apache/samza/system/azureblob/avro/AzureBlobOutputStream.java index 974b7977cc..5268933e08 100644 --- a/samza-azure/src/main/java/org/apache/samza/system/azureblob/avro/AzureBlobOutputStream.java +++ b/samza-azure/src/main/java/org/apache/samza/system/azureblob/avro/AzureBlobOutputStream.java @@ -21,6 +21,7 @@ import com.azure.storage.blob.specialized.BlockBlobAsyncClient; import com.google.common.annotations.VisibleForTesting; +import org.apache.samza.AzureException; import org.apache.samza.system.azureblob.compression.Compression; import org.apache.samza.system.azureblob.producer.AzureBlobWriterMetrics; import java.io.ByteArrayOutputStream; @@ -176,7 +177,7 @@ public synchronized void close() { } catch (Exception e) { String msg = String.format("Close blob %s failed with exception. Total pending sends %d", blobAsyncClient.getBlobUrl().toString(), pendingUpload.size()); - throw new RuntimeException(msg, e); + throw new AzureException(msg, e); } finally { blockList.clear(); pendingUpload.stream().forEach(future -> future.cancel(true)); @@ -276,7 +277,7 @@ public void run() { + " failed for blockid: " + blockId + " due to exception. RetryCount: " + attemptCount; LOG.error(msg, e); if (attemptCount == MAX_ATTEMPT) { - throw new RuntimeException("Exceeded number of retries. Max attempts is: " + MAX_ATTEMPT, e); + throw new AzureException("Exceeded number of retries. Max attempts is: " + MAX_ATTEMPT, e); } } } @@ -296,14 +297,14 @@ public void run() { } return aVoid; } else { - throw new RuntimeException("Blob upload failed for blob " + blobAsyncClient.getBlobUrl().toString() + throw new AzureException("Blob upload failed for blob " + blobAsyncClient.getBlobUrl().toString() + " and block with id: " + blockId, throwable); } }); blockNum += 1; if (blockNum >= MAX_BLOCKS_IN_AZURE_BLOB) { - throw new RuntimeException("Azure blob only supports 50000 blocks in a blob. Current number of blocks is " + blockNum); + throw new AzureException("Azure blob only supports 50000 blocks in a blob. Current number of blocks is " + blockNum); } } } \ No newline at end of file diff --git a/samza-azure/src/main/java/org/apache/samza/system/azureblob/producer/AzureBlobSystemProducer.java b/samza-azure/src/main/java/org/apache/samza/system/azureblob/producer/AzureBlobSystemProducer.java index 7a55635df5..a2fa2acb05 100644 --- a/samza-azure/src/main/java/org/apache/samza/system/azureblob/producer/AzureBlobSystemProducer.java +++ b/samza-azure/src/main/java/org/apache/samza/system/azureblob/producer/AzureBlobSystemProducer.java @@ -180,7 +180,8 @@ public AzureBlobSystemProducer(String systemName, AzureBlobConfig config, Metric @Override public synchronized void start() { if (isStarted) { - throw new SystemProducerException("Attempting to start an already started producer."); + LOG.warn("Attempting to start an already started producer."); + return; } String accountName = config.getAzureAccountName(systemName); @@ -199,11 +200,13 @@ public synchronized void start() { @Override public synchronized void stop() { if (!isStarted) { - throw new SystemProducerException("Attempting to stop a producer that was not started."); + LOG.warn("Attempting to stop a producer that was not started."); + return; } if (isStopped) { - throw new SystemProducerException("Attempting to stop an already stopped producer."); + LOG.warn("Attempting to stop an already stopped producer."); + return; } try { @@ -277,6 +280,11 @@ public void send(String source, OutgoingMessageEnvelope messageEnvelope) { if (!isStarted) { throw new SystemProducerException("Trying to send before producer has started."); } + + if (isStopped) { + throw new SystemProducerException("Sending after producer has been stopped."); + } + ReadWriteLock lock = sourceSendFlushLockMap.get(source); if (lock == null) { throw new SystemProducerException("Attempting to send to source: " + source + " but it was not registered"); @@ -307,6 +315,11 @@ public void flush(String source) { if (!isStarted) { throw new SystemProducerException("Trying to flush before producer has started."); } + + if (isStopped) { + throw new SystemProducerException("Flushing after producer has been stopped."); + } + ReadWriteLock lock = sourceSendFlushLockMap.get(source); if (lock == null) { throw new SystemProducerException("Attempting to flush source: " + source + " but it was not registered"); diff --git a/samza-azure/src/test/java/org/apache/samza/system/azureblob/avro/TestAzureBlobOutputStream.java b/samza-azure/src/test/java/org/apache/samza/system/azureblob/avro/TestAzureBlobOutputStream.java index 9091db7225..a00f563d1c 100644 --- a/samza-azure/src/test/java/org/apache/samza/system/azureblob/avro/TestAzureBlobOutputStream.java +++ b/samza-azure/src/test/java/org/apache/samza/system/azureblob/avro/TestAzureBlobOutputStream.java @@ -66,9 +66,9 @@ public class TestAzureBlobOutputStream { private static final int THRESHOLD = 100; private BlockBlobAsyncClient mockBlobAsyncClient; private AzureBlobOutputStream azureBlobOutputStream; - private static String randomString = "roZzozzLiR7GCEjcB0UsRUNgBAip8cSLGXQSo3RQvbIDoxOaaRs4hrec2s5rMPWgTPRY4UnE959worEtyhRjwUFnRnVuNFZ554yuPQCbI69qFkQX7MmrB4blmpSnFeGjWKjFjIRLFNVSsQBYMkr5jT4T83uVtuGumsjACVrpcilihdd194H8Y71rQcrXZoTQtw5OvmPicbwptawpHoRNzHihyaDVYgAs0dQbvVEu1gitKpamzYdMLFtc5h8PFZSVEB"; - private static byte[] bytes = randomString.substring(0, THRESHOLD).getBytes(); - private static byte[] compressedBytes = randomString.substring(0, THRESHOLD / 2).getBytes(); + private static final String RANDOM_STRING = "roZzozzLiR7GCEjcB0UsRUNgBAip8cSLGXQSo3RQvbIDoxOaaRs4hrec2s5rMPWgTPRY4UnE959worEtyhRjwUFnRnVuNFZ554yuPQCbI69qFkQX7MmrB4blmpSnFeGjWKjFjIRLFNVSsQBYMkr5jT4T83uVtuGumsjACVrpcilihdd194H8Y71rQcrXZoTQtw5OvmPicbwptawpHoRNzHihyaDVYgAs0dQbvVEu1gitKpamzYdMLFtc5h8PFZSVEB"; + private static final byte[] BYTES = RANDOM_STRING.substring(0, THRESHOLD).getBytes(); + private static final byte[] COMPRESSED_BYTES = RANDOM_STRING.substring(0, THRESHOLD / 2).getBytes(); private AzureBlobWriterMetrics mockMetrics; private Compression mockCompression; @@ -91,7 +91,7 @@ public void setup() throws Exception { mockMetrics = mock(AzureBlobWriterMetrics.class); mockCompression = mock(Compression.class); - doReturn(compressedBytes).when(mockCompression).compress(bytes); + doReturn(COMPRESSED_BYTES).when(mockCompression).compress(BYTES); azureBlobOutputStream = spy(new AzureBlobOutputStream(mockBlobAsyncClient, threadPool, mockMetrics, 60000, THRESHOLD, mockByteArrayOutputStream, mockCompression)); @@ -108,14 +108,15 @@ public void testWrite() { @Test public void testWriteLargerThanThreshold() { - byte[] largeRecord = randomString.substring(0, 2 * THRESHOLD).getBytes(); - byte[] largeRecordFirstHalf = randomString.substring(0, THRESHOLD).getBytes(); - byte[] largeRecordSecondHalf = randomString.substring(THRESHOLD, 2 * THRESHOLD).getBytes(); + byte[] largeRecord = RANDOM_STRING.substring(0, 2 * THRESHOLD).getBytes(); + byte[] largeRecordFirstHalf = RANDOM_STRING.substring(0, THRESHOLD).getBytes(); + byte[] largeRecordSecondHalf = RANDOM_STRING.substring(THRESHOLD, 2 * THRESHOLD).getBytes(); - byte[] compressB = randomString.substring(0, THRESHOLD / 2).getBytes(); + byte[] compressB1 = RANDOM_STRING.substring(0, THRESHOLD / 2).getBytes(); + byte[] compressB2 = RANDOM_STRING.substring(THRESHOLD / 2, THRESHOLD).getBytes(); - doReturn(compressB).when(mockCompression).compress(largeRecordFirstHalf); - doReturn(compressB).when(mockCompression).compress(largeRecordSecondHalf); + doReturn(compressB1).when(mockCompression).compress(largeRecordFirstHalf); + doReturn(compressB2).when(mockCompression).compress(largeRecordSecondHalf); azureBlobOutputStream.write(largeRecord, 0, 2 * THRESHOLD); // azureBlobOutputStream.close waits on the CompletableFuture which does the actual stageBlock in uploadBlockAsync @@ -125,11 +126,10 @@ public void testWriteLargerThanThreshold() { verify(mockCompression).compress(largeRecordFirstHalf); verify(mockCompression).compress(largeRecordSecondHalf); ArgumentCaptor argument = ArgumentCaptor.forClass(Flux.class); - verify(mockBlobAsyncClient).stageBlock(eq(blockIdEncoded(0)), argument.capture(), eq((long) compressB.length)); - verify(mockBlobAsyncClient).stageBlock(eq(blockIdEncoded(1)), argument.capture(), eq((long) compressB.length)); - argument.getAllValues().forEach(flux -> { - Assert.assertEquals(ByteBuffer.wrap(compressB), flux.blockFirst()); - }); + verify(mockBlobAsyncClient).stageBlock(eq(blockIdEncoded(0)), argument.capture(), eq((long) compressB1.length)); + verify(mockBlobAsyncClient).stageBlock(eq(blockIdEncoded(1)), argument.capture(), eq((long) compressB2.length)); + Assert.assertEquals(ByteBuffer.wrap(compressB1), argument.getAllValues().get(0).blockFirst()); + Assert.assertEquals(ByteBuffer.wrap(compressB2), argument.getAllValues().get(1).blockFirst()); verify(mockMetrics).updateWriteByteMetrics(2 * THRESHOLD); verify(mockMetrics, times(2)).updateAzureUploadMetrics(); } @@ -175,15 +175,15 @@ public void testWriteLargeRecordWithSmallRecordInBuffer() { @Test public void testWriteThresholdCrossed() throws Exception { - azureBlobOutputStream.write(bytes, 0, THRESHOLD / 2); - azureBlobOutputStream.write(bytes, THRESHOLD / 2, THRESHOLD / 2); + azureBlobOutputStream.write(BYTES, 0, THRESHOLD / 2); + azureBlobOutputStream.write(BYTES, THRESHOLD / 2, THRESHOLD / 2); // azureBlobOutputStream.close waits on the CompletableFuture which does the actual stageBlock in uploadBlockAsync azureBlobOutputStream.close(); - verify(mockCompression).compress(bytes); + verify(mockCompression).compress(BYTES); ArgumentCaptor argument = ArgumentCaptor.forClass(Flux.class); - verify(mockBlobAsyncClient).stageBlock(eq(blockIdEncoded(0)), argument.capture(), eq((long) compressedBytes.length)); // since size of byte[] written is less than threshold - Assert.assertEquals(ByteBuffer.wrap(compressedBytes), ((Flux) argument.getValue()).blockFirst()); + verify(mockBlobAsyncClient).stageBlock(eq(blockIdEncoded(0)), argument.capture(), eq((long) COMPRESSED_BYTES.length)); // since size of byte[] written is less than threshold + Assert.assertEquals(ByteBuffer.wrap(COMPRESSED_BYTES), ((Flux) argument.getValue()).blockFirst()); verify(mockMetrics, times(2)).updateWriteByteMetrics(THRESHOLD / 2); verify(mockMetrics, times(1)).updateAzureUploadMetrics(); } @@ -200,7 +200,7 @@ public void testWriteFailed() { @Test public void testClose() { - azureBlobOutputStream.write(bytes, 0, THRESHOLD); + azureBlobOutputStream.write(BYTES, 0, THRESHOLD); int blockNum = 0; String blockId = String.format("%05d", blockNum); String blockIdEncoded = Base64.getEncoder().encodeToString(blockId.getBytes()); @@ -220,8 +220,8 @@ public void testClose() { @Test public void testCloseMultipleBlocks() { - azureBlobOutputStream.write(bytes, 0, THRESHOLD); - azureBlobOutputStream.write(bytes, 0, THRESHOLD); + azureBlobOutputStream.write(BYTES, 0, THRESHOLD); + azureBlobOutputStream.write(BYTES, 0, THRESHOLD); int blockNum = 0; String blockId = String.format("%05d", blockNum); @@ -255,14 +255,14 @@ public void testCloseFailed() { @Test public void testMultipleClose() { - azureBlobOutputStream.write(bytes, 0, THRESHOLD); + azureBlobOutputStream.write(BYTES, 0, THRESHOLD); azureBlobOutputStream.close(); azureBlobOutputStream.close(); } @Test public void testFlush() throws Exception { - azureBlobOutputStream.write(bytes); + azureBlobOutputStream.write(BYTES); azureBlobOutputStream.flush(); // azureBlobOutputStream.close waits on the CompletableFuture which does the actual stageBlock in uploadBlockAsync azureBlobOutputStream.close(); @@ -271,16 +271,16 @@ public void testFlush() throws Exception { String blockId = String.format("%05d", blockNum); String blockIdEncoded = Base64.getEncoder().encodeToString(blockId.getBytes()); - verify(mockCompression).compress(bytes); + verify(mockCompression).compress(BYTES); ArgumentCaptor argument = ArgumentCaptor.forClass(Flux.class); - verify(mockBlobAsyncClient).stageBlock(eq(blockIdEncoded), argument.capture(), eq((long) compressedBytes.length)); // since size of byte[] written is less than threshold - Assert.assertEquals(ByteBuffer.wrap(compressedBytes), ((Flux) argument.getValue()).blockFirst()); + verify(mockBlobAsyncClient).stageBlock(eq(blockIdEncoded), argument.capture(), eq((long) COMPRESSED_BYTES.length)); // since size of byte[] written is less than threshold + Assert.assertEquals(ByteBuffer.wrap(COMPRESSED_BYTES), ((Flux) argument.getValue()).blockFirst()); verify(mockMetrics).updateAzureUploadMetrics(); } @Test (expected = RuntimeException.class) public void testFlushFailed() throws IOException { - azureBlobOutputStream.write(bytes); + azureBlobOutputStream.write(BYTES); when(mockBlobAsyncClient.stageBlock(anyString(), any(), anyLong())) .thenReturn(Mono.error(new Exception("Test Failed"))); @@ -305,7 +305,7 @@ public void testWriteAfterReleaseBuffer() throws Exception { @Test public void testCloseAfterReleaseBuffer() throws Exception { - azureBlobOutputStream.write(bytes, 0, 100); + azureBlobOutputStream.write(BYTES, 0, 100); azureBlobOutputStream.releaseBuffer(); azureBlobOutputStream.close(); // mockByteArrayOutputStream.close called only once during releaseBuffer and not during azureBlobOutputStream.close @@ -324,18 +324,18 @@ public void testFlushAfterReleaseBuffer() throws Exception { @Test public void testGetSize() throws Exception { Assert.assertEquals(0, azureBlobOutputStream.getSize()); - azureBlobOutputStream.write(bytes, 0, bytes.length); - Assert.assertEquals(bytes.length, azureBlobOutputStream.getSize()); + azureBlobOutputStream.write(BYTES, 0, BYTES.length); + Assert.assertEquals(BYTES.length, azureBlobOutputStream.getSize()); } @Test public void testGetSizeAfterFlush() throws Exception { - azureBlobOutputStream.write(bytes, 0, bytes.length); - Assert.assertEquals(bytes.length, azureBlobOutputStream.getSize()); + azureBlobOutputStream.write(BYTES, 0, BYTES.length); + Assert.assertEquals(BYTES.length, azureBlobOutputStream.getSize()); azureBlobOutputStream.flush(); - Assert.assertEquals(bytes.length, azureBlobOutputStream.getSize()); - azureBlobOutputStream.write(bytes, 0, bytes.length - 10); - Assert.assertEquals(bytes.length + bytes.length - 10, azureBlobOutputStream.getSize()); + Assert.assertEquals(BYTES.length, azureBlobOutputStream.getSize()); + azureBlobOutputStream.write(BYTES, 0, BYTES.length - 10); + Assert.assertEquals(BYTES.length + BYTES.length - 10, azureBlobOutputStream.getSize()); } private String blockIdEncoded(int blockNum) { diff --git a/samza-azure/src/test/java/org/apache/samza/system/azureblob/producer/TestAzureBlobSystemProducer.java b/samza-azure/src/test/java/org/apache/samza/system/azureblob/producer/TestAzureBlobSystemProducer.java index e3e8d542ea..2969cd02e5 100644 --- a/samza-azure/src/test/java/org/apache/samza/system/azureblob/producer/TestAzureBlobSystemProducer.java +++ b/samza-azure/src/test/java/org/apache/samza/system/azureblob/producer/TestAzureBlobSystemProducer.java @@ -107,7 +107,6 @@ public void testStart() { verify(systemProducer).setupAzureContainer(ACCOUNT_NAME, ACCOUNT_KEY); } - @Test(expected = SystemProducerException.class) public void testMultipleStart() { systemProducer.start(); systemProducer.start(); @@ -139,11 +138,6 @@ public void testStopBeforeFlush() throws Exception { verify(mockAzureWriter).close(); // called during flush in STOP } - @Test(expected = SystemProducerException.class) - public void testStopBeforeStart() { - systemProducer.stop(); - } - @Test(expected = SystemProducerException.class) public void testStopWhenThreadpoolShutdownFails() throws Exception { doThrow(new SecurityException("failed")).when(mockThreadPoolExecutor).shutdown(); From 3b30796d3a36ebcbefc97ebb0471fdc415e5b55e Mon Sep 17 00:00:00 2001 From: Manasa Date: Mon, 6 Jan 2020 12:15:51 -0800 Subject: [PATCH 6/9] fix comments --- .../system/azureblob/avro/AzureBlobAvroWriter.java | 2 ++ .../azureblob/avro/AzureBlobOutputStream.java | 13 ++++--------- .../azureblob/compression/CompressionFactory.java | 7 ++----- .../azureblob/avro/TestAzureBlobOutputStream.java | 11 ++++++----- 4 files changed, 14 insertions(+), 19 deletions(-) diff --git a/samza-azure/src/main/java/org/apache/samza/system/azureblob/avro/AzureBlobAvroWriter.java b/samza-azure/src/main/java/org/apache/samza/system/azureblob/avro/AzureBlobAvroWriter.java index 680e404de9..e97b2148b5 100644 --- a/samza-azure/src/main/java/org/apache/samza/system/azureblob/avro/AzureBlobAvroWriter.java +++ b/samza-azure/src/main/java/org/apache/samza/system/azureblob/avro/AzureBlobAvroWriter.java @@ -108,6 +108,8 @@ public class AzureBlobAvroWriter implements AzureBlobWriter { private final long maxRecordsPerBlob; private final boolean useRandomStringInBlobName; private final Object currentDataFileWriterLock = new Object(); + // when either maxRecordsPerBlob or maxBlobSize is exceeded then a new blob is created + // with the same blobURLPrefix and blobNumber is suffixed to the blobURL to differentiate between the blobs private volatile long blobNumber = 0; private volatile long recordsInCurrentBlob = 0; diff --git a/samza-azure/src/main/java/org/apache/samza/system/azureblob/avro/AzureBlobOutputStream.java b/samza-azure/src/main/java/org/apache/samza/system/azureblob/avro/AzureBlobOutputStream.java index 5268933e08..dbe1cf0f68 100644 --- a/samza-azure/src/main/java/org/apache/samza/system/azureblob/avro/AzureBlobOutputStream.java +++ b/samza-azure/src/main/java/org/apache/samza/system/azureblob/avro/AzureBlobOutputStream.java @@ -277,24 +277,19 @@ public void run() { + " failed for blockid: " + blockId + " due to exception. RetryCount: " + attemptCount; LOG.error(msg, e); if (attemptCount == MAX_ATTEMPT) { - throw new AzureException("Exceeded number of retries. Max attempts is: " + MAX_ATTEMPT, e); + throw new AzureException("Exceeded number of attempts. Max attempts is: " + MAX_ATTEMPT, e); } } } } }, blobThreadPool); - if (future.isDone()) { - LOG.info("Upload block for blob: {} with blockid: {} finished.", blobAsyncClient.getBlobUrl().toString(), blockId); - } else { - pendingUpload.add(future); - } + pendingUpload.add(future); + future.handle((aVoid, throwable) -> { if (throwable == null) { LOG.info("Upload block for blob: {} with blockid: {} finished.", blobAsyncClient.getBlobUrl().toString(), blockId); - if (pendingUpload.contains(future)) { - pendingUpload.remove(future); - } + pendingUpload.remove(future); return aVoid; } else { throw new AzureException("Blob upload failed for blob " + blobAsyncClient.getBlobUrl().toString() diff --git a/samza-azure/src/main/java/org/apache/samza/system/azureblob/compression/CompressionFactory.java b/samza-azure/src/main/java/org/apache/samza/system/azureblob/compression/CompressionFactory.java index bcba6c7c5f..0f738fcc09 100644 --- a/samza-azure/src/main/java/org/apache/samza/system/azureblob/compression/CompressionFactory.java +++ b/samza-azure/src/main/java/org/apache/samza/system/azureblob/compression/CompressionFactory.java @@ -25,14 +25,11 @@ * based on the {@link org.apache.samza.system.azureblob.compression.CompressionType}. */ public class CompressionFactory { - private static CompressionFactory compressionFactoryInstance = null; + private final static CompressionFactory COMPRESSION_FACTORY_INSTANCE = new CompressionFactory(); private CompressionFactory() {} public static CompressionFactory getInstance() { - if (compressionFactoryInstance == null) { - compressionFactoryInstance = new CompressionFactory(); - } - return compressionFactoryInstance; + return COMPRESSION_FACTORY_INSTANCE; } public Compression getCompression(CompressionType compressionType) { diff --git a/samza-azure/src/test/java/org/apache/samza/system/azureblob/avro/TestAzureBlobOutputStream.java b/samza-azure/src/test/java/org/apache/samza/system/azureblob/avro/TestAzureBlobOutputStream.java index a00f563d1c..fa21934e29 100644 --- a/samza-azure/src/test/java/org/apache/samza/system/azureblob/avro/TestAzureBlobOutputStream.java +++ b/samza-azure/src/test/java/org/apache/samza/system/azureblob/avro/TestAzureBlobOutputStream.java @@ -125,11 +125,12 @@ public void testWriteLargerThanThreshold() { // invoked 2 times for the data which is 2*threshold verify(mockCompression).compress(largeRecordFirstHalf); verify(mockCompression).compress(largeRecordSecondHalf); - ArgumentCaptor argument = ArgumentCaptor.forClass(Flux.class); - verify(mockBlobAsyncClient).stageBlock(eq(blockIdEncoded(0)), argument.capture(), eq((long) compressB1.length)); - verify(mockBlobAsyncClient).stageBlock(eq(blockIdEncoded(1)), argument.capture(), eq((long) compressB2.length)); - Assert.assertEquals(ByteBuffer.wrap(compressB1), argument.getAllValues().get(0).blockFirst()); - Assert.assertEquals(ByteBuffer.wrap(compressB2), argument.getAllValues().get(1).blockFirst()); + ArgumentCaptor argument0 = ArgumentCaptor.forClass(Flux.class); + ArgumentCaptor argument1 = ArgumentCaptor.forClass(Flux.class); + verify(mockBlobAsyncClient).stageBlock(eq(blockIdEncoded(0)), argument0.capture(), eq((long) compressB1.length)); + verify(mockBlobAsyncClient).stageBlock(eq(blockIdEncoded(1)), argument1.capture(), eq((long) compressB2.length)); + Assert.assertEquals(ByteBuffer.wrap(compressB1), argument0.getAllValues().get(0).blockFirst()); + Assert.assertEquals(ByteBuffer.wrap(compressB2), argument1.getAllValues().get(0).blockFirst()); verify(mockMetrics).updateWriteByteMetrics(2 * THRESHOLD); verify(mockMetrics, times(2)).updateAzureUploadMetrics(); } From 46712c16d7b03def8545b0278656998664d6fd37 Mon Sep 17 00:00:00 2001 From: Manasa Date: Wed, 8 Jan 2020 09:28:30 -0800 Subject: [PATCH 7/9] remove blobnumber --- .../azureblob/avro/AzureBlobAvroWriter.java | 16 +++++--------- .../azureblob/avro/AzureBlobOutputStream.java | 2 +- .../avro/TestAzureBlobAvroWriter.java | 22 +++++++++---------- 3 files changed, 18 insertions(+), 22 deletions(-) diff --git a/samza-azure/src/main/java/org/apache/samza/system/azureblob/avro/AzureBlobAvroWriter.java b/samza-azure/src/main/java/org/apache/samza/system/azureblob/avro/AzureBlobAvroWriter.java index e97b2148b5..7f9a92684a 100644 --- a/samza-azure/src/main/java/org/apache/samza/system/azureblob/avro/AzureBlobAvroWriter.java +++ b/samza-azure/src/main/java/org/apache/samza/system/azureblob/avro/AzureBlobAvroWriter.java @@ -77,8 +77,8 @@ public class AzureBlobAvroWriter implements AzureBlobWriter { private static final Logger LOG = LoggerFactory.getLogger(AzureBlobAvroWriter.class); private static final String PUBLISHED_FILE_NAME_DATE_FORMAT = "yyyy/MM/dd/HH/mm-ss"; - private static final String BLOB_NAME_AVRO = "%s/%s-%s.avro%s"; - private static final String BLOB_NAME_RANDOM_STRING_AVRO = "%s/%s-%s-%s.avro%s"; + private static final String BLOB_NAME_AVRO = "%s/%s.avro%s"; + private static final String BLOB_NAME_RANDOM_STRING_AVRO = "%s/%s-%s.avro%s"; private static final SimpleDateFormat UTC_FORMATTER = buildUTCFormatter(); // Avro's DataFileWriter has internal buffers and also adds metadata. @@ -108,9 +108,6 @@ public class AzureBlobAvroWriter implements AzureBlobWriter { private final long maxRecordsPerBlob; private final boolean useRandomStringInBlobName; private final Object currentDataFileWriterLock = new Object(); - // when either maxRecordsPerBlob or maxBlobSize is exceeded then a new blob is created - // with the same blobURLPrefix and blobNumber is suffixed to the blobURL to differentiate between the blobs - private volatile long blobNumber = 0; private volatile long recordsInCurrentBlob = 0; public AzureBlobAvroWriter(BlobContainerAsyncClient containerAsyncClient, String blobURLPrefix, @@ -220,7 +217,7 @@ public void close() { Executor blobThreadPool, int maxBlockFlushThresholdSize, int flushTimeoutMs, String blobURLPrefix, DataFileWriter dataFileWriter, AzureBlobOutputStream azureBlobOutputStream, BlockBlobAsyncClient blockBlobAsyncClient, - long maxBlobSize, long maxRecordsPerBlob, Compression compression) { + long maxBlobSize, long maxRecordsPerBlob, Compression compression, boolean useRandomStringInBlobName) { if (dataFileWriter == null || azureBlobOutputStream == null || blockBlobAsyncClient == null) { this.currentBlobWriterComponents = null; } else { @@ -235,7 +232,7 @@ public void close() { this.flushTimeoutMs = flushTimeoutMs; this.compression = compression; this.containerAsyncClient = containerAsyncClient; - this.useRandomStringInBlobName = false; + this.useRandomStringInBlobName = useRandomStringInBlobName; this.maxBlobSize = maxBlobSize; this.maxRecordsPerBlob = maxRecordsPerBlob; } @@ -311,15 +308,14 @@ private void startNextBlob(Optional optionalIndexedRecord) throws String blobURL; if (useRandomStringInBlobName) { blobURL = String.format(BLOB_NAME_RANDOM_STRING_AVRO, blobURLPrefix, - UTC_FORMATTER.format(System.currentTimeMillis()), blobNumber, UUID.randomUUID().toString().substring(0, 8), + UTC_FORMATTER.format(System.currentTimeMillis()), UUID.randomUUID().toString().substring(0, 8), compression.getFileExtension()); } else { blobURL = String.format(BLOB_NAME_AVRO, blobURLPrefix, - UTC_FORMATTER.format(System.currentTimeMillis()), blobNumber, compression.getFileExtension()); + UTC_FORMATTER.format(System.currentTimeMillis()), compression.getFileExtension()); } LOG.info("Creating new blob: {}", blobURL); BlockBlobAsyncClient blockBlobAsyncClient = containerAsyncClient.getBlobAsyncClient(blobURL).getBlockBlobAsyncClient(); - blobNumber++; DataFileWriter dataFileWriter = new DataFileWriter<>(datumWriter); AzureBlobOutputStream azureBlobOutputStream = new AzureBlobOutputStream(blockBlobAsyncClient, blobThreadPool, metrics, diff --git a/samza-azure/src/main/java/org/apache/samza/system/azureblob/avro/AzureBlobOutputStream.java b/samza-azure/src/main/java/org/apache/samza/system/azureblob/avro/AzureBlobOutputStream.java index dbe1cf0f68..2b25f96652 100644 --- a/samza-azure/src/main/java/org/apache/samza/system/azureblob/avro/AzureBlobOutputStream.java +++ b/samza-azure/src/main/java/org/apache/samza/system/azureblob/avro/AzureBlobOutputStream.java @@ -274,7 +274,7 @@ public void run() { } catch (Exception e) { attemptCount += 1; String msg = "Upload block for blob: " + blobAsyncClient.getBlobUrl().toString() - + " failed for blockid: " + blockId + " due to exception. RetryCount: " + attemptCount; + + " failed for blockid: " + blockId + " due to exception. AttemptCount: " + attemptCount; LOG.error(msg, e); if (attemptCount == MAX_ATTEMPT) { throw new AzureException("Exceeded number of attempts. Max attempts is: " + MAX_ATTEMPT, e); diff --git a/samza-azure/src/test/java/org/apache/samza/system/azureblob/avro/TestAzureBlobAvroWriter.java b/samza-azure/src/test/java/org/apache/samza/system/azureblob/avro/TestAzureBlobAvroWriter.java index 265847e954..e26a77d98c 100644 --- a/samza-azure/src/test/java/org/apache/samza/system/azureblob/avro/TestAzureBlobAvroWriter.java +++ b/samza-azure/src/test/java/org/apache/samza/system/azureblob/avro/TestAzureBlobAvroWriter.java @@ -52,7 +52,6 @@ import org.junit.Test; import org.junit.runner.RunWith; import org.mockito.ArgumentCaptor; -import org.mockito.Matchers; import org.powermock.api.mockito.PowerMockito; import org.powermock.core.classloader.annotations.PrepareForTest; import org.powermock.modules.junit4.PowerMockRunner; @@ -147,7 +146,7 @@ public void setup() throws Exception { azureBlobAvroWriter = spy(new AzureBlobAvroWriter(mockContainerAsyncClient, mock(AzureBlobWriterMetrics.class), threadPool, THRESHOLD, 60000, "test", mockDataFileWriter, mockAzureBlobOutputStream, mockBlockBlobAsyncClient, Long.MAX_VALUE, - Long.MAX_VALUE, mockCompression)); // keeping blob size and number of records unlimited + Long.MAX_VALUE, mockCompression, false)); // keeping blob size and number of records unlimited doReturn(encodedRecord).when(azureBlobAvroWriter).encodeRecord((IndexedRecord) ome.getMessage()); } @Test @@ -187,7 +186,7 @@ public void testWriteByteArrayWithoutSchema() throws Exception { azureBlobAvroWriter = spy(new AzureBlobAvroWriter(PowerMockito.mock(BlobContainerAsyncClient.class), mock(AzureBlobWriterMetrics.class), threadPool, THRESHOLD, 60000, "test", - null, null, null, 1000, 100, mockCompression)); + null, null, null, 1000, 100, mockCompression, false)); OutgoingMessageEnvelope omeEncoded = new OutgoingMessageEnvelope(new SystemStream(SYSTEM_NAME, "Topic1"), new byte[100]); azureBlobAvroWriter.write(omeEncoded); } @@ -241,13 +240,13 @@ public void testMaxBlobSizeExceeded() throws Exception { BlobContainerAsyncClient mockContainerClient = PowerMockito.mock(BlobContainerAsyncClient.class); azureBlobAvroWriter = spy(new AzureBlobAvroWriter(mockContainerClient, mockMetrics, threadPool, THRESHOLD, 60000, blobUrlPrefix, - null, null, null, maxBlobSize, 10, mockCompression)); + null, null, null, maxBlobSize, 10, mockCompression, true)); DataFileWriter mockDataFileWriter1 = mock(DataFileWriter.class); PowerMockito.whenNew(DataFileWriter.class).withAnyArguments().thenReturn(mockDataFileWriter1); BlobAsyncClient mockBlobAsyncClient1 = mock(BlobAsyncClient.class); - doReturn(mockBlobAsyncClient1).when(mockContainerClient).getBlobAsyncClient(Matchers.endsWith("0.avro.gz")); + doReturn(mockBlobAsyncClient1).when(mockContainerClient).getBlobAsyncClient(anyString()); BlockBlobAsyncClient mockBlockBlobAsyncClient1 = mock(BlockBlobAsyncClient.class); doReturn(mockBlockBlobAsyncClient1).when(mockBlobAsyncClient1).getBlockBlobAsyncClient(); @@ -265,7 +264,7 @@ public void testMaxBlobSizeExceeded() throws Exception { PowerMockito.whenNew(DataFileWriter.class).withAnyArguments().thenReturn(mockDataFileWriter2); BlobAsyncClient mockBlobAsyncClient2 = mock(BlobAsyncClient.class); - doReturn(mockBlobAsyncClient2).when(mockContainerClient).getBlobAsyncClient(Matchers.endsWith("1.avro.gz")); + doReturn(mockBlobAsyncClient2).when(mockContainerClient).getBlobAsyncClient(anyString()); BlockBlobAsyncClient mockBlockBlobAsyncClient2 = mock(BlockBlobAsyncClient.class); doReturn(mockBlockBlobAsyncClient2).when(mockBlobAsyncClient2).getBlockBlobAsyncClient(); @@ -302,13 +301,13 @@ public void testRecordLimitExceeded() throws Exception { BlobContainerAsyncClient mockContainerClient = PowerMockito.mock(BlobContainerAsyncClient.class); azureBlobAvroWriter = spy(new AzureBlobAvroWriter(mockContainerClient, mockMetrics, threadPool, THRESHOLD, 60000, blobUrlPrefix, - null, null, null, maxBlobSize, maxRecordsPerBlob, mockCompression)); + null, null, null, maxBlobSize, maxRecordsPerBlob, mockCompression, true)); DataFileWriter mockDataFileWriter1 = mock(DataFileWriter.class); PowerMockito.whenNew(DataFileWriter.class).withAnyArguments().thenReturn(mockDataFileWriter1); BlobAsyncClient mockBlobAsyncClient1 = mock(BlobAsyncClient.class); - doReturn(mockBlobAsyncClient1).when(mockContainerClient).getBlobAsyncClient(Matchers.endsWith("0.avro.gz")); + doReturn(mockBlobAsyncClient1).when(mockContainerClient).getBlobAsyncClient(anyString()); BlockBlobAsyncClient mockBlockBlobAsyncClient1 = mock(BlockBlobAsyncClient.class); doReturn(mockBlockBlobAsyncClient1).when(mockBlobAsyncClient1).getBlockBlobAsyncClient(); @@ -329,7 +328,7 @@ public void testRecordLimitExceeded() throws Exception { PowerMockito.whenNew(DataFileWriter.class).withAnyArguments().thenReturn(mockDataFileWriter2); BlobAsyncClient mockBlobAsyncClient2 = mock(BlobAsyncClient.class); - doReturn(mockBlobAsyncClient2).when(mockContainerClient).getBlobAsyncClient(Matchers.endsWith("1.avro.gz")); + doReturn(mockBlobAsyncClient2).when(mockContainerClient).getBlobAsyncClient(anyString()); BlockBlobAsyncClient mockBlockBlobAsyncClient2 = mock(BlockBlobAsyncClient.class); doReturn(mockBlockBlobAsyncClient2).when(mockBlobAsyncClient2).getBlockBlobAsyncClient(); @@ -364,7 +363,8 @@ public void testMultipleBlobClose() throws Exception { BlobContainerAsyncClient mockContainerClient = PowerMockito.mock(BlobContainerAsyncClient.class); azureBlobAvroWriter = spy(new AzureBlobAvroWriter(mockContainerClient, mock(AzureBlobWriterMetrics.class), threadPool, THRESHOLD, 60000, blobUrlPrefix, - mockDataFileWriter, mockAzureBlobOutputStream, mockBlockBlobAsyncClient, maxBlobSize, maxRecordsPerBlob, mockCompression)); + mockDataFileWriter, mockAzureBlobOutputStream, mockBlockBlobAsyncClient, maxBlobSize, maxRecordsPerBlob, + mockCompression, false)); DataFileWriter mockDataFileWriter2 = mock(DataFileWriter.class); AzureBlobOutputStream mockAzureBlobOutputStream2 = mock(AzureBlobOutputStream.class); @@ -390,7 +390,7 @@ public void testEncodeRecord() throws Exception { azureBlobAvroWriter = spy(new AzureBlobAvroWriter(PowerMockito.mock(BlobContainerAsyncClient.class), mock(AzureBlobWriterMetrics.class), threadPool, THRESHOLD, 60000, "test", mockDataFileWriter, mockAzureBlobOutputStream, mockBlockBlobAsyncClient, - Long.MAX_VALUE, Long.MAX_VALUE, mockCompression)); + Long.MAX_VALUE, Long.MAX_VALUE, mockCompression, false)); IndexedRecord record = new GenericRecordEvent(); Assert.assertTrue(Arrays.equals(encodeRecord(record), azureBlobAvroWriter.encodeRecord(record))); } From e7b8dfd7de4a558de3c1fb332b7a3f0648ebb5c0 Mon Sep 17 00:00:00 2001 From: Manasa Date: Wed, 15 Jan 2020 11:06:59 -0800 Subject: [PATCH 8/9] fix comments for test --- .../azureblob/avro/TestAzureBlobAvroWriter.java | 11 +++++++---- 1 file changed, 7 insertions(+), 4 deletions(-) diff --git a/samza-azure/src/test/java/org/apache/samza/system/azureblob/avro/TestAzureBlobAvroWriter.java b/samza-azure/src/test/java/org/apache/samza/system/azureblob/avro/TestAzureBlobAvroWriter.java index e26a77d98c..b4ef4b4e95 100644 --- a/samza-azure/src/test/java/org/apache/samza/system/azureblob/avro/TestAzureBlobAvroWriter.java +++ b/samza-azure/src/test/java/org/apache/samza/system/azureblob/avro/TestAzureBlobAvroWriter.java @@ -52,6 +52,7 @@ import org.junit.Test; import org.junit.runner.RunWith; import org.mockito.ArgumentCaptor; +import org.mockito.Matchers; import org.powermock.api.mockito.PowerMockito; import org.powermock.core.classloader.annotations.PrepareForTest; import org.powermock.modules.junit4.PowerMockRunner; @@ -235,6 +236,7 @@ public void testFlushWhenDataFileWriterFails() throws Exception { @Test public void testMaxBlobSizeExceeded() throws Exception { String blobUrlPrefix = "test"; + String blobNameRegex = "test/[0-9]{4}/[0-9]{2}/[0-9]{2}/[0-9]{2}/[0-9]{2}-[0-9]{2}-.{8}.avro.gz"; long maxBlobSize = 1000; AzureBlobWriterMetrics mockMetrics = mock(AzureBlobWriterMetrics.class); BlobContainerAsyncClient mockContainerClient = PowerMockito.mock(BlobContainerAsyncClient.class); @@ -246,7 +248,7 @@ public void testMaxBlobSizeExceeded() throws Exception { PowerMockito.whenNew(DataFileWriter.class).withAnyArguments().thenReturn(mockDataFileWriter1); BlobAsyncClient mockBlobAsyncClient1 = mock(BlobAsyncClient.class); - doReturn(mockBlobAsyncClient1).when(mockContainerClient).getBlobAsyncClient(anyString()); + doReturn(mockBlobAsyncClient1).when(mockContainerClient).getBlobAsyncClient(Matchers.matches(blobNameRegex)); BlockBlobAsyncClient mockBlockBlobAsyncClient1 = mock(BlockBlobAsyncClient.class); doReturn(mockBlockBlobAsyncClient1).when(mockBlobAsyncClient1).getBlockBlobAsyncClient(); @@ -264,7 +266,7 @@ public void testMaxBlobSizeExceeded() throws Exception { PowerMockito.whenNew(DataFileWriter.class).withAnyArguments().thenReturn(mockDataFileWriter2); BlobAsyncClient mockBlobAsyncClient2 = mock(BlobAsyncClient.class); - doReturn(mockBlobAsyncClient2).when(mockContainerClient).getBlobAsyncClient(anyString()); + doReturn(mockBlobAsyncClient2).when(mockContainerClient).getBlobAsyncClient(Matchers.matches(blobNameRegex)); BlockBlobAsyncClient mockBlockBlobAsyncClient2 = mock(BlockBlobAsyncClient.class); doReturn(mockBlockBlobAsyncClient2).when(mockBlobAsyncClient2).getBlockBlobAsyncClient(); @@ -295,6 +297,7 @@ public void testMaxBlobSizeExceeded() throws Exception { @Test public void testRecordLimitExceeded() throws Exception { String blobUrlPrefix = "test"; + String blobNameRegex = "test/[0-9]{4}/[0-9]{2}/[0-9]{2}/[0-9]{2}/[0-9]{2}-[0-9]{2}-.{8}.avro.gz"; AzureBlobWriterMetrics mockMetrics = mock(AzureBlobWriterMetrics.class); long maxBlobSize = AzureBlobAvroWriter.DATAFILEWRITER_OVERHEAD + 1000; long maxRecordsPerBlob = 10; @@ -307,7 +310,7 @@ public void testRecordLimitExceeded() throws Exception { PowerMockito.whenNew(DataFileWriter.class).withAnyArguments().thenReturn(mockDataFileWriter1); BlobAsyncClient mockBlobAsyncClient1 = mock(BlobAsyncClient.class); - doReturn(mockBlobAsyncClient1).when(mockContainerClient).getBlobAsyncClient(anyString()); + doReturn(mockBlobAsyncClient1).when(mockContainerClient).getBlobAsyncClient(Matchers.matches(blobNameRegex)); BlockBlobAsyncClient mockBlockBlobAsyncClient1 = mock(BlockBlobAsyncClient.class); doReturn(mockBlockBlobAsyncClient1).when(mockBlobAsyncClient1).getBlockBlobAsyncClient(); @@ -328,7 +331,7 @@ public void testRecordLimitExceeded() throws Exception { PowerMockito.whenNew(DataFileWriter.class).withAnyArguments().thenReturn(mockDataFileWriter2); BlobAsyncClient mockBlobAsyncClient2 = mock(BlobAsyncClient.class); - doReturn(mockBlobAsyncClient2).when(mockContainerClient).getBlobAsyncClient(anyString()); + doReturn(mockBlobAsyncClient2).when(mockContainerClient).getBlobAsyncClient(Matchers.matches(blobNameRegex)); BlockBlobAsyncClient mockBlockBlobAsyncClient2 = mock(BlockBlobAsyncClient.class); doReturn(mockBlockBlobAsyncClient2).when(mockBlobAsyncClient2).getBlockBlobAsyncClient(); From a9f73ca9cf901b204732589f7e4bb29bbc567d01 Mon Sep 17 00:00:00 2001 From: Manasa Date: Wed, 15 Jan 2020 15:25:33 -0800 Subject: [PATCH 9/9] Trigger Travis Build