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

yangjie01 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 f72b87b90be [SPARK-45515][CORE][SQL] Use enhanced `switch` expressions 
to replace the regular `switch` statement
f72b87b90be is described below

commit f72b87b90bea137050e3e2edceaf962eb7924f13
Author: yangjie01 <[email protected]>
AuthorDate: Fri Oct 13 08:49:21 2023 +0800

    [SPARK-45515][CORE][SQL] Use enhanced `switch` expressions to replace the 
regular `switch` statement
    
    ### What changes were proposed in this pull request?
    This pr use enhanced  `switch` Expressions to replace the regular `switch` 
statement in Spark Java code refer to [JEP 361](https://openjdk.org/jeps/361)
    
    Example:
    
    ```java
    double getPrice(String fruit) {
      switch (fruit) {
        case "Apple":
          return 1.0;
        case "Orange":
          return 1.5;
        case "Mango":
          return 2.0;
        default:
          throw new IllegalArgumentException();
       }
     }
    ```
    
    Can be changed to
    
    ```java
    double getPrice(String fruit) {
      return switch (fruit) {
        case "Apple" -> 1.0;
        case "Orange" -> 1.5;
        case "Mango" -> 2.0;
        default -> throw new IllegalArgumentException();
      };
    }
    ```
    
    This pr does not include parts of the `hive-thriftserver` module.
    
    ### Why are the changes needed?
    Using `JEP 361: Switch Expressions` can bring the following benefits:
    
    1. **More concise syntax**: `switch` can be used as an expression, not just 
a statement. This makes the code more concise and easier to read.
    
    2. **Safer**: In `switch` expressions, if we forget the `break`, there will 
be no unexpected `fall-through` behavior. At the same time, the compiler will 
check whether all possible cases are covered. If not all cases are covered, the 
compiler will report an error.
    
    3. **Easier to understand**: The new `switch` expression syntax is closer 
to our decision-making pattern in daily life, making the code easier to 
understand.
    
    ### Does this PR introduce _any_ user-facing change?
    No
    
    ### How was this patch tested?
    Pass GitHub Actions
    
    ### Was this patch authored or co-authored using generative AI tooling?
    No
    
    Closes #43349 from LuciferYang/jep-361.
    
    Authored-by: yangjie01 <[email protected]>
    Signed-off-by: yangjie01 <[email protected]>
---
 .../org/apache/spark/network/protocol/Message.java | 34 ++++++-------
 .../spark/network/protocol/MessageDecoder.java     | 59 ++++++----------------
 .../network/server/BlockPushNonFatalFailure.java   | 16 +++---
 .../org/apache/spark/network/util/DBProvider.java  | 13 +++--
 .../org/apache/spark/network/util/NettyUtils.java  | 36 +++++--------
 .../apache/spark/network/RpcIntegrationSuite.java  | 18 +++----
 .../org/apache/spark/network/StreamTestHelper.java | 24 ++++-----
 .../shuffle/protocol/BlockTransferMessage.java     | 42 +++++++--------
 .../network/shuffle/ExternalBlockHandlerSuite.java | 34 ++++++-------
 .../shuffle/ExternalShuffleIntegrationSuite.java   | 10 ++--
 .../network/yarn/YarnShuffleServiceMetrics.java    | 17 ++-----
 .../apache/spark/unsafe/UnsafeAlignedOffset.java   | 12 ++---
 .../catalyst/expressions/ExpressionImplUtils.java  | 23 ++++-----
 .../sql/connector/expressions/NullOrdering.java    | 12 ++---
 .../sql/connector/expressions/SortDirection.java   | 12 ++---
 .../sql/connector/util/V2ExpressionSQLBuilder.java |  8 ++-
 .../datasources/parquet/ParquetColumnVector.java   | 12 ++---
 .../parquet/VectorizedRleValuesReader.java         | 11 ++--
 18 files changed, 155 insertions(+), 238 deletions(-)

diff --git 
a/common/network-common/src/main/java/org/apache/spark/network/protocol/Message.java
 
b/common/network-common/src/main/java/org/apache/spark/network/protocol/Message.java
index 12ebee8da96..0bcce788ec4 100644
--- 
a/common/network-common/src/main/java/org/apache/spark/network/protocol/Message.java
+++ 
b/common/network-common/src/main/java/org/apache/spark/network/protocol/Message.java
@@ -55,23 +55,23 @@ public interface Message extends Encodable {
 
     public static Type decode(ByteBuf buf) {
       byte id = buf.readByte();
-      switch (id) {
-        case 0: return ChunkFetchRequest;
-        case 1: return ChunkFetchSuccess;
-        case 2: return ChunkFetchFailure;
-        case 3: return RpcRequest;
-        case 4: return RpcResponse;
-        case 5: return RpcFailure;
-        case 6: return StreamRequest;
-        case 7: return StreamResponse;
-        case 8: return StreamFailure;
-        case 9: return OneWayMessage;
-        case 10: return UploadStream;
-        case 11: return MergedBlockMetaRequest;
-        case 12: return MergedBlockMetaSuccess;
-        case -1: throw new IllegalArgumentException("User type messages cannot 
be decoded.");
-        default: throw new IllegalArgumentException("Unknown message type: " + 
id);
-      }
+      return switch (id) {
+        case 0 -> ChunkFetchRequest;
+        case 1 -> ChunkFetchSuccess;
+        case 2 -> ChunkFetchFailure;
+        case 3 -> RpcRequest;
+        case 4 -> RpcResponse;
+        case 5 -> RpcFailure;
+        case 6 -> StreamRequest;
+        case 7 -> StreamResponse;
+        case 8 -> StreamFailure;
+        case 9 -> OneWayMessage;
+        case 10 -> UploadStream;
+        case 11 -> MergedBlockMetaRequest;
+        case 12 -> MergedBlockMetaSuccess;
+        case -1 -> throw new IllegalArgumentException("User type messages 
cannot be decoded.");
+        default -> throw new IllegalArgumentException("Unknown message type: " 
+ id);
+      };
     }
   }
 }
diff --git 
a/common/network-common/src/main/java/org/apache/spark/network/protocol/MessageDecoder.java
 
b/common/network-common/src/main/java/org/apache/spark/network/protocol/MessageDecoder.java
index 98f7f612a48..29369f6c206 100644
--- 
a/common/network-common/src/main/java/org/apache/spark/network/protocol/MessageDecoder.java
+++ 
b/common/network-common/src/main/java/org/apache/spark/network/protocol/MessageDecoder.java
@@ -49,48 +49,21 @@ public final class MessageDecoder extends 
MessageToMessageDecoder<ByteBuf> {
   }
 
   private Message decode(Message.Type msgType, ByteBuf in) {
-    switch (msgType) {
-      case ChunkFetchRequest:
-        return ChunkFetchRequest.decode(in);
-
-      case ChunkFetchSuccess:
-        return ChunkFetchSuccess.decode(in);
-
-      case ChunkFetchFailure:
-        return ChunkFetchFailure.decode(in);
-
-      case RpcRequest:
-        return RpcRequest.decode(in);
-
-      case RpcResponse:
-        return RpcResponse.decode(in);
-
-      case RpcFailure:
-        return RpcFailure.decode(in);
-
-      case OneWayMessage:
-        return OneWayMessage.decode(in);
-
-      case StreamRequest:
-        return StreamRequest.decode(in);
-
-      case StreamResponse:
-        return StreamResponse.decode(in);
-
-      case StreamFailure:
-        return StreamFailure.decode(in);
-
-      case UploadStream:
-        return UploadStream.decode(in);
-
-      case MergedBlockMetaRequest:
-        return MergedBlockMetaRequest.decode(in);
-
-      case MergedBlockMetaSuccess:
-        return MergedBlockMetaSuccess.decode(in);
-
-      default:
-        throw new IllegalArgumentException("Unexpected message type: " + 
msgType);
-    }
+    return switch (msgType) {
+      case ChunkFetchRequest -> ChunkFetchRequest.decode(in);
+      case ChunkFetchSuccess -> ChunkFetchSuccess.decode(in);
+      case ChunkFetchFailure -> ChunkFetchFailure.decode(in);
+      case RpcRequest -> RpcRequest.decode(in);
+      case RpcResponse -> RpcResponse.decode(in);
+      case RpcFailure -> RpcFailure.decode(in);
+      case OneWayMessage -> OneWayMessage.decode(in);
+      case StreamRequest -> StreamRequest.decode(in);
+      case StreamResponse -> StreamResponse.decode(in);
+      case StreamFailure -> StreamFailure.decode(in);
+      case UploadStream -> UploadStream.decode(in);
+      case MergedBlockMetaRequest -> MergedBlockMetaRequest.decode(in);
+      case MergedBlockMetaSuccess -> MergedBlockMetaSuccess.decode(in);
+      default -> throw new IllegalArgumentException("Unexpected message type: 
" + msgType);
+    };
   }
 }
diff --git 
a/common/network-common/src/main/java/org/apache/spark/network/server/BlockPushNonFatalFailure.java
 
b/common/network-common/src/main/java/org/apache/spark/network/server/BlockPushNonFatalFailure.java
index 4bb22b2e075..f60a74670d1 100644
--- 
a/common/network-common/src/main/java/org/apache/spark/network/server/BlockPushNonFatalFailure.java
+++ 
b/common/network-common/src/main/java/org/apache/spark/network/server/BlockPushNonFatalFailure.java
@@ -154,14 +154,14 @@ public class BlockPushNonFatalFailure extends 
RuntimeException {
   }
 
   public static ReturnCode getReturnCode(byte id) {
-    switch (id) {
-      case 0: return ReturnCode.SUCCESS;
-      case 1: return ReturnCode.TOO_LATE_BLOCK_PUSH;
-      case 2: return ReturnCode.BLOCK_APPEND_COLLISION_DETECTED;
-      case 3: return ReturnCode.STALE_BLOCK_PUSH;
-      case 4: return ReturnCode.TOO_OLD_ATTEMPT_PUSH;
-      default: throw new IllegalArgumentException("Unknown block push return 
code: " + id);
-    }
+    return switch (id) {
+      case 0 -> ReturnCode.SUCCESS;
+      case 1 -> ReturnCode.TOO_LATE_BLOCK_PUSH;
+      case 2 -> ReturnCode.BLOCK_APPEND_COLLISION_DETECTED;
+      case 3 -> ReturnCode.STALE_BLOCK_PUSH;
+      case 4 -> ReturnCode.TOO_OLD_ATTEMPT_PUSH;
+      default -> throw new IllegalArgumentException("Unknown block push return 
code: " + id);
+    };
   }
 
   public static boolean shouldNotRetryErrorCode(ReturnCode returnCode) {
diff --git 
a/common/network-common/src/main/java/org/apache/spark/network/util/DBProvider.java
 
b/common/network-common/src/main/java/org/apache/spark/network/util/DBProvider.java
index 6d77608f7ac..1adb9cfe5d3 100644
--- 
a/common/network-common/src/main/java/org/apache/spark/network/util/DBProvider.java
+++ 
b/common/network-common/src/main/java/org/apache/spark/network/util/DBProvider.java
@@ -56,14 +56,13 @@ public class DBProvider {
     @VisibleForTesting
     public static DB initDB(DBBackend dbBackend, File file) throws IOException 
{
       if (file != null) {
-        switch (dbBackend) {
-          case LEVELDB:
+        return switch (dbBackend) {
+          case LEVELDB -> {
             logger.warn("The LEVELDB is deprecated. Please use ROCKSDB 
instead.");
-            return new LevelDB(LevelDBProvider.initLevelDB(file));
-          case ROCKSDB: return new RocksDB(RocksDBProvider.initRocksDB(file));
-          default:
-            throw new IllegalArgumentException("Unsupported DBBackend: " + 
dbBackend);
-        }
+            yield new LevelDB(LevelDBProvider.initLevelDB(file));
+          }
+          case ROCKSDB -> new RocksDB(RocksDBProvider.initRocksDB(file));
+        };
       }
       return null;
     }
diff --git 
a/common/network-common/src/main/java/org/apache/spark/network/util/NettyUtils.java
 
b/common/network-common/src/main/java/org/apache/spark/network/util/NettyUtils.java
index d8f720e98e3..2dd1c8f2e4a 100644
--- 
a/common/network-common/src/main/java/org/apache/spark/network/util/NettyUtils.java
+++ 
b/common/network-common/src/main/java/org/apache/spark/network/util/NettyUtils.java
@@ -65,38 +65,26 @@ public class NettyUtils {
   public static EventLoopGroup createEventLoop(IOMode mode, int numThreads, 
String threadPrefix) {
     ThreadFactory threadFactory = createThreadFactory(threadPrefix);
 
-    switch (mode) {
-      case NIO:
-        return new NioEventLoopGroup(numThreads, threadFactory);
-      case EPOLL:
-        return new EpollEventLoopGroup(numThreads, threadFactory);
-      default:
-        throw new IllegalArgumentException("Unknown io mode: " + mode);
-    }
+    return switch (mode) {
+      case NIO -> new NioEventLoopGroup(numThreads, threadFactory);
+      case EPOLL -> new EpollEventLoopGroup(numThreads, threadFactory);
+    };
   }
 
   /** Returns the correct (client) SocketChannel class based on IOMode. */
   public static Class<? extends Channel> getClientChannelClass(IOMode mode) {
-    switch (mode) {
-      case NIO:
-        return NioSocketChannel.class;
-      case EPOLL:
-        return EpollSocketChannel.class;
-      default:
-        throw new IllegalArgumentException("Unknown io mode: " + mode);
-    }
+    return switch (mode) {
+      case NIO -> NioSocketChannel.class;
+      case EPOLL -> EpollSocketChannel.class;
+    };
   }
 
   /** Returns the correct ServerSocketChannel class based on IOMode. */
   public static Class<? extends ServerChannel> getServerChannelClass(IOMode 
mode) {
-    switch (mode) {
-      case NIO:
-        return NioServerSocketChannel.class;
-      case EPOLL:
-        return EpollServerSocketChannel.class;
-      default:
-        throw new IllegalArgumentException("Unknown io mode: " + mode);
-    }
+    return switch (mode) {
+      case NIO -> NioServerSocketChannel.class;
+      case EPOLL -> EpollServerSocketChannel.class;
+    };
   }
 
   /**
diff --git 
a/common/network-common/src/test/java/org/apache/spark/network/RpcIntegrationSuite.java
 
b/common/network-common/src/test/java/org/apache/spark/network/RpcIntegrationSuite.java
index 61f222c7275..a7a61588ef1 100644
--- 
a/common/network-common/src/test/java/org/apache/spark/network/RpcIntegrationSuite.java
+++ 
b/common/network-common/src/test/java/org/apache/spark/network/RpcIntegrationSuite.java
@@ -104,9 +104,9 @@ public class RpcIntegrationSuite {
     try {
       if (msg.startsWith("fail/")) {
         String[] parts = msg.split("/");
-        switch (parts[1]) {
-          case "exception-ondata":
-            return new StreamCallbackWithID() {
+        return switch (parts[1]) {
+          case "exception-ondata" ->
+            new StreamCallbackWithID() {
               @Override
               public void onData(String streamId, ByteBuffer buf) throws 
IOException {
                 throw new IOException("failed to read stream data!");
@@ -125,8 +125,8 @@ public class RpcIntegrationSuite {
                 return msg;
               }
             };
-          case "exception-oncomplete":
-            return new StreamCallbackWithID() {
+          case "exception-oncomplete" ->
+            new StreamCallbackWithID() {
               @Override
               public void onData(String streamId, ByteBuffer buf) throws 
IOException {
               }
@@ -145,11 +145,9 @@ public class RpcIntegrationSuite {
                 return msg;
               }
             };
-          case "null":
-            return null;
-          default:
-            throw new IllegalArgumentException("unexpected msg: " + msg);
-        }
+          case "null" -> null;
+          default -> throw new IllegalArgumentException("unexpected msg: " + 
msg);
+        };
       } else {
         VerifyingStreamCallback streamCallback = new 
VerifyingStreamCallback(msg);
         streamCallbacks.put(msg, streamCallback);
diff --git 
a/common/network-common/src/test/java/org/apache/spark/network/StreamTestHelper.java
 
b/common/network-common/src/test/java/org/apache/spark/network/StreamTestHelper.java
index da83e549d1c..7d5db149d4e 100644
--- 
a/common/network-common/src/test/java/org/apache/spark/network/StreamTestHelper.java
+++ 
b/common/network-common/src/test/java/org/apache/spark/network/StreamTestHelper.java
@@ -66,25 +66,19 @@ class StreamTestHelper {
   }
 
   public ByteBuffer srcBuffer(String name) {
-    switch (name) {
-      case "largeBuffer":
-        return largeBuffer;
-      case "smallBuffer":
-        return smallBuffer;
-      case "emptyBuffer":
-        return emptyBuffer;
-      default:
-        throw new IllegalArgumentException("Invalid stream: " + name);
-    }
+    return switch (name) {
+      case "largeBuffer" -> largeBuffer;
+      case "smallBuffer" -> smallBuffer;
+      case "emptyBuffer" -> emptyBuffer;
+      default -> throw new IllegalArgumentException("Invalid stream: " + name);
+    };
   }
 
   public ManagedBuffer openStream(TransportConf conf, String streamId) {
-    switch (streamId) {
-      case "file":
-        return new FileSegmentManagedBuffer(conf, testFile, 0, 
testFile.length());
-      default:
-        return new NioManagedBuffer(srcBuffer(streamId));
+    if ("file".equals(streamId)) {
+      return new FileSegmentManagedBuffer(conf, testFile, 0, 
testFile.length());
     }
+    return new NioManagedBuffer(srcBuffer(streamId));
   }
 
   void cleanup() {
diff --git 
a/common/network-shuffle/src/main/java/org/apache/spark/network/shuffle/protocol/BlockTransferMessage.java
 
b/common/network-shuffle/src/main/java/org/apache/spark/network/shuffle/protocol/BlockTransferMessage.java
index 80143c591d9..be057104a1c 100644
--- 
a/common/network-shuffle/src/main/java/org/apache/spark/network/shuffle/protocol/BlockTransferMessage.java
+++ 
b/common/network-shuffle/src/main/java/org/apache/spark/network/shuffle/protocol/BlockTransferMessage.java
@@ -66,27 +66,27 @@ public abstract class BlockTransferMessage implements 
Encodable {
     public static BlockTransferMessage fromByteBuffer(ByteBuffer msg) {
       ByteBuf buf = Unpooled.wrappedBuffer(msg);
       byte type = buf.readByte();
-      switch (type) {
-        case 0: return OpenBlocks.decode(buf);
-        case 1: return UploadBlock.decode(buf);
-        case 2: return RegisterExecutor.decode(buf);
-        case 3: return StreamHandle.decode(buf);
-        case 6: return UploadBlockStream.decode(buf);
-        case 7: return RemoveBlocks.decode(buf);
-        case 8: return BlocksRemoved.decode(buf);
-        case 9: return FetchShuffleBlocks.decode(buf);
-        case 10: return GetLocalDirsForExecutors.decode(buf);
-        case 11: return LocalDirsForExecutors.decode(buf);
-        case 12: return PushBlockStream.decode(buf);
-        case 13: return FinalizeShuffleMerge.decode(buf);
-        case 14: return MergeStatuses.decode(buf);
-        case 15: return FetchShuffleBlockChunks.decode(buf);
-        case 16: return DiagnoseCorruption.decode(buf);
-        case 17: return CorruptionCause.decode(buf);
-        case 18: return BlockPushReturnCode.decode(buf);
-        case 19: return RemoveShuffleMerge.decode(buf);
-        default: throw new IllegalArgumentException("Unknown message type: " + 
type);
-      }
+      return switch (type) {
+        case 0 -> OpenBlocks.decode(buf);
+        case 1 -> UploadBlock.decode(buf);
+        case 2 -> RegisterExecutor.decode(buf);
+        case 3 -> StreamHandle.decode(buf);
+        case 6 -> UploadBlockStream.decode(buf);
+        case 7 -> RemoveBlocks.decode(buf);
+        case 8 -> BlocksRemoved.decode(buf);
+        case 9 -> FetchShuffleBlocks.decode(buf);
+        case 10 -> GetLocalDirsForExecutors.decode(buf);
+        case 11 -> LocalDirsForExecutors.decode(buf);
+        case 12 -> PushBlockStream.decode(buf);
+        case 13 -> FinalizeShuffleMerge.decode(buf);
+        case 14 -> MergeStatuses.decode(buf);
+        case 15 -> FetchShuffleBlockChunks.decode(buf);
+        case 16 -> DiagnoseCorruption.decode(buf);
+        case 17 -> CorruptionCause.decode(buf);
+        case 18 -> BlockPushReturnCode.decode(buf);
+        case 19 -> RemoveShuffleMerge.decode(buf);
+        default -> throw new IllegalArgumentException("Unknown message type: " 
+ type);
+      };
     }
   }
 
diff --git 
a/common/network-shuffle/src/test/java/org/apache/spark/network/shuffle/ExternalBlockHandlerSuite.java
 
b/common/network-shuffle/src/test/java/org/apache/spark/network/shuffle/ExternalBlockHandlerSuite.java
index 21e2eee76fa..3609aa556de 100644
--- 
a/common/network-shuffle/src/test/java/org/apache/spark/network/shuffle/ExternalBlockHandlerSuite.java
+++ 
b/common/network-shuffle/src/test/java/org/apache/spark/network/shuffle/ExternalBlockHandlerSuite.java
@@ -139,29 +139,27 @@ public class ExternalBlockHandlerSuite {
       ByteStreams.readFully(checkedIn, buffer, 0, (int) 
blockMarkers[0].size());
       long checksumByWriter = checkedIn.getChecksum().getValue();
 
-      switch (expectedCaused) {
+      // when checksumByWriter == checksumRecalculated and checksumByReader != 
checksumByWriter
+      checksumByReader = switch (expectedCaused) {
         // when checksumByWriter != checksumRecalculated
-        case DISK_ISSUE:
+        case DISK_ISSUE -> {
           out.writeLong(checksumByWriter - 1);
-          checksumByReader = checksumByWriter;
-          break;
-
-        // when checksumByWriter == checksumRecalculated and checksumByReader 
!= checksumByWriter
-        case NETWORK_ISSUE:
+          yield checksumByWriter;
+        }
+        case NETWORK_ISSUE -> {
           out.writeLong(checksumByWriter);
-          checksumByReader = checksumByWriter - 1;
-          break;
-
-        case UNKNOWN_ISSUE:
-          // write a int instead of a long to corrupt the checksum file
+          yield checksumByWriter - 1;
+        }
+        case UNKNOWN_ISSUE -> {
+          // write an int instead of a long to corrupt the checksum file
           out.writeInt(0);
-          checksumByReader = checksumByWriter;
-          break;
-
-        default:
+          yield checksumByWriter;
+        }
+        default -> {
           out.writeLong(checksumByWriter);
-          checksumByReader = checksumByWriter;
-      }
+          yield checksumByWriter;
+        }
+      };
     }
     out.close();
 
diff --git 
a/common/network-shuffle/src/test/java/org/apache/spark/network/shuffle/ExternalShuffleIntegrationSuite.java
 
b/common/network-shuffle/src/test/java/org/apache/spark/network/shuffle/ExternalShuffleIntegrationSuite.java
index 694c23b9f4d..b5ffa30f62d 100644
--- 
a/common/network-shuffle/src/test/java/org/apache/spark/network/shuffle/ExternalShuffleIntegrationSuite.java
+++ 
b/common/network-shuffle/src/test/java/org/apache/spark/network/shuffle/ExternalShuffleIntegrationSuite.java
@@ -116,12 +116,10 @@ public class ExternalShuffleIntegrationSuite {
         public ManagedBuffer getRddBlockData(String appId, String execId, int 
rddId, int splitIdx) {
           ManagedBuffer res;
           if (rddId == RDD_ID) {
-            switch (splitIdx) {
-              case SPLIT_INDEX_CORRUPT_LENGTH:
-                res = new FileSegmentManagedBuffer(conf, new 
File("missing.file"), 0, 12);
-                break;
-              default:
-                res = super.getRddBlockData(appId, execId, rddId, splitIdx);
+            if (splitIdx == SPLIT_INDEX_CORRUPT_LENGTH) {
+              res = new FileSegmentManagedBuffer(conf, new 
File("missing.file"), 0, 12);
+            } else {
+              res = super.getRddBlockData(appId, execId, rddId, splitIdx);
             }
           } else {
             res = super.getRddBlockData(appId, execId, rddId, splitIdx);
diff --git 
a/common/network-yarn/src/main/java/org/apache/spark/network/yarn/YarnShuffleServiceMetrics.java
 
b/common/network-yarn/src/main/java/org/apache/spark/network/yarn/YarnShuffleServiceMetrics.java
index 9230c2b40ed..0eb0c10df4c 100644
--- 
a/common/network-yarn/src/main/java/org/apache/spark/network/yarn/YarnShuffleServiceMetrics.java
+++ 
b/common/network-yarn/src/main/java/org/apache/spark/network/yarn/YarnShuffleServiceMetrics.java
@@ -90,18 +90,11 @@ class YarnShuffleServiceMetrics implements MetricsSource {
         .addGauge(
           getShuffleServiceMetricsInfoForGenericValue(name, "stdDev"), 
snapshot.getStdDev());
       for (int percentileThousands : new int[] { 10, 50, 250, 500, 750, 950, 
980, 990, 999 }) {
-        String percentileStr;
-        switch (percentileThousands) {
-          case 10:
-            percentileStr = "1stPercentile";
-            break;
-          case 999:
-            percentileStr = "999thPercentile";
-            break;
-          default:
-            percentileStr = String.format("%dthPercentile", 
percentileThousands / 10);
-            break;
-        }
+        String percentileStr = switch (percentileThousands) {
+          case 10 -> "1stPercentile";
+          case 999 -> "999thPercentile";
+          default -> String.format("%dthPercentile", percentileThousands / 10);
+        };
         metricsRecordBuilder.addGauge(
           getShuffleServiceMetricsInfoForGenericValue(name, percentileStr),
           snapshot.getValue(percentileThousands / 1000.0));
diff --git 
a/common/unsafe/src/main/java/org/apache/spark/unsafe/UnsafeAlignedOffset.java 
b/common/unsafe/src/main/java/org/apache/spark/unsafe/UnsafeAlignedOffset.java
index d399e66aa2a..2dfd1ecb635 100644
--- 
a/common/unsafe/src/main/java/org/apache/spark/unsafe/UnsafeAlignedOffset.java
+++ 
b/common/unsafe/src/main/java/org/apache/spark/unsafe/UnsafeAlignedOffset.java
@@ -41,16 +41,14 @@ public class UnsafeAlignedOffset {
   }
 
   public static int getSize(Object object, long offset) {
-    switch (getUaoSize()) {
-      case 4:
-        return Platform.getInt(object, offset);
-      case 8:
-        return (int)Platform.getLong(object, offset);
-      default:
+    return switch (getUaoSize()) {
+      case 4 -> Platform.getInt(object, offset);
+      case 8 -> (int) Platform.getLong(object, offset);
+      default ->
         // checkstyle.off: RegexpSinglelineJava
         throw new AssertionError("Illegal UAO_SIZE");
         // checkstyle.on: RegexpSinglelineJava
-    }
+    };
   }
 
   public static void putSize(Object object, long offset, int value) {
diff --git 
a/sql/catalyst/src/main/java/org/apache/spark/sql/catalyst/expressions/ExpressionImplUtils.java
 
b/sql/catalyst/src/main/java/org/apache/spark/sql/catalyst/expressions/ExpressionImplUtils.java
index a604e6bf225..45b60ec9f92 100644
--- 
a/sql/catalyst/src/main/java/org/apache/spark/sql/catalyst/expressions/ExpressionImplUtils.java
+++ 
b/sql/catalyst/src/main/java/org/apache/spark/sql/catalyst/expressions/ExpressionImplUtils.java
@@ -145,12 +145,10 @@ public class ExpressionImplUtils {
   }
 
   private static SecretKeySpec getSecretKeySpec(byte[] key) {
-    switch (key.length) {
-      case 16: case 24: case 32:
-        return new SecretKeySpec(key, 0, key.length, "AES");
-      default:
-        throw QueryExecutionErrors.invalidAesKeyLengthError(key.length);
-    }
+    return switch (key.length) {
+      case 16, 24, 32 -> new SecretKeySpec(key, 0, key.length, "AES");
+      default -> throw 
QueryExecutionErrors.invalidAesKeyLengthError(key.length);
+    };
   }
 
   private static byte[] generateIv(CipherMode mode) {
@@ -160,14 +158,11 @@ public class ExpressionImplUtils {
   }
 
   private static AlgorithmParameterSpec getParamSpec(CipherMode mode, byte[] 
input) {
-    switch (mode) {
-      case CBC:
-        return new IvParameterSpec(input, 0, mode.ivLength);
-      case GCM:
-        return new GCMParameterSpec(mode.tagLength, input, 0, mode.ivLength);
-      default:
-        return null;
-    }
+    return switch (mode) {
+      case CBC -> new IvParameterSpec(input, 0, mode.ivLength);
+      case GCM -> new GCMParameterSpec(mode.tagLength, input, 0, 
mode.ivLength);
+      default -> null;
+    };
   }
 
   private static byte[] aesInternal(
diff --git 
a/sql/catalyst/src/main/java/org/apache/spark/sql/connector/expressions/NullOrdering.java
 
b/sql/catalyst/src/main/java/org/apache/spark/sql/connector/expressions/NullOrdering.java
index 669d1c8443b..4aca199c11c 100644
--- 
a/sql/catalyst/src/main/java/org/apache/spark/sql/connector/expressions/NullOrdering.java
+++ 
b/sql/catalyst/src/main/java/org/apache/spark/sql/connector/expressions/NullOrdering.java
@@ -30,13 +30,9 @@ public enum NullOrdering {
 
   @Override
   public String toString() {
-    switch (this) {
-      case NULLS_FIRST:
-        return "NULLS FIRST";
-      case NULLS_LAST:
-        return "NULLS LAST";
-      default:
-        throw new IllegalArgumentException("Unexpected null order: " + this);
-    }
+    return switch (this) {
+      case NULLS_FIRST -> "NULLS FIRST";
+      case NULLS_LAST -> "NULLS LAST";
+    };
   }
 }
diff --git 
a/sql/catalyst/src/main/java/org/apache/spark/sql/connector/expressions/SortDirection.java
 
b/sql/catalyst/src/main/java/org/apache/spark/sql/connector/expressions/SortDirection.java
index e42dd5bcc2e..7e3a29945cc 100644
--- 
a/sql/catalyst/src/main/java/org/apache/spark/sql/connector/expressions/SortDirection.java
+++ 
b/sql/catalyst/src/main/java/org/apache/spark/sql/connector/expressions/SortDirection.java
@@ -49,13 +49,9 @@ public enum SortDirection {
 
   @Override
   public String toString() {
-    switch (this) {
-      case ASCENDING:
-        return "ASC";
-      case DESCENDING:
-        return "DESC";
-      default:
-        throw new IllegalArgumentException("Unexpected sort direction: " + 
this);
-    }
+    return switch (this) {
+      case ASCENDING -> "ASC";
+      case DESCENDING -> "DESC";
+    };
   }
 }
diff --git 
a/sql/catalyst/src/main/java/org/apache/spark/sql/connector/util/V2ExpressionSQLBuilder.java
 
b/sql/catalyst/src/main/java/org/apache/spark/sql/connector/util/V2ExpressionSQLBuilder.java
index e529a8e9250..5c2523943dd 100644
--- 
a/sql/catalyst/src/main/java/org/apache/spark/sql/connector/util/V2ExpressionSQLBuilder.java
+++ 
b/sql/catalyst/src/main/java/org/apache/spark/sql/connector/util/V2ExpressionSQLBuilder.java
@@ -264,12 +264,10 @@ public class V2ExpressionSQLBuilder {
   }
 
   protected String visitBinaryComparison(String name, String l, String r) {
-    switch (name) {
-      case "<=>":
-        return "(" + l + " = " + r + ") OR (" + l + " IS NULL AND " + r + " IS 
NULL)";
-      default:
-        return l + " " + name + " " + r;
+    if (name.equals("<=>")) {
+      return "(" + l + " = " + r + ") OR (" + l + " IS NULL AND " + r + " IS 
NULL)";
     }
+    return l + " " + name + " " + r;
   }
 
   protected String visitBinaryArithmetic(String name, String l, String r) {
diff --git 
a/sql/core/src/main/java/org/apache/spark/sql/execution/datasources/parquet/ParquetColumnVector.java
 
b/sql/core/src/main/java/org/apache/spark/sql/execution/datasources/parquet/ParquetColumnVector.java
index e8ae6f290bc..f00b5b3a88b 100644
--- 
a/sql/core/src/main/java/org/apache/spark/sql/execution/datasources/parquet/ParquetColumnVector.java
+++ 
b/sql/core/src/main/java/org/apache/spark/sql/execution/datasources/parquet/ParquetColumnVector.java
@@ -343,14 +343,10 @@ final class ParquetColumnVector {
   }
 
   private static WritableColumnVector allocateLevelsVector(int capacity, 
MemoryMode memoryMode) {
-    switch (memoryMode) {
-      case ON_HEAP:
-        return new OnHeapColumnVector(capacity, DataTypes.IntegerType);
-      case OFF_HEAP:
-        return new OffHeapColumnVector(capacity, DataTypes.IntegerType);
-      default:
-        throw new IllegalArgumentException("Unknown memory mode: " + 
memoryMode);
-    }
+    return switch (memoryMode) {
+      case ON_HEAP -> new OnHeapColumnVector(capacity, DataTypes.IntegerType);
+      case OFF_HEAP -> new OffHeapColumnVector(capacity, 
DataTypes.IntegerType);
+    };
   }
 
   /**
diff --git 
a/sql/core/src/main/java/org/apache/spark/sql/execution/datasources/parquet/VectorizedRleValuesReader.java
 
b/sql/core/src/main/java/org/apache/spark/sql/execution/datasources/parquet/VectorizedRleValuesReader.java
index 41ce02e3c02..58add7bc04e 100644
--- 
a/sql/core/src/main/java/org/apache/spark/sql/execution/datasources/parquet/VectorizedRleValuesReader.java
+++ 
b/sql/core/src/main/java/org/apache/spark/sql/execution/datasources/parquet/VectorizedRleValuesReader.java
@@ -147,13 +147,10 @@ public final class VectorizedRleValuesReader extends 
ValuesReader
     if (this.currentCount == 0) { this.readNextGroup(); }
 
     this.currentCount--;
-    switch (mode) {
-      case RLE:
-        return this.currentValue;
-      case PACKED:
-        return this.currentBuffer[currentBufferIdx++];
-    }
-    throw new RuntimeException("Unreachable");
+    return switch (mode) {
+      case RLE -> this.currentValue;
+      case PACKED -> this.currentBuffer[currentBufferIdx++];
+    };
   }
 
   /**


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

Reply via email to