Skip to content
Draft
Show file tree
Hide file tree
Changes from all commits
Commits
File filter

Filter by extension

Filter by extension

Conversations
Failed to load comments.
Loading
Jump to
Jump to file
Failed to load files.
Loading
Diff view
Diff view
13 changes: 7 additions & 6 deletions cassandra/cqltypes.py
Original file line number Diff line number Diff line change
Expand Up @@ -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):
Expand Down
1 change: 1 addition & 0 deletions cassandra/cython_utils.pxd
Original file line number Diff line number Diff line change
@@ -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)
23 changes: 23 additions & 0 deletions cassandra/cython_utils.pyx
Original file line number Diff line number Diff line change
Expand Up @@ -60,3 +60,26 @@ cdef datetime_from_timestamp(double timestamp):
microseconds += <int>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
cdef int64_t timestamp_seconds = timestamp_ms // 1000
cdef int64_t remainder_ms = timestamp_ms % 1000
# Handle negative timestamps correctly
if remainder_ms < 0:
remainder_ms += 1000
timestamp_seconds -= 1

cdef int days = <int> (timestamp_seconds // DAY_IN_SECONDS)
cdef int64_t days_in_seconds = (<int64_t> days) * DAY_IN_SECONDS
cdef int seconds = <int> (timestamp_seconds - days_in_seconds)
cdef int microseconds = <int> (remainder_ms * 1000)

return DATETIME_EPOC + timedelta_new(days, seconds, microseconds)
8 changes: 4 additions & 4 deletions cassandra/deserializers.pyx
Original file line number Diff line number Diff line change
Expand Up @@ -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, datetime_from_timestamp_ms

from cython.view cimport array as cython_array
from cassandra.tuple cimport tuple_new, tuple_set
Expand Down Expand Up @@ -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):
Expand Down
23 changes: 23 additions & 0 deletions cassandra/util.py
Original file line number Diff line number Diff line change
Expand Up @@ -62,6 +62,29 @@ 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
timestamp_seconds = timestamp_ms // 1000
remainder_ms = timestamp_ms % 1000
# Handle negative timestamps correctly
if remainder_ms < 0:
remainder_ms += 1000
timestamp_seconds -= 1

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
Expand Down
128 changes: 128 additions & 0 deletions tests/unit/test_timestamp_precision.py
Original file line number Diff line number Diff line change
@@ -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()
Loading