Skip to content

Commit

Permalink
Add integration tests for package management service
Browse files Browse the repository at this point in the history
---

Master Issue: #8676

*Motivation*

Add integration tests for testing the whole package management
service.

*Modifications*

- Add integration tests
- Fix some issue when startup the package management service
  • Loading branch information
zymap committed Dec 9, 2020
1 parent e6549dc commit 1d820e3
Show file tree
Hide file tree
Showing 9 changed files with 158 additions and 15 deletions.
Original file line number Diff line number Diff line change
Expand Up @@ -1350,7 +1350,7 @@ private void startPackagesManagementService() throws IOException {
.newProvider(config.getPackagesManagementStorageProvider());
DefaultPackagesStorageConfiguration storageConfiguration = new DefaultPackagesStorageConfiguration();
storageConfiguration.setProperty(config.getProperties());
PackagesStorage storage = storageProvider.getStorage(new DefaultPackagesStorageConfiguration());
PackagesStorage storage = storageProvider.getStorage(storageConfiguration);
storage.initialize();
packagesManagement.initialize(storage);
}
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -36,7 +36,7 @@ class CmdPackages extends CmdBase {

private final Packages packages;

CmdPackages(PulsarAdmin admin) {
public CmdPackages(PulsarAdmin admin) {
super("packages", admin);
this.packages = admin.packages();

Expand Down Expand Up @@ -133,7 +133,7 @@ private class ListPackageVersionsCmd extends CliCommand {

@Override
void run() throws Exception {
print(packages.listPackageVersions(packageName).toString());
print(packages.listPackageVersions(packageName));
}
}

Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -142,6 +142,8 @@ public class PulsarAdminTool {
// TODO eventually remove this
commandMap.put("source", CmdSources.class);
commandMap.put("sink", CmdSinks.class);

commandMap.put("packages", CmdPackages.class);
}

private void setupCommands(Function<PulsarAdminBuilder, ? extends PulsarAdmin> adminFactory) {
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -35,15 +35,15 @@ public class BookKeeperPackagesStorageConfiguration implements PackagesStorageCo
}

int getNumReplicas() {
return Integer.parseInt(getProperty("numReplicas"));
return Integer.parseInt(getProperty("packagesReplicas"));
}

String getZkServers() {
return getProperty("zkServers");
return getProperty("zookeeperServers");
}

String getLedgersRootPath() {
return getProperty("ledgerRootPath");
return getProperty("packagesManagementLedgerRootPath");
}

String getBookkeeperClientAuthenticationPlugin() {
Expand Down
16 changes: 8 additions & 8 deletions tests/docker-images/latest-version-image/Dockerfile
Original file line number Diff line number Diff line change
Expand Up @@ -17,7 +17,7 @@
# under the License.
#

FROM apachepulsar/pulsar-all:latest
FROM packages-test:latest

RUN rm -rf /var/lib/apt/lists/* && apt update

Expand Down Expand Up @@ -68,10 +68,10 @@ COPY python-examples/producer_schema.py /pulsar/examples/python-examples/
COPY python-examples/consumer_schema.py /pulsar/examples/python-examples/
COPY python-examples/exception_function.py /pulsar/examples/python-examples/

COPY target/pulsar-function-go/ /go/src/github.com/apache/pulsar/pulsar-function-go
RUN cd /go/src/github.com/apache/pulsar/pulsar-function-go && go install ./...
RUN cd /go/src/github.com/apache/pulsar/pulsar-function-go/pf && go install
RUN cd /go/src/github.com/apache/pulsar/pulsar-function-go/examples && go install ./...

COPY target/java-test-functions.jar /pulsar/examples/
RUN cp -a /go/bin/. /pulsar/examples/go-examples/
#COPY target/pulsar-function-go/ /go/src/github.com/apache/pulsar/pulsar-function-go
#RUN cd /go/src/github.com/apache/pulsar/pulsar-function-go && go install ./...
#RUN cd /go/src/github.com/apache/pulsar/pulsar-function-go/pf && go install
#RUN cd /go/src/github.com/apache/pulsar/pulsar-function-go/examples && go install ./...
#
#COPY target/java-test-functions.jar /pulsar/examples/
#RUN cp -a /go/bin/. /pulsar/examples/go-examples/
Original file line number Diff line number Diff line change
@@ -0,0 +1,133 @@
/**
* 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.tests.integration.cli;

import org.apache.pulsar.tests.integration.docker.ContainerExecResult;
import org.apache.pulsar.tests.integration.topologies.PulsarCluster;
import org.apache.pulsar.tests.integration.topologies.PulsarClusterSpec;
import org.testcontainers.shaded.org.apache.commons.lang.RandomStringUtils;
import static org.testng.Assert.assertEquals;
import static org.testng.Assert.assertFalse;
import static org.testng.Assert.assertNotEquals;
import static org.testng.Assert.assertTrue;

import org.testng.annotations.AfterClass;
import org.testng.annotations.BeforeClass;
import org.testng.annotations.Ignore;
import org.testng.annotations.Test;

import java.util.HashMap;
import java.util.Map;

public class PackagesCliTest {

private final static String clusterNamePrefix = "packages-service";
private PulsarCluster pulsarCluster;

@BeforeClass
public void setup() throws Exception {
PulsarClusterSpec spec = PulsarClusterSpec.builder()
.clusterName(String.format("%s-%s", clusterNamePrefix, RandomStringUtils.randomAlphabetic(6)))
.brokerEnvs(getPackagesManagementServiceEnvs())
.build();
pulsarCluster = PulsarCluster.forSpec(spec);
pulsarCluster.start();
}

@AfterClass
public void teardown() {
if (pulsarCluster != null) {
pulsarCluster.stop();
pulsarCluster = null;
}
}

private Map<String, String> getPackagesManagementServiceEnvs() {
Map<String, String> envs = new HashMap<>();
envs.put("enablePackagesManagement", "true");
return envs;
}

@Test
public void testPackagesOperationsWithoutUploadingPackages() throws Exception {
ContainerExecResult result = runPackagesCommand("list", "--type", "functions", "public/default");
assertEquals(result.getExitCode(), 0);
assertTrue(result.getStdout().isEmpty());

result = runPackagesCommand("list-versions", "function://public/default/test");
assertEquals(result.getExitCode(), 0);
assertTrue(result.getStdout().isEmpty());

result = runPackagesCommand("download", "function://public/default/test@v1");
assertNotEquals(result.getExitCode(), 0);
assertFalse(result.getStderr().isEmpty());
assertTrue(result.getStderr().contains("does not exist"));
}

// TODO: the upload command has some problem when uploading packages, enable this tests when issue
// https://github.com/apache/pulsar/issues/8874 is fixed.
@Ignore
@Test
public void testPackagesOperationsWithUploadingPackages() throws Exception {
String testPackageName = "function://public/default/test@v1";
ContainerExecResult result = runPackagesCommand("upload", "--description", "a test package",
"--path", PulsarCluster.ADMIN_SCRIPT, testPackageName);
assertEquals(result.getExitCode(), 0);

result = runPackagesCommand("get-metadata", testPackageName);
assertEquals(result.getExitCode(), 0);
assertFalse(result.getStdout().isEmpty());
assertTrue(result.getStdout().contains("a test package"));

result = runPackagesCommand("list", "--type", "function", "public/default");
assertEquals(result.getExitCode(), 0);
assertFalse(result.getStdout().isEmpty());
assertTrue(result.getStdout().contains(testPackageName));

result = runPackagesCommand("list-versions", "--type", "function://public/default/test");
assertEquals(result.getExitCode(), 0);
assertFalse(result.getStdout().isEmpty());
assertTrue(result.getStdout().contains("v1"));

String contact = "test@apache.org";
result = runPackagesCommand("update-metadata", "--contact", contact, "-PpropertyA=A", testPackageName);
assertEquals(result.getExitCode(), 0);

result = runPackagesCommand("get-metadata", testPackageName);
assertEquals(result.getExitCode(), 0);
assertFalse(result.getStdout().isEmpty());
assertTrue(result.getStdout().contains("a test package"));
assertTrue(result.getStdout().contains(contact));
assertTrue(result.getStdout().contains("propertyA"));

result = runPackagesCommand("delete", testPackageName);
assertEquals(result.getExitCode(), 0);

result = runPackagesCommand("list", "--type", "functions", "public/default");
assertEquals(result.getExitCode(), 0);
assertTrue(result.getStdout().isEmpty());
}

private ContainerExecResult runPackagesCommand(String... commands) throws Exception {
String[] cmds = new String[commands.length + 1];
cmds[0] = "packages";
System.arraycopy(commands, 0, cmds, 1, commands.length);
return pulsarCluster.runAdminCommandOnAnyBroker(cmds);
}
}
Original file line number Diff line number Diff line change
Expand Up @@ -40,7 +40,7 @@ public abstract class PulsarContainer<SelfT extends PulsarContainer<SelfT>> exte
public static final int BROKER_PORT = 6650;
public static final int BROKER_HTTP_PORT = 8080;

public static final String DEFAULT_IMAGE_NAME = "apachepulsar/pulsar-test-latest-version:latest";
public static final String DEFAULT_IMAGE_NAME = "pulsar/packages-tests:latest";
public static final String PULSAR_2_5_IMAGE_NAME = "apachepulsar/pulsar:2.5.0";
public static final String PULSAR_2_4_IMAGE_NAME = "apachepulsar/pulsar:2.4.0";
public static final String PULSAR_2_3_IMAGE_NAME = "apachepulsar/pulsar:2.3.0";
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -171,6 +171,9 @@ private PulsarCluster(PulsarClusterSpec spec) {
brokerContainer.withEnv("bookkeeperExplicitLacIntervalInMills", "10");
brokerContainer.withEnv("bookkeeperUseV2WireProtocol", "false");
}
if (spec.brokerEnvs != null) {
brokerContainer.withEnv(spec.brokerEnvs);
}
return brokerContainer;
}
));
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -137,4 +137,9 @@ public class PulsarClusterSpec {
* Specify envs for proxy.
*/
Map<String, String> proxyEnvs;

/**
* Specify envs for broker.
*/
Map<String, String> brokerEnvs;
}

0 comments on commit 1d820e3

Please sign in to comment.