diff --git a/docs/deployment/settings.md b/docs/deployment/settings.md
index 9f173b3b039..75fc3d91b14 100644
--- a/docs/deployment/settings.md
+++ b/docs/deployment/settings.md
@@ -440,7 +440,7 @@ You can configure the Kyuubi properties in `$KYUUBI_HOME/conf/kyuubi-defaults.co
| kyuubi.operation.plan.only.output.style | plain | Configures the planOnly output style. The value can be 'plain' or 'json', and the default value is 'plain'. This configuration supports only the output styles of the Spark engine | string | 1.7.0 |
| kyuubi.operation.progress.enabled | false | Whether to enable the operation progress. When true, the operation progress will be returned in `GetOperationStatus`. | boolean | 1.6.0 |
| kyuubi.operation.query.timeout | <undefined> | Timeout for query executions at server-side, take effect with client-side timeout(`java.sql.Statement.setQueryTimeout`) together, a running query will be cancelled automatically if timeout. It's off by default, which means only client-side take full control of whether the query should timeout or not. If set, client-side timeout is capped at this point. To cancel the queries right away without waiting for task to finish, consider enabling kyuubi.operation.interrupt.on.cancel together. | duration | 1.2.0 |
-| kyuubi.operation.result.arrow.timestampAsString | true | When true, arrow-based rowsets will convert columns of type timestamp to strings for transmission. Note that the timestamp column has a different behavior compared to thrift when it's false, but it brings better transfer performance. | boolean | 1.7.0 |
+| kyuubi.operation.result.arrow.timestampAsString | false | When true, arrow-based rowsets will convert columns of type timestamp to strings for transmission. | boolean | 1.7.0 |
| kyuubi.operation.result.format | thrift | Specify the result format, available configs are:
- THRIFT: the result will convert to TRow at the engine driver side.
- ARROW: the result will be encoded as Arrow at the executor side before collecting by the driver, and deserialized at the client side. note that it only takes effect for kyuubi-hive-jdbc clients now.
| string | 1.7.0 |
| kyuubi.operation.result.max.rows | 0 | Max rows of Spark query results. Rows exceeding the limit would be ignored. By setting this value to 0 to disable the max rows limit. | int | 1.6.0 |
| kyuubi.operation.scheduler.pool | <undefined> | The scheduler pool of job. Note that, this config should be used after changing Spark config spark.scheduler.mode=FAIR. | string | 1.1.1 |
diff --git a/externals/kyuubi-spark-sql-engine/src/main/scala/org/apache/kyuubi/engine/spark/operation/SparkOperation.scala b/externals/kyuubi-spark-sql-engine/src/main/scala/org/apache/kyuubi/engine/spark/operation/SparkOperation.scala
index 0c178bb4777..a6a7fc896af 100644
--- a/externals/kyuubi-spark-sql-engine/src/main/scala/org/apache/kyuubi/engine/spark/operation/SparkOperation.scala
+++ b/externals/kyuubi-spark-sql-engine/src/main/scala/org/apache/kyuubi/engine/spark/operation/SparkOperation.scala
@@ -280,7 +280,7 @@ abstract class SparkOperation(session: Session)
}
protected def timestampAsString: Boolean = {
- spark.conf.get("kyuubi.operation.result.arrow.timestampAsString", "true").toBoolean
+ spark.conf.get("kyuubi.operation.result.arrow.timestampAsString", "false").toBoolean
}
protected def setSessionUserSign(): Unit = {
diff --git a/externals/kyuubi-spark-sql-engine/src/test/scala/org/apache/kyuubi/engine/spark/operation/SparkArrowbasedOperationSuite.scala b/externals/kyuubi-spark-sql-engine/src/test/scala/org/apache/kyuubi/engine/spark/operation/SparkArrowbasedOperationSuite.scala
index 277a6bd1698..60cc528912d 100644
--- a/externals/kyuubi-spark-sql-engine/src/test/scala/org/apache/kyuubi/engine/spark/operation/SparkArrowbasedOperationSuite.scala
+++ b/externals/kyuubi-spark-sql-engine/src/test/scala/org/apache/kyuubi/engine/spark/operation/SparkArrowbasedOperationSuite.scala
@@ -73,21 +73,15 @@ class SparkArrowbasedOperationSuite extends WithSparkSQLEngine with SparkDataTyp
assert(rs.next())
}
- // timestampAsString = true
- statement.executeQuery(s"set ${KyuubiConf.ARROW_BASED_ROWSET_TIMESTAMP_AS_STRING.key}=true")
- checkArrowBasedRowSetTimestampAsString(statement, "true")
- setTimeZone("UTC")
- check("2022-12-07 17:15:35.0")
- setTimeZone("GMT+8")
- check("2022-12-08 01:15:35.0")
-
- // timestampAsString = false
- statement.executeQuery(s"set ${KyuubiConf.ARROW_BASED_ROWSET_TIMESTAMP_AS_STRING.key}=false")
- checkArrowBasedRowSetTimestampAsString(statement, "false")
- setTimeZone("UTC")
- check("2022-12-07 17:15:35.0")
- setTimeZone("GMT+8")
- check("2022-12-08 01:15:35.0")
+ Seq("true", "false").foreach { timestampAsString =>
+ statement.executeQuery(
+ s"set ${KyuubiConf.ARROW_BASED_ROWSET_TIMESTAMP_AS_STRING.key}=$timestampAsString")
+ checkArrowBasedRowSetTimestampAsString(statement, timestampAsString)
+ setTimeZone("UTC")
+ check("2022-12-07 17:15:35.0")
+ setTimeZone("GMT+8")
+ check("2022-12-08 01:15:35.0")
+ }
}
}
diff --git a/kyuubi-common/src/main/scala/org/apache/kyuubi/config/KyuubiConf.scala b/kyuubi-common/src/main/scala/org/apache/kyuubi/config/KyuubiConf.scala
index 3693e60ffd6..0c6a30c24e2 100644
--- a/kyuubi-common/src/main/scala/org/apache/kyuubi/config/KyuubiConf.scala
+++ b/kyuubi-common/src/main/scala/org/apache/kyuubi/config/KyuubiConf.scala
@@ -1679,11 +1679,10 @@ object KyuubiConf {
val ARROW_BASED_ROWSET_TIMESTAMP_AS_STRING: ConfigEntry[Boolean] =
buildConf("kyuubi.operation.result.arrow.timestampAsString")
.doc("When true, arrow-based rowsets will convert columns of type timestamp to strings for" +
- " transmission. Note that the timestamp column has a different behavior compared to" +
- " thrift when it's false, but it brings better transfer performance.")
+ " transmission.")
.version("1.7.0")
.booleanConf
- .createWithDefault(true)
+ .createWithDefault(false)
val SERVER_OPERATION_LOG_DIR_ROOT: ConfigEntry[String] =
buildConf("kyuubi.operation.log.dir.root")
diff --git a/kyuubi-hive-jdbc/src/main/java/org/apache/kyuubi/jdbc/hive/KyuubiStatement.java b/kyuubi-hive-jdbc/src/main/java/org/apache/kyuubi/jdbc/hive/KyuubiStatement.java
index 6d3bf1fa09b..b452ca6aa36 100644
--- a/kyuubi-hive-jdbc/src/main/java/org/apache/kyuubi/jdbc/hive/KyuubiStatement.java
+++ b/kyuubi-hive-jdbc/src/main/java/org/apache/kyuubi/jdbc/hive/KyuubiStatement.java
@@ -37,7 +37,7 @@ public class KyuubiStatement implements SQLStatement, KyuubiLoggable {
public static final Logger LOG = LoggerFactory.getLogger(KyuubiStatement.class.getName());
public static final int DEFAULT_FETCH_SIZE = 1000;
public static final String DEFAULT_RESULT_FORMAT = "thrift";
- public static final String DEFAULT_ARROW_TIMESTAMP_AS_STRING = "true";
+ public static final String DEFAULT_ARROW_TIMESTAMP_AS_STRING = "false";
private final KyuubiConnection connection;
private TCLIService.Iface client;
private TOperationHandle stmtHandle = null;
diff --git a/kyuubi-hive-jdbc/src/main/java/org/apache/kyuubi/jdbc/hive/arrow/ArrowUtils.java b/kyuubi-hive-jdbc/src/main/java/org/apache/kyuubi/jdbc/hive/arrow/ArrowUtils.java
index 26189dc242b..9a777d4c240 100644
--- a/kyuubi-hive-jdbc/src/main/java/org/apache/kyuubi/jdbc/hive/arrow/ArrowUtils.java
+++ b/kyuubi-hive-jdbc/src/main/java/org/apache/kyuubi/jdbc/hive/arrow/ArrowUtils.java
@@ -86,7 +86,6 @@ public static ArrowType toArrowType(TTypeId ttype, JdbcColumnAttributes jdbcColu
return new ArrowType.Date(DateUnit.DAY);
case TIMESTAMP_TYPE:
if (jdbcColumnAttributes != null) {
- System.out.println("session timeZone: " + jdbcColumnAttributes.timeZone);
return new ArrowType.Timestamp(TimeUnit.MICROSECOND, jdbcColumnAttributes.timeZone);
} else {
throw new IllegalStateException("Missing timezoneId where it is mandatory.");