Return-Path: X-Original-To: archive-asf-public-internal@cust-asf2.ponee.io Delivered-To: archive-asf-public-internal@cust-asf2.ponee.io Received: from cust-asf.ponee.io (cust-asf.ponee.io [163.172.22.183]) by cust-asf2.ponee.io (Postfix) with ESMTP id B8D95200C87 for ; Wed, 26 Apr 2017 19:38:14 +0200 (CEST) Received: by cust-asf.ponee.io (Postfix) id B7743160BC0; Wed, 26 Apr 2017 17:38:14 +0000 (UTC) Delivered-To: archive-asf-public@cust-asf.ponee.io Received: from mail.apache.org (hermes.apache.org [140.211.11.3]) by cust-asf.ponee.io (Postfix) with SMTP id 9A2AB160BC1 for ; Wed, 26 Apr 2017 19:38:11 +0200 (CEST) Received: (qmail 38944 invoked by uid 500); 26 Apr 2017 17:38:10 -0000 Mailing-List: contact commits-help@ignite.apache.org; run by ezmlm Precedence: bulk List-Help: List-Unsubscribe: List-Post: List-Id: Reply-To: dev@ignite.apache.org Delivered-To: mailing list commits@ignite.apache.org Received: (qmail 38589 invoked by uid 99); 26 Apr 2017 17:38:10 -0000 Received: from git1-us-west.apache.org (HELO git1-us-west.apache.org) (140.211.11.23) by apache.org (qpsmtpd/0.29) with ESMTP; Wed, 26 Apr 2017 17:38:10 +0000 Received: by git1-us-west.apache.org (ASF Mail Server at git1-us-west.apache.org, from userid 33) id 78623E2F58; Wed, 26 Apr 2017 17:38:10 +0000 (UTC) Content-Type: text/plain; charset="us-ascii" MIME-Version: 1.0 Content-Transfer-Encoding: 7bit From: sboikov@apache.org To: commits@ignite.apache.org Date: Wed, 26 Apr 2017 17:38:21 -0000 Message-Id: In-Reply-To: References: X-Mailer: ASF-Git Admin Mailer Subject: [12/51] [abbrv] ignite git commit: ignite-1794 Refactored hibernate modules, switched to hibernate 5.1 archived-at: Wed, 26 Apr 2017 17:38:14 -0000 http://git-wip-us.apache.org/repos/asf/ignite/blob/ee1b19d3/modules/hibernate-5.1/src/main/java/org/apache/ignite/cache/store/hibernate/CacheHibernateBlobStore.java ---------------------------------------------------------------------- diff --git a/modules/hibernate-5.1/src/main/java/org/apache/ignite/cache/store/hibernate/CacheHibernateBlobStore.java b/modules/hibernate-5.1/src/main/java/org/apache/ignite/cache/store/hibernate/CacheHibernateBlobStore.java new file mode 100644 index 0000000..6e546b0 --- /dev/null +++ b/modules/hibernate-5.1/src/main/java/org/apache/ignite/cache/store/hibernate/CacheHibernateBlobStore.java @@ -0,0 +1,543 @@ +/* + * 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.ignite.cache.store.hibernate; + +import java.io.File; +import java.io.IOException; +import java.io.InputStream; +import java.net.MalformedURLException; +import java.net.URL; +import java.util.Map; +import java.util.Properties; +import java.util.concurrent.CountDownLatch; +import java.util.concurrent.atomic.AtomicBoolean; +import javax.cache.integration.CacheLoaderException; +import javax.cache.integration.CacheWriterException; +import org.apache.ignite.Ignite; +import org.apache.ignite.IgniteCheckedException; +import org.apache.ignite.IgniteException; +import org.apache.ignite.IgniteLogger; +import org.apache.ignite.cache.store.CacheStore; +import org.apache.ignite.cache.store.CacheStoreAdapter; +import org.apache.ignite.cache.store.CacheStoreSession; +import org.apache.ignite.configuration.CacheConfiguration; +import org.apache.ignite.internal.IgniteInterruptedCheckedException; +import org.apache.ignite.internal.util.tostring.GridToStringExclude; +import org.apache.ignite.internal.util.typedef.F; +import org.apache.ignite.internal.util.typedef.internal.S; +import org.apache.ignite.internal.util.typedef.internal.U; +import org.apache.ignite.marshaller.Marshaller; +import org.apache.ignite.marshaller.jdk.JdkMarshaller; +import org.apache.ignite.resources.CacheStoreSessionResource; +import org.apache.ignite.resources.IgniteInstanceResource; +import org.apache.ignite.resources.LoggerResource; +import org.apache.ignite.transactions.Transaction; +import org.hibernate.HibernateException; +import org.hibernate.Session; +import org.hibernate.SessionFactory; +import org.hibernate.SharedSessionContract; +import org.hibernate.cfg.Configuration; +import org.hibernate.resource.transaction.spi.TransactionStatus; +import org.jetbrains.annotations.Nullable; + +/** + * {@link CacheStore} implementation backed by Hibernate. This implementation + * stores objects in underlying database in {@code BLOB} format. + *

Configuration

