You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@bookkeeper.apache.org by GitBox <gi...@apache.org> on 2022/09/14 09:44:19 UTC

[GitHub] [bookkeeper] diegosalvi opened a new pull request, #3489: Issue #3488 Create an EmbeddedSever extracting code from Main class and adding a builder pattern

diegosalvi opened a new pull request, #3489:
URL: https://github.com/apache/bookkeeper/pull/3489

   Descriptions of the changes in this PR:
   
   ### Motivation
   Is enough common for project to start an embedded version of BK using such classes. Instead of rewrite every time boilerplate code the full server instantiation can be extracted from Main class creating a public API to create BookieService from configurations and/or provided component instances.
   
   ### Changes
   I've took code from Main class and refactored into a builder as EmbeddedServer class.
   Just providing the configuration to the builder produces the same server that Main class was generating.
   Added multiple builder properties to permit to inject custom implementations for various components. "Injected" component lifecycle is responsability of the implementor (ie.: if you provide specific instances you need to remember to start and/or stop such instances accordingly)
   
   Master Issue: #3488


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

To unsubscribe, e-mail: commits-unsubscribe@bookkeeper.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


[GitHub] [bookkeeper] eolivelli commented on a diff in pull request #3489: Issue #3488 Create an EmbeddedSever extracting code from Main class and adding a builder pattern

Posted by GitBox <gi...@apache.org>.
eolivelli commented on code in PR #3489:
URL: https://github.com/apache/bookkeeper/pull/3489#discussion_r1017729963


