diff --git a/docs/ss-migration-guide.md b/docs/ss-migration-guide.md
index 0ca5b00debccf..57fe3a84e12ca 100644
--- a/docs/ss-migration-guide.md
+++ b/docs/ss-migration-guide.md
@@ -30,6 +30,8 @@ Please refer [Migration Guide: SQL, Datasets and DataFrame](sql-migration-guide.
- Since Spark 3.4, `Trigger.Once` is deprecated, and users are encouraged to migrate from `Trigger.Once` to `Trigger.AvailableNow`. Please refer [SPARK-39805](https://issues.apache.org/jira/browse/SPARK-39805) for more details.
+- Since Spark 3.4, the default value of configuration for Kafka offset fetching (`spark.sql.streaming.kafka.useDeprecatedOffsetFetching`) is changed from `true` to `false`. The default no longer relies consumer group based scheduling, which affect the required ACL. For further details please see [Structured Streaming Kafka Integration](structured-streaming-kafka-integration.html#offset-fetching).
+
## Upgrading from Structured Streaming 3.2 to 3.3
- Since Spark 3.3, all stateful operators require hash partitioning with exact grouping keys. In previous versions, all stateful operators except stream-stream join require loose partitioning criteria which opens the possibility on correctness issue. (See [SPARK-38204](https://issues.apache.org/jira/browse/SPARK-38204) for more details.) To ensure backward compatibility, we retain the old behavior with the checkpoint built from older versions.
diff --git a/docs/structured-streaming-kafka-integration.md b/docs/structured-streaming-kafka-integration.md
index 6121f19e806b3..d7b741a3fe6ac 100644
--- a/docs/structured-streaming-kafka-integration.md
+++ b/docs/structured-streaming-kafka-integration.md
@@ -568,8 +568,9 @@ Timestamp offset options require Kafka 0.10.1.0 or higher.
### Offset fetching
In Spark 3.0 and before Spark uses KafkaConsumer
for offset fetching which could cause infinite wait in the driver.
-In Spark 3.1 a new configuration option added spark.sql.streaming.kafka.useDeprecatedOffsetFetching
(default: true
)
-which could be set to `false` allowing Spark to use new offset fetching mechanism using AdminClient
.
+In Spark 3.1 a new configuration option added spark.sql.streaming.kafka.useDeprecatedOffsetFetching
(default: false
)
+which allows Spark to use new offset fetching mechanism using AdminClient
. (Set this to `true` to use old offset fetching with KafkaConsumer
.)
+
When the new mechanism used the following applies.
First of all the new approach supports Kafka brokers `0.11.0.0+`.
diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/internal/SQLConf.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/internal/SQLConf.scala
index a99a795018d7d..72eb420de3749 100644
--- a/sql/catalyst/src/main/scala/org/apache/spark/sql/internal/SQLConf.scala
+++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/internal/SQLConf.scala
@@ -1924,7 +1924,7 @@ object SQLConf {
"Integration Guide.")
.version("3.1.0")
.booleanConf
- .createWithDefault(true)
+ .createWithDefault(false)
val STATEFUL_OPERATOR_CHECK_CORRECTNESS_ENABLED =
buildConf("spark.sql.streaming.statefulOperator.checkCorrectness.enabled")