You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@sis.apache.org by js...@apache.org on 2018/04/10 15:20:10 UTC

svn commit: r1828826 - in /sis/branches/JDK8/storage/sis-sql/src/main/java/org/apache/sis: internal/sql/reverse/ sql/ sql/postgres/

Author: jsorel
Date: Tue Apr 10 15:20:09 2018
New Revision: 1828826

URL: http://svn.apache.org/viewvc?rev=1828826&view=rev
Log:
SQL : add SQLQuery, prepare query feature set, start postgres implementation

Added:
    sis/branches/JDK8/storage/sis-sql/src/main/java/org/apache/sis/sql/SQLQuery.java
    sis/branches/JDK8/storage/sis-sql/src/main/java/org/apache/sis/sql/SQLQueryFeatureSet.java
    sis/branches/JDK8/storage/sis-sql/src/main/java/org/apache/sis/sql/postgres/
    sis/branches/JDK8/storage/sis-sql/src/main/java/org/apache/sis/sql/postgres/PostgresDialect.java
    sis/branches/JDK8/storage/sis-sql/src/main/java/org/apache/sis/sql/postgres/PostgresProvider.java
    sis/branches/JDK8/storage/sis-sql/src/main/java/org/apache/sis/sql/postgres/PostgresStore.java
Modified:
    sis/branches/JDK8/storage/sis-sql/src/main/java/org/apache/sis/internal/sql/reverse/DataBaseModel.java
    sis/branches/JDK8/storage/sis-sql/src/main/java/org/apache/sis/sql/AbstractSQLStore.java

Modified: sis/branches/JDK8/storage/sis-sql/src/main/java/org/apache/sis/internal/sql/reverse/DataBaseModel.java
URL: http://svn.apache.org/viewvc/sis/branches/JDK8/storage/sis-sql/src/main/java/org/apache/sis/internal/sql/reverse/DataBaseModel.java?rev=1828826&r1=1828825&r2=1828826&view=diff
==============================================================================
--- sis/branches/JDK8/storage/sis-sql/src/main/java/org/apache/sis/internal/sql/reverse/DataBaseModel.java [UTF-8] (original)
+++ sis/branches/JDK8/storage/sis-sql/src/main/java/org/apache/sis/internal/sql/reverse/DataBaseModel.java [UTF-8] Tue Apr 10 15:20:09 2018
@@ -34,6 +34,7 @@ import java.util.Objects;
 import java.util.Set;
 import java.util.function.Predicate;
 import java.util.logging.Level;
+import java.util.logging.Logger;
 import org.apache.sis.feature.builder.AttributeRole;
 import org.apache.sis.feature.builder.AttributeTypeBuilder;
 import org.apache.sis.feature.builder.FeatureTypeBuilder;
@@ -86,12 +87,15 @@ public final class DataBaseModel {
     }
 
     private final AbstractSQLStore store;
+    private final Logger logger;
+    private final String databaseSchema;
+    private final String databaseTable;
+
     private FeatureNaming<PrimaryKey> pkIndex = new FeatureNaming<>();
     private Set<GenericName> typeNames = new HashSet<>();
     private FeatureNaming<FeatureType> typeIndex = new FeatureNaming<>();
     private Map<String,SchemaMetaModel> schemas;
     private Set<GenericName> nameCache;
-    private final boolean simpleTypes;
 
     //various cache while analyzing model
     private DatabaseMetaData metadata;
