Skip to content

Commit

Permalink
Fix non-string object arrays in JMX connector
Browse files Browse the repository at this point in the history
  • Loading branch information
nezihyigitbasi authored and electrum committed Jul 21, 2015
1 parent 35223e1 commit ef7b355
Show file tree
Hide file tree
Showing 3 changed files with 42 additions and 4 deletions.
1 change: 1 addition & 0 deletions presto-docs/src/main/sphinx/release/release-0.112.rst
Original file line number Diff line number Diff line change
Expand Up @@ -6,3 +6,4 @@ General Changes
---------------

* Fix handling of ``LIMIT`` when used with a partitioned :func:`row_number`.
* Fix non-string object arrays in JMX connector.
Original file line number Diff line number Diff line change
Expand Up @@ -116,10 +116,7 @@ else if (javaType == double.class) {
else if (javaType == Slice.class) {
if (value.getClass().isArray()) {
// return a string representation of the array
if (value.getClass().getComponentType() == String.class) {
row.add(Arrays.toString((Object[]) value));
}
else if (value.getClass().getComponentType() == boolean.class) {
if (value.getClass().getComponentType() == boolean.class) {
row.add(Arrays.toString((boolean[]) value));
}
else if (value.getClass().getComponentType() == byte.class) {
Expand All @@ -143,6 +140,9 @@ else if (value.getClass().getComponentType() == long.class) {
else if (value.getClass().getComponentType() == short.class) {
row.add(Arrays.toString((short[]) value));
}
else {
row.add(Arrays.toString((Object[]) value));
}
}
else {
row.add(value.toString());
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -15,11 +15,15 @@

import com.facebook.presto.spi.ColumnHandle;
import com.facebook.presto.spi.ConnectorPartitionResult;
import com.facebook.presto.spi.ConnectorSession;
import com.facebook.presto.spi.ConnectorSplit;
import com.facebook.presto.spi.ConnectorSplitSource;
import com.facebook.presto.spi.HostAddress;
import com.facebook.presto.spi.Node;
import com.facebook.presto.spi.NodeManager;
import com.facebook.presto.spi.RecordCursor;
import com.facebook.presto.spi.RecordSet;
import com.facebook.presto.spi.SchemaTableName;
import com.facebook.presto.spi.TupleDomain;
import com.google.common.collect.ImmutableList;
import com.google.common.collect.ImmutableMap;
Expand All @@ -33,19 +37,27 @@
import java.util.Set;
import java.util.concurrent.ExecutionException;

import static com.facebook.presto.spi.type.TimeZoneKey.UTC_KEY;
import static com.facebook.presto.spi.type.VarcharType.VARCHAR;
import static java.lang.String.format;
import static java.lang.management.ManagementFactory.getPlatformMBeanServer;
import static java.util.Locale.ENGLISH;
import static java.util.stream.Collectors.toSet;
import static org.testng.Assert.assertEquals;

public class TestJmxSplitManager
{
private static final ConnectorSession SESSION = new ConnectorSession("user", UTC_KEY, ENGLISH, System.currentTimeMillis(), null);

private final Node localNode = new TestingNode("host1");
private final Set<Node> nodes = ImmutableSet.of(localNode, new TestingNode("host2"), new TestingNode("host3"));

private final TestingNodeManager nodeManager = new TestingNodeManager();
private final JmxColumnHandle columnHandle = new JmxColumnHandle("test", "node", VARCHAR);
private final JmxTableHandle tableHandle = new JmxTableHandle("test", "objectName", ImmutableList.of(columnHandle));
private final JmxSplitManager splitManager = new JmxSplitManager("test", new TestingNodeManager());
private final JmxMetadata metadata = new JmxMetadata("test", getPlatformMBeanServer());
private final JmxRecordSetProvider recordSetProvider = new JmxRecordSetProvider(getPlatformMBeanServer(), nodeManager);

@Test
public void testPredicatePushdown()
Expand Down Expand Up @@ -84,6 +96,31 @@ public void testNoPredicate()
assertEquals(actualNodes, expectedNodes);
}

@Test
public void testRecordSetProvider()
throws Exception
{
for (SchemaTableName schemaTableName : metadata.listTables(SESSION, "jmx")) {
JmxTableHandle tableHandle = metadata.getTableHandle(SESSION, schemaTableName);
List<ColumnHandle> columnHandles = ImmutableList.copyOf(metadata.getColumnHandles(tableHandle).values());

ConnectorPartitionResult connectorPartitionResult = splitManager.getPartitions(tableHandle, TupleDomain.all());
ConnectorSplitSource splitSource = splitManager.getPartitionSplits(tableHandle, connectorPartitionResult.getPartitions());
List<ConnectorSplit> allSplits = getAllSplits(splitSource);
assertEquals(allSplits.size(), nodes.size());
ConnectorSplit split = allSplits.get(0);

RecordSet recordSet = recordSetProvider.getRecordSet(split, columnHandles);
try (RecordCursor cursor = recordSet.cursor()) {
while (cursor.advanceNextPosition()) {
for (int i = 0; i < recordSet.getColumnTypes().size(); i++) {
cursor.isNull(i);
}
}
}
}
}

private static List<ConnectorSplit> getAllSplits(ConnectorSplitSource splitSource)
throws InterruptedException, ExecutionException
{
Expand Down

0 comments on commit ef7b355

Please sign in to comment.