Skip to content
This repository was archived by the owner on May 12, 2021. It is now read-only.
Closed
Original file line number Diff line number Diff line change
Expand Up @@ -20,6 +20,7 @@

import org.apache.tajo.QueryTestCaseBase;
import org.apache.tajo.exception.AmbiguousColumnException;
import org.apache.tajo.exception.NotImplementedException;
import org.apache.tajo.exception.TajoException;
import org.junit.After;
import org.junit.Test;
Expand Down Expand Up @@ -139,6 +140,17 @@ public final void testTableSubquery3() throws Exception {
runSimpleTests();
}

@Test(expected = NotImplementedException.class)
public final void testSelectRecordField() throws Exception {
executeString("select glossary.\"GlossDiv\" " +
"from self_desc_table2 where char_length(glossary.\"GlossDiv\".title) > 0 ");
}

@Test(expected = NotImplementedException.class)
public final void testSelectRecordField2() throws Exception {
executeString("select glossary from self_desc_table2 where char_length(glossary.\"GlossDiv\".title) > 0 ");
}

@Test
@Option(sort = true)
@SimpleTest
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -94,7 +94,7 @@ private static class ExprFinderIncludeSubquery extends SimpleAlgebraVisitor<Find
public static <T extends Expr> Set<T> finds(Expr expr, OpType type) throws TajoException {
FinderContext<T> context = new FinderContext<>(type);
ExprFinderIncludeSubquery finder = new ExprFinderIncludeSubquery();
finder.visit(context, new Stack<>(), expr);
finder.visit(context, new Stack<Expr>(), expr);
return context.set;
}

Expand Down Expand Up @@ -138,7 +138,7 @@ public LogicalNode process(PlanContext context, Expr expr) throws TajoException
if (processor == null) {
processor = new Processor();
}
return processor.visit(new ProcessorContext(context), new Stack<>(), expr);
return processor.visit(new ProcessorContext(context), new Stack<Expr>(), expr);
}

static class ProcessorContext {
Expand Down Expand Up @@ -403,7 +403,17 @@ private Schema buildSchemaFromColumnSet(Set<Column> columns) {
Set<ColumnVertex> rootVertexes = new HashSet<>();
Schema schema = new Schema();

for (Column eachColumn : columns) {
Set<Column> simpleColumns = new HashSet<>();
List<Column> columnList = new ArrayList<>(columns);
Collections.sort(columnList, new Comparator<Column>() {
@Override
public int compare(Column c1, Column c2) {
return c2.getSimpleName().split(NestedPathUtil.PATH_DELIMITER).length -
c1.getSimpleName().split(NestedPathUtil.PATH_DELIMITER).length;
}
});

for (Column eachColumn : columnList) {
String simpleName = eachColumn.getSimpleName();
if (NestedPathUtil.isPath(simpleName)) {
String[] paths = simpleName.split(NestedPathUtil.PATH_DELIMITER);
Expand All @@ -413,24 +423,26 @@ private Schema buildSchemaFromColumnSet(Set<Column> columns) {
parentName = CatalogUtil.buildFQName(eachColumn.getQualifier(), parentName);
}
// Leaf column type is TEXT; otherwise, RECORD.
Type childDataType = (i == paths.length-2) ? Type.TEXT : Type.RECORD;
ColumnVertex childVertex = new ColumnVertex(
paths[i+1],
StringUtils.join(paths, NestedPathUtil.PATH_DELIMITER, 0, i+2),
Type.RECORD
);
if (i == paths.length - 2 && !schemaGraph.hasVertex(childVertex)) {
childVertex = new ColumnVertex(childVertex.name, childVertex.path, Type.TEXT);
}

ColumnVertex parentVertex = new ColumnVertex(
parentName,
StringUtils.join(paths, NestedPathUtil.PATH_DELIMITER, 0, i+1),
Type.RECORD);
schemaGraph.addEdge(
new ColumnEdge(
new ColumnVertex(
paths[i+1],
StringUtils.join(paths, NestedPathUtil.PATH_DELIMITER, 0, i+2), childDataType
),
parentVertex));
schemaGraph.addEdge(new ColumnEdge(childVertex, parentVertex));
if (i == 0) {
rootVertexes.add(parentVertex);
}
}
} else {
schema.addColumn(eachColumn);
simpleColumns.add(eachColumn);
}
}

Expand All @@ -441,6 +453,13 @@ private Schema buildSchemaFromColumnSet(Set<Column> columns) {
schema.addColumn(eachRoot.column);
}

// Add simple columns
for (Column eachColumn : simpleColumns) {
if (!schema.contains(eachColumn)) {
schema.addColumn(eachColumn);
}
}

return schema;
}

Expand All @@ -461,15 +480,14 @@ public boolean equals(Object o) {
if (o instanceof ColumnVertex) {
ColumnVertex other = (ColumnVertex) o;
return this.name.equals(other.name) &&
this.type.equals(other.type) &&
this.path.equals(other.path);
}
return false;
}

@Override
public int hashCode() {
return Objects.hashCode(name, type, path);
return Objects.hashCode(name, path);
}
}

Expand All @@ -487,6 +505,10 @@ private static class SchemaGraph extends SimpleDirectedGraph<ColumnVertex, Colum
public void addEdge(ColumnEdge edge) {
this.addEdge(edge.child, edge.parent, edge);
}

public boolean hasVertex(ColumnVertex vertex) {
return this.directedEdges.containsKey(vertex) || this.reversedEdges.containsKey(vertex);
}
}

private static class RecordColumnBuilder implements DirectedGraphVisitor<ColumnVertex> {
Expand Down