mirror of https://github.com/apache/druid.git
SQL: Add test for concurrent JDBC queries. (#4290)
This commit is contained in:
parent
5c0a7ad2f8
commit
8ca7f9410e
|
@ -19,11 +19,17 @@
|
||||||
|
|
||||||
package io.druid.sql.avatica;
|
package io.druid.sql.avatica;
|
||||||
|
|
||||||
|
import com.google.common.base.Throwables;
|
||||||
import com.google.common.collect.ImmutableList;
|
import com.google.common.collect.ImmutableList;
|
||||||
import com.google.common.collect.ImmutableMap;
|
import com.google.common.collect.ImmutableMap;
|
||||||
import com.google.common.collect.ImmutableSet;
|
import com.google.common.collect.ImmutableSet;
|
||||||
|
import com.google.common.collect.Iterables;
|
||||||
import com.google.common.collect.Lists;
|
import com.google.common.collect.Lists;
|
||||||
import com.google.common.collect.Maps;
|
import com.google.common.collect.Maps;
|
||||||
|
import com.google.common.util.concurrent.Futures;
|
||||||
|
import com.google.common.util.concurrent.ListenableFuture;
|
||||||
|
import com.google.common.util.concurrent.ListeningExecutorService;
|
||||||
|
import com.google.common.util.concurrent.MoreExecutors;
|
||||||
import io.druid.java.util.common.Pair;
|
import io.druid.java.util.common.Pair;
|
||||||
import io.druid.server.DruidNode;
|
import io.druid.server.DruidNode;
|
||||||
import io.druid.sql.calcite.planner.Calcites;
|
import io.druid.sql.calcite.planner.Calcites;
|
||||||
|
@ -57,11 +63,13 @@ import java.sql.SQLException;
|
||||||
import java.sql.Statement;
|
import java.sql.Statement;
|
||||||
import java.sql.Timestamp;
|
import java.sql.Timestamp;
|
||||||
import java.sql.Types;
|
import java.sql.Types;
|
||||||
|
import java.util.ArrayList;
|
||||||
import java.util.List;
|
import java.util.List;
|
||||||
import java.util.Map;
|
import java.util.Map;
|
||||||
import java.util.Properties;
|
import java.util.Properties;
|
||||||
import java.util.Random;
|
import java.util.Random;
|
||||||
import java.util.Set;
|
import java.util.Set;
|
||||||
|
import java.util.concurrent.Executors;
|
||||||
|
|
||||||
public class DruidAvaticaHandlerTest
|
public class DruidAvaticaHandlerTest
|
||||||
{
|
{
|
||||||
|
@ -76,7 +84,7 @@ public class DruidAvaticaHandlerTest
|
||||||
@Override
|
@Override
|
||||||
public int getMaxStatementsPerConnection()
|
public int getMaxStatementsPerConnection()
|
||||||
{
|
{
|
||||||
return 2;
|
return 4;
|
||||||
}
|
}
|
||||||
};
|
};
|
||||||
|
|
||||||
|
@ -353,15 +361,48 @@ public class DruidAvaticaHandlerTest
|
||||||
);
|
);
|
||||||
}
|
}
|
||||||
|
|
||||||
|
@Test(timeout = 30000)
|
||||||
|
public void testConcurrentQueries() throws Exception
|
||||||
|
{
|
||||||
|
final List<ListenableFuture<Integer>> futures = new ArrayList<>();
|
||||||
|
final ListeningExecutorService exec = MoreExecutors.listeningDecorator(
|
||||||
|
Executors.newFixedThreadPool(AVATICA_CONFIG.getMaxStatementsPerConnection())
|
||||||
|
);
|
||||||
|
for (int i = 0; i < 2000; i++) {
|
||||||
|
final String query = String.format("SELECT COUNT(*) + %s AS ci FROM foo", i);
|
||||||
|
futures.add(
|
||||||
|
exec.submit(() -> {
|
||||||
|
try (
|
||||||
|
final Statement statement = client.createStatement();
|
||||||
|
final ResultSet resultSet = statement.executeQuery(query)
|
||||||
|
) {
|
||||||
|
final List<Map<String, Object>> rows = getRows(resultSet);
|
||||||
|
return ((Number) Iterables.getOnlyElement(rows).get("ci")).intValue();
|
||||||
|
}
|
||||||
|
catch (SQLException e) {
|
||||||
|
throw Throwables.propagate(e);
|
||||||
|
}
|
||||||
|
})
|
||||||
|
);
|
||||||
|
}
|
||||||
|
|
||||||
|
final List<Integer> integers = Futures.allAsList(futures).get();
|
||||||
|
for (int i = 0; i < 2000; i++) {
|
||||||
|
Assert.assertEquals(i + 6, (int) integers.get(i));
|
||||||
|
}
|
||||||
|
}
|
||||||
|
|
||||||
@Test
|
@Test
|
||||||
public void testTooManyStatements() throws Exception
|
public void testTooManyStatements() throws Exception
|
||||||
{
|
{
|
||||||
final Statement statement1 = client.createStatement();
|
final Statement statement1 = client.createStatement();
|
||||||
final Statement statement2 = client.createStatement();
|
final Statement statement2 = client.createStatement();
|
||||||
|
final Statement statement3 = client.createStatement();
|
||||||
|
final Statement statement4 = client.createStatement();
|
||||||
|
|
||||||
expectedException.expect(AvaticaClientRuntimeException.class);
|
expectedException.expect(AvaticaClientRuntimeException.class);
|
||||||
expectedException.expectMessage("Too many open statements, limit is[2]");
|
expectedException.expectMessage("Too many open statements, limit is[4]");
|
||||||
final Statement statement3 = client.createStatement();
|
final Statement statement5 = client.createStatement();
|
||||||
}
|
}
|
||||||
|
|
||||||
@Test
|
@Test
|
||||||
|
@ -372,6 +413,11 @@ public class DruidAvaticaHandlerTest
|
||||||
client.createStatement().close();
|
client.createStatement().close();
|
||||||
client.createStatement().close();
|
client.createStatement().close();
|
||||||
client.createStatement().close();
|
client.createStatement().close();
|
||||||
|
client.createStatement().close();
|
||||||
|
client.createStatement().close();
|
||||||
|
client.createStatement().close();
|
||||||
|
client.createStatement().close();
|
||||||
|
client.createStatement().close();
|
||||||
|
|
||||||
Assert.assertTrue(true);
|
Assert.assertTrue(true);
|
||||||
}
|
}
|
||||||
|
|
Loading…
Reference in New Issue