+ * Either {@link #setSessionFactory(SessionFactory)} or + * {@link #setHibernateConfigurationPath(String)} or + * {@link #setHibernateProperties(Properties)} should be set. + *

+ * If session factory is provided it should contain + * {@link CacheHibernateBlobStoreEntry} persistent class (via provided + * mapping file {@code GridCacheHibernateStoreEntry.hbm.xml} or by + * adding {@link CacheHibernateBlobStoreEntry} to annotated classes + * of session factory. + *

+ * Path to hibernate configuration may be either an URL or a file path or + * a classpath resource. This configuration file should include provided + * mapping {@code GridCacheHibernateStoreEntry.hbm.xml} or include annotated + * class {@link CacheHibernateBlobStoreEntry}. + *

+ * If hibernate properties are provided, mapping + * {@code GridCacheHibernateStoreEntry.hbm.xml} is included automatically. + *

+ * Use {@link CacheHibernateBlobStoreFactory} factory to pass {@link CacheHibernateBlobStore} to {@link CacheConfiguration}. + */ +public class CacheHibernateBlobStore extends CacheStoreAdapter { + /** + * Default connection URL + * (value is jdbc:h2:mem:hibernateCacheStore;DB_CLOSE_DELAY=-1;DEFAULT_LOCK_TIMEOUT=5000). + */ + public static final String DFLT_CONN_URL = "jdbc:h2:mem:hibernateCacheStore;DB_CLOSE_DELAY=-1;" + + "DEFAULT_LOCK_TIMEOUT=5000"; + + /** Default show SQL property value (value is true). */ + public static final String DFLT_SHOW_SQL = "true"; + + /** Default hibernate.hbm2ddl.auto property value (value is true). */ + public static final String DFLT_HBM2DDL_AUTO = "update"; + + /** Session attribute name. */ + private static final String ATTR_SES = "HIBERNATE_STORE_SESSION"; + + /** Name of Hibarname mapping resource. */ + private static final String MAPPING_RESOURCE = + "org/apache/ignite/cache/store/hibernate/CacheHibernateBlobStoreEntry.hbm.xml"; + + /** Marshaller. */ + private static final Marshaller marsh = new JdkMarshaller(); + + /** Init guard. */ + @GridToStringExclude + private final AtomicBoolean initGuard = new AtomicBoolean(); + + /** Init latch. */ + @GridToStringExclude + private final CountDownLatch initLatch = new CountDownLatch(1); + + /** Hibernate properties. */ + @GridToStringExclude + private Properties hibernateProps; + + /** Session factory. */ + @GridToStringExclude + private SessionFactory sesFactory; + + /** Path to hibernate configuration file. */ + private String hibernateCfgPath; + + /** Log. */ + @LoggerResource + private IgniteLogger log; + + /** Auto-injected store session. */ + @CacheStoreSessionResource + private CacheStoreSession ses; + + /** Ignite instance. */ + @IgniteInstanceResource + private Ignite ignite; + + /** {@inheritDoc} */ + @SuppressWarnings({"unchecked", "RedundantTypeArguments"}) + @Override public V load(K key) { + init(); + + Transaction tx = transaction(); + + if (log.isDebugEnabled()) + log.debug("Store load [key=" + key + ", tx=" + tx + ']'); + + Session ses = session(tx); + + try { + CacheHibernateBlobStoreEntry entry = (CacheHibernateBlobStoreEntry) + ses.get(CacheHibernateBlobStoreEntry.class, toBytes(key)); + + if (entry == null) + return null; + + return fromBytes(entry.getValue()); + } + catch (IgniteCheckedException | HibernateException e) { + rollback(ses, tx); + + throw new CacheLoaderException("Failed to load value from cache store with key: " + key, e); + } + finally { + end(ses, tx); + } + } + + /** {@inheritDoc} */ + @Override public void write(javax.cache.Cache.Entry entry) { + init(); + + Transaction tx = transaction(); + + K key = entry.getKey(); + V val = entry.getValue(); + + if (log.isDebugEnabled()) + log.debug("Store put [key=" + key + ", val=" + val + ", tx=" + tx + ']'); + + if (val == null) { + delete(key); + + return; + } + + Session ses = session(tx); + + try { + CacheHibernateBlobStoreEntry entry0 = new CacheHibernateBlobStoreEntry(toBytes(key), toBytes(val)); + + ses.saveOrUpdate(entry0); + } + catch (IgniteCheckedException | HibernateException e) { + rollback(ses, tx); + + throw new CacheWriterException("Failed to put value to cache store [key=" + key + ", val" + val + "]", e); + } + finally { + end(ses, tx); + } + } + + /** {@inheritDoc} */ + @SuppressWarnings({"JpaQueryApiInspection", "JpaQlInspection"}) + @Override public void delete(Object key) { + init(); + + Transaction tx = transaction(); + + if (log.isDebugEnabled()) + log.debug("Store remove [key=" + key + ", tx=" + tx + ']'); + + Session ses = session(tx); + + try { + Object obj = ses.get(CacheHibernateBlobStoreEntry.class, toBytes(key)); + + if (obj != null) + ses.delete(obj); + } + catch (IgniteCheckedException | HibernateException e) { + rollback(ses, tx); + + throw new CacheWriterException("Failed to remove value from cache store with key: " + key, e); + } + finally { + end(ses, tx); + } + } + + /** + * Rolls back hibernate session. + * + * @param ses Hibernate session. + * @param tx Cache ongoing transaction. + */ + private void rollback(SharedSessionContract ses, Transaction tx) { + // Rollback only if there is no cache transaction, + // otherwise sessionEnd() will do all required work. + if (tx == null) { + org.hibernate.Transaction hTx = ses.getTransaction(); + + if (hTx != null && hTx.getStatus().canRollback()) + hTx.rollback(); + } + } + + /** + * Ends hibernate session. + * + * @param ses Hibernate session. + * @param tx Cache ongoing transaction. + */ + private void end(Session ses, Transaction tx) { + // Commit only if there is no cache transaction, + // otherwise sessionEnd() will do all required work. + if (tx == null) { + org.hibernate.Transaction hTx = ses.getTransaction(); + + if (hTx != null && hTx.getStatus() == TransactionStatus.ACTIVE) + hTx.commit(); + + ses.close(); + } + } + + /** {@inheritDoc} */ + @Override public void sessionEnd(boolean commit) { + init(); + + Transaction tx = transaction(); + + Map props = session().properties(); + + Session ses = props.remove(ATTR_SES); + + if (ses != null) { + org.hibernate.Transaction hTx = ses.getTransaction(); + + if (hTx != null) { + try { + if (commit) { + ses.flush(); + + hTx.commit(); + } + else + hTx.rollback(); + + if (log.isDebugEnabled()) + log.debug("Transaction ended [xid=" + tx.xid() + ", commit=" + commit + ']'); + } + catch (HibernateException e) { + throw new CacheWriterException("Failed to end transaction [xid=" + tx.xid() + + ", commit=" + commit + ']', e); + } + finally { + ses.close(); + } + } + } + } + + /** + * Gets Hibernate session. + * + * @param tx Cache transaction. + * @return Session. + */ + Session session(@Nullable Transaction tx) { + Session ses; + + if (tx != null) { + Map props = session().properties(); + + ses = props.get(ATTR_SES); + + if (ses == null) { + ses = sesFactory.openSession(); + + ses.beginTransaction(); + + // Store session in transaction metadata, so it can be accessed + // for other operations on the same transaction. + props.put(ATTR_SES, ses); + + if (log.isDebugEnabled()) + log.debug("Hibernate session open [ses=" + ses + ", tx=" + tx.xid() + "]"); + } + } + else { + ses = sesFactory.openSession(); + + ses.beginTransaction(); + } + + return ses; + } + + /** + * Sets session factory. + * + * @param sesFactory Session factory. + */ + public void setSessionFactory(SessionFactory sesFactory) { + this.sesFactory = sesFactory; + } + + /** + * Sets hibernate configuration path. + *

+ * This may be either URL or file path or classpath resource. + * + * @param hibernateCfgPath URL or file path or classpath resource + * pointing to hibernate configuration XML file. + */ + public void setHibernateConfigurationPath(String hibernateCfgPath) { + this.hibernateCfgPath = hibernateCfgPath; + } + + /** + * Sets Hibernate properties. + * + * @param hibernateProps Hibernate properties. + */ + public void setHibernateProperties(Properties hibernateProps) { + this.hibernateProps = hibernateProps; + } + + /** + * Initializes store. + * + * @throws IgniteException If failed to initialize. + */ + private void init() throws IgniteException { + if (initGuard.compareAndSet(false, true)) { + if (log.isDebugEnabled()) + log.debug("Initializing cache store."); + + try { + if (sesFactory != null) + // Session factory has been provided - nothing to do. + return; + + if (!F.isEmpty(hibernateCfgPath)) { + try { + URL url = new URL(hibernateCfgPath); + + sesFactory = new Configuration().configure(url).buildSessionFactory(); + + if (log.isDebugEnabled()) + log.debug("Configured session factory using URL: " + url); + + // Session factory has been successfully initialized. + return; + } + catch (MalformedURLException e) { + if (log.isDebugEnabled()) + log.debug("Caught malformed URL exception: " + e.getMessage()); + } + + // Provided path is not a valid URL. File? + File cfgFile = new File(hibernateCfgPath); + + if (cfgFile.exists()) { + sesFactory = new Configuration().configure(cfgFile).buildSessionFactory(); + + if (log.isDebugEnabled()) + log.debug("Configured session factory using file: " + hibernateCfgPath); + + // Session factory has been successfully initialized. + return; + } + + // Provided path is not a file. Classpath resource? + sesFactory = new Configuration().configure(hibernateCfgPath).buildSessionFactory(); + + if (log.isDebugEnabled()) + log.debug("Configured session factory using classpath resource: " + hibernateCfgPath); + } + else { + if (hibernateProps == null) { + U.warn(log, "No Hibernate configuration has been provided for store (will use default)."); + + hibernateProps = new Properties(); + + hibernateProps.setProperty("hibernate.connection.url", DFLT_CONN_URL); + hibernateProps.setProperty("hibernate.show_sql", DFLT_SHOW_SQL); + hibernateProps.setProperty("hibernate.hbm2ddl.auto", DFLT_HBM2DDL_AUTO); + } + + Configuration cfg = new Configuration(); + + cfg.setProperties(hibernateProps); + + assert resourceAvailable(MAPPING_RESOURCE) : MAPPING_RESOURCE; + + cfg.addResource(MAPPING_RESOURCE); + + sesFactory = cfg.buildSessionFactory(); + + if (log.isDebugEnabled()) + log.debug("Configured session factory using properties: " + hibernateProps); + } + } + catch (HibernateException e) { + throw new IgniteException("Failed to initialize store.", e); + } + finally { + initLatch.countDown(); + } + } + else if (initLatch.getCount() > 0) { + try { + U.await(initLatch); + } + catch (IgniteInterruptedCheckedException e) { + throw new IgniteException(e); + } + } + + if (sesFactory == null) + throw new IgniteException("Cache store was not properly initialized."); + } + + /** + * Checks availability of a classpath resource. + * + * @param name Resource name. + * @return {@code true} if resource is available and ready for read, {@code false} otherwise. + */ + private boolean resourceAvailable(String name) { + InputStream cfgStream = Thread.currentThread().getContextClassLoader().getResourceAsStream(name); + + if (cfgStream == null) { + log.error("Classpath resource not found: " + name); + + return false; + } + + try { + // Read a single byte to force actual content access by JVM. + cfgStream.read(); + + return true; + } + catch (IOException e) { + log.error("Failed to read classpath resource: " + name, e); + + return false; + } + finally { + U.close(cfgStream, log); + } + } + + /** {@inheritDoc} */ + @Override public String toString() { + return S.toString(CacheHibernateBlobStore.class, this); + } + + /** + * Serialize object to byte array using marshaller. + * + * @param obj Object to convert to byte array. + * @return Byte array. + * @throws IgniteCheckedException If failed to convert. + */ + protected byte[] toBytes(Object obj) throws IgniteCheckedException { + return U.marshal(marsh, obj); + } + + /** + * Deserialize object from byte array using marshaller. + * + * @param bytes Bytes to deserialize. + * @param Result object type. + * @return Deserialized object. + * @throws IgniteCheckedException If failed. + */ + protected X fromBytes(byte[] bytes) throws IgniteCheckedException { + if (bytes == null || bytes.length == 0) + return null; + + return U.unmarshal(marsh, bytes, getClass().getClassLoader()); + } + + /** + * @return Current transaction. + */ + @Nullable private Transaction transaction() { + CacheStoreSession ses = session(); + + return ses != null ? ses.transaction() : null; + } + + /** + * @return Store session. + */ + private CacheStoreSession session() { + return ses; + } +} \ No newline at end of file http://git-wip-us.apache.org/repos/asf/ignite/blob/ee1b19d3/modules/hibernate-5.1/src/main/java/org/apache/ignite/cache/store/hibernate/CacheHibernateBlobStoreEntry.hbm.xml ---------------------------------------------------------------------- diff --git a/modules/hibernate-5.1/src/main/java/org/apache/ignite/cache/store/hibernate/CacheHibernateBlobStoreEntry.hbm.xml b/modules/hibernate-5.1/src/main/java/org/apache/ignite/cache/store/hibernate/CacheHibernateBlobStoreEntry.hbm.xml new file mode 100644 index 0000000..5b0be43 --- /dev/null +++ b/modules/hibernate-5.1/src/main/java/org/apache/ignite/cache/store/hibernate/CacheHibernateBlobStoreEntry.hbm.xml @@ -0,0 +1,31 @@ + + + + + + + + + + + + + + http://git-wip-us.apache.org/repos/asf/ignite/blob/ee1b19d3/modules/hibernate-5.1/src/main/java/org/apache/ignite/cache/store/hibernate/CacheHibernateBlobStoreEntry.java ---------------------------------------------------------------------- diff --git a/modules/hibernate-5.1/src/main/java/org/apache/ignite/cache/store/hibernate/CacheHibernateBlobStoreEntry.java b/modules/hibernate-5.1/src/main/java/org/apache/ignite/cache/store/hibernate/CacheHibernateBlobStoreEntry.java new file mode 100644 index 0000000..d40c5ef --- /dev/null +++ b/modules/hibernate-5.1/src/main/java/org/apache/ignite/cache/store/hibernate/CacheHibernateBlobStoreEntry.java @@ -0,0 +1,89 @@ +/* + * 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.ignite.cache.store.hibernate; + +import javax.persistence.Column; +import javax.persistence.Entity; +import javax.persistence.Id; +import javax.persistence.Table; + +/** + * Entry that is used by {@link CacheHibernateBlobStore} implementation. + *

+ * Note that this is a reference implementation for tests only. + * When running on production systems use concrete key-value types to + * get better performance. + */ +@Entity +@Table(name = "ENTRIES") +public class CacheHibernateBlobStoreEntry { + /** Key (use concrete key type in production). */ + @Id + @Column(length = 65535) + private byte[] key; + + /** Value (use concrete value type in production). */ + @Column(length = 65535) + private byte[] val; + + /** + * Constructor. + */ + CacheHibernateBlobStoreEntry() { + // No-op. + } + + /** + * Constructor. + * + * @param key Key. + * @param val Value. + */ + CacheHibernateBlobStoreEntry(byte[] key, byte[] val) { + this.key = key; + this.val = val; + } + + /** + * @return Key. + */ + public byte[] getKey() { + return key; + } + + /** + * @param key Key. + */ + public void setKey(byte[] key) { + this.key = key; + } + + /** + * @return Value. + */ + public byte[] getValue() { + return val; + } + + /** + * @param val Value. + */ + public void setValue(byte[] val) { + this.val = val; + } +} \ No newline at end of file http://git-wip-us.apache.org/repos/asf/ignite/blob/ee1b19d3/modules/hibernate-5.1/src/main/java/org/apache/ignite/cache/store/hibernate/CacheHibernateBlobStoreFactory.java ---------------------------------------------------------------------- diff --git a/modules/hibernate-5.1/src/main/java/org/apache/ignite/cache/store/hibernate/CacheHibernateBlobStoreFactory.java b/modules/hibernate-5.1/src/main/java/org/apache/ignite/cache/store/hibernate/CacheHibernateBlobStoreFactory.java new file mode 100644 index 0000000..ea4df8a --- /dev/null +++ b/modules/hibernate-5.1/src/main/java/org/apache/ignite/cache/store/hibernate/CacheHibernateBlobStoreFactory.java @@ -0,0 +1,235 @@ +/* + * 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.ignite.cache.store.hibernate; + +import java.util.Properties; +import javax.cache.configuration.Factory; +import org.apache.ignite.IgniteCheckedException; +import org.apache.ignite.IgniteException; +import org.apache.ignite.configuration.CacheConfiguration; +import org.apache.ignite.internal.IgniteComponentType; +import org.apache.ignite.internal.util.spring.IgniteSpringHelper; +import org.apache.ignite.internal.util.tostring.GridToStringExclude; +import org.apache.ignite.internal.util.typedef.internal.S; +import org.apache.ignite.resources.SpringApplicationContextResource; +import org.hibernate.SessionFactory; + +/** + * {@link Factory} implementation for {@link CacheHibernateBlobStore}. + * + * Use this factory to pass {@link CacheHibernateBlobStore} to {@link CacheConfiguration}. + * + *

Java Example

+ * In this example existing session factory is provided. + *
+ *     ...
+ *     CacheHibernateBlobStoreFactory<String, String> factory = new CacheHibernateBlobStoreFactory<String, String>();
+ *
+ *     factory.setSessionFactory(sesFactory);
+ *     ...
+ * 
+ * + *

Spring Example (using Spring ORM)

+ *
+ *   ...
+ *   <bean id="cache.hibernate.store.factory"
+ *       class="org.apache.ignite.cache.store.hibernate.CacheHibernateBlobStoreFactory">
+ *       <property name="sessionFactory">
+ *           <bean class="org.springframework.orm.hibernate3.LocalSessionFactoryBean">
+ *               <property name="hibernateProperties">
+ *                   <value>
+ *                       connection.url=jdbc:h2:mem:
+ *                       show_sql=true
+ *                       hbm2ddl.auto=true
+ *                       hibernate.dialect=org.hibernate.dialect.H2Dialect
+ *                   </value>
+ *               </property>
+ *               <property name="mappingResources">
+ *                   <list>
+ *                       <value>
+ *                           org/apache/ignite/cache/store/hibernate/CacheHibernateBlobStoreEntry.hbm.xml
+ *                       </value>
+ *                   </list>
+ *               </property>
+ *           </bean>
+ *       </property>
+ *   </bean>
+ *   ...
+ * 
+ * + *

Spring Example (using Spring ORM and persistent annotations)

+ *
+ *     ...
+ *     <bean id="cache.hibernate.store.factory1"
+ *         class="org.apache.ignite.cache.store.hibernate.CacheHibernateBlobStoreFactory">
+ *         <property name="sessionFactory">
+ *             <bean class="org.springframework.orm.hibernate3.annotation.AnnotationSessionFactoryBean">
+ *                 <property name="hibernateProperties">
+ *                     <value>
+ *                         connection.url=jdbc:h2:mem:
+ *                         show_sql=true
+ *                         hbm2ddl.auto=true
+ *                         hibernate.dialect=org.hibernate.dialect.H2Dialect
+ *                     </value>
+ *                 </property>
+ *                 <property name="annotatedClasses">
+ *                     <list>
+ *                         <value>
+ *                             org.apache.ignite.cache.store.hibernate.CacheHibernateBlobStoreEntry
+ *                         </value>
+ *                     </list>
+ *                 </property>
+ *             </bean>
+ *         </property>
+ *     </bean>
+ *     ...
+ * 
+ * + *

Spring Example

+ *
+ *     ...
+ *     <bean id="cache.hibernate.store.factory2"
+ *         class="org.apache.ignite.cache.store.hibernate.CacheHibernateBlobStoreFactory">
+ *         <property name="hibernateProperties">
+ *             <props>
+ *                 <prop key="connection.url">jdbc:h2:mem:</prop>
+ *                 <prop key="hbm2ddl.auto">update</prop>
+ *                 <prop key="show_sql">true</prop>
+ *             </props>
+ *         </property>
+ *     </bean>
+ *     ...
+ * 
+ *

+ * + *
+ * For information about Spring framework visit www.springframework.org + */ +public class CacheHibernateBlobStoreFactory implements Factory> { + /** */ + private static final long serialVersionUID = 0L; + + /** Session factory. */ + @GridToStringExclude + private transient SessionFactory sesFactory; + + /** Session factory bean name. */ + private String sesFactoryBean; + + /** Path to hibernate configuration file. */ + private String hibernateCfgPath; + + /** Hibernate properties. */ + @GridToStringExclude + private Properties hibernateProps; + + /** Application context. */ + @SpringApplicationContextResource + private Object appContext; + + /** {@inheritDoc} */ + @Override public CacheHibernateBlobStore create() { + CacheHibernateBlobStore store = new CacheHibernateBlobStore<>(); + + 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; + } + + /** + * Sets session factory. + * + * @param sesFactory Session factory. + * @return {@code This} for chaining. + * @see CacheHibernateBlobStore#setSessionFactory(SessionFactory) + */ + public CacheHibernateBlobStoreFactory 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 setSessionFactoryBean(String sesFactory) { + this.sesFactoryBean = sesFactory; + + return this; + } + + /** + * Sets hibernate configuration path. + *

+ * This may be either URL or file path or classpath resource. + * + * @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 CacheHibernateBlobStoreFactory 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 CacheHibernateBlobStoreFactory setHibernateProperties(Properties hibernateProps) { + this.hibernateProps = hibernateProps; + + return this; + } + + /** {@inheritDoc} */ + @Override public String toString() { + return S.toString(CacheHibernateBlobStoreFactory.class, this); + } +} \ No newline at end of file http://git-wip-us.apache.org/repos/asf/ignite/blob/ee1b19d3/modules/hibernate-5.1/src/main/java/org/apache/ignite/cache/store/hibernate/CacheHibernateStoreSessionListener.java ---------------------------------------------------------------------- diff --git a/modules/hibernate-5.1/src/main/java/org/apache/ignite/cache/store/hibernate/CacheHibernateStoreSessionListener.java b/modules/hibernate-5.1/src/main/java/org/apache/ignite/cache/store/hibernate/CacheHibernateStoreSessionListener.java new file mode 100644 index 0000000..8db9568 --- /dev/null +++ b/modules/hibernate-5.1/src/main/java/org/apache/ignite/cache/store/hibernate/CacheHibernateStoreSessionListener.java @@ -0,0 +1,224 @@ +/* + * 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.ignite.cache.store.hibernate; + +import java.io.File; +import java.net.MalformedURLException; +import java.net.URL; +import javax.cache.integration.CacheWriterException; +import org.apache.ignite.IgniteException; +import org.apache.ignite.IgniteLogger; +import org.apache.ignite.cache.store.CacheStore; +import org.apache.ignite.cache.store.CacheStoreSession; +import org.apache.ignite.cache.store.CacheStoreSessionListener; +import org.apache.ignite.internal.util.typedef.F; +import org.apache.ignite.internal.util.typedef.internal.U; +import org.apache.ignite.lifecycle.LifecycleAware; +import org.apache.ignite.resources.LoggerResource; +import org.hibernate.HibernateException; +import org.hibernate.Session; +import org.hibernate.SessionFactory; +import org.hibernate.Transaction; +import org.hibernate.cfg.Configuration; +import org.hibernate.resource.transaction.spi.TransactionStatus; + +/** + * Hibernate-based cache store session listener. + *

+ * This listener creates a new Hibernate session for each store + * session. If there is an ongoing cache transaction, a corresponding + * Hibernate transaction is created as well. + *

+ * The Hibernate session is saved as a store session + * {@link CacheStoreSession#attachment() attachment}. + * The listener guarantees that the session will be + * available for any store operation. If there is an + * ongoing cache transaction, all operations within this + * transaction will share a DB transaction. + *

+ * As an example, here is how the {@link CacheStore#write(javax.cache.Cache.Entry)} + * method can be implemented if {@link CacheHibernateStoreSessionListener} + * is configured: + *

+ * private static class Store extends CacheStoreAdapter<Integer, Integer> {
+ *     @CacheStoreSessionResource
+ *     private CacheStoreSession ses;
+ *
+ *     @Override public void write(Cache.Entry<? extends Integer, ? extends Integer> entry) throws CacheWriterException {
+ *         // Get Hibernate session from the current store session.
+ *         Session hibSes = ses.attachment();
+ *
+ *         // Persist the value.
+ *         hibSes.persist(entry.getValue());
+ *     }
+ * }
+ * 
+ * Hibernate session will be automatically created by the listener + * at the start of the session and closed when it ends. + *

+ * {@link CacheHibernateStoreSessionListener} requires that either + * {@link #setSessionFactory(SessionFactory)} session factory} + * or {@link #setHibernateConfigurationPath(String) Hibernate configuration file} + * is provided. If non of them is set, exception is thrown. Is both are provided, + * session factory will be used. + */ +public class CacheHibernateStoreSessionListener implements CacheStoreSessionListener, LifecycleAware { + /** Hibernate session factory. */ + private SessionFactory sesFactory; + + /** Hibernate configuration file path. */ + private String hibernateCfgPath; + + /** Logger. */ + @LoggerResource + private IgniteLogger log; + + /** Whether to close session on stop. */ + private boolean closeSesOnStop; + + /** + * Sets Hibernate session factory. + *

+ * Either session factory or configuration file is required. + * If none is provided, exception will be thrown on startup. + * + * @param sesFactory Session factory. + */ + public void setSessionFactory(SessionFactory sesFactory) { + this.sesFactory = sesFactory; + } + + /** + * Gets Hibernate session factory. + * + * @return Session factory. + */ + public SessionFactory getSessionFactory() { + return sesFactory; + } + + /** + * Sets hibernate configuration path. + *

+ * Either session factory or configuration file is required. + * If none is provided, exception will be thrown on startup. + * + * @param hibernateCfgPath Hibernate configuration path. + */ + public void setHibernateConfigurationPath(String hibernateCfgPath) { + this.hibernateCfgPath = hibernateCfgPath; + } + + /** + * Gets hibernate configuration path. + * + * @return Hibernate configuration path. + */ + public String getHibernateConfigurationPath() { + return hibernateCfgPath; + } + + /** {@inheritDoc} */ + @SuppressWarnings("deprecation") + @Override public void start() throws IgniteException { + if (sesFactory == null && F.isEmpty(hibernateCfgPath)) + throw new IgniteException("Either session factory or Hibernate configuration file is required by " + + getClass().getSimpleName() + '.'); + + if (!F.isEmpty(hibernateCfgPath)) { + if (sesFactory == null) { + try { + URL url = new URL(hibernateCfgPath); + + sesFactory = new Configuration().configure(url).buildSessionFactory(); + } + catch (MalformedURLException ignored) { + // No-op. + } + + if (sesFactory == null) { + File cfgFile = new File(hibernateCfgPath); + + if (cfgFile.exists()) + sesFactory = new Configuration().configure(cfgFile).buildSessionFactory(); + } + + if (sesFactory == null) + sesFactory = new Configuration().configure(hibernateCfgPath).buildSessionFactory(); + + if (sesFactory == null) + throw new IgniteException("Failed to resolve Hibernate configuration file: " + hibernateCfgPath); + + closeSesOnStop = true; + } + else + U.warn(log, "Hibernate configuration file configured in " + getClass().getSimpleName() + + " will be ignored (session factory is already set)."); + } + } + + /** {@inheritDoc} */ + @Override public void stop() throws IgniteException { + if (closeSesOnStop && sesFactory != null && !sesFactory.isClosed()) + sesFactory.close(); + } + + /** {@inheritDoc} */ + @Override public void onSessionStart(CacheStoreSession ses) { + if (ses.attachment() == null) { + try { + Session hibSes = sesFactory.openSession(); + + ses.attach(hibSes); + + if (ses.isWithinTransaction()) + hibSes.beginTransaction(); + } + catch (HibernateException e) { + throw new CacheWriterException("Failed to start store session [tx=" + ses.transaction() + ']', e); + } + } + } + + /** {@inheritDoc} */ + @Override public void onSessionEnd(CacheStoreSession ses, boolean commit) { + Session hibSes = ses.attach(null); + + if (hibSes != null) { + try { + Transaction tx = hibSes.getTransaction(); + + if (commit) { + if (hibSes.isDirty()) + hibSes.flush(); + + if (tx.getStatus() == TransactionStatus.ACTIVE) + tx.commit(); + } + else if (tx.getStatus().canRollback()) + tx.rollback(); + } + catch (HibernateException e) { + throw new CacheWriterException("Failed to end store session [tx=" + ses.transaction() + ']', e); + } + finally { + hibSes.close(); + } + } + } +} \ No newline at end of file http://git-wip-us.apache.org/repos/asf/ignite/blob/ee1b19d3/modules/hibernate-5.1/src/main/java/org/apache/ignite/cache/store/hibernate/package-info.java ---------------------------------------------------------------------- diff --git a/modules/hibernate-5.1/src/main/java/org/apache/ignite/cache/store/hibernate/package-info.java b/modules/hibernate-5.1/src/main/java/org/apache/ignite/cache/store/hibernate/package-info.java new file mode 100644 index 0000000..891d99a --- /dev/null +++ b/modules/hibernate-5.1/src/main/java/org/apache/ignite/cache/store/hibernate/package-info.java @@ -0,0 +1,22 @@ +/* + * 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. + */ + +/** + * + * Contains reference Hibernate-based cache store implementation. + */ +package org.apache.ignite.cache.store.hibernate; \ No newline at end of file http://git-wip-us.apache.org/repos/asf/ignite/blob/ee1b19d3/modules/hibernate-5.1/src/test/config/factory-cache.xml ---------------------------------------------------------------------- diff --git a/modules/hibernate-5.1/src/test/config/factory-cache.xml b/modules/hibernate-5.1/src/test/config/factory-cache.xml new file mode 100644 index 0000000..a251846 --- /dev/null +++ b/modules/hibernate-5.1/src/test/config/factory-cache.xml @@ -0,0 +1,59 @@ + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + 127.0.0.1:47500..47509 + + + + + + + + http://git-wip-us.apache.org/repos/asf/ignite/blob/ee1b19d3/modules/hibernate-5.1/src/test/config/factory-cache1.xml ---------------------------------------------------------------------- diff --git a/modules/hibernate-5.1/src/test/config/factory-cache1.xml b/modules/hibernate-5.1/src/test/config/factory-cache1.xml new file mode 100644 index 0000000..7a53b1f --- /dev/null +++ b/modules/hibernate-5.1/src/test/config/factory-cache1.xml @@ -0,0 +1,61 @@ + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + 127.0.0.1:47500..47509 + + + + + + + + http://git-wip-us.apache.org/repos/asf/ignite/blob/ee1b19d3/modules/hibernate-5.1/src/test/config/factory-incorrect-store-cache.xml ---------------------------------------------------------------------- diff --git a/modules/hibernate-5.1/src/test/config/factory-incorrect-store-cache.xml b/modules/hibernate-5.1/src/test/config/factory-incorrect-store-cache.xml new file mode 100644 index 0000000..459930c --- /dev/null +++ b/modules/hibernate-5.1/src/test/config/factory-incorrect-store-cache.xml @@ -0,0 +1,56 @@ + + + + + + + + + + + + + + + + + + + + + + + + + + + + + 127.0.0.1:47500..47509 + + + + + + + + http://git-wip-us.apache.org/repos/asf/ignite/blob/ee1b19d3/modules/hibernate-5.1/src/test/java/org/apache/ignite/cache/hibernate/HibernateL2CacheConfigurationSelfTest.java ---------------------------------------------------------------------- diff --git a/modules/hibernate-5.1/src/test/java/org/apache/ignite/cache/hibernate/HibernateL2CacheConfigurationSelfTest.java b/modules/hibernate-5.1/src/test/java/org/apache/ignite/cache/hibernate/HibernateL2CacheConfigurationSelfTest.java new file mode 100644 index 0000000..0363c41 --- /dev/null +++ b/modules/hibernate-5.1/src/test/java/org/apache/ignite/cache/hibernate/HibernateL2CacheConfigurationSelfTest.java @@ -0,0 +1,407 @@ +/* + * 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.ignite.cache.hibernate; + +import java.util.HashSet; +import java.util.Iterator; +import java.util.List; +import java.util.Set; +import javax.cache.Cache; +import javax.persistence.Cacheable; +import javax.persistence.GeneratedValue; +import javax.persistence.Id; +import org.apache.ignite.IgniteCache; +import org.apache.ignite.configuration.CacheConfiguration; +import org.apache.ignite.configuration.IgniteConfiguration; +import org.apache.ignite.internal.IgniteKernal; +import org.apache.ignite.internal.processors.cache.IgniteCacheProxy; +import org.apache.ignite.spi.discovery.tcp.TcpDiscoverySpi; +import org.apache.ignite.spi.discovery.tcp.ipfinder.vm.TcpDiscoveryVmIpFinder; +import org.apache.ignite.testframework.junits.common.GridCommonAbstractTest; +import org.hibernate.Session; +import org.hibernate.SessionFactory; +import org.hibernate.Transaction; +import org.hibernate.annotations.CacheConcurrencyStrategy; +import org.hibernate.boot.registry.StandardServiceRegistryBuilder; +import org.hibernate.cache.spi.access.AccessType; +import org.hibernate.cfg.Configuration; + +import static org.apache.ignite.cache.CacheAtomicityMode.ATOMIC; +import static org.apache.ignite.cache.CacheMode.PARTITIONED; +import static org.apache.ignite.cache.hibernate.HibernateAccessStrategyFactory.DFLT_CACHE_NAME_PROPERTY; +import static org.apache.ignite.cache.hibernate.HibernateAccessStrategyFactory.REGION_CACHE_PROPERTY; +import static org.hibernate.cfg.AvailableSettings.CACHE_REGION_FACTORY; +import static org.hibernate.cfg.AvailableSettings.GENERATE_STATISTICS; +import static org.hibernate.cfg.AvailableSettings.HBM2DDL_AUTO; +import static org.hibernate.cfg.AvailableSettings.RELEASE_CONNECTIONS; +import static org.hibernate.cfg.AvailableSettings.USE_QUERY_CACHE; +import static org.hibernate.cfg.AvailableSettings.USE_SECOND_LEVEL_CACHE; + +/** + * Tests Hibernate L2 cache configuration. + */ +public class HibernateL2CacheConfigurationSelfTest extends GridCommonAbstractTest { + /** */ + public static final String ENTITY1_NAME = Entity1.class.getName(); + + /** */ + public static final String ENTITY2_NAME = Entity2.class.getName(); + + /** */ + public static final String ENTITY3_NAME = Entity3.class.getName(); + + /** */ + public static final String ENTITY4_NAME = Entity4.class.getName(); + + /** */ + public static final String TIMESTAMP_CACHE = "org.hibernate.cache.spi.UpdateTimestampsCache"; + + /** */ + public static final String QUERY_CACHE = "org.hibernate.cache.internal.StandardQueryCache"; + + /** */ + public static final String CONNECTION_URL = "jdbc:h2:mem:example;DB_CLOSE_DELAY=-1"; + + /** If {@code true} then sets default cache in configuration. */ + private boolean dfltCache; + + /** {@inheritDoc} */ + @Override protected void beforeTestsStarted() throws Exception { + startGrid(0); + } + + /** {@inheritDoc} */ + @Override protected void afterTestsStopped() throws Exception { + stopAllGrids(); + } + + /** {@inheritDoc} */ + @Override protected void afterTest() throws Exception { + for (IgniteCacheProxy cache : ((IgniteKernal)grid(0)).caches()) + cache.clear(); + } + + /** {@inheritDoc} */ + @Override protected IgniteConfiguration getConfiguration(String igniteInstanceName) throws Exception { + IgniteConfiguration cfg = super.getConfiguration(igniteInstanceName); + + TcpDiscoverySpi discoSpi = new TcpDiscoverySpi(); + + discoSpi.setIpFinder(new TcpDiscoveryVmIpFinder(true)); + + cfg.setDiscoverySpi(discoSpi); + + cfg.setCacheConfiguration(cacheConfiguration(ENTITY3_NAME), cacheConfiguration(ENTITY4_NAME), + cacheConfiguration("cache1"), cacheConfiguration("cache2"), cacheConfiguration("cache3"), + cacheConfiguration(TIMESTAMP_CACHE), cacheConfiguration(QUERY_CACHE)); + + return cfg; + } + + /** + * @param cacheName Cache name. + * @return Cache configuration. + */ + private CacheConfiguration cacheConfiguration(String cacheName) { + CacheConfiguration cfg = new CacheConfiguration(); + + cfg.setName(cacheName); + + cfg.setCacheMode(PARTITIONED); + + cfg.setAtomicityMode(ATOMIC); + + return cfg; + } + /** + * @param igniteInstanceName Ignite instance name. + * @return Hibernate configuration. + */ + protected Configuration hibernateConfiguration(String igniteInstanceName) { + Configuration cfg = new Configuration(); + + cfg.addAnnotatedClass(Entity1.class); + cfg.addAnnotatedClass(Entity2.class); + cfg.addAnnotatedClass(Entity3.class); + cfg.addAnnotatedClass(Entity4.class); + + cfg.setProperty(HibernateAccessStrategyFactory.DFLT_ACCESS_TYPE_PROPERTY, AccessType.NONSTRICT_READ_WRITE.name()); + + cfg.setProperty(HBM2DDL_AUTO, "create"); + + cfg.setProperty(GENERATE_STATISTICS, "true"); + + cfg.setProperty(USE_SECOND_LEVEL_CACHE, "true"); + + cfg.setProperty(USE_QUERY_CACHE, "true"); + + cfg.setProperty(CACHE_REGION_FACTORY, HibernateRegionFactory.class.getName()); + + cfg.setProperty(RELEASE_CONNECTIONS, "on_close"); + + cfg.setProperty(HibernateAccessStrategyFactory.IGNITE_INSTANCE_NAME_PROPERTY, igniteInstanceName); + + cfg.setProperty(REGION_CACHE_PROPERTY + ENTITY1_NAME, "cache1"); + cfg.setProperty(REGION_CACHE_PROPERTY + ENTITY2_NAME, "cache2"); + cfg.setProperty(REGION_CACHE_PROPERTY + TIMESTAMP_CACHE, TIMESTAMP_CACHE); + cfg.setProperty(REGION_CACHE_PROPERTY + QUERY_CACHE, QUERY_CACHE); + + if (dfltCache) + cfg.setProperty(DFLT_CACHE_NAME_PROPERTY, "cache3"); + + return cfg; + } + + /** + * Tests property {@link HibernateAccessStrategyFactory#REGION_CACHE_PROPERTY}. + */ + public void testPerRegionCacheProperty() { + testCacheUsage(1, 1, 0, 1, 1); + } + + /** + * Tests property {@link HibernateAccessStrategyFactory#DFLT_CACHE_NAME_PROPERTY}. + */ + public void testDefaultCache() { + dfltCache = true; + + testCacheUsage(1, 1, 2, 0, 0); + } + + /** + * @param expCache1 Expected size of cache with name 'cache1'. + * @param expCache2 Expected size of cache with name 'cache2'. + * @param expCache3 Expected size of cache with name 'cache3'. + * @param expCacheE3 Expected size of cache with name {@link #ENTITY3_NAME}. + * @param expCacheE4 Expected size of cache with name {@link #ENTITY4_NAME}. + */ + @SuppressWarnings("unchecked") + private void testCacheUsage(int expCache1, int expCache2, int expCache3, int expCacheE3, int expCacheE4) { + SessionFactory sesFactory = startHibernate(getTestIgniteInstanceName(0)); + + try { + Session ses = sesFactory.openSession(); + + try { + Transaction tx = ses.beginTransaction(); + + ses.save(new Entity1()); + ses.save(new Entity2()); + ses.save(new Entity3()); + ses.save(new Entity4()); + + tx.commit(); + } + finally { + ses.close(); + } + + ses = sesFactory.openSession(); + + try { + List list1 = ses.createCriteria(ENTITY1_NAME).list(); + + assertEquals(1, list1.size()); + + for (Entity1 e : list1) { + ses.load(ENTITY1_NAME, e.getId()); + assertNotNull(e.getId()); + } + + List list2 = ses.createCriteria(ENTITY2_NAME).list(); + + assertEquals(1, list2.size()); + + for (Entity2 e : list2) + assertNotNull(e.getId()); + + List list3 = ses.createCriteria(ENTITY3_NAME).list(); + + assertEquals(1, list3.size()); + + for (Entity3 e : list3) + assertNotNull(e.getId()); + + List list4 = ses.createCriteria(ENTITY4_NAME).list(); + + assertEquals(1, list4.size()); + + for (Entity4 e : list4) + assertNotNull(e.getId()); + } + finally { + ses.close(); + } + + IgniteCache cache1 = grid(0).cache("cache1"); + IgniteCache cache2 = grid(0).cache("cache2"); + IgniteCache cache3 = grid(0).cache("cache3"); + IgniteCache cacheE3 = grid(0).cache(ENTITY3_NAME); + IgniteCache cacheE4 = grid(0).cache(ENTITY4_NAME); + + assertEquals("Unexpected entries: " + toSet(cache1.iterator()), expCache1, cache1.size()); + assertEquals("Unexpected entries: " + toSet(cache2.iterator()), expCache2, cache2.size()); + assertEquals("Unexpected entries: " + toSet(cache3.iterator()), expCache3, cache3.size()); + assertEquals("Unexpected entries: " + toSet(cacheE3.iterator()), expCacheE3, cacheE3.size()); + assertEquals("Unexpected entries: " + toSet(cacheE4.iterator()), expCacheE4, cacheE4.size()); + } + finally { + sesFactory.close(); + } + } + + /** + * + */ + private Set> toSet(Iterator> iter){ + Set> set = new HashSet<>(); + + while (iter.hasNext()) + set.add(iter.next()); + + return set; + } + + /** + * @param igniteInstanceName Name of the grid providing caches. + * @return Session factory. + */ + private SessionFactory startHibernate(String igniteInstanceName) { + Configuration cfg = hibernateConfiguration(igniteInstanceName); + + StandardServiceRegistryBuilder builder = new StandardServiceRegistryBuilder(); + + builder.applySetting("hibernate.connection.url", CONNECTION_URL); + builder.applySetting("hibernate.show_sql", false); + builder.applySettings(cfg.getProperties()); + + return cfg.buildSessionFactory(builder.build()); + } + + /** + * Test Hibernate entity1. + */ + @javax.persistence.Entity + @SuppressWarnings({"PublicInnerClass", "UnnecessaryFullyQualifiedName"}) + @Cacheable + @org.hibernate.annotations.Cache(usage = CacheConcurrencyStrategy.NONSTRICT_READ_WRITE) + public static class Entity1 { + /** */ + private int id; + + /** + * @return ID. + */ + @Id + @GeneratedValue + public int getId() { + return id; + } + + /** + * @param id ID. + */ + public void setId(int id) { + this.id = id; + } + } + + /** + * Test Hibernate entity2. + */ + @javax.persistence.Entity + @SuppressWarnings({"PublicInnerClass", "UnnecessaryFullyQualifiedName"}) + @Cacheable + @org.hibernate.annotations.Cache(usage = CacheConcurrencyStrategy.NONSTRICT_READ_WRITE) + public static class Entity2 { + /** */ + private int id; + + /** + * @return ID. + */ + @Id + @GeneratedValue + public int getId() { + return id; + } + + /** + * @param id ID. + */ + public void setId(int id) { + this.id = id; + } + } + + /** + * Test Hibernate entity3. + */ + @javax.persistence.Entity + @SuppressWarnings({"PublicInnerClass", "UnnecessaryFullyQualifiedName"}) + @Cacheable + @org.hibernate.annotations.Cache(usage = CacheConcurrencyStrategy.NONSTRICT_READ_WRITE) + public static class Entity3 { + /** */ + private int id; + + /** + * @return ID. + */ + @Id + @GeneratedValue + public int getId() { + return id; + } + + /** + * @param id ID. + */ + public void setId(int id) { + this.id = id; + } + } + + /** + * Test Hibernate entity4. + */ + @javax.persistence.Entity + @SuppressWarnings({"PublicInnerClass", "UnnecessaryFullyQualifiedName"}) + @Cacheable + @org.hibernate.annotations.Cache(usage = CacheConcurrencyStrategy.NONSTRICT_READ_WRITE) + public static class Entity4 { + /** */ + private int id; + + /** + * @return ID. + */ + @Id + @GeneratedValue + public int getId() { + return id; + } + + /** + * @param id ID. + */ + public void setId(int id) { + this.id = id; + } + } +} \ No newline at end of file http://git-wip-us.apache.org/repos/asf/ignite/blob/ee1b19d3/modules/hibernate-5.1/src/test/java/org/apache/ignite/cache/hibernate/HibernateL2CacheMultiJvmTest.java ---------------------------------------------------------------------- diff --git a/modules/hibernate-5.1/src/test/java/org/apache/ignite/cache/hibernate/HibernateL2CacheMultiJvmTest.java b/modules/hibernate-5.1/src/test/java/org/apache/ignite/cache/hibernate/HibernateL2CacheMultiJvmTest.java new file mode 100644 index 0000000..57a3f71 --- /dev/null +++ b/modules/hibernate-5.1/src/test/java/org/apache/ignite/cache/hibernate/HibernateL2CacheMultiJvmTest.java @@ -0,0 +1,429 @@ +/* + * 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.ignite.cache.hibernate; + +import java.util.Map; +import javax.persistence.Cacheable; +import javax.persistence.Id; +import org.apache.ignite.Ignite; +import org.apache.ignite.IgniteCompute; +import org.apache.ignite.IgniteLogger; +import org.apache.ignite.configuration.CacheConfiguration; +import org.apache.ignite.configuration.IgniteConfiguration; +import org.apache.ignite.internal.binary.BinaryMarshaller; +import org.apache.ignite.lang.IgniteRunnable; +import org.apache.ignite.resources.IgniteInstanceResource; +import org.apache.ignite.resources.LoggerResource; +import org.apache.ignite.testframework.junits.common.GridCommonAbstractTest; +import org.hibernate.Session; +import org.hibernate.SessionFactory; +import org.hibernate.Transaction; +import org.hibernate.annotations.CacheConcurrencyStrategy; +import org.hibernate.boot.MetadataSources; +import org.hibernate.boot.registry.StandardServiceRegistryBuilder; + +import static org.apache.ignite.cache.CacheWriteSynchronizationMode.FULL_SYNC; +import static org.apache.ignite.cache.hibernate.HibernateAccessStrategyFactory.DFLT_CACHE_NAME_PROPERTY; +import static org.apache.ignite.cache.hibernate.HibernateL2CacheSelfTest.CONNECTION_URL; +import static org.apache.ignite.cache.hibernate.HibernateL2CacheSelfTest.hibernateProperties; +import static org.hibernate.cache.spi.access.AccessType.NONSTRICT_READ_WRITE; + +/** + * + */ +public class HibernateL2CacheMultiJvmTest extends GridCommonAbstractTest { + /** */ + private static final String CACHE_NAME = "hibernateCache"; + + /** {@inheritDoc} */ + @Override protected IgniteConfiguration getConfiguration(String igniteInstanceName) throws Exception { + IgniteConfiguration cfg = super.getConfiguration(igniteInstanceName); + + if (!getTestIgniteInstanceName(0).equals(igniteInstanceName)) + cfg.setClientMode(true); + + CacheConfiguration ccfg = new CacheConfiguration(); + + ccfg.setName(CACHE_NAME); + ccfg.setWriteSynchronizationMode(FULL_SYNC); + + cfg.setCacheConfiguration(ccfg); + + cfg.setMarshaller(new BinaryMarshaller()); + + cfg.setPeerClassLoadingEnabled(false); + + return cfg; + } + + /** {@inheritDoc} */ + @Override protected boolean isMultiJvm() { + return true; + } + + /** {@inheritDoc} */ + @Override protected void beforeTestsStarted() throws Exception { + super.beforeTestsStarted(); + + startGrid(0); + + startGrid(1); + startGrid(2); + } + + /** {@inheritDoc} */ + @Override protected void afterTestsStopped() throws Exception { + stopAllGrids(); + + super.afterTestsStopped(); + } + + /** + * @throws Exception If failed. + */ + public void testL2Cache() throws Exception { + Ignite srv = ignite(0); + + { + IgniteCompute client1Compute = + srv.compute(srv.cluster().forNodeId(ignite(1).cluster().localNode().id())); + + client1Compute.run(new HibernateInsertRunnable()); + } + + { + IgniteCompute client2Compute = + srv.compute(srv.cluster().forNodeId(ignite(2).cluster().localNode().id())); + + client2Compute.run(new HibernateLoadRunnable()); + } + + { + IgniteCompute srvCompute = srv.compute(srv.cluster().forLocal()); + + srvCompute.run(new HibernateLoadRunnable()); + } + } + + /** + * + */ + private static class HibernateInsertRunnable extends HibernateBaseRunnable { + /** {@inheritDoc} */ + @Override public void run() { + SessionFactory sesFactory = startHibernate(ignite.name()); + + Session ses = sesFactory.openSession(); + + try { + Transaction tx = ses.beginTransaction(); + + for (int i = 0; i < 1; i++) { + { + Entity1 e = new Entity1(); + e.setId(i); + e.setName("name-" + i); + + ses.save(e); + } + + { + Entity2 e = new Entity2(); + e.setId(String.valueOf(i)); + e.setName("name-" + i); + + ses.save(e); + } + + { + Entity3 e = new Entity3(); + e.setId(i); + e.setName("name-" + i); + + ses.save(e); + } + } + + tx.commit(); + } + finally { + ses.close(); + } + } + } + + /** + * + */ + private static class HibernateLoadRunnable extends HibernateBaseRunnable { + /** {@inheritDoc} */ + @Override public void run() { + SessionFactory sesFactory = startHibernate(ignite.name()); + + Session ses = sesFactory.openSession(); + + try { + Transaction tx = ses.beginTransaction(); + + for (int i = 0; i < 1; i++) { + { + Entity1 e = (Entity1)ses.load(Entity1.class, i); + + log.info("Found: " + e.getName()); + } + { + Entity2 e = (Entity2)ses.load(Entity2.class, String.valueOf(i)); + + log.info("Found: " + e.getName()); + } + { + Entity3 e = (Entity3)ses.load(Entity3.class, (double)i); + + log.info("Found: " + e.getName()); + } + } + + tx.commit(); + } + finally { + ses.close(); + } + } + } + + /** + * + */ + private abstract static class HibernateBaseRunnable implements IgniteRunnable { + /** */ + @IgniteInstanceResource + protected Ignite ignite; + + /** */ + @LoggerResource + IgniteLogger log; + + /** + * @param nodeName Name of the grid providing caches. + * @return Session factory. + */ + SessionFactory startHibernate(String nodeName) { + log.info("Start hibernate on node: " + nodeName); + + StandardServiceRegistryBuilder builder = new StandardServiceRegistryBuilder(); + + for (Map.Entry e : hibernateProperties(nodeName, NONSTRICT_READ_WRITE.name()).entrySet()) + builder.applySetting(e.getKey(), e.getValue()); + + builder.applySetting("hibernate.connection.url", CONNECTION_URL); + builder.applySetting(DFLT_CACHE_NAME_PROPERTY, CACHE_NAME); + + MetadataSources metadataSources = new MetadataSources(builder.build()); + + metadataSources.addAnnotatedClass(Entity1.class); + metadataSources.addAnnotatedClass(Entity2.class); + metadataSources.addAnnotatedClass(Entity3.class); + + return metadataSources.buildMetadata().buildSessionFactory(); + } + } + + /** + * Test Hibernate entity1. + */ + @javax.persistence.Entity + @Cacheable + @org.hibernate.annotations.Cache(usage = CacheConcurrencyStrategy.NONSTRICT_READ_WRITE) + public static class Entity1 { + /** */ + @Id + private int id; + + /** */ + private String name; + + /** + * @return ID. + */ + public int getId() { + return id; + } + + /** + * @param id ID. + */ + public void setId(int id) { + this.id = id; + } + + /** + * @return Name. + */ + public String getName() { + return name; + } + + /** + * @param name Name. + */ + public void setName(String name) { + this.name = name; + } + + /** {@inheritDoc} */ + @Override public boolean equals(Object o) { + if (this == o) + return true; + + if (o == null || getClass() != o.getClass()) + return false; + + Entity1 entity1 = (Entity1)o; + + return id == entity1.id; + } + + /** {@inheritDoc} */ + @Override public int hashCode() { + return id; + } + } + + /** + * Test Hibernate entity1. + */ + @javax.persistence.Entity + @Cacheable + @org.hibernate.annotations.Cache(usage = CacheConcurrencyStrategy.NONSTRICT_READ_WRITE) + public static class Entity2 { + /** */ + @Id + private String id; + + /** */ + private String name; + + /** + * @return ID. + */ + public String getId() { + return id; + } + + /** + * @param id ID. + */ + public void setId(String id) { + this.id = id; + } + + /** + * @return Name. + */ + public String getName() { + return name; + } + + /** + * @param name Name. + */ + public void setName(String name) { + this.name = name; + } + + /** {@inheritDoc} */ + @Override public boolean equals(Object o) { + if (this == o) + return true; + + if (o == null || getClass() != o.getClass()) + return false; + + Entity2 entity2 = (Entity2)o; + + return id.equals(entity2.id); + } + + /** {@inheritDoc} */ + @Override public int hashCode() { + return id.hashCode(); + } + } + + /** + * Test Hibernate entity1. + */ + @javax.persistence.Entity + @Cacheable + @org.hibernate.annotations.Cache(usage = CacheConcurrencyStrategy.NONSTRICT_READ_WRITE) + public static class Entity3 { + /** */ + @Id + private double id; + + /** */ + private String name; + + /** + * @return ID. + */ + public double getId() { + return id; + } + + /** + * @param id ID. + */ + public void setId(double id) { + this.id = id; + } + + /** + * @return Name. + */ + public String getName() { + return name; + } + + /** + * @param name Name. + */ + public void setName(String name) { + this.name = name; + } + + /** {@inheritDoc} */ + @Override public boolean equals(Object o) { + if (this == o) + return true; + + if (o == null || getClass() != o.getClass()) + return false; + + Entity3 entity3 = (Entity3)o; + + return Double.compare(entity3.id, id) == 0; + } + + /** {@inheritDoc} */ + @Override public int hashCode() { + long temp = Double.doubleToLongBits(id); + return (int)(temp ^ (temp >>> 32)); + } + } +}