From d72ddd17625faf1540d282b35ade4997204c5e22 Mon Sep 17 00:00:00 2001 From: Shijin Date: Tue, 21 Jan 2025 15:12:04 +0530 Subject: [PATCH] Update zookeeper version to fix CVE-2023-44981 --- pom.xml | 6 +- presto-accumulo/pom.xml | 10 +- .../presto/accumulo/AccumuloQueryRunner.java | 99 +++---------------- .../accumulo/MiniAccumuloConfigUtil.java | 52 ---------- .../presto/accumulo/TestAccumuloClient.java | 2 +- .../accumulo/TestingAccumuloServer.java | 89 +++++++++++++++++ 6 files changed, 116 insertions(+), 142 deletions(-) delete mode 100644 presto-accumulo/src/test/java/com/facebook/presto/accumulo/MiniAccumuloConfigUtil.java create mode 100644 presto-accumulo/src/test/java/com/facebook/presto/accumulo/TestingAccumuloServer.java diff --git a/pom.xml b/pom.xml index 4b0659f2054f7..1eeb632cfc2ec 100644 --- a/pom.xml +++ b/pom.xml @@ -1936,7 +1936,7 @@ org.apache.zookeeper zookeeper - 3.4.14 + 3.9.3 jline @@ -1950,6 +1950,10 @@ org.slf4j slf4j-log4j12 + + commons-io + commons-io + diff --git a/presto-accumulo/pom.xml b/presto-accumulo/pom.xml index 86a7e17f8062e..77a89fede4e66 100644 --- a/presto-accumulo/pom.xml +++ b/presto-accumulo/pom.xml @@ -236,8 +236,8 @@ - commons-io - commons-io + com.github.docker-java + docker-java-api @@ -341,6 +341,12 @@ testng test + + + org.testcontainers + testcontainers + test + diff --git a/presto-accumulo/src/test/java/com/facebook/presto/accumulo/AccumuloQueryRunner.java b/presto-accumulo/src/test/java/com/facebook/presto/accumulo/AccumuloQueryRunner.java index 808b0a0a07925..f2d068f7944e0 100644 --- a/presto-accumulo/src/test/java/com/facebook/presto/accumulo/AccumuloQueryRunner.java +++ b/presto-accumulo/src/test/java/com/facebook/presto/accumulo/AccumuloQueryRunner.java @@ -18,48 +18,29 @@ import com.facebook.presto.accumulo.conf.AccumuloConfig; import com.facebook.presto.accumulo.serializers.LexicoderRowSerializer; import com.facebook.presto.common.QualifiedObjectName; -import com.facebook.presto.spi.PrestoException; import com.facebook.presto.testing.QueryRunner; import com.facebook.presto.tests.DistributedQueryRunner; import com.facebook.presto.tpch.TpchPlugin; import com.google.common.collect.ImmutableMap; import com.google.common.collect.ImmutableSortedSet; import io.airlift.tpch.TpchTable; -import org.apache.accumulo.core.client.AccumuloException; -import org.apache.accumulo.core.client.AccumuloSecurityException; -import org.apache.accumulo.core.client.Connector; -import org.apache.accumulo.core.client.Instance; -import org.apache.accumulo.core.client.ZooKeeperInstance; -import org.apache.accumulo.core.client.security.tokens.PasswordToken; -import org.apache.accumulo.minicluster.MiniAccumuloCluster; -import org.apache.commons.io.FileUtils; import org.apache.hadoop.io.Text; import org.intellij.lang.annotations.Language; -import java.io.File; -import java.io.IOException; -import java.nio.file.Files; import java.util.Map; -import static com.facebook.presto.accumulo.AccumuloErrorCode.MINI_ACCUMULO; -import static com.facebook.presto.accumulo.AccumuloErrorCode.UNEXPECTED_ACCUMULO_ERROR; -import static com.facebook.presto.accumulo.MiniAccumuloConfigUtil.setConfigClassPath; import static com.facebook.presto.common.type.BigintType.BIGINT; import static com.facebook.presto.testing.TestingSession.testSessionBuilder; import static com.facebook.presto.tpch.TpchMetadata.TINY_SCHEMA_NAME; import static io.airlift.units.Duration.nanosSince; import static java.lang.String.format; import static java.util.concurrent.TimeUnit.SECONDS; -import static org.apache.accumulo.minicluster.MemoryUnit.MEGABYTE; public final class AccumuloQueryRunner { private static final Logger LOG = Logger.get(AccumuloQueryRunner.class); - private static final String MAC_PASSWORD = "secret"; - private static final String MAC_USER = "root"; private static boolean tpchLoaded; - private static Connector connector = getAccumuloConnector(); private AccumuloQueryRunner() {} @@ -72,13 +53,14 @@ public static synchronized DistributedQueryRunner createAccumuloQueryRunner(Map< queryRunner.installPlugin(new TpchPlugin()); queryRunner.createCatalog("tpch", "tpch"); + TestingAccumuloServer server = TestingAccumuloServer.getInstance(); queryRunner.installPlugin(new AccumuloPlugin()); Map accumuloProperties = ImmutableMap.builder() - .put(AccumuloConfig.INSTANCE, connector.getInstance().getInstanceName()) - .put(AccumuloConfig.ZOOKEEPERS, connector.getInstance().getZooKeepers()) - .put(AccumuloConfig.USERNAME, MAC_USER) - .put(AccumuloConfig.PASSWORD, MAC_PASSWORD) + .put(AccumuloConfig.INSTANCE, server.getInstanceName()) + .put(AccumuloConfig.ZOOKEEPERS, server.getZooKeepers()) + .put(AccumuloConfig.USERNAME, server.getUser()) + .put(AccumuloConfig.PASSWORD, server.getPassword()) .put(AccumuloConfig.ZOOKEEPER_METADATA_ROOT, "/presto-accumulo-test") .build(); @@ -86,7 +68,7 @@ public static synchronized DistributedQueryRunner createAccumuloQueryRunner(Map< if (!tpchLoaded) { copyTpchTables(queryRunner, "tpch", TINY_SCHEMA_NAME, createSession(), TpchTable.getTables()); - connector.tableOperations().addSplits("tpch.orders", ImmutableSortedSet.of(new Text(new LexicoderRowSerializer().encode(BIGINT, 7500L)))); + server.getConnector().tableOperations().addSplits("tpch.orders", ImmutableSortedSet.of(new Text(new LexicoderRowSerializer().encode(BIGINT, 7500L)))); tpchLoaded = true; } @@ -156,68 +138,13 @@ public static Session createSession() return testSessionBuilder().setCatalog("accumulo").setSchema("tpch").build(); } - /** - * Gets the AccumuloConnector singleton, starting the MiniAccumuloCluster on initialization. - * This singleton instance is required so all test cases access the same MiniAccumuloCluster. - * - * @return Accumulo connector - */ - public static Connector getAccumuloConnector() - { - if (connector != null) { - return connector; - } - - try { - MiniAccumuloCluster accumulo = createMiniAccumuloCluster(); - Instance instance = new ZooKeeperInstance(accumulo.getInstanceName(), accumulo.getZooKeepers()); - connector = instance.getConnector(MAC_USER, new PasswordToken(MAC_PASSWORD)); - LOG.info("Connection to MAC instance %s at %s established, user %s password %s", accumulo.getInstanceName(), accumulo.getZooKeepers(), MAC_USER, MAC_PASSWORD); - return connector; - } - catch (AccumuloException | AccumuloSecurityException | InterruptedException | IOException e) { - throw new PrestoException(UNEXPECTED_ACCUMULO_ERROR, "Failed to get connector to Accumulo", e); - } - } - - /** - * Creates and starts an instance of MiniAccumuloCluster, returning the new instance. - * - * @return New MiniAccumuloCluster - */ - private static MiniAccumuloCluster createMiniAccumuloCluster() - throws IOException, InterruptedException + public static void main(String[] args) + throws Exception { - // Create MAC directory - File macDir = Files.createTempDirectory("mac-").toFile(); - LOG.info("MAC is enabled, starting MiniAccumuloCluster at %s", macDir); - - // Start MAC and connect to it - MiniAccumuloCluster accumulo = new MiniAccumuloCluster(macDir, MAC_PASSWORD); - accumulo.getConfig().setDefaultMemory(512, MEGABYTE); - setConfigClassPath(accumulo.getConfig()); - accumulo.start(); - - // Add shutdown hook to stop MAC and cleanup temporary files - Runtime.getRuntime().addShutdownHook(new Thread(() -> { - try { - LOG.info("Shutting down MAC"); - accumulo.stop(); - } - catch (IOException | InterruptedException e) { - Thread.currentThread().interrupt(); - throw new PrestoException(MINI_ACCUMULO, "Failed to shut down MAC instance", e); - } - - try { - LOG.info("Cleaning up MAC directory"); - FileUtils.forceDelete(macDir); - } - catch (IOException e) { - throw new PrestoException(MINI_ACCUMULO, "Failed to clean up MAC directory", e); - } - })); - - return accumulo; + DistributedQueryRunner queryRunner = createAccumuloQueryRunner(ImmutableMap.of("http-server.http.port", "8080")); + Thread.sleep(10); + Logger log = Logger.get(AccumuloQueryRunner.class); + log.info("======== SERVER STARTED ========"); + log.info("\n====\n%s\n====", queryRunner.getCoordinator().getBaseUrl()); } } diff --git a/presto-accumulo/src/test/java/com/facebook/presto/accumulo/MiniAccumuloConfigUtil.java b/presto-accumulo/src/test/java/com/facebook/presto/accumulo/MiniAccumuloConfigUtil.java deleted file mode 100644 index 6ba936d16d009..0000000000000 --- a/presto-accumulo/src/test/java/com/facebook/presto/accumulo/MiniAccumuloConfigUtil.java +++ /dev/null @@ -1,52 +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 com.facebook.presto.accumulo; - -import com.google.common.base.Splitter; -import org.apache.accumulo.minicluster.MiniAccumuloConfig; -import org.apache.accumulo.minicluster.impl.MiniAccumuloConfigImpl; - -import java.io.File; -import java.lang.reflect.Field; -import java.util.List; - -import static java.lang.management.ManagementFactory.getRuntimeMXBean; - -public final class MiniAccumuloConfigUtil -{ - private MiniAccumuloConfigUtil() {} - - /** - * MiniAccumuloClusterImpl will build the class path itself if not set, - * but the code fails on Java 9 due to assumptions about URLClassLoader. - */ - public static void setConfigClassPath(MiniAccumuloConfig config) - { - List items = Splitter.on(File.pathSeparatorChar) - .splitToList(getRuntimeMXBean().getClassPath()); - getConfigImpl(config).setClasspathItems(items.toArray(new String[0])); - } - - private static MiniAccumuloConfigImpl getConfigImpl(MiniAccumuloConfig config) - { - try { - Field field = MiniAccumuloConfig.class.getDeclaredField("impl"); - field.setAccessible(true); - return (MiniAccumuloConfigImpl) field.get(config); - } - catch (ReflectiveOperationException e) { - throw new AssertionError(e); - } - } -} diff --git a/presto-accumulo/src/test/java/com/facebook/presto/accumulo/TestAccumuloClient.java b/presto-accumulo/src/test/java/com/facebook/presto/accumulo/TestAccumuloClient.java index d693d0d33d5c2..d5b43f1defaaa 100644 --- a/presto-accumulo/src/test/java/com/facebook/presto/accumulo/TestAccumuloClient.java +++ b/presto-accumulo/src/test/java/com/facebook/presto/accumulo/TestAccumuloClient.java @@ -46,7 +46,7 @@ public TestAccumuloClient() .setUsername("root") .setPassword("secret"); - Connector connector = AccumuloQueryRunner.getAccumuloConnector(); + Connector connector = TestingAccumuloServer.getInstance().getConnector(); config.setZooKeepers(connector.getInstance().getZooKeepers()); zooKeeperMetadataManager = new ZooKeeperMetadataManager(config, createTestFunctionAndTypeManager()); client = new AccumuloClient(connector, config, zooKeeperMetadataManager, new AccumuloTableManager(connector), new IndexLookup(connector, new ColumnCardinalityCache(connector, config))); diff --git a/presto-accumulo/src/test/java/com/facebook/presto/accumulo/TestingAccumuloServer.java b/presto-accumulo/src/test/java/com/facebook/presto/accumulo/TestingAccumuloServer.java new file mode 100644 index 0000000000000..29d762b6467ba --- /dev/null +++ b/presto-accumulo/src/test/java/com/facebook/presto/accumulo/TestingAccumuloServer.java @@ -0,0 +1,89 @@ +package com.facebook.presto.accumulo; + +/* + * 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. + */ +import org.apache.accumulo.core.client.AccumuloException; +import org.apache.accumulo.core.client.AccumuloSecurityException; +import org.apache.accumulo.core.client.Connector; +import org.apache.accumulo.core.client.ZooKeeperInstance; +import org.apache.accumulo.core.client.security.tokens.PasswordToken; +import org.testcontainers.containers.FixedHostPortGenericContainer; +import org.testcontainers.containers.wait.strategy.Wait; + +import java.io.IOException; +import java.time.Duration; + +import static java.lang.String.format; +public class TestingAccumuloServer +{ + private static final int ACCUMULO_MASTER_PORT = 9999; + private static final int ACCUMULO_TSERVER_PORT = 9997; + private static final int ZOOKEEPER_PORT = 2181; + private static final TestingAccumuloServer instance = new TestingAccumuloServer(); + private final FixedHostPortGenericContainer accumuloContainer; + public static TestingAccumuloServer getInstance() + { + return instance; + } + private TestingAccumuloServer() + { + accumuloContainer = new FixedHostPortGenericContainer<>("ghcr.io/trinodb/testing/accumulo:74"); + accumuloContainer.withFixedExposedPort(ACCUMULO_MASTER_PORT, ACCUMULO_MASTER_PORT); + accumuloContainer.withFixedExposedPort(ACCUMULO_TSERVER_PORT, ACCUMULO_TSERVER_PORT); + accumuloContainer.withExposedPorts(ZOOKEEPER_PORT); + accumuloContainer.withCreateContainerCmdModifier(cmd -> cmd + .withHostName("localhost") + .withEnv("ADDRESS=0.0.0.0") + .withEntrypoint("supervisord", "-c", "/etc/supervisord.conf")); + accumuloContainer.waitingFor(Wait.forHealthcheck().withStartupTimeout(Duration.ofMinutes(10))); + // No need for an explicit stop since this server is a singleton + // and the container will be stopped by TestContainers on shutdown + // TODO Change this class to not be a singleton + // https://github.com/prestosql/presto/issues/5842 + accumuloContainer.start(); + try { + accumuloContainer.execInContainer("wget", "-P", "/usr/local/lib/accumulo/lib/ext/", "https://repo1.maven.org/maven2/com/facebook/presto/presto-accumulo/0.289/presto-accumulo-0.289.jar"); + accumuloContainer.execInContainer("mv", "/usr/local/lib/accumulo/lib/ext/presto-accumulo-0.289.jar", "/usr/local/lib/accumulo/lib/ext/presto-accumulo-0.290-SNAPSHOT.jar"); + } + catch (IOException | InterruptedException e) { + throw new RuntimeException(e); + } + } + public String getInstanceName() + { + return "default"; + } + public String getZooKeepers() + { + return format("%s:%s", accumuloContainer.getHost(), accumuloContainer.getMappedPort(ZOOKEEPER_PORT)); + } + public String getUser() + { + return "root"; + } + public String getPassword() + { + return "secret"; + } + public Connector getConnector() + { + try { + ZooKeeperInstance instance = new ZooKeeperInstance(getInstanceName(), getZooKeepers()); + return instance.getConnector(getUser(), new PasswordToken(getPassword())); + } + catch (AccumuloException | AccumuloSecurityException e) { + throw new RuntimeException(e); + } + } +}