diff --git a/cassandra/cqltypes.py b/cassandra/cqltypes.py index e36c48563c..a3658ba3ea 100644 --- a/cassandra/cqltypes.py +++ b/cassandra/cqltypes.py @@ -642,25 +642,26 @@ def interpret_datestring(val): @staticmethod def deserialize(byts, protocol_version): - timestamp = int64_unpack(byts) / 1000.0 - return util.datetime_from_timestamp(timestamp) + timestamp_ms = int64_unpack(byts) + return util.datetime_from_timestamp_ms(timestamp_ms) @staticmethod def serialize(v, protocol_version): try: # v is datetime timestamp_seconds = calendar.timegm(v.utctimetuple()) - timestamp = timestamp_seconds * 1e3 + getattr(v, 'microsecond', 0) / 1e3 + # Use integer arithmetic to preserve precision + timestamp_ms = timestamp_seconds * 1000 + getattr(v, 'microsecond', 0) // 1000 except AttributeError: try: - timestamp = calendar.timegm(v.timetuple()) * 1e3 + timestamp_ms = calendar.timegm(v.timetuple()) * 1000 except AttributeError: # Ints and floats are valid timestamps too if type(v) not in _number_types: raise TypeError('DateType arguments must be a datetime, date, or timestamp') - timestamp = v + timestamp_ms = v - return int64_pack(int(timestamp)) + return int64_pack(int(timestamp_ms)) @classmethod def serial_size(cls): diff --git a/cassandra/cython_utils.pxd b/cassandra/cython_utils.pxd index 4a1e71dba5..738a3d908d 100644 --- a/cassandra/cython_utils.pxd +++ b/cassandra/cython_utils.pxd @@ -1,2 +1,3 @@ from libc.stdint cimport int64_t cdef datetime_from_timestamp(double timestamp) +cdef datetime_from_timestamp_ms(int64_t timestamp_ms) diff --git a/cassandra/cython_utils.pyx b/cassandra/cython_utils.pyx index 7539f33f31..2f5a2c665a 100644 --- a/cassandra/cython_utils.pyx +++ b/cassandra/cython_utils.pyx @@ -60,3 +60,23 @@ cdef datetime_from_timestamp(double timestamp): microseconds += tmp return DATETIME_EPOC + timedelta_new(days, seconds, microseconds) + + +cdef datetime_from_timestamp_ms(int64_t timestamp_ms): + """ + Creates a timezone-agnostic datetime from timestamp in milliseconds. + Avoids floating-point conversion to maintain precision for large timestamps. + + :param timestamp_ms: a unix timestamp, in milliseconds + """ + # Break down milliseconds into components to avoid float conversion + # Cython's % operator uses Python semantics, always returns non-negative for positive divisor + cdef int64_t timestamp_seconds = timestamp_ms // 1000 + cdef int64_t remainder_ms = timestamp_ms % 1000 + + cdef int days = (timestamp_seconds // DAY_IN_SECONDS) + cdef int64_t days_in_seconds = ( days) * DAY_IN_SECONDS + cdef int seconds = (timestamp_seconds - days_in_seconds) + cdef int microseconds = (remainder_ms * 1000) + + return DATETIME_EPOC + timedelta_new(days, seconds, microseconds) diff --git a/cassandra/deserializers.pyx b/cassandra/deserializers.pyx index 7c256674b0..ab94af74a0 100644 --- a/cassandra/deserializers.pyx +++ b/cassandra/deserializers.pyx @@ -13,11 +13,11 @@ # limitations under the License. -from libc.stdint cimport int32_t, uint16_t +from libc.stdint cimport int32_t, uint16_t, int64_t include 'cython_marshal.pyx' from cassandra.buffer cimport Buffer, to_bytes, slice_buffer -from cassandra.cython_utils cimport datetime_from_timestamp +from cassandra.cython_utils cimport datetime_from_timestamp_ms from cython.view cimport array as cython_array from cassandra.tuple cimport tuple_new, tuple_set @@ -135,8 +135,8 @@ cdef class DesCounterColumnType(DesLongType): cdef class DesDateType(Deserializer): cdef deserialize(self, Buffer *buf, int protocol_version): - cdef double timestamp = unpack_num[int64_t](buf) / 1000.0 - return datetime_from_timestamp(timestamp) + cdef int64_t timestamp_ms = unpack_num[int64_t](buf) + return datetime_from_timestamp_ms(timestamp_ms) cdef class TimestampType(DesDateType): diff --git a/cassandra/util.py b/cassandra/util.py index 12886d05ab..93256977af 100644 --- a/cassandra/util.py +++ b/cassandra/util.py @@ -62,6 +62,25 @@ def datetime_from_timestamp(timestamp): return dt +def datetime_from_timestamp_ms(timestamp_ms): + """ + Creates a timezone-agnostic datetime from timestamp in milliseconds. + Avoids floating-point conversion to maintain precision for large timestamps. + + Works around precision loss issues with large timestamps (far from epoch) + by using integer arithmetic throughout. + + :param timestamp_ms: a unix timestamp, in milliseconds (as integer) + """ + # Break down milliseconds into components to avoid float conversion + # Python's % operator always returns non-negative result for positive divisor + timestamp_seconds = timestamp_ms // 1000 + remainder_ms = timestamp_ms % 1000 + microseconds = remainder_ms * 1000 + dt = DATETIME_EPOC + datetime.timedelta(seconds=timestamp_seconds, microseconds=microseconds) + return dt + + def utc_datetime_from_ms_timestamp(timestamp): """ Creates a UTC datetime from a timestamp in milliseconds. See diff --git a/tests/integration/standard/test_types.py b/tests/integration/standard/test_types.py index 4ee9b70cde..0df4f44abc 100644 --- a/tests/integration/standard/test_types.py +++ b/tests/integration/standard/test_types.py @@ -429,6 +429,55 @@ def test_timezone_aware_datetimes_are_timestamps(self): result = s.execute("SELECT b FROM tz_aware WHERE a='key2'").one().b assert dt.utctimetuple() == result.utctimetuple() + def test_large_timestamp_precision(self): + """ + Test that large timestamp values (far from epoch) maintain precision + through round-trip serialization/deserialization. + + Verifies the fix for precision loss issue where timestamps more than + ~300 years from Unix epoch would lose millisecond precision due to + floating-point conversion. + + @jira_ticket PYTHON-XXXXX + @expected_result Timestamps far from epoch should round-trip with exact precision + """ + s = self.session + s.execute("CREATE TABLE large_timestamps (pk timestamp PRIMARY KEY)") + + # Test timestamps far from epoch (year 2300 and year 1640) + # These would lose precision with float conversion + test_timestamps = [ + datetime(2300, 1, 1, 0, 0, 0, 1000), # 1 millisecond in year 2300 + datetime(2300, 1, 1, 0, 0, 0, 999000), # 999 milliseconds in year 2300 + datetime(1640, 1, 1, 0, 0, 0, 1000), # 1 millisecond in year 1640 + datetime(2500, 12, 31, 23, 59, 59, 999000), # Very far future + ] + + for original_timestamp in test_timestamps: + with self.subTest(timestamp=original_timestamp): + # Insert using prepared statement (uses serialization) + insert = s.prepare("INSERT INTO large_timestamps (pk) VALUES (?)") + s.execute(insert, [original_timestamp]) + + # Retrieve the timestamp (uses deserialization) + result = s.execute("SELECT pk FROM large_timestamps WHERE pk = ?", [original_timestamp]).one() + assert result is not None, f"Failed to retrieve timestamp {original_timestamp}" + retrieved_timestamp = result.pk + + # Verify exact equality - microseconds should match + assert retrieved_timestamp == original_timestamp, \ + f"Timestamp mismatch: original={original_timestamp}, retrieved={retrieved_timestamp}" + + # Verify we can query using the retrieved timestamp (round-trip test) + result2 = s.execute("SELECT pk FROM large_timestamps WHERE pk = ?", [retrieved_timestamp]).one() + assert result2 is not None, \ + f"Failed to query with retrieved timestamp {retrieved_timestamp}" + assert result2.pk == original_timestamp, \ + f"Second round-trip failed: expected={original_timestamp}, got={result2.pk}" + + # Clean up for next test + s.execute("DELETE FROM large_timestamps WHERE pk = ?", [original_timestamp]) + def test_can_insert_tuples(self): """ Basic test of tuple functionality diff --git a/tests/unit/test_timestamp_precision.py b/tests/unit/test_timestamp_precision.py new file mode 100644 index 0000000000..47d62147a5 --- /dev/null +++ b/tests/unit/test_timestamp_precision.py @@ -0,0 +1,128 @@ +# Copyright DataStax, Inc. +# +# Licensed under the Apache License, Version 2.0 (the "License"); +# you may not use this file except in compliance with the License. +# You may obtain a copy of the License at +# +# http://www.apache.org/licenses/LICENSE-2.0 +# +# Unless required by applicable law or agreed to in writing, software +# distributed under the License is distributed on an "AS IS" BASIS, +# WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. +# See the License for the specific language governing permissions and +# limitations under the License. + +import unittest +import datetime +from cassandra.cqltypes import DateType +from cassandra.marshal import int64_pack + + +class TimestampPrecisionTests(unittest.TestCase): + """ + Tests for timestamp precision with large values (far from epoch). + See: https://github.com/scylladb/python-driver/issues/XXX + """ + + def test_large_timestamp_roundtrip(self): + """ + Test that timestamps far from epoch (> 300 years) maintain precision + through serialize/deserialize cycle. + """ + # Timestamp for "2300-01-01 00:00:00.001" in milliseconds + # This is far enough from epoch that float precision is lost + original_ms = 10413792000001 # 2300-01-01 00:00:00.001 + + # Pack as int64 (simulating database storage) + packed = int64_pack(original_ms) + + # Deserialize back + dt = DateType.deserialize(packed, 0) + + # Serialize again + repacked = DateType.serialize(dt, 0) + + # Unpack and compare + from cassandra.marshal import int64_unpack + result_ms = int64_unpack(repacked) + + # Should be exactly equal + assert result_ms == original_ms, \ + f"Expected {original_ms}, got {result_ms}, difference: {result_ms - original_ms}" + + def test_year_2300_timestamp_precision(self): + """ + Test the specific case from the issue report: + timestamp "2300-01-01 00:00:00.001" should maintain precision. + """ + # Create datetime for 2300-01-01 00:00:00.001 + dt = datetime.datetime(2300, 1, 1, 0, 0, 0, 1000) # 1000 microseconds = 1 millisecond + + # Serialize to bytes + packed = DateType.serialize(dt, 0) + + # Deserialize back + dt_restored = DateType.deserialize(packed, 0) + + # Serialize again + repacked = DateType.serialize(dt_restored, 0) + + # They should be exactly equal + assert packed == repacked, \ + f"Serialization not stable: {packed.hex()} != {repacked.hex()}" + + # The microseconds should be preserved + assert dt_restored.microsecond == 1000, \ + f"Expected 1000 microseconds, got {dt_restored.microsecond}" + + def test_various_large_timestamps(self): + """ + Test multiple timestamps far from epoch to ensure precision is maintained. + """ + # Various timestamps > 300 years from epoch (in milliseconds) + test_timestamps_ms = [ + 10413792000001, # 2300-01-01 00:00:00.001 + 10413792000999, # 2300-01-01 00:00:00.999 + 15768000000000, # 2469-12-31 12:00:00.000 + 20000000000001, # ~2603 with millisecond precision + -10413792000001, # ~1640 BCE + ] + + for original_ms in test_timestamps_ms: + with self.subTest(timestamp_ms=original_ms): + # Pack as int64 + packed = int64_pack(original_ms) + + # Deserialize + dt = DateType.deserialize(packed, 0) + + # Serialize again + repacked = DateType.serialize(dt, 0) + + # Unpack and compare + from cassandra.marshal import int64_unpack + result_ms = int64_unpack(repacked) + + # Should be exactly equal + assert result_ms == original_ms, \ + f"Expected {original_ms}, got {result_ms}, difference: {result_ms - original_ms}" + + def test_small_timestamp_still_works(self): + """ + Ensure that timestamps close to epoch still work correctly. + """ + # Timestamp close to epoch (well within float precision) + original_ms = 1000000000000 # 2001-09-09 01:46:40.000 + + packed = int64_pack(original_ms) + dt = DateType.deserialize(packed, 0) + repacked = DateType.serialize(dt, 0) + + from cassandra.marshal import int64_unpack + result_ms = int64_unpack(repacked) + + assert result_ms == original_ms + + +if __name__ == '__main__': + unittest.main()