cameronlee314 commented on a change in pull request #1239: SAMZA-2421: Add 
SystemProducer for Azure Blob Storage
URL: https://github.com/apache/samza/pull/1239#discussion_r360991285
 
 

 ##########
 File path: 
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<String, Map<String, AzureBlobWriter>> 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<String, Object> sourceWriterCreationLockMap = new 
ConcurrentHashMap<>();
+  private final Map<String, ReadWriteLock> 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<Runnable>
+        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.");
+    }
 
 Review comment:
   Should this throw an exception in these cases? What do other 
`SystemProducer` impls do?

----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.
 
For queries about this service, please contact Infrastructure at:
[email protected]


With regards,
Apache Git Services

Reply via email to