16
16
17
17
package com .google .cloud .pubsublite .spark ;
18
18
19
- import static com .google .cloud .pubsublite .internal .wire .ServiceClients .addDefaultSettings ;
20
-
19
+ import com .github .benmanes .caffeine .cache .Ticker ;
21
20
import com .google .auto .service .AutoService ;
22
21
import com .google .cloud .pubsublite .AdminClient ;
23
- import com .google .cloud .pubsublite .Partition ;
24
22
import com .google .cloud .pubsublite .PartitionLookupUtils ;
25
23
import com .google .cloud .pubsublite .SubscriptionPath ;
26
24
import com .google .cloud .pubsublite .TopicPath ;
27
- import com .google .cloud .pubsublite .internal .CheckedApiException ;
28
25
import com .google .cloud .pubsublite .internal .CursorClient ;
29
- import com .google .cloud .pubsublite .internal .wire .CommitterBuilder ;
30
- import com .google .cloud .pubsublite .internal .wire .PubsubContext ;
31
- import com .google .cloud .pubsublite .internal .wire .RoutingMetadata ;
32
- import com .google .cloud .pubsublite .internal .wire .ServiceClients ;
33
- import com .google .cloud .pubsublite .internal .wire .SubscriberBuilder ;
34
- import com .google .cloud .pubsublite .v1 .SubscriberServiceClient ;
35
- import com .google .cloud .pubsublite .v1 .SubscriberServiceSettings ;
36
- import com .google .common .collect .ImmutableMap ;
37
- import java .io .IOException ;
38
26
import java .util .Objects ;
39
27
import java .util .Optional ;
40
28
import org .apache .spark .sql .sources .DataSourceRegister ;
@@ -69,20 +57,10 @@ public ContinuousReader createContinuousReader(
69
57
AdminClient adminClient = pslDataSourceOptions .newAdminClient ();
70
58
SubscriptionPath subscriptionPath = pslDataSourceOptions .subscriptionPath ();
71
59
long topicPartitionCount = PartitionLookupUtils .numPartitions (subscriptionPath , adminClient );
72
- MultiPartitionCommitter committer =
73
- new MultiPartitionCommitterImpl (
74
- topicPartitionCount ,
75
- (partition ) ->
76
- CommitterBuilder .newBuilder ()
77
- .setSubscriptionPath (subscriptionPath )
78
- .setPartition (partition )
79
- .setServiceClient (pslDataSourceOptions .newCursorServiceClient ())
80
- .build ());
81
-
82
60
return new PslContinuousReader (
83
61
cursorClient ,
84
- committer ,
85
- getSubscriberFactory (new PslCredentialsProvider ( pslDataSourceOptions ), subscriptionPath ),
62
+ pslDataSourceOptions . newMultiPartitionCommitter ( topicPartitionCount ) ,
63
+ pslDataSourceOptions . getSubscriberFactory (),
86
64
subscriptionPath ,
87
65
Objects .requireNonNull (pslDataSourceOptions .flowControlSettings ()),
88
66
topicPartitionCount );
@@ -98,7 +76,6 @@ public MicroBatchReader createMicroBatchReader(
98
76
99
77
PslDataSourceOptions pslDataSourceOptions =
100
78
PslDataSourceOptions .fromSparkDataSourceOptions (options );
101
- PslCredentialsProvider credentialsProvider = new PslCredentialsProvider (pslDataSourceOptions );
102
79
CursorClient cursorClient = pslDataSourceOptions .newCursorClient ();
103
80
AdminClient adminClient = pslDataSourceOptions .newAdminClient ();
104
81
SubscriptionPath subscriptionPath = pslDataSourceOptions .subscriptionPath ();
@@ -110,72 +87,17 @@ public MicroBatchReader createMicroBatchReader(
110
87
"Unable to get topic for subscription " + subscriptionPath , t );
111
88
}
112
89
long topicPartitionCount = PartitionLookupUtils .numPartitions (topicPath , adminClient );
113
- MultiPartitionCommitter committer =
114
- new MultiPartitionCommitterImpl (
115
- topicPartitionCount ,
116
- (partition ) ->
117
- CommitterBuilder .newBuilder ()
118
- .setSubscriptionPath (subscriptionPath )
119
- .setPartition (partition )
120
- .setServiceClient (pslDataSourceOptions .newCursorServiceClient ())
121
- .build ());
122
-
123
90
return new PslMicroBatchReader (
124
91
cursorClient ,
125
- committer ,
126
- getSubscriberFactory (new PslCredentialsProvider (pslDataSourceOptions ), subscriptionPath ),
92
+ pslDataSourceOptions .newMultiPartitionCommitter (topicPartitionCount ),
93
+ pslDataSourceOptions .getSubscriberFactory (),
94
+ new LimitingHeadOffsetReader (
95
+ pslDataSourceOptions .newTopicStatsClient (),
96
+ topicPath ,
97
+ topicPartitionCount ,
98
+ Ticker .systemTicker ()),
127
99
subscriptionPath ,
128
- PslSparkUtils .toSparkSourceOffset (getHeadOffset (topicPath )),
129
100
Objects .requireNonNull (pslDataSourceOptions .flowControlSettings ()),
130
101
topicPartitionCount );
131
102
}
132
-
133
- private static PslSourceOffset getHeadOffset (TopicPath topicPath ) {
134
- // TODO(jiangmichael): Replace it with real implementation.
135
- HeadOffsetReader headOffsetReader =
136
- new HeadOffsetReader () {
137
- @ Override
138
- public PslSourceOffset getHeadOffset (TopicPath topic ) {
139
- return PslSourceOffset .builder ()
140
- .partitionOffsetMap (
141
- ImmutableMap .of (
142
- Partition .of (0 ), com .google .cloud .pubsublite .Offset .of (50 ),
143
- Partition .of (1 ), com .google .cloud .pubsublite .Offset .of (50 )))
144
- .build ();
145
- }
146
-
147
- @ Override
148
- public void close () {}
149
- };
150
- try {
151
- return headOffsetReader .getHeadOffset (topicPath );
152
- } catch (CheckedApiException e ) {
153
- throw new IllegalStateException ("Unable to get head offset for topic " + topicPath , e );
154
- }
155
- }
156
-
157
- private static PartitionSubscriberFactory getSubscriberFactory (
158
- PslCredentialsProvider credentialsProvider , SubscriptionPath subscriptionPath ) {
159
- return (partition , consumer ) -> {
160
- PubsubContext context = PubsubContext .of (Constants .FRAMEWORK );
161
- SubscriberServiceSettings .Builder settingsBuilder =
162
- SubscriberServiceSettings .newBuilder ().setCredentialsProvider (credentialsProvider );
163
- ServiceClients .addDefaultMetadata (
164
- context , RoutingMetadata .of (subscriptionPath , partition ), settingsBuilder );
165
- try {
166
- SubscriberServiceClient serviceClient =
167
- SubscriberServiceClient .create (
168
- addDefaultSettings (subscriptionPath .location ().region (), settingsBuilder ));
169
- return SubscriberBuilder .newBuilder ()
170
- .setSubscriptionPath (subscriptionPath )
171
- .setPartition (partition )
172
- .setContext (context )
173
- .setServiceClient (serviceClient )
174
- .setMessageConsumer (consumer )
175
- .build ();
176
- } catch (IOException e ) {
177
- throw new IllegalStateException ("Failed to create subscriber service." , e );
178
- }
179
- };
180
- }
181
103
}
0 commit comments