You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@baremaps.apache.org by bc...@apache.org on 2023/10/27 09:25:28 UTC

(incubator-baremaps) branch main updated: Improve postgres tilestore (#792)

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

bchapuis pushed a commit to branch main
in repository https://gitbox.apache.org/repos/asf/incubator-baremaps.git


The following commit(s) were added to refs/heads/main by this push:
     new 647a2734 Improve postgres tilestore (#792)
647a2734 is described below

commit 647a27349d973828e374d9f03f7aeb4562bab031
Author: Bertil Chapuis <bc...@gmail.com>
AuthorDate: Fri Oct 27 11:25:24 2023 +0200

    Improve postgres tilestore (#792)
    
    * Align tilestore queries with the baremaps-exporter
    
    * Remove dependency to jsqlparser
    
    * Improve documentation and naming
    
    Co-authored-by: Joe Polastre <jo...@flightaware.com>
---
 baremaps-core/pom.xml                              |   4 -
 .../org/apache/baremaps/iploc/IpLocRepository.java |   1 -
 .../baremaps/stream/PartitionedSpliterator.java    |  16 +-
 .../org/apache/baremaps/stream/StreamUtils.java    |   3 +-
 .../org/apache/baremaps/tilestore/TileEntry.java}  |  29 +-
 .../org/apache/baremaps/tilestore/TileStore.java   |  41 +++
 .../baremaps/tilestore/mbtiles/MBTilesStore.java   |  43 ++-
 .../baremaps/tilestore/postgres/PostgresGroup.java | 111 -------
 .../baremaps/tilestore/postgres/PostgresQuery.java | 130 ---------
 .../tilestore/postgres/PostgresQueryGenerator.java |   8 +-
 .../tilestore/postgres/PostgresTileStore.java      | 319 +++++++++------------
 .../org/apache/baremaps/utils/PostgresUtils.java   |  11 +-
 .../org/apache/baremaps/utils/SqliteUtils.java     |   2 +-
 .../baremaps/workflow/tasks/ExportVectorTiles.java |  33 ++-
 .../stream/PartitionedSpliteratorTest.java         |  20 +-
 .../apache/baremaps/stream/StreamUtilsTest.java    |   4 +-
 .../tilestore/postgres/JSQLParserTest.java         |  64 -----
 .../postgres/PostgresQueryGeneratorTest.java       |   3 +-
 .../postgres/PostgresQueryParserTest.java          | 107 -------
 .../tilestore/postgres/PostgresTileStoreTest.java  |  50 ++--
 pom.xml                                            |   6 -
 21 files changed, 308 insertions(+), 697 deletions(-)

diff --git a/baremaps-core/pom.xml b/baremaps-core/pom.xml
index 68a427db..2b40f130 100644
--- a/baremaps-core/pom.xml
+++ b/baremaps-core/pom.xml
@@ -50,10 +50,6 @@ limitations under the License.
       <groupId>com.github.ben-manes.caffeine</groupId>
       <artifactId>caffeine</artifactId>
     </dependency>
-    <dependency>
-      <groupId>com.github.jsqlparser</groupId>
-      <artifactId>jsqlparser</artifactId>
-    </dependency>
     <dependency>
       <groupId>com.google.guava</groupId>
       <artifactId>guava</artifactId>
diff --git a/baremaps-core/src/main/java/org/apache/baremaps/iploc/IpLocRepository.java b/baremaps-core/src/main/java/org/apache/baremaps/iploc/IpLocRepository.java
index f3782730..cc726d9f 100644
--- a/baremaps-core/src/main/java/org/apache/baremaps/iploc/IpLocRepository.java
+++ b/baremaps-core/src/main/java/org/apache/baremaps/iploc/IpLocRepository.java
@@ -217,7 +217,6 @@ public final class IpLocRepository {
    */
   public void save(Stream<IpLocObject> ipLocObjects) {
     StreamUtils.partition(ipLocObjects, 100)
-        .map(Stream::toList)
         .forEach(this::save);
   }
 }
diff --git a/baremaps-core/src/main/java/org/apache/baremaps/stream/PartitionedSpliterator.java b/baremaps-core/src/main/java/org/apache/baremaps/stream/PartitionedSpliterator.java
index ab0079dd..de42729a 100644
--- a/baremaps-core/src/main/java/org/apache/baremaps/stream/PartitionedSpliterator.java
+++ b/baremaps-core/src/main/java/org/apache/baremaps/stream/PartitionedSpliterator.java
@@ -19,6 +19,8 @@ package org.apache.baremaps.stream;
 
 
 
+import java.util.ArrayList;
+import java.util.List;
 import java.util.Spliterator;
 import java.util.function.Consumer;
 import java.util.stream.Stream;
@@ -28,7 +30,7 @@ import java.util.stream.Stream;
  *
  * @param <T> the type of elements returned by this {@code Spliterator}
  */
-public class PartitionedSpliterator<T> implements Spliterator<Stream<T>> {
+public class PartitionedSpliterator<T> implements Spliterator<List<T>> {
 
   private final Spliterator<T> spliterator;
 
@@ -47,23 +49,23 @@ public class PartitionedSpliterator<T> implements Spliterator<Stream<T>> {
 
   /** {@inheritDoc} */
   @Override
-  public boolean tryAdvance(Consumer<? super Stream<T>> action) {
-    Stream.Builder<T> partition = Stream.builder();
+  public boolean tryAdvance(Consumer<? super List<T>> action) {
+    var list = new ArrayList<T>(partitionSize);
     int size = 0;
-    while (size < partitionSize && spliterator.tryAdvance(partition::add)) {
+    while (size < partitionSize && spliterator.tryAdvance(list::add)) {
       size++;
     }
     if (size == 0) {
       return false;
     }
-    action.accept(partition.build());
+    action.accept(list);
     return true;
   }
 
   /** {@inheritDoc} */
   @Override
-  public Spliterator<Stream<T>> trySplit() {
-    HoldingConsumer<Stream<T>> consumer = new HoldingConsumer<>();
+  public Spliterator<List<T>> trySplit() {
+    HoldingConsumer<List<T>> consumer = new HoldingConsumer<>();
     tryAdvance(consumer);
     return Stream.ofNullable(consumer.value()).spliterator();
   }
diff --git a/baremaps-core/src/main/java/org/apache/baremaps/stream/StreamUtils.java b/baremaps-core/src/main/java/org/apache/baremaps/stream/StreamUtils.java
index 17397487..349a2c06 100644
--- a/baremaps-core/src/main/java/org/apache/baremaps/stream/StreamUtils.java
+++ b/baremaps-core/src/main/java/org/apache/baremaps/stream/StreamUtils.java
@@ -20,6 +20,7 @@ package org.apache.baremaps.stream;
 
 
 import java.util.Iterator;
+import java.util.List;
 import java.util.Spliterator;
 import java.util.Spliterators;
 import java.util.concurrent.CompletableFuture;
@@ -190,7 +191,7 @@ public class StreamUtils {
   }
 
   /** Partition the provided stream according to a partition size. */
-  public static <T> Stream<Stream<T>> partition(
+  public static <T> Stream<List<T>> partition(
       Stream<T> stream,
       int partitionSize) {
     return StreamSupport.stream(
diff --git a/baremaps-core/src/test/java/org/apache/baremaps/stream/StreamUtilsTest.java b/baremaps-core/src/main/java/org/apache/baremaps/tilestore/TileEntry.java
similarity index 60%
copy from baremaps-core/src/test/java/org/apache/baremaps/stream/StreamUtilsTest.java
copy to baremaps-core/src/main/java/org/apache/baremaps/tilestore/TileEntry.java
index eb2ecc0b..58333c7a 100644
--- a/baremaps-core/src/test/java/org/apache/baremaps/stream/StreamUtilsTest.java
+++ b/baremaps-core/src/main/java/org/apache/baremaps/tilestore/TileEntry.java
@@ -15,22 +15,25 @@
  * limitations under the License.
  */
 
-package org.apache.baremaps.stream;
+package org.apache.baremaps.tilestore;
 
-import static org.junit.jupiter.api.Assertions.assertEquals;
+import java.nio.ByteBuffer;
 
-import java.util.List;
-import java.util.stream.Collectors;
-import java.util.stream.IntStream;
-import org.junit.jupiter.api.Test;
+public class TileEntry {
 
-class StreamUtilsTest {
+  private final TileCoord tileCoord;
+  private final ByteBuffer byteBuffer;
 
-  @Test
-  void partition() {
-    List<Integer> list = IntStream.range(0, 100).boxed().toList();
-    List<List<Integer>> partitions = StreamUtils.partition(list.stream(), 10)
-        .map(stream -> stream.collect(Collectors.toList())).toList();
-    assertEquals(partitions.size(), 10);
+  public TileEntry(TileCoord tileCoord, ByteBuffer byteBuffer) {
+    this.tileCoord = tileCoord;
+    this.byteBuffer = byteBuffer;
+  }
+
+  public TileCoord getTileCoord() {
+    return tileCoord;
+  }
+
+  public ByteBuffer getByteBuffer() {
+    return byteBuffer;
   }
 }
diff --git a/baremaps-core/src/main/java/org/apache/baremaps/tilestore/TileStore.java b/baremaps-core/src/main/java/org/apache/baremaps/tilestore/TileStore.java
index 5d2b1a2f..c0332150 100644
--- a/baremaps-core/src/main/java/org/apache/baremaps/tilestore/TileStore.java
+++ b/baremaps-core/src/main/java/org/apache/baremaps/tilestore/TileStore.java
@@ -20,6 +20,8 @@ package org.apache.baremaps.tilestore;
 
 
 import java.nio.ByteBuffer;
+import java.util.ArrayList;
+import java.util.List;
 
 /** Represents a store for tiles. */
 public interface TileStore {
@@ -33,6 +35,21 @@ public interface TileStore {
    */
   ByteBuffer read(TileCoord tileCoord) throws TileStoreException;
 
+  /**
+   * Reads the content of a list of tiles.
+   *
+   * @param tileCoords the tile coordinates
+   * @return the content of the tiles
+   * @throws TileStoreException
+   */
+  default List<ByteBuffer> read(List<TileCoord> tileCoords) throws TileStoreException {
+    var blobs = new ArrayList<ByteBuffer>();
+    for (var tileCoord : tileCoords) {
+      blobs.add(read(tileCoord));
+    }
+    return blobs;
+  }
+
   /**
    * Writes the content of a tile.
    *
@@ -42,6 +59,18 @@ public interface TileStore {
    */
   void write(TileCoord tileCoord, ByteBuffer blob) throws TileStoreException;
 
+  /**
+   * Writes the content of a list of tiles.
+   *
+   * @param entries the tile entries
+   * @throws TileStoreException
+   */
+  default void write(List<TileEntry> entries) throws TileStoreException {
+    for (var entry : entries) {
+      write(entry.getTileCoord(), entry.getByteBuffer());
+    }
+  }
+
   /**
    * Deletes the content of a tile.
    *
@@ -49,4 +78,16 @@ public interface TileStore {
    * @throws TileStoreException
    */
   void delete(TileCoord tileCoord) throws TileStoreException;
+
+  /**
+   * Deletes the content of a list of tiles.
+   *
+   * @param tileCoords the tile coordinates
+   * @throws TileStoreException
+   */
+  default void delete(List<TileCoord> tileCoords) throws TileStoreException {
+    for (var tileCoord : tileCoords) {
+      delete(tileCoord);
+    }
+  }
 }
diff --git a/baremaps-core/src/main/java/org/apache/baremaps/tilestore/mbtiles/MBTilesStore.java b/baremaps-core/src/main/java/org/apache/baremaps/tilestore/mbtiles/MBTilesStore.java
index 22baf690..b6560778 100644
--- a/baremaps-core/src/main/java/org/apache/baremaps/tilestore/mbtiles/MBTilesStore.java
+++ b/baremaps-core/src/main/java/org/apache/baremaps/tilestore/mbtiles/MBTilesStore.java
@@ -18,7 +18,6 @@
 package org.apache.baremaps.tilestore.mbtiles;
 
 
-
 import java.io.IOException;
 import java.nio.ByteBuffer;
 import java.sql.Connection;
@@ -27,9 +26,11 @@ import java.sql.ResultSet;
 import java.sql.SQLException;
 import java.sql.Statement;
 import java.util.HashMap;
+import java.util.List;
 import java.util.Map;
 import javax.sql.DataSource;
 import org.apache.baremaps.tilestore.TileCoord;
+import org.apache.baremaps.tilestore.TileEntry;
 import org.apache.baremaps.tilestore.TileStore;
 import org.apache.baremaps.tilestore.TileStoreException;
 
@@ -40,7 +41,7 @@ import org.apache.baremaps.tilestore.TileStoreException;
  */
 public class MBTilesStore implements TileStore {
 
-  private static final String CREATE_TABLE_METADATA =
+  public static final String CREATE_TABLE_METADATA =
       "CREATE TABLE IF NOT EXISTS metadata (name TEXT, value TEXT, PRIMARY KEY (name))";
 
   private static final String CREATE_TABLE_TILES =
@@ -75,7 +76,9 @@ public class MBTilesStore implements TileStore {
     this.dataSource = dataSource;
   }
 
-  /** {@inheritDoc} */
+  /**
+   * {@inheritDoc}
+   */
   @Override
   public ByteBuffer read(TileCoord tileCoord) throws TileStoreException {
     try (Connection connection = dataSource.getConnection();
@@ -95,7 +98,9 @@ public class MBTilesStore implements TileStore {
     }
   }
 
-  /** {@inheritDoc} */
+  /**
+   * {@inheritDoc}
+   */
   @Override
   public void write(TileCoord tileCoord, ByteBuffer blob) throws TileStoreException {
     try (Connection connection = dataSource.getConnection();
@@ -104,13 +109,39 @@ public class MBTilesStore implements TileStore {
       statement.setInt(2, tileCoord.x());
       statement.setInt(3, reverseY(tileCoord.y(), tileCoord.z()));
       statement.setBytes(4, blob.array());
-      statement.executeUpdate();
+      statement.execute();
     } catch (SQLException e) {
       throw new TileStoreException(e);
     }
   }
 
-  /** {@inheritDoc} */
+  /**
+   * {@inheritDoc}
+   */
+  @Override
+  public void write(List<TileEntry> tileEntries) throws TileStoreException {
+    try (Connection connection = dataSource.getConnection()) {
+      // connection.setAutoCommit(false);
+      try (PreparedStatement statement = connection.prepareStatement(INSERT_TILE)) {
+        for (TileEntry tileEntry : tileEntries) {
+          TileCoord tileCoord = tileEntry.getTileCoord();
+          ByteBuffer byteBuffer = tileEntry.getByteBuffer();
+          statement.setInt(1, tileCoord.z());
+          statement.setInt(2, tileCoord.x());
+          statement.setInt(3, reverseY(tileCoord.y(), tileCoord.z()));
+          statement.setBytes(4, byteBuffer.array());
+          statement.execute();
+        }
+      }
+      // connection.commit();
+    } catch (SQLException e) {
+      throw new TileStoreException(e);
+    }
+  }
+
+  /**
+   * {@inheritDoc}
+   */
   @Override
   public void delete(TileCoord tileCoord) throws TileStoreException {
     try (Connection connection = dataSource.getConnection();
diff --git a/baremaps-core/src/main/java/org/apache/baremaps/tilestore/postgres/PostgresGroup.java b/baremaps-core/src/main/java/org/apache/baremaps/tilestore/postgres/PostgresGroup.java
deleted file mode 100644
index de473ab4..00000000
--- a/baremaps-core/src/main/java/org/apache/baremaps/tilestore/postgres/PostgresGroup.java
+++ /dev/null
@@ -1,111 +0,0 @@
-/*
- * 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.
- */
-
-package org.apache.baremaps.tilestore.postgres;
-
-
-
-import java.util.List;
-import java.util.Objects;
-import java.util.Optional;
-import java.util.stream.Collectors;
-import net.sf.jsqlparser.statement.select.FromItem;
-import net.sf.jsqlparser.statement.select.Join;
-import net.sf.jsqlparser.statement.select.SelectItem;
-
-/**
- * Models the groups identified in the input queries of a {@code PostgresTileStore}. These groups
- * are used to form common table expressions (CTE).
- */
-class PostgresGroup {
-
-  private final List<SelectItem<?>> selectItems;
-  private final FromItem fromItem;
-  private final List<Join> joins;
-
-  /**
-   * Constructs a {@code PostgresGroup} with objects extracted from an AST obtained by parsing a SQL
-   * query with JSQLParser.
-   *
-   * @param selectItems the selected columns.
-   * @param fromItem the from clause
-   * @param joins the join clauses
-   */
-  public PostgresGroup(List<SelectItem<?>> selectItems, FromItem fromItem, List<Join> joins) {
-    this.selectItems = selectItems;
-    this.fromItem = fromItem;
-    this.joins = joins;
-  }
-
-  /**
-   * Returns the selected columns.
-   *
-   * @return the selected columns
-   */
-  public List<SelectItem<?>> getSelectItems() {
-    return selectItems;
-  }
-
-  /**
-   * Returns the from clause.
-   *
-   * @return the from clause
-   */
-  public FromItem getFromItem() {
-    return fromItem;
-  }
-
-  /**
-   * Returns the join clauses.
-   *
-   * @return the join clauses
-   */
-  public List<Join> getJoins() {
-    return joins;
-  }
-
-  /**
-   * Returns the unique alias of this group.
-   *
-   * @return the alias
-   */
-  public String getAlias() {
-    return String.format("h%x", hashCode());
-  }
-
-  /** {@inheritDoc} */
-  @Override
-  public boolean equals(Object o) {
-    if (this == o) {
-      return true;
-    }
-    if (!(o instanceof PostgresGroup)) {
-      return false;
-    }
-    return hashCode() == o.hashCode();
-  }
-
-  /** {@inheritDoc} */
-  @Override
-  public int hashCode() {
-    String selectItemsString = selectItems.toString();
-    String fromItemString = fromItem.toString();
-    String joinsString = Optional.ofNullable(joins).stream().flatMap(List::stream)
-        .map(Join::toString).collect(Collectors.joining());
-    return Objects.hash(selectItemsString, fromItemString, joinsString);
-  }
-}
diff --git a/baremaps-core/src/main/java/org/apache/baremaps/tilestore/postgres/PostgresQuery.java b/baremaps-core/src/main/java/org/apache/baremaps/tilestore/postgres/PostgresQuery.java
deleted file mode 100644
index 68cbf750..00000000
--- a/baremaps-core/src/main/java/org/apache/baremaps/tilestore/postgres/PostgresQuery.java
+++ /dev/null
@@ -1,130 +0,0 @@
-/*
- * 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.
- */
-
-package org.apache.baremaps.tilestore.postgres;
-
-
-
-import net.sf.jsqlparser.JSQLParserException;
-import net.sf.jsqlparser.parser.CCJSqlParserUtil;
-import net.sf.jsqlparser.statement.select.PlainSelect;
-import net.sf.jsqlparser.statement.select.Select;
-import net.sf.jsqlparser.statement.select.SelectItem;
-import net.sf.jsqlparser.statement.select.SelectItemVisitorAdapter;
-
-/** Models the input queries of a {@code PostgresTileStore}. */
-public class PostgresQuery {
-
-  private final String layer;
-  private final Integer minzoom;
-  private final Integer maxzoom;
-  private final String sql;
-  private final PlainSelect ast;
-
-  /**
-   * Constructs a {@code PostgresQuery}.
-   *
-   * @param layer the layer name
-   * @param minzoom the min zoom
-   * @param maxzoom the max zoom
-   * @param sql the sql query
-   */
-  public PostgresQuery(String layer, Integer minzoom, Integer maxzoom, String sql) {
-    this.layer = layer;
-    this.minzoom = minzoom;
-    this.maxzoom = maxzoom;
-    this.sql = sql;
-    this.ast = parse(sql);
-  }
-
-  /**
-   * Returns the layer name.
-   *
-   * @return the layer name
-   */
-  public String getLayer() {
-    return layer;
-  }
-
-  /**
-   * Returns the min zoom.
-   *
-   * @return the min zoom
-   */
-  public Integer getMinzoom() {
-    return minzoom;
-  }
-
-  /**
-   * Returns the max zoom.
-   *
-   * @return the max zoom
-   */
-  public Integer getMaxzoom() {
-    return maxzoom;
-  }
-
-  /**
-   * Returns the SQL query.
-   *
-   * @return the SQL query
-   */
-  public String getSql() {
-    return sql;
-  }
-
-  /**
-   * Returns the abstract syntax tree (AST) obtained by parsing the query with JSQLParser.
-   *
-   * @return the AST
-   */
-  public PlainSelect getAst() {
-    return ast;
-  }
-
-  private PlainSelect parse(String query) {
-    // Try to parse the query
-    PlainSelect plainSelect;
-    try {
-      Select select = (Select) CCJSqlParserUtil.parse(query);
-      plainSelect = (PlainSelect) select.getSelectBody();
-    } catch (JSQLParserException e) {
-      String message = String.format("The query is malformed.\n" + "\tQuery:\n\t\t%s", query);
-      throw new IllegalArgumentException(message, e);
-    }
-
-    // Check the number of columns
-    if (plainSelect.getSelectItems().size() != 3) {
-      String message = String.format("The query is malformed.\n"
-          + "\tExpected format:\n\t\tSELECT c1::bigint, c2::jsonb, c3::geometry FROM t WHERE c\n"
-          + "\tActual query:\n\t\t%s", query);
-      throw new IllegalArgumentException(message);
-    }
-
-    // Remove all the aliases
-    for (SelectItem selectItem : plainSelect.getSelectItems()) {
-      selectItem.accept(new SelectItemVisitorAdapter() {
-        @Override
-        public void visit(SelectItem selectExpressionItem) {
-          selectExpressionItem.setAlias(null);
-        }
-      });
-    }
-
-    return plainSelect;
-  }
-}
diff --git a/baremaps-core/src/main/java/org/apache/baremaps/tilestore/postgres/PostgresQueryGenerator.java b/baremaps-core/src/main/java/org/apache/baremaps/tilestore/postgres/PostgresQueryGenerator.java
index 64f65ca7..5774bfba 100644
--- a/baremaps-core/src/main/java/org/apache/baremaps/tilestore/postgres/PostgresQueryGenerator.java
+++ b/baremaps-core/src/main/java/org/apache/baremaps/tilestore/postgres/PostgresQueryGenerator.java
@@ -18,12 +18,12 @@
 package org.apache.baremaps.tilestore.postgres;
 
 
-
 import java.util.List;
 import java.util.stream.Collectors;
 import javax.sql.DataSource;
 import org.apache.baremaps.postgres.metadata.DatabaseMetadata;
 import org.apache.baremaps.postgres.metadata.TableMetadata;
+import org.apache.baremaps.vectortile.tileset.TilesetQuery;
 
 /**
  * A generator that uses PostgreSQL metadata to generate input queries for a {@code
@@ -82,14 +82,14 @@ public class PostgresQueryGenerator {
    *
    * @return the queries
    */
-  public List<PostgresQuery> generate() {
+  public List<TilesetQuery> generate() {
     return new DatabaseMetadata(dataSource)
         .getTableMetaData(catalog, schemaPattern, tableNamePattern, types).stream()
         .filter(table -> table.primaryKeys().size() == 1)
         .filter(table -> table.getGeometryColumns().size() == 1).map(this::getLayer).toList();
   }
 
-  private PostgresQuery getLayer(TableMetadata table) {
+  private TilesetQuery getLayer(TableMetadata table) {
     String tableSchema = table.table().tableSchem();
     String tableName = table.table().tableName();
     String layer = String.format("%s.%s", tableSchema, tableName);
@@ -102,6 +102,6 @@ public class PostgresQueryGenerator {
             .collect(Collectors.joining(", ", "hstore(array[", "])"));
     String sql = String.format("SELECT %s, %s, %s FROM %s", idColumn, tagsColumns, geometryColumn,
         tableName);
-    return new PostgresQuery(layer, 0, 20, sql);
+    return new TilesetQuery(0, 20, sql);
   }
 }
diff --git a/baremaps-core/src/main/java/org/apache/baremaps/tilestore/postgres/PostgresTileStore.java b/baremaps-core/src/main/java/org/apache/baremaps/tilestore/postgres/PostgresTileStore.java
index b5edbd83..a1125033 100644
--- a/baremaps-core/src/main/java/org/apache/baremaps/tilestore/postgres/PostgresTileStore.java
+++ b/baremaps-core/src/main/java/org/apache/baremaps/tilestore/postgres/PostgresTileStore.java
@@ -18,30 +18,17 @@
 package org.apache.baremaps.tilestore.postgres;
 
 
-
 import java.io.ByteArrayOutputStream;
-import java.io.IOException;
+import java.io.OutputStream;
 import java.nio.ByteBuffer;
-import java.sql.Connection;
-import java.sql.ResultSet;
-import java.sql.SQLException;
-import java.sql.Statement;
-import java.util.LinkedHashMap;
-import java.util.List;
+import java.sql.*;
 import java.util.Map;
-import java.util.Optional;
-import java.util.stream.Collectors;
+import java.util.concurrent.ConcurrentHashMap;
 import java.util.zip.GZIPOutputStream;
 import javax.sql.DataSource;
-import net.sf.jsqlparser.expression.Expression;
-import net.sf.jsqlparser.expression.Parenthesis;
-import net.sf.jsqlparser.expression.operators.conditional.OrExpression;
-import net.sf.jsqlparser.schema.Column;
-import net.sf.jsqlparser.statement.select.Join;
 import org.apache.baremaps.tilestore.TileCoord;
 import org.apache.baremaps.tilestore.TileStore;
 import org.apache.baremaps.tilestore.TileStoreException;
-import org.apache.baremaps.tilestore.VariableUtils;
 import org.apache.baremaps.vectortile.tileset.Tileset;
 import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
@@ -49,218 +36,182 @@ import org.slf4j.LoggerFactory;
 /**
  * A read-only {@code TileStore} implementation that uses the PostgreSQL to generate vector tiles.
  * This {@code TileStore} combines the input queries, identifies common table expressions (CTE), and
- * generates a single optimized query that hits the database.
+ * generates a single optimized sql that hits the database.
  */
 public class PostgresTileStore implements TileStore {
 
   private static final Logger logger = LoggerFactory.getLogger(PostgresTileStore.class);
 
-  private static final String TILE_ENVELOPE = "st_tileenvelope(%1$s, %2$s, %3$s)";
-
-  private static final String WITH_QUERY = "with %1$s %2$s";
-
-  private static final String CTE_QUERY =
-      "%1$s as (select * from %3$s%4$s where %5$s st_intersects(%2$s, $envelope))";
-
-  private static final String CTE_WHERE = "(%s) and";
-
-  private static final String STATEMENT_QUERY =
-      "select st_asmvt(target, '%1$s', 4096, 'geom', 'id') from (%2$s) as target";
-
-  private static final String STATEMENT_LAYER_QUERY = "select " + "%1$s as id, "
-      + "(%2$s ||  jsonb_build_object('geometry', lower(replace(st_geometrytype(%3$s), 'ST_', '')))) as tags, "
-      + "st_asmvtgeom(%3$s, $envelope, 4096, 256, true) as geom " + "from %4$s %5$s";
-
-  private static final String STATEMENT_WHERE = "where %s";
-
-  private static final String UNION = " union all ";
-
-  private static final String COMMA = ", ";
-
-  private static final String SPACE = " ";
-
-  private static final String EMPTY = "";
-
-  public static final String CONTENT_ENCODING = "gzip";
-
-  public static final String CONTENT_TYPE = "application/vnd.mapbox-vector-tile";
-
   private final DataSource datasource;
 
-  private final List<PostgresQuery> queries;
+  private final Tileset tileset;
 
-  public PostgresTileStore(DataSource datasource, List<PostgresQuery> queries) {
+  /**
+   * Constructs a {@code PostgresTileStore}.
+   *
+   * @param datasource the datasource
+   * @param tileset the tileset
+   */
+  public PostgresTileStore(DataSource datasource, Tileset tileset) {
     this.datasource = datasource;
-    this.queries = queries;
+    this.tileset = tileset;
   }
 
-  public PostgresTileStore(DataSource datasource, Tileset tileset) {
-    this.datasource = datasource;
-    this.queries = tileset.getVectorLayers().stream()
-        .flatMap(layer -> layer.getQueries().stream().map(query -> new PostgresQuery(layer.getId(),
-            query.getMinzoom(), query.getMaxzoom(), query.getSql())))
-        .toList();
+  /**
+   * A cache of queries.
+   */
+  private Map<Integer, Query> cache = new ConcurrentHashMap<>();
+
+  /**
+   * A record that holds the sql of a prepared statement and the number of parameters.
+   * 
+   * @param sql
+   * @param parameters
+   */
+  protected record Query(String sql, int parameters) {
   }
 
-  /** {@inheritDoc} */
   @Override
   public ByteBuffer read(TileCoord tileCoord) throws TileStoreException {
-    String sql = withQuery(tileCoord);
-    try (Connection connection = datasource.getConnection();
-        Statement statement = connection.createStatement();
-        ByteArrayOutputStream data = new ByteArrayOutputStream()) {
-
-      int length = 0;
-      if (queries.stream().anyMatch(query -> zoomPredicate(query, tileCoord.z()))) {
-        logger.debug("Executing query: {}", sql);
-        long start = System.currentTimeMillis();
-        try (GZIPOutputStream gzip = new GZIPOutputStream(data);
-            ResultSet resultSet = statement.executeQuery(sql)) {
-          while (resultSet.next()) {
-            byte[] bytes = resultSet.getBytes(1);
-            length += bytes.length;
-            gzip.write(bytes);
-          }
-        }
-        long stop = System.currentTimeMillis();
-        long duration = stop - start;
+    var start = System.currentTimeMillis();
 
-        // Log slow queries (> 10s)
-        if (duration > 10_000) {
-          logger.warn("Executed query for tile {} in {} ms: {}", tileCoord, duration, sql);
+    // Prepare and cache the query
+    var query = cache.computeIfAbsent(tileCoord.z(), z -> prepareQuery(tileset, z));
+
+    // Fetch and compress the tile data
+    try (var connection = datasource.getConnection();
+        ByteArrayOutputStream data = new ByteArrayOutputStream();
+        var statement = connection.prepareStatement(query.sql())) {
+
+      // Set the parameters for the tile
+      for (int i = 0; i < query.parameters(); i += 3) {
+        statement.setInt(i + 1, tileCoord.z());
+        statement.setInt(i + 2, tileCoord.x());
+        statement.setInt(i + 3, tileCoord.y());
+      }
+
+      try (ResultSet resultSet = statement.executeQuery();
+          OutputStream gzip = new GZIPOutputStream(data)) {
+        while (resultSet.next()) {
+          byte[] bytes = resultSet.getBytes(1);
+          gzip.write(bytes);
         }
       }
 
-      if (length > 0) {
-        return ByteBuffer.wrap(data.toByteArray());
-      } else {
-        return null;
+      // Log slow queries (> 10s)
+      long stop = System.currentTimeMillis();
+      long duration = stop - start;
+      if (duration > 10_000) {
+        logger.warn("Executed sql for tile {} in {} ms", tileCoord, duration);
       }
-    } catch (SQLException | IOException e) {
+
+      return ByteBuffer.wrap(data.toByteArray());
+
+    } catch (Exception e) {
       throw new TileStoreException(e);
     }
   }
 
   /**
-   * Returns a WITH query for the provided tile.
+   * Prepare the sql query for a given tileset and zoom level.
    *
-   * @param tileCoord the tile
-   * @return the WITH query
-   */
-  protected String withQuery(TileCoord tileCoord) {
-    int zoom = tileCoord.z();
-    String sourceQueries = ctes(queries, zoom);
-    String targetQueries = statements(queries, zoom);
-    String withQuery = String.format(WITH_QUERY, sourceQueries, targetQueries);
-    Map<String, String> variables =
-        Map.of("envelope", tileEnvelope(tileCoord), "zoom", String.valueOf(zoom));
-    return VariableUtils.interpolate(variables, withQuery);
-  }
-
-  /**
-   * Returns the common table expressions for a list of input queries at a specified zoom level.
-   *
-   * @param queries the queries
+   * @param tileset the tileset
    * @param zoom the zoom level
-   * @return the common table expressions
+   * @return
    */
-  protected String ctes(List<PostgresQuery> queries, int zoom) {
-    return queries.stream().filter(query -> zoomPredicate(query, zoom))
-        .collect(Collectors.groupingBy(this::commonTableExpression, LinkedHashMap::new,
-            Collectors.toList()))
-        .entrySet().stream().map(entry -> cte(entry.getKey(), entry.getValue())).distinct()
-        .collect(Collectors.joining(COMMA));
-  }
+  protected static Query prepareQuery(Tileset tileset, int zoom) {
+    // Initialize a builder for the tile sql
+    var tileSql = new StringBuilder();
+    tileSql.append("SELECT (");
+
+    // Iterate over the layers and keep track of the number of layers and parameters included in the
+    // final sql
+    var layers = tileset.getVectorLayers();
+    var layerCount = 0;
+    var paramCount = 0;
+    for (var layer : layers) {
+
+      // Initialize a builder for the layer sql
+      var layerSql = new StringBuilder();
+      var layerHead = "(WITH mvtGeom AS (";
+      layerSql.append(layerHead);
+
+      // Iterate over the queries and keep track of the number of queries included in the final
+      // sql
+      var queries = layer.getQueries();
+      var queryCount = 0;
+      for (var query : queries) {
+
+        // Only include the sql if the zoom level is in the range
+        if (query.getMinzoom() <= zoom && zoom < query.getMaxzoom()) {
+
+          // Add a union between queries
+          if (queryCount > 0) {
+            layerSql.append("UNION ");
+          }
 
-  /**
-   * Returns the common table expression for a group of queries.
-   *
-   * @param group the common table expression
-   * @param queries the input queries associated with the provided group
-   * @return the common table expression
-   */
-  protected String cte(PostgresGroup group, List<PostgresQuery> queries) {
-    String alias = group.getAlias();
-    String geom = group.getSelectItems().get(2).toString();
-    String from = group.getFromItem().toString();
-    String joins = Optional.ofNullable(group.getJoins()).stream().flatMap(List::stream)
-        .map(Join::toString).collect(Collectors.joining(SPACE));
-    String where = queries.stream().map(query -> query.getAst().getWhere())
-        .map(Optional::ofNullable).map(o -> o.orElse(new Column("true"))).map(Parenthesis::new)
-        .map(Expression.class::cast).reduce(OrExpression::new)
-        .map(expression -> String.format(CTE_WHERE, expression)).orElse(EMPTY);
-    return String.format(CTE_QUERY, alias, geom, from, joins, where);
-  }
+          // Add the sql to the layer sql
+          var querySql = query.getSql()
+              .replace(";", "")
+              .replace("?", "??")
+              .replace("$zoom", String.valueOf(zoom));
+          var querySqlWithParams = String.format(
+              "SELECT ST_AsMVTGeom(t.geom, ST_TileEnvelope(?, ?, ?)) AS geom, t.tags, t.id " +
+                  "FROM (%s) AS t WHERE t.geom && ST_TileEnvelope(?, ?, ?, margin => (64.0/4096))",
+              querySql);
+          layerSql.append(querySqlWithParams);
+
+          // Increase the parameter count (e.g. ?) and sql count
+          paramCount += 6;
+          queryCount++;
+        }
+      }
 
-  /**
-   * Returns the statements for a list of input queries at a specified zoom level.
-   *
-   * @param queries the queries
-   * @param zoom the zoom level
-   * @return the statements
-   */
-  protected String statements(List<PostgresQuery> queries, int zoom) {
-    return queries.stream().filter(query -> zoomPredicate(query, zoom))
-        .collect(
-            Collectors.groupingBy(PostgresQuery::getLayer, LinkedHashMap::new, Collectors.toList()))
-        .entrySet().stream().map(entry -> layerStatements(entry.getValue(), entry.getKey()))
-        .collect(Collectors.joining(UNION));
-  }
+      // Add the tail of the layer sql
+      var layerQueryTail =
+          String.format(") SELECT ST_AsMVT(mvtGeom.*, '%s') FROM mvtGeom)", layer.getId());
+      layerSql.append(layerQueryTail);
 
-  /**
-   * Returns the statements for a list of input queries corresponding to a layer.
-   *
-   * @param queries the queries
-   * @param layer the layer name
-   * @return the statements
-   */
-  protected String layerStatements(List<PostgresQuery> queries, String layer) {
-    return String.format(STATEMENT_QUERY, layer, queries.stream()
-        .map(queryValue -> layerStatement(queryValue)).collect(Collectors.joining(UNION)));
-  }
+      // Only include the layer sql if queries were included for this layer
+      if (queryCount > 0) {
 
-  /**
-   * Returns the statement for a query in a layer.
-   *
-   * @param query the query
-   * @return the statement
-   */
-  protected String layerStatement(PostgresQuery query) {
-    String alias = commonTableExpression(query).getAlias();
-    var ast = query.getAst();
-    String id = ast.getSelectItems().get(0).toString();
-    String tags = ast.getSelectItems().get(1).toString();
-    String geom = ast.getSelectItems().get(2).toString();
-    String where = Optional.ofNullable(query.getAst().getWhere())
-        .map(expression -> String.format(STATEMENT_WHERE, expression)).orElse(EMPTY);
-    return String.format(STATEMENT_LAYER_QUERY, id, tags, geom, alias, where);
-  }
+        // Add the concatenation between layer queries
+        if (layerCount > 0) {
+          tileSql.append(" || ");
+        }
 
-  protected boolean zoomPredicate(PostgresQuery query, int zoom) {
-    return query.getMinzoom() <= zoom && zoom < query.getMaxzoom();
-  }
+        // Add the layer sql to the mvt sql
+        tileSql.append(layerSql);
 
-  protected PostgresGroup commonTableExpression(PostgresQuery query) {
-    return new PostgresGroup(
-        query.getAst().getSelectItems(),
-        query.getAst().getFromItem(),
-        query.getAst().getJoins());
-  }
+        // Increase the layer count
+        layerCount++;
+      }
+    }
+
+    // Add the tail of the tile sql
+    var tileQueryTail = ") mvtTile";
+    tileSql.append(tileQueryTail);
 
-  protected String tileEnvelope(TileCoord tileCoord) {
-    return String.format(TILE_ENVELOPE, tileCoord.z(), tileCoord.x(), tileCoord.y());
+    // Log the resulting sql
+    var sql = tileSql.toString().replace("\n", " ");
+    logger.debug("sql: {}", sql);
+
+    return new Query(sql, paramCount);
   }
 
-  /** This operation is not supported. */
+  /**
+   * This operation is not supported.
+   */
   @Override
   public void write(TileCoord tileCoord, ByteBuffer blob) {
     throw new UnsupportedOperationException("The postgis tile store is read only");
   }
 
-  /** This operation is not supported. */
+  /**
+   * This operation is not supported.
+   */
   @Override
   public void delete(TileCoord tileCoord) {
     throw new UnsupportedOperationException("The postgis tile store is read only");
   }
-
 }
diff --git a/baremaps-core/src/main/java/org/apache/baremaps/utils/PostgresUtils.java b/baremaps-core/src/main/java/org/apache/baremaps/utils/PostgresUtils.java
index cacfab00..681743fa 100644
--- a/baremaps-core/src/main/java/org/apache/baremaps/utils/PostgresUtils.java
+++ b/baremaps-core/src/main/java/org/apache/baremaps/utils/PostgresUtils.java
@@ -100,10 +100,13 @@ public final class PostgresUtils {
     if (poolSize < 1) {
       throw new IllegalArgumentException("PoolSize cannot be inferior to 1");
     }
-    var multiQueriesJdbcUrl = withAllowMultiQueriesParameter(jdbcUrl);
+
     var config = new HikariConfig();
-    config.setJdbcUrl(multiQueriesJdbcUrl);
+    config.setJdbcUrl(jdbcUrl);
     config.setMaximumPoolSize(poolSize);
+    config.addDataSourceProperty("allowMultiQueries", true);
+    config.addDataSourceProperty("prepareThreshold", 100);
+
     return new HikariDataSource(config);
   }
 
@@ -154,6 +157,10 @@ public final class PostgresUtils {
     if (datasource.getReadOnly() != null) {
       config.setReadOnly(datasource.getReadOnly());
     }
+
+    config.addDataSourceProperty("allowMultiQueries", true);
+    config.addDataSourceProperty("prepareThreshold", 100);
+
     return new HikariDataSource(config);
   }
 
diff --git a/baremaps-core/src/main/java/org/apache/baremaps/utils/SqliteUtils.java b/baremaps-core/src/main/java/org/apache/baremaps/utils/SqliteUtils.java
index 9bf6a582..d9c7d151 100644
--- a/baremaps-core/src/main/java/org/apache/baremaps/utils/SqliteUtils.java
+++ b/baremaps-core/src/main/java/org/apache/baremaps/utils/SqliteUtils.java
@@ -52,7 +52,7 @@ public final class SqliteUtils {
   public static DataSource createDataSource(Path path, boolean readOnly) {
     var sqliteConfig = new SQLiteConfig();
     sqliteConfig.setReadOnly(readOnly);
-    sqliteConfig.setCacheSize(1000000);
+    sqliteConfig.setCacheSize(-1000000);
     sqliteConfig.setPageSize(65536);
     sqliteConfig.setJournalMode(JournalMode.OFF);
     sqliteConfig.setLockingMode(LockingMode.EXCLUSIVE);
diff --git a/baremaps-core/src/main/java/org/apache/baremaps/workflow/tasks/ExportVectorTiles.java b/baremaps-core/src/main/java/org/apache/baremaps/workflow/tasks/ExportVectorTiles.java
index 06d1455c..3f5e1854 100644
--- a/baremaps-core/src/main/java/org/apache/baremaps/workflow/tasks/ExportVectorTiles.java
+++ b/baremaps-core/src/main/java/org/apache/baremaps/workflow/tasks/ExportVectorTiles.java
@@ -24,8 +24,7 @@ import com.fasterxml.jackson.databind.ObjectMapper;
 import java.io.IOException;
 import java.nio.file.Files;
 import java.nio.file.Path;
-import java.util.HashMap;
-import java.util.Map;
+import java.util.*;
 import java.util.stream.Collectors;
 import javax.sql.DataSource;
 import org.apache.baremaps.config.ConfigReader;
@@ -75,12 +74,30 @@ public record ExportVectorTiles(
         : new Envelope(-180, 180, -85.0511, 85.0511);
 
     var count = TileCoord.count(envelope, tileset.getMinzoom(), tileset.getMaxzoom());
-
-    var stream =
-        StreamUtils.stream(TileCoord.iterator(envelope, tileset.getMinzoom(), tileset.getMaxzoom()))
-            .peek(new ProgressLogger<>(count, 5000));
-
-    StreamUtils.batch(stream).forEach(new TileChannel(sourceTileStore, targetTileStore));
+    var start = System.currentTimeMillis();
+
+    var tileCoordIterator =
+        TileCoord.iterator(envelope, tileset.getMinzoom(), tileset.getMaxzoom());
+    var tileCoordStream =
+        StreamUtils.stream(tileCoordIterator).peek(new ProgressLogger<>(count, 5000));
+    var bufferedTileEntryStream = StreamUtils.bufferInCompletionOrder(tileCoordStream, tile -> {
+      try {
+        return new TileEntry(tile, sourceTileStore.read(tile));
+      } catch (TileStoreException e) {
+        throw new RuntimeException(e);
+      }
+    }, 1000);
+    var partitionedTileEntryStream = StreamUtils.partition(bufferedTileEntryStream, 1000);
+    partitionedTileEntryStream.forEach(batch -> {
+      try {
+        targetTileStore.write(batch);
+      } catch (TileStoreException e) {
+        throw new RuntimeException(e);
+      }
+    });
+
+    var stop = System.currentTimeMillis();
+    logger.info("Exported {} tiles in {}s", count, (stop - start) / 1000);
   }
 
   private TileStore sourceTileStore(Tileset tileset, DataSource datasource) {
diff --git a/baremaps-core/src/test/java/org/apache/baremaps/stream/PartitionedSpliteratorTest.java b/baremaps-core/src/test/java/org/apache/baremaps/stream/PartitionedSpliteratorTest.java
index 79476740..ad39b098 100644
--- a/baremaps-core/src/test/java/org/apache/baremaps/stream/PartitionedSpliteratorTest.java
+++ b/baremaps-core/src/test/java/org/apache/baremaps/stream/PartitionedSpliteratorTest.java
@@ -22,10 +22,10 @@ import static org.junit.Assert.assertFalse;
 import static org.junit.Assert.assertTrue;
 import static org.junit.Assert.fail;
 
+import java.util.List;
 import java.util.Spliterator;
 import java.util.concurrent.atomic.AtomicInteger;
 import java.util.stream.IntStream;
-import java.util.stream.Stream;
 import org.junit.jupiter.api.Test;
 
 class PartitionedSpliteratorTest {
@@ -67,11 +67,11 @@ class PartitionedSpliteratorTest {
     PartitionedSpliterator<Integer> stream =
         new PartitionedSpliterator<>(IntStream.range(0, 1000).spliterator(), 250);
 
-    Spliterator<Stream<Integer>> a = stream.trySplit();
-    Spliterator<Stream<Integer>> b = stream.trySplit();
-    Spliterator<Stream<Integer>> c = stream.trySplit();
-    Spliterator<Stream<Integer>> d = stream.trySplit();
-    Spliterator<Stream<Integer>> e = stream.trySplit();
+    Spliterator<List<Integer>> a = stream.trySplit();
+    Spliterator<List<Integer>> b = stream.trySplit();
+    Spliterator<List<Integer>> c = stream.trySplit();
+    Spliterator<List<Integer>> d = stream.trySplit();
+    Spliterator<List<Integer>> e = stream.trySplit();
     AtomicInteger i = new AtomicInteger();
     a.forEachRemaining(s -> s.forEach(item -> {
       assertEquals(i.get(), (long) item); // cast necessary otherwise call is ambiguous
@@ -100,10 +100,10 @@ class PartitionedSpliteratorTest {
     PartitionedSpliterator<Integer> stream =
         new PartitionedSpliterator<>(IntStream.range(0, 600).spliterator(), 250);
 
-    Spliterator<Stream<Integer>> a = stream.trySplit();
-    Spliterator<Stream<Integer>> b = stream.trySplit();
-    Spliterator<Stream<Integer>> c = stream.trySplit();
-    Spliterator<Stream<Integer>> d = stream.trySplit();
+    Spliterator<List<Integer>> a = stream.trySplit();
+    Spliterator<List<Integer>> b = stream.trySplit();
+    Spliterator<List<Integer>> c = stream.trySplit();
+    Spliterator<List<Integer>> d = stream.trySplit();
     AtomicInteger i = new AtomicInteger();
     a.forEachRemaining(s -> s.forEach(item -> {
       assertEquals(i.get(), (long) item); // cast necessary otherwise call is ambiguous
diff --git a/baremaps-core/src/test/java/org/apache/baremaps/stream/StreamUtilsTest.java b/baremaps-core/src/test/java/org/apache/baremaps/stream/StreamUtilsTest.java
index eb2ecc0b..ee1cc354 100644
--- a/baremaps-core/src/test/java/org/apache/baremaps/stream/StreamUtilsTest.java
+++ b/baremaps-core/src/test/java/org/apache/baremaps/stream/StreamUtilsTest.java
@@ -20,7 +20,6 @@ package org.apache.baremaps.stream;
 import static org.junit.jupiter.api.Assertions.assertEquals;
 
 import java.util.List;
-import java.util.stream.Collectors;
 import java.util.stream.IntStream;
 import org.junit.jupiter.api.Test;
 
@@ -29,8 +28,7 @@ class StreamUtilsTest {
   @Test
   void partition() {
     List<Integer> list = IntStream.range(0, 100).boxed().toList();
-    List<List<Integer>> partitions = StreamUtils.partition(list.stream(), 10)
-        .map(stream -> stream.collect(Collectors.toList())).toList();
+    List<List<Integer>> partitions = StreamUtils.partition(list.stream(), 10).toList();
     assertEquals(partitions.size(), 10);
   }
 }
diff --git a/baremaps-core/src/test/java/org/apache/baremaps/tilestore/postgres/JSQLParserTest.java b/baremaps-core/src/test/java/org/apache/baremaps/tilestore/postgres/JSQLParserTest.java
deleted file mode 100644
index 464eab81..00000000
--- a/baremaps-core/src/test/java/org/apache/baremaps/tilestore/postgres/JSQLParserTest.java
+++ /dev/null
@@ -1,64 +0,0 @@
-/*
- * 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.
- */
-
-package org.apache.baremaps.tilestore.postgres;
-
-import static org.junit.jupiter.api.Assertions.assertNotNull;
-
-import net.sf.jsqlparser.JSQLParserException;
-import net.sf.jsqlparser.parser.CCJSqlParserUtil;
-import net.sf.jsqlparser.statement.Statement;
-import net.sf.jsqlparser.statement.select.Select;
-import org.junit.jupiter.api.Test;
-
-class JSQLParserTest {
-
-  @Test
-  void parseArray() throws JSQLParserException {
-    Statement statement = CCJSqlParserUtil
-        .parse("SELECT id, hstore(array['tag1', 'tag2'], array[tag1, tag2]), geom FROM table");
-    assertNotNull(statement);
-  }
-
-  @Test
-  void parseWithStatement() throws JSQLParserException {
-    String sql = "WITH a AS (SELECT c FROM t) SELECT c FROM a";
-    Select select = (Select) CCJSqlParserUtil.parse(sql);
-    assertNotNull(select);
-  }
-
-  @Test
-  void parseUnionStatement() throws JSQLParserException {
-    String sql = "SELECT a FROM t1 UNION ALL SELECT a FROM t2";
-    Select select = (Select) CCJSqlParserUtil.parse(sql);
-    assertNotNull(select);
-  }
-
-  @Test
-  void parseVariable() throws JSQLParserException {
-    String sql = "SELECT $variable FROM table";
-    Select select = (Select) CCJSqlParserUtil.parse(sql);
-    assertNotNull(select);
-  }
-
-  @Test
-  void parseBoolean() throws JSQLParserException {
-    String sql = "SELECT true";
-    Select select = (Select) CCJSqlParserUtil.parse(sql);
-    assertNotNull(select);
-  }
-}
diff --git a/baremaps-core/src/test/java/org/apache/baremaps/tilestore/postgres/PostgresQueryGeneratorTest.java b/baremaps-core/src/test/java/org/apache/baremaps/tilestore/postgres/PostgresQueryGeneratorTest.java
index ae2f0a3e..e421e753 100644
--- a/baremaps-core/src/test/java/org/apache/baremaps/tilestore/postgres/PostgresQueryGeneratorTest.java
+++ b/baremaps-core/src/test/java/org/apache/baremaps/tilestore/postgres/PostgresQueryGeneratorTest.java
@@ -22,6 +22,7 @@ import static org.junit.jupiter.api.Assertions.assertEquals;
 import java.util.List;
 import javax.sql.DataSource;
 import org.apache.baremaps.openstreetmap.postgres.PostgresRepositoryTest;
+import org.apache.baremaps.vectortile.tileset.TilesetQuery;
 import org.junit.jupiter.api.Tag;
 import org.junit.jupiter.api.Test;
 
@@ -31,7 +32,7 @@ class PostgresQueryGeneratorTest extends PostgresRepositoryTest {
   @Tag("integration")
   void generate() {
     DataSource dataSource = dataSource();
-    List<PostgresQuery> queries =
+    List<TilesetQuery> queries =
         new PostgresQueryGenerator(dataSource, null, "public", null, null, "TABLE").generate();
     assertEquals(3, queries.size());
     assertEquals(
diff --git a/baremaps-core/src/test/java/org/apache/baremaps/tilestore/postgres/PostgresQueryParserTest.java b/baremaps-core/src/test/java/org/apache/baremaps/tilestore/postgres/PostgresQueryParserTest.java
deleted file mode 100644
index 56cf27c8..00000000
--- a/baremaps-core/src/test/java/org/apache/baremaps/tilestore/postgres/PostgresQueryParserTest.java
+++ /dev/null
@@ -1,107 +0,0 @@
-/*
- * 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.
- */
-
-package org.apache.baremaps.tilestore.postgres;
-
-import static org.junit.jupiter.api.Assertions.assertEquals;
-
-import java.util.Optional;
-import org.junit.jupiter.api.Test;
-
-class PostgresQueryParserTest {
-
-  @Test
-  void parse1() {
-    parse("SELECT id, tags, geom FROM table", "id", "tags", "geom", "table", Optional.empty());
-  }
-
-  @Test
-  void parse2() {
-    parse("select id, tags, geom from table", "id", "tags", "geom", "table", Optional.empty());
-  }
-
-  @Test
-  void parse3() {
-    parse("SELECT id AS a, tags AS b, geom AS c FROM table", "id", "tags", "geom", "table",
-        Optional.empty());
-  }
-
-  @Test
-  void parse4() {
-    parse("select id as a, tags as b, geom as c from table", "id", "tags", "geom", "table",
-        Optional.empty());
-  }
-
-  @Test
-  void parse5() {
-    parse("SELECT id, tags, geom FROM table WHERE condition", "id", "tags", "geom", "table",
-        Optional.of("condition"));
-  }
-
-  @Test
-  void parse6() {
-    parse(
-        "SELECT id, tags, geom FROM table WHERE tags ? 'building' AND st_geometrytype(geom) LIKE 'ST_Polygon'",
-        "id", "tags", "geom", "table",
-        Optional.of("tags ? 'building' AND st_geometrytype(geom) LIKE 'ST_Polygon'"));
-  }
-
-  @Test
-  void parse7() {
-    parse("select id, tags, geom from table where condition", "id", "tags", "geom", "table",
-        Optional.of("condition"));
-  }
-
-  @Test
-  void parse8() {
-    parse("SELECT id, hstore(ARRAY['tag1', 'tag2'], ARRAY[tag1, tag2]), geom FROM table", "id",
-        "hstore(ARRAY['tag1', 'tag2'], ARRAY[tag1, tag2])", "geom", "table", Optional.empty());
-  }
-
-  @Test
-  void parse9() {
-    parse("SELECT id, hstore('tag', tag), geom FROM table", "id", "hstore('tag', tag)", "geom",
-        "table", Optional.empty());
-  }
-
-  @Test
-  void parse10() {
-    parse("SELECT id, hstore('tag', tag) as tags, geom FROM table", "id", "hstore('tag', tag)",
-        "geom", "table", Optional.empty());
-  }
-
-  @Test
-  void parse11() {
-    parse("SELECT id, tags, st_transform(geom, '1234') as geom FROM table", "id", "tags",
-        "st_transform(geom, '1234')", "table", Optional.empty());
-  }
-
-  @Test
-  void parse12() {
-    parse("SELECT id, a(b(c), d(e)), geom FROM table", "id", "a(b(c), d(e))", "geom", "table",
-        Optional.empty());
-  }
-
-  void parse(String sql, String id, String tags, String geom, String from, Optional<String> where) {
-    PostgresQuery query = new PostgresQuery("layer", 0, 1, sql);
-    assertEquals(id, String.valueOf(query.getAst().getSelectItems().get(0)));
-    assertEquals(tags, String.valueOf(query.getAst().getSelectItems().get(1)));
-    assertEquals(geom, String.valueOf(query.getAst().getSelectItems().get(2)));
-    assertEquals(from, String.valueOf(query.getAst().getFromItem()));
-    assertEquals(where, Optional.ofNullable(query.getAst().getWhere()).map(String::valueOf));
-  }
-}
diff --git a/baremaps-core/src/test/java/org/apache/baremaps/tilestore/postgres/PostgresTileStoreTest.java b/baremaps-core/src/test/java/org/apache/baremaps/tilestore/postgres/PostgresTileStoreTest.java
index 8b0bc3fe..f146a2cf 100644
--- a/baremaps-core/src/test/java/org/apache/baremaps/tilestore/postgres/PostgresTileStoreTest.java
+++ b/baremaps-core/src/test/java/org/apache/baremaps/tilestore/postgres/PostgresTileStoreTest.java
@@ -19,46 +19,28 @@ package org.apache.baremaps.tilestore.postgres;
 
 import static org.junit.jupiter.api.Assertions.assertEquals;
 
-import java.util.Arrays;
 import java.util.List;
-import org.apache.baremaps.tilestore.TileCoord;
+import java.util.Map;
+import org.apache.baremaps.vectortile.tileset.Tileset;
+import org.apache.baremaps.vectortile.tileset.TilesetLayer;
+import org.apache.baremaps.vectortile.tileset.TilesetQuery;
 import org.junit.jupiter.api.Test;
 
 class PostgresTileStoreTest {
 
   @Test
-  void sameQueries() {
-    List<PostgresQuery> queries =
-        Arrays.asList(new PostgresQuery("a", 0, 20, "SELECT id, tags, geom FROM table"),
-            new PostgresQuery("b", 0, 20, "SELECT id, tags, geom FROM table"));
-    PostgresTileStore tileStore = new PostgresTileStore(null, queries);
-    String query = tileStore.withQuery(new TileCoord(0, 0, 10));
+  void prepareQuery() {
+    var tileset = new Tileset();
+    tileset.setMinzoom(0);
+    tileset.setMaxzoom(20);
+    tileset.setVectorLayers(List.of(
+        new TilesetLayer("a", Map.of(), "", 0, 20,
+            List.of(new TilesetQuery(0, 20, "SELECT id, tags, geom FROM table"))),
+        new TilesetLayer("b", Map.of(), "", 0, 20,
+            List.of(new TilesetQuery(0, 20, "SELECT id, tags, geom FROM table")))));
+    var query = PostgresTileStore.prepareQuery(tileset, 10);
     assertEquals(
-        "with ha14cb45b as (select * from table where ((true) OR (true)) and st_intersects(geom, st_tileenvelope(10, 0, 0))) select st_asmvt(target, 'a', 4096, 'geom', 'id') from (select id as id, (tags ||  jsonb_build_object('geometry', lower(replace(st_geometrytype(geom), 'ST_', '')))) as tags, st_asmvtgeom(geom, st_tileenvelope(10, 0, 0), 4096, 256, true) as geom from ha14cb45b ) as target union all select st_asmvt(target, 'b', 4096, 'geom', 'id') from (select id as id, (tags ||  json [...]
-        query);
-  }
-
-  @Test
-  void differentConditions1() {
-    List<PostgresQuery> queries =
-        Arrays.asList(new PostgresQuery("a", 0, 20, "SELECT id, tags, geom FROM table"),
-            new PostgresQuery("b", 0, 20, "SELECT id, tags, geom FROM table WHERE condition = 1"));
-    PostgresTileStore tileStore = new PostgresTileStore(null, queries);
-    String query = tileStore.withQuery(new TileCoord(0, 0, 10));
-    assertEquals(
-        "with ha14cb45b as (select * from table where ((true) OR (condition = 1)) and st_intersects(geom, st_tileenvelope(10, 0, 0))) select st_asmvt(target, 'a', 4096, 'geom', 'id') from (select id as id, (tags ||  jsonb_build_object('geometry', lower(replace(st_geometrytype(geom), 'ST_', '')))) as tags, st_asmvtgeom(geom, st_tileenvelope(10, 0, 0), 4096, 256, true) as geom from ha14cb45b ) as target union all select st_asmvt(target, 'b', 4096, 'geom', 'id') from (select id as id, (tags [...]
-        query);
-  }
-
-  @Test
-  void differentConditions2() {
-    List<PostgresQuery> queries = Arrays.asList(
-        new PostgresQuery("a", 0, 20, "SELECT id, tags, geom FROM table WHERE condition = 1"),
-        new PostgresQuery("b", 0, 20, "SELECT id, tags, geom FROM table WHERE condition = 2"));
-    PostgresTileStore tileStore = new PostgresTileStore(null, queries);
-    String query = tileStore.withQuery(new TileCoord(0, 0, 10));
-    assertEquals(
-        "with ha14cb45b as (select * from table where ((condition = 1) OR (condition = 2)) and st_intersects(geom, st_tileenvelope(10, 0, 0))) select st_asmvt(target, 'a', 4096, 'geom', 'id') from (select id as id, (tags ||  jsonb_build_object('geometry', lower(replace(st_geometrytype(geom), 'ST_', '')))) as tags, st_asmvtgeom(geom, st_tileenvelope(10, 0, 0), 4096, 256, true) as geom from ha14cb45b where condition = 1) as target union all select st_asmvt(target, 'b', 4096, 'geom', 'id')  [...]
-        query);
+        "SELECT ((WITH mvtGeom AS (SELECT ST_AsMVTGeom(t.geom, ST_TileEnvelope(?, ?, ?)) AS geom, t.tags, t.id FROM (SELECT id, tags, geom FROM table) AS t WHERE t.geom && ST_TileEnvelope(?, ?, ?, margin => (64.0/4096))) SELECT ST_AsMVT(mvtGeom.*, 'a') FROM mvtGeom) || (WITH mvtGeom AS (SELECT ST_AsMVTGeom(t.geom, ST_TileEnvelope(?, ?, ?)) AS geom, t.tags, t.id FROM (SELECT id, tags, geom FROM table) AS t WHERE t.geom && ST_TileEnvelope(?, ?, ?, margin => (64.0/4096))) SELECT ST_AsMVT(mv [...]
+        query.sql());
   }
 }
diff --git a/pom.xml b/pom.xml
index 7f0f308d..7bb72aff 100644
--- a/pom.xml
+++ b/pom.xml
@@ -96,7 +96,6 @@ limitations under the License.
     <version.lib.jakarta>2.1.6</version.lib.jakarta>
     <version.lib.jersey>2.37</version.lib.jersey>
     <version.lib.jmh>1.35</version.lib.jmh>
-    <version.lib.jsqlparser>4.7</version.lib.jsqlparser>
     <version.lib.jts>1.19.0</version.lib.jts>
     <version.lib.junit>5.10.0</version.lib.junit>
     <version.lib.junit-vintage>5.10.0</version.lib.junit-vintage>
@@ -170,11 +169,6 @@ limitations under the License.
         <artifactId>caffeine</artifactId>
         <version>${version.lib.caffeine}</version>
       </dependency>
-      <dependency>
-        <groupId>com.github.jsqlparser</groupId>
-        <artifactId>jsqlparser</artifactId>
-        <version>${version.lib.jsqlparser}</version>
-      </dependency>
       <dependency>
         <groupId>com.google.guava</groupId>
         <artifactId>guava</artifactId>