### What changes were proposed in this pull request?
Support Flink 1.19.
### Why are the changes needed?
Flink 1.19.0 is announced to release: [Announcing the Release of Apache Flink 1.19] (https://flink.apache.org/2024/03/18/announcing-the-release-of-apache-flink-1.19).
The main changes includes:
- `org.apache.flink.runtime.io.network.partition.consumer.RemoteInputChannel` constructor change parameters:
- `consumedSubpartitionIndex` changes to `consumedSubpartitionIndexSet`: [[FLINK-33743][runtime] Support consuming multiple subpartitions on a single channel](https://github.com/apache/flink/pull/23927).
- adds `partitionRequestListenerTimeout`: [[FLINK-25055][network] Support listen and notify mechanism for partition request](https://github.com/apache/flink/pull/23565).
- `org.apache.flink.runtime.io.network.partition.consumer.SingleInputGate` constructor removes parameters `subpartitionIndexRange`, `tieredStorageConsumerClient`, `nettyService` and `tieredStorageConsumerSpecs`: [[FLINK-33743][runtime] Support consuming multiple subpartitions on a single channel](https://github.com/apache/flink/pull/23927).
- Change the default config file to `config.yaml` in `flink-dist`: [[FLINK-33577][dist] Change the default config file to config.yaml in flink-dist](https://github.com/apache/flink/pull/24177).
- `org.apache.flink.configuration.RestartStrategyOptions` uses `org.apache.commons.compress.utils.Sets` of `commons-compress` dependency: [[FLINK-33865][runtime] Adding an ITCase to ensure exponential-delay.attempts-before-reset-backoff works well](https://github.com/apache/flink/pull/23942).
### Does this PR introduce _any_ user-facing change?
No.
### How was this patch tested?
Local test:
- Flink batch job submission
```
$ ./bin/flink run examples/streaming/WordCount.jar --execution-mode BATCH
Executing example with default input data.
Use --input to specify file input.
Printing result to stdout. Use --output to specify output path.
Job has been submitted with JobID 2e9fb659991a9c29d376151783bdf6de
Program execution finished
Job with JobID 2e9fb659991a9c29d376151783bdf6de has finished.
Job Runtime: 1912 ms
```
- Flink batch job execution

- Celeborn master log
```
24/03/18 20:52:47,513 INFO [celeborn-dispatcher-42] Master: Offer slots successfully for 1 reducers of 1710766312631-2e9fb659991a9c29d376151783bdf6de-0 on 1 workers.
```
- Celeborn worker log
```
24/03/18 20:52:47,704 INFO [celeborn-dispatcher-1] StorageManager: created file at /Users/nicholas/Software/Celeborn/apache-celeborn-0.5.0-SNAPSHOT/shuffle/celeborn-worker/shuffle_data/1710766312631-2e9fb659991a9c29d376151783bdf6de/0/0-0-0
24/03/18 20:52:47,707 INFO [celeborn-dispatcher-1] Controller: Reserved 1 primary location and 0 replica location for 1710766312631-2e9fb659991a9c29d376151783bdf6de-0
24/03/18 20:52:47,874 INFO [celeborn-dispatcher-2] Controller: Start commitFiles for 1710766312631-2e9fb659991a9c29d376151783bdf6de-0
24/03/18 20:52:47,890 INFO [worker-rpc-async-replier] Controller: CommitFiles for 1710766312631-2e9fb659991a9c29d376151783bdf6de-0 success with 1 committed primary partitions, 0 empty primary partitions, 0 failed primary partitions, 0 committed replica partitions, 0 empty replica partitions, 0 failed replica partitions.
```
Closes#2399 from SteNicholas/CELEBORN-1310.
Authored-by: SteNicholas <programgeek@163.com>
Signed-off-by: Shuang <lvshuang.xjs@alibaba-inc.com>

### What changes were proposed in this pull request?
- Added Husky to enforce code quality with automated tasks during Git events.
- Added lint-staged for optimized linting on staged files before each commit.
### Why are the changes needed?
Enhances code quality.
### Does this PR introduce _any_ user-facing change?
No.
### How was this patch tested?
Local test.
Closes#2250 from tiny-dust/CELEBORN-1240.
Lead-authored-by: tiny-dust <idioticzhou@foxmail.com>
Co-authored-by: 周顺顺 <idioticzhou@foxmail.com>
Signed-off-by: SteNicholas <programgeek@163.com>
### What changes were proposed in this pull request?
### Why are the changes needed?
```bash
flink-1.18.0
./bin/start-cluster.sh
./bin/flink run examples/streaming/WordCount.jar --execution-mode BATCH
```
```java
Caused by: java.lang.NoSuchMethodError: org.apache.flink.runtime.io.network.partition.consumer.SingleInputGate.<init>(Ljava/lang/String;ILorg/apache/flink/runtime/jobgraph/IntermediateDataSetID;Lorg/apache/flink/runtime/io/network/partition/ResultPartitionType;Lorg/apache/flink/runtime/executiongraph/IndexRange;ILorg/apache/flink/runtime/io/network/partition/PartitionProducerStateProvider;Lorg/apache/flink/util/function/SupplierWithException;Lorg/apache/flink/runtime/io/network/buffer/BufferDecompressor;Lorg/apache/flink/core/memory/MemorySegmentProvider;ILorg/apache/flink/runtime/throughput/ThroughputCalculator;Lorg/apache/flink/runtime/throughput/BufferDebloater;)V
at org.apache.celeborn.plugin.flink.RemoteShuffleInputGate$FakedRemoteInputChannel.<init>(RemoteShuffleInputGate.java:225)
at org.apache.celeborn.plugin.flink.RemoteShuffleInputGate.getChannel(RemoteShuffleInputGate.java:179)
at org.apache.flink.runtime.io.network.partition.consumer.InputGate.setChannelStateWriter(InputGate.java:90)
at org.apache.flink.runtime.taskmanager.InputGateWithMetrics.setChannelStateWriter(InputGateWithMetrics.java:120)
at org.apache.flink.streaming.runtime.tasks.StreamTask.injectChannelStateWriterIntoChannels(StreamTask.java:524)
at org.apache.flink.streaming.runtime.tasks.StreamTask.<init>(StreamTask.java:496)
```
Flink 1.18.0 release
https://flink.apache.org/2023/10/24/announcing-the-release-of-apache-flink-1.18/
Interface `org.apache.flink.runtime.io.network.buffer.Buffer` adds `setRecycler` method.
[[FLINK-32549](https://issues.apache.org/jira/browse/FLINK-32549)][network] Tiered storage memory manager supports ownership transfer for buffers
`org.apache.flink.runtime.io.network.partition.consumer.SingleInputGate` constructor adds parameters.
[[FLINK-31638](https://issues.apache.org/jira/browse/FLINK-31638)][network] Introduce the TieredStorageConsumerClient to SingleInputGate
[[FLINK-31642](https://issues.apache.org/jira/browse/FLINK-31642)][network] Introduce the MemoryTierConsumerAgent to TieredStorageConsumerClient
### Does this PR introduce _any_ user-facing change?
### How was this patch tested?
```bash
flink-1.18.0 ./bin/flink run examples/streaming/WordCount.jar --execution-mode BATCH
Executing example with default input data.
Use --input to specify file input.
Printing result to stdout. Use --output to specify output path.
Job has been submitted with JobID d7fc5f0ca018a54e9453c4d35f7c598a
Program execution finished
Job with JobID d7fc5f0ca018a54e9453c4d35f7c598a has finished.
Job Runtime: 1635 ms
```
<img width="1297" alt="image" src="https://github.com/apache/incubator-celeborn/assets/3898450/6a5266bf-2386-4386-b98b-a60d2570fa99">
Closes#2063 from cxzl25/CELEBORN-1105.
Authored-by: sychen <sychen@ctrip.com>
Signed-off-by: Shuang <lvshuang.tb@gmail.com>