浏览代码

ESQL: Move serialization for Literal and Order (#109709)

This moves the serialization for `Literal` and `Order` into the classes
themselves to line up better with how everything else in Elasticsearch
is serialized.
Nik Everett 1 年之前
父节点
当前提交
3bfecc8d22

+ 84 - 1
x-pack/plugin/esql-core/src/main/java/org/elasticsearch/xpack/esql/core/expression/Literal.java

@@ -6,17 +6,35 @@
  */
  */
 package org.elasticsearch.xpack.esql.core.expression;
 package org.elasticsearch.xpack.esql.core.expression;
 
 
+import org.apache.lucene.util.BytesRef;
+import org.elasticsearch.TransportVersions;
+import org.elasticsearch.common.io.stream.NamedWriteableRegistry;
+import org.elasticsearch.common.io.stream.StreamInput;
+import org.elasticsearch.common.io.stream.StreamOutput;
 import org.elasticsearch.xpack.esql.core.QlIllegalArgumentException;
 import org.elasticsearch.xpack.esql.core.QlIllegalArgumentException;
 import org.elasticsearch.xpack.esql.core.tree.NodeInfo;
 import org.elasticsearch.xpack.esql.core.tree.NodeInfo;
 import org.elasticsearch.xpack.esql.core.tree.Source;
 import org.elasticsearch.xpack.esql.core.tree.Source;
 import org.elasticsearch.xpack.esql.core.type.DataType;
 import org.elasticsearch.xpack.esql.core.type.DataType;
+import org.elasticsearch.xpack.esql.core.util.PlanStreamInput;
 
 
+import java.io.IOException;
+import java.util.List;
 import java.util.Objects;
 import java.util.Objects;
 
 
+import static org.elasticsearch.xpack.esql.core.type.DataType.CARTESIAN_POINT;
+import static org.elasticsearch.xpack.esql.core.type.DataType.GEO_POINT;
+import static org.elasticsearch.xpack.esql.core.util.SpatialCoordinateTypes.CARTESIAN;
+import static org.elasticsearch.xpack.esql.core.util.SpatialCoordinateTypes.GEO;
+
 /**
 /**
- * SQL Literal or constant.
+ * Literal or constant.
  */
  */
 public class Literal extends LeafExpression {
 public class Literal extends LeafExpression {
+    public static final NamedWriteableRegistry.Entry ENTRY = new NamedWriteableRegistry.Entry(
+        Expression.class,
+        "Literal",
+        Literal::readFrom
+    );
 
 
     public static final Literal TRUE = new Literal(Source.EMPTY, Boolean.TRUE, DataType.BOOLEAN);
     public static final Literal TRUE = new Literal(Source.EMPTY, Boolean.TRUE, DataType.BOOLEAN);
     public static final Literal FALSE = new Literal(Source.EMPTY, Boolean.FALSE, DataType.BOOLEAN);
     public static final Literal FALSE = new Literal(Source.EMPTY, Boolean.FALSE, DataType.BOOLEAN);
@@ -31,6 +49,25 @@ public class Literal extends LeafExpression {
         this.value = value;
         this.value = value;
     }
     }
 
 
+    private static Literal readFrom(StreamInput in) throws IOException {
+        Source source = Source.readFrom((StreamInput & PlanStreamInput) in);
+        Object value = in.readGenericValue();
+        DataType dataType = DataType.readFrom(in);
+        return new Literal(source, mapToLiteralValue(in, dataType, value), dataType);
+    }
+
+    @Override
+    public void writeTo(StreamOutput out) throws IOException {
+        Source.EMPTY.writeTo(out);
+        out.writeGenericValue(mapFromLiteralValue(out, dataType, value));
+        dataType.writeTo(out);
+    }
+
+    @Override
+    public String getWriteableName() {
+        return ENTRY.name;
+    }
+
     @Override
     @Override
     protected NodeInfo<? extends Literal> info() {
     protected NodeInfo<? extends Literal> info() {
         return NodeInfo.create(this, Literal::new, value, dataType);
         return NodeInfo.create(this, Literal::new, value, dataType);
@@ -112,4 +149,50 @@ public class Literal extends LeafExpression {
     public static Literal of(Expression source, Object value) {
     public static Literal of(Expression source, Object value) {
         return new Literal(source.source(), value, source.dataType());
         return new Literal(source.source(), value, source.dataType());
     }
     }
+
+    /**
+     * Not all literal values are currently supported in StreamInput/StreamOutput as generic values.
+     * This mapper allows for addition of new and interesting values without (yet) adding to StreamInput/Output.
+     * This makes the most sense during the pre-GA version of ESQL. When we get near GA we might want to push this down.
+     * <p>
+     * For the spatial point type support we need to care about the fact that 8.12.0 uses encoded longs for serializing
+     * while 8.13 uses WKB.
+     */
+    private static Object mapFromLiteralValue(StreamOutput out, DataType dataType, Object value) {
+        if (dataType == GEO_POINT || dataType == CARTESIAN_POINT) {
+            // In 8.12.0 we serialized point literals as encoded longs, but now use WKB
+            if (out.getTransportVersion().before(TransportVersions.V_8_13_0)) {
+                if (value instanceof List<?> list) {
+                    return list.stream().map(v -> mapFromLiteralValue(out, dataType, v)).toList();
+                }
+                return wkbAsLong(dataType, (BytesRef) value);
+            }
+        }
+        return value;
+    }
+
+    /**
+     * Not all literal values are currently supported in StreamInput/StreamOutput as generic values.
+     * This mapper allows for addition of new and interesting values without (yet) changing StreamInput/Output.
+     */
+    private static Object mapToLiteralValue(StreamInput in, DataType dataType, Object value) {
+        if (dataType == GEO_POINT || dataType == CARTESIAN_POINT) {
+            // In 8.12.0 we serialized point literals as encoded longs, but now use WKB
+            if (in.getTransportVersion().before(TransportVersions.V_8_13_0)) {
+                if (value instanceof List<?> list) {
+                    return list.stream().map(v -> mapToLiteralValue(in, dataType, v)).toList();
+                }
+                return longAsWKB(dataType, (Long) value);
+            }
+        }
+        return value;
+    }
+
+    private static BytesRef longAsWKB(DataType dataType, long encoded) {
+        return dataType == GEO_POINT ? GEO.longAsWkb(encoded) : CARTESIAN.longAsWkb(encoded);
+    }
+
+    private static long wkbAsLong(DataType dataType, BytesRef wkb) {
+        return dataType == GEO_POINT ? GEO.wkbAsLong(wkb) : CARTESIAN.wkbAsLong(wkb);
+    }
 }
 }

+ 6 - 2
x-pack/plugin/esql-core/src/test/java/org/elasticsearch/xpack/esql/core/expression/LiteralTests.java

@@ -76,6 +76,10 @@ public class LiteralTests extends AbstractNodeTestCase<Literal, Expression> {
 
 
     @Override
     @Override
     protected Literal mutate(Literal instance) {
     protected Literal mutate(Literal instance) {
+        return mutateLiteral(instance);
+    }
+
+    public static Literal mutateLiteral(Literal instance) {
         List<Function<Literal, Literal>> mutators = new ArrayList<>();
         List<Function<Literal, Literal>> mutators = new ArrayList<>();
         // Changing the location doesn't count as mutation because..... it just doesn't, ok?!
         // Changing the location doesn't count as mutation because..... it just doesn't, ok?!
         // Change the value to another valid value
         // Change the value to another valid value
@@ -116,7 +120,7 @@ public class LiteralTests extends AbstractNodeTestCase<Literal, Expression> {
         assertEquals("this type of node doesn't have any children to replace", e.getMessage());
         assertEquals("this type of node doesn't have any children to replace", e.getMessage());
     }
     }
 
 
-    private Object randomValueOfTypeOtherThan(Object original, DataType type) {
+    private static Object randomValueOfTypeOtherThan(Object original, DataType type) {
         for (ValueAndCompatibleTypes gen : GENERATORS) {
         for (ValueAndCompatibleTypes gen : GENERATORS) {
             if (gen.validDataTypes.get(0) == type) {
             if (gen.validDataTypes.get(0) == type) {
                 return randomValueOtherThan(original, () -> DataTypeConverter.convert(gen.valueSupplier.get(), type));
                 return randomValueOtherThan(original, () -> DataTypeConverter.convert(gen.valueSupplier.get(), type));
@@ -125,7 +129,7 @@ public class LiteralTests extends AbstractNodeTestCase<Literal, Expression> {
         throw new IllegalArgumentException("No native generator for [" + type + "]");
         throw new IllegalArgumentException("No native generator for [" + type + "]");
     }
     }
 
 
-    private List<DataType> validReplacementDataTypes(Object value, DataType type) {
+    private static List<DataType> validReplacementDataTypes(Object value, DataType type) {
         List<DataType> validDataTypes = new ArrayList<>();
         List<DataType> validDataTypes = new ArrayList<>();
         List<DataType> options = Arrays.asList(BYTE, SHORT, INTEGER, LONG, FLOAT, DOUBLE, BOOLEAN);
         List<DataType> options = Arrays.asList(BYTE, SHORT, INTEGER, LONG, FLOAT, DOUBLE, BOOLEAN);
         for (DataType candidate : options) {
         for (DataType candidate : options) {

+ 30 - 1
x-pack/plugin/esql/src/main/java/org/elasticsearch/xpack/esql/expression/Order.java

@@ -7,18 +7,48 @@
 
 
 package org.elasticsearch.xpack.esql.expression;
 package org.elasticsearch.xpack.esql.expression;
 
 
+import org.elasticsearch.common.io.stream.NamedWriteableRegistry;
+import org.elasticsearch.common.io.stream.StreamInput;
+import org.elasticsearch.common.io.stream.StreamOutput;
 import org.elasticsearch.xpack.esql.core.expression.Expression;
 import org.elasticsearch.xpack.esql.core.expression.Expression;
 import org.elasticsearch.xpack.esql.core.tree.NodeInfo;
 import org.elasticsearch.xpack.esql.core.tree.NodeInfo;
 import org.elasticsearch.xpack.esql.core.tree.Source;
 import org.elasticsearch.xpack.esql.core.tree.Source;
 import org.elasticsearch.xpack.esql.core.type.DataType;
 import org.elasticsearch.xpack.esql.core.type.DataType;
+import org.elasticsearch.xpack.esql.io.stream.PlanStreamInput;
+import org.elasticsearch.xpack.esql.io.stream.PlanStreamOutput;
 
 
+import java.io.IOException;
 import java.util.List;
 import java.util.List;
 
 
 public class Order extends org.elasticsearch.xpack.esql.core.expression.Order {
 public class Order extends org.elasticsearch.xpack.esql.core.expression.Order {
+    public static final NamedWriteableRegistry.Entry ENTRY = new NamedWriteableRegistry.Entry(Expression.class, "Order", Order::new);
+
     public Order(Source source, Expression child, OrderDirection direction, NullsPosition nulls) {
     public Order(Source source, Expression child, OrderDirection direction, NullsPosition nulls) {
         super(source, child, direction, nulls);
         super(source, child, direction, nulls);
     }
     }
 
 
+    public Order(StreamInput in) throws IOException {
+        this(
+            Source.readFrom((PlanStreamInput) in),
+            ((PlanStreamInput) in).readExpression(),
+            in.readEnum(org.elasticsearch.xpack.esql.core.expression.Order.OrderDirection.class),
+            in.readEnum(org.elasticsearch.xpack.esql.core.expression.Order.NullsPosition.class)
+        );
+    }
+
+    @Override
+    public void writeTo(StreamOutput out) throws IOException {
+        Source.EMPTY.writeTo(out);
+        ((PlanStreamOutput) out).writeExpression(child());
+        out.writeEnum(direction());
+        out.writeEnum(nullsPosition());
+    }
+
+    @Override
+    public String getWriteableName() {
+        return ENTRY.name;
+    }
+
     @Override
     @Override
     protected TypeResolution resolveType() {
     protected TypeResolution resolveType() {
         if (DataType.isString(child().dataType())) {
         if (DataType.isString(child().dataType())) {
@@ -36,5 +66,4 @@ public class Order extends org.elasticsearch.xpack.esql.core.expression.Order {
     protected NodeInfo<org.elasticsearch.xpack.esql.core.expression.Order> info() {
     protected NodeInfo<org.elasticsearch.xpack.esql.core.expression.Order> info() {
         return NodeInfo.create(this, Order::new, child(), direction(), nullsPosition());
         return NodeInfo.create(this, Order::new, child(), direction(), nullsPosition());
     }
     }
-
 }
 }

+ 12 - 96
x-pack/plugin/esql/src/main/java/org/elasticsearch/xpack/esql/io/stream/PlanNamedTypes.java

@@ -7,7 +7,6 @@
 
 
 package org.elasticsearch.xpack.esql.io.stream;
 package org.elasticsearch.xpack.esql.io.stream;
 
 
-import org.apache.lucene.util.BytesRef;
 import org.elasticsearch.TransportVersion;
 import org.elasticsearch.TransportVersion;
 import org.elasticsearch.TransportVersions;
 import org.elasticsearch.TransportVersions;
 import org.elasticsearch.common.TriFunction;
 import org.elasticsearch.common.TriFunction;
@@ -46,7 +45,6 @@ import org.elasticsearch.xpack.esql.core.plan.logical.Limit;
 import org.elasticsearch.xpack.esql.core.plan.logical.LogicalPlan;
 import org.elasticsearch.xpack.esql.core.plan.logical.LogicalPlan;
 import org.elasticsearch.xpack.esql.core.plan.logical.OrderBy;
 import org.elasticsearch.xpack.esql.core.plan.logical.OrderBy;
 import org.elasticsearch.xpack.esql.core.tree.Source;
 import org.elasticsearch.xpack.esql.core.tree.Source;
-import org.elasticsearch.xpack.esql.core.type.DataType;
 import org.elasticsearch.xpack.esql.core.type.EsField;
 import org.elasticsearch.xpack.esql.core.type.EsField;
 import org.elasticsearch.xpack.esql.expression.function.UnsupportedAttribute;
 import org.elasticsearch.xpack.esql.expression.function.UnsupportedAttribute;
 import org.elasticsearch.xpack.esql.expression.function.aggregate.AggregateFunction;
 import org.elasticsearch.xpack.esql.expression.function.aggregate.AggregateFunction;
@@ -178,10 +176,6 @@ import java.util.function.BiFunction;
 import java.util.function.Function;
 import java.util.function.Function;
 
 
 import static java.util.Map.entry;
 import static java.util.Map.entry;
-import static org.elasticsearch.xpack.esql.core.type.DataType.CARTESIAN_POINT;
-import static org.elasticsearch.xpack.esql.core.type.DataType.GEO_POINT;
-import static org.elasticsearch.xpack.esql.core.util.SpatialCoordinateTypes.CARTESIAN;
-import static org.elasticsearch.xpack.esql.core.util.SpatialCoordinateTypes.GEO;
 import static org.elasticsearch.xpack.esql.io.stream.PlanNameRegistry.Entry.of;
 import static org.elasticsearch.xpack.esql.io.stream.PlanNameRegistry.Entry.of;
 import static org.elasticsearch.xpack.esql.io.stream.PlanNameRegistry.PlanReader.readerFromPlanReader;
 import static org.elasticsearch.xpack.esql.io.stream.PlanNameRegistry.PlanReader.readerFromPlanReader;
 import static org.elasticsearch.xpack.esql.io.stream.PlanNameRegistry.PlanWriter.writerFromPlanWriter;
 import static org.elasticsearch.xpack.esql.io.stream.PlanNameRegistry.PlanWriter.writerFromPlanWriter;
@@ -363,10 +357,7 @@ public final class PlanNamedTypes {
             of(ScalarFunction.class, MvSort.class, PlanNamedTypes::writeMvSort, PlanNamedTypes::readMvSort),
             of(ScalarFunction.class, MvSort.class, PlanNamedTypes::writeMvSort, PlanNamedTypes::readMvSort),
             of(ScalarFunction.class, MvSlice.class, PlanNamedTypes::writeMvSlice, PlanNamedTypes::readMvSlice),
             of(ScalarFunction.class, MvSlice.class, PlanNamedTypes::writeMvSlice, PlanNamedTypes::readMvSlice),
             of(ScalarFunction.class, MvSum.class, PlanNamedTypes::writeMvFunction, PlanNamedTypes::readMvFunction),
             of(ScalarFunction.class, MvSum.class, PlanNamedTypes::writeMvFunction, PlanNamedTypes::readMvFunction),
-            of(ScalarFunction.class, MvZip.class, PlanNamedTypes::writeMvZip, PlanNamedTypes::readMvZip),
-            // Expressions (other)
-            of(Expression.class, Literal.class, PlanNamedTypes::writeLiteral, PlanNamedTypes::readLiteral),
-            of(Expression.class, Order.class, PlanNamedTypes::writeOrder, PlanNamedTypes::readOrder)
+            of(ScalarFunction.class, MvZip.class, PlanNamedTypes::writeMvZip, PlanNamedTypes::readMvZip)
         );
         );
         List<PlanNameRegistry.Entry> entries = new ArrayList<>(declared);
         List<PlanNameRegistry.Entry> entries = new ArrayList<>(declared);
 
 
@@ -378,6 +369,8 @@ public final class PlanNamedTypes {
             entries.add(of(Expression.class, e));
             entries.add(of(Expression.class, e));
         }
         }
         entries.add(of(Expression.class, UnsupportedAttribute.ENTRY));
         entries.add(of(Expression.class, UnsupportedAttribute.ENTRY));
+        entries.add(of(Expression.class, Literal.ENTRY));
+        entries.add(of(Expression.class, org.elasticsearch.xpack.esql.expression.Order.ENTRY));
 
 
         return entries;
         return entries;
     }
     }
@@ -678,14 +671,14 @@ public final class PlanNamedTypes {
         return new OrderExec(
         return new OrderExec(
             Source.readFrom(in),
             Source.readFrom(in),
             in.readPhysicalPlanNode(),
             in.readPhysicalPlanNode(),
-            in.readCollectionAsList(readerFromPlanReader(PlanNamedTypes::readOrder))
+            in.readCollectionAsList(org.elasticsearch.xpack.esql.expression.Order::new)
         );
         );
     }
     }
 
 
     static void writeOrderExec(PlanStreamOutput out, OrderExec orderExec) throws IOException {
     static void writeOrderExec(PlanStreamOutput out, OrderExec orderExec) throws IOException {
         Source.EMPTY.writeTo(out);
         Source.EMPTY.writeTo(out);
         out.writePhysicalPlanNode(orderExec.child());
         out.writePhysicalPlanNode(orderExec.child());
-        out.writeCollection(orderExec.order(), writerFromPlanWriter(PlanNamedTypes::writeOrder));
+        out.writeCollection(orderExec.order());
     }
     }
 
 
     static ProjectExec readProjectExec(PlanStreamInput in) throws IOException {
     static ProjectExec readProjectExec(PlanStreamInput in) throws IOException {
@@ -731,7 +724,7 @@ public final class PlanNamedTypes {
         return new TopNExec(
         return new TopNExec(
             Source.readFrom(in),
             Source.readFrom(in),
             in.readPhysicalPlanNode(),
             in.readPhysicalPlanNode(),
-            in.readCollectionAsList(readerFromPlanReader(PlanNamedTypes::readOrder)),
+            in.readCollectionAsList(org.elasticsearch.xpack.esql.expression.Order::new),
             in.readNamed(Expression.class),
             in.readNamed(Expression.class),
             in.readOptionalVInt()
             in.readOptionalVInt()
         );
         );
@@ -740,7 +733,7 @@ public final class PlanNamedTypes {
     static void writeTopNExec(PlanStreamOutput out, TopNExec topNExec) throws IOException {
     static void writeTopNExec(PlanStreamOutput out, TopNExec topNExec) throws IOException {
         Source.EMPTY.writeTo(out);
         Source.EMPTY.writeTo(out);
         out.writePhysicalPlanNode(topNExec.child());
         out.writePhysicalPlanNode(topNExec.child());
-        out.writeCollection(topNExec.order(), writerFromPlanWriter(PlanNamedTypes::writeOrder));
+        out.writeCollection(topNExec.order());
         out.writeExpression(topNExec.limit());
         out.writeExpression(topNExec.limit());
         out.writeOptionalVInt(topNExec.estimatedRowSize());
         out.writeOptionalVInt(topNExec.estimatedRowSize());
     }
     }
@@ -969,14 +962,14 @@ public final class PlanNamedTypes {
         return new OrderBy(
         return new OrderBy(
             Source.readFrom(in),
             Source.readFrom(in),
             in.readLogicalPlanNode(),
             in.readLogicalPlanNode(),
-            in.readCollectionAsList(readerFromPlanReader(PlanNamedTypes::readOrder))
+            in.readCollectionAsList(org.elasticsearch.xpack.esql.expression.Order::new)
         );
         );
     }
     }
 
 
     static void writeOrderBy(PlanStreamOutput out, OrderBy order) throws IOException {
     static void writeOrderBy(PlanStreamOutput out, OrderBy order) throws IOException {
         Source.EMPTY.writeTo(out);
         Source.EMPTY.writeTo(out);
         out.writeLogicalPlanNode(order.child());
         out.writeLogicalPlanNode(order.child());
-        out.writeCollection(order.order(), writerFromPlanWriter(PlanNamedTypes::writeOrder));
+        out.writeCollection(order.order());
     }
     }
 
 
     static Project readProject(PlanStreamInput in) throws IOException {
     static Project readProject(PlanStreamInput in) throws IOException {
@@ -993,15 +986,15 @@ public final class PlanNamedTypes {
         return new TopN(
         return new TopN(
             Source.readFrom(in),
             Source.readFrom(in),
             in.readLogicalPlanNode(),
             in.readLogicalPlanNode(),
-            in.readCollectionAsList(readerFromPlanReader(PlanNamedTypes::readOrder)),
-            in.readNamed(Expression.class)
+            in.readCollectionAsList(org.elasticsearch.xpack.esql.expression.Order::new),
+            in.readExpression()
         );
         );
     }
     }
 
 
     static void writeTopN(PlanStreamOutput out, TopN topN) throws IOException {
     static void writeTopN(PlanStreamOutput out, TopN topN) throws IOException {
         Source.EMPTY.writeTo(out);
         Source.EMPTY.writeTo(out);
         out.writeLogicalPlanNode(topN.child());
         out.writeLogicalPlanNode(topN.child());
-        out.writeCollection(topN.order(), writerFromPlanWriter(PlanNamedTypes::writeOrder));
+        out.writeCollection(topN.order());
         out.writeExpression(topN.limit());
         out.writeExpression(topN.limit());
     }
     }
 
 
@@ -1552,83 +1545,6 @@ public final class PlanNamedTypes {
         out.writeExpression(fn.right());
         out.writeExpression(fn.right());
     }
     }
 
 
-    // -- Expressions (other)
-
-    static Literal readLiteral(PlanStreamInput in) throws IOException {
-        Source source = Source.readFrom(in);
-        Object value = in.readGenericValue();
-        DataType dataType = DataType.readFrom(in);
-        return new Literal(source, mapToLiteralValue(in, dataType, value), dataType);
-    }
-
-    static void writeLiteral(PlanStreamOutput out, Literal literal) throws IOException {
-        Source.EMPTY.writeTo(out);
-        out.writeGenericValue(mapFromLiteralValue(out, literal.dataType(), literal.value()));
-        out.writeString(literal.dataType().typeName());
-    }
-
-    /**
-     * Not all literal values are currently supported in StreamInput/StreamOutput as generic values.
-     * This mapper allows for addition of new and interesting values without (yet) adding to StreamInput/Output.
-     * This makes the most sense during the pre-GA version of ESQL. When we get near GA we might want to push this down.
-     * <p>
-     * For the spatial point type support we need to care about the fact that 8.12.0 uses encoded longs for serializing
-     * while 8.13 uses WKB.
-     */
-    private static Object mapFromLiteralValue(PlanStreamOutput out, DataType dataType, Object value) {
-        if (dataType == GEO_POINT || dataType == CARTESIAN_POINT) {
-            // In 8.12.0 we serialized point literals as encoded longs, but now use WKB
-            if (out.getTransportVersion().before(TransportVersions.V_8_13_0)) {
-                if (value instanceof List<?> list) {
-                    return list.stream().map(v -> mapFromLiteralValue(out, dataType, v)).toList();
-                }
-                return wkbAsLong(dataType, (BytesRef) value);
-            }
-        }
-        return value;
-    }
-
-    /**
-     * Not all literal values are currently supported in StreamInput/StreamOutput as generic values.
-     * This mapper allows for addition of new and interesting values without (yet) changing StreamInput/Output.
-     */
-    private static Object mapToLiteralValue(PlanStreamInput in, DataType dataType, Object value) {
-        if (dataType == GEO_POINT || dataType == CARTESIAN_POINT) {
-            // In 8.12.0 we serialized point literals as encoded longs, but now use WKB
-            if (in.getTransportVersion().before(TransportVersions.V_8_13_0)) {
-                if (value instanceof List<?> list) {
-                    return list.stream().map(v -> mapToLiteralValue(in, dataType, v)).toList();
-                }
-                return longAsWKB(dataType, (Long) value);
-            }
-        }
-        return value;
-    }
-
-    private static BytesRef longAsWKB(DataType dataType, long encoded) {
-        return dataType == GEO_POINT ? GEO.longAsWkb(encoded) : CARTESIAN.longAsWkb(encoded);
-    }
-
-    private static long wkbAsLong(DataType dataType, BytesRef wkb) {
-        return dataType == GEO_POINT ? GEO.wkbAsLong(wkb) : CARTESIAN.wkbAsLong(wkb);
-    }
-
-    static Order readOrder(PlanStreamInput in) throws IOException {
-        return new org.elasticsearch.xpack.esql.expression.Order(
-            Source.readFrom(in),
-            in.readNamed(Expression.class),
-            in.readEnum(Order.OrderDirection.class),
-            in.readEnum(Order.NullsPosition.class)
-        );
-    }
-
-    static void writeOrder(PlanStreamOutput out, Order order) throws IOException {
-        Source.EMPTY.writeTo(out);
-        out.writeExpression(order.child());
-        out.writeEnum(order.direction());
-        out.writeEnum(order.nullsPosition());
-    }
-
     // -- ancillary supporting classes of plan nodes, etc
     // -- ancillary supporting classes of plan nodes, etc
 
 
     static EsQueryExec.FieldSort readFieldSort(PlanStreamInput in) throws IOException {
     static EsQueryExec.FieldSort readFieldSort(PlanStreamInput in) throws IOException {

+ 10 - 1
x-pack/plugin/esql/src/test/java/org/elasticsearch/xpack/esql/expression/AbstractExpressionSerializationTests.java

@@ -31,6 +31,7 @@ import java.util.Map;
 import java.util.stream.Collectors;
 import java.util.stream.Collectors;
 
 
 import static org.hamcrest.Matchers.equalTo;
 import static org.hamcrest.Matchers.equalTo;
+import static org.hamcrest.Matchers.sameInstance;
 
 
 public abstract class AbstractExpressionSerializationTests<T extends Expression> extends AbstractWireTestCase<T> {
 public abstract class AbstractExpressionSerializationTests<T extends Expression> extends AbstractWireTestCase<T> {
     public static Source randomSource() {
     public static Source randomSource() {
@@ -59,13 +60,21 @@ public abstract class AbstractExpressionSerializationTests<T extends Expression>
                 PlanStreamInput pin = new PlanStreamInput(in, new PlanNameRegistry(), in.namedWriteableRegistry(), config);
                 PlanStreamInput pin = new PlanStreamInput(in, new PlanNameRegistry(), in.namedWriteableRegistry(), config);
                 @SuppressWarnings("unchecked")
                 @SuppressWarnings("unchecked")
                 T deser = (T) pin.readNamedWriteable(Expression.class);
                 T deser = (T) pin.readNamedWriteable(Expression.class);
-                assertThat(deser.source(), equalTo(instance.source()));
+                if (alwaysEmptySource()) {
+                    assertThat(deser.source(), sameInstance(Source.EMPTY));
+                } else {
+                    assertThat(deser.source(), equalTo(instance.source()));
+                }
                 return deser;
                 return deser;
             },
             },
             version
             version
         );
         );
     }
     }
 
 
+    protected boolean alwaysEmptySource() {
+        return false;
+    }
+
     protected abstract List<NamedWriteableRegistry.Entry> getNamedWriteables();
     protected abstract List<NamedWriteableRegistry.Entry> getNamedWriteables();
 
 
     @Override
     @Override

+ 37 - 0
x-pack/plugin/esql/src/test/java/org/elasticsearch/xpack/esql/expression/LiteralSerializationTests.java

@@ -0,0 +1,37 @@
+/*
+ * Copyright Elasticsearch B.V. and/or licensed to Elasticsearch B.V. under one
+ * or more contributor license agreements. Licensed under the Elastic License
+ * 2.0; you may not use this file except in compliance with the Elastic License
+ * 2.0.
+ */
+
+package org.elasticsearch.xpack.esql.expression;
+
+import org.elasticsearch.common.io.stream.NamedWriteableRegistry;
+import org.elasticsearch.xpack.esql.core.expression.Literal;
+import org.elasticsearch.xpack.esql.core.expression.LiteralTests;
+
+import java.io.IOException;
+import java.util.List;
+
+public class LiteralSerializationTests extends AbstractExpressionSerializationTests<Literal> {
+    @Override
+    protected Literal createTestInstance() {
+        return LiteralTests.randomLiteral();
+    }
+
+    @Override
+    protected Literal mutateInstance(Literal instance) throws IOException {
+        return LiteralTests.mutateLiteral(instance);
+    }
+
+    @Override
+    protected List<NamedWriteableRegistry.Entry> getNamedWriteables() {
+        return List.of(Literal.ENTRY);
+    }
+
+    @Override
+    protected boolean alwaysEmptySource() {
+        return true;
+    }
+}

+ 54 - 0
x-pack/plugin/esql/src/test/java/org/elasticsearch/xpack/esql/expression/OrderSerializationTests.java

@@ -0,0 +1,54 @@
+/*
+ * Copyright Elasticsearch B.V. and/or licensed to Elasticsearch B.V. under one
+ * or more contributor license agreements. Licensed under the Elastic License
+ * 2.0; you may not use this file except in compliance with the Elastic License
+ * 2.0.
+ */
+
+package org.elasticsearch.xpack.esql.expression;
+
+import org.elasticsearch.common.io.stream.NamedWriteableRegistry;
+import org.elasticsearch.xpack.esql.core.expression.Expression;
+import org.elasticsearch.xpack.esql.core.tree.Source;
+
+import java.io.IOException;
+import java.util.List;
+
+public class OrderSerializationTests extends AbstractExpressionSerializationTests<Order> {
+    @Override
+    protected Order createTestInstance() {
+        return new Order(randomSource(), randomChild(), randomDirection(), randomNulls());
+    }
+
+    private static org.elasticsearch.xpack.esql.core.expression.Order.OrderDirection randomDirection() {
+        return randomFrom(org.elasticsearch.xpack.esql.core.expression.Order.OrderDirection.values());
+    }
+
+    private static org.elasticsearch.xpack.esql.core.expression.Order.NullsPosition randomNulls() {
+        return randomFrom(org.elasticsearch.xpack.esql.core.expression.Order.NullsPosition.values());
+    }
+
+    @Override
+    protected Order mutateInstance(Order instance) throws IOException {
+        Source source = instance.source();
+        Expression child = instance.child();
+        org.elasticsearch.xpack.esql.core.expression.Order.OrderDirection direction = instance.direction();
+        org.elasticsearch.xpack.esql.core.expression.Order.NullsPosition nulls = instance.nullsPosition();
+        switch (between(0, 2)) {
+            case 0 -> child = randomValueOtherThan(child, AbstractExpressionSerializationTests::randomChild);
+            case 1 -> direction = randomValueOtherThan(direction, OrderSerializationTests::randomDirection);
+            case 2 -> nulls = randomValueOtherThan(nulls, OrderSerializationTests::randomNulls);
+        }
+        return new Order(source, child, direction, nulls);
+    }
+
+    @Override
+    protected List<NamedWriteableRegistry.Entry> getNamedWriteables() {
+        return List.of(Order.ENTRY);
+    }
+
+    @Override
+    protected boolean alwaysEmptySource() {
+        return true;
+    }
+}

+ 0 - 18
x-pack/plugin/esql/src/test/java/org/elasticsearch/xpack/esql/io/stream/PlanNamedTypesTests.java

@@ -308,24 +308,6 @@ public class PlanNamedTypesTests extends ESTestCase {
         EqualsHashCodeTestUtils.checkEqualsAndHashCode(orig, unused -> deser);
         EqualsHashCodeTestUtils.checkEqualsAndHashCode(orig, unused -> deser);
     }
     }
 
 
-    public void testLiteralSimple() throws IOException {
-        var orig = new Literal(Source.EMPTY, 1, DataType.INTEGER);
-        BytesStreamOutput bso = new BytesStreamOutput();
-        PlanStreamOutput out = new PlanStreamOutput(bso, planNameRegistry, null);
-        PlanNamedTypes.writeLiteral(out, orig);
-        var deser = PlanNamedTypes.readLiteral(planStreamInput(bso));
-        EqualsHashCodeTestUtils.checkEqualsAndHashCode(orig, unused -> deser);
-    }
-
-    public void testOrderSimple() throws IOException {
-        var orig = new Order(Source.EMPTY, field("val", DataType.INTEGER), Order.OrderDirection.ASC, Order.NullsPosition.FIRST);
-        BytesStreamOutput bso = new BytesStreamOutput();
-        PlanStreamOutput out = new PlanStreamOutput(bso, planNameRegistry, null);
-        PlanNamedTypes.writeOrder(out, orig);
-        var deser = (Order) PlanNamedTypes.readOrder(planStreamInput(bso));
-        EqualsHashCodeTestUtils.checkEqualsAndHashCode(orig, unused -> deser);
-    }
-
     public void testFieldSortSimple() throws IOException {
     public void testFieldSortSimple() throws IOException {
         var orig = new EsQueryExec.FieldSort(field("val", DataType.LONG), Order.OrderDirection.ASC, Order.NullsPosition.FIRST);
         var orig = new EsQueryExec.FieldSort(field("val", DataType.LONG), Order.OrderDirection.ASC, Order.NullsPosition.FIRST);
         BytesStreamOutput bso = new BytesStreamOutput();
         BytesStreamOutput bso = new BytesStreamOutput();