@@ -15,18 +15,20 @@ package org.apache.spark.sql.pulsar
15
15
16
16
import java .{util => ju }
17
17
import java .io .Closeable
18
- import java .util .{ Optional , UUID }
18
+ import java .util .Optional
19
19
import java .util .concurrent .TimeUnit
20
20
import java .util .regex .Pattern
21
21
22
22
import org .apache .pulsar .client .admin .{PulsarAdmin , PulsarAdminException }
23
- import org .apache .pulsar .client .api .{Message , MessageId , PulsarClient , SubscriptionInitialPosition , SubscriptionType }
23
+ import org .apache .pulsar .client .api .{Message , MessageId , PulsarClient }
24
24
import org .apache .pulsar .client .impl .schema .BytesSchema
25
+ import org .apache .pulsar .client .internal .DefaultImplementation
25
26
import org .apache .pulsar .common .naming .TopicName
26
27
import org .apache .pulsar .common .schema .SchemaInfo
27
28
28
29
import org .apache .spark .internal .Logging
29
- import org .apache .spark .sql .pulsar .PulsarOptions .{AUTH_PARAMS , AUTH_PLUGIN_CLASS_NAME , TLS_ALLOW_INSECURE_CONNECTION , TLS_HOSTNAME_VERIFICATION_ENABLE , TLS_TRUST_CERTS_FILE_PATH , TOPIC_OPTION_KEYS }
30
+ import org .apache .spark .sql .pulsar .PulsarOptions .TOPIC_OPTION_KEYS
31
+ import org .apache .spark .sql .pulsar .topicinternalstats .forward ._
30
32
import org .apache .spark .sql .types .StructType
31
33
32
34
/**
@@ -205,6 +207,88 @@ private[pulsar] case class PulsarMetadataReader(
205
207
}.toMap)
206
208
}
207
209
210
+
211
+ def forwardOffset (actualOffset : Option [Map [String , MessageId ]],
212
+ strategy : String ,
213
+ numberOfEntriesToForward : Long ,
214
+ ensureEntriesPerTopic : Long ): SpecificPulsarOffset = {
215
+ getTopicPartitions()
216
+
217
+ // Collect internal stats for all topics
218
+ val topicStats = topicPartitions.map( topic => {
219
+ val internalStats = admin.topics().getInternalStats(topic)
220
+ val topicActualMessageId = actualOffset match {
221
+ case Some (value) => value.getOrElse(topic, MessageId .earliest)
222
+ case None => MessageId .earliest
223
+ }
224
+ topic -> TopicState (internalStats,
225
+ PulsarSourceUtils .getLedgerId(topicActualMessageId),
226
+ PulsarSourceUtils .getEntryId(topicActualMessageId))
227
+ } ).toMap
228
+
229
+ val forwarder = strategy match {
230
+ case PulsarOptions .PROPORTIONAL_FORWARD_STRATEGY =>
231
+ new ProportionalForwardStrategy (numberOfEntriesToForward, ensureEntriesPerTopic)
232
+ case PulsarOptions .LARGE_FIRST_FORWARD_STRATEGY =>
233
+ new LargeFirstForwardStrategy (numberOfEntriesToForward, ensureEntriesPerTopic)
234
+ case _ =>
235
+ new LinearForwardStrategy (numberOfEntriesToForward)
236
+ }
237
+
238
+ SpecificPulsarOffset (topicPartitions.map { topic =>
239
+ topic -> PulsarSourceUtils .seekableLatestMid {
240
+ // Fetch actual offset for topic
241
+ val topicActualMessageId = actualOffset match {
242
+ case Some (value) => value.getOrElse(topic, MessageId .earliest)
243
+ case None => MessageId .earliest
244
+ }
245
+ try {
246
+ // Get the actual ledger
247
+ val actualLedgerId = PulsarSourceUtils .getLedgerId(topicActualMessageId)
248
+ // Get the actual entry ID
249
+ val actualEntryId = PulsarSourceUtils .getEntryId(topicActualMessageId)
250
+ // Get the partition index
251
+ val partitionIndex = PulsarSourceUtils .getPartitionIndex(topicActualMessageId)
252
+ // Cache topic internal stats
253
+ val internalStats = topicStats.get(topic).get.internalStat
254
+ // Calculate the amount of messages we will pull in
255
+ val numberOfEntriesPerTopic = forwarder.forward(topicStats)(topic)
256
+ // Get a future message ID which corresponds
257
+ // to the maximum number of messages
258
+ val (nextLedgerId, nextEntryId) = TopicInternalStatsUtils .forwardMessageId(
259
+ internalStats,
260
+ actualLedgerId,
261
+ actualEntryId,
262
+ numberOfEntriesPerTopic)
263
+ // Build a message id
264
+ val forwardedMessageId =
265
+ DefaultImplementation .newMessageId(nextLedgerId, nextEntryId, partitionIndex)
266
+ // Log state
267
+ val forwardedEntry = TopicInternalStatsUtils .numOfEntriesUntil(
268
+ internalStats, nextLedgerId, nextEntryId)
269
+ val entryCount = TopicInternalStatsUtils .entryCount(internalStats)
270
+ val progress = f " ${forwardedEntry.toFloat / entryCount.toFloat}%1.3f "
271
+ val logMessage = s " Pulsar Connector forward on topic. " +
272
+ s " [ $numberOfEntriesPerTopic/ $numberOfEntriesToForward] " +
273
+ s " ${topic.reverse.take(30 ).reverse} $topicActualMessageId -> " +
274
+ s " $forwardedMessageId ( $forwardedEntry/ $entryCount) [ $progress] "
275
+ log.debug(logMessage)
276
+ // Return the message ID
277
+ forwardedMessageId
278
+ } catch {
279
+ case e : PulsarAdminException if e.getStatusCode == 404 =>
280
+ MessageId .earliest
281
+ case e : Throwable =>
282
+ throw new RuntimeException (
283
+ s " Failed to get forwarded messageId for ${TopicName .get(topic).toString} " +
284
+ s " (tried to forward ${forwarder.forward(topicStats)(topic)} messages " +
285
+ s " starting from ` $topicActualMessageId` using strategy $strategy) " , e)
286
+ }
287
+
288
+ }
289
+ }.toMap)
290
+ }
291
+
208
292
def fetchLatestOffsetForTopic (topic : String ): MessageId = {
209
293
PulsarSourceUtils .seekableLatestMid( try {
210
294
admin.topics().getLastMessageId(topic)
0 commit comments