65
65
import java .util .concurrent .CompletableFuture ;
66
66
import java .util .concurrent .TimeUnit ;
67
67
68
+ import static com .automq .stream .s3 .operator .ObjectStorageFactory .EXTENSION_TYPE_BACKGROUND ;
69
+ import static com .automq .stream .s3 .operator .ObjectStorageFactory .EXTENSION_TYPE_KEY ;
70
+ import static com .automq .stream .s3 .operator .ObjectStorageFactory .EXTENSION_TYPE_MAIN ;
71
+
68
72
public class DefaultS3Client implements Client {
69
73
private static final Logger LOGGER = LoggerFactory .getLogger (DefaultS3Client .class );
70
74
protected final Config config ;
@@ -74,8 +78,8 @@ public class DefaultS3Client implements Client {
74
78
75
79
protected ControllerRequestSender requestSender ;
76
80
77
- protected ObjectStorage objectStorage ;
78
- protected ObjectStorage compactionObjectStorage ;
81
+ protected ObjectStorage mainObjectStorage ;
82
+ protected ObjectStorage backgroundObjectStorage ;
79
83
80
84
protected WriteAheadLog writeAheadLog ;
81
85
protected StorageFailureHandlerChain storageFailureHandlerChain ;
@@ -109,7 +113,6 @@ public DefaultS3Client(BrokerServer brokerServer, Config config) {
109
113
110
114
@ Override
111
115
public void start () {
112
- BucketURI dataBucket = config .dataBuckets ().get (0 );
113
116
long refillToken = (long ) (config .networkBaselineBandwidth () * ((double ) config .refillPeriodMs () / 1000 ));
114
117
if (refillToken <= 0 ) {
115
118
throw new IllegalArgumentException (String .format ("refillToken must be greater than 0, bandwidth: %d, refill period: %dms" ,
@@ -127,34 +130,30 @@ public void start() {
127
130
S3StreamMetricsManager .registerNetworkAvailableBandwidthSupplier (AsyncNetworkBandwidthLimiter .Type .OUTBOUND , () ->
128
131
config .networkBaselineBandwidth () - (long ) networkOutboundRate .derive (
129
132
TimeUnit .NANOSECONDS .toSeconds (System .nanoTime ()), NetworkStats .getInstance ().networkOutboundUsageTotal ().get ()));
130
- this .objectStorage = ObjectStorageFactory .instance ().builder (dataBucket ).tagging (config .objectTagging ())
131
- .inboundLimiter (networkInboundLimiter ).outboundLimiter (networkOutboundLimiter ).readWriteIsolate (true )
132
- .threadPrefix ("dataflow" ).build ();
133
- if (!objectStorage .readinessCheck ()) {
133
+
134
+ this .localIndexCache = new LocalStreamRangeIndexCache ();
135
+ this .objectReaderFactory = new DefaultObjectReaderFactory (() -> this .mainObjectStorage );
136
+ this .metadataManager = new StreamMetadataManager (brokerServer , config .nodeId (), objectReaderFactory , localIndexCache );
137
+ this .requestSender = new ControllerRequestSender (brokerServer , new ControllerRequestSender .RetryPolicyContext (config .controllerRequestRetryMaxCount (),
138
+ config .controllerRequestRetryBaseDelayMs ()));
139
+ this .streamManager = newStreamManager (config .nodeId (), config .nodeEpoch (), false );
140
+ this .objectManager = newObjectManager (config .nodeId (), config .nodeEpoch (), false );
141
+ this .mainObjectStorage = newMainObjectStorage ();
142
+ if (!mainObjectStorage .readinessCheck ()) {
134
143
throw new IllegalArgumentException (String .format ("%s is not ready" , config .dataBuckets ()));
135
144
}
136
- this .compactionObjectStorage = ObjectStorageFactory .instance ().builder (dataBucket ).tagging (config .objectTagging ())
137
- .inboundLimiter (networkInboundLimiter ).outboundLimiter (networkOutboundLimiter )
138
- .threadPrefix ("compaction" ).build ();
139
- ControllerRequestSender .RetryPolicyContext retryPolicyContext = new ControllerRequestSender .RetryPolicyContext (config .controllerRequestRetryMaxCount (),
140
- config .controllerRequestRetryBaseDelayMs ());
141
- localIndexCache = new LocalStreamRangeIndexCache ();
142
- localIndexCache .init (config .nodeId (), objectStorage );
145
+ this .backgroundObjectStorage = newBackgroundObjectStorage ();
146
+ localIndexCache .init (config .nodeId (), backgroundObjectStorage );
143
147
localIndexCache .start ();
144
- this .objectReaderFactory = new DefaultObjectReaderFactory (objectStorage );
145
- this .metadataManager = new StreamMetadataManager (brokerServer , config .nodeId (), objectReaderFactory , localIndexCache );
146
- this .requestSender = new ControllerRequestSender (brokerServer , retryPolicyContext );
147
- this .streamManager = newStreamManager (config .nodeId (), config .nodeEpoch (), false );
148
148
this .streamManager .setStreamCloseHook (streamId -> localIndexCache .uploadOnStreamClose ());
149
- this .objectManager = newObjectManager (config .nodeId (), config .nodeEpoch (), false );
150
149
this .objectManager .setCommitStreamSetObjectHook (localIndexCache ::updateIndexFromRequest );
151
- this .blockCache = new StreamReaders (this .config .blockCacheSize (), objectManager , objectStorage , objectReaderFactory );
152
- this .compactionManager = new CompactionManager (this .config , this .objectManager , this .streamManager , compactionObjectStorage );
150
+ this .blockCache = new StreamReaders (this .config .blockCacheSize (), objectManager , mainObjectStorage , objectReaderFactory );
151
+ this .compactionManager = new CompactionManager (this .config , this .objectManager , this .streamManager , backgroundObjectStorage );
153
152
this .writeAheadLog = buildWAL ();
154
153
this .storageFailureHandlerChain = new StorageFailureHandlerChain ();
155
154
this .storage = newS3Storage ();
156
155
// stream object compactions share the same object storage with stream set object compactions
157
- this .streamClient = new S3StreamClient (this .streamManager , this .storage , this .objectManager , compactionObjectStorage , this .config , networkInboundLimiter , networkOutboundLimiter );
156
+ this .streamClient = new S3StreamClient (this .streamManager , this .storage , this .objectManager , backgroundObjectStorage , this .config , networkInboundLimiter , networkOutboundLimiter );
158
157
storageFailureHandlerChain .addHandler (new ForceCloseStorageFailureHandler (streamClient ));
159
158
storageFailureHandlerChain .addHandler (new HaltStorageFailureHandler ());
160
159
this .streamClient .registerStreamLifeCycleListener (localIndexCache );
@@ -221,6 +220,22 @@ protected WriteAheadLog buildWAL() {
221
220
}
222
221
}
223
222
223
+ protected ObjectStorage newMainObjectStorage () {
224
+ return ObjectStorageFactory .instance ().builder ()
225
+ .buckets (config .dataBuckets ())
226
+ .extension (EXTENSION_TYPE_KEY , EXTENSION_TYPE_MAIN )
227
+ .readWriteIsolate (true )
228
+ .build ();
229
+ }
230
+
231
+ protected ObjectStorage newBackgroundObjectStorage () {
232
+ return ObjectStorageFactory .instance ().builder ()
233
+ .buckets (config .dataBuckets ())
234
+ .extension (EXTENSION_TYPE_KEY , EXTENSION_TYPE_BACKGROUND )
235
+ .readWriteIsolate (false )
236
+ .build ();
237
+ }
238
+
224
239
protected StreamManager newStreamManager (int nodeId , long nodeEpoch , boolean failoverMode ) {
225
240
return new ControllerStreamManager (this .metadataManager , this .requestSender , nodeId , nodeEpoch ,
226
241
this ::getAutoMQVersion , failoverMode );
@@ -232,7 +247,7 @@ protected ObjectManager newObjectManager(int nodeId, long nodeEpoch, boolean fai
232
247
}
233
248
234
249
protected S3Storage newS3Storage () {
235
- return new S3Storage (config , writeAheadLog , streamManager , objectManager , blockCache , objectStorage , storageFailureHandlerChain );
250
+ return new S3Storage (config , writeAheadLog , streamManager , objectManager , blockCache , mainObjectStorage , storageFailureHandlerChain );
236
251
}
237
252
238
253
protected Failover failover () {
0 commit comments