You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@impala.apache.org by jb...@apache.org on 2019/05/19 17:21:17 UTC

[impala] branch master updated (98a79c8 -> d72f333)

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

jbapple pushed a change to branch master
in repository https://gitbox.apache.org/repos/asf/impala.git.


    from 98a79c8  Add USE_CDP_HIVE=true case to build-all-flag-combinations.sh
     new 358e92f  IMPALA-8563: Update SSL ciphers used in BE tests
     new d72f333  IMPALA-8545: Test Ldap authentication

The 2 revisions listed above as "new" are entirely new to this
repository and will be described in separate emails.  The revisions
listed as "add" were already present in the repository and have only
been added to this reference.


Summary of changes:
 be/src/rpc/rpc-mgr-test.h                          |   4 +-
 be/src/rpc/thrift-server-test.cc                   |   4 +-
 be/src/util/webserver-test.cc                      |   2 +-
 bin/rat_exclude_files.txt                          |   1 +
 bin/run-all-tests.sh                               |  11 ++
 fe/pom.xml                                         |   7 ++
 .../impala/customcluster/CustomClusterRunner.java  |  44 ++++++++
 .../apache/impala/customcluster/LdapJdbcTest.java  | 105 ++++++++++++++++++
 .../java/org/apache/impala/service/JdbcTest.java   |  90 +--------------
 .../org/apache/impala/service/JdbcTestBase.java    | 122 +++++++++++++++++++++
 .../apache/impala/testutil/ImpalaJdbcClient.java   |  17 ++-
 fe/src/test/resources/users.ldif                   |  25 +++++
 12 files changed, 338 insertions(+), 94 deletions(-)
 create mode 100644 fe/src/test/java/org/apache/impala/customcluster/CustomClusterRunner.java
 create mode 100644 fe/src/test/java/org/apache/impala/customcluster/LdapJdbcTest.java
 create mode 100644 fe/src/test/java/org/apache/impala/service/JdbcTestBase.java
 create mode 100644 fe/src/test/resources/users.ldif


[impala] 02/02: IMPALA-8545: Test Ldap authentication

Posted by jb...@apache.org.
This is an automated email from the ASF dual-hosted git repository.

jbapple pushed a commit to branch master
in repository https://gitbox.apache.org/repos/asf/impala.git

commit d72f3330c1edc9086ba120e6d3469a75c0aea083
Author: Thomas Tauber-Marshall <tm...@cloudera.com>
AuthorDate: Tue May 14 12:21:03 2019 -0700

    IMPALA-8545: Test Ldap authentication
    
    Currently, Impala does not have any automated tests for LDAP auth
    functionality, due to the challenge of setting up an LDAP server
    for use by the minicluster.
    
    This patch adds LDAP tests by utilizing the ApacheDS project's
    unit testing functionality, which works with JUnit to setup up
    a local LDAP server for the duration of a test suite.
    
    This requires running an Impala cluster with custom arguments to
    set up LDAP auth. This patch introduces a concept of FE custom cluster
    tests which must be in the package org.apache.impala.customcluster.
    These tests are filtered out from the other FE tests in
    bin/run-all-tests.sh and run with the other custom cluster tests so
    that they don't affect other tests that expect Impala to have been
    started with particular flags.
    
    Testing:
    - Ran a full core run and confirmed that new tests run as expected.
    
    Change-Id: I92b5e60860c60209c1bd8afe5b3ea201fb7a7513
    Reviewed-on: http://gerrit.cloudera.org:8080/13337
    Tested-by: Impala Public Jenkins <im...@cloudera.com>
    Reviewed-by: Tim Armstrong <ta...@cloudera.com>
---
 bin/rat_exclude_files.txt                          |   1 +
 bin/run-all-tests.sh                               |  11 ++
 fe/pom.xml                                         |   7 ++
 .../impala/customcluster/CustomClusterRunner.java  |  44 ++++++++
 .../apache/impala/customcluster/LdapJdbcTest.java  | 105 ++++++++++++++++++
 .../java/org/apache/impala/service/JdbcTest.java   |  90 +--------------
 .../org/apache/impala/service/JdbcTestBase.java    | 122 +++++++++++++++++++++
 .../apache/impala/testutil/ImpalaJdbcClient.java   |  17 ++-
 fe/src/test/resources/users.ldif                   |  25 +++++
 9 files changed, 333 insertions(+), 89 deletions(-)