@@ -107,13 +111,11 @@ public final class DataBaseModel {
     private Set<String> requieredSchemas;
 
 
-    public DataBaseModel(final AbstractSQLStore store, final boolean simpleTypes){
+    public DataBaseModel(final AbstractSQLStore store, Logger logger, String schema, String table){
         this.store = store;
-        this.simpleTypes = simpleTypes;
-    }
-
-    public boolean isSimpleTypes() {
-        return simpleTypes;
+        this.logger = logger;
+        this.databaseSchema = schema;
+        this.databaseTable = table;
     }
 
     public Collection<SchemaMetaModel> getSchemaMetaModels() throws DataStoreException {
@@ -186,7 +188,6 @@ public final class DataBaseModel {
         clearCache();
         schemas = new HashMap<>();
         final SQLDialect dialect = store.getDialect();
-        final String databaseSchema = store.getDatabaseSchema();
 
         visitedSchemas = new HashSet<>();
         requieredSchemas = new HashSet<>();
@@ -313,7 +314,7 @@ public final class DataBaseModel {
 
 
         //build indexes---------------------------------------------------------
-        final String baseSchemaName = store.getDatabaseSchema();
+        final String baseSchemaName = databaseSchema;
 
         final Collection<SchemaMetaModel> candidates;
         if (baseSchemaName == null) {
@@ -327,12 +328,7 @@ public final class DataBaseModel {
            if (schema != null) {
                 for (TableMetaModel table : schema.tables.values()) {
 
-                    final FeatureTypeBuilder ft;
-                    if (simpleTypes) {
-                        ft = table.getType(TableMetaModel.View.SIMPLE_FEATURE_TYPE);
-                    } else {
-                        ft = table.getType(TableMetaModel.View.COMPLEX_FEATURE_TYPE);
-                    }
+                    final FeatureTypeBuilder ft = table.getType(TableMetaModel.View.SIMPLE_FEATURE_TYPE);
                     final GenericName name = ft.getName();
                     pkIndex.add(store, name, table.key);
                     if (table.isSubType()) {
@@ -352,7 +348,6 @@ public final class DataBaseModel {
     private SchemaMetaModel analyzeSchema(final String schemaName, final Connection cx) throws DataStoreException {
 
         final SchemaMetaModel schema = new SchemaMetaModel(schemaName);
-        final String databaseTable = store.getDatabaseTable();
 
         try {
             for (Map<String,Object> info : cacheTables.records()) {
@@ -409,7 +404,7 @@ public final class DataBaseModel {
                 Class<?> columnType = dialect.getJavaType(sqlType, sqlTypeName);
 
                 if (columnType == null) {
-                    store.getLogger().log(Level.WARNING, "No class for sql type {0}", sqlType);
+                    logger.log(Level.WARNING, "No class for sql type {0}", sqlType);
                     columnType = Object.class;
                 }
 
@@ -515,7 +510,7 @@ public final class DataBaseModel {
 
             if (cols.isEmpty()) {
                 if (Table.VALUE_TYPE_TABLE.equals(tableType)) {
-                    store.getLogger().log(Level.INFO, "No primary key found for {0}.", tableName);
+                    logger.log(Level.INFO, "No primary key found for {0}.", tableName);
                 }
             }
             table.key = new PrimaryKey(tableName, cols);
@@ -691,7 +686,7 @@ public final class DataBaseModel {
                 atb.setMaximumOccurs(1);
                 atb.setName(columnLabel);
 
-                try (Connection  cx = store.getDataSource().getConnection()) {
+                try (Connection cx = store.getDataSource().getConnection()) {
                     final Class<?> type = dialect.getJavaType(sqlType, sqlTypeName);
                     //TODO : avoid jts, global geometry interface common to all ?
 //                    if (type.equals(Geometry.class)) {
@@ -750,13 +745,13 @@ public final class DataBaseModel {
                         Integer srid = null;
                         CoordinateReferenceSystem crs = null;
                         try {
-                            srid = dialect.getGeometrySRID(store.getDatabaseSchema(), tableName, name, metas, cx);
+                            srid = dialect.getGeometrySRID(databaseSchema, tableName, name, metas, cx);
                             if (srid != null) {
                                 crs = dialect.createCRS(srid, cx);
                             }
                         } catch (SQLException e) {
                             String msg = "Error occured determing srid for " + tableName + "."+ name;
-                            store.getLogger().log(Level.WARNING, msg, e);
+                            logger.log(Level.WARNING, msg, e);
                         }
 
                         atb.setCRS(crs);
@@ -781,13 +776,13 @@ public final class DataBaseModel {
                         Integer srid = null;
                         CoordinateReferenceSystem crs = null;
                         try {
-                            srid = dialect.getGeometrySRID(store.getDatabaseSchema(), tableName, name, metas, cx);
+                            srid = dialect.getGeometrySRID(databaseSchema, tableName, name, metas, cx);
                             if (srid != null) {
                                 crs = dialect.createCRS(srid, cx);
                             }
                         } catch (SQLException e) {
                             String msg = "Error occured determing srid for " + tableName + "."+ name;
-                            store.getLogger().log(Level.WARNING, msg, e);
+                            logger.log(Level.WARNING, msg, e);
                         }
 
                         atb.setCRS(crs);

Modified: sis/branches/JDK8/storage/sis-sql/src/main/java/org/apache/sis/sql/AbstractSQLStore.java
URL: http://svn.apache.org/viewvc/sis/branches/JDK8/storage/sis-sql/src/main/java/org/apache/sis/sql/AbstractSQLStore.java?rev=1828826&r1=1828825&r2=1828826&view=diff
==============================================================================
--- sis/branches/JDK8/storage/sis-sql/src/main/java/org/apache/sis/sql/AbstractSQLStore.java [UTF-8] (original)
+++ sis/branches/JDK8/storage/sis-sql/src/main/java/org/apache/sis/sql/AbstractSQLStore.java [UTF-8] Tue Apr 10 15:20:09 2018
@@ -16,13 +16,11 @@
  */
 package org.apache.sis.sql;
 
-import java.util.logging.Logger;
 import javax.sql.DataSource;
-import org.apache.sis.feature.builder.FeatureTypeBuilder;
-import org.apache.sis.internal.sql.reverse.RelationMetaModel;
+import org.apache.sis.internal.sql.reverse.DataBaseModel;
 import org.apache.sis.sql.dialect.SQLDialect;
 import org.apache.sis.storage.DataStore;
-import org.opengis.feature.AttributeType;
+import org.apache.sis.storage.FeatureSet;
 
 /**
  * Parent store class for DataStore implementation using java JDBC backend.
@@ -41,12 +39,18 @@ public abstract class AbstractSQLStore e
      */
     public abstract SQLDialect getDialect();
 
-    public abstract Logger getLogger();
-
     public abstract DataSource getDataSource();
 
-    public abstract String getDatabaseSchema();
+    public abstract DataBaseModel getDatabaseModel();
 
-    public abstract String getDatabaseTable();
+    /**
+     * Execute a query directly on the database.
+     *
+     * @param query , not null
+     * @return resulting features from given query.
+     */
+    public FeatureSet query(SQLQuery query) {
+        return new SQLQueryFeatureSet(this, query);
+    }
 
 }

Added: sis/branches/JDK8/storage/sis-sql/src/main/java/org/apache/sis/sql/SQLQuery.java
URL: http://svn.apache.org/viewvc/sis/branches/JDK8/storage/sis-sql/src/main/java/org/apache/sis/sql/SQLQuery.java?rev=1828826&view=auto
==============================================================================
--- sis/branches/JDK8/storage/sis-sql/src/main/java/org/apache/sis/sql/SQLQuery.java (added)
+++ sis/branches/JDK8/storage/sis-sql/src/main/java/org/apache/sis/sql/SQLQuery.java Tue Apr 10 15:20:09 2018
@@ -0,0 +1,68 @@
+/*
+ * 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.sis.sql;
+
+import org.apache.sis.storage.Query;
+import org.apache.sis.util.ArgumentChecks;
+
+/**
+ * An SQL query executed directly on the database.
+ * Such query should be transfered to the JDBC without or with minimum modifications.
+ *
+ * @author Johann Sorel (Geomatys)
+ * @version 1.0
+ * @since   1.0
+ * @module
+ */
+public class SQLQuery implements Query {
+
+    private final String statement;
+    private final String name;
+
+    /**
+     * Create a new SQL query object.
+     *
+     * @param statement SQL query text, not null
+     * @param name common name, used for returned feature types, not null
+     */
+    public SQLQuery(String statement, String name) {
+        ArgumentChecks.ensureNonNull("statement", statement);
+        ArgumentChecks.ensureNonNull("name", name);
+        this.statement = statement;
+        this.name = name;
+    }
+
+    /**
+     * Returns the query text.
+     * Example : SELECT * FROM Road
+     *
+     * @return SQL query text, not null
+     */
+    public String getStatement() {
+        return statement;
+    }
+
+    /**
+     * Returns the query name.
+     * This name is used to build the returned feature type.
+     *
+     * @return common query name
+     */
+    public String getName() {
+        return name;
+    }
+}

Added: sis/branches/JDK8/storage/sis-sql/src/main/java/org/apache/sis/sql/SQLQueryFeatureSet.java
URL: http://svn.apache.org/viewvc/sis/branches/JDK8/storage/sis-sql/src/main/java/org/apache/sis/sql/SQLQueryFeatureSet.java?rev=1828826&view=auto
==============================================================================
--- sis/branches/JDK8/storage/sis-sql/src/main/java/org/apache/sis/sql/SQLQueryFeatureSet.java (added)
+++ sis/branches/JDK8/storage/sis-sql/src/main/java/org/apache/sis/sql/SQLQueryFeatureSet.java Tue Apr 10 15:20:09 2018
@@ -0,0 +1,90 @@
+/*
+ * 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.sis.sql;
+
+import java.sql.Connection;
+import java.sql.ResultSet;
+import java.sql.SQLException;
+import java.sql.Statement;
+import java.util.stream.Stream;
+import org.apache.sis.storage.DataStoreException;
+import org.apache.sis.storage.FeatureSet;
+import org.apache.sis.storage.event.ChangeEvent;
+import org.apache.sis.storage.event.ChangeListener;
+import org.opengis.feature.Feature;
+import org.opengis.feature.FeatureType;
+import org.opengis.geometry.Envelope;
+import org.opengis.metadata.Metadata;
+
+/**
+ * A FeatureSet above a custom SQL query.
+ *
+ * @author Johann Sorel (Geomatys)
+ * @version 1.0
+ * @since   1.0
+ * @module
+ */
+final class SQLQueryFeatureSet implements FeatureSet {
+
+    private final AbstractSQLStore store;
+    private final SQLQuery query;
+    private FeatureType type;
+
+    public SQLQueryFeatureSet(AbstractSQLStore store, SQLQuery query) {
+        this.store = store;
+        this.query = query;
+    }
+
+    @Override
+    public FeatureType getType() throws DataStoreException {
+        if (type == null) {
+            final String sql = query.getStatement();
+            try (Connection cnx = store.getDataSource().getConnection();
+                 Statement stmt = cnx.createStatement();
+                 ResultSet rs = stmt.executeQuery(sql)) {
+                type = store.getDatabaseModel().analyzeResult(rs, query.getName());
+            } catch (SQLException ex) {
+                throw new DataStoreException(ex);
+            }
+        }
+        return type;
+    }
+
+    @Override
+    public Stream<Feature> features(boolean parallel) throws DataStoreException {
+        throw new UnsupportedOperationException("Not supported yet.");
+    }
+
+    @Override
+    public Envelope getEnvelope() throws DataStoreException {
+        return null;
+    }
+
+    @Override
+    public Metadata getMetadata() throws DataStoreException {
+        throw new UnsupportedOperationException("Not supported yet.");
+    }
+
+    @Override
+    public <T extends ChangeEvent> void addListener(ChangeListener<? super T> listener, Class<T> eventType) {
+    }
+
+    @Override
+    public <T extends ChangeEvent> void removeListener(ChangeListener<? super T> listener, Class<T> eventType) {
+    }
+
+}

Added: sis/branches/JDK8/storage/sis-sql/src/main/java/org/apache/sis/sql/postgres/PostgresDialect.java
URL: http://svn.apache.org/viewvc/sis/branches/JDK8/storage/sis-sql/src/main/java/org/apache/sis/sql/postgres/PostgresDialect.java?rev=1828826&view=auto
==============================================================================
--- sis/branches/JDK8/storage/sis-sql/src/main/java/org/apache/sis/sql/postgres/PostgresDialect.java (added)
+++ sis/branches/JDK8/storage/sis-sql/src/main/java/org/apache/sis/sql/postgres/PostgresDialect.java Tue Apr 10 15:20:09 2018
@@ -0,0 +1,107 @@
+/*
+ * 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.sis.sql.postgres;
+
+import java.sql.Connection;
+import java.sql.ResultSet;
+import java.sql.SQLException;
+import java.util.HashSet;
+import java.util.Map;
+import java.util.Set;
+import org.apache.sis.internal.sql.SingleAttributeTypeBuilder;
+import org.apache.sis.internal.sql.reverse.ColumnMetaModel;
+import org.apache.sis.sql.dialect.SQLDialect;
+import org.apache.sis.storage.DataStoreException;
+import org.opengis.referencing.crs.CoordinateReferenceSystem;
+
+/**
+ *
+ * @author Johann Sorel (Geomatys)
+ * @version 1.0
+ * @since   1.0
+ * @module
+ */
+class PostgresDialect implements SQLDialect {
+
+    private static final Set<String> IGNORE_TABLES = new HashSet<>();
+    static {
+        //postgis 1+ geometry and referencing
+        IGNORE_TABLES.add("spatial_ref_sys");
+        IGNORE_TABLES.add("geometry_columns");
+        IGNORE_TABLES.add("geography_columns");
+        //postgis 2 raster
+        IGNORE_TABLES.add("raster_columns");
+        IGNORE_TABLES.add("raster_overviews");
+    }
+
+
+    @Override
+    public boolean supportGlobalMetadata() {
+        return true;
+    }
+
+    @Override
+    public boolean ignoreTable(String name) {
+        return IGNORE_TABLES.contains(name.toLowerCase());
+    }
+
+    @Override
+    public Class<?> getJavaType(int sqlType, String sqlTypeName) {
+        throw new UnsupportedOperationException("Not supported yet."); //To change body of generated methods, choose Tools | Templates.
+    }
+
+    @Override
+    public void encodeColumnName(StringBuilder sql, String name) {
+        throw new UnsupportedOperationException("Not supported yet."); //To change body of generated methods, choose Tools | Templates.
+    }
+
+    @Override
+    public void encodeSchemaAndTableName(StringBuilder sql, String databaseSchema, String tableName) {
+        throw new UnsupportedOperationException("Not supported yet."); //To change body of generated methods, choose Tools | Templates.
+    }
+
+    @Override
+    public Object nextValue(ColumnMetaModel column, Connection cx) throws SQLException, DataStoreException {
+        throw new UnsupportedOperationException("Not supported yet."); //To change body of generated methods, choose Tools | Templates.
+    }
+
+    @Override
+    public String getColumnSequence(Connection cx, String schemaName, String tableName, String columnName) throws SQLException {
+        throw new UnsupportedOperationException("Not supported yet."); //To change body of generated methods, choose Tools | Templates.
+    }
+
+    @Override
+    public void decodeColumnType(SingleAttributeTypeBuilder atb, Connection cx, String typeName, int datatype, String schemaName, String tableName, String columnName) throws SQLException {
+        throw new UnsupportedOperationException("Not supported yet."); //To change body of generated methods, choose Tools | Templates.
+    }
+
+    @Override
+    public void decodeGeometryColumnType(SingleAttributeTypeBuilder atb, Connection cx, ResultSet rs, int columnIndex, boolean customquery) throws SQLException {
+        throw new UnsupportedOperationException("Not supported yet."); //To change body of generated methods, choose Tools | Templates.
+    }
+
+    @Override
+    public Integer getGeometrySRID(String schemaName, String tableName, String columnName, Map<String, Object> metas, Connection cx) throws SQLException {
+        throw new UnsupportedOperationException("Not supported yet."); //To change body of generated methods, choose Tools | Templates.
+    }
+
+    @Override
+    public CoordinateReferenceSystem createCRS(int srid, Connection cx) throws SQLException {
+        throw new UnsupportedOperationException("Not supported yet."); //To change body of generated methods, choose Tools | Templates.
+    }
+
+}

Added: sis/branches/JDK8/storage/sis-sql/src/main/java/org/apache/sis/sql/postgres/PostgresProvider.java
URL: http://svn.apache.org/viewvc/sis/branches/JDK8/storage/sis-sql/src/main/java/org/apache/sis/sql/postgres/PostgresProvider.java?rev=1828826&view=auto
==============================================================================
--- sis/branches/JDK8/storage/sis-sql/src/main/java/org/apache/sis/sql/postgres/PostgresProvider.java (added)
+++ sis/branches/JDK8/storage/sis-sql/src/main/java/org/apache/sis/sql/postgres/PostgresProvider.java Tue Apr 10 15:20:09 2018
@@ -0,0 +1,57 @@
+/*
+ * 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.sis.sql.postgres;
+
+import org.apache.sis.storage.DataStore;
+import org.apache.sis.storage.DataStoreException;
+import org.apache.sis.storage.DataStoreProvider;
+import org.apache.sis.storage.ProbeResult;
+import org.apache.sis.storage.StorageConnector;
+import org.opengis.parameter.ParameterDescriptorGroup;
+
+/**
+ *
+ * @author Johann Sorel (Geomatys)
+ * @version 1.0
+ * @since   1.0
+ * @module
+ */
+public class PostgresProvider extends DataStoreProvider {
+
+    public static final String NAME = "postgres";
+
+    @Override
+    public String getShortName() {
+        return NAME;
+    }
+
+    @Override
+    public ParameterDescriptorGroup getOpenParameters() {
+        throw new UnsupportedOperationException("Not supported yet."); //To change body of generated methods, choose Tools | Templates.
+    }
+
+    @Override
+    public ProbeResult probeContent(StorageConnector connector) throws DataStoreException {
+        throw new UnsupportedOperationException("Not supported yet."); //To change body of generated methods, choose Tools | Templates.
+    }
+
+    @Override
+    public DataStore open(StorageConnector connector) throws DataStoreException {
+        throw new UnsupportedOperationException("Not supported yet."); //To change body of generated methods, choose Tools | Templates.
+    }
+
+}

Added: sis/branches/JDK8/storage/sis-sql/src/main/java/org/apache/sis/sql/postgres/PostgresStore.java
URL: http://svn.apache.org/viewvc/sis/branches/JDK8/storage/sis-sql/src/main/java/org/apache/sis/sql/postgres/PostgresStore.java?rev=1828826&view=auto
==============================================================================
--- sis/branches/JDK8/storage/sis-sql/src/main/java/org/apache/sis/sql/postgres/PostgresStore.java (added)
+++ sis/branches/JDK8/storage/sis-sql/src/main/java/org/apache/sis/sql/postgres/PostgresStore.java Tue Apr 10 15:20:09 2018
@@ -0,0 +1,94 @@
+/*
+ * 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.sis.sql.postgres;
+
+import java.util.logging.Logger;
+import javax.sql.DataSource;
+import org.apache.sis.internal.sql.reverse.DataBaseModel;
+import org.apache.sis.sql.AbstractSQLStore;
+import org.apache.sis.sql.dialect.SQLDialect;
+import org.apache.sis.storage.DataStoreException;
+import org.apache.sis.storage.event.ChangeEvent;
+import org.apache.sis.storage.event.ChangeListener;
+import org.apache.sis.util.logging.Logging;
+import org.opengis.metadata.Metadata;
+import org.opengis.parameter.ParameterValueGroup;
+
+/**
+ *
+ * @author Johann Sorel (Geomatys)
+ * @version 1.0
+ * @since   1.0
+ * @module
+ */
+class PostgresStore extends AbstractSQLStore {
+
+    private static final Logger LOGGER = Logging.getLogger("org.apache.sis.sql");
+
+    private final PostgresDialect dialect = new PostgresDialect();
+
+    private final DataSource source;
+    private final String schema;
+    private final String table;
+
+    private final DataBaseModel model;
+
+    public PostgresStore(DataSource source, String schema, String table) {
+        this.source = source;
+        this.schema = schema;
+        this.table = table;
+        this.model = new DataBaseModel(this, LOGGER, schema, table);
+    }
+
+    @Override
+    public SQLDialect getDialect() {
+        return dialect;
+    }
+
+    @Override
+    public DataSource getDataSource() {
+        return source;
+    }
+
+    @Override
+    public DataBaseModel getDatabaseModel() {
+        return model;
+    }
+
+    @Override
+    public ParameterValueGroup getOpenParameters() {
+        throw new UnsupportedOperationException("Not supported yet.");
+    }
+
+    @Override
+    public Metadata getMetadata() throws DataStoreException {
+        throw new UnsupportedOperationException("Not supported yet.");
+    }
+
+    @Override
+    public void close() throws DataStoreException {
+    }
+
+    @Override
+    public <T extends ChangeEvent> void addListener(ChangeListener<? super T> listener, Class<T> eventType) {
+    }
+
+    @Override
+    public <T extends ChangeEvent> void removeListener(ChangeListener<? super T> listener, Class<T> eventType) {
+    }
+
+}