heesung-sn commented on code in PR #19045: URL: https://github.com/apache/pulsar/pull/19045#discussion_r1084364025
########## pulsar-broker/src/main/java/org/apache/pulsar/broker/loadbalance/extensions/channel/ServiceUnitStateCompactionStrategy.java: ########## @@ -0,0 +1,90 @@ +/* + * 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.pulsar.broker.loadbalance.extensions.channel; + +import static org.apache.pulsar.broker.loadbalance.extensions.channel.ServiceUnitState.Assigned; +import static org.apache.pulsar.broker.loadbalance.extensions.channel.ServiceUnitState.Free; +import static org.apache.pulsar.broker.loadbalance.extensions.channel.ServiceUnitState.Owned; +import static org.apache.pulsar.broker.loadbalance.extensions.channel.ServiceUnitState.Released; +import static org.apache.pulsar.broker.loadbalance.extensions.channel.ServiceUnitState.Splitting; +import com.google.common.annotations.VisibleForTesting; +import org.apache.commons.lang.StringUtils; +import org.apache.pulsar.client.api.Schema; +import org.apache.pulsar.client.impl.schema.JSONSchema; +import org.apache.pulsar.common.topics.TopicCompactionStrategy; + +public class ServiceUnitStateCompactionStrategy implements TopicCompactionStrategy<ServiceUnitStateData> { + + private final Schema<ServiceUnitStateData> schema; + + private boolean checkBrokers = true; + + public ServiceUnitStateCompactionStrategy() { + schema = JSONSchema.of(ServiceUnitStateData.class); Review Comment: Updated ########## pulsar-broker/src/test/java/org/apache/pulsar/compaction/ServiceUnitStateCompactionTest.java: ########## @@ -0,0 +1,833 @@ +/* + * 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.pulsar.compaction; + +import static org.apache.pulsar.broker.loadbalance.extensions.channel.ServiceUnitState.Free; +import static org.apache.pulsar.broker.loadbalance.extensions.channel.ServiceUnitState.Owned; +import static org.apache.pulsar.broker.loadbalance.extensions.channel.ServiceUnitState.Assigned; +import static org.apache.pulsar.broker.loadbalance.extensions.channel.ServiceUnitState.Splitting; +import static org.apache.pulsar.broker.loadbalance.extensions.channel.ServiceUnitState.isValidTransition; +import static org.testng.Assert.assertEquals; +import static org.testng.Assert.assertNotNull; +import static org.testng.Assert.assertNull; +import static org.testng.Assert.assertTrue; +import com.google.common.collect.Sets; +import com.google.common.util.concurrent.ThreadFactoryBuilder; +import java.util.ArrayList; +import java.util.Arrays; +import java.util.HashMap; +import java.util.List; +import java.util.Map; +import java.util.Optional; +import java.util.Random; +import java.util.Set; +import java.util.UUID; +import java.util.concurrent.CompletableFuture; +import java.util.concurrent.ExecutionException; +import java.util.concurrent.Executors; +import java.util.concurrent.ScheduledExecutorService; +import java.util.concurrent.TimeUnit; +import java.util.stream.Collectors; +import org.apache.bookkeeper.client.BookKeeper; +import org.apache.commons.lang.reflect.FieldUtils; +import org.apache.commons.lang3.tuple.Pair; +import org.apache.pulsar.broker.auth.MockedPulsarServiceBaseTest; +import org.apache.pulsar.broker.loadbalance.extensions.channel.ServiceUnitState; +import org.apache.pulsar.broker.loadbalance.extensions.channel.ServiceUnitStateCompactionStrategy; +import org.apache.pulsar.broker.loadbalance.extensions.channel.ServiceUnitStateData; +import org.apache.pulsar.client.admin.PulsarAdminException; +import org.apache.pulsar.client.api.Consumer; +import org.apache.pulsar.client.api.Message; +import org.apache.pulsar.client.api.MessageId; +import org.apache.pulsar.client.api.MessageRoutingMode; +import org.apache.pulsar.client.api.Producer; +import org.apache.pulsar.client.api.ProducerBuilder; +import org.apache.pulsar.client.api.PulsarClientException; +import org.apache.pulsar.client.api.Reader; +import org.apache.pulsar.client.api.SubscriptionInitialPosition; +import org.apache.pulsar.client.impl.schema.JSONSchema; +import org.apache.pulsar.common.policies.data.ClusterData; +import org.apache.pulsar.common.policies.data.PersistentTopicInternalStats; +import org.apache.pulsar.common.policies.data.RetentionPolicies; +import org.apache.pulsar.common.policies.data.TenantInfoImpl; +import org.apache.pulsar.common.util.FutureUtil; +import org.awaitility.Awaitility; + +import org.testng.Assert; +import org.testng.annotations.AfterMethod; +import org.testng.annotations.BeforeMethod; +import org.testng.annotations.Test; + +@Test(groups = "broker-compaction") +public class ServiceUnitStateCompactionTest extends MockedPulsarServiceBaseTest { + private ScheduledExecutorService compactionScheduler; + private BookKeeper bk; + private JSONSchema<ServiceUnitStateData> schema; + private ServiceUnitStateCompactionStrategy strategy; + + private ServiceUnitState testState0 = Free; + private ServiceUnitState testState1 = Free; + private ServiceUnitState testState2 = Free; + private ServiceUnitState testState3 = Free; + private ServiceUnitState testState4 = Free; + + private static Random RANDOM = new Random(); + + + private ServiceUnitStateData testValue(ServiceUnitState state, String broker) { + if (state == Free) { + return null; + } + return new ServiceUnitStateData(state, broker); + } + + private ServiceUnitStateData testValue0(String broker) { + ServiceUnitState to = nextValidState(testState0); + testState0 = to; + return testValue(to, broker); + } + + private ServiceUnitStateData testValue1(String broker) { + ServiceUnitState to = nextValidState(testState1); + testState1 = to; + return testValue(to, broker); + } + + private ServiceUnitStateData testValue2(String broker) { + ServiceUnitState to = nextValidState(testState2); + testState2 = to; + return testValue(to, broker); + } + + private ServiceUnitStateData testValue3(String broker) { + ServiceUnitState to = nextValidState(testState3); + testState3 = to; + return testValue(to, broker); + } + + private ServiceUnitStateData testValue4(String broker) { + ServiceUnitState to = nextValidState(testState4); + testState4 = to; + return testValue(to, broker); + } + + private ServiceUnitState nextValidState(ServiceUnitState from) { + List<ServiceUnitState> candidates = Arrays.stream(ServiceUnitState.values()) + .filter(to -> to != Free && to != Splitting && isValidTransition(from, to)) + .collect(Collectors.toList()); + var state= candidates.get(RANDOM.nextInt(candidates.size())); + System.out.println("test state:" + state); + return state; + } + + private ServiceUnitState nextInvalidState(ServiceUnitState from) { + List<ServiceUnitState> candidates = Arrays.stream(ServiceUnitState.values()) + .filter(to -> !isValidTransition(from, to)) + .collect(Collectors.toList()); + if (candidates.size() == 0) { + return null; + } + return candidates.get(RANDOM.nextInt(candidates.size())); + } + + private List<ServiceUnitState> nextStatesToNull(ServiceUnitState from) { + if (from == null) { + return List.of(); + } + return switch (from) { + case Assigned -> List.of(Owned); + case Owned -> List.of(); + case Splitting -> List.of(); + default -> List.of(); + }; + } + + @BeforeMethod + @Override + public void setup() throws Exception { + super.internalSetup(); + + admin.clusters().createCluster("use", ClusterData.builder().serviceUrl(pulsar.getWebServiceAddress()).build()); + admin.tenants().createTenant("my-property", + new TenantInfoImpl(Sets.newHashSet("appid1", "appid2"), Sets.newHashSet("use"))); + admin.namespaces().createNamespace("my-property/use/my-ns"); + + compactionScheduler = Executors.newSingleThreadScheduledExecutor( + new ThreadFactoryBuilder().setNameFormat("compaction-%d").setDaemon(true).build()); + bk = pulsar.getBookKeeperClientFactory().create(this.conf, null, null, Optional.empty(), null); + schema = JSONSchema.of(ServiceUnitStateData.class); Review Comment: updated. ########## pulsar-broker/src/main/java/org/apache/pulsar/compaction/StrategicTwoPhaseCompactor.java: ########## @@ -385,6 +385,7 @@ private <T> void phaseTwoLoop(String topic, Iterator<Message<T>> reader, promise.completeExceptionally(e); return; } + outstanding.release(MAX_OUTSTANDING); Review Comment: No, I found a deadlock if we don't release this semaphore before completing the future. deadLock example ``` import java.util.concurrent.CompletableFuture; import java.util.concurrent.ExecutorService; import java.util.concurrent.Executors; import java.util.concurrent.Semaphore; import org.junit.Test; public class SemaphoreTest { @Test public void deadlockTest() { Semaphore sm = new Semaphore(2); ExecutorService executor = Executors.newSingleThreadExecutor(); CompletableFuture<String> future = new CompletableFuture(); CompletableFuture.supplyAsync(() -> { System.out.println("starting"); try { sm.acquire(2); System.out.println("acquired 2"); } catch (InterruptedException e) { } try { Thread.sleep(1000*5); } catch (InterruptedException e) { } System.out.println("return!"); future.complete(""); // hangs here. System.out.println("proceeding"); sm.release(2); System.out.println("released 2"); return ""; }, executor); future.thenCompose(x -> { try { System.out.println("acquiring one!"); sm.acquire(); // hangs here. System.out.println("acquired 1"); sm.release(); } catch (InterruptedException e) { } return CompletableFuture.completedFuture(""); }).join(); } } ``` -- 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: [email protected] For queries about this service, please contact Infrastructure at: [email protected]