##########
bookkeeper-server/src/main/java/org/apache/bookkeeper/server/EmbeddedServer.java:
##########
@@ -0,0 +1,648 @@
+/**
+ *
+ * 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.bookkeeper.server;
+
+import static com.google.common.base.Preconditions.checkNotNull;
+import static org.apache.bookkeeper.bookie.BookKeeperServerStats.BOOKIE_SCOPE;
+import static org.apache.bookkeeper.bookie.BookKeeperServerStats.LD_INDEX_SCOPE;
+import static org.apache.bookkeeper.bookie.BookKeeperServerStats.LD_LEDGER_SCOPE;
+import static org.apache.bookkeeper.client.BookKeeperClientStats.CLIENT_SCOPE;
+import static org.apache.bookkeeper.replication.ReplicationStats.REPLICATION_SCOPE;
+import static org.apache.bookkeeper.server.Main.storageDirectoriesFromConf;
+import static org.apache.bookkeeper.server.component.ServerLifecycleComponent.loadServerComponents;
+
+import com.google.common.base.Ticker;
+import com.google.common.util.concurrent.ThreadFactoryBuilder;
+import io.netty.buffer.ByteBuf;
+import io.netty.buffer.ByteBufAllocator;
+import io.netty.buffer.CompositeByteBuf;
+import io.reactivex.rxjava3.core.Scheduler;
+import io.reactivex.rxjava3.schedulers.Schedulers;
+import java.util.List;
+import java.util.concurrent.ExecutorService;
+import java.util.concurrent.Executors;
+import java.util.function.Consumer;
+import java.util.function.Supplier;
+import java.util.stream.Collectors;
+import lombok.extern.slf4j.Slf4j;
+import org.apache.bookkeeper.bookie.Bookie;
+import org.apache.bookkeeper.bookie.BookieImpl;
+import org.apache.bookkeeper.bookie.BookieResources;
+import org.apache.bookkeeper.bookie.CookieValidation;
+import org.apache.bookkeeper.bookie.LedgerDirsManager;
+import org.apache.bookkeeper.bookie.LedgerStorage;
+import org.apache.bookkeeper.bookie.LegacyCookieValidation;
+import org.apache.bookkeeper.bookie.ReadOnlyBookie;
+import org.apache.bookkeeper.bookie.ScrubberStats;
+import org.apache.bookkeeper.bookie.UncleanShutdownDetection;
+import org.apache.bookkeeper.bookie.UncleanShutdownDetectionImpl;
+import org.apache.bookkeeper.bookie.datainteg.DataIntegrityCheck;
+import org.apache.bookkeeper.bookie.datainteg.DataIntegrityCheckImpl;
+import org.apache.bookkeeper.bookie.datainteg.DataIntegrityCookieValidation;
+import org.apache.bookkeeper.bookie.datainteg.DataIntegrityService;
+import org.apache.bookkeeper.bookie.datainteg.EntryCopier;
+import org.apache.bookkeeper.bookie.datainteg.EntryCopierImpl;
+import org.apache.bookkeeper.client.BookKeeper;
+import org.apache.bookkeeper.client.BookKeeperAdmin;
+import org.apache.bookkeeper.common.allocator.ByteBufAllocatorWithOomHandler;
+import org.apache.bookkeeper.common.component.AutoCloseableLifecycleComponent;
+import org.apache.bookkeeper.common.component.ComponentInfoPublisher;
+import org.apache.bookkeeper.common.component.LifecycleComponentStack;
+import org.apache.bookkeeper.common.component.RxSchedulerLifecycleComponent;
+import org.apache.bookkeeper.conf.ClientConfiguration;
+import org.apache.bookkeeper.discover.BookieServiceInfo;
+import org.apache.bookkeeper.discover.RegistrationManager;
+import org.apache.bookkeeper.meta.LedgerManager;
+import org.apache.bookkeeper.meta.LedgerManagerFactory;
+import org.apache.bookkeeper.meta.MetadataBookieDriver;
+import org.apache.bookkeeper.net.BookieId;
+import org.apache.bookkeeper.server.component.ServerLifecycleComponent;
+import org.apache.bookkeeper.server.conf.BookieConfiguration;
+import org.apache.bookkeeper.server.http.BKHttpServiceProvider;
+import org.apache.bookkeeper.server.service.AutoRecoveryService;
+import org.apache.bookkeeper.server.service.BookieService;
+import org.apache.bookkeeper.server.service.HttpService;
+import org.apache.bookkeeper.server.service.ScrubberService;
+import org.apache.bookkeeper.server.service.StatsProviderService;
+import org.apache.bookkeeper.stats.StatsLogger;
+import org.apache.bookkeeper.stats.StatsProvider;
+import org.apache.bookkeeper.util.DiskChecker;
+import org.apache.commons.lang3.StringUtils;
+
+/**
+ * An embedded server is a server that run bookie and serving rpc requests.
+ *
+ * <p>
+ * It is a rewritten server using {@link org.apache.bookkeeper.common.component.LifecycleComponent}, replacing the
+ * legacy server {@link org.apache.bookkeeper.proto.BookieServer}.
+ */
+public class EmbeddedServer {
+
+    private final LifecycleComponentStack lifecycleComponentStack;
+
+    private final StatsProvider statsProvider;
+
+    private final RegistrationManager registrationManager;
+
+    private final LedgerManagerFactory ledgerManagerFactory;
+
+    private final DiskChecker diskChecker;
+    private final LedgerDirsManager ledgerDirsManager;
+    private final LedgerDirsManager indexDirsManager;
+
+    private final BookieService bookieService;
+    private final AutoRecoveryService autoRecoveryService;
+    private final DataIntegrityService dataIntegrityService;
+    private final HttpService httpService;
+
+    private EmbeddedServer(LifecycleComponentStack lifecycleComponentStack, StatsProvider statsProvider,
+                           RegistrationManager registrationManager, LedgerManagerFactory ledgerManagerFactory,
+                           DiskChecker diskChecker, LedgerDirsManager ledgerDirsManager,
+                           LedgerDirsManager indexDirsManager, BookieService bookieService,
+                           AutoRecoveryService autoRecoveryService, DataIntegrityService dataIntegrityService,
+                           HttpService httpService) {
+        this.lifecycleComponentStack = lifecycleComponentStack;
+        this.statsProvider = statsProvider;
+        this.registrationManager = registrationManager;
+        this.ledgerManagerFactory = ledgerManagerFactory;
+        this.diskChecker = diskChecker;
+        this.ledgerDirsManager = ledgerDirsManager;
+        this.indexDirsManager = indexDirsManager;
+        this.bookieService = bookieService;
+        this.autoRecoveryService = autoRecoveryService;
+        this.dataIntegrityService = dataIntegrityService;
+        this.httpService = httpService;
+    }
+
+    public LifecycleComponentStack getLifecycleComponentStack() {
+        return lifecycleComponentStack;
+    }
+
+    public StatsProvider getStatsProvider() {
+        return statsProvider;
+    }
+
+    public RegistrationManager getRegistrationManager() {
+        return registrationManager;
+    }
+
+    public LedgerManagerFactory getLedgerManagerFactory() {
+        return ledgerManagerFactory;
+    }
+
+    public DiskChecker getDiskChecker() {
+        return diskChecker;
+    }
+
+    public LedgerDirsManager getLedgerDirsManager() {
+        return ledgerDirsManager;
+    }
+
+    public LedgerDirsManager getIndexDirsManager() {
+        return indexDirsManager;
+    }
+
+    public BookieService getBookieService() {
+        return bookieService;
+    }
+
+    public AutoRecoveryService getAutoRecoveryService() {
+        return autoRecoveryService;
+    }
+
+    public DataIntegrityService getDataIntegrityService() {
+        return dataIntegrityService;
+    }
+
+    public HttpService getHttpService() {
+        return httpService;
+    }
+
+    /**
+     * Create a new builder from given configuration. Actual services implementations can be provided to the builder and
+     * will override ones defined in the configuration.
+     * <p>
+     * Invoker is responsible to start and stop provided services implementations, components from
+     * {@link EmbeddedServer#getLifecycleComponentStack()} will reflect only those created from provided configuration.
+     *
+     * @param conf bookie configuration
+     * @return a new embedded server builder
+     */
+    public static final Builder builder(BookieConfiguration conf) {
+        return new Builder(conf);
+    }
+
+    @Slf4j
+    public static class Builder {
+
+        private BookieConfiguration conf;
+
+        boolean addExternalResourcesToLifecycle = true;

Review Comment:
   What does this mean?
   If a component is passed from outside we shouldn't handle it.
   Why are you adding this feature?
   Also, if you really need this it must be turned off by default 



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

To unsubscribe, e-mail: commits-unsubscribe@bookkeeper.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


[GitHub] [bookkeeper] RaulGracia commented on a diff in pull request #3489: Issue #3488 Create an EmbeddedSever extracting code from Main class and adding a builder pattern

Posted by GitBox <gi...@apache.org>.
RaulGracia commented on code in PR #3489:
URL: https://github.com/apache/bookkeeper/pull/3489#discussion_r970875276


##########
bookkeeper-server/src/main/java/org/apache/bookkeeper/server/EmbeddedServer.java:
##########
@@ -0,0 +1,636 @@
+/**
+ *
+ * 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.bookkeeper.server;
+
+import static com.google.common.base.Preconditions.checkNotNull;
+import static org.apache.bookkeeper.bookie.BookKeeperServerStats.BOOKIE_SCOPE;
+import static org.apache.bookkeeper.bookie.BookKeeperServerStats.LD_INDEX_SCOPE;
+import static org.apache.bookkeeper.bookie.BookKeeperServerStats.LD_LEDGER_SCOPE;
+import static org.apache.bookkeeper.client.BookKeeperClientStats.CLIENT_SCOPE;
+import static org.apache.bookkeeper.replication.ReplicationStats.REPLICATION_SCOPE;
+import static org.apache.bookkeeper.server.Main.storageDirectoriesFromConf;
+import static org.apache.bookkeeper.server.component.ServerLifecycleComponent.loadServerComponents;
+
+import com.google.common.base.Ticker;
+import com.google.common.util.concurrent.ThreadFactoryBuilder;
+import io.netty.buffer.ByteBuf;
+import io.netty.buffer.ByteBufAllocator;
+import io.netty.buffer.CompositeByteBuf;
+import io.reactivex.rxjava3.core.Scheduler;
+import io.reactivex.rxjava3.schedulers.Schedulers;
+import java.util.List;
+import java.util.concurrent.ExecutorService;
+import java.util.concurrent.Executors;
+import java.util.function.Consumer;
+import java.util.function.Supplier;
+import java.util.stream.Collectors;
+import lombok.extern.slf4j.Slf4j;
+import org.apache.bookkeeper.bookie.Bookie;
+import org.apache.bookkeeper.bookie.BookieImpl;
+import org.apache.bookkeeper.bookie.BookieResources;
+import org.apache.bookkeeper.bookie.CookieValidation;
+import org.apache.bookkeeper.bookie.LedgerDirsManager;
+import org.apache.bookkeeper.bookie.LedgerStorage;
+import org.apache.bookkeeper.bookie.LegacyCookieValidation;
+import org.apache.bookkeeper.bookie.ReadOnlyBookie;
+import org.apache.bookkeeper.bookie.ScrubberStats;
+import org.apache.bookkeeper.bookie.UncleanShutdownDetection;
+import org.apache.bookkeeper.bookie.UncleanShutdownDetectionImpl;
+import org.apache.bookkeeper.bookie.datainteg.DataIntegrityCheck;
+import org.apache.bookkeeper.bookie.datainteg.DataIntegrityCheckImpl;
+import org.apache.bookkeeper.bookie.datainteg.DataIntegrityCookieValidation;
+import org.apache.bookkeeper.bookie.datainteg.DataIntegrityService;
+import org.apache.bookkeeper.bookie.datainteg.EntryCopier;
+import org.apache.bookkeeper.bookie.datainteg.EntryCopierImpl;
+import org.apache.bookkeeper.client.BookKeeper;
+import org.apache.bookkeeper.client.BookKeeperAdmin;
+import org.apache.bookkeeper.common.allocator.ByteBufAllocatorWithOomHandler;
+import org.apache.bookkeeper.common.component.AutoCloseableLifecycleComponent;
+import org.apache.bookkeeper.common.component.ComponentInfoPublisher;
+import org.apache.bookkeeper.common.component.LifecycleComponentStack;
+import org.apache.bookkeeper.common.component.RxSchedulerLifecycleComponent;
+import org.apache.bookkeeper.conf.ClientConfiguration;
+import org.apache.bookkeeper.discover.BookieServiceInfo;
+import org.apache.bookkeeper.discover.RegistrationManager;
+import org.apache.bookkeeper.meta.LedgerManager;
+import org.apache.bookkeeper.meta.LedgerManagerFactory;
+import org.apache.bookkeeper.meta.MetadataBookieDriver;
+import org.apache.bookkeeper.net.BookieId;
+import org.apache.bookkeeper.server.component.ServerLifecycleComponent;
+import org.apache.bookkeeper.server.conf.BookieConfiguration;
+import org.apache.bookkeeper.server.http.BKHttpServiceProvider;
+import org.apache.bookkeeper.server.service.AutoRecoveryService;
+import org.apache.bookkeeper.server.service.BookieService;
+import org.apache.bookkeeper.server.service.HttpService;
+import org.apache.bookkeeper.server.service.ScrubberService;
+import org.apache.bookkeeper.server.service.StatsProviderService;
+import org.apache.bookkeeper.stats.StatsLogger;
+import org.apache.bookkeeper.stats.StatsProvider;
+import org.apache.bookkeeper.util.DiskChecker;
+import org.apache.commons.lang3.StringUtils;
+
+/**
+ * An embedded server is a server that run bookie and serving rpc requests.
+ *
+ * <p>
+ * It is a rewritten server using {@link org.apache.bookkeeper.common.component.LifecycleComponent}, replacing the
+ * legacy server {@link org.apache.bookkeeper.proto.BookieServer}.
+ */
+public class EmbeddedServer {

Review Comment:
   If I understand correctly, this is embedding the Bookie server and resources, right? So it can be used both in tests or when running a real Bookie. I think that I'm in favor of this approach, mainly because it facilitates testing to other projects using Bookkeeper. For instance, in this PR I had to do some gymnastics to instantiate test Bookies when upgrading to Bookkeeper 4.15 in Pravega: https://github.com/pravega/pravega/pull/6676
   
   My understanding is that, with this change, just instantiating a new `EmbeddedServer` (maybe the name of the class could be discussed) with the right configuration, the external code should not care about the resources of the Bookie itself, right? If this is the case, I think that it is an improvement in usability.



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

To unsubscribe, e-mail: commits-unsubscribe@bookkeeper.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


[GitHub] [bookkeeper] hangc0276 commented on pull request #3489: Issue #3488 Create an EmbeddedSever extracting code from Main class and adding a builder pattern

Posted by GitBox <gi...@apache.org>.
hangc0276 commented on PR #3489:
URL: https://github.com/apache/bookkeeper/pull/3489#issuecomment-1280910873

   rerun failure checks


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

To unsubscribe, e-mail: commits-unsubscribe@bookkeeper.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


[GitHub] [bookkeeper] diegosalvi commented on pull request #3489: Issue #3488 Create an EmbeddedSever extracting code from Main class and adding a builder pattern

Posted by GitBox <gi...@apache.org>.
diegosalvi commented on PR #3489:
URL: https://github.com/apache/bookkeeper/pull/3489#issuecomment-1246904547

   Righ, it is exactly the case.
   
   Il Mer 14 Set 2022, 16:20 Raúl Gracia ***@***.***> ha scritto:
   
   > ***@***.**** commented on this pull request.
   > ------------------------------
   >
   > In
   > bookkeeper-server/src/main/java/org/apache/bookkeeper/server/EmbeddedServer.java
   > <https://github.com/apache/bookkeeper/pull/3489#discussion_r970875276>:
   >
   > > +import org.apache.bookkeeper.server.service.HttpService;
   > +import org.apache.bookkeeper.server.service.ScrubberService;
   > +import org.apache.bookkeeper.server.service.StatsProviderService;
   > +import org.apache.bookkeeper.stats.StatsLogger;
   > +import org.apache.bookkeeper.stats.StatsProvider;
   > +import org.apache.bookkeeper.util.DiskChecker;
   > +import org.apache.commons.lang3.StringUtils;
   > +
   > +/**
   > + * An embedded server is a server that run bookie and serving rpc requests.
   > + *
   > + * <p>
   > + * It is a rewritten server using ***@***.*** org.apache.bookkeeper.common.component.LifecycleComponent}, replacing the
   > + * legacy server ***@***.*** org.apache.bookkeeper.proto.BookieServer}.
   > + */
   > +public class EmbeddedServer {
   >
   > If I understand correctly, this is embedding the Bookie server and
   > resources, right? So it can be used both in tests or when running a real
   > Bookie. I think that I'm in favor of this approach, mainly because it
   > facilitates testing to other projects using Bookkeeper. For instance, in
   > this PR I had to do some gymnastics to instantiate test Bookies when
   > upgrading to Bookkeeper 4.15 in Pravega: pravega/pravega#6676
   > <https://github.com/pravega/pravega/pull/6676>
   >
   > My understanding is that, with this change, just instantiating a new
   > EmbeddedServer (maybe the name of the class could be discussed) with the
   > right configuration, the external code should not care about the resources
   > of the Bookie itself, right? If this is the case, I think that it is an
   > improvement in usability.
   >
   > —
   > Reply to this email directly, view it on GitHub
   > <https://github.com/apache/bookkeeper/pull/3489#pullrequestreview-1107586467>,
   > or unsubscribe
   > <https://github.com/notifications/unsubscribe-auth/AA2R46N34KUN7AVAOA2ELGDV6HNJ7ANCNFSM6AAAAAAQMHOROE>
   > .
   > You are receiving this because you authored the thread.Message ID:
   > ***@***.***>
   >
   


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

To unsubscribe, e-mail: commits-unsubscribe@bookkeeper.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


[GitHub] [bookkeeper] diegosalvi commented on a diff in pull request #3489: Issue #3488 Create an EmbeddedSever extracting code from Main class and adding a builder pattern

Posted by GitBox <gi...@apache.org>.
diegosalvi commented on code in PR #3489:
URL: https://github.com/apache/bookkeeper/pull/3489#discussion_r970609966


##########
bookkeeper-server/src/test/java/org/apache/bookkeeper/server/TestEmbeddedServer.java:
##########
@@ -1,60 +1,67 @@
 /*
- * 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
+ * Copyright 2022 The Apache Software Foundation.

Review Comment:
   Changed with a copy took from other files



##########
bookkeeper-server/src/main/java/org/apache/bookkeeper/server/EmbeddedServer.java:
##########
@@ -0,0 +1,630 @@
+/*
+ * Copyright 2022 The Apache Software Foundation.

Review Comment:
   Changed with a copy took from other files



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

To unsubscribe, e-mail: commits-unsubscribe@bookkeeper.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


[GitHub] [bookkeeper] diegosalvi commented on a diff in pull request #3489: Issue #3488 Create an EmbeddedSever extracting code from Main class and adding a builder pattern

Posted by GitBox <gi...@apache.org>.
diegosalvi commented on code in PR #3489:
URL: https://github.com/apache/bookkeeper/pull/3489#discussion_r970991351


##########
bookkeeper-server/src/main/java/org/apache/bookkeeper/server/EmbeddedServer.java:
##########
@@ -0,0 +1,636 @@
+/**
+ *
+ * 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.bookkeeper.server;
+
+import static com.google.common.base.Preconditions.checkNotNull;
+import static org.apache.bookkeeper.bookie.BookKeeperServerStats.BOOKIE_SCOPE;
+import static org.apache.bookkeeper.bookie.BookKeeperServerStats.LD_INDEX_SCOPE;
+import static org.apache.bookkeeper.bookie.BookKeeperServerStats.LD_LEDGER_SCOPE;
+import static org.apache.bookkeeper.client.BookKeeperClientStats.CLIENT_SCOPE;
+import static org.apache.bookkeeper.replication.ReplicationStats.REPLICATION_SCOPE;
+import static org.apache.bookkeeper.server.Main.storageDirectoriesFromConf;
+import static org.apache.bookkeeper.server.component.ServerLifecycleComponent.loadServerComponents;
+
+import com.google.common.base.Ticker;
+import com.google.common.util.concurrent.ThreadFactoryBuilder;
+import io.netty.buffer.ByteBuf;
+import io.netty.buffer.ByteBufAllocator;
+import io.netty.buffer.CompositeByteBuf;
+import io.reactivex.rxjava3.core.Scheduler;
+import io.reactivex.rxjava3.schedulers.Schedulers;
+import java.util.List;
+import java.util.concurrent.ExecutorService;
+import java.util.concurrent.Executors;
+import java.util.function.Consumer;
+import java.util.function.Supplier;
+import java.util.stream.Collectors;
+import lombok.extern.slf4j.Slf4j;
+import org.apache.bookkeeper.bookie.Bookie;
+import org.apache.bookkeeper.bookie.BookieImpl;
+import org.apache.bookkeeper.bookie.BookieResources;
+import org.apache.bookkeeper.bookie.CookieValidation;
+import org.apache.bookkeeper.bookie.LedgerDirsManager;
+import org.apache.bookkeeper.bookie.LedgerStorage;
+import org.apache.bookkeeper.bookie.LegacyCookieValidation;
+import org.apache.bookkeeper.bookie.ReadOnlyBookie;
+import org.apache.bookkeeper.bookie.ScrubberStats;
+import org.apache.bookkeeper.bookie.UncleanShutdownDetection;
+import org.apache.bookkeeper.bookie.UncleanShutdownDetectionImpl;
+import org.apache.bookkeeper.bookie.datainteg.DataIntegrityCheck;
+import org.apache.bookkeeper.bookie.datainteg.DataIntegrityCheckImpl;
+import org.apache.bookkeeper.bookie.datainteg.DataIntegrityCookieValidation;
+import org.apache.bookkeeper.bookie.datainteg.DataIntegrityService;
+import org.apache.bookkeeper.bookie.datainteg.EntryCopier;
+import org.apache.bookkeeper.bookie.datainteg.EntryCopierImpl;
+import org.apache.bookkeeper.client.BookKeeper;
+import org.apache.bookkeeper.client.BookKeeperAdmin;
+import org.apache.bookkeeper.common.allocator.ByteBufAllocatorWithOomHandler;
+import org.apache.bookkeeper.common.component.AutoCloseableLifecycleComponent;
+import org.apache.bookkeeper.common.component.ComponentInfoPublisher;
+import org.apache.bookkeeper.common.component.LifecycleComponentStack;
+import org.apache.bookkeeper.common.component.RxSchedulerLifecycleComponent;
+import org.apache.bookkeeper.conf.ClientConfiguration;
+import org.apache.bookkeeper.discover.BookieServiceInfo;
+import org.apache.bookkeeper.discover.RegistrationManager;
+import org.apache.bookkeeper.meta.LedgerManager;
+import org.apache.bookkeeper.meta.LedgerManagerFactory;
+import org.apache.bookkeeper.meta.MetadataBookieDriver;
+import org.apache.bookkeeper.net.BookieId;
+import org.apache.bookkeeper.server.component.ServerLifecycleComponent;
+import org.apache.bookkeeper.server.conf.BookieConfiguration;
+import org.apache.bookkeeper.server.http.BKHttpServiceProvider;
+import org.apache.bookkeeper.server.service.AutoRecoveryService;
+import org.apache.bookkeeper.server.service.BookieService;
+import org.apache.bookkeeper.server.service.HttpService;
+import org.apache.bookkeeper.server.service.ScrubberService;
+import org.apache.bookkeeper.server.service.StatsProviderService;
+import org.apache.bookkeeper.stats.StatsLogger;
+import org.apache.bookkeeper.stats.StatsProvider;
+import org.apache.bookkeeper.util.DiskChecker;
+import org.apache.commons.lang3.StringUtils;
+
+/**
+ * An embedded server is a server that run bookie and serving rpc requests.
+ *
+ * <p>
+ * It is a rewritten server using {@link org.apache.bookkeeper.common.component.LifecycleComponent}, replacing the
+ * legacy server {@link org.apache.bookkeeper.proto.BookieServer}.
+ */
+public class EmbeddedServer {

Review Comment:
   Right this is exactly the case. I needed it both for tests and embedding a bookie server



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

To unsubscribe, e-mail: commits-unsubscribe@bookkeeper.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


[GitHub] [bookkeeper] eolivelli commented on a diff in pull request #3489: Issue #3488 Create an EmbeddedSever extracting code from Main class and adding a builder pattern

Posted by GitBox <gi...@apache.org>.
eolivelli commented on code in PR #3489:
URL: https://github.com/apache/bookkeeper/pull/3489#discussion_r970580986


##########
bookkeeper-server/src/test/java/org/apache/bookkeeper/server/TestEmbeddedServer.java:
##########
@@ -1,60 +1,67 @@
 /*
- * 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
+ * Copyright 2022 The Apache Software Foundation.

Review Comment:
   The license seems wrong



##########
bookkeeper-server/src/main/java/org/apache/bookkeeper/server/EmbeddedServer.java:
##########
@@ -0,0 +1,630 @@
+/*
+ * Copyright 2022 The Apache Software Foundation.

Review Comment:
   This license seems wrong, please copy/paste from other files



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

To unsubscribe, e-mail: commits-unsubscribe@bookkeeper.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


[GitHub] [bookkeeper] eolivelli commented on pull request #3489: Issue #3488 Create an EmbeddedSever extracting code from Main class and adding a builder pattern

Posted by GitBox <gi...@apache.org>.
eolivelli commented on PR #3489:
URL: https://github.com/apache/bookkeeper/pull/3489#issuecomment-1274546284

   @diegosalvi PTAL


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

To unsubscribe, e-mail: commits-unsubscribe@bookkeeper.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


[GitHub] [bookkeeper] diegosalvi commented on a diff in pull request #3489: Issue #3488 Create an EmbeddedSever extracting code from Main class and adding a builder pattern

Posted by GitBox <gi...@apache.org>.
diegosalvi commented on code in PR #3489:
URL: https://github.com/apache/bookkeeper/pull/3489#discussion_r1017790940


##########
bookkeeper-server/src/main/java/org/apache/bookkeeper/server/EmbeddedServer.java:
##########
@@ -0,0 +1,648 @@
+/**
+ *
+ * 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.bookkeeper.server;
+
+import static com.google.common.base.Preconditions.checkNotNull;
+import static org.apache.bookkeeper.bookie.BookKeeperServerStats.BOOKIE_SCOPE;
+import static org.apache.bookkeeper.bookie.BookKeeperServerStats.LD_INDEX_SCOPE;
+import static org.apache.bookkeeper.bookie.BookKeeperServerStats.LD_LEDGER_SCOPE;
+import static org.apache.bookkeeper.client.BookKeeperClientStats.CLIENT_SCOPE;
+import static org.apache.bookkeeper.replication.ReplicationStats.REPLICATION_SCOPE;
+import static org.apache.bookkeeper.server.Main.storageDirectoriesFromConf;
+import static org.apache.bookkeeper.server.component.ServerLifecycleComponent.loadServerComponents;
+
+import com.google.common.base.Ticker;
+import com.google.common.util.concurrent.ThreadFactoryBuilder;
+import io.netty.buffer.ByteBuf;
+import io.netty.buffer.ByteBufAllocator;
+import io.netty.buffer.CompositeByteBuf;
+import io.reactivex.rxjava3.core.Scheduler;
+import io.reactivex.rxjava3.schedulers.Schedulers;
+import java.util.List;
+import java.util.concurrent.ExecutorService;
+import java.util.concurrent.Executors;
+import java.util.function.Consumer;
+import java.util.function.Supplier;
+import java.util.stream.Collectors;
+import lombok.extern.slf4j.Slf4j;
+import org.apache.bookkeeper.bookie.Bookie;
+import org.apache.bookkeeper.bookie.BookieImpl;
+import org.apache.bookkeeper.bookie.BookieResources;
+import org.apache.bookkeeper.bookie.CookieValidation;
+import org.apache.bookkeeper.bookie.LedgerDirsManager;
+import org.apache.bookkeeper.bookie.LedgerStorage;
+import org.apache.bookkeeper.bookie.LegacyCookieValidation;
+import org.apache.bookkeeper.bookie.ReadOnlyBookie;
+import org.apache.bookkeeper.bookie.ScrubberStats;
+import org.apache.bookkeeper.bookie.UncleanShutdownDetection;
+import org.apache.bookkeeper.bookie.UncleanShutdownDetectionImpl;
+import org.apache.bookkeeper.bookie.datainteg.DataIntegrityCheck;
+import org.apache.bookkeeper.bookie.datainteg.DataIntegrityCheckImpl;
+import org.apache.bookkeeper.bookie.datainteg.DataIntegrityCookieValidation;
+import org.apache.bookkeeper.bookie.datainteg.DataIntegrityService;
+import org.apache.bookkeeper.bookie.datainteg.EntryCopier;
+import org.apache.bookkeeper.bookie.datainteg.EntryCopierImpl;
+import org.apache.bookkeeper.client.BookKeeper;
+import org.apache.bookkeeper.client.BookKeeperAdmin;
+import org.apache.bookkeeper.common.allocator.ByteBufAllocatorWithOomHandler;
+import org.apache.bookkeeper.common.component.AutoCloseableLifecycleComponent;
+import org.apache.bookkeeper.common.component.ComponentInfoPublisher;
+import org.apache.bookkeeper.common.component.LifecycleComponentStack;
+import org.apache.bookkeeper.common.component.RxSchedulerLifecycleComponent;
+import org.apache.bookkeeper.conf.ClientConfiguration;
+import org.apache.bookkeeper.discover.BookieServiceInfo;
+import org.apache.bookkeeper.discover.RegistrationManager;
+import org.apache.bookkeeper.meta.LedgerManager;
+import org.apache.bookkeeper.meta.LedgerManagerFactory;
+import org.apache.bookkeeper.meta.MetadataBookieDriver;
+import org.apache.bookkeeper.net.BookieId;
+import org.apache.bookkeeper.server.component.ServerLifecycleComponent;
+import org.apache.bookkeeper.server.conf.BookieConfiguration;
+import org.apache.bookkeeper.server.http.BKHttpServiceProvider;
+import org.apache.bookkeeper.server.service.AutoRecoveryService;
+import org.apache.bookkeeper.server.service.BookieService;
+import org.apache.bookkeeper.server.service.HttpService;
+import org.apache.bookkeeper.server.service.ScrubberService;
+import org.apache.bookkeeper.server.service.StatsProviderService;
+import org.apache.bookkeeper.stats.StatsLogger;
+import org.apache.bookkeeper.stats.StatsProvider;
+import org.apache.bookkeeper.util.DiskChecker;
+import org.apache.commons.lang3.StringUtils;
+
+/**
+ * An embedded server is a server that run bookie and serving rpc requests.
+ *
+ * <p>
+ * It is a rewritten server using {@link org.apache.bookkeeper.common.component.LifecycleComponent}, replacing the
+ * legacy server {@link org.apache.bookkeeper.proto.BookieServer}.
+ */
+public class EmbeddedServer {
+
+    private final LifecycleComponentStack lifecycleComponentStack;
+
+    private final StatsProvider statsProvider;
+
+    private final RegistrationManager registrationManager;
+
+    private final LedgerManagerFactory ledgerManagerFactory;
+
+    private final DiskChecker diskChecker;
+    private final LedgerDirsManager ledgerDirsManager;
+    private final LedgerDirsManager indexDirsManager;
+
+    private final BookieService bookieService;
+    private final AutoRecoveryService autoRecoveryService;
+    private final DataIntegrityService dataIntegrityService;
+    private final HttpService httpService;
+
+    private EmbeddedServer(LifecycleComponentStack lifecycleComponentStack, StatsProvider statsProvider,
+                           RegistrationManager registrationManager, LedgerManagerFactory ledgerManagerFactory,
+                           DiskChecker diskChecker, LedgerDirsManager ledgerDirsManager,
+                           LedgerDirsManager indexDirsManager, BookieService bookieService,
+                           AutoRecoveryService autoRecoveryService, DataIntegrityService dataIntegrityService,
+                           HttpService httpService) {
+        this.lifecycleComponentStack = lifecycleComponentStack;
+        this.statsProvider = statsProvider;
+        this.registrationManager = registrationManager;
+        this.ledgerManagerFactory = ledgerManagerFactory;
+        this.diskChecker = diskChecker;
+        this.ledgerDirsManager = ledgerDirsManager;
+        this.indexDirsManager = indexDirsManager;
+        this.bookieService = bookieService;
+        this.autoRecoveryService = autoRecoveryService;
+        this.dataIntegrityService = dataIntegrityService;
+        this.httpService = httpService;
+    }
+
+    public LifecycleComponentStack getLifecycleComponentStack() {
+        return lifecycleComponentStack;
+    }
+
+    public StatsProvider getStatsProvider() {
+        return statsProvider;
+    }
+
+    public RegistrationManager getRegistrationManager() {
+        return registrationManager;
+    }
+
+    public LedgerManagerFactory getLedgerManagerFactory() {
+        return ledgerManagerFactory;
+    }
+
+    public DiskChecker getDiskChecker() {
+        return diskChecker;
+    }
+
+    public LedgerDirsManager getLedgerDirsManager() {
+        return ledgerDirsManager;
+    }
+
+    public LedgerDirsManager getIndexDirsManager() {
+        return indexDirsManager;
+    }
+
+    public BookieService getBookieService() {
+        return bookieService;
+    }
+
+    public AutoRecoveryService getAutoRecoveryService() {
+        return autoRecoveryService;
+    }
+
+    public DataIntegrityService getDataIntegrityService() {
+        return dataIntegrityService;
+    }
+
+    public HttpService getHttpService() {
+        return httpService;
+    }
+
+    /**
+     * Create a new builder from given configuration. Actual services implementations can be provided to the builder and
+     * will override ones defined in the configuration.
+     * <p>
+     * Invoker is responsible to start and stop provided services implementations, components from
+     * {@link EmbeddedServer#getLifecycleComponentStack()} will reflect only those created from provided configuration.
+     *
+     * @param conf bookie configuration
+     * @return a new embedded server builder
+     */
+    public static final Builder builder(BookieConfiguration conf) {
+        return new Builder(conf);
+    }
+
+    @Slf4j
+    public static class Builder {
+
+        private BookieConfiguration conf;
+
+        boolean addExternalResourcesToLifecycle = true;

Review Comment:
   Set to default false in the last commit



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

To unsubscribe, e-mail: commits-unsubscribe@bookkeeper.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


[GitHub] [bookkeeper] eolivelli commented on a diff in pull request #3489: Issue #3488 Create an EmbeddedSever extracting code from Main class and adding a builder pattern

Posted by GitBox <gi...@apache.org>.
eolivelli commented on code in PR #3489:
URL: https://github.com/apache/bookkeeper/pull/3489#discussion_r1017943696


##########
bookkeeper-server/src/main/java/org/apache/bookkeeper/server/EmbeddedServer.java:
##########
@@ -0,0 +1,648 @@
+/**
+ *
+ * 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.bookkeeper.server;
+
+import static com.google.common.base.Preconditions.checkNotNull;
+import static org.apache.bookkeeper.bookie.BookKeeperServerStats.BOOKIE_SCOPE;
+import static org.apache.bookkeeper.bookie.BookKeeperServerStats.LD_INDEX_SCOPE;
+import static org.apache.bookkeeper.bookie.BookKeeperServerStats.LD_LEDGER_SCOPE;
+import static org.apache.bookkeeper.client.BookKeeperClientStats.CLIENT_SCOPE;
+import static org.apache.bookkeeper.replication.ReplicationStats.REPLICATION_SCOPE;
+import static org.apache.bookkeeper.server.Main.storageDirectoriesFromConf;
+import static org.apache.bookkeeper.server.component.ServerLifecycleComponent.loadServerComponents;
+
+import com.google.common.base.Ticker;
+import com.google.common.util.concurrent.ThreadFactoryBuilder;
+import io.netty.buffer.ByteBuf;
+import io.netty.buffer.ByteBufAllocator;
+import io.netty.buffer.CompositeByteBuf;
+import io.reactivex.rxjava3.core.Scheduler;
+import io.reactivex.rxjava3.schedulers.Schedulers;
+import java.util.List;
+import java.util.concurrent.ExecutorService;
+import java.util.concurrent.Executors;
+import java.util.function.Consumer;
+import java.util.function.Supplier;
+import java.util.stream.Collectors;
+import lombok.extern.slf4j.Slf4j;
+import org.apache.bookkeeper.bookie.Bookie;
+import org.apache.bookkeeper.bookie.BookieImpl;
+import org.apache.bookkeeper.bookie.BookieResources;
+import org.apache.bookkeeper.bookie.CookieValidation;
+import org.apache.bookkeeper.bookie.LedgerDirsManager;
+import org.apache.bookkeeper.bookie.LedgerStorage;
+import org.apache.bookkeeper.bookie.LegacyCookieValidation;
+import org.apache.bookkeeper.bookie.ReadOnlyBookie;
+import org.apache.bookkeeper.bookie.ScrubberStats;
+import org.apache.bookkeeper.bookie.UncleanShutdownDetection;
+import org.apache.bookkeeper.bookie.UncleanShutdownDetectionImpl;
+import org.apache.bookkeeper.bookie.datainteg.DataIntegrityCheck;
+import org.apache.bookkeeper.bookie.datainteg.DataIntegrityCheckImpl;
+import org.apache.bookkeeper.bookie.datainteg.DataIntegrityCookieValidation;
+import org.apache.bookkeeper.bookie.datainteg.DataIntegrityService;
+import org.apache.bookkeeper.bookie.datainteg.EntryCopier;
+import org.apache.bookkeeper.bookie.datainteg.EntryCopierImpl;
+import org.apache.bookkeeper.client.BookKeeper;
+import org.apache.bookkeeper.client.BookKeeperAdmin;
+import org.apache.bookkeeper.common.allocator.ByteBufAllocatorWithOomHandler;
+import org.apache.bookkeeper.common.component.AutoCloseableLifecycleComponent;
+import org.apache.bookkeeper.common.component.ComponentInfoPublisher;
+import org.apache.bookkeeper.common.component.LifecycleComponentStack;
+import org.apache.bookkeeper.common.component.RxSchedulerLifecycleComponent;
+import org.apache.bookkeeper.conf.ClientConfiguration;
+import org.apache.bookkeeper.discover.BookieServiceInfo;
+import org.apache.bookkeeper.discover.RegistrationManager;
+import org.apache.bookkeeper.meta.LedgerManager;
+import org.apache.bookkeeper.meta.LedgerManagerFactory;
+import org.apache.bookkeeper.meta.MetadataBookieDriver;
+import org.apache.bookkeeper.net.BookieId;
+import org.apache.bookkeeper.server.component.ServerLifecycleComponent;
+import org.apache.bookkeeper.server.conf.BookieConfiguration;
+import org.apache.bookkeeper.server.http.BKHttpServiceProvider;
+import org.apache.bookkeeper.server.service.AutoRecoveryService;
+import org.apache.bookkeeper.server.service.BookieService;
+import org.apache.bookkeeper.server.service.HttpService;
+import org.apache.bookkeeper.server.service.ScrubberService;
+import org.apache.bookkeeper.server.service.StatsProviderService;
+import org.apache.bookkeeper.stats.StatsLogger;
+import org.apache.bookkeeper.stats.StatsProvider;
+import org.apache.bookkeeper.util.DiskChecker;
+import org.apache.commons.lang3.StringUtils;
+
+/**
+ * An embedded server is a server that run bookie and serving rpc requests.
+ *
+ * <p>
+ * It is a rewritten server using {@link org.apache.bookkeeper.common.component.LifecycleComponent}, replacing the
+ * legacy server {@link org.apache.bookkeeper.proto.BookieServer}.
+ */
+public class EmbeddedServer {
+
+    private final LifecycleComponentStack lifecycleComponentStack;
+
+    private final StatsProvider statsProvider;
+
+    private final RegistrationManager registrationManager;
+
+    private final LedgerManagerFactory ledgerManagerFactory;
+
+    private final DiskChecker diskChecker;
+    private final LedgerDirsManager ledgerDirsManager;
+    private final LedgerDirsManager indexDirsManager;
+
+    private final BookieService bookieService;
+    private final AutoRecoveryService autoRecoveryService;
+    private final DataIntegrityService dataIntegrityService;
+    private final HttpService httpService;
+
+    private EmbeddedServer(LifecycleComponentStack lifecycleComponentStack, StatsProvider statsProvider,
+                           RegistrationManager registrationManager, LedgerManagerFactory ledgerManagerFactory,
+                           DiskChecker diskChecker, LedgerDirsManager ledgerDirsManager,
+                           LedgerDirsManager indexDirsManager, BookieService bookieService,
+                           AutoRecoveryService autoRecoveryService, DataIntegrityService dataIntegrityService,
+                           HttpService httpService) {
+        this.lifecycleComponentStack = lifecycleComponentStack;
+        this.statsProvider = statsProvider;
+        this.registrationManager = registrationManager;
+        this.ledgerManagerFactory = ledgerManagerFactory;
+        this.diskChecker = diskChecker;
+        this.ledgerDirsManager = ledgerDirsManager;
+        this.indexDirsManager = indexDirsManager;
+        this.bookieService = bookieService;
+        this.autoRecoveryService = autoRecoveryService;
+        this.dataIntegrityService = dataIntegrityService;
+        this.httpService = httpService;
+    }
+
+    public LifecycleComponentStack getLifecycleComponentStack() {
+        return lifecycleComponentStack;
+    }
+
+    public StatsProvider getStatsProvider() {
+        return statsProvider;
+    }
+
+    public RegistrationManager getRegistrationManager() {
+        return registrationManager;
+    }
+
+    public LedgerManagerFactory getLedgerManagerFactory() {
+        return ledgerManagerFactory;
+    }
+
+    public DiskChecker getDiskChecker() {
+        return diskChecker;
+    }
+
+    public LedgerDirsManager getLedgerDirsManager() {
+        return ledgerDirsManager;
+    }
+
+    public LedgerDirsManager getIndexDirsManager() {
+        return indexDirsManager;
+    }
+
+    public BookieService getBookieService() {
+        return bookieService;
+    }
+
+    public AutoRecoveryService getAutoRecoveryService() {
+        return autoRecoveryService;
+    }
+
+    public DataIntegrityService getDataIntegrityService() {
+        return dataIntegrityService;
+    }
+
+    public HttpService getHttpService() {
+        return httpService;
+    }
+
+    /**
+     * Create a new builder from given configuration. Actual services implementations can be provided to the builder and
+     * will override ones defined in the configuration.
+     * <p>
+     * Invoker is responsible to start and stop provided services implementations, components from
+     * {@link EmbeddedServer#getLifecycleComponentStack()} will reflect only those created from provided configuration.
+     *
+     * @param conf bookie configuration
+     * @return a new embedded server builder
+     */
+    public static final Builder builder(BookieConfiguration conf) {
+        return new Builder(conf);
+    }
+
+    @Slf4j
+    public static class Builder {
+
+        private BookieConfiguration conf;
+
+        boolean addExternalResourcesToLifecycle = true;

Review Comment:
   if this is not required to make CI pass I prefer to drop the flag at all.
   Otherwise we add a feature that won't be used by anyone but we will have to support it forever.
   Currently in the case of Pravega, Pulsar and HerdDB that I know, there is no need to let the Bookie take control of externally provided components 



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

To unsubscribe, e-mail: commits-unsubscribe@bookkeeper.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


[GitHub] [bookkeeper] StevenLuMT commented on pull request #3489: Issue #3488 Create an EmbeddedSever extracting code from Main class and adding a builder pattern

Posted by GitBox <gi...@apache.org>.
StevenLuMT commented on PR #3489:
URL: https://github.com/apache/bookkeeper/pull/3489#issuecomment-1257371368

   rerun failure checks


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

To unsubscribe, e-mail: commits-unsubscribe@bookkeeper.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


[GitHub] [bookkeeper] diegosalvi commented on a diff in pull request #3489: Issue #3488 Create an EmbeddedSever extracting code from Main class and adding a builder pattern

Posted by GitBox <gi...@apache.org>.
diegosalvi commented on code in PR #3489:
URL: https://github.com/apache/bookkeeper/pull/3489#discussion_r1017956640


##########
bookkeeper-server/src/main/java/org/apache/bookkeeper/server/EmbeddedServer.java:
##########
@@ -0,0 +1,648 @@
+/**
+ *
+ * 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.bookkeeper.server;
+
+import static com.google.common.base.Preconditions.checkNotNull;
+import static org.apache.bookkeeper.bookie.BookKeeperServerStats.BOOKIE_SCOPE;
+import static org.apache.bookkeeper.bookie.BookKeeperServerStats.LD_INDEX_SCOPE;
+import static org.apache.bookkeeper.bookie.BookKeeperServerStats.LD_LEDGER_SCOPE;
+import static org.apache.bookkeeper.client.BookKeeperClientStats.CLIENT_SCOPE;
+import static org.apache.bookkeeper.replication.ReplicationStats.REPLICATION_SCOPE;
+import static org.apache.bookkeeper.server.Main.storageDirectoriesFromConf;
+import static org.apache.bookkeeper.server.component.ServerLifecycleComponent.loadServerComponents;
+
+import com.google.common.base.Ticker;
+import com.google.common.util.concurrent.ThreadFactoryBuilder;
+import io.netty.buffer.ByteBuf;
+import io.netty.buffer.ByteBufAllocator;
+import io.netty.buffer.CompositeByteBuf;
+import io.reactivex.rxjava3.core.Scheduler;
+import io.reactivex.rxjava3.schedulers.Schedulers;
+import java.util.List;
+import java.util.concurrent.ExecutorService;
+import java.util.concurrent.Executors;
+import java.util.function.Consumer;
+import java.util.function.Supplier;
+import java.util.stream.Collectors;
+import lombok.extern.slf4j.Slf4j;
+import org.apache.bookkeeper.bookie.Bookie;
+import org.apache.bookkeeper.bookie.BookieImpl;
+import org.apache.bookkeeper.bookie.BookieResources;
+import org.apache.bookkeeper.bookie.CookieValidation;
+import org.apache.bookkeeper.bookie.LedgerDirsManager;
+import org.apache.bookkeeper.bookie.LedgerStorage;
+import org.apache.bookkeeper.bookie.LegacyCookieValidation;
+import org.apache.bookkeeper.bookie.ReadOnlyBookie;
+import org.apache.bookkeeper.bookie.ScrubberStats;
+import org.apache.bookkeeper.bookie.UncleanShutdownDetection;
+import org.apache.bookkeeper.bookie.UncleanShutdownDetectionImpl;
+import org.apache.bookkeeper.bookie.datainteg.DataIntegrityCheck;
+import org.apache.bookkeeper.bookie.datainteg.DataIntegrityCheckImpl;
+import org.apache.bookkeeper.bookie.datainteg.DataIntegrityCookieValidation;
+import org.apache.bookkeeper.bookie.datainteg.DataIntegrityService;
+import org.apache.bookkeeper.bookie.datainteg.EntryCopier;
+import org.apache.bookkeeper.bookie.datainteg.EntryCopierImpl;
+import org.apache.bookkeeper.client.BookKeeper;
+import org.apache.bookkeeper.client.BookKeeperAdmin;
+import org.apache.bookkeeper.common.allocator.ByteBufAllocatorWithOomHandler;
+import org.apache.bookkeeper.common.component.AutoCloseableLifecycleComponent;
+import org.apache.bookkeeper.common.component.ComponentInfoPublisher;
+import org.apache.bookkeeper.common.component.LifecycleComponentStack;
+import org.apache.bookkeeper.common.component.RxSchedulerLifecycleComponent;
+import org.apache.bookkeeper.conf.ClientConfiguration;
+import org.apache.bookkeeper.discover.BookieServiceInfo;
+import org.apache.bookkeeper.discover.RegistrationManager;
+import org.apache.bookkeeper.meta.LedgerManager;
+import org.apache.bookkeeper.meta.LedgerManagerFactory;
+import org.apache.bookkeeper.meta.MetadataBookieDriver;
+import org.apache.bookkeeper.net.BookieId;
+import org.apache.bookkeeper.server.component.ServerLifecycleComponent;
+import org.apache.bookkeeper.server.conf.BookieConfiguration;
+import org.apache.bookkeeper.server.http.BKHttpServiceProvider;
+import org.apache.bookkeeper.server.service.AutoRecoveryService;
+import org.apache.bookkeeper.server.service.BookieService;
+import org.apache.bookkeeper.server.service.HttpService;
+import org.apache.bookkeeper.server.service.ScrubberService;
+import org.apache.bookkeeper.server.service.StatsProviderService;
+import org.apache.bookkeeper.stats.StatsLogger;
+import org.apache.bookkeeper.stats.StatsProvider;
+import org.apache.bookkeeper.util.DiskChecker;
+import org.apache.commons.lang3.StringUtils;
+
+/**
+ * An embedded server is a server that run bookie and serving rpc requests.
+ *
+ * <p>
+ * It is a rewritten server using {@link org.apache.bookkeeper.common.component.LifecycleComponent}, replacing the
+ * legacy server {@link org.apache.bookkeeper.proto.BookieServer}.
+ */
+public class EmbeddedServer {
+
+    private final LifecycleComponentStack lifecycleComponentStack;
+
+    private final StatsProvider statsProvider;
+
+    private final RegistrationManager registrationManager;
+
+    private final LedgerManagerFactory ledgerManagerFactory;
+
+    private final DiskChecker diskChecker;
+    private final LedgerDirsManager ledgerDirsManager;
+    private final LedgerDirsManager indexDirsManager;
+
+    private final BookieService bookieService;
+    private final AutoRecoveryService autoRecoveryService;
+    private final DataIntegrityService dataIntegrityService;
+    private final HttpService httpService;
+
+    private EmbeddedServer(LifecycleComponentStack lifecycleComponentStack, StatsProvider statsProvider,
+                           RegistrationManager registrationManager, LedgerManagerFactory ledgerManagerFactory,
+                           DiskChecker diskChecker, LedgerDirsManager ledgerDirsManager,
+                           LedgerDirsManager indexDirsManager, BookieService bookieService,
+                           AutoRecoveryService autoRecoveryService, DataIntegrityService dataIntegrityService,
+                           HttpService httpService) {
+        this.lifecycleComponentStack = lifecycleComponentStack;
+        this.statsProvider = statsProvider;
+        this.registrationManager = registrationManager;
+        this.ledgerManagerFactory = ledgerManagerFactory;
+        this.diskChecker = diskChecker;
+        this.ledgerDirsManager = ledgerDirsManager;
+        this.indexDirsManager = indexDirsManager;
+        this.bookieService = bookieService;
+        this.autoRecoveryService = autoRecoveryService;
+        this.dataIntegrityService = dataIntegrityService;
+        this.httpService = httpService;
+    }
+
+    public LifecycleComponentStack getLifecycleComponentStack() {
+        return lifecycleComponentStack;
+    }
+
+    public StatsProvider getStatsProvider() {
+        return statsProvider;
+    }
+
+    public RegistrationManager getRegistrationManager() {
+        return registrationManager;
+    }
+
+    public LedgerManagerFactory getLedgerManagerFactory() {
+        return ledgerManagerFactory;
+    }
+
+    public DiskChecker getDiskChecker() {
+        return diskChecker;
+    }
+
+    public LedgerDirsManager getLedgerDirsManager() {
+        return ledgerDirsManager;
+    }
+
+    public LedgerDirsManager getIndexDirsManager() {
+        return indexDirsManager;
+    }
+
+    public BookieService getBookieService() {
+        return bookieService;
+    }
+
+    public AutoRecoveryService getAutoRecoveryService() {
+        return autoRecoveryService;
+    }
+
+    public DataIntegrityService getDataIntegrityService() {
+        return dataIntegrityService;
+    }
+
+    public HttpService getHttpService() {
+        return httpService;
+    }
+
+    /**
+     * Create a new builder from given configuration. Actual services implementations can be provided to the builder and
+     * will override ones defined in the configuration.
+     * <p>
+     * Invoker is responsible to start and stop provided services implementations, components from
+     * {@link EmbeddedServer#getLifecycleComponentStack()} will reflect only those created from provided configuration.
+     *
+     * @param conf bookie configuration
+     * @return a new embedded server builder
+     */
+    public static final Builder builder(BookieConfiguration conf) {
+        return new Builder(conf);
+    }
+
+    @Slf4j
+    public static class Builder {
+
+        private BookieConfiguration conf;
+
+        boolean addExternalResourcesToLifecycle = true;

Review Comment:
   Removed addExternalResourcesToLifecycle



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

To unsubscribe, e-mail: commits-unsubscribe@bookkeeper.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


[GitHub] [bookkeeper] diegosalvi commented on pull request #3489: Issue #3488 Create an EmbeddedSever extracting code from Main class and adding a builder pattern

Posted by GitBox <gi...@apache.org>.
diegosalvi commented on PR #3489:
URL: https://github.com/apache/bookkeeper/pull/3489#issuecomment-1308642056

   Finally checks all green


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

To unsubscribe, e-mail: commits-unsubscribe@bookkeeper.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


[GitHub] [bookkeeper] eolivelli merged pull request #3489: Issue #3488 Create an EmbeddedSever extracting code from Main class and adding a builder pattern

Posted by GitBox <gi...@apache.org>.
eolivelli merged PR #3489:
URL: https://github.com/apache/bookkeeper/pull/3489


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

To unsubscribe, e-mail: commits-unsubscribe@bookkeeper.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


[GitHub] [bookkeeper] StevenLuMT commented on pull request #3489: Issue #3488 Create an EmbeddedSever extracting code from Main class and adding a builder pattern

Posted by GitBox <gi...@apache.org>.
StevenLuMT commented on PR #3489:
URL: https://github.com/apache/bookkeeper/pull/3489#issuecomment-1257160863

   rerun failure checks


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

To unsubscribe, e-mail: commits-unsubscribe@bookkeeper.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


[GitHub] [bookkeeper] StevenLuMT commented on pull request #3489: Issue #3488 Create an EmbeddedSever extracting code from Main class and adding a builder pattern

Posted by GitBox <gi...@apache.org>.
StevenLuMT commented on PR #3489:
URL: https://github.com/apache/bookkeeper/pull/3489#issuecomment-1260542203

   > LGTM. There are couple of failing tests on the CI, on the first glance seem to be related to the change. Please take a look.
   
   @diegosalvi have a look at failing tests on the CI


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

To unsubscribe, e-mail: commits-unsubscribe@bookkeeper.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


[GitHub] [bookkeeper] diegosalvi commented on a diff in pull request #3489: Issue #3488 Create an EmbeddedSever extracting code from Main class and adding a builder pattern

Posted by GitBox <gi...@apache.org>.
diegosalvi commented on code in PR #3489:
URL: https://github.com/apache/bookkeeper/pull/3489#discussion_r1017763187


##########
bookkeeper-server/src/main/java/org/apache/bookkeeper/server/EmbeddedServer.java:
##########
@@ -0,0 +1,648 @@
+/**
+ *
+ * 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.bookkeeper.server;
+
+import static com.google.common.base.Preconditions.checkNotNull;
+import static org.apache.bookkeeper.bookie.BookKeeperServerStats.BOOKIE_SCOPE;
+import static org.apache.bookkeeper.bookie.BookKeeperServerStats.LD_INDEX_SCOPE;
+import static org.apache.bookkeeper.bookie.BookKeeperServerStats.LD_LEDGER_SCOPE;
+import static org.apache.bookkeeper.client.BookKeeperClientStats.CLIENT_SCOPE;
+import static org.apache.bookkeeper.replication.ReplicationStats.REPLICATION_SCOPE;
+import static org.apache.bookkeeper.server.Main.storageDirectoriesFromConf;
+import static org.apache.bookkeeper.server.component.ServerLifecycleComponent.loadServerComponents;
+
+import com.google.common.base.Ticker;
+import com.google.common.util.concurrent.ThreadFactoryBuilder;
+import io.netty.buffer.ByteBuf;
+import io.netty.buffer.ByteBufAllocator;
+import io.netty.buffer.CompositeByteBuf;
+import io.reactivex.rxjava3.core.Scheduler;
+import io.reactivex.rxjava3.schedulers.Schedulers;
+import java.util.List;
+import java.util.concurrent.ExecutorService;
+import java.util.concurrent.Executors;
+import java.util.function.Consumer;
+import java.util.function.Supplier;
+import java.util.stream.Collectors;
+import lombok.extern.slf4j.Slf4j;
+import org.apache.bookkeeper.bookie.Bookie;
+import org.apache.bookkeeper.bookie.BookieImpl;
+import org.apache.bookkeeper.bookie.BookieResources;
+import org.apache.bookkeeper.bookie.CookieValidation;
+import org.apache.bookkeeper.bookie.LedgerDirsManager;
+import org.apache.bookkeeper.bookie.LedgerStorage;
+import org.apache.bookkeeper.bookie.LegacyCookieValidation;
+import org.apache.bookkeeper.bookie.ReadOnlyBookie;
+import org.apache.bookkeeper.bookie.ScrubberStats;
+import org.apache.bookkeeper.bookie.UncleanShutdownDetection;
+import org.apache.bookkeeper.bookie.UncleanShutdownDetectionImpl;
+import org.apache.bookkeeper.bookie.datainteg.DataIntegrityCheck;
+import org.apache.bookkeeper.bookie.datainteg.DataIntegrityCheckImpl;
+import org.apache.bookkeeper.bookie.datainteg.DataIntegrityCookieValidation;
+import org.apache.bookkeeper.bookie.datainteg.DataIntegrityService;
+import org.apache.bookkeeper.bookie.datainteg.EntryCopier;
+import org.apache.bookkeeper.bookie.datainteg.EntryCopierImpl;
+import org.apache.bookkeeper.client.BookKeeper;
+import org.apache.bookkeeper.client.BookKeeperAdmin;
+import org.apache.bookkeeper.common.allocator.ByteBufAllocatorWithOomHandler;
+import org.apache.bookkeeper.common.component.AutoCloseableLifecycleComponent;
+import org.apache.bookkeeper.common.component.ComponentInfoPublisher;
+import org.apache.bookkeeper.common.component.LifecycleComponentStack;
+import org.apache.bookkeeper.common.component.RxSchedulerLifecycleComponent;
+import org.apache.bookkeeper.conf.ClientConfiguration;
+import org.apache.bookkeeper.discover.BookieServiceInfo;
+import org.apache.bookkeeper.discover.RegistrationManager;
+import org.apache.bookkeeper.meta.LedgerManager;
+import org.apache.bookkeeper.meta.LedgerManagerFactory;
+import org.apache.bookkeeper.meta.MetadataBookieDriver;
+import org.apache.bookkeeper.net.BookieId;
+import org.apache.bookkeeper.server.component.ServerLifecycleComponent;
+import org.apache.bookkeeper.server.conf.BookieConfiguration;
+import org.apache.bookkeeper.server.http.BKHttpServiceProvider;
+import org.apache.bookkeeper.server.service.AutoRecoveryService;
+import org.apache.bookkeeper.server.service.BookieService;
+import org.apache.bookkeeper.server.service.HttpService;
+import org.apache.bookkeeper.server.service.ScrubberService;
+import org.apache.bookkeeper.server.service.StatsProviderService;
+import org.apache.bookkeeper.stats.StatsLogger;
+import org.apache.bookkeeper.stats.StatsProvider;
+import org.apache.bookkeeper.util.DiskChecker;
+import org.apache.commons.lang3.StringUtils;
+
+/**
+ * An embedded server is a server that run bookie and serving rpc requests.
+ *
+ * <p>
+ * It is a rewritten server using {@link org.apache.bookkeeper.common.component.LifecycleComponent}, replacing the
+ * legacy server {@link org.apache.bookkeeper.proto.BookieServer}.
+ */
+public class EmbeddedServer {
+
+    private final LifecycleComponentStack lifecycleComponentStack;
+
+    private final StatsProvider statsProvider;
+
+    private final RegistrationManager registrationManager;
+
+    private final LedgerManagerFactory ledgerManagerFactory;
+
+    private final DiskChecker diskChecker;
+    private final LedgerDirsManager ledgerDirsManager;
+    private final LedgerDirsManager indexDirsManager;
+
+    private final BookieService bookieService;
+    private final AutoRecoveryService autoRecoveryService;
+    private final DataIntegrityService dataIntegrityService;
+    private final HttpService httpService;
+
+    private EmbeddedServer(LifecycleComponentStack lifecycleComponentStack, StatsProvider statsProvider,
+                           RegistrationManager registrationManager, LedgerManagerFactory ledgerManagerFactory,
+                           DiskChecker diskChecker, LedgerDirsManager ledgerDirsManager,
+                           LedgerDirsManager indexDirsManager, BookieService bookieService,
+                           AutoRecoveryService autoRecoveryService, DataIntegrityService dataIntegrityService,
+                           HttpService httpService) {
+        this.lifecycleComponentStack = lifecycleComponentStack;
+        this.statsProvider = statsProvider;
+        this.registrationManager = registrationManager;
+        this.ledgerManagerFactory = ledgerManagerFactory;
+        this.diskChecker = diskChecker;
+        this.ledgerDirsManager = ledgerDirsManager;
+        this.indexDirsManager = indexDirsManager;
+        this.bookieService = bookieService;
+        this.autoRecoveryService = autoRecoveryService;
+        this.dataIntegrityService = dataIntegrityService;
+        this.httpService = httpService;
+    }
+
+    public LifecycleComponentStack getLifecycleComponentStack() {
+        return lifecycleComponentStack;
+    }
+
+    public StatsProvider getStatsProvider() {
+        return statsProvider;
+    }
+
+    public RegistrationManager getRegistrationManager() {
+        return registrationManager;
+    }
+
+    public LedgerManagerFactory getLedgerManagerFactory() {
+        return ledgerManagerFactory;
+    }
+
+    public DiskChecker getDiskChecker() {
+        return diskChecker;
+    }
+
+    public LedgerDirsManager getLedgerDirsManager() {
+        return ledgerDirsManager;
+    }
+
+    public LedgerDirsManager getIndexDirsManager() {
+        return indexDirsManager;
+    }
+
+    public BookieService getBookieService() {
+        return bookieService;
+    }
+
+    public AutoRecoveryService getAutoRecoveryService() {
+        return autoRecoveryService;
+    }
+
+    public DataIntegrityService getDataIntegrityService() {
+        return dataIntegrityService;
+    }
+
+    public HttpService getHttpService() {
+        return httpService;
+    }
+
+    /**
+     * Create a new builder from given configuration. Actual services implementations can be provided to the builder and
+     * will override ones defined in the configuration.
+     * <p>
+     * Invoker is responsible to start and stop provided services implementations, components from
+     * {@link EmbeddedServer#getLifecycleComponentStack()} will reflect only those created from provided configuration.
+     *
+     * @param conf bookie configuration
+     * @return a new embedded server builder
+     */
+    public static final Builder builder(BookieConfiguration conf) {
+        return new Builder(conf);
+    }
+
+    @Slf4j
+    public static class Builder {
+
+        private BookieConfiguration conf;
+
+        boolean addExternalResourcesToLifecycle = true;

Review Comment:
   Sorry is a WIP fix but I had to see running it on CI.
   I think is useful to let BK to handle and close provided components but this behaviour should be off by default



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

To unsubscribe, e-mail: commits-unsubscribe@bookkeeper.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


[GitHub] [bookkeeper] diegosalvi commented on a diff in pull request #3489: Issue #3488 Create an EmbeddedSever extracting code from Main class and adding a builder pattern

Posted by GitBox <gi...@apache.org>.
diegosalvi commented on code in PR #3489:
URL: https://github.com/apache/bookkeeper/pull/3489#discussion_r1017763187


##########
bookkeeper-server/src/main/java/org/apache/bookkeeper/server/EmbeddedServer.java:
##########
@@ -0,0 +1,648 @@
+/**
+ *
+ * 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.bookkeeper.server;
+
+import static com.google.common.base.Preconditions.checkNotNull;
+import static org.apache.bookkeeper.bookie.BookKeeperServerStats.BOOKIE_SCOPE;
+import static org.apache.bookkeeper.bookie.BookKeeperServerStats.LD_INDEX_SCOPE;
+import static org.apache.bookkeeper.bookie.BookKeeperServerStats.LD_LEDGER_SCOPE;
+import static org.apache.bookkeeper.client.BookKeeperClientStats.CLIENT_SCOPE;
+import static org.apache.bookkeeper.replication.ReplicationStats.REPLICATION_SCOPE;
+import static org.apache.bookkeeper.server.Main.storageDirectoriesFromConf;
+import static org.apache.bookkeeper.server.component.ServerLifecycleComponent.loadServerComponents;
+
+import com.google.common.base.Ticker;
+import com.google.common.util.concurrent.ThreadFactoryBuilder;
+import io.netty.buffer.ByteBuf;
+import io.netty.buffer.ByteBufAllocator;
+import io.netty.buffer.CompositeByteBuf;
+import io.reactivex.rxjava3.core.Scheduler;
+import io.reactivex.rxjava3.schedulers.Schedulers;
+import java.util.List;
+import java.util.concurrent.ExecutorService;
+import java.util.concurrent.Executors;
+import java.util.function.Consumer;
+import java.util.function.Supplier;
+import java.util.stream.Collectors;
+import lombok.extern.slf4j.Slf4j;
+import org.apache.bookkeeper.bookie.Bookie;
+import org.apache.bookkeeper.bookie.BookieImpl;
+import org.apache.bookkeeper.bookie.BookieResources;
+import org.apache.bookkeeper.bookie.CookieValidation;
+import org.apache.bookkeeper.bookie.LedgerDirsManager;
+import org.apache.bookkeeper.bookie.LedgerStorage;
+import org.apache.bookkeeper.bookie.LegacyCookieValidation;
+import org.apache.bookkeeper.bookie.ReadOnlyBookie;
+import org.apache.bookkeeper.bookie.ScrubberStats;
+import org.apache.bookkeeper.bookie.UncleanShutdownDetection;
+import org.apache.bookkeeper.bookie.UncleanShutdownDetectionImpl;
+import org.apache.bookkeeper.bookie.datainteg.DataIntegrityCheck;
+import org.apache.bookkeeper.bookie.datainteg.DataIntegrityCheckImpl;
+import org.apache.bookkeeper.bookie.datainteg.DataIntegrityCookieValidation;
+import org.apache.bookkeeper.bookie.datainteg.DataIntegrityService;
+import org.apache.bookkeeper.bookie.datainteg.EntryCopier;
+import org.apache.bookkeeper.bookie.datainteg.EntryCopierImpl;
+import org.apache.bookkeeper.client.BookKeeper;
+import org.apache.bookkeeper.client.BookKeeperAdmin;
+import org.apache.bookkeeper.common.allocator.ByteBufAllocatorWithOomHandler;
+import org.apache.bookkeeper.common.component.AutoCloseableLifecycleComponent;
+import org.apache.bookkeeper.common.component.ComponentInfoPublisher;
+import org.apache.bookkeeper.common.component.LifecycleComponentStack;
+import org.apache.bookkeeper.common.component.RxSchedulerLifecycleComponent;
+import org.apache.bookkeeper.conf.ClientConfiguration;
+import org.apache.bookkeeper.discover.BookieServiceInfo;
+import org.apache.bookkeeper.discover.RegistrationManager;
+import org.apache.bookkeeper.meta.LedgerManager;
+import org.apache.bookkeeper.meta.LedgerManagerFactory;
+import org.apache.bookkeeper.meta.MetadataBookieDriver;
+import org.apache.bookkeeper.net.BookieId;
+import org.apache.bookkeeper.server.component.ServerLifecycleComponent;
+import org.apache.bookkeeper.server.conf.BookieConfiguration;
+import org.apache.bookkeeper.server.http.BKHttpServiceProvider;
+import org.apache.bookkeeper.server.service.AutoRecoveryService;
+import org.apache.bookkeeper.server.service.BookieService;
+import org.apache.bookkeeper.server.service.HttpService;
+import org.apache.bookkeeper.server.service.ScrubberService;
+import org.apache.bookkeeper.server.service.StatsProviderService;
+import org.apache.bookkeeper.stats.StatsLogger;
+import org.apache.bookkeeper.stats.StatsProvider;
+import org.apache.bookkeeper.util.DiskChecker;
+import org.apache.commons.lang3.StringUtils;
+
+/**
+ * An embedded server is a server that run bookie and serving rpc requests.
+ *
+ * <p>
+ * It is a rewritten server using {@link org.apache.bookkeeper.common.component.LifecycleComponent}, replacing the
+ * legacy server {@link org.apache.bookkeeper.proto.BookieServer}.
+ */
+public class EmbeddedServer {
+
+    private final LifecycleComponentStack lifecycleComponentStack;
+
+    private final StatsProvider statsProvider;
+
+    private final RegistrationManager registrationManager;
+
+    private final LedgerManagerFactory ledgerManagerFactory;
+
+    private final DiskChecker diskChecker;
+    private final LedgerDirsManager ledgerDirsManager;
+    private final LedgerDirsManager indexDirsManager;
+
+    private final BookieService bookieService;
+    private final AutoRecoveryService autoRecoveryService;
+    private final DataIntegrityService dataIntegrityService;
+    private final HttpService httpService;
+
+    private EmbeddedServer(LifecycleComponentStack lifecycleComponentStack, StatsProvider statsProvider,
+                           RegistrationManager registrationManager, LedgerManagerFactory ledgerManagerFactory,
+                           DiskChecker diskChecker, LedgerDirsManager ledgerDirsManager,
+                           LedgerDirsManager indexDirsManager, BookieService bookieService,
+                           AutoRecoveryService autoRecoveryService, DataIntegrityService dataIntegrityService,
+                           HttpService httpService) {
+        this.lifecycleComponentStack = lifecycleComponentStack;
+        this.statsProvider = statsProvider;
+        this.registrationManager = registrationManager;
+        this.ledgerManagerFactory = ledgerManagerFactory;
+        this.diskChecker = diskChecker;
+        this.ledgerDirsManager = ledgerDirsManager;
+        this.indexDirsManager = indexDirsManager;
+        this.bookieService = bookieService;
+        this.autoRecoveryService = autoRecoveryService;
+        this.dataIntegrityService = dataIntegrityService;
+        this.httpService = httpService;
+    }
+
+    public LifecycleComponentStack getLifecycleComponentStack() {
+        return lifecycleComponentStack;
+    }
+
+    public StatsProvider getStatsProvider() {
+        return statsProvider;
+    }
+
+    public RegistrationManager getRegistrationManager() {
+        return registrationManager;
+    }
+
+    public LedgerManagerFactory getLedgerManagerFactory() {
+        return ledgerManagerFactory;
+    }
+
+    public DiskChecker getDiskChecker() {
+        return diskChecker;
+    }
+
+    public LedgerDirsManager getLedgerDirsManager() {
+        return ledgerDirsManager;
+    }
+
+    public LedgerDirsManager getIndexDirsManager() {
+        return indexDirsManager;
+    }
+
+    public BookieService getBookieService() {
+        return bookieService;
+    }
+
+    public AutoRecoveryService getAutoRecoveryService() {
+        return autoRecoveryService;
+    }
+
+    public DataIntegrityService getDataIntegrityService() {
+        return dataIntegrityService;
+    }
+
+    public HttpService getHttpService() {
+        return httpService;
+    }
+
+    /**
+     * Create a new builder from given configuration. Actual services implementations can be provided to the builder and
+     * will override ones defined in the configuration.
+     * <p>
+     * Invoker is responsible to start and stop provided services implementations, components from
+     * {@link EmbeddedServer#getLifecycleComponentStack()} will reflect only those created from provided configuration.
+     *
+     * @param conf bookie configuration
+     * @return a new embedded server builder
+     */
+    public static final Builder builder(BookieConfiguration conf) {
+        return new Builder(conf);
+    }
+
+    @Slf4j
+    public static class Builder {
+
+        private BookieConfiguration conf;
+
+        boolean addExternalResourcesToLifecycle = true;

Review Comment:
   Sorry is a WIP fix but I had to see running it on CI



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

To unsubscribe, e-mail: commits-unsubscribe@bookkeeper.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org