You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@ignite.apache.org by sb...@apache.org on 2015/06/17 11:10:19 UTC

[23/29] incubator-ignite git commit: #IGNITE-960 Added tests.

#IGNITE-960 Added tests.


Project: http://git-wip-us.apache.org/repos/asf/incubator-ignite/repo
Commit: http://git-wip-us.apache.org/repos/asf/incubator-ignite/commit/d530c8c6
Tree: http://git-wip-us.apache.org/repos/asf/incubator-ignite/tree/d530c8c6
Diff: http://git-wip-us.apache.org/repos/asf/incubator-ignite/diff/d530c8c6

Branch: refs/heads/ignite-1020
Commit: d530c8c64477634576007ce6cacf21fc1113f93c
Parents: dd8048b
Author: nikolay_tikhonov <nt...@gridgain.com>
Authored: Tue Jun 16 18:47:55 2015 +0300
Committer: nikolay_tikhonov <nt...@gridgain.com>
Committed: Tue Jun 16 18:47:55 2015 +0300

----------------------------------------------------------------------
 .../store/jdbc/CacheJdbcBlobStoreFactory.java   |   2 +-
 .../store/jdbc/CacheJdbcPojoStoreFactory.java   |   4 +-
 modules/hibernate/pom.xml                       |  14 ++
 .../CacheHibernateBlobStoreFactory.java         |  62 +++++-
 .../hibernate/src/test/config/factory-cache.xml |  59 ++++++
 .../src/test/config/factory-cache1.xml          |  61 ++++++
 .../config/factory-incorrect-store-cache.xml    |  56 +++++
 .../CacheHibernateStoreFactorySelfTest.java     | 207 +++++++++++++++++++
 8 files changed, 457 insertions(+), 8 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/d530c8c6/modules/core/src/main/java/org/apache/ignite/cache/store/jdbc/CacheJdbcBlobStoreFactory.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/cache/store/jdbc/CacheJdbcBlobStoreFactory.java b/modules/core/src/main/java/org/apache/ignite/cache/store/jdbc/CacheJdbcBlobStoreFactory.java
index 29a83c9..c145641 100644
--- a/modules/core/src/main/java/org/apache/ignite/cache/store/jdbc/CacheJdbcBlobStoreFactory.java
+++ b/modules/core/src/main/java/org/apache/ignite/cache/store/jdbc/CacheJdbcBlobStoreFactory.java
@@ -92,7 +92,7 @@ public class CacheJdbcBlobStoreFactory<K, V> implements Factory<CacheJdbcBlobSto
     private String dataSrcBean;
 
     /** Data source. */
-    private DataSource dataSrc;
+    private transient DataSource dataSrc;
 
     /** Application context. */
     @SpringApplicationContextResource

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/d530c8c6/modules/core/src/main/java/org/apache/ignite/cache/store/jdbc/CacheJdbcPojoStoreFactory.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/cache/store/jdbc/CacheJdbcPojoStoreFactory.java b/modules/core/src/main/java/org/apache/ignite/cache/store/jdbc/CacheJdbcPojoStoreFactory.java
index 4cb680a..54f890f 100644
--- a/modules/core/src/main/java/org/apache/ignite/cache/store/jdbc/CacheJdbcPojoStoreFactory.java
+++ b/modules/core/src/main/java/org/apache/ignite/cache/store/jdbc/CacheJdbcPojoStoreFactory.java
@@ -63,10 +63,10 @@ public class CacheJdbcPojoStoreFactory<K, V> implements Factory<CacheJdbcPojoSto
     private String dataSrcBean;
 
     /** Data source. */
-    private DataSource dataSrc;
+    private transient DataSource dataSrc;
 
     /** Database dialect. */
-    protected JdbcDialect dialect;
+    private transient JdbcDialect dialect;
 
     /** Application context. */
     @SpringApplicationContextResource

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/d530c8c6/modules/hibernate/pom.xml
----------------------------------------------------------------------
diff --git a/modules/hibernate/pom.xml b/modules/hibernate/pom.xml
index 10d91a2..d7e098c 100644
--- a/modules/hibernate/pom.xml
+++ b/modules/hibernate/pom.xml
@@ -90,6 +90,20 @@
         </dependency>
 
         <dependency>
