Return-Path: X-Original-To: archive-asf-public-internal@cust-asf2.ponee.io Delivered-To: archive-asf-public-internal@cust-asf2.ponee.io Received: from cust-asf.ponee.io (cust-asf.ponee.io [163.172.22.183]) by cust-asf2.ponee.io (Postfix) with ESMTP id 67065200D2B for ; Thu, 19 Oct 2017 02:34:06 +0200 (CEST) Received: by cust-asf.ponee.io (Postfix) id 657F4160BED; Thu, 19 Oct 2017 00:34:06 +0000 (UTC) Delivered-To: archive-asf-public@cust-asf.ponee.io Received: from mail.apache.org (hermes.apache.org [140.211.11.3]) by cust-asf.ponee.io (Postfix) with SMTP id 3524F160BEA for ; Thu, 19 Oct 2017 02:34:05 +0200 (CEST) Received: (qmail 33505 invoked by uid 500); 19 Oct 2017 00:34:04 -0000 Mailing-List: contact commits-help@aurora.apache.org; run by ezmlm Precedence: bulk List-Help: List-Unsubscribe: List-Post: List-Id: Reply-To: dev@aurora.apache.org Delivered-To: mailing list commits@aurora.apache.org Received: (qmail 33496 invoked by uid 99); 19 Oct 2017 00:34:04 -0000 Received: from git1-us-west.apache.org (HELO git1-us-west.apache.org) (140.211.11.23) by apache.org (qpsmtpd/0.29) with ESMTP; Thu, 19 Oct 2017 00:34:04 +0000 Received: by git1-us-west.apache.org (ASF Mail Server at git1-us-west.apache.org, from userid 33) id 4A0F8DFA3C; Thu, 19 Oct 2017 00:34:04 +0000 (UTC) Content-Type: text/plain; charset="us-ascii" MIME-Version: 1.0 Content-Transfer-Encoding: 7bit From: wfarner@apache.org To: commits@aurora.apache.org Date: Thu, 19 Oct 2017 00:34:04 -0000 Message-Id: X-Mailer: ASF-Git Admin Mailer Subject: [1/3] aurora git commit: Remove legacy commons ZK code archived-at: Thu, 19 Oct 2017 00:34:06 -0000 Repository: aurora Updated Branches: refs/heads/master c6388774b -> 15cb049f3 http://git-wip-us.apache.org/repos/asf/aurora/blob/15cb049f/src/test/java/org/apache/aurora/scheduler/config/CommandLineTest.java ---------------------------------------------------------------------- diff --git a/src/test/java/org/apache/aurora/scheduler/config/CommandLineTest.java b/src/test/java/org/apache/aurora/scheduler/config/CommandLineTest.java index 1e2e01d..8f4f63c 100644 --- a/src/test/java/org/apache/aurora/scheduler/config/CommandLineTest.java +++ b/src/test/java/org/apache/aurora/scheduler/config/CommandLineTest.java @@ -159,7 +159,6 @@ public class CommandLineTest { expected.scheduling.schedulingMaxBatchSize = 42; expected.scheduling.maxTasksPerScheduleAttempt = 42; expected.async.asyncWorkerThreads = 42; - expected.zk.useCurator = false; expected.zk.inProcess = true; expected.zk.zkEndpoints = ImmutableList.of(InetSocketAddress.createUnresolved("testing", 42)); expected.zk.chrootPath = "testing"; @@ -311,7 +310,6 @@ public class CommandLineTest { "-scheduling_max_batch_size=42", "-max_tasks_per_schedule_attempt=42", "-async_worker_threads=42", - "-zk_use_curator=false", "-zk_in_proc=true", "-zk_endpoints=testing:42", "-zk_chroot_path=testing", http://git-wip-us.apache.org/repos/asf/aurora/blob/15cb049f/src/test/java/org/apache/aurora/scheduler/discovery/AbstractDiscoveryModuleTest.java ---------------------------------------------------------------------- diff --git a/src/test/java/org/apache/aurora/scheduler/discovery/AbstractDiscoveryModuleTest.java b/src/test/java/org/apache/aurora/scheduler/discovery/AbstractDiscoveryModuleTest.java deleted file mode 100644 index cec54e5..0000000 --- a/src/test/java/org/apache/aurora/scheduler/discovery/AbstractDiscoveryModuleTest.java +++ /dev/null @@ -1,82 +0,0 @@ -/** - * Licensed 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.aurora.scheduler.discovery; - -import java.net.InetSocketAddress; - -import com.google.common.base.Optional; -import com.google.common.collect.ImmutableList; -import com.google.inject.AbstractModule; -import com.google.inject.Binder; -import com.google.inject.Guice; -import com.google.inject.Injector; -import com.google.inject.Module; - -import org.apache.aurora.common.quantity.Amount; -import org.apache.aurora.common.quantity.Time; -import org.apache.aurora.common.stats.StatsProvider; -import org.apache.aurora.common.testing.TearDownTestCase; -import org.apache.aurora.common.zookeeper.Credentials; -import org.apache.aurora.common.zookeeper.SingletonService; -import org.apache.aurora.common.zookeeper.ZooKeeperUtils; -import org.apache.aurora.scheduler.app.ServiceGroupMonitor; -import org.apache.aurora.scheduler.testing.FakeStatsProvider; -import org.junit.Test; - -import static org.junit.Assert.assertNotNull; - -abstract class AbstractDiscoveryModuleTest extends TearDownTestCase { - - @Test - public void testBindingContract() { - ZooKeeperConfig zooKeeperConfig = - new ZooKeeperConfig( - isCurator(), - ImmutableList.of(InetSocketAddress.createUnresolved("localhost", 42)), - Optional.of("/chroot"), - false, // inProcess - Amount.of(1, Time.DAYS), - Amount.of(1, Time.DAYS), - Optional.of(Credentials.digestCredentials("test", "user"))); - - Injector injector = - Guice.createInjector( - new AbstractModule() { - @Override - protected void configure() { - bind(ServiceDiscoveryBindings.ZOO_KEEPER_CLUSTER_KEY) - .toInstance( - ImmutableList.of(InetSocketAddress.createUnresolved("localhost", 42))); - bind(ServiceDiscoveryBindings.ZOO_KEEPER_ACL_KEY) - .toInstance(ZooKeeperUtils.OPEN_ACL_UNSAFE); - - bind(StatsProvider.class).toInstance(new FakeStatsProvider()); - - bindExtraRequirements(binder()); - } - }, - createModule("/discovery/path", zooKeeperConfig)); - - assertNotNull(injector.getBinding(SingletonService.class).getProvider().get()); - assertNotNull(injector.getBinding(ServiceGroupMonitor.class).getProvider().get()); - } - - void bindExtraRequirements(Binder binder) { - // Noop. - } - - abstract Module createModule(String discoveryPath, ZooKeeperConfig zooKeeperConfig); - - abstract boolean isCurator(); -} http://git-wip-us.apache.org/repos/asf/aurora/blob/15cb049f/src/test/java/org/apache/aurora/scheduler/discovery/BaseCuratorDiscoveryTest.java ---------------------------------------------------------------------- diff --git a/src/test/java/org/apache/aurora/scheduler/discovery/BaseCuratorDiscoveryTest.java b/src/test/java/org/apache/aurora/scheduler/discovery/BaseCuratorDiscoveryTest.java index 226b068..02c8183 100644 --- a/src/test/java/org/apache/aurora/scheduler/discovery/BaseCuratorDiscoveryTest.java +++ b/src/test/java/org/apache/aurora/scheduler/discovery/BaseCuratorDiscoveryTest.java @@ -25,7 +25,7 @@ import org.apache.aurora.common.io.Codec; import org.apache.aurora.common.thrift.Endpoint; import org.apache.aurora.common.thrift.ServiceInstance; import org.apache.aurora.common.thrift.Status; -import org.apache.aurora.common.zookeeper.ServerSet; +import org.apache.aurora.common.zookeeper.Encoding; import org.apache.aurora.common.zookeeper.testing.BaseZooKeeperTest; import org.apache.aurora.scheduler.app.ServiceGroupMonitor; import org.apache.curator.framework.CuratorFramework; @@ -38,7 +38,7 @@ class BaseCuratorDiscoveryTest extends BaseZooKeeperTest { static final String GROUP_PATH = "/group/root"; static final String MEMBER_TOKEN = "member_"; - static final Codec CODEC = ServerSet.JSON_CODEC; + static final Codec CODEC = Encoding.JSON_CODEC; static final int PRIMARY_PORT = 42; private CuratorFramework client; @@ -55,7 +55,7 @@ class BaseCuratorDiscoveryTest extends BaseZooKeeperTest { groupCache.getListenable().addListener((c, event) -> groupEvents.put(event)); Predicate memberSelector = name -> name.contains(MEMBER_TOKEN); - groupMonitor = new CuratorServiceGroupMonitor(groupCache, memberSelector, ServerSet.JSON_CODEC); + groupMonitor = new CuratorServiceGroupMonitor(groupCache, memberSelector, Encoding.JSON_CODEC); } final CuratorFramework startNewClient() { http://git-wip-us.apache.org/repos/asf/aurora/blob/15cb049f/src/test/java/org/apache/aurora/scheduler/discovery/CommonsDiscoveryModuleTest.java ---------------------------------------------------------------------- diff --git a/src/test/java/org/apache/aurora/scheduler/discovery/CommonsDiscoveryModuleTest.java b/src/test/java/org/apache/aurora/scheduler/discovery/CommonsDiscoveryModuleTest.java deleted file mode 100644 index 7a4c4dd..0000000 --- a/src/test/java/org/apache/aurora/scheduler/discovery/CommonsDiscoveryModuleTest.java +++ /dev/null @@ -1,29 +0,0 @@ -/** - * Licensed 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.aurora.scheduler.discovery; - -import com.google.inject.Module; - -public class CommonsDiscoveryModuleTest extends AbstractDiscoveryModuleTest { - - @Override - Module createModule(String discoveryPath, ZooKeeperConfig zooKeeperConfig) { - return new CommonsServiceDiscoveryModule(discoveryPath, zooKeeperConfig); - } - - @Override - boolean isCurator() { - return false; - } -} http://git-wip-us.apache.org/repos/asf/aurora/blob/15cb049f/src/test/java/org/apache/aurora/scheduler/discovery/CommonsServiceGroupMonitorTest.java ---------------------------------------------------------------------- diff --git a/src/test/java/org/apache/aurora/scheduler/discovery/CommonsServiceGroupMonitorTest.java b/src/test/java/org/apache/aurora/scheduler/discovery/CommonsServiceGroupMonitorTest.java deleted file mode 100644 index 42a2224..0000000 --- a/src/test/java/org/apache/aurora/scheduler/discovery/CommonsServiceGroupMonitorTest.java +++ /dev/null @@ -1,137 +0,0 @@ -/** - * Licensed 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.aurora.scheduler.discovery; - -import com.google.common.collect.ImmutableMap; -import com.google.common.collect.ImmutableSet; - -import org.apache.aurora.common.base.Command; -import org.apache.aurora.common.net.pool.DynamicHostSet; -import org.apache.aurora.common.net.pool.DynamicHostSet.HostChangeMonitor; -import org.apache.aurora.common.testing.easymock.EasyMockTest; -import org.apache.aurora.common.thrift.Endpoint; -import org.apache.aurora.common.thrift.ServiceInstance; -import org.apache.aurora.common.thrift.Status; -import org.apache.aurora.scheduler.app.ServiceGroupMonitor; -import org.easymock.Capture; -import org.junit.Before; -import org.junit.Test; - -import static org.easymock.EasyMock.capture; -import static org.easymock.EasyMock.expect; -import static org.easymock.EasyMock.expectLastCall; -import static org.junit.Assert.assertEquals; -import static org.junit.Assert.fail; - -public class CommonsServiceGroupMonitorTest extends EasyMockTest { - - private DynamicHostSet serverSet; - private Capture> hostChangeMonitorCapture; - private Command stopCommand; - - @Before - public void setUp() throws Exception { - serverSet = createMock(new Clazz>() { }); - hostChangeMonitorCapture = createCapture(); - stopCommand = createMock(Command.class); - } - - private void expectSuccessfulWatch() throws Exception { - expect(serverSet.watch(capture(hostChangeMonitorCapture))).andReturn(stopCommand); - } - - private void expectFailedWatch() throws Exception { - DynamicHostSet.MonitorException watchError = - new DynamicHostSet.MonitorException( - "Problem watching service group", - new RuntimeException()); - expect(serverSet.watch(capture(hostChangeMonitorCapture))).andThrow(watchError); - } - - @Test - public void testNominalLifecycle() throws Exception { - expectSuccessfulWatch(); - - stopCommand.execute(); - expectLastCall(); - - control.replay(); - - CommonsServiceGroupMonitor groupMonitor = new CommonsServiceGroupMonitor(serverSet); - groupMonitor.start(); - groupMonitor.close(); - } - - @Test - public void testExceptionalLifecycle() throws Exception { - expectFailedWatch(); - control.replay(); - - CommonsServiceGroupMonitor groupMonitor = new CommonsServiceGroupMonitor(serverSet); - try { - groupMonitor.start(); - fail(); - } catch (ServiceGroupMonitor.MonitorException e) { - // expected - } - - // Close on a non-started monitor should be allowed. - groupMonitor.close(); - } - - @Test - public void testNoHosts() throws Exception { - expectSuccessfulWatch(); - control.replay(); - - CommonsServiceGroupMonitor groupMonitor = new CommonsServiceGroupMonitor(serverSet); - assertEquals(ImmutableSet.of(), groupMonitor.get()); - - groupMonitor.start(); - assertEquals(ImmutableSet.of(), groupMonitor.get()); - - hostChangeMonitorCapture.getValue().onChange(ImmutableSet.of()); - assertEquals(ImmutableSet.of(), groupMonitor.get()); - } - - @Test - public void testHostUpdates() throws Exception { - expectSuccessfulWatch(); - control.replay(); - - CommonsServiceGroupMonitor groupMonitor = new CommonsServiceGroupMonitor(serverSet); - groupMonitor.start(); - - ImmutableSet twoHosts = - ImmutableSet.of(serviceInstance("one"), serviceInstance("two")); - hostChangeMonitorCapture.getValue().onChange(twoHosts); - assertEquals(twoHosts, groupMonitor.get()); - - ImmutableSet oneHost = ImmutableSet.of(serviceInstance("one")); - hostChangeMonitorCapture.getValue().onChange(oneHost); - assertEquals(oneHost, groupMonitor.get()); - - ImmutableSet anotherHost = ImmutableSet.of(serviceInstance("three")); - hostChangeMonitorCapture.getValue().onChange(anotherHost); - assertEquals(anotherHost, groupMonitor.get()); - - ImmutableSet noHosts = ImmutableSet.of(); - hostChangeMonitorCapture.getValue().onChange(noHosts); - assertEquals(noHosts, groupMonitor.get()); - } - - private ServiceInstance serviceInstance(String hostName) { - return new ServiceInstance(new Endpoint(hostName, 42), ImmutableMap.of(), Status.ALIVE); - } -} http://git-wip-us.apache.org/repos/asf/aurora/blob/15cb049f/src/test/java/org/apache/aurora/scheduler/discovery/CuratorDiscoveryModuleTest.java ---------------------------------------------------------------------- diff --git a/src/test/java/org/apache/aurora/scheduler/discovery/CuratorDiscoveryModuleTest.java b/src/test/java/org/apache/aurora/scheduler/discovery/CuratorDiscoveryModuleTest.java index f1a02e4..f847882 100644 --- a/src/test/java/org/apache/aurora/scheduler/discovery/CuratorDiscoveryModuleTest.java +++ b/src/test/java/org/apache/aurora/scheduler/discovery/CuratorDiscoveryModuleTest.java @@ -13,36 +13,67 @@ */ package org.apache.aurora.scheduler.discovery; +import java.net.InetSocketAddress; + +import com.google.common.base.Optional; import com.google.common.collect.ImmutableList; -import com.google.inject.Binder; -import com.google.inject.Module; +import com.google.inject.AbstractModule; +import com.google.inject.Guice; +import com.google.inject.Injector; import org.apache.aurora.common.application.ShutdownRegistry; import org.apache.aurora.common.application.ShutdownRegistry.ShutdownRegistryImpl; +import org.apache.aurora.common.quantity.Amount; +import org.apache.aurora.common.quantity.Time; +import org.apache.aurora.common.stats.StatsProvider; +import org.apache.aurora.common.testing.TearDownTestCase; +import org.apache.aurora.common.zookeeper.Credentials; +import org.apache.aurora.common.zookeeper.SingletonService; import org.apache.aurora.common.zookeeper.ZooKeeperUtils; +import org.apache.aurora.scheduler.app.ServiceGroupMonitor; +import org.apache.aurora.scheduler.testing.FakeStatsProvider; import org.apache.curator.framework.api.ACLProvider; import org.apache.zookeeper.data.ACL; import org.junit.Test; import static org.junit.Assert.assertEquals; +import static org.junit.Assert.assertNotNull; -public class CuratorDiscoveryModuleTest extends AbstractDiscoveryModuleTest { +public class CuratorDiscoveryModuleTest extends TearDownTestCase { - @Override - void bindExtraRequirements(Binder binder) { - ShutdownRegistryImpl shutdownRegistry = new ShutdownRegistryImpl(); - binder.bind(ShutdownRegistry.class).toInstance(shutdownRegistry); - addTearDown(shutdownRegistry::execute); - } + @Test + public void testBindingContract() { + ZooKeeperConfig zooKeeperConfig = + new ZooKeeperConfig( + ImmutableList.of(InetSocketAddress.createUnresolved("localhost", 42)), + Optional.of("/chroot"), + false, // inProcess + Amount.of(1, Time.DAYS), + Amount.of(1, Time.DAYS), + Optional.of(Credentials.digestCredentials("test", "user"))); - @Override - Module createModule(String discoveryPath, ZooKeeperConfig zooKeeperConfig) { - return new CuratorServiceDiscoveryModule(discoveryPath, zooKeeperConfig); - } + Injector injector = + Guice.createInjector( + new AbstractModule() { + @Override + protected void configure() { + bind(ServiceDiscoveryBindings.ZOO_KEEPER_CLUSTER_KEY) + .toInstance( + ImmutableList.of(InetSocketAddress.createUnresolved("localhost", 42))); + bind(ServiceDiscoveryBindings.ZOO_KEEPER_ACL_KEY) + .toInstance(ZooKeeperUtils.OPEN_ACL_UNSAFE); + + bind(StatsProvider.class).toInstance(new FakeStatsProvider()); + + ShutdownRegistryImpl shutdownRegistry = new ShutdownRegistryImpl(); + bind(ShutdownRegistry.class).toInstance(shutdownRegistry); + addTearDown(shutdownRegistry::execute); + } + }, + new CuratorServiceDiscoveryModule("/discovery/path", zooKeeperConfig)); - @Override - boolean isCurator() { - return false; + assertNotNull(injector.getBinding(SingletonService.class).getProvider().get()); + assertNotNull(injector.getBinding(ServiceGroupMonitor.class).getProvider().get()); } @Test http://git-wip-us.apache.org/repos/asf/aurora/blob/15cb049f/src/test/java/org/apache/aurora/scheduler/discovery/ZooKeeperConfigTest.java ---------------------------------------------------------------------- diff --git a/src/test/java/org/apache/aurora/scheduler/discovery/ZooKeeperConfigTest.java b/src/test/java/org/apache/aurora/scheduler/discovery/ZooKeeperConfigTest.java index baee123..f1ea9e6 100644 --- a/src/test/java/org/apache/aurora/scheduler/discovery/ZooKeeperConfigTest.java +++ b/src/test/java/org/apache/aurora/scheduler/discovery/ZooKeeperConfigTest.java @@ -37,7 +37,6 @@ public class ZooKeeperConfigTest { @Test(expected = IllegalArgumentException.class) public void testEmptyServers() { new ZooKeeperConfig( - false, ImmutableList.of(), Optional.absent(), false, @@ -50,7 +49,6 @@ public class ZooKeeperConfigTest { public void testWithCredentials() { ZooKeeperConfig config = new ZooKeeperConfig( - false, SERVERS, Optional.absent(), false, @@ -72,9 +70,8 @@ public class ZooKeeperConfigTest { @Test public void testCreateFactory() { - ZooKeeperConfig config = ZooKeeperConfig.create(true, SERVERS); + ZooKeeperConfig config = ZooKeeperConfig.create(SERVERS); - assertTrue(config.isUseCurator()); assertEquals(SERVERS, ImmutableList.copyOf(config.getServers())); assertFalse(config.getChrootPath().isPresent()); assertFalse(config.isInProcess()); http://git-wip-us.apache.org/repos/asf/aurora/blob/15cb049f/src/test/java/org/apache/aurora/scheduler/http/LeaderHealthTest.java ---------------------------------------------------------------------- diff --git a/src/test/java/org/apache/aurora/scheduler/http/LeaderHealthTest.java b/src/test/java/org/apache/aurora/scheduler/http/LeaderHealthTest.java index d21a38e..d3e4580 100644 --- a/src/test/java/org/apache/aurora/scheduler/http/LeaderHealthTest.java +++ b/src/test/java/org/apache/aurora/scheduler/http/LeaderHealthTest.java @@ -16,7 +16,6 @@ package org.apache.aurora.scheduler.http; import javax.servlet.http.HttpServletResponse; import javax.ws.rs.core.Response; -import org.apache.aurora.common.net.pool.DynamicHostSet.MonitorException; import org.apache.aurora.common.testing.easymock.EasyMockTest; import org.apache.aurora.scheduler.http.LeaderRedirect.LeaderStatus; import org.junit.Before; @@ -31,7 +30,7 @@ public class LeaderHealthTest extends EasyMockTest { private LeaderHealth leaderHealth; @Before - public void setUp() throws MonitorException { + public void setUp() { leaderRedirect = createMock(LeaderRedirect.class); leaderHealth = new LeaderHealth(leaderRedirect); }