diff --git a/bin/rat_exclude_files.txt b/bin/rat_exclude_files.txt
index 4c255e8..e352a17 100644
--- a/bin/rat_exclude_files.txt
+++ b/bin/rat_exclude_files.txt
@@ -102,6 +102,7 @@ be/src/testutil/*.pem
 fe/src/test/resources/*.xml
 fe/src/test/resources/hbase-jaas-client.conf.template
 fe/src/test/resources/hbase-jaas-server.conf.template
+fe/src/test/resources/users.ldif
 testdata/AllTypesError/*.txt
 testdata/AllTypesErrorNoNulls/*.txt
 *.avsc
diff --git a/bin/run-all-tests.sh b/bin/run-all-tests.sh
index b74d03c..d124e03 100755
--- a/bin/run-all-tests.sh
+++ b/bin/run-all-tests.sh
@@ -195,6 +195,9 @@ do
     if [[ "$CODE_COVERAGE" == true ]]; then
       MVN_ARGS+="-DcodeCoverage"
     fi
+    # Don't run the FE custom cluster tests here since they restart Impala. We'll run them
+    # with the other custom cluster tests below.
+    MVN_ARGS+=" -Dtest=!org.apache.impala.customcluster.*Test "
     if ! "${IMPALA_HOME}/bin/mvn-quiet.sh" -fae test ${MVN_ARGS}; then
       TEST_RET_CODE=1
     fi
@@ -242,6 +245,14 @@ do
       TEST_RET_CODE=1
     fi
     export IMPALA_MAX_LOG_FILES="${IMPALA_MAX_LOG_FILES_SAVE}"
+
+    # Run the FE custom cluster tests.
+    pushd "${IMPALA_FE_DIR}"
+    MVN_ARGS=" -Dtest=org.apache.impala.customcluster.*Test "
+    if ! "${IMPALA_HOME}/bin/mvn-quiet.sh" -fae test ${MVN_ARGS}; then
+      TEST_RET_CODE=1
+    fi
+    popd
   fi
 
   # Run the process failure tests.
diff --git a/fe/pom.xml b/fe/pom.xml
index 62fc500..d6c9908 100644
--- a/fe/pom.xml
+++ b/fe/pom.xml
@@ -412,6 +412,13 @@ under the License.
       <version>2.23.4</version>
       <scope>test</scope>
     </dependency>
+
+    <dependency>
+      <groupId>org.apache.directory.server</groupId>
+      <artifactId>apacheds-test-framework</artifactId>
+      <version>2.0.0.AM25</version>
+      <scope>test</scope>
+    </dependency>
   </dependencies>
 
   <reporting>
diff --git a/fe/src/test/java/org/apache/impala/customcluster/CustomClusterRunner.java b/fe/src/test/java/org/apache/impala/customcluster/CustomClusterRunner.java
new file mode 100644
index 0000000..7081a9d
--- /dev/null
+++ b/fe/src/test/java/org/apache/impala/customcluster/CustomClusterRunner.java
@@ -0,0 +1,44 @@
+// 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.impala.customcluster;
+
+import java.io.IOException;
+
+/**
+ * Runs an Impala cluster with custom flags.
+ *
+ * In order to prevent this from affecting other tests, we filter on the package name to
+ * run FE custom cluster tests when we run the python custom cluster tests, so this should
+ * not be used outside of this package.
+ */
+class CustomClusterRunner {
+  public static int StartImpalaCluster() throws IOException, InterruptedException {
+    return StartImpalaCluster("");
+  }
+
+  /**
+   * Starts Impala and passes 'args' to the impalads, catalogd, and statestored.
+   */
+  public static int StartImpalaCluster(String args)
+      throws IOException, InterruptedException {
+    Process p = Runtime.getRuntime().exec(new String[] {"start-impala-cluster.py",
+        "--impalad_args", args, "--catalogd_args", args, "--state_store_args", args});
+    p.waitFor();
+    return p.exitValue();
+  }
+}
diff --git a/fe/src/test/java/org/apache/impala/customcluster/LdapJdbcTest.java b/fe/src/test/java/org/apache/impala/customcluster/LdapJdbcTest.java
new file mode 100644
index 0000000..8c77cc4
--- /dev/null
+++ b/fe/src/test/java/org/apache/impala/customcluster/LdapJdbcTest.java
@@ -0,0 +1,105 @@
+// 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.impala.customcluster;
+
+import static org.junit.Assert.assertEquals;
+import static org.junit.Assert.assertFalse;
+import static org.junit.Assert.assertTrue;
+import static org.junit.Assert.fail;
+
+import java.sql.Connection;
+import java.sql.ResultSet;
+import java.sql.SQLException;
+
+import org.apache.directory.server.core.annotations.CreateDS;
+import org.apache.directory.server.core.annotations.CreatePartition;
+import org.apache.directory.server.annotations.CreateLdapServer;
+import org.apache.directory.server.annotations.CreateTransport;
+import org.apache.directory.server.core.annotations.ApplyLdifFiles;
+import org.apache.directory.server.core.integ.CreateLdapServerRule;
+import org.apache.impala.testutil.ImpalaJdbcClient;
+import org.junit.AfterClass;
+import org.junit.BeforeClass;
+import org.junit.ClassRule;
+import org.junit.Test;
+
+import org.apache.impala.service.JdbcTestBase;
+
+/**
+ * Tests for connecting to Impala when LDAP authentication is in use.
+ */
+@CreateDS(name = "myDS",
+    partitions = { @CreatePartition(name = "test", suffix = "dc=myorg,dc=com") })
+@CreateLdapServer(
+    transports = { @CreateTransport(protocol = "LDAP", address = "localhost") })
+@ApplyLdifFiles({"users.ldif"})
+public class LdapJdbcTest extends JdbcTestBase {
+  @ClassRule
+  public static CreateLdapServerRule serverRule = new CreateLdapServerRule();
+
+  // These correspond to the values in fe/src/test/resources/users.ldif
+  private static final String testUser_ = "Test1Ldap";
+  private static final String testPassword_ = "12345";
+
+  @BeforeClass
+  public static void setUp() throws Exception {
+    String uri =
+        String.format("ldap://localhost:%s", serverRule.getLdapServer().getPort());
+    String dn = "cn=#UID,ou=Users,dc=myorg,dc=com";
+    String ldapArgs = String.format("--enable_ldap_auth --ldap_uri='%s' "
+        + "--ldap_bind_pattern='%s' --ldap_passwords_in_clear_ok", uri, dn);
+    int ret = CustomClusterRunner.StartImpalaCluster(ldapArgs);
+    assertEquals(ret, 0);
+
+    con_ =
+        createConnection(ImpalaJdbcClient.getLdapConnectionStr(testUser_, testPassword_));
+  }
+
+  @AfterClass
+  public static void cleanUp() throws Exception {
+    JdbcTestBase.cleanUp();
+    CustomClusterRunner.StartImpalaCluster();
+  }
+
+  @Test
+  public void testLoggedInUser() throws Exception {
+    ResultSet rs = con_.createStatement().executeQuery("select logged_in_user() user");
+    assertTrue(rs.next());
+    assertEquals(rs.getString("user"), testUser_);
+    assertFalse(rs.next());
+  }
+
+  @Test
+  public void testFailedConnection() throws Exception {
+    try {
+      Connection con = createConnection(
+          ImpalaJdbcClient.getLdapConnectionStr(testUser_, "invalid-password"));
+      fail("Connecting with an invalid password should throw an error.");
+    } catch (SQLException e) {
+      assertTrue(e.getMessage().contains("Could not open client transport"));
+    }
+
+    try {
+      Connection con = createConnection(
+          ImpalaJdbcClient.getLdapConnectionStr("invalid-user", testPassword_));
+      fail("Connecting with an invalid user name should throw an error.");
+    } catch (SQLException e) {
+      assertTrue(e.getMessage().contains("Could not open client transport"));
+    }
+  }
+}
diff --git a/fe/src/test/java/org/apache/impala/service/JdbcTest.java b/fe/src/test/java/org/apache/impala/service/JdbcTest.java
index 1749fbe..1fbc0d0 100644
--- a/fe/src/test/java/org/apache/impala/service/JdbcTest.java
+++ b/fe/src/test/java/org/apache/impala/service/JdbcTest.java
@@ -23,12 +23,10 @@ import static org.junit.Assert.assertNotNull;
 import static org.junit.Assert.assertNull;
 import static org.junit.Assert.assertTrue;
 
