This is an automated email from the ASF dual-hosted git repository.

ruifengz pushed a commit to branch master
in repository https://gitbox.apache.org/repos/asf/spark.git


The following commit(s) were added to refs/heads/master by this push:
     new a99f3106c80 Revert "[SPARK-45088][PYTHON][CONNECT] Make `getitem` work 
with duplicated column"
a99f3106c80 is described below

commit a99f3106c802710a1fe31773ffbae709bcd5b690
Author: Ruifeng Zheng <[email protected]>
AuthorDate: Wed Oct 4 19:30:42 2023 +0800

    Revert "[SPARK-45088][PYTHON][CONNECT] Make `getitem` work with duplicated 
column"
    
    ### What changes were proposed in this pull request?
    This reverts commit 73d3c49c97ae1be3f9f96fbc86be1c91cd17a656.
    
    ### Why are the changes needed?
    to address 
https://github.com/apache/spark/pull/42828#discussion_r1338141627 and 
https://github.com/apache/spark/pull/43115#discussion_r1338097284, should not 
use `GetColumnByOrdinal` in this case.
    
    Need to find another approach, but let's revert it first.
    
    ### Does this PR introduce _any_ user-facing change?
    No
    
    ### How was this patch tested?
    CI
    
    ### Was this patch authored or co-authored using generative AI tooling?
    No
    
    Closes #43211 from zhengruifeng/revert_SPARK_45088.
    
    Authored-by: Ruifeng Zheng <[email protected]>
    Signed-off-by: Ruifeng Zheng <[email protected]>
---
 .../main/protobuf/spark/connect/expressions.proto  |  10 --
 .../sql/connect/planner/SparkConnectPlanner.scala  |  14 +--
 python/pyspark/sql/connect/dataframe.py            |  21 +---
 python/pyspark/sql/connect/expressions.py          |  34 ------
 .../pyspark/sql/connect/proto/expressions_pb2.py   | 126 ++++++++++-----------
 .../pyspark/sql/connect/proto/expressions_pb2.pyi  |  40 -------
 python/pyspark/sql/dataframe.py                    |   5 +-
 python/pyspark/sql/tests/test_dataframe.py         |  47 +-------
 python/pyspark/sql/tests/test_group.py             |  27 -----
 .../main/scala/org/apache/spark/sql/Dataset.scala  |  12 --
 10 files changed, 71 insertions(+), 265 deletions(-)