+            <groupId>org.apache.ignite</groupId>
+            <artifactId>ignite-spring</artifactId>
+            <version>${project.version}</version>
+            <scope>test</scope>
+        </dependency>
+
+        <dependency>
+            <groupId>org.apache.ignite</groupId>
+            <artifactId>ignite-log4j</artifactId>
+            <version>${project.version}</version>
+            <scope>test</scope>
+        </dependency>
+
+        <dependency>
             <groupId>org.springframework</groupId>
             <artifactId>spring-beans</artifactId>
             <version>${spring.version}</version>

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/d530c8c6/modules/hibernate/src/main/java/org/apache/ignite/cache/store/hibernate/CacheHibernateBlobStoreFactory.java
----------------------------------------------------------------------
diff --git a/modules/hibernate/src/main/java/org/apache/ignite/cache/store/hibernate/CacheHibernateBlobStoreFactory.java b/modules/hibernate/src/main/java/org/apache/ignite/cache/store/hibernate/CacheHibernateBlobStoreFactory.java
index 6b89a9e..5154faa 100644
--- a/modules/hibernate/src/main/java/org/apache/ignite/cache/store/hibernate/CacheHibernateBlobStoreFactory.java
+++ b/modules/hibernate/src/main/java/org/apache/ignite/cache/store/hibernate/CacheHibernateBlobStoreFactory.java
@@ -17,9 +17,13 @@
 
 package org.apache.ignite.cache.store.hibernate;
 
+import org.apache.ignite.*;
 import org.apache.ignite.configuration.*;
+import org.apache.ignite.internal.*;
+import org.apache.ignite.internal.util.spring.*;
 import org.apache.ignite.internal.util.tostring.*;
 import org.apache.ignite.internal.util.typedef.internal.*;
+import org.apache.ignite.resources.*;
 import org.hibernate.*;
 
 import javax.cache.configuration.*;