-import java.io.StringReader;
 import java.sql.Connection;
 import java.sql.ResultSet;
 import java.sql.ResultSetMetaData;
 import java.sql.SQLException;
-import java.sql.Statement;
 import java.sql.Types;
 import java.util.ArrayList;
 import java.util.Arrays;
@@ -36,22 +34,12 @@ import java.util.HashMap;
 import java.util.List;
 import java.util.Map;
 
-import org.apache.impala.analysis.CreateTableStmt;
-import org.apache.impala.analysis.Parser;
-import org.apache.impala.analysis.Parser.ParseException;
-import org.apache.impala.analysis.SqlParser;
-import org.apache.impala.analysis.SqlScanner;
-import org.apache.impala.analysis.StatementBase;
 import org.apache.impala.testutil.ImpalaJdbcClient;
-import org.apache.impala.thrift.TQueryOptions;
 import org.apache.impala.util.Metrics;
-import org.junit.After;
 import org.junit.AfterClass;
 import org.junit.BeforeClass;
 import org.junit.Test;
 
-import com.google.common.collect.Lists;
-
 /**
  * JdbcTest
  *
@@ -59,82 +47,10 @@ import com.google.common.collect.Lists;
  * getTableTypes, getColumnNames.
  *
  */
-public class JdbcTest {
-  private static Connection con_;
-
-  // Test-local list of test tables. These are cleaned up in @After.
-  private final List<String> testTableNames_ = Lists.newArrayList();
-
+public class JdbcTest extends JdbcTestBase {
   @BeforeClass
   public static void setUp() throws Exception {
-    con_ = createConnection();
-  }
-
-  @AfterClass
-  public static void cleanUp() throws Exception {
-    con_.close();
-    assertTrue("Connection should be closed", con_.isClosed());
-
-    Exception expectedException = null;
-    try {
-      con_.createStatement();
-    } catch (Exception e) {
-      expectedException = e;
-    }
-
-    assertNotNull("createStatement() on closed connection should throw exception",
-        expectedException);
-  }
-
-  private static Connection createConnection() throws Exception {
-    ImpalaJdbcClient client = ImpalaJdbcClient.createClientUsingHiveJdbcDriver();
-    client.connect();
-    Connection connection = client.getConnection();
-
-    assertNotNull("Connection is null", connection);
-    assertFalse("Connection should not be closed", connection.isClosed());
-    Statement stmt = connection.createStatement();
-    assertNotNull("Statement is null", stmt);
-
-    return connection;
-  }
-
-  protected void addTestTable(String createTableSql) throws Exception {
-    // Parse the stmt to extract the table name. We do this first to ensure
-    // that we do not execute arbitrary SQL here and pollute the test setup.
-    StatementBase result = Parser.parse(createTableSql);
-    if (!(result instanceof CreateTableStmt)) {
-      throw new Exception("Given stmt is not a CREATE TABLE stmt: " + createTableSql);
-    }
-
-    // Execute the stmt.
-    Statement stmt = con_.createStatement();
-    try {
-      stmt.execute(createTableSql);
-    } finally {
-      stmt.close();
-    }
-
-    // Once the stmt was executed successfully, add the fully-qualified table name
-    // for cleanup in @After.
-    CreateTableStmt parsedStmt = (CreateTableStmt) result;
-    testTableNames_.add(parsedStmt.getTblName().toString());
-  }
-
-  protected void dropTestTable(String tableName) throws SQLException {
-    Statement stmt = con_.createStatement();
-    try {
-      stmt.execute("DROP TABLE " + tableName);
-    } finally {
-      stmt.close();
-    }
-  }
-
-  @After
-  public void testCleanUp() throws SQLException {
-    for (String tableName: testTableNames_) {
-      dropTestTable(tableName);
-    }
+    con_ = createConnection(ImpalaJdbcClient.getNoAuthConnectionStr());
   }
 
   @Test
@@ -638,7 +554,7 @@ public class JdbcTest {
     List<Long> lastTimeSessionActive = new ArrayList<>();
 
     for (int timeout : timeoutPeriods) {
-      connections.add(createConnection());
+      connections.add(createConnection(ImpalaJdbcClient.getNoAuthConnectionStr()));
     }
 
     Long numOpenSessions = (Long)metrics.getMetric(
diff --git a/fe/src/test/java/org/apache/impala/service/JdbcTestBase.java b/fe/src/test/java/org/apache/impala/service/JdbcTestBase.java
new file mode 100644
index 0000000..74e309a
--- /dev/null
+++ b/fe/src/test/java/org/apache/impala/service/JdbcTestBase.java
@@ -0,0 +1,122 @@
+// 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.impala.service;
+
+import static org.junit.Assert.assertFalse;
+import static org.junit.Assert.assertNotNull;
+import static org.junit.Assert.assertTrue;
+
+import java.sql.Connection;
+import java.sql.SQLException;
+import java.sql.Statement;
+import java.util.List;
+
+import org.apache.impala.analysis.CreateTableStmt;
+import org.apache.impala.analysis.Parser;
+import org.apache.impala.analysis.StatementBase;
+import org.apache.impala.testutil.ImpalaJdbcClient;
+import org.junit.After;
+import org.junit.AfterClass;
+
+import com.google.common.collect.Lists;
+
+/**
+ * Base class providing utility functions for tests that need a Jdbc connection.
+ */
+public abstract class JdbcTestBase {
+  protected static Connection con_;
+
+  // Test-local list of test tables. These are cleaned up in @After.
+  protected final List<String> testTableNames_ = Lists.newArrayList();
+
+  /**
+   * Closes 'con_'. Any subclasses that specify their own 'AfterClass' will need to call
+   * this function there.
+   */
+  @AfterClass
+  public static void cleanUp() throws Exception {
+    con_.close();
+    assertTrue("Connection should be closed", con_.isClosed());
+
+    Exception expectedException = null;
+    try {
+      con_.createStatement();
+    } catch (Exception e) {
+      expectedException = e;
+    }
+
+    assertNotNull("createStatement() on closed connection should throw exception",
+        expectedException);
+  }
+
+  protected static Connection createConnection(String connStr) throws Exception {
+    ImpalaJdbcClient client = ImpalaJdbcClient.createClientUsingHiveJdbcDriver(connStr);
+    client.connect();
+    Connection connection = client.getConnection();
+
+    assertNotNull("Connection is null", connection);
+    assertFalse("Connection should not be closed", connection.isClosed());
+    Statement stmt = connection.createStatement();
+    assertNotNull("Statement is null", stmt);
+
+    return connection;
+  }
+
+  /**
+   * Runs 'createTableSql', which must be a "CREATE TABLE" sql statement, and stores the
+   * name of the created table in 'testTableNames_' so that it can be dropped after the
+   * test case by testCleanUp().
+   */
+  protected void addTestTable(String createTableSql) throws Exception {
+    // Parse the stmt to extract the table name. We do this first to ensure
+    // that we do not execute arbitrary SQL here and pollute the test setup.
+    StatementBase result = Parser.parse(createTableSql);
+    if (!(result instanceof CreateTableStmt)) {
+      throw new Exception("Given stmt is not a CREATE TABLE stmt: " + createTableSql);
+    }
+
+    // Execute the stmt.
+    Statement stmt = con_.createStatement();
+    try {
+      stmt.execute(createTableSql);
+    } finally {
+      stmt.close();
+    }
+
+    // Once the stmt was executed successfully, add the fully-qualified table name
+    // for cleanup in @After.
+    CreateTableStmt parsedStmt = (CreateTableStmt) result;
+    testTableNames_.add(parsedStmt.getTblName().toString());
+  }
+
+  protected void dropTestTable(String tableName) throws SQLException {
+    Statement stmt = con_.createStatement();
+    try {
+      stmt.execute("DROP TABLE " + tableName);
+    } finally {
+      stmt.close();
+    }
+  }
+
+  @After
+  public void testCleanUp() throws SQLException {
+    for (String tableName : testTableNames_) {
+      dropTestTable(tableName);
+    }
+  }
+}
diff --git a/fe/src/test/java/org/apache/impala/testutil/ImpalaJdbcClient.java b/fe/src/test/java/org/apache/impala/testutil/ImpalaJdbcClient.java
index 1d34ef0..32b3e77 100644
--- a/fe/src/test/java/org/apache/impala/testutil/ImpalaJdbcClient.java
+++ b/fe/src/test/java/org/apache/impala/testutil/ImpalaJdbcClient.java
@@ -58,6 +58,8 @@ public class ImpalaJdbcClient {
   // As of Hive 0.11 'noSasl' is case sensitive. See HIVE-4232 for more details.
   private final static String NOSASL_AUTH_SPEC = ";auth=noSasl";
 
+  private final static String LDAP_AUTH_SPEC = ";user=%s;password=%s";
+
   // The default connection string connects to localhost at the default hs2_port without
   // Sasl.
   private final static String DEFAULT_CONNECTION_STRING =
@@ -137,14 +139,25 @@ public class ImpalaJdbcClient {
   }
 
   public static ImpalaJdbcClient createClientUsingHiveJdbcDriver() {
-    return new ImpalaJdbcClient(
-        HIVE_SERVER2_DRIVER_NAME, DEFAULT_CONNECTION_STRING + NOSASL_AUTH_SPEC);
+    return new ImpalaJdbcClient(HIVE_SERVER2_DRIVER_NAME, getNoAuthConnectionStr());
   }
 
   public static ImpalaJdbcClient createClientUsingHiveJdbcDriver(String connString) {
     return new ImpalaJdbcClient(HIVE_SERVER2_DRIVER_NAME, connString);
   }
 
+  public static String getNoAuthConnectionStr() {
+    return getConnectionStr(NOSASL_AUTH_SPEC);
+  }
+
+  public static String getLdapConnectionStr(String username, String password) {
+    return getConnectionStr(String.format(LDAP_AUTH_SPEC, username, password));
+  }
+
+  private static String getConnectionStr(String authStr) {
+    return DEFAULT_CONNECTION_STRING + authStr;
+  }
+
   /**
    * Used to store the execution options passed via command line
    */
diff --git a/fe/src/test/resources/users.ldif b/fe/src/test/resources/users.ldif
new file mode 100644
index 0000000..93ced04
--- /dev/null
+++ b/fe/src/test/resources/users.ldif
@@ -0,0 +1,25 @@
+version: 1
+dn: dc=myorg,dc=com
+objectClass: domain
+objectClass: top
+dc: myorg
+
+dn: ou=Users,dc=myorg,dc=com
+objectClass: organizationalUnit
+objectClass: top
+ou: Users
+
+dn: ou=Groups,dc=myorg,dc=com
+objectClass: organizationalUnit
+objectClass: top
+ou: Groups
+
+dn: cn=Test1Ldap,ou=Users,dc=myorg,dc=com
+objectClass: inetOrgPerson
+objectClass: organizationalPerson
+objectClass: person
+objectClass: top
+cn: Test1Ldap
+sn: Ldap
+uid: ldaptest1
+userPassword: 12345
\ No newline at end of file


[impala] 01/02: IMPALA-8563: Update SSL ciphers used in BE tests

Posted by jb...@apache.org.
This is an automated email from the ASF dual-hosted git repository.

jbapple pushed a commit to branch master
in repository https://gitbox.apache.org/repos/asf/impala.git

commit 358e92ffa25270adb0cec90710409a7a6f2151da
Author: Laszlo Gaal <la...@cloudera.com>
AuthorDate: Fri May 17 16:49:27 2019 +0200

    IMPALA-8563: Update SSL ciphers used in BE tests
    
    Impala BE tests used RC4-based ciphers in a few BE tests, where
    OpenSSL contexts were set up manually.
    Since OpenSSL v1.1.0 these ciphers are not considered strong enough
    any more, so they are rejected, which made these tests fail on platforms
    using OpenSSL 1.1.0 (or higher), e.g. on Ubuntu 18.04.
    
    This patch changes the affected tests to use AES128 and AES256.
    
    The updated tests were verified on the following platforms:
    - Ubuntu 14.04, 16.04, 18.04
    - CentOS 6.4, 7.4
    
    Change-Id: I12b014361fb90afe63aed4b4608f6d6031e49cca
    Reviewed-on: http://gerrit.cloudera.org:8080/13364
    Reviewed-by: Tim Armstrong <ta...@cloudera.com>
    Reviewed-by: Michael Ho <kw...@cloudera.com>
    Tested-by: Impala Public Jenkins <im...@cloudera.com>
---
 be/src/rpc/rpc-mgr-test.h        | 4 ++--
 be/src/rpc/thrift-server-test.cc | 4 ++--
 be/src/util/webserver-test.cc    | 2 +-
 3 files changed, 5 insertions(+), 5 deletions(-)

diff --git a/be/src/rpc/rpc-mgr-test.h b/be/src/rpc/rpc-mgr-test.h
index 42c47fe..8b749b5 100644
--- a/be/src/rpc/rpc-mgr-test.h
+++ b/be/src/rpc/rpc-mgr-test.h
@@ -108,8 +108,8 @@ class ScopedSetTlsFlags {
 
 // Only use TLSv1.0 compatible ciphers, as tests might run on machines with only TLSv1.0
 // support.
-const string TLS1_0_COMPATIBLE_CIPHER = "RC4-SHA";
-const string TLS1_0_COMPATIBLE_CIPHER_2 = "RC4-MD5";
+const string TLS1_0_COMPATIBLE_CIPHER = "AES128-SHA";
+const string TLS1_0_COMPATIBLE_CIPHER_2 = "AES256-SHA";
 
 #define PAYLOAD_SIZE (4096)
 
diff --git a/be/src/rpc/thrift-server-test.cc b/be/src/rpc/thrift-server-test.cc
index 621f557..68b55e5 100644
--- a/be/src/rpc/thrift-server-test.cc
+++ b/be/src/rpc/thrift-server-test.cc
@@ -66,8 +66,8 @@ static string principal_kt_path;
 
 // Only use TLSv1.0 compatible ciphers, as tests might run on machines with only TLSv1.0
 // support.
-static const string TLS1_0_COMPATIBLE_CIPHER = "RC4-SHA";
-static const string TLS1_0_COMPATIBLE_CIPHER_2 = "RC4-MD5";
+static const string TLS1_0_COMPATIBLE_CIPHER = "AES128-SHA";
+static const string TLS1_0_COMPATIBLE_CIPHER_2 = "AES256-SHA";
 
 /// Dummy server class (chosen because it has the smallest interface to implement) that
 /// tests can use to start Thrift servers.
diff --git a/be/src/util/webserver-test.cc b/be/src/util/webserver-test.cc
index b100c74..c0c542a 100644
--- a/be/src/util/webserver-test.cc
+++ b/be/src/util/webserver-test.cc
@@ -268,7 +268,7 @@ TEST(Webserver, SslCipherSuite) {
 
   {
     auto ciphers = ScopedFlagSetter<string>::Make(
-        &FLAGS_ssl_cipher_list, "RC4-SHA");
+        &FLAGS_ssl_cipher_list, "AES128-SHA");
     Webserver webserver(FLAGS_webserver_port);
     ASSERT_OK(webserver.Start());
   }