diff --git 
a/connector/connect/common/src/main/protobuf/spark/connect/expressions.proto 
b/connector/connect/common/src/main/protobuf/spark/connect/expressions.proto
index 782bcc5d1fc..4aac2bcc612 100644
--- a/connector/connect/common/src/main/protobuf/spark/connect/expressions.proto
+++ b/connector/connect/common/src/main/protobuf/spark/connect/expressions.proto
@@ -48,7 +48,6 @@ message Expression {
     CommonInlineUserDefinedFunction common_inline_user_defined_function = 15;
     CallFunction call_function = 16;
     NamedArgumentExpression named_argument_expression = 17;
-    GetColumnByOrdinal get_column_by_ordinal = 18;
 
     // This field is used to mark extensions to the protocol. When plugins 
generate arbitrary
     // relations they can add them here. During the planning the correct 
resolution is done.
@@ -229,15 +228,6 @@ message Expression {
     optional bool is_metadata_column = 3;
   }
 
-  // An unresolved attribute that is represented by its column index.
-  message GetColumnByOrdinal {
-    // (Required) 0-based column index.
-    int32 ordinal = 1;
-
-    // (Optional) The id of corresponding connect plan.
-    optional int64 plan_id = 2;
-  }
-
   // An unresolved function is not explicitly bound to one explicit function, 
but the function
   // is resolved during analysis following Sparks name resolution rules.
   message UnresolvedFunction {
diff --git 
a/connector/connect/server/src/main/scala/org/apache/spark/sql/connect/planner/SparkConnectPlanner.scala
 
b/connector/connect/server/src/main/scala/org/apache/spark/sql/connect/planner/SparkConnectPlanner.scala
index 3dd2be9c54c..52c61c2cbe9 100644
--- 
a/connector/connect/server/src/main/scala/org/apache/spark/sql/connect/planner/SparkConnectPlanner.scala
+++ 
b/connector/connect/server/src/main/scala/org/apache/spark/sql/connect/planner/SparkConnectPlanner.scala
@@ -45,7 +45,7 @@ import org.apache.spark.ml.{functions => MLFunctions}
 import org.apache.spark.sql.{Column, Dataset, Encoders, ForeachWriter, 
RelationalGroupedDataset, SparkSession}
 import org.apache.spark.sql.avro.{AvroDataToCatalyst, CatalystDataToAvro}
 import org.apache.spark.sql.catalyst.{expressions, AliasIdentifier, 
FunctionIdentifier}
-import org.apache.spark.sql.catalyst.analysis.{GetColumnByOrdinal, 
GlobalTempView, LocalTempView, MultiAlias, NameParameterizedQuery, 
PosParameterizedQuery, UnresolvedAlias, UnresolvedAttribute, 
UnresolvedDeserializer, UnresolvedExtractValue, UnresolvedFunction, 
UnresolvedRegex, UnresolvedRelation, UnresolvedStar}
+import org.apache.spark.sql.catalyst.analysis.{GlobalTempView, LocalTempView, 
MultiAlias, NameParameterizedQuery, PosParameterizedQuery, UnresolvedAlias, 
UnresolvedAttribute, UnresolvedDeserializer, UnresolvedExtractValue, 
UnresolvedFunction, UnresolvedRegex, UnresolvedRelation, UnresolvedStar}
 import org.apache.spark.sql.catalyst.encoders.{AgnosticEncoder, 
ExpressionEncoder, RowEncoder}
 import 
org.apache.spark.sql.catalyst.encoders.AgnosticEncoders.UnboundRowEncoder
 import org.apache.spark.sql.catalyst.expressions._
@@ -1353,8 +1353,6 @@ class SparkConnectPlanner(val sessionHolder: 
SessionHolder) extends Logging {
       case proto.Expression.ExprTypeCase.LITERAL => 
transformLiteral(exp.getLiteral)
       case proto.Expression.ExprTypeCase.UNRESOLVED_ATTRIBUTE =>
         transformUnresolvedAttribute(exp.getUnresolvedAttribute)
-      case proto.Expression.ExprTypeCase.GET_COLUMN_BY_ORDINAL =>
-        transformGetColumnByOrdinal(exp.getGetColumnByOrdinal)
       case proto.Expression.ExprTypeCase.UNRESOLVED_FUNCTION =>
         transformUnregisteredFunction(exp.getUnresolvedFunction)
           .getOrElse(transformUnresolvedFunction(exp.getUnresolvedFunction))
@@ -1408,16 +1406,6 @@ class SparkConnectPlanner(val sessionHolder: 
SessionHolder) extends Logging {
     expr
   }
 
-  private def transformGetColumnByOrdinal(
-      attr: proto.Expression.GetColumnByOrdinal): GetColumnByOrdinal = {
-    // always set dataType field null, since it is not used in Analyzer
-    val expr = GetColumnByOrdinal(attr.getOrdinal, null)
-    if (attr.hasPlanId) {
-      expr.setTagValue(LogicalPlan.PLAN_ID_TAG, attr.getPlanId)
-    }
-    expr
-  }
-
   private def transformExpressionPlugin(extension: ProtoAny): Expression = {
     SparkConnectPluginRegistry.expressionRegistry
       // Lazily traverse the collection.
diff --git a/python/pyspark/sql/connect/dataframe.py 
b/python/pyspark/sql/connect/dataframe.py
index b0f229a8a43..e4e6613f997 100644
--- a/python/pyspark/sql/connect/dataframe.py
+++ b/python/pyspark/sql/connect/dataframe.py
@@ -68,10 +68,7 @@ from pyspark.sql.connect.group import GroupedData
 from pyspark.sql.connect.readwriter import DataFrameWriter, DataFrameWriterV2
 from pyspark.sql.connect.streaming.readwriter import DataStreamWriter
 from pyspark.sql.connect.column import Column
-from pyspark.sql.connect.expressions import (
-    UnresolvedRegex,
-    GetColumnByOrdinal,
-)
+from pyspark.sql.connect.expressions import UnresolvedRegex
 from pyspark.sql.connect.functions import (
     _to_col_with_plan_id,
     _to_col,
@@ -1691,10 +1688,10 @@ class DataFrame:
         ...
 
     def __getitem__(self, item: Union[int, str, Column, List, Tuple]) -> 
Union[Column, "DataFrame"]:
-        if self._plan is None:
-            raise SparkConnectException("Cannot analyze on empty plan.")
-
         if isinstance(item, str):
+            if self._plan is None:
+                raise SparkConnectException("Cannot analyze on empty plan.")
+
             # validate the column name
             if not hasattr(self._session, "is_mock_session"):
                 self.select(item).isLocal()
@@ -1708,15 +1705,7 @@ class DataFrame:
         elif isinstance(item, (list, tuple)):
             return self.select(*item)
         elif isinstance(item, int):
-            n = len(self.columns)
-            # 1, convert bool; 2, covert negative index; 3, validate index
-            item = range(0, n)[int(item)]
-            return Column(
-                GetColumnByOrdinal(
-                    ordinal=item,
-                    plan_id=self._plan._plan_id,
-                )
-            )
+            return col(self.columns[item])
         else:
             raise PySparkTypeError(
                 error_class="NOT_COLUMN_OR_INT_OR_LIST_OR_STR_OR_TUPLE",
diff --git a/python/pyspark/sql/connect/expressions.py 
b/python/pyspark/sql/connect/expressions.py
index 797910eba0b..88c4f4d267b 100644
--- a/python/pyspark/sql/connect/expressions.py
+++ b/python/pyspark/sql/connect/expressions.py
@@ -488,40 +488,6 @@ class ColumnReference(Expression):
         )
 
 
-class GetColumnByOrdinal(Expression):
-    """Represents a column index (0-based). There is no guarantee that this 
column
-    actually exists. In the context of this project, we refer by its index and
-    treat it as an unresolved GetColumnByOrdinal"""
-
-    def __init__(self, ordinal: int, plan_id: Optional[int] = None):
-        super().__init__()
-
-        assert isinstance(ordinal, int) and ordinal >= 0
-        self._ordinal = ordinal
-
-        assert plan_id is None or isinstance(plan_id, int)
-        self._plan_id = plan_id
-
-    def to_plan(self, session: "SparkConnectClient") -> proto.Expression:
-        """Returns the Proto representation of the expression."""
-        expr = proto.Expression()
-        expr.get_column_by_ordinal.ordinal = self._ordinal
-        if self._plan_id is not None:
-            expr.get_column_by_ordinal.plan_id = self._plan_id
-        return expr
-
-    def __repr__(self) -> str:
-        return f"getcolumnbyordinal({self._ordinal})"
-
-    def __eq__(self, other: Any) -> bool:
-        return (
-            other is not None
-            and isinstance(other, GetColumnByOrdinal)
-            and other._ordinal == self._ordinal
-            and other._plan_id == self._plan_id
-        )
-
-
 class UnresolvedStar(Expression):
     def __init__(self, unparsed_target: Optional[str]):
         super().__init__()
diff --git a/python/pyspark/sql/connect/proto/expressions_pb2.py 
b/python/pyspark/sql/connect/proto/expressions_pb2.py
index 13f7f2c75de..1e943b8978c 100644
--- a/python/pyspark/sql/connect/proto/expressions_pb2.py
+++ b/python/pyspark/sql/connect/proto/expressions_pb2.py
@@ -33,7 +33,7 @@ from pyspark.sql.connect.proto import types_pb2 as 
spark_dot_connect_dot_types__
 
 
 DESCRIPTOR = _descriptor_pool.Default().AddSerializedFile(
-    
b'\n\x1fspark/connect/expressions.proto\x12\rspark.connect\x1a\x19google/protobuf/any.proto\x1a\x19spark/connect/types.proto"\xc7.\n\nExpression\x12=\n\x07literal\x18\x01
 
\x01(\x0b\x32!.spark.connect.Expression.LiteralH\x00R\x07literal\x12\x62\n\x14unresolved_attribute\x18\x02
 
\x01(\x0b\x32-.spark.connect.Expression.UnresolvedAttributeH\x00R\x13unresolvedAttribute\x12_\n\x13unresolved_function\x18\x03
 
\x01(\x0b\x32,.spark.connect.Expression.UnresolvedFunctionH\x00R\x12unresolvedFunct
 [...]
+    
b'\n\x1fspark/connect/expressions.proto\x12\rspark.connect\x1a\x19google/protobuf/any.proto\x1a\x19spark/connect/types.proto"\x8a-\n\nExpression\x12=\n\x07literal\x18\x01
 
\x01(\x0b\x32!.spark.connect.Expression.LiteralH\x00R\x07literal\x12\x62\n\x14unresolved_attribute\x18\x02
 
\x01(\x0b\x32-.spark.connect.Expression.UnresolvedAttributeH\x00R\x13unresolvedAttribute\x12_\n\x13unresolved_function\x18\x03
 
\x01(\x0b\x32,.spark.connect.Expression.UnresolvedFunctionH\x00R\x12unresolvedFunct
 [...]
 )
 
 _builder.BuildMessageAndEnumDescriptors(DESCRIPTOR, globals())
@@ -44,67 +44,65 @@ if _descriptor._USE_C_DESCRIPTORS == False:
         b"\n\036org.apache.spark.connect.protoP\001Z\022internal/generated"
     )
     _EXPRESSION._serialized_start = 105
-    _EXPRESSION._serialized_end = 6064
-    _EXPRESSION_WINDOW._serialized_start = 1744
-    _EXPRESSION_WINDOW._serialized_end = 2527
-    _EXPRESSION_WINDOW_WINDOWFRAME._serialized_start = 2034
-    _EXPRESSION_WINDOW_WINDOWFRAME._serialized_end = 2527
-    _EXPRESSION_WINDOW_WINDOWFRAME_FRAMEBOUNDARY._serialized_start = 2301
-    _EXPRESSION_WINDOW_WINDOWFRAME_FRAMEBOUNDARY._serialized_end = 2446
-    _EXPRESSION_WINDOW_WINDOWFRAME_FRAMETYPE._serialized_start = 2448
-    _EXPRESSION_WINDOW_WINDOWFRAME_FRAMETYPE._serialized_end = 2527
-    _EXPRESSION_SORTORDER._serialized_start = 2530
-    _EXPRESSION_SORTORDER._serialized_end = 2955
-    _EXPRESSION_SORTORDER_SORTDIRECTION._serialized_start = 2760
-    _EXPRESSION_SORTORDER_SORTDIRECTION._serialized_end = 2868
-    _EXPRESSION_SORTORDER_NULLORDERING._serialized_start = 2870
-    _EXPRESSION_SORTORDER_NULLORDERING._serialized_end = 2955
-    _EXPRESSION_CAST._serialized_start = 2958
-    _EXPRESSION_CAST._serialized_end = 3103
-    _EXPRESSION_LITERAL._serialized_start = 3106
-    _EXPRESSION_LITERAL._serialized_end = 4669
-    _EXPRESSION_LITERAL_DECIMAL._serialized_start = 3941
-    _EXPRESSION_LITERAL_DECIMAL._serialized_end = 4058
-    _EXPRESSION_LITERAL_CALENDARINTERVAL._serialized_start = 4060
-    _EXPRESSION_LITERAL_CALENDARINTERVAL._serialized_end = 4158
-    _EXPRESSION_LITERAL_ARRAY._serialized_start = 4161
-    _EXPRESSION_LITERAL_ARRAY._serialized_end = 4291
-    _EXPRESSION_LITERAL_MAP._serialized_start = 4294
-    _EXPRESSION_LITERAL_MAP._serialized_end = 4521
-    _EXPRESSION_LITERAL_STRUCT._serialized_start = 4524
-    _EXPRESSION_LITERAL_STRUCT._serialized_end = 4653
-    _EXPRESSION_UNRESOLVEDATTRIBUTE._serialized_start = 4672
-    _EXPRESSION_UNRESOLVEDATTRIBUTE._serialized_end = 4858
-    _EXPRESSION_GETCOLUMNBYORDINAL._serialized_start = 4860
-    _EXPRESSION_GETCOLUMNBYORDINAL._serialized_end = 4948
-    _EXPRESSION_UNRESOLVEDFUNCTION._serialized_start = 4951
-    _EXPRESSION_UNRESOLVEDFUNCTION._serialized_end = 5155
-    _EXPRESSION_EXPRESSIONSTRING._serialized_start = 5157
-    _EXPRESSION_EXPRESSIONSTRING._serialized_end = 5207
-    _EXPRESSION_UNRESOLVEDSTAR._serialized_start = 5209
-    _EXPRESSION_UNRESOLVEDSTAR._serialized_end = 5291
-    _EXPRESSION_UNRESOLVEDREGEX._serialized_start = 5293
-    _EXPRESSION_UNRESOLVEDREGEX._serialized_end = 5379
-    _EXPRESSION_UNRESOLVEDEXTRACTVALUE._serialized_start = 5382
-    _EXPRESSION_UNRESOLVEDEXTRACTVALUE._serialized_end = 5514
-    _EXPRESSION_UPDATEFIELDS._serialized_start = 5517
-    _EXPRESSION_UPDATEFIELDS._serialized_end = 5704
-    _EXPRESSION_ALIAS._serialized_start = 5706
-    _EXPRESSION_ALIAS._serialized_end = 5826
-    _EXPRESSION_LAMBDAFUNCTION._serialized_start = 5829
-    _EXPRESSION_LAMBDAFUNCTION._serialized_end = 5987
-    _EXPRESSION_UNRESOLVEDNAMEDLAMBDAVARIABLE._serialized_start = 5989
-    _EXPRESSION_UNRESOLVEDNAMEDLAMBDAVARIABLE._serialized_end = 6051
-    _COMMONINLINEUSERDEFINEDFUNCTION._serialized_start = 6067
-    _COMMONINLINEUSERDEFINEDFUNCTION._serialized_end = 6431
-    _PYTHONUDF._serialized_start = 6434
-    _PYTHONUDF._serialized_end = 6589
-    _SCALARSCALAUDF._serialized_start = 6592
-    _SCALARSCALAUDF._serialized_end = 6776
-    _JAVAUDF._serialized_start = 6779
-    _JAVAUDF._serialized_end = 6928
-    _CALLFUNCTION._serialized_start = 6930
-    _CALLFUNCTION._serialized_end = 7038
-    _NAMEDARGUMENTEXPRESSION._serialized_start = 7040
-    _NAMEDARGUMENTEXPRESSION._serialized_end = 7132
+    _EXPRESSION._serialized_end = 5875
+    _EXPRESSION_WINDOW._serialized_start = 1645
+    _EXPRESSION_WINDOW._serialized_end = 2428
+    _EXPRESSION_WINDOW_WINDOWFRAME._serialized_start = 1935
+    _EXPRESSION_WINDOW_WINDOWFRAME._serialized_end = 2428
+    _EXPRESSION_WINDOW_WINDOWFRAME_FRAMEBOUNDARY._serialized_start = 2202
+    _EXPRESSION_WINDOW_WINDOWFRAME_FRAMEBOUNDARY._serialized_end = 2347
+    _EXPRESSION_WINDOW_WINDOWFRAME_FRAMETYPE._serialized_start = 2349
+    _EXPRESSION_WINDOW_WINDOWFRAME_FRAMETYPE._serialized_end = 2428
+    _EXPRESSION_SORTORDER._serialized_start = 2431
+    _EXPRESSION_SORTORDER._serialized_end = 2856
+    _EXPRESSION_SORTORDER_SORTDIRECTION._serialized_start = 2661
+    _EXPRESSION_SORTORDER_SORTDIRECTION._serialized_end = 2769
+    _EXPRESSION_SORTORDER_NULLORDERING._serialized_start = 2771
+    _EXPRESSION_SORTORDER_NULLORDERING._serialized_end = 2856
+    _EXPRESSION_CAST._serialized_start = 2859
+    _EXPRESSION_CAST._serialized_end = 3004
+    _EXPRESSION_LITERAL._serialized_start = 3007
+    _EXPRESSION_LITERAL._serialized_end = 4570
+    _EXPRESSION_LITERAL_DECIMAL._serialized_start = 3842
+    _EXPRESSION_LITERAL_DECIMAL._serialized_end = 3959
+    _EXPRESSION_LITERAL_CALENDARINTERVAL._serialized_start = 3961
+    _EXPRESSION_LITERAL_CALENDARINTERVAL._serialized_end = 4059
+    _EXPRESSION_LITERAL_ARRAY._serialized_start = 4062
+    _EXPRESSION_LITERAL_ARRAY._serialized_end = 4192
+    _EXPRESSION_LITERAL_MAP._serialized_start = 4195
+    _EXPRESSION_LITERAL_MAP._serialized_end = 4422
+    _EXPRESSION_LITERAL_STRUCT._serialized_start = 4425
+    _EXPRESSION_LITERAL_STRUCT._serialized_end = 4554
+    _EXPRESSION_UNRESOLVEDATTRIBUTE._serialized_start = 4573
+    _EXPRESSION_UNRESOLVEDATTRIBUTE._serialized_end = 4759
+    _EXPRESSION_UNRESOLVEDFUNCTION._serialized_start = 4762
+    _EXPRESSION_UNRESOLVEDFUNCTION._serialized_end = 4966
+    _EXPRESSION_EXPRESSIONSTRING._serialized_start = 4968
+    _EXPRESSION_EXPRESSIONSTRING._serialized_end = 5018
+    _EXPRESSION_UNRESOLVEDSTAR._serialized_start = 5020
+    _EXPRESSION_UNRESOLVEDSTAR._serialized_end = 5102
+    _EXPRESSION_UNRESOLVEDREGEX._serialized_start = 5104
+    _EXPRESSION_UNRESOLVEDREGEX._serialized_end = 5190
+    _EXPRESSION_UNRESOLVEDEXTRACTVALUE._serialized_start = 5193
+    _EXPRESSION_UNRESOLVEDEXTRACTVALUE._serialized_end = 5325
+    _EXPRESSION_UPDATEFIELDS._serialized_start = 5328
+    _EXPRESSION_UPDATEFIELDS._serialized_end = 5515
+    _EXPRESSION_ALIAS._serialized_start = 5517
+    _EXPRESSION_ALIAS._serialized_end = 5637
+    _EXPRESSION_LAMBDAFUNCTION._serialized_start = 5640
+    _EXPRESSION_LAMBDAFUNCTION._serialized_end = 5798
+    _EXPRESSION_UNRESOLVEDNAMEDLAMBDAVARIABLE._serialized_start = 5800
+    _EXPRESSION_UNRESOLVEDNAMEDLAMBDAVARIABLE._serialized_end = 5862
+    _COMMONINLINEUSERDEFINEDFUNCTION._serialized_start = 5878
+    _COMMONINLINEUSERDEFINEDFUNCTION._serialized_end = 6242
+    _PYTHONUDF._serialized_start = 6245
+    _PYTHONUDF._serialized_end = 6400
+    _SCALARSCALAUDF._serialized_start = 6403
+    _SCALARSCALAUDF._serialized_end = 6587
+    _JAVAUDF._serialized_start = 6590
+    _JAVAUDF._serialized_end = 6739
+    _CALLFUNCTION._serialized_start = 6741
+    _CALLFUNCTION._serialized_end = 6849
+    _NAMEDARGUMENTEXPRESSION._serialized_start = 6851
+    _NAMEDARGUMENTEXPRESSION._serialized_end = 6943
 # @@protoc_insertion_point(module_scope)
diff --git a/python/pyspark/sql/connect/proto/expressions_pb2.pyi 
b/python/pyspark/sql/connect/proto/expressions_pb2.pyi
index ba24da263cf..93a431dcc86 100644
--- a/python/pyspark/sql/connect/proto/expressions_pb2.pyi
+++ b/python/pyspark/sql/connect/proto/expressions_pb2.pyi
@@ -807,37 +807,6 @@ class Expression(google.protobuf.message.Message):
             self, oneof_group: typing_extensions.Literal["_plan_id", 
b"_plan_id"]
         ) -> typing_extensions.Literal["plan_id"] | None: ...
 
-    class GetColumnByOrdinal(google.protobuf.message.Message):
-        """An unresolved attribute that is represented by its column index."""
-
-        DESCRIPTOR: google.protobuf.descriptor.Descriptor
-
-        ORDINAL_FIELD_NUMBER: builtins.int
-        PLAN_ID_FIELD_NUMBER: builtins.int
-        ordinal: builtins.int
-        """(Required) 0-based column index."""
-        plan_id: builtins.int
-        """(Optional) The id of corresponding connect plan."""
-        def __init__(
-            self,
-            *,
-            ordinal: builtins.int = ...,
-            plan_id: builtins.int | None = ...,
-        ) -> None: ...
-        def HasField(
-            self,
-            field_name: typing_extensions.Literal["_plan_id", b"_plan_id", 
"plan_id", b"plan_id"],
-        ) -> builtins.bool: ...
-        def ClearField(
-            self,
-            field_name: typing_extensions.Literal[
-                "_plan_id", b"_plan_id", "ordinal", b"ordinal", "plan_id", 
b"plan_id"
-            ],
-        ) -> None: ...
-        def WhichOneof(
-            self, oneof_group: typing_extensions.Literal["_plan_id", 
b"_plan_id"]
-        ) -> typing_extensions.Literal["plan_id"] | None: ...
-
     class UnresolvedFunction(google.protobuf.message.Message):
         """An unresolved function is not explicitly bound to one explicit 
function, but the function
         is resolved during analysis following Sparks name resolution rules.
@@ -1160,7 +1129,6 @@ class Expression(google.protobuf.message.Message):
     COMMON_INLINE_USER_DEFINED_FUNCTION_FIELD_NUMBER: builtins.int
     CALL_FUNCTION_FIELD_NUMBER: builtins.int
     NAMED_ARGUMENT_EXPRESSION_FIELD_NUMBER: builtins.int
-    GET_COLUMN_BY_ORDINAL_FIELD_NUMBER: builtins.int
     EXTENSION_FIELD_NUMBER: builtins.int
     @property
     def literal(self) -> global___Expression.Literal: ...
@@ -1199,8 +1167,6 @@ class Expression(google.protobuf.message.Message):
     @property
     def named_argument_expression(self) -> global___NamedArgumentExpression: 
...
     @property
-    def get_column_by_ordinal(self) -> global___Expression.GetColumnByOrdinal: 
...
-    @property
     def extension(self) -> google.protobuf.any_pb2.Any:
         """This field is used to mark extensions to the protocol. When plugins 
generate arbitrary
         relations they can add them here. During the planning the correct 
resolution is done.
@@ -1226,7 +1192,6 @@ class Expression(google.protobuf.message.Message):
         common_inline_user_defined_function: 
global___CommonInlineUserDefinedFunction | None = ...,
         call_function: global___CallFunction | None = ...,
         named_argument_expression: global___NamedArgumentExpression | None = 
...,
-        get_column_by_ordinal: global___Expression.GetColumnByOrdinal | None = 
...,
         extension: google.protobuf.any_pb2.Any | None = ...,
     ) -> None: ...
     def HasField(
@@ -1246,8 +1211,6 @@ class Expression(google.protobuf.message.Message):
             b"expression_string",
             "extension",
             b"extension",
-            "get_column_by_ordinal",
-            b"get_column_by_ordinal",
             "lambda_function",
             b"lambda_function",
             "literal",
@@ -1291,8 +1254,6 @@ class Expression(google.protobuf.message.Message):
             b"expression_string",
             "extension",
             b"extension",
-            "get_column_by_ordinal",
-            b"get_column_by_ordinal",
             "lambda_function",
             b"lambda_function",
             "literal",
@@ -1340,7 +1301,6 @@ class Expression(google.protobuf.message.Message):
             "common_inline_user_defined_function",
             "call_function",
             "named_argument_expression",
-            "get_column_by_ordinal",
             "extension",
         ]
         | None
diff --git a/python/pyspark/sql/dataframe.py b/python/pyspark/sql/dataframe.py
index 51f18e7b6f3..07a672d4063 100644
--- a/python/pyspark/sql/dataframe.py
+++ b/python/pyspark/sql/dataframe.py
@@ -3458,10 +3458,7 @@ class DataFrame(PandasMapOpsMixin, 
PandasConversionMixin):
         elif isinstance(item, (list, tuple)):
             return self.select(*item)
         elif isinstance(item, int):
-            n = len(self.columns)
-            # 1, convert bool; 2, covert negative index; 3, validate index
-            item = range(0, n)[int(item)]
-            jc = self._jdf.apply(item)
+            jc = self._jdf.apply(self.columns[item])
             return Column(jc)
         else:
             raise PySparkTypeError(
diff --git a/python/pyspark/sql/tests/test_dataframe.py 
b/python/pyspark/sql/tests/test_dataframe.py
index 150130f3c1a..3c493a8ae3a 100644
--- a/python/pyspark/sql/tests/test_dataframe.py
+++ b/python/pyspark/sql/tests/test_dataframe.py
@@ -26,6 +26,7 @@ from typing import cast
 import io
 from contextlib import redirect_stdout
 
+from pyspark import StorageLevel
 from pyspark.sql import SparkSession, Row, functions
 from pyspark.sql.functions import col, lit, count, sum, mean, struct
 from pyspark.sql.types import (
@@ -61,51 +62,6 @@ from pyspark.testing.utils import QuietTest
 
 
 class DataFrameTestsMixin:
-    def test_getitem_invalid_indices(self):
-        df = self.spark.sql(
-            "SELECT * FROM VALUES "
-            "(1, 1.1, 'a'), "
-            "(2, 2.2, 'b'), "
-            "(4, 4.4, 'c') "
-            "AS TAB(a, b, c)"
-        )
-
-        # accepted type and values
-        for index in [False, True, 0, 1, 2, -1, -2, -3]:
-            df[index]
-
-        # negative cases: ordinal out of range
-        for index in [-10, -4, 3, 10, 100]:
-            with self.assertRaises(IndexError):
-                df[index]
-
-        # negative cases: unsupported types
-        for index in [None, 1.0, Decimal(1)]:
-            with self.assertRaises(PySparkTypeError):
-                df[index]
-
-    def test_getitem_duplicated_column(self):
-        df = self.spark.sql(
-            "SELECT * FROM VALUES "
-            "(1, 1.1, 'a'), "
-            "(2, 2.2, 'b'), "
-            "(4, 4.4, 'c') "
-            "AS TAB(a, a, a)"
-        )
-
-        self.assertEqual(
-            df.select(df[0]).schema.simpleString(),
-            "struct<a:int>",
-        )
-        self.assertEqual(
-            df.select(df[1]).schema.simpleString(),
-            "struct<a:decimal(2,1)>",
-        )
-        self.assertEqual(
-            df.select(df[2]).schema.simpleString(),
-            "struct<a:string>",
-        )
-
     def test_range(self):
         self.assertEqual(self.spark.range(1, 1).count(), 0)
         self.assertEqual(self.spark.range(1, 0, -1).count(), 1)
@@ -120,6 +76,7 @@ class DataFrameTestsMixin:
         self.assertEqual(2, row[1])
         self.assertEqual("Row(c=1, c=2)", str(row))
         # Cannot access columns
+        self.assertRaises(AnalysisException, lambda: df.select(df[0]).first())
         self.assertRaises(AnalysisException, lambda: df.select(df.c).first())
         self.assertRaises(AnalysisException, lambda: 
df.select(df["c"]).first())
 
diff --git a/python/pyspark/sql/tests/test_group.py 
b/python/pyspark/sql/tests/test_group.py
index 85d25357741..6981601cb12 100644
--- a/python/pyspark/sql/tests/test_group.py
+++ b/python/pyspark/sql/tests/test_group.py
@@ -149,33 +149,6 @@ class GroupTestsMixin:
             with self.assertRaises(IndexError):
                 df.orderBy(-3)
 
-    def test_order_by_ordinal_duplicated_column(self):
-        spark = self.spark
-        df = spark.createDataFrame(
-            [
-                (1, 1),
-                (1, 2),
-                (2, 1),
-                (2, 2),
-                (3, 1),
-                (3, 2),
-            ],
-            ["a", "a"],
-        )
-
-        with self.tempView("v"):
-            df.createOrReplaceTempView("v")
-
-            df1 = spark.sql("select * from v order by 2, 1;")
-            df2 = df.orderBy(2, 1)
-            assertSchemaEqual(df1.schema, df2.schema)
-            assertDataFrameEqual(df1, df2)
-
-            df1 = spark.sql("select * from v order by 1 desc, 2;")
-            df2 = df.orderBy(-1, 2)
-            assertSchemaEqual(df1.schema, df2.schema)
-            assertDataFrameEqual(df1, df2)
-
 
 class GroupTests(GroupTestsMixin, ReusedSQLTestCase):
     pass
diff --git a/sql/core/src/main/scala/org/apache/spark/sql/Dataset.scala 
b/sql/core/src/main/scala/org/apache/spark/sql/Dataset.scala
index a3f8a2cf4a0..0cc037b157e 100644
--- a/sql/core/src/main/scala/org/apache/spark/sql/Dataset.scala
+++ b/sql/core/src/main/scala/org/apache/spark/sql/Dataset.scala
@@ -1393,11 +1393,6 @@ class Dataset[T] private[sql](
    */
   def apply(colName: String): Column = col(colName)
 
-  /**
-   * Selects column based on the column index (0-based) and returns it as a 
[[Column]].
-   */
-  private[sql] def apply(index: Int): Column = col(index)
-
   /**
    * Specifies some hint on the current Dataset. As an example, the following 
code specifies
    * that one of the plan can be broadcasted:
@@ -1450,13 +1445,6 @@ class Dataset[T] private[sql](
       }
   }
 
-  /**
-   * Selects column based on the column index (0-based) and returns it as a 
[[Column]].
-   */
-  private[sql] def col(index: Int): Column = {
-    Column(addDataFrameIdToCol(queryExecution.analyzed.output(index)))
-  }
-
   /**
    * Selects a metadata column based on its logical column name, and returns 
it as a [[Column]].
    *


---------------------------------------------------------------------
To unsubscribe, e-mail: [email protected]
For additional commands, e-mail: [email protected]

Reply via email to