You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@sedona.apache.org by ji...@apache.org on 2023/02/15 17:20:28 UTC

[sedona] branch master updated: [SEDONA-227] Implemented Python geometry serializer as a native extension (#767)

This is an automated email from the ASF dual-hosted git repository.

jiayu pushed a commit to branch master
in repository https://gitbox.apache.org/repos/asf/sedona.git


The following commit(s) were added to refs/heads/master by this push:
     new abe41fab [SEDONA-227] Implemented Python geometry serializer as a native extension (#767)
abe41fab is described below

commit abe41fab92c12d9c0693fd70312abd5723adad24
Author: Kristin Cowalcijk <bo...@wherobots.com>
AuthorDate: Thu Feb 16 01:20:21 2023 +0800

    [SEDONA-227] Implemented Python geometry serializer as a native extension (#767)
---
 .github/workflows/python-extension.yml             |  55 ++
 .github/workflows/python-wheel.yml                 |  33 +
 .github/workflows/python.yml                       |   3 +-
 python/sedona/utils/geometry_serde.py              | 602 +++-------------
 ...geometry_serde.py => geometry_serde_general.py} |   0
 python/setup.py                                    |  27 +-
 python/src/geom_buf.c                              | 514 ++++++++++++++
 python/src/geom_buf.h                              | 146 ++++
 python/src/geomserde.c                             | 771 +++++++++++++++++++++
 python/src/geomserde.h                             |  76 ++
 python/src/geomserde_speedup_module.c              | 288 ++++++++
 python/src/geos_c_dyn.c                            | 164 +++++
 python/src/geos_c_dyn.h                            |  91 +++
 python/src/geos_c_dyn_funcs.h                      | 144 ++++
 python/src/pygeos/c_api.h                          |  85 +++
 python/tests/utils/test_geometry_serde.py          | 103 ---
 python/tests/utils/test_geomserde_speedup.py       | 137 ++++
 17 files changed, 2627 insertions(+), 612 deletions(-)

diff --git a/.github/workflows/python-extension.yml b/.github/workflows/python-extension.yml
new file mode 100644
index 00000000..906ce19f
--- /dev/null
+++ b/.github/workflows/python-extension.yml
@@ -0,0 +1,55 @@
+name: Python build extension
+
+on:
+  push:
+    branches:
+      - master
+  pull_request:
+    branches:
+      - '*'
+      
+jobs:
+  build:
+    strategy:
+      matrix:
+        os: ['ubuntu-latest', 'windows-latest', 'macos-latest']
+        python: ['3.10', '3.9', '3.8']
+
+    runs-on: ${{ matrix.os }}
+    defaults:
+      run:
+        shell: bash
+
+    steps:
+    - uses: actions/checkout@v2
+    - uses: actions/setup-python@v2
+      with:
+        python-version: ${{ matrix.python }}
+    - name: Install pipenv
+      run: pip install -U pipenv
+    - name: Install dependencies
+      run: |
+        cd python
+        pipenv --python ${{ matrix.python }}
+        pipenv install --dev
+    - name: Build extension
+      run: |
+        cd python
+        pipenv run python setup.py build_ext --inplace
+    - name: Run tests
+      run: |
+        cd python
+        pipenv run pytest tests/utils/test_geomserde_speedup.py
+    - name: Run tests on Shapely 2.0
+      run: |
+        cd python
+        pipenv install shapely~=2.0
+        pipenv run pytest tests/utils/test_geomserde_speedup.py
+    - name: Run tests on Shapley 1.7
+      # Shapely 1.7 only provides wheels for cp36 ~ cp39, so we'll skip running
+      # this test for recent python versions.
+      if: ${{ matrix.python == '3.9' || matrix.python == '3.8' }}
+      run: |
+        cd python
+        pipenv install shapely~=1.7
+        pipenv run pytest tests/utils/test_geomserde_speedup.py
diff --git a/.github/workflows/python-wheel.yml b/.github/workflows/python-wheel.yml
new file mode 100644
index 00000000..0d5b229c
--- /dev/null
+++ b/.github/workflows/python-wheel.yml
@@ -0,0 +1,33 @@
+name: Python build wheels
+
+on:
+  push:
+    branches:
+      - master
+  pull_request:
+    branches:
+      - '*'
+      
+jobs:
+  build:
+    strategy:
+      matrix:
+        os: ['ubuntu-latest', 'windows-latest', 'macos-latest']
+
+    runs-on: ${{ matrix.os }}
+    defaults:
+      run:
+        shell: bash
+
+    steps:
+    - uses: actions/checkout@v2
+    - name: Build wheels
+      uses: pypa/cibuildwheel@v2.12.0
+      env:
+        CIBW_SKIP: 'pp* *musl*'
+        CIBW_ARCHS: 'auto64'
+      with:
+        package-dir: python
+    - uses: actions/upload-artifact@v3
+      with:
+        path: ./wheelhouse/*.whl
diff --git a/.github/workflows/python.yml b/.github/workflows/python.yml
index 0d5cc9b4..41e9b16a 100644
--- a/.github/workflows/python.yml
+++ b/.github/workflows/python.yml
@@ -71,11 +71,12 @@ jobs:
         SPARK_VERSION: ${{ matrix.spark }}
         HADOOP_VERSION: ${{ matrix.hadoop }}
       run: tar -xzf spark-${SPARK_VERSION}-bin-hadoop${HADOOP_VERSION}.tgz
-    - run: sudo apt-get -y install python3-pip python-dev libgeos-dev
+    - run: sudo apt-get -y install python3-pip python-dev
     - run: sudo pip3 install -U setuptools
     - run: sudo pip3 install -U wheel
     - run: sudo pip3 install -U virtualenvwrapper
     - run: python3 -m pip install pipenv
+    - run: cd python; python3 setup.py build_ext --inplace
     - env:
         SPARK_VERSION: ${{ matrix.spark }}
         PYTHON_VERSION: ${{ matrix.python }}
diff --git a/python/sedona/utils/geometry_serde.py b/python/sedona/utils/geometry_serde.py
index 9efd33eb..257ae759 100644
--- a/python/sedona/utils/geometry_serde.py
+++ b/python/sedona/utils/geometry_serde.py
@@ -15,520 +15,118 @@
 #  specific language governing permissions and limitations
 #  under the License.
 
-import array
-import struct
-import math
-from typing import List, Optional, Tuple, Union
+from typing import Optional
+from warnings import warn
+import sys
+import os
 
-import numpy as np
-from shapely.geometry import (
-    GeometryCollection,
-    LinearRing,
-    LineString,
-    MultiLineString,
-    MultiPoint,
-    MultiPolygon,
-    Point,
-    Polygon,
-)
+import shapely
 from shapely.geometry.base import BaseGeometry
-from shapely.wkb import dumps as wkb_dumps
-from shapely.wkt import loads as wkt_loads
 
 
-CoordType = Union[Tuple[float, float], Tuple[float, float, float], Tuple[float, float, float, float]]
-ListCoordType = Union[List[Tuple[float, float]], List[Tuple[float, float, float]], List[Tuple[float, float, float, float]]]
+speedup_enabled = False
 
-GET_COORDS_NUMPY_THRESHOLD = 50
 
+# Use geomserde_speedup when available, otherwise fallback to general pure
+# python implementation.
+try:
+    from . import geomserde_speedup
 
-class GeometryTypeID:
-    """
-    Constants used to identify the geometry type in the serialized bytearray of geometry.
-    """
-    POINT = 1
-    LINESTRING = 2
-    POLYGON = 3
-    MULTIPOINT = 4
-    MULTILINESTRING = 5
-    MULTIPOLYGON = 6
-    GEOMETRYCOLLECTION = 7
+    def find_geos_c_dll():
+        packages_dir = os.path.dirname(os.path.dirname(shapely.__file__))
+        for lib_dirname in ['shapely.libs', 'Shapely.libs']:
+            lib_dirpath = os.path.join(packages_dir, lib_dirname)
+            if not os.path.exists(lib_dirpath):
+                continue
+            for filename in os.listdir(lib_dirpath):
+                if filename.lower().startswith('geos_c') and filename.lower().endswith('.dll'):
+                    return os.path.join(lib_dirpath, filename)
+        raise RuntimeError('geos_c DLL not found in {}\\[S|s]hapely.libs'.format(packages_dir))
 
-
-class CoordinateType:
-    """
-    Constants used to identify geometry dimensions in the serialized bytearray of geometry.
-    """
-    XY = 1
-    XYZ = 2
-    XYM = 3
-    XYZM = 4
-
-    BYTES_PER_COORDINATE = [16, 24, 24, 32]
-    NUM_COORD_COMPONENTS = [2, 3, 3, 4]
-    UNPACK_FORMAT = ['dd', 'ddd', 'ddxxxxxxxx', 'dddxxxxxxxx']
-
-    @staticmethod
-    def type_of(geom) -> int:
-        if geom._ndim == 2:
-            return CoordinateType.XY
-        elif geom._ndim == 3:
-            return CoordinateType.XYZ
+    if shapely.__version__.startswith('2.'):
+        if sys.platform != 'win32':
+            # We load geos_c library indirectly by loading shapely.lib
+            import shapely.lib
+            geomserde_speedup.load_libgeos_c(shapely.lib.__file__)
         else:
-            raise ValueError("Invalid coordinate dimension: {}".format(geom._ndim))
-
-    @staticmethod
-    def bytes_per_coord(coord_type: int) -> int:
-        return CoordinateType.BYTES_PER_COORDINATE[coord_type - 1]
-
-    @staticmethod
-    def components_per_coord(coord_type: int) -> int:
-        return CoordinateType.NUM_COORD_COMPONENTS[coord_type - 1]
-
-    @staticmethod
-    def unpack_format(coord_type: int) -> str:
-        return CoordinateType.UNPACK_FORMAT[coord_type - 1]
-
-
-class GeometryBuffer:
-    buffer: bytearray
-    coord_type: int
-    bytes_per_coord: int
-    num_coords: int
-    coords_offset: int
-    ints_offset: int
-
-    def __init__(
-        self,
-        buffer: bytearray,
-        coord_type: int,
-        coords_offset: int,
-        num_coords: int
-        ) -> None:
-        self.buffer = buffer
-        self.coord_type = coord_type
-        self.bytes_per_coord = CoordinateType.bytes_per_coord(coord_type)
-        self.num_coords = num_coords
-        self.coords_offset = coords_offset
-        self.ints_offset = coords_offset + self.bytes_per_coord * num_coords
-
-    def read_linestring(self) -> LineString:
-        num_points = self.read_int()
-        return LineString(self.read_coordinates(num_points))
-
-    def read_linearring(self) -> LineString:
-        num_points = self.read_int()
-        return LinearRing(self.read_coordinates(num_points))
-
-    def read_polygon(self) -> Polygon:
-        num_rings = self.read_int()
-
-        if num_rings == 0:
-            return Polygon()
-
-        rings = [
-            self.read_coordinates(self.read_int())
-            for _ in range(num_rings)
-        ]
-
-        return Polygon(rings[0], rings[1:])
-
-    def write_linestring(self, line: LineString) -> None:
-        coords = [tuple(c) for c in line.coords]
-        self.write_int(len(coords))
-        self.coords_offset = put_coordinates(self.buffer, self.coords_offset, self.coord_type, coords)
-
-    def write_polygon(self, polygon: Polygon) -> None:
-        exterior = polygon.exterior
-        if not exterior.coords:
-            self.write_int(0)
-            return
-        self.write_int(len(polygon.interiors) + 1)
-        self.write_linestring(exterior)
-        for interior in polygon.interiors:
-            self.write_linestring(interior)
-
-    def read_coordinates(self, num_coords: int) -> ListCoordType:
-        coords = get_coordinates(self.buffer, self.coords_offset, self.coord_type, num_coords)
-        self.coords_offset += num_coords * self.bytes_per_coord
-        return coords
-
-    def read_coordinate(self) -> CoordType:
-        coord = get_coordinate(self.buffer, self.coords_offset, self.coord_type)
-        self.coords_offset += self.bytes_per_coord
-        return coord
-
-    def read_int(self) -> int:
-        value = struct.unpack_from("i", self.buffer, self.ints_offset)[0]
-        if value > len(self.buffer):
-            raise ValueError('Unexpected large integer in structural data')
-        self.ints_offset += 4
-        return value
-
-    def write_int(self, value: int) -> None:
-        struct.pack_into("i", self.buffer, self.ints_offset, value)
-        self.ints_offset += 4
-
-def serialize(geom: BaseGeometry) -> Optional[Union[bytes, bytearray]]:
-    """
-    Serialize a shapely geometry object to the internal representation of GeometryUDT.
-    :param geom: shapely geometry object
-    :return: internal representation of GeometryUDT
-    """
-    if geom is None:
-        return None
-
-    if isinstance(geom, Point):
-        return serialize_point(geom)
-    elif isinstance(geom, LineString):
-        return serialize_linestring(geom)
-    elif isinstance(geom, Polygon):
-        return serialize_polygon(geom)
-    elif isinstance(geom, MultiPoint):
-        return serialize_multi_point(geom)
-    elif isinstance(geom, MultiLineString):
-        return serialize_multi_linestring(geom)
-    elif isinstance(geom, MultiPolygon):
-        return serialize_multi_polygon(geom)
-    elif geom.geom_type == "GeometryCollection":
-        # this check is slightly different due to how shapely
-        # handles empty geometries in version 1.x
-        return serialize_geometry_collection(geom)
-    else:
-        raise ValueError(f"Unsupported geometry type: {type(geom)}")
-
-def deserialize(buffer: bytes) -> Optional[BaseGeometry]:
-    """
-    Deserialize a shapely geometry object from the internal representation of GeometryUDT.
-    :param buffer: internal representation of GeometryUDT
-    :return: shapely geometry object
-    """
-    if buffer is None:
-        return None
-    preamble_byte = buffer[0]
-    geom_type = (preamble_byte >> 4) & 0x0F
-    coord_type = (preamble_byte >> 1) & 0x07
-    num_coords = struct.unpack_from('i', buffer, 4)[0]
-    if num_coords > len(buffer):
-        raise ValueError('num_coords cannot be larger than buffer size')
-    geom_buffer = GeometryBuffer(buffer, coord_type, 8, num_coords)
-    if geom_type == GeometryTypeID.POINT:
-        geom = deserialize_point(geom_buffer)
-    elif geom_type == GeometryTypeID.LINESTRING:
-        geom = deserialize_linestring(geom_buffer)
-    elif geom_type == GeometryTypeID.POLYGON:
-        geom = deserialize_polygon(geom_buffer)
-    elif geom_type == GeometryTypeID.MULTIPOINT:
-        geom = deserialize_multi_point(geom_buffer)
-    elif geom_type == GeometryTypeID.MULTILINESTRING:
-        geom = deserialize_multi_linestring(geom_buffer)
-    elif geom_type == GeometryTypeID.MULTIPOLYGON:
-        geom = deserialize_multi_polygon(geom_buffer)
-    elif geom_type == GeometryTypeID.GEOMETRYCOLLECTION:
-        geom = deserialize_geometry_collection(geom_buffer)
-    else:
-        raise ValueError("Unsupported geometry type ID: {}".format(geom_type))
-    return geom, geom_buffer.ints_offset
-
-
-def create_buffer_for_geom(geom_type: int, coord_type: int, size: int, num_coords: int) -> bytearray:
-    buffer = bytearray(size)
-    preamble_byte = (geom_type << 4) | (coord_type << 1)
-    buffer[0] = preamble_byte
-    struct.pack_into('i', buffer, 4, num_coords)
-    return buffer
-
-def generate_header_bytes(geom_type: int, coord_type: int, num_coords: int) -> bytes:
-    preamble_byte = (geom_type << 4) | (coord_type << 1)
-    return struct.pack(
-        'BBBBi',
-        preamble_byte,
-        0,
-        0,
-        0,
-        num_coords
-    )
-
-
-def put_coordinates(buffer: bytearray, offset: int, coord_type: int, coords: ListCoordType) -> int:
-    for coord in coords:
-        struct.pack_into(CoordinateType.unpack_format(coord_type, buffer, offset, *coord))
-        offset += CoordinateType.bytes_per_coord(coord_type)
-    return offset
-
-
-def put_coordinate(buffer: bytearray, offset: int, coord_type: int, coord: CoordType) -> int:
-    struct.pack_into(CoordinateType.unpack_format(coord_type, buffer, offset, *coord))
-    offset += CoordinateType.bytes_per_coord(coord_type)
-    return offset
-
-
-def get_coordinates(buffer: bytearray, offset: int, coord_type: int, num_coords: int) -> Union[np.ndarray, ListCoordType]:
-    if num_coords < GET_COORDS_NUMPY_THRESHOLD:
-        coords = [
-            struct.unpack_from(CoordinateType.unpack_format(coord_type), buffer, offset + (i * CoordinateType.bytes_per_coord(coord_type)))
-            for i in range(num_coords)
-        ]
-    else:
-        nums_per_coord = CoordinateType.components_per_coord(coord_type)
-        coords = np.frombuffer(buffer, np.float64, num_coords * nums_per_coord, offset).reshape((num_coords, nums_per_coord))
-
-    return coords
-
-
-def get_coordinate(buffer: bytearray, offset: int, coord_type: int) -> CoordType:
-    return struct.unpack_from(CoordinateType.unpack_format(coord_type), buffer, offset)
-
-
-def aligned_offset(offset: int) -> int:
-    return (offset + 7) & ~7
-
-
-def serialize_point(geom: Point) -> bytes:
-    coords = [tuple(c) for c in geom.coords]
-    if coords:
-        # FIXME this does not handle M yet, but geom.has_z is extremely slow
-        pack_format = "BBBBi" + "d" * geom._ndim
-        coord_type = CoordinateType.type_of(geom)
-        preamble_byte = ((GeometryTypeID.POINT << 4) | (coord_type << 1))
-        coords = coords[0]
-        return struct.pack(
-            pack_format,
-            preamble_byte,
-            0,
-            0,
-            0,
-            1,
-            *coords
-        )
-    else:
-        return struct.pack(
-            'BBBBi',
-            18,
-            0,
-            0,
-            0,
-            0
+            # Find geos_c library and load it
+            geos_c_dllpath = find_geos_c_dll()
+            geomserde_speedup.load_libgeos_c(geos_c_dllpath)
+
+        from .geomserde_speedup import serialize
+
+        def deserialize(buf: bytearray) -> Optional[BaseGeometry]:
+            if buf is None:
+                return None
+            return geomserde_speedup.deserialize(buf)
+
+        speedup_enabled = True
+
+    elif shapely.__version__.startswith('1.'):
+        # Shapely 1.x uses ctypes.CDLL to load geos_c library. We can obtain the
+        # handle of geos_c library from `shapely.geos._lgeos._handle`
+        import shapely.geos
+        import shapely.geometry.base
+        from shapely.geometry import (
+            Point,
+            LineString,
+            LinearRing,
+            Polygon,
+            MultiPoint,
+            MultiLineString,
+            MultiPolygon,
+            GeometryCollection
         )
 
-def deserialize_point(geom_buffer: GeometryBuffer) -> Point:
-    if geom_buffer.num_coords == 0:
-        # Here we don't call Point() directly since it would create an empty GeometryCollection
-        # in shapely 1.x. You'll find similar code for creating empty geometries in other
-        # deserialization functions.
-        return wkt_loads("POINT EMPTY")
-    coord = geom_buffer.read_coordinate()
-    return Point(coord)
-
-
-def serialize_multi_point(geom: MultiPoint) -> bytes:
-    points = list(geom.geoms)
-    num_points = len(points)
-    if num_points == 0:
-        return generate_header_bytes(GeometryTypeID.MULTIPOINT, CoordinateType.XY, 0)
-    coord_type = CoordinateType.type_of(geom)
-
-    header = generate_header_bytes(GeometryTypeID.MULTIPOINT, coord_type, num_points)
-    coords = []
-    for point in points:
-        if point.coords:
-            for coord in point.coords[0]:
-                coords.append(coord)
-        else:
-            for k in range(geom._ndim):
-                coords.append(math.nan)
-
-    body = array.array('d', coords).tobytes()
-
-    return header + body
-
-
-def deserialize_multi_point(geom_buffer: GeometryBuffer) -> MultiPoint:
-    if geom_buffer.num_coords == 0:
-        return wkt_loads("MULTIPOINT EMPTY")
-    coords = geom_buffer.read_coordinates(geom_buffer.num_coords)
-    points = []
-    for coord in coords:
-        if math.isnan(coord[0]):
-            # Shapely does not allow creating MultiPoint with empty components
-            pass
-        else:
-            points.append(Point(coord))
-    return MultiPoint(points)
-
-
-def serialize_linestring(geom: LineString) -> bytes:
-    coords = [tuple(c) for c in geom.coords]
-    if coords:
-        coord_type = CoordinateType.type_of(geom)
-        header = generate_header_bytes(GeometryTypeID.LINESTRING, coord_type, len(coords))
-        return header + array.array('d', [x for c in coords for x in c]).tobytes()
-    else:
-        return generate_header_bytes(GeometryTypeID.LINESTRING, 1, 0)
-
-
-def deserialize_linestring(geom_buffer: GeometryBuffer) -> LineString:
-    if geom_buffer.num_coords == 0:
-        return wkt_loads("LINESTRING EMPTY")
-    coords = geom_buffer.read_coordinates(geom_buffer.num_coords)
-    return LineString(coords)
-
-
-def serialize_multi_linestring(geom: MultiLineString) -> bytes:
-    linestrings = list(geom.geoms)
-
-    coord_type = CoordinateType.type_of(geom)
-    lines = [[list(coord) for coord in ls.coords] for ls in linestrings]
-    line_lengths = [len(l) for l in lines]
-    num_coords = sum(line_lengths)
-
-    header = generate_header_bytes(GeometryTypeID.MULTILINESTRING, coord_type, num_coords)
-    coord_data = array.array('d', [c for l in lines for coord in l for c in coord]).tobytes()
-    num_lines = struct.pack('i', len(lines))
-    structure_data = array.array('i', line_lengths).tobytes()
-
-    result = header + coord_data + num_lines + structure_data
-
-    return result
-
-
-def deserialize_multi_linestring(geom_buffer: GeometryBuffer) -> MultiLineString:
-    num_linestrings = geom_buffer.read_int()
-    linestrings = []
-    for k in range(0, num_linestrings):
-        linestring = geom_buffer.read_linestring()
-        if not linestring.is_empty:
-            linestrings.append(linestring)
-    if not linestrings:
-        return wkt_loads("MULTILINESTRING EMPTY")
-    return MultiLineString(linestrings)
-
-def serialize_polygon(geom: Polygon) -> bytes:
-    # it may seem odd, but dumping to wkb and parsing proved to be the fastest here
-    wkb_string = wkb_dumps(geom)
-
-    int_format = ">i" if struct.unpack_from('B', wkb_string) == 0 else "<i"
-    num_rings = struct.unpack_from(int_format, wkb_string, 5)[0]
-
-    if num_rings == 0:
-        return generate_header_bytes(GeometryTypeID.POLYGON, CoordinateType.XY, 0)
-
-    coord_bytes = b''
-    ring_lengths = []
-    offset = 9  #  1 byte endianness + 4 byte geom type + 4 byte num rings
-    bytes_per_coord = geom._ndim * 8
-    num_coords = 0
-
-    for _ in range(num_rings):
-        ring_len = struct.unpack_from(int_format, wkb_string, offset)[0]
-        ring_lengths.append(ring_len)
-        num_coords += ring_len
-        offset += 4
-
-        coord_bytes += wkb_string[offset: (offset + bytes_per_coord * ring_len)]
-        offset += bytes_per_coord * ring_len
-
-    coord_type = CoordinateType.type_of(geom)
-
-    header = generate_header_bytes(GeometryTypeID.POLYGON, coord_type, num_coords)
-    structure_data_bytes = array.array('i', [num_rings] + ring_lengths).tobytes()
-
-    return header + coord_bytes + structure_data_bytes
-
-
-def deserialize_polygon(geom_buffer: GeometryBuffer) -> Polygon:
-    if geom_buffer.num_coords == 0:
-        return wkt_loads("POLYGON EMPTY")
-    return geom_buffer.read_polygon()
-
-
-def serialize_multi_polygon(geom: MultiPolygon) -> bytes:
-    coords_for = lambda x: [y for y in list(x)]
-    polygons = [[coords_for(polygon.exterior.coords)] + [coords_for(ring.coords) for ring in polygon.interiors] for polygon in list(geom.geoms)]
-
-    coord_type = CoordinateType.type_of(geom)
-
-    structure_data = array.array('i', [val for polygon in polygons for val in [len(polygon)] + [len(ring) for ring in polygon]]).tobytes()
-    coords = array.array('d', [component for polygon in polygons for ring in polygon for coord in ring for component in coord]).tobytes()
-
-    num_coords = len(coords) // CoordinateType.bytes_per_coord(coord_type)
-    header = generate_header_bytes(GeometryTypeID.MULTIPOLYGON, coord_type, num_coords)
-
-    num_polygons = struct.pack('i', len(polygons))
-
-    result = header + coords + num_polygons + structure_data
-    return result
-
-
-def deserialize_multi_polygon(geom_buffer: GeometryBuffer) -> MultiPolygon:
-    num_polygons = geom_buffer.read_int()
-    polygons = []
-    for k in range(0, num_polygons):
-        polygon = geom_buffer.read_polygon()
-        if not polygon.is_empty:
-            polygons.append(polygon)
-    if not polygons:
-        return wkt_loads("MULTIPOLYGON EMPTY")
-    return MultiPolygon(polygons)
-
-
-def serialize_geometry_collection(geom: GeometryCollection) -> bytearray:
-    if not isinstance(geom, GeometryCollection):
-        # In shapely 1.x, Empty geometries such as empty points, empty polygons etc.
-        # have geom_type property evaluated to 'GeometryCollection'. Such geometries are not
-        # instances of GeometryCollection and do not have `geom` property.
-        return serialize_shapely_1_empty_geom(geom)
-    geometries = geom.geoms
-    if not geometries:
-        return create_buffer_for_geom(GeometryTypeID.GEOMETRYCOLLECTION, CoordinateType.XY, 8, 0)
-    num_geometries = len(geometries)
-    total_size = 8
-    buffers = []
-    for geom in geometries:
-        buf = serialize(geom)
-        buffers.append(buf)
-        total_size += aligned_offset(len(buf))
-    buffer = create_buffer_for_geom(GeometryTypeID.GEOMETRYCOLLECTION, CoordinateType.XY, total_size, num_geometries)
-    offset = 8
-    for buf in buffers:
-        buffer[offset:(offset + len(buf))] = buf
-        offset += aligned_offset(len(buf))
-    return buffer
+        lgeos_handle = shapely.geos._lgeos._handle
+        geomserde_speedup.load_libgeos_c(lgeos_handle)
+
+        GEOMETRY_CLASSES = [
+            Point,
+            LineString,
+            LinearRing,
+            Polygon,
+            MultiPoint,
+            MultiLineString,
+            MultiPolygon,
+            GeometryCollection,
+        ]
 
+        def serialize(geom: BaseGeometry) -> Optional[bytearray]:
+            if geom is None:
+                return None
+            return geomserde_speedup.serialize_1(geom._geom)
+
+        def deserialize(buf: bytearray) -> Optional[BaseGeometry]:
+            if buf is None:
+                return None
+            g, geom_type_id, has_z, bytes_read = geomserde_speedup.deserialize_1(buf)
+
+            # The following code is mostly taken from the geom_factory function
+            # in shapely/geometry/base.py, with a few tweaks to eliminate
+            # invocations to GEOS functions. We've also replaced direct
+            # attribute reference with __dict__['attr'] to get rid of the extra
+            # cost of __setattr__ in shapely 1.8.
+            if not g:
+                raise ValueError("No Shapely geometry can be created from null value")
+            ob = BaseGeometry()
+            geom_type = shapely.geometry.base.GEOMETRY_TYPES[geom_type_id]
+            ob.__class__ = GEOMETRY_CLASSES[geom_type_id]
+            ob.__dict__['__geom__'] = g
+            ob.__dict__['__p__'] = None
+            if has_z != 0:
+                ob.__dict__['_ndim'] = 3
+            else:
+                ob.__dict__['_ndim'] = 2
+            ob.__dict__['_is_empty'] = False
+            return ob, bytes_read
+
+        speedup_enabled = True
 
-def serialize_shapely_1_empty_geom(geom: BaseGeometry) -> bytearray:
-    total_size = 8
-    if isinstance(geom, Point):
-        geom_type = GeometryTypeID.POINT
-    elif isinstance(geom, LineString):
-        geom_type = GeometryTypeID.LINESTRING
-    elif isinstance(geom, Polygon):
-        geom_type = GeometryTypeID.POLYGON
-    elif isinstance(geom, MultiPoint):
-        geom_type = GeometryTypeID.MULTIPOINT
-    elif isinstance(geom, MultiLineString):
-        geom_type = GeometryTypeID.MULTILINESTRING
-        total_size = 12
-    elif isinstance(geom, MultiPolygon):
-        geom_type = GeometryTypeID.MULTIPOLYGON
-        total_size = 12
     else:
-        raise ValueError("Invalid empty geometry collection object: {}".format(geom))
-    return create_buffer_for_geom(geom_type, CoordinateType.XY, total_size, 0)
-
+        # fallback to our general pure python implementation
+        from .geomserde_general import serialize, deserialize
 
-def deserialize_geometry_collection(geom_buffer: GeometryBuffer) -> GeometryCollection:
-    num_geometries = geom_buffer.num_coords
-    if num_geometries == 0:
-        return wkt_loads("GEOMETRYCOLLECTION EMPTY")
-    geometries = []
-    geom_end_offset = 8
-    buffer = geom_buffer.buffer[8:]
-    for k in range(0, num_geometries):
-        geom, offset = deserialize(buffer)
-        geometries.append(geom)
-        offset = aligned_offset(offset)
-        buffer = buffer[offset:]
-        geom_end_offset += offset
-    geom_buffer.ints_offset = geom_end_offset
-    return GeometryCollection(geometries)
+except Exception as e:
+    warn(f'Cannot load geomserde_speedup, fallback to general python implementation. Reason: {e}')
+    from .geomserde_general import serialize, deserialize
diff --git a/python/sedona/utils/geometry_serde.py b/python/sedona/utils/geometry_serde_general.py
similarity index 100%
copy from python/sedona/utils/geometry_serde.py
copy to python/sedona/utils/geometry_serde_general.py
diff --git a/python/setup.py b/python/setup.py
index 91715697..a6bc7b2c 100644
--- a/python/setup.py
+++ b/python/setup.py
@@ -15,15 +15,30 @@
 # specific language governing permissions and limitations
 # under the License.
 
-from setuptools import setup, find_packages
-from os import path
+from setuptools import setup, find_packages, Extension
+import os
 from sedona import version
 
-here = path.abspath(path.dirname(__file__))
-
 with open("README.md", "r") as fh:
     long_description = fh.read()
 
+extension_args = {}
+
+if os.getenv('ENABLE_ASAN'):
+    extension_args = {
+        'extra_compile_args': ["-fsanitize=address"],
+        'extra_link_args': ["-fsanitize=address"]
+    }
+
+ext_modules = [
+    Extension('sedona.utils.geomserde_speedup', sources=[
+        'src/geomserde_speedup_module.c',
+        'src/geomserde.c',
+        'src/geom_buf.c',
+        'src/geos_c_dyn.c'
+    ], **extension_args)
+]
+
 setup(
     name='apache-sedona',
     version=version,
@@ -33,10 +48,11 @@ setup(
     author='Apache Sedona',
     author_email='dev@sedona.apache.org',
     packages=find_packages(exclude=["*.tests", "*.tests.*", "tests.*", "tests"]),
+    ext_modules=ext_modules,
     long_description=long_description,
     long_description_content_type="text/markdown",
     python_requires='>=3.6',
-    install_requires=['attrs', "shapely"],
+    install_requires=['attrs', "shapely>=1.7.0"],
     extras_require={"spark": ['pyspark>=2.3.0']},
     project_urls={
         'Documentation': 'https://sedona.apache.org',
@@ -48,4 +64,3 @@ setup(
         "License :: OSI Approved :: Apache Software License"
     ]
 )
-
diff --git a/python/src/geom_buf.c b/python/src/geom_buf.c
new file mode 100644
index 00000000..e37fa28b
--- /dev/null
+++ b/python/src/geom_buf.c
@@ -0,0 +1,514 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you 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.
+ */
+
+#include "geom_buf.h"
+
+#include <string.h>
+
+#include "geomserde.h"
+#include "geos_c_dyn.h"
+
+static CoordinateType coordinate_type_of(int has_z, int has_m) {
+  if (has_z && has_m) {
+    return XYZM;
+  } else if (has_z) {
+    return XYZ;
+  } else if (has_m) {
+    return XYM;
+  } else {
+    return XY;
+  }
+}
+
+static unsigned int get_bytes_per_coordinate(CoordinateType coord_type) {
+  switch (coord_type) {
+    case XY:
+      return 16;
+    case XYZ:
+    case XYM:
+      return 24;
+    case XYZM:
+    default:
+      return 32;
+  }
+}
+
+SedonaErrorCode get_coord_seq_info_from_geom(
+    GEOSContextHandle_t handle, const GEOSGeometry *geom,
+    CoordinateSequenceInfo *coord_seq_info) {
+  int dims = dyn_GEOSGeom_getCoordinateDimension_r(handle, geom);
+  if (dims == 0) {
+    return SEDONA_GEOS_ERROR;
+  }
+  int has_z = (dims >= 3);
+  int has_m = 0; /* libgeos does not support M dimension for now */
+  CoordinateType coord_type = coordinate_type_of(has_z, has_m);
+  unsigned int bytes_per_coord = get_bytes_per_coordinate(coord_type);
+  int num_coords = dyn_GEOSGetNumCoordinates_r(handle, geom);
+  if (num_coords == -1) {
+    return SEDONA_GEOS_ERROR;
+  }
+
+  coord_seq_info->dims = dims;
+  coord_seq_info->has_z = has_z;
+  coord_seq_info->has_m = has_m;
+  coord_seq_info->coord_type = coord_type;
+  coord_seq_info->bytes_per_coord = bytes_per_coord;
+  coord_seq_info->num_coords = num_coords;
+  coord_seq_info->total_bytes = bytes_per_coord * num_coords;
+  return SEDONA_SUCCESS;
+}
+
+void *alloc_buffer_for_geom(GeometryTypeId geom_type_id,
+                            CoordinateType coord_type, int srid, int buf_size,
+                            int num_coords) {
+  unsigned char *buf = malloc(buf_size);
+  int *buf_int = (int *)buf;
+  if (buf == NULL) {
+    return buf;
+  }
+  int has_srid = (srid != 0) ? 1 : 0;
+  unsigned char preamble_byte =
+      (geom_type_id << 4) | (coord_type << 1) | has_srid;
+  buf[0] = preamble_byte;
+  buf[1] = srid >> 16;
+  buf[2] = srid >> 8;
+  buf[3] = srid;
+  buf_int[1] = num_coords;
+  return buf;
+}
+
+static SedonaErrorCode copy_coord_seq_to_buffer(
+    GEOSContextHandle_t handle, const GEOSCoordSequence *coord_seq, double *buf,
+    int num_coords, int has_z, int has_m) {
+  if (dyn_GEOSCoordSeq_copyToBuffer_r != NULL) {
+    /* fast path for libgeos >= 3.10.0 */
+    if (dyn_GEOSCoordSeq_copyToBuffer_r(handle, coord_seq, buf, has_z, has_m) ==
+        0) {
+      return SEDONA_GEOS_ERROR;
+    }
+    return SEDONA_SUCCESS;
+  }
+
+  /* slow path for old libgeos */
+  for (int k = 0; k < num_coords; k++) {
+    if (has_z) {
+      double x, y, z;
+      if (dyn_GEOSCoordSeq_getXYZ_r(handle, coord_seq, k, &x, &y, &z) == 0) {
+        return SEDONA_GEOS_ERROR;
+      }
+      *buf++ = x;
+      *buf++ = y;
+      *buf++ = z;
+    } else {
+      double x, y;
+      if (dyn_GEOSCoordSeq_getXY_r(handle, coord_seq, k, &x, &y) == 0) {
+        return SEDONA_GEOS_ERROR;
+      }
+      *buf++ = x;
+      *buf++ = y;
+    }
+    if (has_m) {
+      /* XYZ/XYZM is not supported for now, just fill in 0 for M ordinate as a
+       * fallback value. */
+      *buf++ = 0;
+    }
+  }
+  return SEDONA_SUCCESS;
+}
+
+static SedonaErrorCode copy_buffer_to_coord_seq(
+    GEOSContextHandle_t handle, double *buf, int num_coords, int has_z,
+    int has_m, GEOSCoordSequence **p_coord_seq) {
+  if (dyn_GEOSCoordSeq_copyFromBuffer_r != NULL) {
+    /* fast path for libgeos >= 3.10.0 */
+    GEOSCoordSequence *coord_seq = dyn_GEOSCoordSeq_copyFromBuffer_r(
+        handle, buf, num_coords, has_z, has_m);
+    if (coord_seq == NULL) {
+      return SEDONA_GEOS_ERROR;
+    }
+    *p_coord_seq = coord_seq;
+    return SEDONA_SUCCESS;
+  }
+
+  /* slow path for old libgeos */
+  GEOSCoordSequence *coord_seq =
+      dyn_GEOSCoordSeq_create_r(handle, num_coords, 2 + has_z + has_m);
+  if (coord_seq == NULL) {
+    return SEDONA_GEOS_ERROR;
+  }
+  for (int k = 0; k < num_coords; k++) {
+    double x = *buf++;
+    double y = *buf++;
+    double z;
+    if (has_z) {
+      z = *buf++;
+    }
+    if (has_m) {
+      /* M ordinate is not supported for now, just skip it. */
+      buf++;
+    }
+    if (has_z) {
+      if (dyn_GEOSCoordSeq_setXYZ_r(handle, coord_seq, k, x, y, z) == 0) {
+        dyn_GEOSCoordSeq_destroy_r(handle, coord_seq);
+        return SEDONA_GEOS_ERROR;
+      }
+    } else {
+      if (dyn_GEOSCoordSeq_setXY_r(handle, coord_seq, k, x, y) == 0) {
+        dyn_GEOSCoordSeq_destroy_r(handle, coord_seq);
+        return SEDONA_GEOS_ERROR;
+      }
+    }
+  }
+
+  *p_coord_seq = coord_seq;
+  return SEDONA_SUCCESS;
+}
+
+static void geom_buf_init(GeomBuffer *geom_buf, void *buf,
+                          const CoordinateSequenceInfo *cs_info, int num_coords,
+                          int num_ints) {
+  geom_buf->buf = buf;
+  geom_buf->buf_size = 8 + num_coords * cs_info->bytes_per_coord + 4 * num_ints;
+  geom_buf->buf_coord = (double *)buf + 1;
+  geom_buf->buf_coord_end = geom_buf->buf_coord + num_coords * cs_info->dims;
+  geom_buf->buf_int = (int *)geom_buf->buf_coord_end;
+  geom_buf->buf_int_end = geom_buf->buf_int + num_ints;
+}
+
+SedonaErrorCode geom_buf_alloc(GeomBuffer *geom_buf,
+                               GeometryTypeId geom_type_id, int srid,
+                               const CoordinateSequenceInfo *cs_info,
+                               int num_ints) {
+  int num_coords = cs_info->num_coords;
+  int bytes_per_coord = cs_info->bytes_per_coord;
+  int buf_size = 8 + num_coords * bytes_per_coord + 4 * num_ints;
+  void *buf = alloc_buffer_for_geom(geom_type_id, cs_info->coord_type, srid,
+                                    buf_size, num_coords);
+  if (buf == NULL) {
+    return SEDONA_ALLOC_ERROR;
+  }
+  geom_buf_init(geom_buf, buf, cs_info, num_coords, num_ints);
+  return SEDONA_SUCCESS;
+}
+
+SedonaErrorCode read_geom_buf_header(const char *buf, int buf_size,
+                                     GeomBuffer *geom_buf,
+                                     CoordinateSequenceInfo *cs_info,
+                                     GeometryTypeId *p_geom_type_id,
+                                     int *p_srid) {
+  if (buf_size < 8) {
+    return SEDONA_INCOMPLETE_BUFFER;
+  }
+  unsigned int preamble = (unsigned int)buf[0];
+  int srid = 0;
+  int geom_type_id = preamble >> 4;
+  int coord_type = (preamble & 0x0F) >> 1;
+  if ((preamble & 0x01) != 0) {
+    srid = (((unsigned int)buf[1]) << 16) | (((unsigned int)buf[2]) << 8) |
+           ((unsigned int)buf[3]);
+  }
+  int num_coords = ((int *)buf)[1];
+  if (geom_type_id < 0 || geom_type_id > GEOMETRYCOLLECTION) {
+    return SEDONA_UNKNOWN_GEOM_TYPE;
+  }
+  if (coord_type < 0 || coord_type > XYZM) {
+    return SEDONA_UNKNOWN_COORD_TYPE;
+  }
+  if (num_coords < 0 || num_coords > buf_size) {
+    return SEDONA_BAD_GEOM_BUFFER;
+  }
+
+  int bytes_per_coord = get_bytes_per_coordinate(coord_type);
+  if (geom_type_id != GEOMETRYCOLLECTION) {
+    if (8 + num_coords * bytes_per_coord > buf_size) {
+      return SEDONA_INCOMPLETE_BUFFER;
+    }
+
+    int dims = bytes_per_coord / 8;
+    int has_z = ((coord_type == XYZ || coord_type == XYZM) ? 1 : 0);
+    int has_m = ((coord_type == XYM || coord_type == XYZM) ? 1 : 0);
+    cs_info->bytes_per_coord = bytes_per_coord;
+    cs_info->coord_type = coord_type;
+    cs_info->num_coords = num_coords;
+    cs_info->dims = dims;
+    cs_info->has_z = has_z;
+    cs_info->has_m = has_m;
+
+    geom_buf->buf = (void *)buf;
+    geom_buf->buf_coord = (double *)(buf + 8);
+    geom_buf->buf_coord_end = geom_buf->buf_coord + num_coords * dims;
+    geom_buf->buf_int = (int *)geom_buf->buf_coord_end;
+    geom_buf->buf_int_end = (int *)(buf + buf_size);
+    geom_buf->buf_size = buf_size;
+  } else {
+    /* num_coords for GeometryCollection is number of geometries in the
+     * collection, other fields in cs_info are unused. */
+    cs_info->num_coords = num_coords;
+
+    /* geom_buf contains a series of serialized geometries. buf_coord is the
+     * begining of its first child geometry, and buf_int is unused. */
+    const void *buf_coord = buf + 8;
+    geom_buf->buf = (void *)buf;
+    geom_buf->buf_coord = (double *)(buf_coord);
+    geom_buf->buf_coord_end = (double *)buf_coord;
+    geom_buf->buf_int = (int *)buf_coord;
+    geom_buf->buf_int_end = (int *)buf_coord;
+    geom_buf->buf_size = buf_size;
+  }
+
+  *p_geom_type_id = geom_type_id;
+  *p_srid = srid;
+  return SEDONA_SUCCESS;
+}
+
+SedonaErrorCode geom_buf_write_int(GeomBuffer *geom_buf, int value) {
+  if (geom_buf->buf_int >= geom_buf->buf_int_end) {
+    return SEDONA_INTERNAL_ERROR;
+  }
+  *geom_buf->buf_int++ = value;
+  return SEDONA_SUCCESS;
+}
+
+SedonaErrorCode geom_buf_read_bounded_int(GeomBuffer *geom_buf, int *p_value) {
+  if (geom_buf->buf_int >= geom_buf->buf_int_end) {
+    return SEDONA_INCOMPLETE_BUFFER;
+  }
+  int value = *geom_buf->buf_int++;
+  if (value < 0 || value > geom_buf->buf_size) {
+    return SEDONA_BAD_GEOM_BUFFER;
+  }
+  *p_value = value;
+  return SEDONA_SUCCESS;
+}
+
+SedonaErrorCode geom_buf_write_coords(GeomBuffer *geom_buf,
+                                      GEOSContextHandle_t handle,
+                                      const GEOSCoordSequence *coord_seq,
+                                      const CoordinateSequenceInfo *cs_info) {
+  int num_coords = cs_info->num_coords;
+  if (num_coords == 0) {
+    return SEDONA_SUCCESS;
+  }
+  int num_doubles = num_coords * cs_info->dims;
+  if (geom_buf->buf_coord + num_doubles > geom_buf->buf_coord_end) {
+    return SEDONA_INTERNAL_ERROR;
+  }
+  SedonaErrorCode err = copy_coord_seq_to_buffer(
+      handle, coord_seq, geom_buf->buf_coord, cs_info->num_coords,
+      cs_info->has_z, cs_info->has_m);
+  if (err != SEDONA_SUCCESS) {
+    return err;
+  }
+  geom_buf->buf_coord += num_doubles;
+  return SEDONA_SUCCESS;
+}
+
+SedonaErrorCode geom_buf_read_coords(GeomBuffer *geom_buf,
+                                     GEOSContextHandle_t handle,
+                                     const CoordinateSequenceInfo *cs_info,
+                                     GEOSCoordSequence **p_coord_seq) {
+  int num_coords = cs_info->num_coords;
+  int dims = cs_info->dims;
+  int num_ordinates = num_coords * dims;
+  if (geom_buf->buf_coord + num_ordinates > geom_buf->buf_coord_end) {
+    return SEDONA_INCOMPLETE_BUFFER;
+  }
+  int err =
+      copy_buffer_to_coord_seq(handle, geom_buf->buf_coord, num_coords,
+                               cs_info->has_z, cs_info->has_m, p_coord_seq);
+  if (err != SEDONA_SUCCESS) {
+    return err;
+  }
+  geom_buf->buf_coord += num_ordinates;
+  return SEDONA_SUCCESS;
+}
+
+SedonaErrorCode geom_buf_write_linear_segment(GeomBuffer *geom_buf,
+                                              GEOSContextHandle_t handle,
+                                              const GEOSGeometry *geom,
+                                              CoordinateSequenceInfo *cs_info) {
+  const GEOSCoordSequence *coord_seq = dyn_GEOSGeom_getCoordSeq_r(handle, geom);
+  if (coord_seq == NULL) {
+    return SEDONA_GEOS_ERROR;
+  }
+  unsigned int num_coords = 0;
+  if (dyn_GEOSCoordSeq_getSize_r(handle, coord_seq, &num_coords) == 0) {
+    return SEDONA_GEOS_ERROR;
+  }
+  cs_info->num_coords = num_coords;
+  SedonaErrorCode err =
+      geom_buf_write_coords(geom_buf, handle, coord_seq, cs_info);
+  if (err != SEDONA_SUCCESS) {
+    return err;
+  }
+  err = geom_buf_write_int(geom_buf, num_coords);
+  if (err != SEDONA_SUCCESS) {
+    return err;
+  }
+  return SEDONA_SUCCESS;
+}
+
+SedonaErrorCode geom_buf_read_linear_segment(GeomBuffer *geom_buf,
+                                             GEOSContextHandle_t handle,
+                                             CoordinateSequenceInfo *cs_info,
+                                             int type, GEOSGeometry **p_geom) {
+  SedonaErrorCode err =
+      geom_buf_read_bounded_int(geom_buf, (int *)&cs_info->num_coords);
+  if (err != SEDONA_SUCCESS) {
+    return err;
+  }
+
+  GEOSCoordSequence *coord_seq = NULL;
+  err = geom_buf_read_coords(geom_buf, handle, cs_info, &coord_seq);
+  if (err != SEDONA_SUCCESS) {
+    return err;
+  }
+
+  GEOSGeometry *segment = NULL;
+  switch (type) {
+    case GEOS_LINESTRING:
+      segment = dyn_GEOSGeom_createLineString_r(handle, coord_seq);
+      break;
+    case GEOS_LINEARRING:
+      segment = dyn_GEOSGeom_createLinearRing_r(handle, coord_seq);
+      break;
+    default:
+      return SEDONA_INTERNAL_ERROR;
+  }
+
+  if (segment == NULL) {
+    dyn_GEOSCoordSeq_destroy_r(handle, coord_seq);
+    return SEDONA_GEOS_ERROR;
+  }
+
+  *p_geom = segment;
+  return SEDONA_SUCCESS;
+}
+
+SedonaErrorCode geom_buf_write_polygon(GeomBuffer *geom_buf,
+                                       GEOSContextHandle_t handle,
+                                       const GEOSGeometry *geom,
+                                       CoordinateSequenceInfo *cs_info) {
+  const GEOSGeometry *exterior_ring = dyn_GEOSGetExteriorRing_r(handle, geom);
+  if (exterior_ring == NULL) {
+    return SEDONA_GEOS_ERROR;
+  }
+
+  /* if exterior ring is empty, the serialized polygon is an empty polygon */
+  char is_empty = dyn_GEOSisEmpty_r(handle, exterior_ring);
+  if (is_empty == 2) {
+    return SEDONA_GEOS_ERROR;
+  }
+  if (is_empty == 1) {
+    return geom_buf_write_int(geom_buf, 0);
+  }
+
+  int num_interior_rings = dyn_GEOSGetNumInteriorRings_r(handle, geom);
+  if (num_interior_rings == -1) {
+    return SEDONA_GEOS_ERROR;
+  }
+
+  int num_rings = num_interior_rings + 1;
+  SedonaErrorCode err = geom_buf_write_int(geom_buf, num_rings);
+  if (err != SEDONA_SUCCESS) {
+    return err;
+  }
+
+  /* Write exterior ring */
+  err = geom_buf_write_linear_segment(geom_buf, handle, exterior_ring, cs_info);
+  if (err != SEDONA_SUCCESS) {
+    return err;
+  }
+
+  /* Write interior rings */
+  for (int k = 0; k < num_interior_rings; k++) {
+    const GEOSGeometry *interior_ring =
+        dyn_GEOSGetInteriorRingN_r(handle, geom, k);
+    if (interior_ring == NULL) {
+      return SEDONA_GEOS_ERROR;
+    }
+
+    err =
+        geom_buf_write_linear_segment(geom_buf, handle, interior_ring, cs_info);
+    if (err != SEDONA_SUCCESS) {
+      return err;
+    }
+  }
+
+  return SEDONA_SUCCESS;
+}
+
+SedonaErrorCode geom_buf_read_polygon(GeomBuffer *geom_buf,
+                                      GEOSContextHandle_t handle,
+                                      CoordinateSequenceInfo *cs_info,
+                                      GEOSGeometry **p_geom) {
+  int num_rings = 0;
+  SedonaErrorCode err = geom_buf_read_bounded_int(geom_buf, &num_rings);
+  if (err != SEDONA_SUCCESS) {
+    return err;
+  }
+  if (num_rings == 0) {
+    GEOSGeometry *geom = dyn_GEOSGeom_createEmptyPolygon_r(handle);
+    if (geom == NULL) {
+      return SEDONA_GEOS_ERROR;
+    }
+    *p_geom = geom;
+    return SEDONA_SUCCESS;
+  }
+
+  GEOSGeometry **rings = calloc(num_rings, sizeof(GEOSGeometry *));
+  if (rings == NULL) {
+    return SEDONA_ALLOC_ERROR;
+  }
+  for (int k = 0; k < num_rings; k++) {
+    GEOSGeometry *ring = NULL;
+    err = geom_buf_read_linear_segment(geom_buf, handle, cs_info,
+                                       GEOS_LINEARRING, &ring);
+    if (err != SEDONA_SUCCESS) {
+      goto handle_error;
+    }
+    rings[k] = ring;
+  }
+
+  GEOSGeometry *geom =
+      dyn_GEOSGeom_createPolygon_r(handle, rings[0], &rings[1], num_rings - 1);
+  if (geom == NULL) {
+    err = SEDONA_GEOS_ERROR;
+    goto handle_error;
+  }
+
+  free(rings);
+  *p_geom = geom;
+  return SEDONA_SUCCESS;
+
+handle_error:
+  destroy_geometry_array(handle, rings, num_rings);
+  return err;
+}
+
+void destroy_geometry_array(GEOSContextHandle_t handle, GEOSGeometry **geoms,
+                            int num_geoms) {
+  for (int k = 0; k < num_geoms; k++) {
+    if (geoms[k] != NULL) {
+      dyn_GEOSGeom_destroy_r(handle, geoms[k]);
+    }
+  }
+  free(geoms);
+}
diff --git a/python/src/geom_buf.h b/python/src/geom_buf.h
new file mode 100644
index 00000000..3bf18026
--- /dev/null
+++ b/python/src/geom_buf.h
@@ -0,0 +1,146 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you 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.
+ */
+
+#ifndef GEOM_BUF
+#define GEOM_BUF
+
+#include <stdlib.h>
+
+#include "geomserde.h"
+#include "geos_c_dyn.h"
+
+/*
+ * Constants for identifying geometry dimensions in the serialized buffer of
+ * geometry.
+ */
+typedef enum CoordinateType_enum {
+  XY = 1,
+  XYZ = 2,
+  XYM = 3,
+  XYZM = 4
+} CoordinateType;
+
+/*
+ * Constants for identifying the geometry type in the serialized buffer of
+ * geometry.
+ */
+typedef enum GeometryTypeId_enum {
+  POINT = 1,
+  LINESTRING = 2,
+  POLYGON = 3,
+  MULTIPOINT = 4,
+  MULTILINESTRING = 5,
+  MULTIPOLYGON = 6,
+  GEOMETRYCOLLECTION = 7
+} GeometryTypeId;
+
+/*
+ * Basic info of coordinate sequence retrieved by calling multiple libgeos_c
+ * APIs.
+ */
+typedef struct CoordinateSequenceInfo {
+  unsigned int dims;
+  int has_z;
+  int has_m;
+  CoordinateType coord_type;
+  unsigned int bytes_per_coord;
+  unsigned int num_coords;
+  unsigned int total_bytes;
+} CoordinateSequenceInfo;
+
+/*
+ * Geometry buffer data structure for tracking the reading progress of
+ * coordinate part and offset part separately.
+ */
+typedef struct GeomBuffer {
+  void *buf;
+  int buf_size;
+  double *buf_coord;
+  double *buf_coord_end;
+  int *buf_int;
+  int *buf_int_end;
+} GeomBuffer;
+
+SedonaErrorCode get_coord_seq_info_from_geom(
+    GEOSContextHandle_t handle, const GEOSGeometry *geom,
+    CoordinateSequenceInfo *coord_seq_info);
+
+void *alloc_buffer_for_geom(GeometryTypeId geom_type_id,
+                            CoordinateType coord_type, int srid, int buf_size,
+                            int num_coords);
+
+SedonaErrorCode geom_buf_alloc(GeomBuffer *geom_buf,
+                               GeometryTypeId geom_type_id, int srid,
+                               const CoordinateSequenceInfo *cs_info,
+                               int num_ints);
+
+SedonaErrorCode read_geom_buf_header(const char *buf, int buf_size,
+                                     GeomBuffer *geom_buf,
+                                     CoordinateSequenceInfo *cs_info,
+                                     GeometryTypeId *p_geom_type_id,
+                                     int *p_srid);
+
+SedonaErrorCode geom_buf_write_int(GeomBuffer *geom_buf, int value);
+SedonaErrorCode geom_buf_read_bounded_int(GeomBuffer *geom_buf, int *p_value);
+
+SedonaErrorCode geom_buf_write_coords(GeomBuffer *geom_buf,
+                                      GEOSContextHandle_t handle,
+                                      const GEOSCoordSequence *coord_seq,
+                                      const CoordinateSequenceInfo *cs_info);
+
+SedonaErrorCode geom_buf_read_coords(GeomBuffer *geom_buf,
+                                     GEOSContextHandle_t handle,
+                                     const CoordinateSequenceInfo *cs_info,
+                                     GEOSCoordSequence **p_coord_seq);
+
+SedonaErrorCode geom_buf_write_linear_segment(GeomBuffer *geom_buf,
+                                              GEOSContextHandle_t handle,
+                                              const GEOSGeometry *geom,
+                                              CoordinateSequenceInfo *cs_info);
+
+SedonaErrorCode geom_buf_read_linear_segment(GeomBuffer *geom_buf,
+                                             GEOSContextHandle_t handle,
+                                             CoordinateSequenceInfo *cs_info,
+                                             int type, GEOSGeometry **p_geom);
+
+SedonaErrorCode geom_buf_write_polygon(GeomBuffer *geom_buf,
+                                       GEOSContextHandle_t handle,
+                                       const GEOSGeometry *geom,
+                                       CoordinateSequenceInfo *cs_info);
+
+SedonaErrorCode geom_buf_read_polygon(GeomBuffer *geom_buf,
+                                      GEOSContextHandle_t handle,
+                                      CoordinateSequenceInfo *cs_info,
+                                      GEOSGeometry **p_geom);
+
+void destroy_geometry_array(GEOSContextHandle_t handle, GEOSGeometry **geoms,
+                            int num_geoms);
+
+#define RETURN_BUFFER_FOR_EMPTY_GEOM(geom_type_id, coord_type, srid)         \
+  do {                                                                       \
+    void *buf = alloc_buffer_for_geom(geom_type_id, coord_type, srid, 8, 0); \
+    if (buf == NULL) {                                                       \
+      return SEDONA_ALLOC_ERROR;                                             \
+    }                                                                        \
+    *p_buf = buf;                                                            \
+    *p_buf_size = 8;                                                         \
+    return SEDONA_SUCCESS;                                                   \
+  } while (0)
+
+#endif /* GEOM_BUF */
diff --git a/python/src/geomserde.c b/python/src/geomserde.c
new file mode 100644
index 00000000..18a57714
--- /dev/null
+++ b/python/src/geomserde.c
@@ -0,0 +1,771 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you 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.
+ */
+
+#include "geomserde.h"
+
+#include <math.h>
+#include <stdlib.h>
+#include <string.h>
+
+#include "geom_buf.h"
+#include "geos_c_dyn.h"
+
+static SedonaErrorCode sedona_serialize_point(GEOSContextHandle_t handle,
+                                              const GEOSGeometry *geom,
+                                              int srid,
+                                              CoordinateSequenceInfo *cs_info,
+                                              char **p_buf, int *p_buf_size) {
+  if (cs_info->total_bytes == 0) {
+    RETURN_BUFFER_FOR_EMPTY_GEOM(POINT, cs_info->coord_type, srid);
+  }
+
+  const GEOSCoordSequence *coord_seq = dyn_GEOSGeom_getCoordSeq_r(handle, geom);
+  if (coord_seq == NULL) {
+    return SEDONA_GEOS_ERROR;
+  }
+
+  GeomBuffer geom_buf;
+  SedonaErrorCode err = geom_buf_alloc(&geom_buf, POINT, srid, cs_info, 0);
+  if (err != SEDONA_SUCCESS) {
+    return err;
+  }
+
+  err = geom_buf_write_coords(&geom_buf, handle, coord_seq, cs_info);
+  if (err != SEDONA_SUCCESS) {
+    goto handle_error;
+  }
+
+  *p_buf = geom_buf.buf;
+  *p_buf_size = geom_buf.buf_size;
+  return SEDONA_SUCCESS;
+
+handle_error:
+  free(geom_buf.buf);
+  return SEDONA_GEOS_ERROR;
+}
+
+static SedonaErrorCode sedona_deserialize_point(GEOSContextHandle_t handle,
+                                                int srid, GeomBuffer *geom_buf,
+                                                CoordinateSequenceInfo *cs_info,
+                                                GEOSGeometry **p_geom) {
+  GEOSGeometry *geom = NULL;
+  if (cs_info->num_coords == 0) {
+    geom = dyn_GEOSGeom_createEmptyPoint_r(handle);
+  } else if (cs_info->dims == 2) {
+    /* fast path for 2D points */
+    double x = *geom_buf->buf_coord++;
+    double y = *geom_buf->buf_coord++;
+    geom = dyn_GEOSGeom_createPointFromXY_r(handle, x, y);
+  } else {
+    GEOSCoordSequence *coord_seq = NULL;
+    SedonaErrorCode err =
+        geom_buf_read_coords(geom_buf, handle, cs_info, &coord_seq);
+    if (err != SEDONA_SUCCESS) {
+      return err;
+    }
+    geom = dyn_GEOSGeom_createPoint_r(handle, coord_seq);
+    if (geom == NULL) {
+      dyn_GEOSCoordSeq_destroy_r(handle, coord_seq);
+      return SEDONA_GEOS_ERROR;
+    }
+  }
+
+  *p_geom = geom;
+  return SEDONA_SUCCESS;
+}
+
+static SedonaErrorCode sedona_serialize_linestring(
+    GEOSContextHandle_t handle, const GEOSGeometry *geom, int srid,
+    CoordinateSequenceInfo *cs_info, char **p_buf, int *p_buf_size) {
+  if (cs_info->num_coords == 0) {
+    RETURN_BUFFER_FOR_EMPTY_GEOM(LINESTRING, cs_info->coord_type, srid);
+  }
+
+  const GEOSCoordSequence *coord_seq = dyn_GEOSGeom_getCoordSeq_r(handle, geom);
+  if (coord_seq == NULL) {
+    return SEDONA_GEOS_ERROR;
+  }
+
+  GeomBuffer geom_buf;
+  SedonaErrorCode err = geom_buf_alloc(&geom_buf, LINESTRING, srid, cs_info, 0);
+  if (err != SEDONA_SUCCESS) {
+    return err;
+  }
+
+  err = geom_buf_write_coords(&geom_buf, handle, coord_seq, cs_info);
+  if (err != SEDONA_SUCCESS) {
+    free(geom_buf.buf);
+    return err;
+  }
+
+  *p_buf = geom_buf.buf;
+  *p_buf_size = geom_buf.buf_size;
+  return SEDONA_SUCCESS;
+}
+
+static SedonaErrorCode sedona_deserialize_linestring(
+    GEOSContextHandle_t handle, int srid, GeomBuffer *geom_buf,
+    CoordinateSequenceInfo *cs_info, GEOSGeometry **p_geom) {
+  if (cs_info->num_coords == 0) {
+    GEOSGeometry *geom = dyn_GEOSGeom_createEmptyLineString_r(handle);
+    if (geom == NULL) {
+      return SEDONA_GEOS_ERROR;
+    }
+    *p_geom = geom;
+    return SEDONA_SUCCESS;
+  }
+
+  GEOSCoordSequence *coord_seq = NULL;
+  SedonaErrorCode err =
+      geom_buf_read_coords(geom_buf, handle, cs_info, &coord_seq);
+  if (err != SEDONA_SUCCESS) {
+    return err;
+  }
+
+  GEOSGeometry *geom = dyn_GEOSGeom_createLineString_r(handle, coord_seq);
+  if (geom == NULL) {
+    dyn_GEOSCoordSeq_destroy_r(handle, coord_seq);
+    return SEDONA_GEOS_ERROR;
+  }
+
+  *p_geom = geom;
+  return SEDONA_SUCCESS;
+}
+
+static SedonaErrorCode sedona_serialize_polygon(GEOSContextHandle_t handle,
+                                                const GEOSGeometry *geom,
+                                                int srid,
+                                                CoordinateSequenceInfo *cs_info,
+                                                char **p_buf, int *p_buf_size) {
+  if (cs_info->num_coords == 0) {
+    RETURN_BUFFER_FOR_EMPTY_GEOM(POLYGON, cs_info->coord_type, srid);
+  }
+
+  int num_interior_rings = dyn_GEOSGetNumInteriorRings_r(handle, geom);
+  if (num_interior_rings == -1) {
+    return SEDONA_GEOS_ERROR;
+  }
+
+  GeomBuffer geom_buf;
+  int num_rings = num_interior_rings + 1;
+  SedonaErrorCode err =
+      geom_buf_alloc(&geom_buf, POLYGON, srid, cs_info, num_rings + 1);
+  if (err != SEDONA_SUCCESS) {
+    return err;
+  }
+
+  err = geom_buf_write_polygon(&geom_buf, handle, geom, cs_info);
+  if (err != SEDONA_SUCCESS) {
+    free(geom_buf.buf);
+    return err;
+  }
+
+  *p_buf = geom_buf.buf;
+  *p_buf_size = geom_buf.buf_size;
+  return SEDONA_SUCCESS;
+}
+
+static SedonaErrorCode sedona_deserialize_polygon(
+    GEOSContextHandle_t handle, int srid, GeomBuffer *geom_buf,
+    CoordinateSequenceInfo *cs_info, GEOSGeometry **p_geom) {
+  if (cs_info->num_coords == 0) {
+    GEOSGeometry *geom = dyn_GEOSGeom_createEmptyPolygon_r(handle);
+    if (geom == NULL) {
+      return SEDONA_GEOS_ERROR;
+    }
+    *p_geom = geom;
+    return SEDONA_SUCCESS;
+  }
+  return geom_buf_read_polygon(geom_buf, handle, cs_info, p_geom);
+}
+
+static SedonaErrorCode sedona_serialize_multipoint(
+    GEOSContextHandle_t handle, const GEOSGeometry *geom, int srid,
+    CoordinateSequenceInfo *cs_info, char **p_buf, int *p_buf_size) {
+  int num_points = dyn_GEOSGetNumGeometries_r(handle, geom);
+  if (num_points == -1) {
+    return SEDONA_GEOS_ERROR;
+  }
+
+  /* cs_info->num_coords will be smaller than actual number of serialized
+   * coordinates when there're empty points in the multipoint, so let's fix
+   * it. */
+  cs_info->num_coords = num_points;
+
+  GeomBuffer geom_buf;
+  SedonaErrorCode err = geom_buf_alloc(&geom_buf, MULTIPOINT, srid, cs_info, 0);
+  if (err != SEDONA_SUCCESS) {
+    return err;
+  }
+
+  for (int k = 0; k < num_points; k++) {
+    const GEOSGeometry *point = dyn_GEOSGetGeometryN_r(handle, geom, k);
+    if (point == NULL) {
+      err = SEDONA_GEOS_ERROR;
+      goto handle_error;
+    }
+
+    const GEOSCoordSequence *coord_seq =
+        dyn_GEOSGeom_getCoordSeq_r(handle, point);
+    if (coord_seq == NULL) {
+      err = SEDONA_GEOS_ERROR;
+      goto handle_error;
+    }
+
+    /* check if this point is empty */
+    unsigned int num_coords = 0;
+    if (dyn_GEOSCoordSeq_getSize_r(handle, coord_seq, &num_coords) == 0) {
+      err = SEDONA_GEOS_ERROR;
+      goto handle_error;
+    }
+
+    if (num_coords == 1) {
+      /* non-empty point */
+      cs_info->num_coords = 1;
+      err = geom_buf_write_coords(&geom_buf, handle, coord_seq, cs_info);
+      if (err != SEDONA_SUCCESS) {
+        goto handle_error;
+      }
+    } else {
+      /* point is empty, we have to manually write NaNs */
+      *geom_buf.buf_coord++ = NAN;
+      *geom_buf.buf_coord++ = NAN;
+      if (cs_info->has_z) {
+        *geom_buf.buf_coord++ = NAN;
+      }
+      if (cs_info->has_m) {
+        *geom_buf.buf_coord++ = NAN;
+      }
+    }
+  }
+
+  *p_buf = geom_buf.buf;
+  *p_buf_size = geom_buf.buf_size;
+  return SEDONA_SUCCESS;
+
+handle_error:
+  free(geom_buf.buf);
+  return err;
+}
+
+static SedonaErrorCode sedona_deserialize_multipoint(
+    GEOSContextHandle_t handle, int srid, GeomBuffer *geom_buf,
+    CoordinateSequenceInfo *cs_info, GEOSGeometry **p_geom) {
+  int num_points = cs_info->num_coords;
+  GEOSGeometry **points = calloc(num_points, sizeof(GEOSGeometry *));
+  if (points == NULL) {
+    return SEDONA_ALLOC_ERROR;
+  }
+
+  SedonaErrorCode err = SEDONA_SUCCESS;
+  for (int k = 0; k < num_points; k++) {
+    GEOSGeometry *point = NULL;
+    if (cs_info->dims == 2) {
+      /* fast path for 2D points. We can get rid of constructing a coordinate
+       * sequence object explicitly */
+      double x = *geom_buf->buf_coord++;
+      double y = *geom_buf->buf_coord++;
+      /* x and y will be NaN when serialized point was an empty point. GEOS
+       * will treat point with Nan ordinates as an empty point so we don't need
+       * to handle NaN specially. */
+      point = dyn_GEOSGeom_createPointFromXY_r(handle, x, y);
+      if (point == NULL) {
+        err = SEDONA_GEOS_ERROR;
+        goto handle_error;
+      }
+    } else {
+      GEOSCoordSequence *coord_seq = NULL;
+      cs_info->num_coords = 1;
+      err = geom_buf_read_coords(geom_buf, handle, cs_info, &coord_seq);
+      if (err != SEDONA_SUCCESS) {
+        goto handle_error;
+      }
+      point = dyn_GEOSGeom_createPoint_r(handle, coord_seq);
+      if (point == NULL) {
+        dyn_GEOSCoordSeq_destroy_r(handle, coord_seq);
+        err = SEDONA_GEOS_ERROR;
+        goto handle_error;
+      }
+    }
+    points[k] = point;
+  }
+
+  GEOSGeometry *geom = dyn_GEOSGeom_createCollection_r(handle, GEOS_MULTIPOINT,
+                                                       points, num_points);
+  if (geom == NULL) {
+    err = SEDONA_GEOS_ERROR;
+    goto handle_error;
+  }
+
+  free(points);
+  *p_geom = geom;
+  return SEDONA_SUCCESS;
+
+handle_error:
+  destroy_geometry_array(handle, points, num_points);
+  return err;
+}
+
+static SedonaErrorCode sedona_serialize_multilinestring(
+    GEOSContextHandle_t handle, const GEOSGeometry *geom, int srid,
+    CoordinateSequenceInfo *cs_info, char **p_buf, int *p_buf_size) {
+  int num_geoms = dyn_GEOSGetNumGeometries_r(handle, geom);
+  if (num_geoms == -1) {
+    return SEDONA_GEOS_ERROR;
+  }
+
+  GeomBuffer geom_buf;
+  SedonaErrorCode err =
+      geom_buf_alloc(&geom_buf, MULTILINESTRING, srid, cs_info, num_geoms + 1);
+  if (err != SEDONA_SUCCESS) {
+    return err;
+  }
+
+  if ((err = geom_buf_write_int(&geom_buf, num_geoms)) != SEDONA_SUCCESS) {
+    return err;
+  }
+  for (int k = 0; k < num_geoms; k++) {
+    const GEOSGeometry *linestring = dyn_GEOSGetGeometryN_r(handle, geom, k);
+    if (linestring == NULL) {
+      err = SEDONA_GEOS_ERROR;
+      goto handle_error;
+    }
+
+    err = geom_buf_write_linear_segment(&geom_buf, handle, linestring, cs_info);
+    if (err != SEDONA_SUCCESS) {
+      goto handle_error;
+    }
+  }
+
+  *p_buf = geom_buf.buf;
+  *p_buf_size = geom_buf.buf_size;
+  return SEDONA_SUCCESS;
+
+handle_error:
+  free(geom_buf.buf);
+  return err;
+}
+
+static SedonaErrorCode sedona_deserialize_multilinestring(
+    GEOSContextHandle_t handle, int srid, GeomBuffer *geom_buf,
+    CoordinateSequenceInfo *cs_info, GEOSGeometry **p_geom) {
+  int num_geoms = 0;
+  SedonaErrorCode err = geom_buf_read_bounded_int(geom_buf, &num_geoms);
+  if (err != SEDONA_SUCCESS) {
+    return err;
+  }
+
+  GEOSGeometry **linestrings = calloc(num_geoms, sizeof(GEOSGeometry *));
+  for (int k = 0; k < num_geoms; k++) {
+    GEOSGeometry *linestring = NULL;
+    if ((err = geom_buf_read_linear_segment(geom_buf, handle, cs_info,
+                                            GEOS_LINESTRING, &linestring)) !=
+        SEDONA_SUCCESS) {
+      goto handle_error;
+    }
+    linestrings[k] = linestring;
+  }
+
+  GEOSGeometry *geom = dyn_GEOSGeom_createCollection_r(
+      handle, GEOS_MULTILINESTRING, linestrings, num_geoms);
+  if (geom == NULL) {
+    err = SEDONA_GEOS_ERROR;
+    goto handle_error;
+  }
+
+  free(linestrings);
+  *p_geom = geom;
+  return SEDONA_SUCCESS;
+
+handle_error:
+  destroy_geometry_array(handle, linestrings, num_geoms);
+  return err;
+}
+
+static SedonaErrorCode sedona_serialize_multipolygon(
+    GEOSContextHandle_t handle, const GEOSGeometry *geom, int srid,
+    CoordinateSequenceInfo *cs_info, char **p_buf, int *p_buf_size) {
+  int num_geoms = dyn_GEOSGetNumGeometries_r(handle, geom);
+  if (num_geoms == -1) {
+    return SEDONA_GEOS_ERROR;
+  }
+
+  /* collect size of structural data */
+  int num_rings = 0;
+  for (int k = 0; k < num_geoms; k++) {
+    const GEOSGeometry *polygon = dyn_GEOSGetGeometryN_r(handle, geom, k);
+    if (polygon == NULL) {
+      return SEDONA_GEOS_ERROR;
+    }
+    int num_interior_rings = dyn_GEOSGetNumInteriorRings_r(handle, polygon);
+    if (num_interior_rings == -1) {
+      return SEDONA_GEOS_ERROR;
+    }
+    if (num_interior_rings > 0) {
+      num_rings += (num_interior_rings + 1);
+    } else {
+      /* check if polygon is empty */
+      char is_empty = dyn_GEOSisEmpty_r(handle, polygon);
+      if (is_empty == 2) {
+        return SEDONA_GEOS_ERROR;
+      }
+      num_rings += (is_empty == 1 ? 0 : 1);
+    }
+  }
+
+  GeomBuffer geom_buf;
+  SedonaErrorCode err = geom_buf_alloc(&geom_buf, MULTIPOLYGON, srid, cs_info,
+                                       1 + num_geoms + num_rings);
+  if (err != SEDONA_SUCCESS) {
+    return err;
+  }
+  if ((err = geom_buf_write_int(&geom_buf, num_geoms)) != SEDONA_SUCCESS) {
+    return err;
+  }
+  for (int k = 0; k < num_geoms; k++) {
+    const GEOSGeometry *polygon = dyn_GEOSGetGeometryN_r(handle, geom, k);
+    if (polygon == NULL) {
+      err = SEDONA_GEOS_ERROR;
+      goto handle_error;
+    }
+
+    if ((err = geom_buf_write_polygon(&geom_buf, handle, polygon, cs_info)) !=
+        SEDONA_SUCCESS) {
+      goto handle_error;
+    }
+  }
+
+  *p_buf = geom_buf.buf;
+  *p_buf_size = geom_buf.buf_size;
+  return SEDONA_SUCCESS;
+
+handle_error:
+  free(geom_buf.buf);
+  return err;
+}
+
+static SedonaErrorCode sedona_deserialize_multipolygon(
+    GEOSContextHandle_t handle, int srid, GeomBuffer *geom_buf,
+    CoordinateSequenceInfo *cs_info, GEOSGeometry **p_geom) {
+  int num_geoms = 0;
+  SedonaErrorCode err = geom_buf_read_bounded_int(geom_buf, &num_geoms);
+  if (err != SEDONA_SUCCESS) {
+    return err;
+  }
+
+  GEOSGeometry **polygons = calloc(num_geoms, sizeof(GEOSGeometry *));
+  for (int k = 0; k < num_geoms; k++) {
+    GEOSGeometry *polygon = NULL;
+    if ((err = geom_buf_read_polygon(geom_buf, handle, cs_info, &polygon)) !=
+        SEDONA_SUCCESS) {
+      goto handle_error;
+    }
+    polygons[k] = polygon;
+  }
+
+  GEOSGeometry *geom = dyn_GEOSGeom_createCollection_r(handle, MULTIPOLYGON,
+                                                       polygons, num_geoms);
+  if (geom == NULL) {
+    err = SEDONA_GEOS_ERROR;
+    goto handle_error;
+  }
+
+  free(polygons);
+  *p_geom = geom;
+  return SEDONA_SUCCESS;
+
+handle_error:
+  destroy_geometry_array(handle, polygons, num_geoms);
+  return err;
+}
+
+static inline int aligned_offset(int offset) { return (offset + 7) & ~7; }
+
+static SedonaErrorCode sedona_serialize_geometrycollection(
+    GEOSContextHandle_t handle, const GEOSGeometry *geom, int srid,
+    char **p_buf, int *p_buf_size) {
+  int num_geoms = dyn_GEOSGetNumGeometries_r(handle, geom);
+  if (num_geoms == -1) {
+    return SEDONA_GEOS_ERROR;
+  }
+  int total_size = 8;
+  unsigned char *scratch = calloc(num_geoms, sizeof(char *) + sizeof(int));
+  if (scratch == NULL) {
+    return SEDONA_ALLOC_ERROR;
+  }
+  char **geom_bufs = (char **)scratch;
+  int *geom_buf_sizes = (int *)(scratch + num_geoms * sizeof(char *));
+  SedonaErrorCode err = SEDONA_SUCCESS;
+
+  /* Serialize geometries individually */
+  for (int k = 0; k < num_geoms; k++) {
+    const GEOSGeometry *child_geom = dyn_GEOSGetGeometryN_r(handle, geom, k);
+    if (child_geom == NULL) {
+      err = SEDONA_GEOS_ERROR;
+      goto handle_error;
+    }
+
+    char *buf = NULL;
+    int buf_size = 0;
+    err = sedona_serialize_geom(handle, child_geom, &buf, &buf_size);
+    if (err != SEDONA_SUCCESS) {
+      goto handle_error;
+    }
+    geom_bufs[k] = buf;
+    geom_buf_sizes[k] = buf_size;
+    total_size += aligned_offset(buf_size);
+  }
+
+  char *buf = alloc_buffer_for_geom(GEOMETRYCOLLECTION, XY, srid, total_size,
+                                    num_geoms);
+  if (buf == NULL) {
+    err = SEDONA_ALLOC_ERROR;
+    goto handle_error;
+  }
+
+  char *p_next_geom = buf + 8;
+  for (int k = 0; k < num_geoms; k++) {
+    int buf_size = geom_buf_sizes[k];
+    memcpy(p_next_geom, geom_bufs[k], buf_size);
+    free(geom_bufs[k]);
+    int padding_size = aligned_offset(buf_size) - buf_size;
+    p_next_geom += buf_size;
+    if (padding_size > 0) {
+      memset(p_next_geom, 0, padding_size);
+      p_next_geom += padding_size;
+    }
+  }
+
+  free(scratch);
+  *p_buf = buf;
+  *p_buf_size = total_size;
+  return SEDONA_SUCCESS;
+
+handle_error:
+  for (int k = 0; k < num_geoms; k++) {
+    free(geom_bufs[k]);
+  }
+  free(scratch);
+  return err;
+}
+
+static SedonaErrorCode deserialize_geom_buf(GEOSContextHandle_t handle,
+                                            GeometryTypeId geom_type_id,
+                                            int srid, GeomBuffer *geom_buf,
+                                            CoordinateSequenceInfo *cs_info,
+                                            GEOSGeometry **p_geom);
+
+static SedonaErrorCode sedona_deserialize_geometrycollection(
+    GEOSContextHandle_t handle, int srid, GeomBuffer *geom_buf,
+    CoordinateSequenceInfo *cs_info, GEOSGeometry **p_geom) {
+  SedonaErrorCode err = SEDONA_SUCCESS;
+  int num_geoms = cs_info->num_coords;
+  GEOSGeometry **child_geoms = calloc(num_geoms, sizeof(GEOSGeometry *));
+  if (child_geoms == NULL) {
+    return SEDONA_ALLOC_ERROR;
+  }
+
+  const char *buf = (const char *)geom_buf->buf + 8;
+  int remaining_size = geom_buf->buf_size - 8;
+  for (int k = 0; k < num_geoms; k++) {
+    GEOSGeometry *child_geom = NULL;
+    int bytes_read = 0;
+    err = sedona_deserialize_geom(handle, buf, remaining_size, &child_geom,
+                                  &bytes_read);
+    if (err != SEDONA_SUCCESS) {
+      goto handle_error;
+    }
+
+    child_geoms[k] = child_geom;
+
+    bytes_read = aligned_offset(bytes_read);
+    if (remaining_size < bytes_read) {
+      err = SEDONA_INCOMPLETE_BUFFER;
+      goto handle_error;
+    }
+    remaining_size -= bytes_read;
+    buf += bytes_read;
+  }
+
+  GEOSGeometry *geom_collection = dyn_GEOSGeom_createCollection_r(
+      handle, GEOS_GEOMETRYCOLLECTION, child_geoms, num_geoms);
+  if (geom_collection == NULL) {
+    err = SEDONA_GEOS_ERROR;
+    goto handle_error;
+  }
+
+  free(child_geoms);
+  *p_geom = geom_collection;
+
+  /* set geom_buf.buf_int to mark the end of the buffer for this geometry
+   * collection */
+  geom_buf->buf_int = (int *)buf;
+  return SEDONA_SUCCESS;
+
+handle_error:
+  destroy_geometry_array(handle, child_geoms, num_geoms);
+  return err;
+}
+
+SedonaErrorCode sedona_serialize_geom(GEOSContextHandle_t handle,
+                                      const GEOSGeometry *geom, char **p_buf,
+                                      int *p_buf_size) {
+  int srid = dyn_GEOSGetSRID_r(handle, geom);
+  int geom_type_id = dyn_GEOSGeomTypeId_r(handle, geom);
+  if (geom_type_id == GEOS_GEOMETRYCOLLECTION) {
+    return sedona_serialize_geometrycollection(handle, geom, srid, p_buf,
+                                               p_buf_size);
+  }
+
+  CoordinateSequenceInfo cs_info;
+  SedonaErrorCode err = get_coord_seq_info_from_geom(handle, geom, &cs_info);
+  if (err != SEDONA_SUCCESS) {
+    return err;
+  }
+
+  switch (geom_type_id) {
+    case GEOS_POINT:
+      err = sedona_serialize_point(handle, geom, srid, &cs_info, p_buf,
+                                   p_buf_size);
+      break;
+    case GEOS_LINESTRING:
+      err = sedona_serialize_linestring(handle, geom, srid, &cs_info, p_buf,
+                                        p_buf_size);
+      break;
+    case GEOS_LINEARRING:
+      err = SEDONA_UNSUPPORTED_GEOM_TYPE;
+      break;
+    case GEOS_POLYGON:
+      err = sedona_serialize_polygon(handle, geom, srid, &cs_info, p_buf,
+                                     p_buf_size);
+      break;
+    case GEOS_MULTIPOINT:
+      err = sedona_serialize_multipoint(handle, geom, srid, &cs_info, p_buf,
+                                        p_buf_size);
+      break;
+    case GEOS_MULTILINESTRING:
+      err = sedona_serialize_multilinestring(handle, geom, srid, &cs_info,
+                                             p_buf, p_buf_size);
+      break;
+    case GEOS_MULTIPOLYGON:
+      err = sedona_serialize_multipolygon(handle, geom, srid, &cs_info, p_buf,
+                                          p_buf_size);
+      break;
+    default:
+      err = SEDONA_UNKNOWN_GEOM_TYPE;
+  }
+
+  return err;
+}
+
+static SedonaErrorCode deserialize_geom_buf(GEOSContextHandle_t handle,
+                                            GeometryTypeId geom_type_id,
+                                            int srid, GeomBuffer *geom_buf,
+                                            CoordinateSequenceInfo *cs_info,
+                                            GEOSGeometry **p_geom) {
+  SedonaErrorCode err = SEDONA_SUCCESS;
+  switch (geom_type_id) {
+    case POINT:
+      err = sedona_deserialize_point(handle, srid, geom_buf, cs_info, p_geom);
+      break;
+    case LINESTRING:
+      err = sedona_deserialize_linestring(handle, srid, geom_buf, cs_info,
+                                          p_geom);
+      break;
+    case POLYGON:
+      err = sedona_deserialize_polygon(handle, srid, geom_buf, cs_info, p_geom);
+      break;
+    case MULTIPOINT:
+      err = sedona_deserialize_multipoint(handle, srid, geom_buf, cs_info,
+                                          p_geom);
+      break;
+    case MULTILINESTRING:
+      err = sedona_deserialize_multilinestring(handle, srid, geom_buf, cs_info,
+                                               p_geom);
+      break;
+    case MULTIPOLYGON:
+      err = sedona_deserialize_multipolygon(handle, srid, geom_buf, cs_info,
+                                            p_geom);
+      break;
+    case GEOMETRYCOLLECTION:
+      err = sedona_deserialize_geometrycollection(handle, srid, geom_buf,
+                                                  cs_info, p_geom);
+      break;
+    default:
+      return SEDONA_UNSUPPORTED_GEOM_TYPE;
+  }
+
+  if (err != SEDONA_SUCCESS) {
+    return err;
+  }
+  if (srid != 0) {
+    dyn_GEOSSetSRID_r(handle, *p_geom, srid);
+  }
+  return SEDONA_SUCCESS;
+}
+
+SedonaErrorCode sedona_deserialize_geom(GEOSContextHandle_t handle,
+                                        const char *buf, int buf_size,
+                                        GEOSGeometry **p_geom,
+                                        int *p_bytes_read) {
+  GeomBuffer geom_buf;
+  CoordinateSequenceInfo cs_info;
+  GeometryTypeId geom_type_id;
+  int srid;
+  SedonaErrorCode err = read_geom_buf_header(buf, buf_size, &geom_buf, &cs_info,
+                                             &geom_type_id, &srid);
+  if (err != SEDONA_SUCCESS) {
+    return err;
+  }
+
+  err = deserialize_geom_buf(handle, geom_type_id, srid, &geom_buf, &cs_info,
+                             p_geom);
+  if (err != SEDONA_SUCCESS) {
+    return err;
+  }
+
+  *p_bytes_read =
+      (int)((unsigned char *)geom_buf.buf_int - (unsigned char *)geom_buf.buf);
+  return SEDONA_SUCCESS;
+}
+
+const char *sedona_get_error_message(int err) {
+  switch (err) {
+    case SEDONA_SUCCESS:
+      return "";
+    case SEDONA_UNKNOWN_GEOM_TYPE:
+      return "Unknown geometry type";
+    case SEDONA_UNKNOWN_COORD_TYPE:
+      return "Unknown coordinate type";
+    case SEDONA_UNSUPPORTED_GEOM_TYPE:
+      return "Unsupported geometry type";
+    case SEDONA_INCOMPLETE_BUFFER:
+      return "Buffer to be deserialized is incomplete";
+    case SEDONA_BAD_GEOM_BUFFER:
+      return "Bad serialized geometry buffer";
+    case SEDONA_GEOS_ERROR:
+      return "GEOS error";
+    case SEDONA_ALLOC_ERROR:
+      return "Out of memory";
+    case SEDONA_INTERNAL_ERROR:
+      return "Internal error";
+    default:
+      return "Unknown failure occurred";
+  }
+}
diff --git a/python/src/geomserde.h b/python/src/geomserde.h
new file mode 100644
index 00000000..f205b619
--- /dev/null
+++ b/python/src/geomserde.h
@@ -0,0 +1,76 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you 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.
+ */
+
+#ifndef GEOM_SERDE
+#define GEOM_SERDE
+
+#include "geos_c_dyn.h"
+
+/**
+ * Error code for internal APIs of the sedona python extension module.
+ */
+typedef enum SedonaErrorCode {
+  SEDONA_SUCCESS,
+  SEDONA_UNKNOWN_GEOM_TYPE,
+  SEDONA_UNKNOWN_COORD_TYPE,
+  SEDONA_UNSUPPORTED_GEOM_TYPE,
+  SEDONA_INCOMPLETE_BUFFER,
+  SEDONA_BAD_GEOM_BUFFER,
+  SEDONA_GEOS_ERROR,
+  SEDONA_ALLOC_ERROR,
+  SEDONA_INTERNAL_ERROR,
+} SedonaErrorCode;
+
+/**
+ * Converts error code to human readable error message
+ *
+ * @param err error code
+ * @return error message
+ */
+extern const char *sedona_get_error_message(int err);
+
+/**
+ * Serializes a GEOS geometry object as a binary buffer
+ *
+ * @param handle the GEOS context handle
+ * @param geom The GEOS geometry object to serialize
+ * @param p_buf OUTPUT parameter for receiving the pointer to buffer
+ * @param p_buf_size OUTPUT parameter for receiving size of the buffer
+ * @return error code
+ */
+SedonaErrorCode sedona_serialize_geom(GEOSContextHandle_t handle,
+                                      const GEOSGeometry *geom, char **p_buf,
+                                      int *p_buf_size);
+
+/**
+ * Deserializes a serialized geometry to a GEOS geometry object
+ *
+ * @param handle GEOS context handle
+ * @param buf buffer containing serialized geometry
+ * @param buf_size size of the buffer
+ * @param p_geom OUTPUT parameter for receiving deserialized GEOS geometry
+ * @param p_bytes_read OUTPUT parameter for receiving number of bytes read
+ * @return error code
+ */
+SedonaErrorCode sedona_deserialize_geom(GEOSContextHandle_t handle,
+                                        const char *buf, int buf_size,
+                                        GEOSGeometry **p_geom,
+                                        int *p_bytes_read);
+
+#endif /* GEOM_SERDE */
diff --git a/python/src/geomserde_speedup_module.c b/python/src/geomserde_speedup_module.c
new file mode 100644
index 00000000..e49efb4f
--- /dev/null
+++ b/python/src/geomserde_speedup_module.c
@@ -0,0 +1,288 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you 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.
+ */
+
+#define PY_SSIZE_T_CLEAN
+#include <Python.h>
+#include <stdio.h>
+
+#include "geomserde.h"
+#include "geos_c_dyn.h"
+#include "pygeos/c_api.h"
+
+PyDoc_STRVAR(module_doc, "Geometry serialization/deserialization module.");
+
+#define ERR_MSG_BUF_SIZE 1024
+
+#ifdef __GNUC__
+#define thread_local __thread
+#elif __STDC_VERSION__ >= 201112L
+#define thread_local _Thread_local
+#elif defined(_MSC_VER)
+#define thread_local __declspec(thread)
+#else
+#error Cannot define thread_local
+#endif
+
+static PyObject *load_libgeos_c(PyObject *self, PyObject *args) {
+  PyObject *obj;
+  char err_msg[ERR_MSG_BUF_SIZE];
+  if (!PyArg_ParseTuple(args, "O", &obj)) {
+    return NULL;
+  }
+
+  if (PyLong_Check(obj)) {
+    unsigned long long handle = PyLong_AsUnsignedLongLong(obj);
+    if (load_geos_c_from_handle((void *)handle, err_msg, sizeof(err_msg)) !=
+        0) {
+      PyErr_Format(PyExc_RuntimeError, "Failed to find libgeos_c functions: %s",
+                   err_msg);
+      return NULL;
+    }
+  } else if (PyUnicode_Check(obj)) {
+    const char *libname = PyUnicode_AsUTF8(obj);
+    if (load_geos_c_library(libname, err_msg, sizeof(err_msg)) != 0) {
+      PyErr_Format(PyExc_RuntimeError, "Failed to find libgeos_c functions: %s",
+                   err_msg);
+      return NULL;
+    }
+  } else {
+    PyErr_SetString(PyExc_TypeError,
+                    "load_libgeos_c expects a string or long argument");
+    return NULL;
+  }
+
+  Py_INCREF(Py_None);
+  return Py_None;
+}
+
+static thread_local GEOSContextHandle_t handle;
+static thread_local char *geos_err_msg;
+
+static void geos_msg_handler(const char *fmt, ...) {
+  if (geos_err_msg == NULL) return;
+  va_list ap;
+  va_start(ap, fmt);
+  vprintf(fmt, ap);
+  vsnprintf(geos_err_msg, ERR_MSG_BUF_SIZE, fmt, ap);
+  va_end(ap);
+}
+
+static GEOSContextHandle_t get_geos_context_handle() {
+  if (handle == NULL) {
+    if (!is_geos_c_loaded()) {
+      PyErr_SetString(
+          PyExc_RuntimeError,
+          "libgeos_c was not loaded, please call load_libgeos_c first");
+      return NULL;
+    }
+
+    handle = dyn_GEOS_init_r();
+    if (handle == NULL) {
+      goto oom_failure;
+    }
+    geos_err_msg = malloc(ERR_MSG_BUF_SIZE);
+    if (geos_err_msg == NULL) {
+      goto oom_failure;
+    }
+    dyn_GEOSContext_setErrorHandler_r(handle, geos_msg_handler);
+  }
+
+  geos_err_msg[0] = '\0';
+  return handle;
+
+oom_failure:
+  PyErr_NoMemory();
+  if (handle != NULL) {
+    dyn_GEOS_finish_r(handle);
+    handle = NULL;
+  }
+  if (geos_err_msg != NULL) {
+    free(geos_err_msg);
+    geos_err_msg = NULL;
+  }
+  return NULL;
+}
+
+static void handle_geomserde_error(SedonaErrorCode err) {
+  const char *errmsg = sedona_get_error_message(err);
+  if (err == SEDONA_ALLOC_ERROR) {
+    PyErr_NoMemory();
+  } else if (err == SEDONA_INTERNAL_ERROR) {
+    PyErr_Format(PyExc_RuntimeError, "%s", errmsg);
+  } else if (err == SEDONA_GEOS_ERROR) {
+    const char *errmsg = sedona_get_error_message(err);
+    PyErr_Format(PyExc_RuntimeError, "%s: %s", errmsg, geos_err_msg);
+  } else {
+    const char *errmsg = sedona_get_error_message(err);
+    PyErr_Format(PyExc_ValueError, "%s", errmsg);
+  }
+}
+
+static PyObject *do_serialize(GEOSGeometry *geos_geom) {
+  if (geos_geom == NULL) {
+    Py_INCREF(Py_None);
+    return Py_None;
+  }
+
+  GEOSContextHandle_t handle = get_geos_context_handle();
+  if (handle == NULL) {
+    return NULL;
+  }
+
+  char *buf = NULL;
+  int buf_size = 0;
+  SedonaErrorCode err =
+      sedona_serialize_geom(handle, geos_geom, &buf, &buf_size);
+  if (err != SEDONA_SUCCESS) {
+    handle_geomserde_error(err);
+    return NULL;
+  }
+
+  PyObject *bytearray = PyByteArray_FromStringAndSize(buf, buf_size);
+  free(buf);
+  return bytearray;
+}
+
+static GEOSGeometry *do_deserialize(PyObject *args,
+                                    GEOSContextHandle_t *out_handle,
+                                    int *p_bytes_read) {
+  Py_buffer view;
+  if (!PyArg_ParseTuple(args, "y*", &view)) {
+    return NULL;
+  }
+
+  GEOSContextHandle_t handle = get_geos_context_handle();
+  if (handle == NULL) {
+    return NULL;
+  }
+
+  /* The Py_buffer filled by PyArg_ParseTuple is guaranteed to be C-contiguous,
+   * so we can simply proceed with view.buf and view.len */
+  const char *buf = view.buf;
+  int buf_size = view.len;
+  GEOSGeometry *geom = NULL;
+  SedonaErrorCode err =
+      sedona_deserialize_geom(handle, buf, buf_size, &geom, p_bytes_read);
+  PyBuffer_Release(&view);
+  if (err != SEDONA_SUCCESS) {
+    handle_geomserde_error(err);
+    return NULL;
+  }
+
+  *out_handle = handle;
+  return geom;
+}
+
+/* serialize/deserialize functions for Shapely 2.x */
+
+static PyObject *serialize(PyObject *self, PyObject *args) {
+  PyObject *pygeos_geom = NULL;
+  if (!PyArg_ParseTuple(args, "O", &pygeos_geom)) {
+    return NULL;
+  }
+
+  GEOSGeometry *geos_geom = NULL;
+  char success = PyGEOS_GetGEOSGeometry(pygeos_geom, &geos_geom);
+  if (success == 0) {
+    PyErr_SetString(
+        PyExc_TypeError,
+        "Argument is of incorrect type. Please provide only Geometry objects.");
+    return NULL;
+  }
+
+  return do_serialize(geos_geom);
+}
+
+static PyObject *deserialize(PyObject *self, PyObject *args) {
+  GEOSContextHandle_t handle = NULL;
+  int length = 0;
+  GEOSGeometry *geom = do_deserialize(args, &handle, &length);
+  if (geom == NULL) {
+    return NULL;
+  }
+  PyObject *pygeom = PyGEOS_CreateGeometry(geom, handle);
+  return Py_BuildValue("(Ni)", pygeom, length);
+}
+
+/* serialize/deserialize functions for Shapely 1.x */
+
+static PyObject *serialize_1(PyObject *self, PyObject *args) {
+  GEOSGeometry *geos_geom = NULL;
+  if (!PyArg_ParseTuple(args, "K", &geos_geom)) {
+    return NULL;
+  }
+  return do_serialize(geos_geom);
+}
+
+static PyObject *deserialize_1(PyObject *self, PyObject *args) {
+  GEOSContextHandle_t handle = NULL;
+  int length = 0;
+  GEOSGeometry *geom = do_deserialize(args, &handle, &length);
+  if (geom == NULL) {
+    return NULL;
+  }
+
+  /* These functions would be called by Shapely using ctypes when constructing
+   * a Shapely BaseGeometry object from GEOSGeometry pointer. We call them here
+   * to get rid of the extra overhead introduced by ctypes. */
+  int geom_type_id = dyn_GEOSGeomTypeId_r(handle, geom);
+  char has_z = dyn_GEOSHasZ_r(handle, geom);
+  return Py_BuildValue("(Kibi)", geom, geom_type_id, has_z, length);
+}
+
+/* Module definition for Shapely 2.x */
+
+static PyMethodDef geomserde_methods_shapely_2[] = {
+    {"load_libgeos_c", load_libgeos_c, METH_VARARGS, "Load libgeos_c."},
+    {"serialize", serialize, METH_VARARGS,
+     "Serialize geometry object as bytearray."},
+    {"deserialize", deserialize, METH_VARARGS,
+     "Deserialize bytes-like object to geometry object."},
+    {NULL, NULL, 0, NULL}, /* Sentinel */
+};
+
+static struct PyModuleDef geomserde_module_shapely_2 = {
+    PyModuleDef_HEAD_INIT, "geomserde_speedup", module_doc, 0,
+    geomserde_methods_shapely_2};
+
+/* Module definition for Shapely 1.x */
+
+static PyMethodDef geomserde_methods_shapely_1[] = {
+    {"load_libgeos_c", load_libgeos_c, METH_VARARGS, "Load libgeos_c."},
+    {"serialize_1", serialize_1, METH_VARARGS,
+     "Serialize geometry object as bytearray."},
+    {"deserialize_1", deserialize_1, METH_VARARGS,
+     "Deserialize bytes-like object to geometry object."},
+    {NULL, NULL, 0, NULL}, /* Sentinel */
+};
+
+static struct PyModuleDef geomserde_module_shapely_1 = {
+    PyModuleDef_HEAD_INIT, "geomserde_speedup", module_doc, 0,
+    geomserde_methods_shapely_1};
+
+PyMODINIT_FUNC PyInit_geomserde_speedup(void) {
+  if (import_shapely_c_api() != 0) {
+    /* As long as the capsule provided by Shapely 2.0 cannot be loaded, we
+     * assume that we're working with Shapely 1.0 */
+    PyErr_Clear();
+    return PyModuleDef_Init(&geomserde_module_shapely_1);
+  }
+
+  return PyModuleDef_Init(&geomserde_module_shapely_2);
+}
diff --git a/python/src/geos_c_dyn.c b/python/src/geos_c_dyn.c
new file mode 100644
index 00000000..ae21137d
--- /dev/null
+++ b/python/src/geos_c_dyn.c
@@ -0,0 +1,164 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you 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.
+ */
+
+#define _GNU_SOURCE
+#include "geos_c_dyn.h"
+
+#if defined(_WIN32) || defined(_WIN64)
+#define TARGETING_WINDOWS
+#include <tchar.h>
+#include <windows.h>
+#else
+#include <dlfcn.h>
+#include <errno.h>
+#endif
+
+#include <stdio.h>
+#include <stdlib.h>
+#include <string.h>
+
+#define GEOS_FP_QUALIFIER
+#include "geos_c_dyn_funcs.h"
+#undef GEOS_FP_QUALIFIER
+
+#define LOAD_GEOS_FUNCTION(func)                                               \
+  if (load_geos_c_symbol(handle, #func, (void **)&dyn_##func, err_msg, len) != \
+      0) {                                                                     \
+    return -1;                                                                 \
+  }
+
+#ifdef TARGETING_WINDOWS
+static void win32_get_last_error(char *err_msg, int len) {
+  wchar_t info[256];
+  unsigned int error_code = GetLastError();
+  int info_length = FormatMessageW(
+      FORMAT_MESSAGE_FROM_SYSTEM | FORMAT_MESSAGE_IGNORE_INSERTS, /* flags */
+      NULL,                                      /* message source*/
+      error_code,                                /* the message (error) ID */
+      MAKELANGID(LANG_NEUTRAL, SUBLANG_DEFAULT), /* default language */
+      info,                                      /* the buffer */
+      sizeof(info) / sizeof(wchar_t),            /* size in wchars */
+      NULL);
+  int num_bytes = WideCharToMultiByte(CP_UTF8, 0, info, info_length, err_msg,
+                                      len, NULL, NULL);
+  num_bytes = min(num_bytes, len - 1);
+  err_msg[num_bytes] = '\0';
+}
+#endif
+
+static void *try_load_geos_c_symbol(void *handle, const char *func_name) {
+#ifndef TARGETING_WINDOWS
+  return dlsym(handle, func_name);
+#else
+  return GetProcAddress((HMODULE)handle, func_name);
+#endif
+}
+
+static int load_geos_c_symbol(void *handle, const char *func_name,
+                              void **p_func, char *err_msg, int len) {
+  void *func = try_load_geos_c_symbol(handle, func_name);
+  if (func == NULL) {
+#ifndef TARGETING_WINDOWS
+    snprintf(err_msg, len, "%s", dlerror());
+#else
+    win32_get_last_error(err_msg, len);
+#endif
+    return -1;
+  }
+  *p_func = func;
+  return 0;
+}
+
+int load_geos_c_library(const char *path, char *err_msg, int len) {
+#ifndef TARGETING_WINDOWS
+  void *handle = dlopen(path, RTLD_LOCAL | RTLD_NOW);
+  if (handle == NULL) {
+    snprintf(err_msg, len, "%s", dlerror());
+    return -1;
+  }
+#else
+  int num_chars = MultiByteToWideChar(CP_UTF8, 0, path, -1, NULL, 0);
+  wchar_t *wpath = calloc(num_chars, sizeof(wchar_t));
+  if (wpath == NULL) {
+    snprintf(err_msg, len, "%s", "Cannot allocate memory for wpath");
+    return -1;
+  }
+  MultiByteToWideChar(CP_UTF8, 0, path, -1, wpath, num_chars);
+  HMODULE module = LoadLibraryW(wpath);
+  free(wpath);
+  if (module == NULL) {
+    win32_get_last_error(err_msg, len);
+    return -1;
+  }
+  void *handle = module;
+#endif
+  return load_geos_c_from_handle(handle, err_msg, len);
+}
+
+int is_geos_c_loaded() { return (dyn_GEOS_init_r != NULL) ? 1 : 0; }
+
+int load_geos_c_from_handle(void *handle, char *err_msg, int len) {
+  LOAD_GEOS_FUNCTION(GEOS_finish_r);
+  LOAD_GEOS_FUNCTION(GEOSContext_setErrorHandler_r);
+  LOAD_GEOS_FUNCTION(GEOSGeomTypeId_r);
+  LOAD_GEOS_FUNCTION(GEOSHasZ_r);
+  LOAD_GEOS_FUNCTION(GEOSGetSRID_r);
+  LOAD_GEOS_FUNCTION(GEOSSetSRID_r);
+  LOAD_GEOS_FUNCTION(GEOSGeom_getCoordSeq_r);
+  LOAD_GEOS_FUNCTION(GEOSCoordSeq_getDimensions_r);
+  LOAD_GEOS_FUNCTION(GEOSCoordSeq_getSize_r);
+  LOAD_GEOS_FUNCTION(GEOSCoordSeq_create_r);
+  LOAD_GEOS_FUNCTION(GEOSCoordSeq_destroy_r);
+  LOAD_GEOS_FUNCTION(GEOSCoordSeq_getXY_r);
+  LOAD_GEOS_FUNCTION(GEOSCoordSeq_getXYZ_r);
+  LOAD_GEOS_FUNCTION(GEOSCoordSeq_setXY_r);
+  LOAD_GEOS_FUNCTION(GEOSCoordSeq_setXYZ_r);
+  LOAD_GEOS_FUNCTION(GEOSGetExteriorRing_r);
+  LOAD_GEOS_FUNCTION(GEOSGetNumInteriorRings_r);
+  LOAD_GEOS_FUNCTION(GEOSGetInteriorRingN_r);
+  LOAD_GEOS_FUNCTION(GEOSGetNumCoordinates_r);
+  LOAD_GEOS_FUNCTION(GEOSGeom_getCoordinateDimension_r);
+  LOAD_GEOS_FUNCTION(GEOSGetNumGeometries_r);
+  LOAD_GEOS_FUNCTION(GEOSGetGeometryN_r);
+  LOAD_GEOS_FUNCTION(GEOSisEmpty_r);
+  LOAD_GEOS_FUNCTION(GEOSGeom_createEmptyPoint_r);
+  LOAD_GEOS_FUNCTION(GEOSGeom_createPoint_r);
+  LOAD_GEOS_FUNCTION(GEOSGeom_createPointFromXY_r);
+  LOAD_GEOS_FUNCTION(GEOSGeom_createEmptyLineString_r);
+  LOAD_GEOS_FUNCTION(GEOSGeom_createLineString_r);
+  LOAD_GEOS_FUNCTION(GEOSGeom_createEmptyPolygon_r);
+  LOAD_GEOS_FUNCTION(GEOSGeom_createPolygon_r);
+  LOAD_GEOS_FUNCTION(GEOSGeom_createLinearRing_r);
+  LOAD_GEOS_FUNCTION(GEOSGeom_createCollection_r);
+  LOAD_GEOS_FUNCTION(GEOSGeom_createEmptyCollection_r);
+  LOAD_GEOS_FUNCTION(GEOSGeom_destroy_r);
+
+  /* These functions are not mandantory, only libgeos (>=3.10.0) bundled with
+   * shapely>=1.8.0 has these functions. */
+  dyn_GEOSCoordSeq_copyFromBuffer_r =
+      try_load_geos_c_symbol(handle, "GEOSCoordSeq_copyFromBuffer_r");
+  dyn_GEOSCoordSeq_copyToBuffer_r =
+      try_load_geos_c_symbol(handle, "GEOSCoordSeq_copyToBuffer_r");
+
+  /* Deliberately load GEOS_init_r after all other functions, so that we can
+   * check if all functions were loaded by checking if GEOS_init_r was
+   * loaded. */
+  LOAD_GEOS_FUNCTION(GEOS_init_r);
+  return 0;
+}
diff --git a/python/src/geos_c_dyn.h b/python/src/geos_c_dyn.h
new file mode 100644
index 00000000..2f9cba9f
--- /dev/null
+++ b/python/src/geos_c_dyn.h
@@ -0,0 +1,91 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you 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.
+ */
+
+#ifndef GEOS_C_DYN
+#define GEOS_C_DYN
+
+/* We don't need to depend on geos_c.h in libgeos directly. We can add forward
+ * type declarations for them since the libgeos C API only deals with pointer
+ * types of them, so they are essentially opaque to us. */
+struct GEOSGeom_t;
+struct GEOSPrepGeom_t;
+struct GEOSCoordSeq_t;
+struct GEOSContextHandle;
+
+typedef struct GEOSGeom_t GEOSGeometry;
+typedef struct GEOSPrepGeom_t GEOSPreparedGeometry;
+typedef struct GEOSContextHandle *GEOSContextHandle_t;
+typedef struct GEOSCoordSeq_t GEOSCoordSequence;
+
+typedef void (*GEOSMessageHandler)(const char *fmt, ...);
+
+/**
+ * Check if GEOS C was loaded
+
+ * @return 1 if GEOS C was loaded, otherwise return 0.
+ */
+int is_geos_c_loaded();
+
+/**
+ * Load GEOS C functions from libgeos_c library on specified path.
+ *
+ * We don't link to libgeos_c directly because shapely already brought its own
+ * copy of libgeos_c, it is better to use the same copy of libgeos_c in our
+ * extension since our extension is an augmentation of shapely. Mixing various
+ * versions of the same library together is likely to cause nasty bugs.
+ *
+ * @param path path to the libgeos_c library
+ * @param err_msg buffer for receiving error message in case of errors
+ * @param len length of the error message buffer
+ * @return 0 when GEOS functions were loaded correctly, otherwise returns a
+ * non-zero value
+ */
+int load_geos_c_library(const char *path, char *err_msg, int len);
+
+/**
+ * Load GEOS C functions from specified (platform-specific) library handle
+ *
+ * This function is similar to `load_geos_c_library`. The only exception is
+ * that it does not load the libgeos_c library from file.
+ *
+ * @param handle platform-specific handle to load functions from
+ * @param err_msg buffer for receiving error message in case of errors
+ * @param len length of the error message buffer
+ * @return 0 when GEOS functions were loaded correctly, otherwise returns a
+ * non-zero value
+ */
+int load_geos_c_from_handle(void *handle, char *err_msg, int len);
+
+#define GEOS_FP_QUALIFIER extern
+#include "geos_c_dyn_funcs.h"
+#undef GEOS_FP_QUALIFIER
+
+/* Supported geometry types */
+enum GEOSGeomTypes {
+  GEOS_POINT,
+  GEOS_LINESTRING,
+  GEOS_LINEARRING,
+  GEOS_POLYGON,
+  GEOS_MULTIPOINT,
+  GEOS_MULTILINESTRING,
+  GEOS_MULTIPOLYGON,
+  GEOS_GEOMETRYCOLLECTION
+};
+
+#endif /* GEOS_C_DYN */
diff --git a/python/src/geos_c_dyn_funcs.h b/python/src/geos_c_dyn_funcs.h
new file mode 100644
index 00000000..d0fca8b2
--- /dev/null
+++ b/python/src/geos_c_dyn_funcs.h
@@ -0,0 +1,144 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you 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.
+ */
+
+/* The following are function pointers to GEOS C APIs provided by
+ * libgeos_c. These functions must be called after a successful invocation of
+ * `load_geos_c_library` or `load_geos_c_from_handle` */
+
+GEOS_FP_QUALIFIER GEOSContextHandle_t (*dyn_GEOS_init_r)();
+
+GEOS_FP_QUALIFIER void (*dyn_GEOS_finish_r)(GEOSContextHandle_t handle);
+
+GEOS_FP_QUALIFIER GEOSMessageHandler (*dyn_GEOSContext_setErrorHandler_r)(
+    GEOSContextHandle_t extHandle, GEOSMessageHandler ef);
+
+GEOS_FP_QUALIFIER int (*dyn_GEOSGeomTypeId_r)(GEOSContextHandle_t handle,
+                                              const GEOSGeometry *g);
+
+GEOS_FP_QUALIFIER char (*dyn_GEOSHasZ_r)(GEOSContextHandle_t handle,
+                                         const GEOSGeometry *g);
+
+GEOS_FP_QUALIFIER int (*dyn_GEOSGetSRID_r)(GEOSContextHandle_t handle,
+                                           const GEOSGeometry *g);
+
+GEOS_FP_QUALIFIER void (*dyn_GEOSSetSRID_r)(GEOSContextHandle_t handle,
+                                            GEOSGeometry *g, int SRID);
+
+GEOS_FP_QUALIFIER const GEOSCoordSequence *(*dyn_GEOSGeom_getCoordSeq_r)(
+    GEOSContextHandle_t handle, const GEOSGeometry *g);
+
+GEOS_FP_QUALIFIER int (*dyn_GEOSCoordSeq_getDimensions_r)(
+    GEOSContextHandle_t handle, const GEOSCoordSequence *s, unsigned int *dims);
+
+GEOS_FP_QUALIFIER int (*dyn_GEOSCoordSeq_getSize_r)(GEOSContextHandle_t handle,
+                                                    const GEOSCoordSequence *s,
+                                                    unsigned int *size);
+
+GEOS_FP_QUALIFIER GEOSCoordSequence *(*dyn_GEOSCoordSeq_copyFromBuffer_r)(
+    GEOSContextHandle_t handle, const double *buf, unsigned int size, int hasZ,
+    int hasM);
+
+GEOS_FP_QUALIFIER int (*dyn_GEOSCoordSeq_copyToBuffer_r)(
+    GEOSContextHandle_t handle, const GEOSCoordSequence *s, double *buf,
+    int hasZ, int hasM);
+
+GEOS_FP_QUALIFIER GEOSCoordSequence *(*dyn_GEOSCoordSeq_create_r)(
+    GEOSContextHandle_t handle, unsigned int size, unsigned int dims);
+
+GEOS_FP_QUALIFIER void (*dyn_GEOSCoordSeq_destroy_r)(GEOSContextHandle_t handle,
+                                                     GEOSCoordSequence *s);
+
+GEOS_FP_QUALIFIER int (*dyn_GEOSCoordSeq_getXY_r)(GEOSContextHandle_t handle,
+                                                  const GEOSCoordSequence *s,
+                                                  unsigned int idx, double *x,
+                                                  double *y);
+
+GEOS_FP_QUALIFIER int (*dyn_GEOSCoordSeq_getXYZ_r)(GEOSContextHandle_t handle,
+                                                   const GEOSCoordSequence *s,
+                                                   unsigned int idx, double *x,
+                                                   double *y, double *z);
+
+GEOS_FP_QUALIFIER int (*dyn_GEOSCoordSeq_setXY_r)(GEOSContextHandle_t handle,
+                                                  GEOSCoordSequence *s,
+                                                  unsigned int idx, double x,
+                                                  double y);
+
+GEOS_FP_QUALIFIER int (*dyn_GEOSCoordSeq_setXYZ_r)(GEOSContextHandle_t handle,
+                                                   GEOSCoordSequence *s,
+                                                   unsigned int idx, double x,
+                                                   double y, double z);
+
+GEOS_FP_QUALIFIER const GEOSGeometry *(*dyn_GEOSGetExteriorRing_r)(
+    GEOSContextHandle_t handle, const GEOSGeometry *g);
+
+GEOS_FP_QUALIFIER int (*dyn_GEOSGetNumInteriorRings_r)(
+    GEOSContextHandle_t handle, const GEOSGeometry *g);
+
+GEOS_FP_QUALIFIER int (*dyn_GEOSGetNumCoordinates_r)(GEOSContextHandle_t handle,
+                                                     const GEOSGeometry *g);
+
+GEOS_FP_QUALIFIER int (*dyn_GEOSGeom_getCoordinateDimension_r)(
+    GEOSContextHandle_t handle, const GEOSGeometry *g);
+
+GEOS_FP_QUALIFIER const GEOSGeometry *(*dyn_GEOSGetInteriorRingN_r)(
+    GEOSContextHandle_t handle, const GEOSGeometry *g, int n);
+
+GEOS_FP_QUALIFIER int (*dyn_GEOSGetNumGeometries_r)(GEOSContextHandle_t handle,
+                                                    const GEOSGeometry *g);
+
+GEOS_FP_QUALIFIER const GEOSGeometry *(*dyn_GEOSGetGeometryN_r)(
+    GEOSContextHandle_t handle, const GEOSGeometry *g, int n);
+
+GEOS_FP_QUALIFIER char (*dyn_GEOSisEmpty_r)(GEOSContextHandle_t handle,
+                                            const GEOSGeometry *g);
+
+GEOS_FP_QUALIFIER GEOSGeometry *(*dyn_GEOSGeom_createEmptyPoint_r)(
+    GEOSContextHandle_t handle);
+
+GEOS_FP_QUALIFIER GEOSGeometry *(*dyn_GEOSGeom_createPoint_r)(
+    GEOSContextHandle_t handle, GEOSCoordSequence *s);
+
+GEOS_FP_QUALIFIER GEOSGeometry *(*dyn_GEOSGeom_createPointFromXY_r)(
+    GEOSContextHandle_t handle, double x, double y);
+
+GEOS_FP_QUALIFIER GEOSGeometry *(*dyn_GEOSGeom_createEmptyLineString_r)(
+    GEOSContextHandle_t handle);
+
+GEOS_FP_QUALIFIER GEOSGeometry *(*dyn_GEOSGeom_createLineString_r)(
+    GEOSContextHandle_t handle, GEOSCoordSequence *s);
+
+GEOS_FP_QUALIFIER GEOSGeometry *(*dyn_GEOSGeom_createEmptyPolygon_r)(
+    GEOSContextHandle_t handle);
+
+GEOS_FP_QUALIFIER GEOSGeometry *(*dyn_GEOSGeom_createPolygon_r)(
+    GEOSContextHandle_t handle, GEOSGeometry *shell, GEOSGeometry **holes,
+    unsigned int nholes);
+
+GEOS_FP_QUALIFIER GEOSGeometry *(*dyn_GEOSGeom_createLinearRing_r)(
+    GEOSContextHandle_t handle, GEOSCoordSequence *s);
+
+GEOS_FP_QUALIFIER void (*dyn_GEOSGeom_destroy_r)(GEOSContextHandle_t handle,
+                                                 GEOSGeometry *g);
+
+GEOS_FP_QUALIFIER GEOSGeometry *(*dyn_GEOSGeom_createCollection_r)(
+    GEOSContextHandle_t handle, int type, GEOSGeometry **geoms,
+    unsigned int ngeoms);
+
+GEOS_FP_QUALIFIER GEOSGeometry *(*dyn_GEOSGeom_createEmptyCollection_r)(
+    GEOSContextHandle_t handle, int type);
diff --git a/python/src/pygeos/c_api.h b/python/src/pygeos/c_api.h
new file mode 100644
index 00000000..d3b129b8
--- /dev/null
+++ b/python/src/pygeos/c_api.h
@@ -0,0 +1,85 @@
+/************************************************************************
+ * PyGEOS C API
+ *
+ * This file wraps internal PyGEOS C extension functions for use in other
+ * extensions.  These are specifically wrapped to enable dynamic loading
+ * after Python initialization.
+ *
+ * Each function must provide 3 defines for use in the dynamic loading process:
+ * NUM: the index in function pointer array
+ * RETURN: the return type
+ * PROTO: function prototype
+ *
+ * IMPORTANT: each function must provide 2 sets of defines below and
+ * provide an entry into PyGEOS_API in lib.c module declaration block.
+ *
+ ***********************************************************************/
+
+#ifndef _PYGEOS_API_H
+#define _PYGEOS_API_H
+
+#include <Python.h>
+#include "../geos_c_dyn.h"
+
+/* PyObject* PyGEOS_CreateGeometry(GEOSGeometry *ptr, GEOSContextHandle_t ctx) */
+#define PyGEOS_CreateGeometry_NUM 0
+#define PyGEOS_CreateGeometry_RETURN PyObject *
+#define PyGEOS_CreateGeometry_PROTO (GEOSGeometry * ptr, GEOSContextHandle_t ctx)
+
+/* char PyGEOS_GetGEOSGeometry(GeometryObject *obj, GEOSGeometry **out) */
+#define PyGEOS_GetGEOSGeometry_NUM 1
+#define PyGEOS_GetGEOSGeometry_RETURN char
+#define PyGEOS_GetGEOSGeometry_PROTO (PyObject * obj, GEOSGeometry * *out)
+
+/* GEOSCoordSequence* PyGEOS_CoordSeq_FromBuffer(GEOSContextHandle_t ctx, const double* buf,
+                                                unsigned int size, unsigned int dims, char ring_closure)*/
+#define PyGEOS_CoordSeq_FromBuffer_NUM 2
+#define PyGEOS_CoordSeq_FromBuffer_RETURN GEOSCoordSequence*
+#define PyGEOS_CoordSeq_FromBuffer_PROTO (GEOSContextHandle_t ctx, const double* buf, unsigned int size, unsigned int dims, char ring_closure)
+
+/* Total number of C API pointers */
+#define PyGEOS_API_num_pointers 3
+
+#ifdef PyGEOS_API_Module
+/* This section is used when compiling shapely.lib C extension.
+ * Each API function needs to provide a corresponding *_PROTO here.
+ */
+
+extern PyGEOS_CreateGeometry_RETURN PyGEOS_CreateGeometry PyGEOS_CreateGeometry_PROTO;
+extern PyGEOS_GetGEOSGeometry_RETURN PyGEOS_GetGEOSGeometry PyGEOS_GetGEOSGeometry_PROTO;
+extern PyGEOS_CoordSeq_FromBuffer_RETURN PyGEOS_CoordSeq_FromBuffer PyGEOS_CoordSeq_FromBuffer_PROTO;
+
+#else
+/* This section is used in modules that use the PyGEOS C API
+ * Each API function needs to provide the lookup into PyGEOS_API as a
+ * define statement.
+*/
+
+static void **PyGEOS_API;
+
+#define PyGEOS_CreateGeometry \
+    (*(PyGEOS_CreateGeometry_RETURN(*) PyGEOS_CreateGeometry_PROTO)PyGEOS_API[PyGEOS_CreateGeometry_NUM])
+
+#define PyGEOS_GetGEOSGeometry \
+    (*(PyGEOS_GetGEOSGeometry_RETURN(*) PyGEOS_GetGEOSGeometry_PROTO)PyGEOS_API[PyGEOS_GetGEOSGeometry_NUM])
+
+#define PyGEOS_CoordSeq_FromBuffer \
+    (*(PyGEOS_CoordSeq_FromBuffer_RETURN(*) PyGEOS_CoordSeq_FromBuffer_PROTO)PyGEOS_API[PyGEOS_CoordSeq_FromBuffer_NUM])
+
+/* Dynamically load C API from PyCapsule.
+ * This MUST be called prior to using C API functions in other modules; otherwise
+ * segfaults will occur when the PyGEOS C API functions are called.
+ *
+ * Returns 0 on success, -1 if error.
+ * PyCapsule_Import will set an exception on error.
+ */
+static int
+import_shapely_c_api(void)
+{
+    PyGEOS_API = (void **)PyCapsule_Import("shapely.lib._C_API", 0);
+    return (PyGEOS_API == NULL) ? -1 : 0;
+}
+
+#endif
+
+#endif /* !defined(_PYGEOS_API_H) */
diff --git a/python/tests/utils/test_geometry_serde.py b/python/tests/utils/test_geometry_serde.py
index e6c8fc8c..adb12202 100644
--- a/python/tests/utils/test_geometry_serde.py
+++ b/python/tests/utils/test_geometry_serde.py
@@ -19,9 +19,7 @@ import pytest
 from pyspark.sql.types import (StructType, StringType)
 from sedona.sql.types import GeometryType
 from pyspark.sql.functions import expr
-from sedona.utils import geometry_serde
 
-from shapely.geometry.base import BaseGeometry
 from shapely.geometry import (
     GeometryCollection,
     LineString,
@@ -36,8 +34,6 @@ from shapely.wkt import loads as wkt_loads
 from tests.test_base import TestBase
 
 class TestGeometrySerde(TestBase):
-
-
     @pytest.mark.parametrize("geom", [
         GeometryCollection([Point([10.0, 20.0]), Polygon([(10.0, 10.0), (20.0, 20.0), (20.0, 10.0)])]),
         LineString([(10.0, 20.0), (30.0, 40.0)]),
@@ -151,102 +147,3 @@ class TestGeometrySerde(TestBase):
         assert n_dims == 3
         assert z_min is None
         assert geom.equals(returned_geom)
-
-    def test_point(self):
-        points = [
-            wkt_loads("POINT EMPTY"),
-            Point(10, 20),
-            Point(10, 20, 30)
-        ]
-        self._test_serde_roundtrip(points)
-
-    def test_linestring(self):
-        linestrings = [
-            wkt_loads("LINESTRING EMPTY"),
-            LineString([(10, 20), (30, 40)]),
-            LineString([(10, 20), (30, 40), (50, 60)]),
-            LineString([(10, 20, 30), (30, 40, 50), (50, 60, 70)]),
-        ]
-        self._test_serde_roundtrip(linestrings)
-
-    def test_multi_point(self):
-        multi_points = [
-            wkt_loads("MULTIPOINT EMPTY"),
-            MultiPoint([(10, 20)]),
-            MultiPoint([(10, 20), (30, 40)]),
-            MultiPoint([(10, 20), (30, 40), (50, 60)]),
-            MultiPoint([(10, 20, 30), (30, 40, 50), (50, 60, 70)]),
-        ]
-        self._test_serde_roundtrip(multi_points)
-
-    def test_multi_linestring(self):
-        multi_linestrings = [
-            wkt_loads("MULTILINESTRING EMPTY"),
-            MultiLineString([[(10, 20), (30, 40)]]),
-            MultiLineString([[(10, 20), (30, 40)], [(50, 60), (70, 80)]]),
-            MultiLineString([[(10, 20, 30), (30, 40, 50)], [(50, 60, 70), (70, 80, 90)]]),
-        ]
-        self._test_serde_roundtrip(multi_linestrings)
-
-    def test_polygon(self):
-        ext = [(0, 0), (10, 0), (10, 10), (0, 10), (0, 0)]
-        int0 = [(1, 1), (1, 1.5), (1.5, 1.5), (1.5, 1), (1, 1)]
-        int1 = [(2, 2), (2, 2.5), (2.5, 2.5), (2.5, 2), (2, 2)]
-        polygons = [
-            wkt_loads("POLYGON EMPTY"),
-            Polygon(ext),
-            Polygon(ext, [int0]),
-            Polygon(ext, [int0, int1]),
-        ]
-        self._test_serde_roundtrip(polygons)
-
-    def test_multi_polygon(self):
-        ext = [(0, 0), (100, 0), (100, 100), (0, 100), (0, 0)]
-        int0 = [(10, 10), (10, 15), (15, 15), (15, 10), (10, 10)]
-        int1 = [(2, 2), (2, 2.5), (2.5, 2.5), (2.5, 2), (2, 2)]
-        multi_polygons = [
-            wkt_loads("MULTIPOLYGON EMPTY"),
-            MultiPolygon([Polygon(ext)]),
-            MultiPolygon([Polygon(ext), Polygon(ext, [int0])]),
-            MultiPolygon([Polygon(ext), Polygon(ext, [int0, int1])]),
-            MultiPolygon([Polygon(ext, [int1]), Polygon(ext), Polygon(ext, [int0, int1])]),
-        ]
-        self._test_serde_roundtrip(multi_polygons)
-
-    def test_geometry_collection(self):
-        geometry_collections = [
-            wkt_loads("GEOMETRYCOLLECTION EMPTY"),
-            GeometryCollection([Point(10, 20), LineString([(10, 20), (30, 40)]), Point(30, 40)]),
-            GeometryCollection([
-                MultiPoint([(10, 20), (30, 40)]),
-                MultiLineString([[(10, 20), (30, 40)], [(50, 60), (70, 80)]]),
-                MultiPolygon([
-                    Polygon(
-                        [(0, 0), (100, 0), (100, 100), (0, 100), (0, 0)],
-                        [[(10, 10), (10, 15), (15, 15), (15, 10), (10, 10)]])
-                ]),
-                Point(100, 200)
-            ]),
-            GeometryCollection([
-                GeometryCollection([Point(10, 20), LineString([(10, 20), (30, 40)]), Point(30, 40)]),
-                GeometryCollection([
-                    MultiPoint([(10, 20), (30, 40)]),
-                    MultiLineString([[(10, 20), (30, 40)], [(50, 60), (70, 80)]]),
-                    Point(10, 20)
-                ])
-            ])
-        ]
-        self._test_serde_roundtrip(geometry_collections)
-
-    @staticmethod
-    def _test_serde_roundtrip(geoms):
-        for geom in geoms:
-            geom_actual = TestGeometrySerde.serde_roundtrip(geom)
-            assert geom_actual.equals_exact(geom, 1e-6)
-            assert geom.wkt == geom_actual.wkt
-
-    @staticmethod
-    def serde_roundtrip(geom: BaseGeometry) -> BaseGeometry:
-        buffer = geometry_serde.serialize(geom)
-        geom2, offset = geometry_serde.deserialize(buffer)
-        return geom2
diff --git a/python/tests/utils/test_geomserde_speedup.py b/python/tests/utils/test_geomserde_speedup.py
new file mode 100644
index 00000000..4dcf6383
--- /dev/null
+++ b/python/tests/utils/test_geomserde_speedup.py
@@ -0,0 +1,137 @@
+#  Licensed to the Apache Software Foundation (ASF) under one
+#  or more contributor license agreements.  See the NOTICE file
+#  distributed with this work for additional information
+#  regarding copyright ownership.  The ASF licenses this file
+#  to you 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.
+
+from sedona.utils import geometry_serde
+
+from shapely.geometry.base import BaseGeometry
+from shapely.geometry import (
+    GeometryCollection,
+    LineString,
+    MultiLineString,
+    MultiPoint,
+    MultiPolygon,
+    Point,
+    Polygon,
+)
+from shapely.wkt import loads as wkt_loads
+
+class TestGeomSerdeSpeedup:
+    def test_speedup_enabled(self):
+        assert geometry_serde.speedup_enabled
+
+    def test_point(self):
+        points = [
+            wkt_loads("POINT EMPTY"),
+            Point(10, 20),
+            Point(10, 20, 30)
+        ]
+        self._test_serde_roundtrip(points)
+
+    def test_linestring(self):
+        linestrings = [
+            wkt_loads("LINESTRING EMPTY"),
+            LineString([(10, 20), (30, 40)]),
+            LineString([(10, 20), (30, 40), (50, 60)]),
+            LineString([(10, 20, 30), (30, 40, 50), (50, 60, 70)]),
+        ]
+        self._test_serde_roundtrip(linestrings)
+
+    def test_multi_point(self):
+        multi_points = [
+            wkt_loads("MULTIPOINT EMPTY"),
+            MultiPoint([(10, 20)]),
+            MultiPoint([(10, 20), (30, 40)]),
+            MultiPoint([(10, 20), (30, 40), (50, 60)]),
+            MultiPoint([(10, 20, 30), (30, 40, 50), (50, 60, 70)]),
+        ]
+        self._test_serde_roundtrip(multi_points)
+
+    def test_multi_linestring(self):
+        multi_linestrings = [
+            wkt_loads("MULTILINESTRING EMPTY"),
+            MultiLineString([[(10, 20), (30, 40)]]),
+            MultiLineString([[(10, 20), (30, 40)], [(50, 60), (70, 80)]]),
+            MultiLineString([[(10, 20, 30), (30, 40, 50)], [(50, 60, 70), (70, 80, 90)]]),
+        ]
+        self._test_serde_roundtrip(multi_linestrings)
+
+    def test_polygon(self):
+        ext = [(0, 0), (10, 0), (10, 10), (0, 10), (0, 0)]
+        int0 = [(1, 1), (1, 1.5), (1.5, 1.5), (1.5, 1), (1, 1)]
+        int1 = [(2, 2), (2, 2.5), (2.5, 2.5), (2.5, 2), (2, 2)]
+        polygons = [
+            wkt_loads("POLYGON EMPTY"),
+            Polygon(ext),
+            Polygon(ext, [int0]),
+            Polygon(ext, [int0, int1]),
+        ]
+        self._test_serde_roundtrip(polygons)
+
+    def test_multi_polygon(self):
+        ext = [(0, 0), (100, 0), (100, 100), (0, 100), (0, 0)]
+        int0 = [(10, 10), (10, 15), (15, 15), (15, 10), (10, 10)]
+        int1 = [(2, 2), (2, 2.5), (2.5, 2.5), (2.5, 2), (2, 2)]
+        multi_polygons = [
+            wkt_loads("MULTIPOLYGON EMPTY"),
+            MultiPolygon([Polygon(ext)]),
+            MultiPolygon([Polygon(ext), Polygon(ext, [int0])]),
+            MultiPolygon([Polygon(ext), Polygon(ext, [int0, int1])]),
+            MultiPolygon([Polygon(ext, [int1]), Polygon(ext), Polygon(ext, [int0, int1])]),
+        ]
+        self._test_serde_roundtrip(multi_polygons)
+
+    def test_geometry_collection(self):
+        geometry_collections = [
+            wkt_loads("GEOMETRYCOLLECTION EMPTY"),
+            GeometryCollection([Point(10, 20), LineString([(10, 20), (30, 40)]), Point(30, 40)]),
+            GeometryCollection([
+                MultiPoint([(10, 20), (30, 40)]),
+                MultiLineString([[(10, 20), (30, 40)], [(50, 60), (70, 80)]]),
+                MultiPolygon([
+                    Polygon(
+                        [(0, 0), (100, 0), (100, 100), (0, 100), (0, 0)],
+                        [[(10, 10), (10, 15), (15, 15), (15, 10), (10, 10)]])
+                ]),
+                Point(100, 200)
+            ]),
+            GeometryCollection([
+                GeometryCollection([Point(10, 20), LineString([(10, 20), (30, 40)]), Point(30, 40)]),
+                GeometryCollection([
+                    MultiPoint([(10, 20), (30, 40)]),
+                    MultiLineString([[(10, 20), (30, 40)], [(50, 60), (70, 80)]]),
+                    Point(10, 20)
+                ])
+            ])
+        ]
+        self._test_serde_roundtrip(geometry_collections)
+
+    @staticmethod
+    def _test_serde_roundtrip(geoms):
+        for geom in geoms:
+            geom_actual = TestGeomSerdeSpeedup.serde_roundtrip(geom)
+            assert geom_actual.equals_exact(geom, 1e-6)
+            # GEOSGeom_createEmptyLineString in libgeos creates LineString with
+            # Z dimension, This bug has been fixed by
+            # https://github.com/libgeos/geos/pull/745
+            geom_actual_wkt = geom_actual.wkt.replace('LINESTRING Z EMPTY', 'LINESTRING EMPTY')
+            assert geom.wkt == geom_actual_wkt
+
+    @staticmethod
+    def serde_roundtrip(geom: BaseGeometry) -> BaseGeometry:
+        buffer = geometry_serde.serialize(geom)
+        geom2, offset = geometry_serde.deserialize(buffer)
+        return geom2