Skip to content

Commit

Permalink
Make Redis minimal functionality test abstract
Browse files Browse the repository at this point in the history
  • Loading branch information
polaris6 authored and ebyhr committed Jun 16, 2022
1 parent c440595 commit 37bda5e
Showing 1 changed file with 23 additions and 63 deletions.
Original file line number Diff line number Diff line change
Expand Up @@ -15,60 +15,41 @@

import com.google.common.collect.ImmutableMap;
import io.trino.Session;
import io.trino.metadata.QualifiedObjectName;
import io.trino.metadata.TableHandle;
import io.trino.plugin.redis.util.JsonEncoder;
import io.trino.plugin.redis.util.RedisServer;
import io.trino.security.AllowAllAccessControl;
import io.trino.spi.connector.SchemaTableName;
import io.trino.spi.type.BigintType;
import io.trino.testing.MaterializedResult;
import io.trino.testing.StandaloneQueryRunner;
import org.testng.annotations.AfterClass;
import org.testng.annotations.AfterMethod;
import org.testng.annotations.BeforeClass;
import org.testng.annotations.BeforeMethod;
import org.testng.annotations.Test;
import redis.clients.jedis.Jedis;

import java.util.Optional;
import java.util.Map;
import java.util.UUID;

import static io.trino.plugin.redis.util.RedisTestUtils.createEmptyTableDescription;
import static io.trino.plugin.redis.util.RedisTestUtils.installRedisPlugin;
import static io.trino.testing.TestingSession.testSessionBuilder;
import static io.trino.testing.assertions.Assert.assertEquals;
import static io.trino.transaction.TransactionBuilder.transaction;
import static org.testng.Assert.assertTrue;

@Test(singleThreaded = true)
public class TestMinimalFunctionality
public abstract class AbstractTestMinimalFunctionality
{
private static final Session SESSION = testSessionBuilder()
protected static final Session SESSION = testSessionBuilder()
.setCatalog("redis")
.setSchema("default")
.build();

private RedisServer redisServer;
private String tableName;
private StandaloneQueryRunner queryRunner;
protected RedisServer redisServer;
protected String tableName;
protected StandaloneQueryRunner queryRunner;

protected abstract Map<String, String> connectorProperties();

@BeforeClass
public void startRedis()
{
redisServer = new RedisServer();
}

@AfterClass(alwaysRun = true)
public void stopRedis()
{
redisServer.close();
redisServer = null;
}

@BeforeMethod
public void spinUp()
{
this.tableName = "test_" + UUID.randomUUID().toString().replaceAll("-", "_");

this.queryRunner = new StandaloneQueryRunner(SESSION);
Expand All @@ -77,17 +58,24 @@ public void spinUp()
ImmutableMap.<SchemaTableName, RedisTableDescription>builder()
.put(createEmptyTableDescription(new SchemaTableName("default", tableName)))
.buildOrThrow(),
ImmutableMap.of());
connectorProperties());

populateData(1000);
}

@AfterMethod(alwaysRun = true)
public void tearDown()
@AfterClass(alwaysRun = true)
public void stopRedis()
{
clearData();

queryRunner.close();
queryRunner = null;

redisServer.close();
redisServer = null;
}

private void populateData(int count)
protected void populateData(int count)
{
JsonEncoder jsonEncoder = new JsonEncoder();
for (long i = 0; i < count; i++) {
Expand All @@ -98,38 +86,10 @@ private void populateData(int count)
}
}

@Test
public void testTableExists()
protected void clearData()
{
QualifiedObjectName name = new QualifiedObjectName("redis", "default", tableName);
transaction(queryRunner.getTransactionManager(), new AllowAllAccessControl())
.singleStatement()
.execute(SESSION, session -> {
Optional<TableHandle> handle = queryRunner.getServer().getMetadata().getTableHandle(session, name);
assertTrue(handle.isPresent());
});
}

@Test
public void testTableHasData()
{
MaterializedResult result = queryRunner.execute("SELECT count(1) from " + tableName);

MaterializedResult expected = MaterializedResult.resultBuilder(SESSION, BigintType.BIGINT)
.row(0L)
.build();

assertEquals(result, expected);

int count = 1000;
populateData(count);

result = queryRunner.execute("SELECT count(1) from " + tableName);

expected = MaterializedResult.resultBuilder(SESSION, BigintType.BIGINT)
.row((long) count)
.build();

assertEquals(result, expected);
try (Jedis jedis = redisServer.getJedisPool().getResource()) {
jedis.flushAll();
}
}
}

0 comments on commit 37bda5e

Please sign in to comment.