This is an automated email from the ASF dual-hosted git repository.

eolivelli pushed a commit to branch branch-2.7
in repository https://gitbox.apache.org/repos/asf/pulsar.git


The following commit(s) were added to refs/heads/branch-2.7 by this push:
     new 26ad7e4  Branch-2.7: Remove BookKeeper OpportunisticStriping tests 
(#10451)
26ad7e4 is described below

commit 26ad7e4568b4250a17ac0bfd9fdd0f5bb655e32e
Author: Enrico Olivelli <eolive...@gmail.com>
AuthorDate: Fri Apr 30 08:21:46 2021 +0200

    Branch-2.7: Remove BookKeeper OpportunisticStriping tests (#10451)
    
    - OpportunisticStriping is not available on Pulsar 2.7.x
    - remove tests and demo configuration
    - adapt test in order to verify 'bookkeeper_xxx' configuration
    
    Co-authored-by: Enrico Olivelli <eolive...@apache.org>
---
 conf/broker.conf                                   |  4 -
 .../broker/BookKeeperClientFactoryImplTest.java    | 12 +--
 .../broker/service/OpportunisticStripingTest.java  | 95 ----------------------
 3 files changed, 6 insertions(+), 105 deletions(-)

diff --git a/conf/broker.conf b/conf/broker.conf
index 4e689c6..f286961 100644
--- a/conf/broker.conf
+++ b/conf/broker.conf
@@ -791,10 +791,6 @@ managedLedgerDefaultWriteQuorum=2
 # Number of guaranteed copies (acks to wait before write is complete)
 managedLedgerDefaultAckQuorum=2
 
-# with OpportunisticStriping=true the ensembleSize is adapted automatically to 
writeQuorum
-# in case of lack of enough bookies
-#bookkeeper_opportunisticStriping=false
-
 # you can add other configuration options for the BookKeeper client
 # by prefixing them with bookkeeper_
 
diff --git 
a/pulsar-broker/src/test/java/org/apache/pulsar/broker/BookKeeperClientFactoryImplTest.java
 
b/pulsar-broker/src/test/java/org/apache/pulsar/broker/BookKeeperClientFactoryImplTest.java
index 13c352c..183c7c9 100644
--- 
a/pulsar-broker/src/test/java/org/apache/pulsar/broker/BookKeeperClientFactoryImplTest.java
+++ 
b/pulsar-broker/src/test/java/org/apache/pulsar/broker/BookKeeperClientFactoryImplTest.java
@@ -206,15 +206,15 @@ public class BookKeeperClientFactoryImplTest {
     }
 
     @Test
-    public void testOpportunisticStripingConfiguration() {
+    public void testBookKeeperCustomConfiguration() {
         BookKeeperClientFactoryImpl factory = new 
BookKeeperClientFactoryImpl();
         ServiceConfiguration conf = new ServiceConfiguration();
         // default value
-        
assertEquals(factory.createBkClientConfiguration(conf).getOpportunisticStriping(),
 false);
-        conf.getProperties().setProperty("bookkeeper_opportunisticStriping", 
"true");
-        
assertEquals(factory.createBkClientConfiguration(conf).getOpportunisticStriping(),
 true);
-        conf.getProperties().setProperty("bookkeeper_opportunisticStriping", 
"false");
-        
assertEquals(factory.createBkClientConfiguration(conf).getOpportunisticStriping(),
 false);
+        
assertEquals(factory.createBkClientConfiguration(conf).getAddEntryQuorumTimeout(),
 -1);
+        
conf.getProperties().setProperty("bookkeeper_addEntryQuorumTimeoutSec", "10");
+        
assertEquals(factory.createBkClientConfiguration(conf).getAddEntryQuorumTimeout(),
 10);
+        
conf.getProperties().setProperty("bookkeeper_addEntryQuorumTimeoutSec", "20");
+        
assertEquals(factory.createBkClientConfiguration(conf).getAddEntryQuorumTimeout(),
 20);
 
     }
 
diff --git 
a/pulsar-broker/src/test/java/org/apache/pulsar/broker/service/OpportunisticStripingTest.java
 
b/pulsar-broker/src/test/java/org/apache/pulsar/broker/service/OpportunisticStripingTest.java
deleted file mode 100644
index 221367c..0000000
--- 
a/pulsar-broker/src/test/java/org/apache/pulsar/broker/service/OpportunisticStripingTest.java
+++ /dev/null
@@ -1,95 +0,0 @@
-/**
- * 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.service;
-
-import java.util.concurrent.TimeUnit;
-import org.apache.bookkeeper.client.api.BookKeeper;
-import org.apache.bookkeeper.client.api.LedgerMetadata;
-import org.apache.bookkeeper.client.api.ListLedgersResult;
-import org.apache.bookkeeper.conf.ClientConfiguration;
-import org.apache.pulsar.broker.ServiceConfiguration;
-import org.apache.pulsar.client.api.Producer;
-import org.apache.pulsar.client.api.PulsarClient;
-import static org.testng.AssertJUnit.assertEquals;
-import static org.testng.AssertJUnit.assertTrue;
-import org.testng.annotations.Test;
-
-/**
- * With BookKeeper Opportunistic Striping feature we can allow Pulsar to work
- * with only WQ bookie during temporary outages of some bookie.
- */
-public class OpportunisticStripingTest extends BkEnsemblesTestBase {
-
-    public OpportunisticStripingTest() {
-        // starting only two bookies
-        super(2);
-    }
-
-    @Override
-    protected void configurePulsar(ServiceConfiguration config) throws 
Exception {
-        // we would like to stripe over 5 bookies
-        config.setManagedLedgerDefaultEnsembleSize(5);
-        // we want 2 copies for each entry
-        config.setManagedLedgerDefaultWriteQuorum(2);
-        config.setManagedLedgerDefaultAckQuorum(2);
-
-        config.setBrokerDeleteInactiveTopicsEnabled(false);
-        config.getProperties().setProperty("bookkeeper_opportunisticStriping", 
"true");
-    }
-
-    @Test
-    public void testOpportunisticStriping() throws Exception {
-
-        try (PulsarClient client = PulsarClient.builder()
-                .serviceUrl(pulsar.getWebServiceAddress())
-                .statsInterval(0, TimeUnit.SECONDS)
-                .build();) {
-
-            final String ns1 = "prop/usc/opportunistic1";
-            admin.namespaces().createNamespace(ns1);
-
-            final String topic1 = "persistent://" + ns1 + "/my-topic";
-            Producer<byte[]> producer = 
client.newProducer().topic(topic1).create();
-            for (int i = 0; i < 10; i++) {
-                String message = "my-message-" + i;
-                producer.send(message.getBytes());
-            }
-
-            // verify that all ledgers has the proper writequorumsize,
-            // equals to the number of available bookies (in this case 2)
-            ClientConfiguration clientConfiguration = new 
ClientConfiguration();
-            clientConfiguration.setZkServers("localhost:" + 
this.bkEnsemble.getZookeeperPort());
-
-            try (BookKeeper bkAdmin = 
BookKeeper.newBuilder(clientConfiguration).build()) {
-                try (ListLedgersResult list = 
bkAdmin.newListLedgersOp().execute().get();) {
-                    int count = 0;
-                    for (long ledgerId : list.toIterable()) {
-                        LedgerMetadata ledgerMetadata = 
bkAdmin.getLedgerMetadata(ledgerId).get();
-                        assertEquals(2, ledgerMetadata.getEnsembleSize());
-                        assertEquals(2, ledgerMetadata.getWriteQuorumSize());
-                        assertEquals(2, ledgerMetadata.getAckQuorumSize());
-                        count++;
-                    }
-                    assertTrue(count > 0);
-                }
-            }
-        }
-    }
-
-}

Reply via email to