Skip to content
Open
Show file tree
Hide file tree
Changes from all commits
Commits
File filter

Filter by extension

Filter by extension

Conversations
Failed to load comments.
Loading
Jump to
Jump to file
Failed to load files.
Loading
Diff view
Diff view
30 changes: 30 additions & 0 deletions core/src/main/java/io/substrait/relation/AbstractRelVisitor.java
Original file line number Diff line number Diff line change
@@ -1,8 +1,13 @@
package io.substrait.relation;

import io.substrait.relation.physical.BroadcastExchange;
import io.substrait.relation.physical.HashJoin;
import io.substrait.relation.physical.MergeJoin;
import io.substrait.relation.physical.MultiBucketExchange;
import io.substrait.relation.physical.NestedLoopJoin;
import io.substrait.relation.physical.RoundRobinExchange;
import io.substrait.relation.physical.ScatterExchange;
import io.substrait.relation.physical.SingleBucketExchange;
import io.substrait.util.VisitationContext;

public abstract class AbstractRelVisitor<O, C extends VisitationContext, E extends Exception>
Expand Down Expand Up @@ -138,4 +143,29 @@ public O visit(ExtensionDdl ddl, C context) throws E {
public O visit(NamedUpdate update, C context) throws E {
return visitFallback(update, context);
}

@Override
public O visit(ScatterExchange exchange, C context) throws E {
return visitFallback(exchange, context);
}

@Override
public O visit(SingleBucketExchange exchange, C context) throws E {
return visitFallback(exchange, context);
}

@Override
public O visit(MultiBucketExchange exchange, C context) throws E {
return visitFallback(exchange, context);
}

@Override
public O visit(BroadcastExchange exchange, C context) throws E {
return visitFallback(exchange, context);
}

@Override
public O visit(RoundRobinExchange exchange, C context) throws E {
return visitFallback(exchange, context);
}
}
174 changes: 174 additions & 0 deletions core/src/main/java/io/substrait/relation/ProtoRelConverter.java
Original file line number Diff line number Diff line change
@@ -1,6 +1,7 @@
package io.substrait.relation;

import io.substrait.expression.Expression;
import io.substrait.expression.FieldReference;
import io.substrait.expression.proto.ProtoExpressionConverter;
import io.substrait.extension.AdvancedExtension;
import io.substrait.extension.DefaultExtensionCatalog;
Expand All @@ -18,6 +19,7 @@
import io.substrait.proto.ConsistentPartitionWindowRel;
import io.substrait.proto.CrossRel;
import io.substrait.proto.DdlRel;
import io.substrait.proto.ExchangeRel;
import io.substrait.proto.ExpandRel;
import io.substrait.proto.ExtensionLeafRel;
import io.substrait.proto.ExtensionMultiRel;
Expand All @@ -37,9 +39,22 @@
import io.substrait.relation.extensions.EmptyDetail;
import io.substrait.relation.files.FileFormat;
import io.substrait.relation.files.FileOrFiles;
import io.substrait.relation.physical.AbstractExchangeRel;
import io.substrait.relation.physical.BroadcastExchange;
import io.substrait.relation.physical.HashJoin;
import io.substrait.relation.physical.ImmutableBroadcastExchange;
import io.substrait.relation.physical.ImmutableExchangeTarget;
import io.substrait.relation.physical.ImmutableMultiBucketExchange;
import io.substrait.relation.physical.ImmutableRoundRobinExchange;
import io.substrait.relation.physical.ImmutableScatterExchange;
import io.substrait.relation.physical.ImmutableSingleBucketExchange;
import io.substrait.relation.physical.MergeJoin;
import io.substrait.relation.physical.MultiBucketExchange;
import io.substrait.relation.physical.NestedLoopJoin;
import io.substrait.relation.physical.RoundRobinExchange;
import io.substrait.relation.physical.ScatterExchange;
import io.substrait.relation.physical.SingleBucketExchange;
import io.substrait.relation.physical.TargetType;
import io.substrait.type.NamedStruct;
import io.substrait.type.Type;
import io.substrait.type.proto.ProtoTypeConverter;
Expand Down Expand Up @@ -163,6 +178,8 @@ public Rel from(io.substrait.proto.Rel rel) {
return newDdl(rel.getDdl());
case UPDATE:
return newUpdate(rel.getUpdate());
case EXCHANGE:
return newExchange(rel.getExchange());
default:
throw new UnsupportedOperationException("Unsupported RelTypeCase of " + relType);
}
Expand Down Expand Up @@ -977,6 +994,163 @@ protected ConsistentPartitionWindow newConsistentPartitionWindow(
return builder.build();
}

