steveloughran commented on a change in pull request #24934: [SPARK-28124] [SS] 
SQS source for Structured Streaming
URL: https://github.com/apache/spark/pull/24934#discussion_r307689936
 
 

 ##########
 File path: 
external/s3-sqs/src/main/scala/org/apache/spark/sql/sqs/SqsClient.scala
 ##########
 @@ -0,0 +1,267 @@
+/*
+ * 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.spark.sql.sqs
+
+import java.text.SimpleDateFormat
+import java.util.TimeZone
+import java.util.concurrent.TimeUnit
+
+import scala.collection.JavaConverters._
+
+import com.amazonaws.{AmazonClientException, AmazonServiceException, 
ClientConfiguration}
+import com.amazonaws.services.sqs.{AmazonSQS, AmazonSQSClientBuilder}
+import com.amazonaws.services.sqs.model.{DeleteMessageBatchRequestEntry, 
Message, ReceiveMessageRequest}
+import org.apache.hadoop.conf.Configuration
+import org.json4s.{DefaultFormats, MappingException}
+import org.json4s.JsonAST.JValue
+import org.json4s.jackson.JsonMethods.parse
+
+import org.apache.spark.SparkException
+import org.apache.spark.internal.Logging
+import org.apache.spark.util.ThreadUtils
+
+class SqsClient(sourceOptions: SqsSourceOptions,
+                hadoopConf: Configuration) extends Logging {
+
+  private val sqsFetchIntervalSeconds = sourceOptions.fetchIntervalSeconds
+  private val sqsLongPollWaitTimeSeconds = 
sourceOptions.longPollWaitTimeSeconds
+  private val sqsMaxRetries = sourceOptions.maxRetries
+  private val maxConnections = sourceOptions.maxConnections
+  private val ignoreFileDeletion = sourceOptions.ignoreFileDeletion
+  private val region = sourceOptions.region
+  val sqsUrl = sourceOptions.sqsUrl
+
+  @volatile var exception: Option[Exception] = None
+
+  private val timestampFormat = new 
SimpleDateFormat("yyyy-MM-dd'T'HH:mm:ss.SSS'Z'") // ISO8601
+  timestampFormat.setTimeZone(TimeZone.getTimeZone("UTC"))
+  private var retriesOnFailure = 0
+  private val sqsClient = createSqsClient()
+
+  val sqsScheduler = 
ThreadUtils.newDaemonSingleThreadScheduledExecutor("sqs-scheduler")
+
+  val sqsFileCache = new SqsFileCache(sourceOptions.maxFileAgeMs, 
sourceOptions.fileNameOnly)
+
+  val deleteMessageQueue = new 
java.util.concurrent.ConcurrentLinkedQueue[String]()
+
+  private val sqsFetchMessagesThread = new Runnable {
+    override def run(): Unit = {
+      try {
+        // Fetching messages from Amazon SQS
+        val newMessages = sqsFetchMessages()
+
+        // Filtering the new messages which are already not seen
+        if (newMessages.nonEmpty) {
+          newMessages.filter(message => sqsFileCache.isNewFile(message._1, 
message._2))
+            .foreach(message =>
+              sqsFileCache.add(message._1, MessageDescription(message._2, 
false, message._3)))
+        }
+      } catch {
+        case e: Exception =>
+          exception = Some(e)
+      }
+    }
+  }
+
+  sqsScheduler.scheduleWithFixedDelay(
+    sqsFetchMessagesThread,
+    0,
+    sqsFetchIntervalSeconds,
+    TimeUnit.SECONDS)
+
+  private def sqsFetchMessages(): Seq[(String, Long, String)] = {
+    val messageList = try {
+      val receiveMessageRequest = new ReceiveMessageRequest()
+        .withQueueUrl(sqsUrl)
+        .withWaitTimeSeconds(sqsLongPollWaitTimeSeconds)
+      val messages = 
sqsClient.receiveMessage(receiveMessageRequest).getMessages.asScala
+      retriesOnFailure = 0
+      logDebug(s"successfully received ${messages.size} messages")
+      messages
+    } catch {
+      case ase: AmazonServiceException =>
 
 Review comment:
   I'm going to point you at the S3A logic to interpret AWS exceptions and remap
   
https://github.com/steveloughran/hadoop/blob/trunk/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/S3AUtils.java#L179
   
   And then retry policy we've evolved over time
   
https://github.com/steveloughran/hadoop/blob/trunk/hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/S3ARetryPolicy.java#L158
   
   Each AWS service has its own failure modes which you get to learn over time.

----------------------------------------------------------------
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

---------------------------------------------------------------------
To unsubscribe, e-mail: [email protected]
For additional commands, e-mail: [email protected]

Reply via email to