better derby test database

This commit is contained in:
Xavier Léauté 2014-10-31 16:25:36 -07:00
parent 80c72eb486
commit cad4d22f35
4 changed files with 111 additions and 102 deletions

View File

@ -36,9 +36,14 @@ public class DerbyConnector extends SQLMetadataConnector
@Inject @Inject
public DerbyConnector(Supplier<MetadataStorageConnectorConfig> config, Supplier<MetadataStorageTablesConfig> dbTables) public DerbyConnector(Supplier<MetadataStorageConnectorConfig> config, Supplier<MetadataStorageTablesConfig> dbTables)
{
this(config, dbTables, new DBI(getConnectionFactory("druidDerbyDb")));
}
public DerbyConnector(Supplier<MetadataStorageConnectorConfig> config, Supplier<MetadataStorageTablesConfig> dbTables, DBI dbi)
{ {
super(config, dbTables); super(config, dbTables);
this.dbi = new DBI(getConnectionFactory("druidDerbyDb")); this.dbi = dbi;
} }
@Override @Override
@ -59,7 +64,7 @@ public class DerbyConnector extends SQLMetadataConnector
@Override @Override
public DBI getDBI() { return dbi; } public DBI getDBI() { return dbi; }
private ConnectionFactory getConnectionFactory(String dbName) private static ConnectionFactory getConnectionFactory(String dbName)
{ {
try { try {
NetworkServerControl server = new NetworkServerControl(InetAddress.getByName("localhost"),1527); NetworkServerControl server = new NetworkServerControl(InetAddress.getByName("localhost"),1527);

View File

@ -30,6 +30,7 @@ import org.skife.jdbi.v2.Handle;
import org.skife.jdbi.v2.IDBI; import org.skife.jdbi.v2.IDBI;
import org.skife.jdbi.v2.tweak.HandleCallback; import org.skife.jdbi.v2.tweak.HandleCallback;
import org.skife.jdbi.v2.util.ByteArrayMapper; import org.skife.jdbi.v2.util.ByteArrayMapper;
import org.skife.jdbi.v2.util.IntegerMapper;
import java.sql.Connection; import java.sql.Connection;
import java.util.List; import java.util.List;
@ -256,10 +257,14 @@ public abstract class SQLMetadataConnector implements MetadataStorageConnector
Connection conn = getDBI().open().getConnection(); Connection conn = getDBI().open().getConnection();
handle.begin(); handle.begin();
conn.setAutoCommit(false); conn.setAutoCommit(false);
List<Map<String, Object>> entry = handle.createQuery( int count = handle
String.format("SELECT * FROM %1$s WHERE %2$s=:key", tableName, keyColumn) .createQuery(
).list(); String.format("SELECT COUNT(*) FROM %1$s WHERE %2$s=:key", tableName, keyColumn, valueColumn)
if (entry == null || entry.isEmpty()) { )
.bind("key", key)
.map(IntegerMapper.FIRST)
.first();
if (count == 0) {
handle.createStatement(String.format("INSERT INTO %1$s (%2$s, %3$s) VALUES (:key, :value)", handle.createStatement(String.format("INSERT INTO %1$s (%2$s, %3$s) VALUES (:key, :value)",
tableName, keyColumn, valueColumn)) tableName, keyColumn, valueColumn))
.bind("key", key) .bind("key", key)

View File

@ -20,30 +20,25 @@ package io.druid.metadata;
import com.fasterxml.jackson.databind.ObjectMapper; import com.fasterxml.jackson.databind.ObjectMapper;
import com.google.common.base.Suppliers; import com.google.common.base.Suppliers;
import com.google.common.base.Throwables;
import io.druid.jackson.DefaultObjectMapper; import io.druid.jackson.DefaultObjectMapper;
import org.junit.Assert; import org.junit.Assert;
import org.junit.Before; import org.junit.Before;
import org.junit.Test; import org.junit.Test;
import org.skife.jdbi.v2.Batch; import org.skife.jdbi.v2.Batch;
import org.skife.jdbi.v2.DBI;
import org.skife.jdbi.v2.Handle; import org.skife.jdbi.v2.Handle;
import org.skife.jdbi.v2.tweak.HandleCallback; import org.skife.jdbi.v2.tweak.HandleCallback;
import java.lang.Exception;
import java.util.LinkedList; import java.util.LinkedList;
public class SQLMetadataConnectorTest public class SQLMetadataConnectorTest
{ {
private static final ObjectMapper jsonMapper = new DefaultObjectMapper(); private static final ObjectMapper jsonMapper = new DefaultObjectMapper();
private DerbyConnector connector; private TestDerbyConnector connector;
private DBI dbi;
private MetadataStorageTablesConfig tablesConfig = MetadataStorageTablesConfig.fromBase("test"); private MetadataStorageTablesConfig tablesConfig = MetadataStorageTablesConfig.fromBase("test");
private String TABLE_NAME = tablesConfig.getSegmentsTable();
@Before @Before
public void setUp() { public void setUp() throws Exception {
MetadataStorageConnectorConfig config = jsonReadWriteRead( MetadataStorageConnectorConfig config = jsonReadWriteRead(
"{" "{"
+ "\"type\" : \"db\",\n" + "\"type\" : \"db\",\n"
@ -52,15 +47,14 @@ public class SQLMetadataConnectorTest
MetadataStorageConnectorConfig.class MetadataStorageConnectorConfig.class
); );
connector = new DerbyConnector( connector = new TestDerbyConnector(
Suppliers.ofInstance(config), Suppliers.ofInstance(config),
Suppliers.ofInstance(tablesConfig) Suppliers.ofInstance(tablesConfig)
); );
dbi = connector.getDBI();
} }
@Test @Test
public void testCreateTables() public void testCreateTables() throws Exception
{ {
final LinkedList<String> tables = new LinkedList<String>(); final LinkedList<String> tables = new LinkedList<String>();
tables.add(tablesConfig.getConfigTable()); tables.add(tablesConfig.getConfigTable());
@ -69,13 +63,13 @@ public class SQLMetadataConnectorTest
tables.add(tablesConfig.getTaskLockTable()); tables.add(tablesConfig.getTaskLockTable());
tables.add(tablesConfig.getTaskLogTable()); tables.add(tablesConfig.getTaskLogTable());
tables.add(tablesConfig.getTasksTable()); tables.add(tablesConfig.getTasksTable());
try {
connector.createSegmentTable(); connector.createSegmentTable();
connector.createConfigTable(); connector.createConfigTable();
connector.createRulesTable(); connector.createRulesTable();
connector.createTaskTables(); connector.createTaskTables();
dbi.withHandle( connector.getDBI().withHandle(
new HandleCallback<Void>() new HandleCallback<Void>()
{ {
@Override @Override
@ -92,83 +86,54 @@ public class SQLMetadataConnectorTest
} }
} }
); );
}
finally {
dbi.withHandle(
new HandleCallback<Void>()
{
@Override
public Void withHandle(Handle handle) throws Exception
{
final Batch batch = handle.createBatch();
for (String table : tables) { for (String table : tables) {
batch.add(String.format("DROP TABLE %s", table)); dropTable(table);
}
batch.execute();
return null;
}
}
);
} }
} }
@Test @Test
public void testInsertOrUpdate() { public void testInsertOrUpdate() throws Exception
try {
connector.createSegmentTable(dbi, TABLE_NAME);
connector.insertOrUpdate(TABLE_NAME, "dummy1", "dummy2", "emperor", "penguin".getBytes());
dbi.withHandle(
new HandleCallback<Void>()
{ {
@Override final String tableName = "test";
public Void withHandle(Handle handle) throws Exception connector.createConfigTable(connector.getDBI(), tableName);
{
Assert.assertEquals(connector.lookup(TABLE_NAME, "dummy1", "dummy2", "emperor"), Assert.assertNull(connector.lookup(tableName, "name", "payload", "emperor"));
"penguin".getBytes());
return null; connector.insertOrUpdate(tableName, "name", "payload", "emperor", "penguin".getBytes());
} Assert.assertArrayEquals(
} "penguin".getBytes(),
connector.lookup(tableName, "name", "payload", "emperor")
); );
connector.insertOrUpdate(TABLE_NAME, "dummy1", "dummy2", "emperor", "penguin chick".getBytes()); connector.insertOrUpdate(tableName, "name", "payload", "emperor", "penguin chick".getBytes());
dbi.withHandle( Assert.assertArrayEquals(
new HandleCallback<Void>() "penguin chick".getBytes(),
{ connector.lookup(tableName, "name", "payload", "emperor")
@Override
public Void withHandle(Handle handle) throws Exception
{
Assert.assertEquals(connector.lookup(TABLE_NAME, "dummy1", "dummy2", "emperor"),
"penguin chick".getBytes());
return null;
}
}
); );
} catch (Exception e) { dropTable(tableName);
} finally { }
dbi.withHandle(
private void dropTable(final String tableName)
{
connector.getDBI().withHandle(
new HandleCallback<Void>() new HandleCallback<Void>()
{ {
@Override @Override
public Void withHandle(Handle handle) throws Exception public Void withHandle(Handle handle) throws Exception
{ {
handle.createStatement(String.format("DROP TABLE %s", TABLE_NAME)) handle.createStatement(String.format("DROP TABLE %s", tableName))
.execute(); .execute();
return null; return null;
} }
} }
); );
} }
}
private <T> T jsonReadWriteRead(String s, Class<T> klass) private <T> T jsonReadWriteRead(String s, Class<T> klass) throws Exception
{ {
try {
return jsonMapper.readValue(jsonMapper.writeValueAsBytes(jsonMapper.readValue(s, klass)), klass); return jsonMapper.readValue(jsonMapper.writeValueAsBytes(jsonMapper.readValue(s, klass)), klass);
} }
catch (Exception e) {
throw Throwables.propagate(e);
}
}
} }

View File

@ -0,0 +1,34 @@
/*
* Druid - a distributed column store.
* Copyright (C) 2014 Metamarkets Group Inc.
*
* This program is free software; you can redistribute it and/or
* modify it under the terms of the GNU General Public License
* as published by the Free Software Foundation; either version 2
* of the License, or (at your option) any later version.
*
* This program is distributed in the hope that it will be useful,
* but WITHOUT ANY WARRANTY; without even the implied warranty of
* MERCHANTABILITY or FITNESS FOR A PARTICULAR PURPOSE. See the
* GNU General Public License for more details.
*
* You should have received a copy of the GNU General Public License
* along with this program; if not, write to the Free Software
* Foundation, Inc., 51 Franklin Street, Fifth Floor, Boston, MA 02110-1301, USA.
*/
package io.druid.metadata;
import com.google.common.base.Supplier;
import org.skife.jdbi.v2.DBI;
public class TestDerbyConnector extends DerbyConnector
{
public TestDerbyConnector(
Supplier<MetadataStorageConnectorConfig> config,
Supplier<MetadataStorageTablesConfig> dbTables
)
{
super(config, dbTables, new DBI("jdbc:derby:memory:druidTest;create=true"));
}
}