protected AbstractExchangeRel newExchange(ExchangeRel rel) {
ExchangeRel.ExchangeKindCase exchangeKind = rel.getExchangeKindCase();
switch (exchangeKind) {
case SCATTER_BY_FIELDS:
return newScatterExchange(rel);
case SINGLE_TARGET:
return newSingleBucketExchange(rel);
case MULTI_TARGET:
return newMultiBucketExchange(rel);
case BROADCAST:
return newBroadcastExchange(rel);
case ROUND_ROBIN:
return newRoundRobinExchange(rel);
default:
throw new UnsupportedOperationException("Unsupported ExchangeKindCase of " + exchangeKind);
}
}

protected ScatterExchange newScatterExchange(ExchangeRel rel) {
Rel input = from(rel.getInput());
List<AbstractExchangeRel.ExchangeTarget> targets =
rel.getTargetsList().stream().map(this::newExchangeTarget).collect(Collectors.toList());

ProtoExpressionConverter protoExprConverter =
new ProtoExpressionConverter(lookup, extensions, input.getRecordType(), this);
List<FieldReference> fieldReferences =
rel.getScatterByFields().getFieldsList().stream()
.map(protoExprConverter::from)
.collect(Collectors.toList());

ImmutableScatterExchange.Builder builder =
ScatterExchange.builder()
.input(input)
.addAllFields(fieldReferences)
.partitionCount(rel.getPartitionCount())
.targets(targets);

builder
.commonExtension(optionalAdvancedExtension(rel.getCommon()))
.remap(optionalRelmap(rel.getCommon()))
.hint(optionalHint(rel.getCommon()));
if (rel.hasAdvancedExtension()) {
builder.extension(protoExtensionConverter.fromProto(rel.getAdvancedExtension()));
}
return builder.build();
}

protected SingleBucketExchange newSingleBucketExchange(ExchangeRel rel) {
Rel input = from(rel.getInput());
List<AbstractExchangeRel.ExchangeTarget> targets =
rel.getTargetsList().stream().map(this::newExchangeTarget).collect(Collectors.toList());
ProtoExpressionConverter protoExprConverter =
new ProtoExpressionConverter(lookup, extensions, input.getRecordType(), this);

ImmutableSingleBucketExchange.Builder builder =
SingleBucketExchange.builder()
.input(input)
.partitionCount(rel.getPartitionCount())
.targets(targets)
.expression(protoExprConverter.from(rel.getSingleTarget().getExpression()));

builder
.commonExtension(optionalAdvancedExtension(rel.getCommon()))
.remap(optionalRelmap(rel.getCommon()))
.hint(optionalHint(rel.getCommon()));
if (rel.hasAdvancedExtension()) {
builder.extension(protoExtensionConverter.fromProto(rel.getAdvancedExtension()));
}
return builder.build();
}

protected MultiBucketExchange newMultiBucketExchange(ExchangeRel rel) {
Rel input = from(rel.getInput());
List<AbstractExchangeRel.ExchangeTarget> targets =
rel.getTargetsList().stream().map(this::newExchangeTarget).collect(Collectors.toList());
ProtoExpressionConverter protoExprConverter =
new ProtoExpressionConverter(lookup, extensions, input.getRecordType(), this);

ImmutableMultiBucketExchange.Builder builder =
MultiBucketExchange.builder()
.input(input)
.partitionCount(rel.getPartitionCount())
.targets(targets)
.expression(protoExprConverter.from(rel.getMultiTarget().getExpression()))
.constrainedToCount(rel.getMultiTarget().getConstrainedToCount());

builder
.commonExtension(optionalAdvancedExtension(rel.getCommon()))
.remap(optionalRelmap(rel.getCommon()))
.hint(optionalHint(rel.getCommon()));
if (rel.hasAdvancedExtension()) {
builder.extension(protoExtensionConverter.fromProto(rel.getAdvancedExtension()));
}
return builder.build();
}

protected RoundRobinExchange newRoundRobinExchange(ExchangeRel rel) {
Rel input = from(rel.getInput());
List<AbstractExchangeRel.ExchangeTarget> targets =
rel.getTargetsList().stream().map(this::newExchangeTarget).collect(Collectors.toList());

ImmutableRoundRobinExchange.Builder builder =
RoundRobinExchange.builder()
.input(input)
.partitionCount(rel.getPartitionCount())
.targets(targets)
.exact(rel.getRoundRobin().getExact());

builder
.commonExtension(optionalAdvancedExtension(rel.getCommon()))
.remap(optionalRelmap(rel.getCommon()))
.hint(optionalHint(rel.getCommon()));
if (rel.hasAdvancedExtension()) {
builder.extension(protoExtensionConverter.fromProto(rel.getAdvancedExtension()));
}
return builder.build();
}