@@ -119,7 +123,10 @@ import java.util.*;
 public class CacheHibernateBlobStoreFactory<K, V> implements Factory<CacheHibernateBlobStore<K, V>> {
     /** Session factory. */
     @GridToStringExclude
-    private SessionFactory sesFactory;
+    private transient SessionFactory sesFactory;
+
+    /** Session factory bean name. */
+    private String sesFactoryBean;
 
     /** Path to hibernate configuration file. */
     private String hibernateCfgPath;
@@ -128,15 +135,38 @@ public class CacheHibernateBlobStoreFactory<K, V> implements Factory<CacheHibern
     @GridToStringExclude
     private Properties hibernateProps;
 
+    /** Application context. */
+    @SpringApplicationContextResource
+    private Object appContext;
 
     /** {@inheritDoc} */
     @Override public CacheHibernateBlobStore<K, V> create() {
         CacheHibernateBlobStore<K, V> store = new CacheHibernateBlobStore<>();
 
-        store.setSessionFactory(sesFactory);
         store.setHibernateConfigurationPath(hibernateCfgPath);
         store.setHibernateProperties(hibernateProps);
 
+        if (sesFactory != null)
+            store.setSessionFactory(sesFactory);
+        else if (sesFactoryBean != null) {
+            if (appContext == null)
+                throw new IgniteException("Spring application context resource is not injected.");
+
+            IgniteSpringHelper spring;
+
+            try {
+                spring = IgniteComponentType.SPRING.create(false);
+
+                SessionFactory sesFac = spring.loadBeanFromAppContext(appContext, sesFactoryBean);
+
+                store.setSessionFactory(sesFac);
+            }
+            catch (IgniteCheckedException e) {
+                throw new IgniteException("Failed to load bean in application context [beanName=" + sesFactoryBean +
+                        ", igniteConfig=" + appContext + ']');
+            }
+        }
+
         return store;
     }
 
@@ -144,10 +174,26 @@ public class CacheHibernateBlobStoreFactory<K, V> implements Factory<CacheHibern
      * Sets session factory.
      *
      * @param sesFactory Session factory.
+     * @return {@code This} for chaining.
      * @see CacheHibernateBlobStore#setSessionFactory(SessionFactory)
      */
-    public void setSessionFactory(SessionFactory sesFactory) {
+    public CacheHibernateBlobStoreFactory<K, V> setSessionFactory(SessionFactory sesFactory) {
         this.sesFactory = sesFactory;
+
+        return this;
+    }
+
+    /**
+     * Sets name of the data source bean.
+     *
+     * @param sesFactory Session factory bean name.
+     * @return {@code This} for chaining.
+     * @see CacheHibernateBlobStore#setSessionFactory(SessionFactory)
+     */
+    public CacheHibernateBlobStoreFactory<K, V> setSessionFactoryBean(String sesFactory) {
+        this.sesFactoryBean = sesFactory;
+
+        return this;
     }
 
     /**
@@ -157,20 +203,26 @@ public class CacheHibernateBlobStoreFactory<K, V> implements Factory<CacheHibern
      *
      * @param hibernateCfgPath URL or file path or classpath resource
      *      pointing to hibernate configuration XML file.
+     * @return {@code This} for chaining.
      * @see CacheHibernateBlobStore#setHibernateConfigurationPath(String)
      */
-    public void setHibernateConfigurationPath(String hibernateCfgPath) {
+    public CacheHibernateBlobStoreFactory<K, V> setHibernateConfigurationPath(String hibernateCfgPath) {
         this.hibernateCfgPath = hibernateCfgPath;
+
+        return this;
     }
 
     /**
      * Sets Hibernate properties.
      *
      * @param hibernateProps Hibernate properties.
+     * @return {@code This} for chaining.
      * @see CacheHibernateBlobStore#setHibernateProperties(Properties)
      */
-    public void setHibernateProperties(Properties hibernateProps) {
+    public CacheHibernateBlobStoreFactory<K, V> setHibernateProperties(Properties hibernateProps) {
         this.hibernateProps = hibernateProps;
+
+        return this;
     }
 
     /** {@inheritDoc} */

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/d530c8c6/modules/hibernate/src/test/config/factory-cache.xml
----------------------------------------------------------------------
diff --git a/modules/hibernate/src/test/config/factory-cache.xml b/modules/hibernate/src/test/config/factory-cache.xml
new file mode 100644
index 0000000..a251846
--- /dev/null
+++ b/modules/hibernate/src/test/config/factory-cache.xml
@@ -0,0 +1,59 @@
+<?xml version="1.0" encoding="UTF-8"?>
+
+<!--
+  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.
+-->
+<beans xmlns="http://www.springframework.org/schema/beans"
+       xmlns:xsi="http://www.w3.org/2001/XMLSchema-instance"
+       xsi:schemaLocation="
+        http://www.springframework.org/schema/beans
+        http://www.springframework.org/schema/beans/spring-beans.xsd">
+
+    <bean id="simpleSessionFactory"
+          class="org.apache.ignite.cache.store.hibernate.CacheHibernateStoreFactorySelfTest$DummySessionFactoryExt"/>
+
+    <bean id="ignite.cfg" class="org.apache.ignite.configuration.IgniteConfiguration">
+        <property name="cacheConfiguration">
+            <list>
+                <bean class="org.apache.ignite.configuration.CacheConfiguration">
+                    <property name="name" value="test"/>
+                    <property name="atomicityMode" value="ATOMIC"/>
+                    <property name="backups" value="1"/>
+                    <property name="cacheStoreFactory">
+                        <bean class="org.apache.ignite.cache.store.hibernate.CacheHibernateBlobStoreFactory">
+                            <property name="sessionFactoryBean" value = "simpleSessionFactory"/>
+                        </bean>
+                    </property>
+                </bean>
+            </list>
+        </property>
+
+        <!-- Explicitly configure TCP discovery SPI to provide list of initial nodes. -->
+        <property name="discoverySpi">
+            <bean class="org.apache.ignite.spi.discovery.tcp.TcpDiscoverySpi">
+                <property name="ipFinder">
+                    <bean class="org.apache.ignite.spi.discovery.tcp.ipfinder.vm.TcpDiscoveryVmIpFinder">
+                        <property name="addresses">
+                            <list>
+                                <value>127.0.0.1:47500..47509</value>
+                            </list>
+                        </property>
+                    </bean>
+                </property>
+            </bean>
+        </property>
+    </bean>
+</beans>

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/d530c8c6/modules/hibernate/src/test/config/factory-cache1.xml
----------------------------------------------------------------------
diff --git a/modules/hibernate/src/test/config/factory-cache1.xml b/modules/hibernate/src/test/config/factory-cache1.xml
new file mode 100644
index 0000000..7f751fb
--- /dev/null
+++ b/modules/hibernate/src/test/config/factory-cache1.xml
@@ -0,0 +1,61 @@
+<?xml version="1.0" encoding="UTF-8"?>
+
+<!--
+  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.
+-->
+<beans xmlns="http://www.springframework.org/schema/beans"
+       xmlns:xsi="http://www.w3.org/2001/XMLSchema-instance"
+       xsi:schemaLocation="
+        http://www.springframework.org/schema/beans
+        http://www.springframework.org/schema/beans/spring-beans.xsd">
+
+    <bean id="simpleSessionFactory1"
+          class="org.apache.ignite.cache.store.hibernate.CacheHibernateStoreFactorySelfTest$DummySessionFactory"/>
+
+    <bean id="ignite.cfg" class="org.apache.ignite.configuration.IgniteConfiguration">
+        <property name="gridName" value="ignite1"/>
+
+        <property name="cacheConfiguration">
+            <list>
+                <bean class="org.apache.ignite.configuration.CacheConfiguration">
+                    <property name="name" value="test"/>
+                    <property name="atomicityMode" value="ATOMIC"/>
+                    <property name="backups" value="1"/>
+                    <property name="cacheStoreFactory">
+                        <bean class="org.apache.ignite.cache.store.hibernate.CacheHibernateBlobStoreFactory">
+                            <property name="sessionFactoryBean" value = "simpleSessionFactory1"/>
+                        </bean>
+                    </property>
+                </bean>
+            </list>
+        </property>
+
+        <!-- Explicitly configure TCP discovery SPI to provide list of initial nodes. -->
+        <property name="discoverySpi">
+            <bean class="org.apache.ignite.spi.discovery.tcp.TcpDiscoverySpi">
+                <property name="ipFinder">
+                    <bean class="org.apache.ignite.spi.discovery.tcp.ipfinder.vm.TcpDiscoveryVmIpFinder">
+                        <property name="addresses">
+                            <list>
+                                <value>127.0.0.1:47500..47509</value>
+                            </list>
+                        </property>
+                    </bean>
+                </property>
+            </bean>
+        </property>
+    </bean>
+</beans>

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/d530c8c6/modules/hibernate/src/test/config/factory-incorrect-store-cache.xml
----------------------------------------------------------------------
diff --git a/modules/hibernate/src/test/config/factory-incorrect-store-cache.xml b/modules/hibernate/src/test/config/factory-incorrect-store-cache.xml
new file mode 100644
index 0000000..459930c
--- /dev/null
+++ b/modules/hibernate/src/test/config/factory-incorrect-store-cache.xml
@@ -0,0 +1,56 @@
+<?xml version="1.0" encoding="UTF-8"?>
+
+<!--
+  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.
+-->
+<beans xmlns="http://www.springframework.org/schema/beans"
+       xmlns:xsi="http://www.w3.org/2001/XMLSchema-instance"
+       xsi:schemaLocation="
+        http://www.springframework.org/schema/beans
+        http://www.springframework.org/schema/beans/spring-beans.xsd">
+
+    <bean id="ignite.cfg" class="org.apache.ignite.configuration.IgniteConfiguration">
+        <property name="cacheConfiguration">
+            <list>
+                <bean class="org.apache.ignite.configuration.CacheConfiguration">
+                    <property name="name" value="test"/>
+                    <property name="atomicityMode" value="ATOMIC"/>
+                    <property name="backups" value="1"/>
+                    <property name="cacheStoreFactory">
+                        <bean class="org.apache.ignite.cache.store.hibernate.CacheHibernateBlobStoreFactory">
+                            <property name="sessionFactoryBean" value = "simpleSessionFactory1"/>
+                        </bean>
+                    </property>
+                </bean>
+            </list>
+        </property>
+
+        <!-- Explicitly configure TCP discovery SPI to provide list of initial nodes. -->
+        <property name="discoverySpi">
+            <bean class="org.apache.ignite.spi.discovery.tcp.TcpDiscoverySpi">
+                <property name="ipFinder">
+                    <bean class="org.apache.ignite.spi.discovery.tcp.ipfinder.vm.TcpDiscoveryVmIpFinder">
+                        <property name="addresses">
+                            <list>
+                                <value>127.0.0.1:47500..47509</value>
+                            </list>
+                        </property>
+                    </bean>
+                </property>
+            </bean>
+        </property>
+    </bean>
+</beans>

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/d530c8c6/modules/hibernate/src/test/java/org/apache/ignite/cache/store/hibernate/CacheHibernateStoreFactorySelfTest.java
----------------------------------------------------------------------
diff --git a/modules/hibernate/src/test/java/org/apache/ignite/cache/store/hibernate/CacheHibernateStoreFactorySelfTest.java b/modules/hibernate/src/test/java/org/apache/ignite/cache/store/hibernate/CacheHibernateStoreFactorySelfTest.java
index 225fc11..ad0bcfc 100644
--- a/modules/hibernate/src/test/java/org/apache/ignite/cache/store/hibernate/CacheHibernateStoreFactorySelfTest.java
+++ b/modules/hibernate/src/test/java/org/apache/ignite/cache/store/hibernate/CacheHibernateStoreFactorySelfTest.java
@@ -21,11 +21,24 @@ import org.apache.ignite.*;
 import org.apache.ignite.configuration.*;
 import org.apache.ignite.testframework.*;
 import org.apache.ignite.testframework.junits.common.*;
+import org.hibernate.*;
+import org.hibernate.engine.spi.*;
+import org.hibernate.metadata.*;
+import org.hibernate.stat.*;
+
+import javax.naming.*;
+import java.io.*;
+import java.sql.*;
+import java.util.*;
+import java.util.concurrent.*;
 
 /**
  * Test for Cache jdbc blob store factory.
  */
 public class CacheHibernateStoreFactorySelfTest extends GridCommonAbstractTest {
+    /** Cache name. */
+    private static final String CACHE_NAME = "test";
+
     /**
      * @throws Exception If failed.
      */
@@ -38,6 +51,36 @@ public class CacheHibernateStoreFactorySelfTest extends GridCommonAbstractTest {
     }
 
     /**
+     * @throws Exception If failed.
+     */
+    public void testXmlConfiguration() throws Exception {
+        try (Ignite ignite = Ignition.start("modules/hibernate/src/test/config/factory-cache.xml")) {
+            try(Ignite ignite1 = Ignition.start("modules/hibernate/src/test/config/factory-cache1.xml")) {
+                checkStore(ignite.<Integer, String>cache(CACHE_NAME), DummySessionFactoryExt.class);
+
+                checkStore(ignite1.<Integer, String>cache(CACHE_NAME), DummySessionFactory.class);
+            }
+        }
+    }
+
+
+    /**
+     * @throws Exception If failed.
+     */
+    public void testIncorrectBeanConfiguration() throws Exception {
+        GridTestUtils.assertThrows(log, new Callable<Object>() {
+            @Override public Object call() throws Exception {
+                try(Ignite ignite =
+                    Ignition.start("modules/hibernate/src/test/config/factory-incorrect-store-cache.xml")) {
+                    ignite.cache(CACHE_NAME).getConfiguration(CacheConfiguration.class).
+                            getCacheStoreFactory().create();
+                }
+                return null;
+            }
+        }, IgniteException.class, "Failed to load bean in application context");
+    }
+
+    /**
      * @return Cache configuration with store.
      */
     private CacheConfiguration<Integer, String> cacheConfiguration() {
@@ -54,6 +97,19 @@ public class CacheHibernateStoreFactorySelfTest extends GridCommonAbstractTest {
 
     /**
      * @param cache Ignite cache.
+     * @param dataSrcClass Data source class.
+     * @throws Exception If store parameters is not the same as in configuration xml.
+     */
+    private void checkStore(IgniteCache<Integer, String> cache, Class<?> dataSrcClass) throws Exception {
+        CacheHibernateBlobStore store = (CacheHibernateBlobStore)cache
+            .getConfiguration(CacheConfiguration.class).getCacheStoreFactory().create();
+
+        assertEquals(dataSrcClass,
+            GridTestUtils.getFieldValue(store, CacheHibernateBlobStore.class, "sesFactory").getClass());
+    }
+
+    /**
+     * @param cache Ignite cache.
      * @throws Exception If store parameters is not the same as in configuration xml.
      */
     private void checkStore(IgniteCache<Integer, String> cache) throws Exception {
@@ -63,4 +119,155 @@ public class CacheHibernateStoreFactorySelfTest extends GridCommonAbstractTest {
         assertEquals("/org/apache/ignite/cache/store/hibernate/hibernate.cfg.xml",
             GridTestUtils.getFieldValue(store, CacheHibernateBlobStore.class, "hibernateCfgPath"));
     }
+
+    /**
+     *
+     */
+    public static class DummySessionFactoryExt extends DummySessionFactory {
+        /** */
+        public DummySessionFactoryExt() {
+            // No-op.
+        }
+    }
+
+    /**
+     *
+     */
+    public static class DummySessionFactory implements SessionFactory {
+        /** {@inheritDoc} */
+        @Override public SessionFactoryOptions getSessionFactoryOptions() {
+            return null;
+        }
+
+        /** {@inheritDoc} */
+        @Override public SessionBuilder withOptions() {
+            return null;
+        }
+
+        /** {@inheritDoc} */
+        @Override public Session openSession() throws HibernateException {
+            return null;
+        }
+
+        /** {@inheritDoc} */
+        @Override public Session getCurrentSession() throws HibernateException {
+            return null;
+        }
+
+        /** {@inheritDoc} */
+        @Override public StatelessSessionBuilder withStatelessOptions() {
+            return null;
+        }
+
+        /** {@inheritDoc} */
+        @Override public StatelessSession openStatelessSession() {
+            return null;
+        }
+
+        /** {@inheritDoc} */
+        @Override public StatelessSession openStatelessSession(Connection connection) {
+            return null;
+        }
+
+        /** {@inheritDoc} */
+        @Override public ClassMetadata getClassMetadata(Class entityClass) {
+            return null;
+        }
+
+        /** {@inheritDoc} */
+        @Override public ClassMetadata getClassMetadata(String entityName) {
+            return null;
+        }
+
+        /** {@inheritDoc} */
+        @Override public CollectionMetadata getCollectionMetadata(String roleName) {
+            return null;
+        }
+
+        /** {@inheritDoc} */
+        @Override public Map<String, ClassMetadata> getAllClassMetadata() {
+            return null;
+        }
+
+        /** {@inheritDoc} */
+        @Override public Map getAllCollectionMetadata() {
+            return null;
+        }
+
+        /** {@inheritDoc} */
+        @Override public Statistics getStatistics() {
+            return null;
+        }
+
+        /** {@inheritDoc} */
+        @Override public void close() throws HibernateException {
+        }
+
+        /** {@inheritDoc} */
+        @Override public boolean isClosed() {
+            return false;
+        }
+
+        /** {@inheritDoc} */
+        @Override public Cache getCache() {
+            return null;
+        }
+
+        /** {@inheritDoc} */
+        @Override public void evict(Class persistentClass) throws HibernateException {
+        }
+
+        /** {@inheritDoc} */
+        @Override public void evict(Class persistentClass, Serializable id) throws HibernateException {
+        }
+
+        /** {@inheritDoc} */
+        @Override public void evictEntity(String entityName) throws HibernateException {
+        }
+
+        /** {@inheritDoc} */
+        @Override public void evictEntity(String entityName, Serializable id) throws HibernateException {
+        }
+
+        /** {@inheritDoc} */
+        @Override public void evictCollection(String roleName) throws HibernateException {
+        }
+
+        /** {@inheritDoc} */
+        @Override public void evictCollection(String roleName, Serializable id) throws HibernateException {
+        }
+
+        /** {@inheritDoc} */
+        @Override public void evictQueries(String cacheRegion) throws HibernateException {
+        }
+
+        /** {@inheritDoc} */
+        @Override public void evictQueries() throws HibernateException {
+        }
+
+        /** {@inheritDoc} */
+        @Override public Set getDefinedFilterNames() {
+            return null;
+        }
+
+        /** {@inheritDoc} */
+        @Override public FilterDefinition getFilterDefinition(String filterName) throws HibernateException {
+            return null;
+        }
+
+        /** {@inheritDoc} */
+        @Override public boolean containsFetchProfileDefinition(String name) {
+            return false;
+        }
+
+        /** {@inheritDoc} */
+        @Override public TypeHelper getTypeHelper() {
+            return null;
+        }
+
+        /** {@inheritDoc} */
+        @Override public Reference getReference() throws NamingException {
+            return null;
+        }
+    }
 }