protected BroadcastExchange newBroadcastExchange(ExchangeRel rel) {
Rel input = from(rel.getInput());
List<AbstractExchangeRel.ExchangeTarget> targets =
rel.getTargetsList().stream().map(this::newExchangeTarget).collect(Collectors.toList());

ImmutableBroadcastExchange.Builder builder =
BroadcastExchange.builder()
.input(input)
.partitionCount(rel.getPartitionCount())
.targets(targets);

builder
.commonExtension(optionalAdvancedExtension(rel.getCommon()))
.remap(optionalRelmap(rel.getCommon()))
.hint(optionalHint(rel.getCommon()));
if (rel.hasAdvancedExtension()) {
builder.extension(protoExtensionConverter.fromProto(rel.getAdvancedExtension()));
}
return builder.build();
}

protected AbstractExchangeRel.ExchangeTarget newExchangeTarget(
ExchangeRel.ExchangeTarget target) {
ImmutableExchangeTarget.Builder builder = AbstractExchangeRel.ExchangeTarget.builder();
builder.addAllPartitionIds(target.getPartitionIdList());
switch (target.getTargetTypeCase()) {
case URI:
builder.type(TargetType.Uri.builder().uri(target.getUri()).build());
break;
case EXTENDED:
builder.type(TargetType.Extended.builder().extended(target.getExtended()).build());
break;
default:
throw new UnsupportedOperationException(
"Unsupported TargetTypeCase of " + target.getTargetTypeCase());
}
return builder.build();
}

protected static Optional<Rel.Remap> optionalRelmap(io.substrait.proto.RelCommon relCommon) {
return Optional.ofNullable(
relCommon.hasEmit() ? Rel.Remap.of(relCommon.getEmit().getOutputMappingList()) : null);
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -8,9 +8,14 @@
import io.substrait.expression.Expression;
import io.substrait.expression.FieldReference;
import io.substrait.expression.FunctionArg;
import io.substrait.relation.physical.BroadcastExchange;
import io.substrait.relation.physical.HashJoin;
import io.substrait.relation.physical.MergeJoin;
import io.substrait.relation.physical.MultiBucketExchange;
import io.substrait.relation.physical.NestedLoopJoin;
import io.substrait.relation.physical.RoundRobinExchange;
import io.substrait.relation.physical.ScatterExchange;
import io.substrait.relation.physical.SingleBucketExchange;
import io.substrait.util.EmptyVisitationContext;
import java.util.List;
import java.util.Optional;
Expand Down Expand Up @@ -274,6 +279,91 @@ public Optional<Rel> visit(NamedUpdate update, EmptyVisitationContext context) t
.build());
}

@Override
public Optional<Rel> visit(ScatterExchange exchange, EmptyVisitationContext context) throws E {
Optional<Rel> input = exchange.getInput().accept(this, context);
Optional<List<FieldReference>> fields =
transformList(exchange.getFields(), context, this::visitFieldReference);

if (allEmpty(input, fields)) {
return Optional.empty();
}

return Optional.of(
ScatterExchange.builder()
.from(exchange)
.input(input.orElse(exchange.getInput()))
.fields(fields.orElse(exchange.getFields()))
.build());
}

@Override
public Optional<Rel> visit(SingleBucketExchange exchange, EmptyVisitationContext context)
throws E {
Optional<Rel> input = exchange.getInput().accept(this, context);

Optional<Expression> expression =
exchange.getExpression().accept(getExpressionCopyOnWriteVisitor(), context);

if (allEmpty(input, expression)) {
return Optional.empty();
}

return Optional.of(
SingleBucketExchange.builder()
.from(exchange)
.input(input.orElse(exchange.getInput()))
.expression(expression.orElse(exchange.getExpression()))
.build());
}

@Override
public Optional<Rel> visit(MultiBucketExchange exchange, EmptyVisitationContext context)
throws E {
Optional<Rel> input = exchange.getInput().accept(this, context);
Optional<Expression> expression =
exchange.getExpression().accept(getExpressionCopyOnWriteVisitor(), context);

if (allEmpty(input)) {
return Optional.empty();
}

return Optional.of(
MultiBucketExchange.builder()
.from(exchange)
.input(input.orElse(exchange.getInput()))
.expression(expression.orElse(exchange.getExpression()))
.build());
}

@Override
public Optional<Rel> visit(RoundRobinExchange exchange, EmptyVisitationContext context) throws E {
Optional<Rel> input = exchange.getInput().accept(this, context);
if (allEmpty(input)) {
return Optional.empty();
}

return Optional.of(
RoundRobinExchange.builder()
.from(exchange)
.input(input.orElse(exchange.getInput()))
.build());
}

@Override
public Optional<Rel> visit(BroadcastExchange exchange, EmptyVisitationContext context) throws E {
Optional<Rel> input = exchange.getInput().accept(this, context);
if (allEmpty(input)) {
return Optional.empty();
}

return Optional.of(
BroadcastExchange.builder()
.from(exchange)
.input(input.orElse(exchange.getInput()))
.build());
}

@Override
public Optional<Rel> visit(Sort sort, EmptyVisitationContext context) throws E {
Optional<Rel> input = sort.getInput().accept(this, context);
Expand Down
Loading