You are viewing a plain text version of this content. The canonical link for it is here.
Posted to dev@gora.apache.org by djkevincr <gi...@git.apache.org> on 2016/08/01 04:54:16 UTC

[GitHub] gora pull request #72: Gora 409

GitHub user djkevincr opened a pull request:

    https://github.com/apache/gora/pull/72

    Gora 409

    

You can merge this pull request into a Git repository by running:

    $ git pull https://github.com/djkevincr/gora GORA-409

Alternatively you can review and apply these changes as the patch at:

    https://github.com/apache/gora/pull/72.patch

To close this pull request, make a commit to your master/trunk branch
with (at least) the following in the commit message:

    This closes #72
    
----
commit 2b8b0610767f4cc51f0a393a0627b721069f052c
Author: Kevin Ratnasekera <dj...@yahoo.com>
Date:   2016-05-29T13:47:57Z

    GORA-409 JCache datastore initial project structure commit

commit e28c661bccd40a0b03221ad6af08c2bcb3075149
Author: Kevin Ratnasekera <dj...@yahoo.com>
Date:   2016-06-19T09:46:41Z

    initial poc commit

commit 5684bfa9be52505e49dec532b5b070184de0cfcb
Author: Kevin Ratnasekera <dj...@yahoo.com>
Date:   2016-07-25T20:05:23Z

    Initial datastore commit

commit 29a3f615eb2c5a451378c1695ab250fbd7d4bb1a
Author: Kevin Ratnasekera <dj...@yahoo.com>
Date:   2016-07-27T11:11:28Z

    make cache properties configurable from property file

commit fd80478e376790856b399f472283e58c3969f8fa
Author: Kevin Ratnasekera <dj...@yahoo.com>
Date:   2016-07-31T08:46:00Z

    refactored and improved logging

commit a05928a7e347e841ed0bb4359908b5feca390fb2
Author: Kevin Ratnasekera <dj...@yahoo.com>
Date:   2016-07-31T20:57:40Z

    adding avro serialization insted pure java and unit tests

commit 897c787e5f05793c710a96a3741fe9cde4af1ef7
Author: Kevin Ratnasekera <dj...@yahoo.com>
Date:   2016-07-31T21:10:37Z

    added formatting for JCacheStore

commit c9d11af58f80f200972f4c59a11cfbb6afbbb532
Author: Kevin Ratnasekera <dj...@yahoo.com>
Date:   2016-07-31T21:47:00Z

    adding default values for readthrough and writethrough

commit 4fdbf6c810a951918ed142d8fc3e83f985b5672c
Author: Kevin Ratnasekera <dj...@yahoo.com>
Date:   2016-08-01T04:32:33Z

    adding missing data bean serializable changes

commit b4cb434b726deedec52d570479289c1c655003bc
Author: Kevin Ratnasekera <dj...@yahoo.com>
Date:   2016-08-01T04:37:21Z

    reverting previous changes on persitent base

commit 8ba235dd6cf2d61e714293fddb9239735a6d8e71
Author: Kevin Ratnasekera <dj...@yahoo.com>
Date:   2016-08-01T04:48:26Z

    adding mockpersistent serializable changes

commit 351eb6323824acc79e997c254bdebcfa3a29c3d9
Author: Kevin Ratnasekera <dj...@yahoo.com>
Date:   2016-08-01T04:52:05Z

    removing un necessary hadoop dependancy from JCache module pom.xml

----


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

[GitHub] gora pull request #72: Gora 409

Posted by lewismc <gi...@git.apache.org>.
Github user lewismc commented on a diff in the pull request:

    https://github.com/apache/gora/pull/72#discussion_r73616418
  
    --- Diff: gora-jcache/src/main/java/org/apache/gora/jcache/store/JCacheCacheFactoryBuilder.java ---
    @@ -0,0 +1,41 @@
    +/**
    + * 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.gora.jcache.store;
    +
    +import org.apache.gora.persistency.impl.PersistentBase;
    +import org.apache.gora.store.DataStore;
    +import javax.cache.configuration.Factory;
    +
    +public class JCacheCacheFactoryBuilder {
    +
    +  public static <K, T extends PersistentBase> Factory<JCacheCacheLoader<K,T>>
    +  factoryOfCacheLoader(DataStore<K, T> dataStore) {
    +    return new JCacheCacheLoaderFactory<>(new JCacheCacheLoader<>(dataStore));
    +  }
    +
    +  public static <K, T extends PersistentBase> Factory<JCacheCacheWriter<K,T>>
    --- End diff --
    
    Add documentation


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

[GitHub] gora issue #72: Gora 409

Posted by djkevincr <gi...@git.apache.org>.
Github user djkevincr commented on the issue:

    https://github.com/apache/gora/pull/72
  
    Hi Lewis,
    
    I haven't used any hadoop dependency over jcache module. Anyway I ll have a look into this. Can you please give some time I am still working on the adding java docs to module.
    
    Regards
    Kevin


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

[GitHub] gora pull request #72: Gora 409

Posted by lewismc <gi...@git.apache.org>.
Github user lewismc commented on a diff in the pull request:

    https://github.com/apache/gora/pull/72#discussion_r73616273
  
    --- Diff: gora-jcache/src/main/java/org/apache/gora/jcache/store/JCacheCacheEntryListenerFactory.java ---
    @@ -0,0 +1,56 @@
    +/**
    + * 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.gora.jcache.store;
    +
    +import org.apache.gora.persistency.impl.PersistentBase;
    +import org.slf4j.Logger;
    +import org.slf4j.LoggerFactory;
    +
    +import javax.cache.configuration.Factory;
    +
    +public class JCacheCacheEntryListenerFactory <K,T extends PersistentBase>
    +        implements Factory<JCacheCacheEntryListener<K, T>> {
    +
    +  private static final Logger LOG = LoggerFactory.getLogger(JCacheCacheEntryListenerFactory.class);
    +  public static final long serialVersionUID = 201305101634L;
    +  private transient JCacheCacheEntryListener<K, T> instance;
    +
    +  public JCacheCacheEntryListenerFactory(JCacheCacheEntryListener<K, T> instance) {
    --- End diff --
    
    Add documentation


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

[GitHub] gora pull request #72: Gora 409

Posted by lewismc <gi...@git.apache.org>.
Github user lewismc commented on a diff in the pull request:

    https://github.com/apache/gora/pull/72#discussion_r73616112
  
    --- Diff: gora-jcache/src/main/java/org/apache/gora/jcache/query/JCacheResult.java ---
    @@ -0,0 +1,69 @@
    +/**
    + * 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.gora.jcache.query;
    +
    +import java.io.IOException;
    +import java.util.Iterator;
    +import java.util.NavigableSet;
    +
    +import org.apache.gora.jcache.store.JCacheStore;
    +import org.apache.gora.persistency.impl.PersistentBase;
    +import org.apache.gora.query.Query;
    +import org.apache.gora.query.impl.ResultBase;
    +import org.apache.gora.store.DataStore;
    +
    +public class JCacheResult<K, T extends PersistentBase> extends ResultBase<K, T> {
    +
    +  private NavigableSet<K> cacheKeySet;
    +  private Iterator<K> iterator;
    +
    +  public JCacheResult(DataStore<K, T> dataStore, Query<K, T> query) {
    +    super(dataStore, query);
    +  }
    +
    +  public JCacheResult(DataStore<K, T> dataStore, Query<K, T> query, NavigableSet<K> cacheKeySet) {
    +    super(dataStore, query);
    +    this.cacheKeySet = cacheKeySet;
    +    this.iterator = cacheKeySet.iterator();
    +  }
    +
    +  public JCacheStore<K, T> getDataStore() {
    --- End diff --
    
    Add documentation


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

[GitHub] gora pull request #72: Gora 409

Posted by lewismc <gi...@git.apache.org>.
Github user lewismc commented on a diff in the pull request:

    https://github.com/apache/gora/pull/72#discussion_r73615685
  
    --- Diff: gora-core/src/main/java/org/apache/gora/persistency/impl/DirtyListIterator.java ---
    @@ -1,3 +1,21 @@
    +/**
    --- End diff --
    
    
    
    Thank you for adding license header here.



---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

[GitHub] gora pull request #72: Gora 409

Posted by lewismc <gi...@git.apache.org>.
Github user lewismc commented on a diff in the pull request:

    https://github.com/apache/gora/pull/72#discussion_r73616287
  
    --- Diff: gora-jcache/src/main/java/org/apache/gora/jcache/store/JCacheCacheEntryListenerFactory.java ---
    @@ -0,0 +1,56 @@
    +/**
    + * 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.gora.jcache.store;
    +
    +import org.apache.gora.persistency.impl.PersistentBase;
    +import org.slf4j.Logger;
    +import org.slf4j.LoggerFactory;
    +
    +import javax.cache.configuration.Factory;
    +
    +public class JCacheCacheEntryListenerFactory <K,T extends PersistentBase>
    +        implements Factory<JCacheCacheEntryListener<K, T>> {
    +
    +  private static final Logger LOG = LoggerFactory.getLogger(JCacheCacheEntryListenerFactory.class);
    +  public static final long serialVersionUID = 201305101634L;
    +  private transient JCacheCacheEntryListener<K, T> instance;
    +
    +  public JCacheCacheEntryListenerFactory(JCacheCacheEntryListener<K, T> instance) {
    +    LOG.info("JCache cache entry listener factory initialized successfully.");
    +    this.instance = instance;
    +  }
    +
    +  public JCacheCacheEntryListener<K, T> create() {
    --- End diff --
    
    Add documentation


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

[GitHub] gora pull request #72: Gora 409

Posted by lewismc <gi...@git.apache.org>.
Github user lewismc commented on a diff in the pull request:

    https://github.com/apache/gora/pull/72#discussion_r73615754
  
    --- Diff: gora-core/src/main/java/org/apache/gora/store/DataStoreFactory.java ---
    @@ -42,6 +42,8 @@
     
       public static final String GORA_DEFAULT_DATASTORE_KEY = "gora.datastore.default";
     
    +  public static final String GORA_DEFAULT_CACHE_DATASTORE_KEY = "gora.cache.datastore.default";
    --- End diff --
    
    
    
    Can you document this please.



---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

[GitHub] gora pull request #72: Gora 409

Posted by lewismc <gi...@git.apache.org>.
Github user lewismc commented on a diff in the pull request:

    https://github.com/apache/gora/pull/72#discussion_r73616490
  
    --- Diff: gora-jcache/src/main/java/org/apache/gora/jcache/store/JCacheCacheLoaderFactory.java ---
    @@ -0,0 +1,56 @@
    +/**
    + * 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.gora.jcache.store;
    +
    +import org.apache.gora.persistency.impl.PersistentBase;
    +import org.slf4j.Logger;
    +import org.slf4j.LoggerFactory;
    +
    +import javax.cache.configuration.Factory;
    +
    +public class JCacheCacheLoaderFactory<K, T extends PersistentBase>
    +        implements Factory<JCacheCacheLoader<K,T>> {
    +
    +  private static final Logger LOG = LoggerFactory.getLogger(JCacheCacheLoaderFactory.class);
    +  public static final long serialVersionUID = 201305101626L;
    +  private transient JCacheCacheLoader<K, T> instance;
    +
    +  public JCacheCacheLoaderFactory(JCacheCacheLoader<K, T> instance) {
    +    LOG.info("JCache cache entry loader factory initialized successfully.");
    +    this.instance = instance;
    +  }
    +
    +  public JCacheCacheLoader<K, T> create() {
    --- End diff --
    
    Add documentation


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

[GitHub] gora pull request #72: Gora 409

Posted by lewismc <gi...@git.apache.org>.
Github user lewismc commented on a diff in the pull request:

    https://github.com/apache/gora/pull/72#discussion_r73616466
  
    --- Diff: gora-jcache/src/main/java/org/apache/gora/jcache/store/JCacheCacheLoader.java ---
    @@ -0,0 +1,65 @@
    +/**
    + * 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.gora.jcache.store;
    +
    +import org.apache.gora.persistency.impl.PersistentBase;
    +import org.apache.gora.store.DataStore;
    +import org.slf4j.Logger;
    +import org.slf4j.LoggerFactory;
    +
    +import javax.cache.integration.CacheLoader;
    +import javax.cache.integration.CacheLoaderException;
    +import java.util.HashMap;
    +import java.util.Map;
    +
    +
    +public class JCacheCacheLoader<K, T extends PersistentBase> implements CacheLoader<K, T> {
    +
    +  private static final Logger LOG = LoggerFactory.getLogger(JCacheCacheLoader.class);
    +  private DataStore<K, T> dataStore;
    +
    +  public JCacheCacheLoader(DataStore<K, T> dataStore) {
    +      this.dataStore = dataStore;
    +  }
    +
    +  @Override
    +  public T load(K key) throws CacheLoaderException {
    +    T persistent = null;
    +    try {
    +      persistent = dataStore.get(key);
    +      LOG.info("Loaded data bean from persistent datastore on key " + key.toString());
    +    } catch (CacheLoaderException ex) {
    +      throw ex;
    +    }
    +    return persistent;
    +  }
    +
    +  @Override
    +  public Map<K, T> loadAll(Iterable<? extends K> keys) throws CacheLoaderException {
    --- End diff --
    
    Add documentation


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

[GitHub] gora pull request #72: Gora 409

Posted by lewismc <gi...@git.apache.org>.
Github user lewismc commented on a diff in the pull request:

    https://github.com/apache/gora/pull/72#discussion_r73616227
  
    --- Diff: gora-jcache/src/main/java/org/apache/gora/jcache/store/JCacheCacheEntryListener.java ---
    @@ -0,0 +1,77 @@
    +/**
    + * 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.gora.jcache.store;
    +
    +import org.apache.gora.persistency.impl.PersistentBase;
    +import org.slf4j.Logger;
    +import org.slf4j.LoggerFactory;
    +
    +import javax.cache.event.CacheEntryExpiredListener;
    +import javax.cache.event.CacheEntryUpdatedListener;
    +import javax.cache.event.CacheEntryCreatedListener;
    +import javax.cache.event.CacheEntryRemovedListener;
    +import javax.cache.event.CacheEntryEvent;
    +import javax.cache.event.CacheEntryListenerException;
    +import java.util.concurrent.ConcurrentSkipListSet;
    +
    +public class JCacheCacheEntryListener<K, T extends PersistentBase>
    +        implements CacheEntryCreatedListener<K, T>,
    +        CacheEntryRemovedListener<K, T>, CacheEntryUpdatedListener<K, T>, CacheEntryExpiredListener<K, T> {
    +
    +  private static final Logger LOG = LoggerFactory.getLogger(JCacheCacheEntryListener.class);
    +  private ConcurrentSkipListSet<K> cacheEntryList;
    +
    +  public JCacheCacheEntryListener(ConcurrentSkipListSet cacheEntryList) {
    +    this.cacheEntryList = cacheEntryList;
    +  }
    +
    +  @Override
    +  public void onCreated(Iterable<CacheEntryEvent<? extends K, ? extends T>> cacheEntryEvents)
    +          throws CacheEntryListenerException {
    +    for (CacheEntryEvent<? extends K, ? extends T> event : cacheEntryEvents) {
    +      cacheEntryList.add(event.getKey());
    +      LOG.info("Cache entry added on key " + event.getKey().toString());
    +    }
    +  }
    +
    +  @Override
    +  public void onRemoved(Iterable<CacheEntryEvent<? extends K, ? extends T>> cacheEntryEvents)
    +          throws CacheEntryListenerException {
    +    for (CacheEntryEvent<? extends K, ? extends T> event : cacheEntryEvents) {
    +      cacheEntryList.remove(event.getKey());
    +      LOG.info("Cache entry removed on key " + event.getKey().toString());
    +    }
    +  }
    +
    +  @Override
    +  public void onUpdated(Iterable<CacheEntryEvent<? extends K, ? extends T>> cacheEntryEvents)
    --- End diff --
    
    Add documentation


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

[GitHub] gora pull request #72: Gora 409

Posted by lewismc <gi...@git.apache.org>.
Github user lewismc commented on a diff in the pull request:

    https://github.com/apache/gora/pull/72#discussion_r73616717
  
    --- Diff: gora-jcache/src/main/java/org/apache/gora/jcache/store/JCacheStore.java ---
    @@ -0,0 +1,412 @@
    +/**
    + * 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.gora.jcache.store;
    +
    +import java.io.IOException;
    +import java.net.URI;
    +import java.net.URISyntaxException;
    +import java.util.List;
    +import java.util.Arrays;
    +import java.util.ArrayList;
    +import java.util.Properties;
    +import java.util.concurrent.ConcurrentSkipListSet;
    +import java.util.concurrent.TimeUnit;
    +
    +import com.hazelcast.cache.HazelcastCachingProvider;
    +import com.hazelcast.cache.ICache;
    +import com.hazelcast.config.CacheConfig;
    +import com.hazelcast.config.EvictionConfig;
    +import com.hazelcast.config.EvictionPolicy;
    +import com.hazelcast.config.InMemoryFormat;
    +import com.hazelcast.core.Hazelcast;
    +import com.hazelcast.core.HazelcastInstance;
    +import com.hazelcast.core.Member;
    +import com.hazelcast.core.Partition;
    +import org.apache.avro.Schema;
    +import org.apache.gora.jcache.query.JCacheQuery;
    +import org.apache.gora.jcache.query.JCacheResult;
    +import org.apache.gora.persistency.impl.PersistentBase;
    +import org.apache.gora.query.PartitionQuery;
    +import org.apache.gora.query.Query;
    +import org.apache.gora.query.Result;
    +import org.apache.gora.query.impl.PartitionQueryImpl;
    +import org.apache.gora.store.DataStore;
    +import org.apache.gora.store.DataStoreFactory;
    +import org.apache.gora.store.impl.DataStoreBase;
    +import org.apache.gora.util.AvroUtils;
    +import org.apache.gora.util.GoraException;
    +import org.apache.hadoop.conf.Configuration;
    +import org.slf4j.Logger;
    +import org.slf4j.LoggerFactory;
    +
    +import javax.cache.CacheManager;
    +import javax.cache.Caching;
    +import javax.cache.configuration.FactoryBuilder;
    +import javax.cache.configuration.MutableCacheEntryListenerConfiguration;
    +import javax.cache.expiry.AccessedExpiryPolicy;
    +import javax.cache.expiry.ModifiedExpiryPolicy;
    +import javax.cache.expiry.CreatedExpiryPolicy;
    +import javax.cache.expiry.TouchedExpiryPolicy;
    +import javax.cache.expiry.Duration;
    +import javax.cache.spi.CachingProvider;
    +
    +public class JCacheStore<K, T extends PersistentBase> extends DataStoreBase<K, T> {
    +
    +  private static final String GORA_DEFAULT_JCACHE_NAMESPACE = "gora.jcache.namespace";
    +  private static final String GORA_DEFAULT_JCACHE_PROVIDER_KEY = "gora.datastore.jcache.provider";
    +  private static final String JCACHE_READ_THROUGH_PROPERTY_KEY = "jcache.read.through.enable";
    +  private static final String JCACHE_WRITE_THROUGH_PROPERTY_KEY = "jcache.write.through.enable";
    +  private static final String JCACHE_STORE_BY_VALUE_PROPERTY_KEY = "jcache.store.by.value.enable";
    +  private static final String JCACHE_STATISTICS_PROPERTY_KEY = "jcache.statistics.enable";
    +  private static final String JCACHE_MANAGEMENT_PROPERTY_KEY = "jcache.management.enable";
    +  private static final String JCACHE_CACHE_NAMESPACE_PROPERTY_KEY = "jcache.cache.namespace";
    +  private static final String JCACHE_EVICTION_POLICY_PROPERTY_KEY = "jcache.eviction.policy";
    +  private static final String JCACHE_EVICTION_MAX_SIZE_POLICY_PROPERTY_KEY = "jcache.eviction.max.size.policy";
    +  private static final String JCACHE_EVICTION_SIZE_PROPERTY_KEY = "jcache.eviction.size";
    +  private static final String JCACHE_EXPIRE_POLICY_PROPERTY_KEY = "jcache.expire.policy";
    +  private static final String JCACHE_EXPIRE_POLICY_DURATION_PROPERTY_KEY = "jcache.expire.policy";
    +  private static final String JCACHE_ACCESSED_EXPIRY_IDENTIFIER = "ACCESSED";
    +  private static final String JCACHE_CREATED_EXPIRY_IDENTIFIER = "CREATED";
    +  private static final String JCACHE_MODIFIED_EXPIRY_IDENTIFIER = "MODIFIED";
    +  private static final String JCACHE_TOUCHED_EXPIRY_IDENTIFIER = "TOUCHED";
    +  private static final String HAZELCAST_CACHE_IN_MEMORY_FORMAT_PROPERTY_KEY = "jcache.cache.inmemory.format";
    +  private static final String HAZELCAST_CACHE_BINARY_IN_MEMORY_FORMAT_IDENTIFIER = "BINARY";
    +  private static final String HAZELCAST_CACHE_OBJECT_IN_MEMORY_FORMAT_IDENTIFIER = "OBJECT";
    +  private static final String HAZELCAST_CACHE_NATIVE_IN_MEMORY_FORMAT_IDENTIFIER = "NATIVE";
    +  private static final String JCACHE_AUTO_CREATE_CACHE_PROPERTY_KEY = "jcache.auto.create.cache";
    +  private static final Logger LOG = LoggerFactory.getLogger(JCacheStore.class);
    +  private ICache<K, T> cache;
    +  private CacheManager manager;
    +  private ConcurrentSkipListSet<K> cacheEntryList;
    +  private String goraCacheNamespace = GORA_DEFAULT_JCACHE_NAMESPACE;
    +  private DataStore<K, T> persistentDataStore;
    +  private CacheConfig<K, T> cacheConfig;
    +  private HazelcastInstance hazelcastInstance;
    +
    +  private static <T extends PersistentBase> T getPersistent(T persitent, String[] fields) {
    +    List<Schema.Field> otherFields = persitent.getSchema().getFields();
    +    String[] otherFieldStrings = new String[otherFields.size()];
    +    for (int i = 0; i < otherFields.size(); i++) {
    +      otherFieldStrings[i] = otherFields.get(i).name();
    +    }
    +    if (Arrays.equals(fields, otherFieldStrings)) {
    +      return persitent;
    +    }
    +    T clonedPersistent = AvroUtils.deepClonePersistent(persitent);
    +    clonedPersistent.clear();
    +    if (fields != null && fields.length > 0) {
    +      for (String field : fields) {
    +        Schema.Field otherField = persitent.getSchema().getField(field);
    +        int index = otherField.pos();
    +        clonedPersistent.put(index, persitent.get(index));
    +      }
    +    } else {
    +      for (String field : otherFieldStrings) {
    +        Schema.Field otherField = persitent.getSchema().getField(field);
    +        int index = otherField.pos();
    +        clonedPersistent.put(index, persitent.get(index));
    +      }
    +    }
    +    return clonedPersistent;
    +  }
    +
    +  @Override
    +  public void initialize(Class<K> keyClass, Class<T> persistentClass, Properties properties) {
    --- End diff --
    
    Add documentation


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

[GitHub] gora issue #72: Gora 409

Posted by lewismc <gi...@git.apache.org>.
Github user lewismc commented on the issue:

    https://github.com/apache/gora/pull/72
  
    Yes, trying right now.


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

[GitHub] gora pull request #72: Gora 409

Posted by lewismc <gi...@git.apache.org>.
Github user lewismc commented on a diff in the pull request:

    https://github.com/apache/gora/pull/72#discussion_r73615730
  
    --- Diff: gora-core/src/main/java/org/apache/gora/persistency/impl/PersistentBase.java ---
    @@ -181,10 +181,14 @@ public void setDirty(String field) {
         setDirty(getSchema().getField(field).pos());
       }
     
    -  private ByteBuffer getDirtyBytes() {
    +  public ByteBuffer getDirtyBytes() {
    --- End diff --
    
    Can you document this please.


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

[GitHub] gora pull request #72: Gora 409

Posted by lewismc <gi...@git.apache.org>.
Github user lewismc commented on a diff in the pull request:

    https://github.com/apache/gora/pull/72#discussion_r73616543
  
    --- Diff: gora-jcache/src/main/java/org/apache/gora/jcache/store/JCacheCacheWriter.java ---
    @@ -0,0 +1,72 @@
    +/**
    + * 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.gora.jcache.store;
    +
    +import org.apache.gora.persistency.impl.PersistentBase;
    +import org.apache.gora.store.DataStore;
    +import org.slf4j.Logger;
    +import org.slf4j.LoggerFactory;
    +
    +import javax.cache.Cache;
    +import javax.cache.integration.CacheWriter;
    +import javax.cache.integration.CacheWriterException;
    +import java.util.Collection;
    +import java.util.Iterator;
    +
    +public class JCacheCacheWriter<K, T extends PersistentBase> implements CacheWriter<K, T> {
    +
    +  private static final Logger LOG = LoggerFactory.getLogger(JCacheCacheWriter.class);
    +  private DataStore<K, T> dataStore;
    +
    +  public JCacheCacheWriter(DataStore<K, T> dataStore) {
    +    this.dataStore = dataStore;
    +  }
    +
    +  @Override
    +  public void write(Cache.Entry<? extends K,
    +          ? extends T> entry) throws CacheWriterException {
    +    dataStore.put(entry.getKey(), entry.getValue());
    +    LOG.info("Written data bean to persistent datastore on key " + entry.getKey().toString());
    +  }
    +
    +  @Override
    +  public void writeAll(Collection<Cache.Entry<? extends K,
    +          ? extends T>> entries) throws CacheWriterException {
    +    Iterator<Cache.Entry<? extends K, ? extends T>> iterator = entries.iterator();
    +    while (iterator.hasNext()) {
    +      write(iterator.next());
    +      iterator.remove();
    +    }
    +  }
    +
    +  @Override
    +  public void delete(Object key) throws CacheWriterException {
    --- End diff --
    
    Add documentation


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

[GitHub] gora pull request #72: Gora 409

Posted by lewismc <gi...@git.apache.org>.
Github user lewismc commented on a diff in the pull request:

    https://github.com/apache/gora/pull/72#discussion_r73616255
  
    --- Diff: gora-jcache/src/main/java/org/apache/gora/jcache/store/JCacheCacheEntryListenerFactory.java ---
    @@ -0,0 +1,56 @@
    +/**
    + * 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.gora.jcache.store;
    +
    +import org.apache.gora.persistency.impl.PersistentBase;
    +import org.slf4j.Logger;
    +import org.slf4j.LoggerFactory;
    +
    +import javax.cache.configuration.Factory;
    +
    +public class JCacheCacheEntryListenerFactory <K,T extends PersistentBase>
    --- End diff --
    
    Add documentation


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

[GitHub] gora pull request #72: Gora 409

Posted by lewismc <gi...@git.apache.org>.
Github user lewismc commented on a diff in the pull request:

    https://github.com/apache/gora/pull/72#discussion_r73616486
  
    --- Diff: gora-jcache/src/main/java/org/apache/gora/jcache/store/JCacheCacheLoaderFactory.java ---
    @@ -0,0 +1,56 @@
    +/**
    + * 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.gora.jcache.store;
    +
    +import org.apache.gora.persistency.impl.PersistentBase;
    +import org.slf4j.Logger;
    +import org.slf4j.LoggerFactory;
    +
    +import javax.cache.configuration.Factory;
    +
    +public class JCacheCacheLoaderFactory<K, T extends PersistentBase>
    +        implements Factory<JCacheCacheLoader<K,T>> {
    +
    +  private static final Logger LOG = LoggerFactory.getLogger(JCacheCacheLoaderFactory.class);
    +  public static final long serialVersionUID = 201305101626L;
    +  private transient JCacheCacheLoader<K, T> instance;
    +
    +  public JCacheCacheLoaderFactory(JCacheCacheLoader<K, T> instance) {
    --- End diff --
    
    Add documentation


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

[GitHub] gora pull request #72: Gora 409

Posted by lewismc <gi...@git.apache.org>.
Github user lewismc commented on a diff in the pull request:

    https://github.com/apache/gora/pull/72#discussion_r73616685
  
    --- Diff: gora-jcache/src/main/java/org/apache/gora/jcache/store/JCacheCacheWriterFactory.java ---
    @@ -0,0 +1,55 @@
    +/**
    + * 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.gora.jcache.store;
    +
    +import org.apache.gora.persistency.impl.PersistentBase;
    +import org.slf4j.Logger;
    +import org.slf4j.LoggerFactory;
    +
    +import javax.cache.configuration.Factory;
    +
    +public class JCacheCacheWriterFactory<K, T extends PersistentBase> implements Factory<JCacheCacheWriter<K,T>> {
    +
    +  private static final Logger LOG = LoggerFactory.getLogger(JCacheCacheWriterFactory.class);
    +  public static final long serialVersionUID = 201205101621L;
    +  private transient JCacheCacheWriter<K,T> instance;
    +
    +  public JCacheCacheWriterFactory(JCacheCacheWriter<K,T> instance) {
    +    LOG.info("JCache cache writer factory initialized successfully.");
    +    this.instance = instance;
    +  }
    +
    +  public JCacheCacheWriter<K,T> create() {
    +    return (JCacheCacheWriter<K,T>)this.instance;
    +  }
    +
    +  public boolean equals(Object other) {
    +    if (this == other) {
    +      return true;
    +    } else if (other != null && this.getClass() == other.getClass()) {
    +      return true;
    +    } else {
    +      return false;
    +    }
    +  }
    +
    +  public int hashCode() {
    --- End diff --
    
    Add documentation


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

[GitHub] gora pull request #72: Gora 409

Posted by lewismc <gi...@git.apache.org>.
Github user lewismc commented on a diff in the pull request:

    https://github.com/apache/gora/pull/72#discussion_r73616218
  
    --- Diff: gora-jcache/src/main/java/org/apache/gora/jcache/store/JCacheCacheEntryListener.java ---
    @@ -0,0 +1,77 @@
    +/**
    + * 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.gora.jcache.store;
    +
    +import org.apache.gora.persistency.impl.PersistentBase;
    +import org.slf4j.Logger;
    +import org.slf4j.LoggerFactory;
    +
    +import javax.cache.event.CacheEntryExpiredListener;
    +import javax.cache.event.CacheEntryUpdatedListener;
    +import javax.cache.event.CacheEntryCreatedListener;
    +import javax.cache.event.CacheEntryRemovedListener;
    +import javax.cache.event.CacheEntryEvent;
    +import javax.cache.event.CacheEntryListenerException;
    +import java.util.concurrent.ConcurrentSkipListSet;
    +
    +public class JCacheCacheEntryListener<K, T extends PersistentBase>
    +        implements CacheEntryCreatedListener<K, T>,
    +        CacheEntryRemovedListener<K, T>, CacheEntryUpdatedListener<K, T>, CacheEntryExpiredListener<K, T> {
    +
    +  private static final Logger LOG = LoggerFactory.getLogger(JCacheCacheEntryListener.class);
    +  private ConcurrentSkipListSet<K> cacheEntryList;
    +
    +  public JCacheCacheEntryListener(ConcurrentSkipListSet cacheEntryList) {
    +    this.cacheEntryList = cacheEntryList;
    +  }
    +
    +  @Override
    +  public void onCreated(Iterable<CacheEntryEvent<? extends K, ? extends T>> cacheEntryEvents)
    +          throws CacheEntryListenerException {
    +    for (CacheEntryEvent<? extends K, ? extends T> event : cacheEntryEvents) {
    +      cacheEntryList.add(event.getKey());
    +      LOG.info("Cache entry added on key " + event.getKey().toString());
    +    }
    +  }
    +
    +  @Override
    +  public void onRemoved(Iterable<CacheEntryEvent<? extends K, ? extends T>> cacheEntryEvents)
    --- End diff --
    
    Add documentation


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

[GitHub] gora pull request #72: Gora 409

Posted by lewismc <gi...@git.apache.org>.
Github user lewismc commented on a diff in the pull request:

    https://github.com/apache/gora/pull/72#discussion_r73615702
  
    --- Diff: gora-core/src/main/java/org/apache/gora/persistency/impl/DirtyMapWrapper.java ---
    @@ -1,3 +1,21 @@
    +/**
    --- End diff --
    
    
    
    Thank you for adding license header here.



---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

[GitHub] gora pull request #72: Gora 409

Posted by lewismc <gi...@git.apache.org>.
Github user lewismc commented on a diff in the pull request:

    https://github.com/apache/gora/pull/72#discussion_r73616641
  
    --- Diff: gora-jcache/src/main/java/org/apache/gora/jcache/store/JCacheCacheWriterFactory.java ---
    @@ -0,0 +1,55 @@
    +/**
    + * 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.gora.jcache.store;
    +
    +import org.apache.gora.persistency.impl.PersistentBase;
    +import org.slf4j.Logger;
    +import org.slf4j.LoggerFactory;
    +
    +import javax.cache.configuration.Factory;
    +
    +public class JCacheCacheWriterFactory<K, T extends PersistentBase> implements Factory<JCacheCacheWriter<K,T>> {
    --- End diff --
    
    Add documentation


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

[GitHub] gora pull request #72: Gora 409

Posted by lewismc <gi...@git.apache.org>.
Github user lewismc commented on a diff in the pull request:

    https://github.com/apache/gora/pull/72#discussion_r73616496
  
    --- Diff: gora-jcache/src/main/java/org/apache/gora/jcache/store/JCacheCacheLoaderFactory.java ---
    @@ -0,0 +1,56 @@
    +/**
    + * 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.gora.jcache.store;
    +
    +import org.apache.gora.persistency.impl.PersistentBase;
    +import org.slf4j.Logger;
    +import org.slf4j.LoggerFactory;
    +
    +import javax.cache.configuration.Factory;
    +
    +public class JCacheCacheLoaderFactory<K, T extends PersistentBase>
    +        implements Factory<JCacheCacheLoader<K,T>> {
    +
    +  private static final Logger LOG = LoggerFactory.getLogger(JCacheCacheLoaderFactory.class);
    +  public static final long serialVersionUID = 201305101626L;
    +  private transient JCacheCacheLoader<K, T> instance;
    +
    +  public JCacheCacheLoaderFactory(JCacheCacheLoader<K, T> instance) {
    +    LOG.info("JCache cache entry loader factory initialized successfully.");
    +    this.instance = instance;
    +  }
    +
    +  public JCacheCacheLoader<K, T> create() {
    +    return (JCacheCacheLoader<K, T>) this.instance;
    +  }
    +
    +  public boolean equals(Object other) {
    --- End diff --
    
    Add documentation


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

[GitHub] gora pull request #72: Gora 409

Posted by lewismc <gi...@git.apache.org>.
Github user lewismc commented on a diff in the pull request:

    https://github.com/apache/gora/pull/72#discussion_r73616463
  
    --- Diff: gora-jcache/src/main/java/org/apache/gora/jcache/store/JCacheCacheLoader.java ---
    @@ -0,0 +1,65 @@
    +/**
    + * 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.gora.jcache.store;
    +
    +import org.apache.gora.persistency.impl.PersistentBase;
    +import org.apache.gora.store.DataStore;
    +import org.slf4j.Logger;
    +import org.slf4j.LoggerFactory;
    +
    +import javax.cache.integration.CacheLoader;
    +import javax.cache.integration.CacheLoaderException;
    +import java.util.HashMap;
    +import java.util.Map;
    +
    +
    +public class JCacheCacheLoader<K, T extends PersistentBase> implements CacheLoader<K, T> {
    +
    +  private static final Logger LOG = LoggerFactory.getLogger(JCacheCacheLoader.class);
    +  private DataStore<K, T> dataStore;
    +
    +  public JCacheCacheLoader(DataStore<K, T> dataStore) {
    +      this.dataStore = dataStore;
    +  }
    +
    +  @Override
    +  public T load(K key) throws CacheLoaderException {
    --- End diff --
    
    Add documentation


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

[GitHub] gora pull request #72: Gora 409

Posted by lewismc <gi...@git.apache.org>.
Github user lewismc commented on a diff in the pull request:

    https://github.com/apache/gora/pull/72#discussion_r73616396
  
    --- Diff: gora-jcache/src/main/java/org/apache/gora/jcache/store/JCacheCacheFactoryBuilder.java ---
    @@ -0,0 +1,41 @@
    +/**
    + * 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.gora.jcache.store;
    +
    +import org.apache.gora.persistency.impl.PersistentBase;
    +import org.apache.gora.store.DataStore;
    +import javax.cache.configuration.Factory;
    +
    +public class JCacheCacheFactoryBuilder {
    +
    +  public static <K, T extends PersistentBase> Factory<JCacheCacheLoader<K,T>>
    --- End diff --
    
    Add documentation


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

[GitHub] gora pull request #72: Gora 409

Posted by lewismc <gi...@git.apache.org>.
Github user lewismc commented on a diff in the pull request:

    https://github.com/apache/gora/pull/72#discussion_r73616098
  
    --- Diff: gora-jcache/src/main/java/org/apache/gora/jcache/query/JCacheResult.java ---
    @@ -0,0 +1,69 @@
    +/**
    + * 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.gora.jcache.query;
    +
    +import java.io.IOException;
    +import java.util.Iterator;
    +import java.util.NavigableSet;
    +
    +import org.apache.gora.jcache.store.JCacheStore;
    +import org.apache.gora.persistency.impl.PersistentBase;
    +import org.apache.gora.query.Query;
    +import org.apache.gora.query.impl.ResultBase;
    +import org.apache.gora.store.DataStore;
    +
    +public class JCacheResult<K, T extends PersistentBase> extends ResultBase<K, T> {
    +
    +  private NavigableSet<K> cacheKeySet;
    +  private Iterator<K> iterator;
    +
    +  public JCacheResult(DataStore<K, T> dataStore, Query<K, T> query) {
    +    super(dataStore, query);
    +  }
    +
    +  public JCacheResult(DataStore<K, T> dataStore, Query<K, T> query, NavigableSet<K> cacheKeySet) {
    --- End diff --
    
    Add documentation


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

[GitHub] gora issue #72: Gora 409

Posted by djkevincr <gi...@git.apache.org>.
Github user djkevincr commented on the issue:

    https://github.com/apache/gora/pull/72
  
    @lewismc Please find PR on [1] which addresses some of your review comments as well as improvements to unit tests. 
    [1] https://github.com/apache/gora/pull/73


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

[GitHub] gora issue #72: Gora 409

Posted by djkevincr <gi...@git.apache.org>.
Github user djkevincr commented on the issue:

    https://github.com/apache/gora/pull/72
  
    Can you please try now?


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

[GitHub] gora pull request #72: Gora 409

Posted by lewismc <gi...@git.apache.org>.
Github user lewismc commented on a diff in the pull request:

    https://github.com/apache/gora/pull/72#discussion_r73617004
  
    --- Diff: gora-jcache/src/main/java/org/apache/gora/jcache/store/JCacheStore.java ---
    @@ -0,0 +1,412 @@
    +/**
    + * 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.gora.jcache.store;
    +
    +import java.io.IOException;
    +import java.net.URI;
    +import java.net.URISyntaxException;
    +import java.util.List;
    +import java.util.Arrays;
    +import java.util.ArrayList;
    +import java.util.Properties;
    +import java.util.concurrent.ConcurrentSkipListSet;
    +import java.util.concurrent.TimeUnit;
    +
    +import com.hazelcast.cache.HazelcastCachingProvider;
    +import com.hazelcast.cache.ICache;
    +import com.hazelcast.config.CacheConfig;
    +import com.hazelcast.config.EvictionConfig;
    +import com.hazelcast.config.EvictionPolicy;
    +import com.hazelcast.config.InMemoryFormat;
    +import com.hazelcast.core.Hazelcast;
    +import com.hazelcast.core.HazelcastInstance;
    +import com.hazelcast.core.Member;
    +import com.hazelcast.core.Partition;
    +import org.apache.avro.Schema;
    +import org.apache.gora.jcache.query.JCacheQuery;
    +import org.apache.gora.jcache.query.JCacheResult;
    +import org.apache.gora.persistency.impl.PersistentBase;
    +import org.apache.gora.query.PartitionQuery;
    +import org.apache.gora.query.Query;
    +import org.apache.gora.query.Result;
    +import org.apache.gora.query.impl.PartitionQueryImpl;
    +import org.apache.gora.store.DataStore;
    +import org.apache.gora.store.DataStoreFactory;
    +import org.apache.gora.store.impl.DataStoreBase;
    +import org.apache.gora.util.AvroUtils;
    +import org.apache.gora.util.GoraException;
    +import org.apache.hadoop.conf.Configuration;
    +import org.slf4j.Logger;
    +import org.slf4j.LoggerFactory;
    +
    +import javax.cache.CacheManager;
    +import javax.cache.Caching;
    +import javax.cache.configuration.FactoryBuilder;
    +import javax.cache.configuration.MutableCacheEntryListenerConfiguration;
    +import javax.cache.expiry.AccessedExpiryPolicy;
    +import javax.cache.expiry.ModifiedExpiryPolicy;
    +import javax.cache.expiry.CreatedExpiryPolicy;
    +import javax.cache.expiry.TouchedExpiryPolicy;
    +import javax.cache.expiry.Duration;
    +import javax.cache.spi.CachingProvider;
    +
    +public class JCacheStore<K, T extends PersistentBase> extends DataStoreBase<K, T> {
    +
    +  private static final String GORA_DEFAULT_JCACHE_NAMESPACE = "gora.jcache.namespace";
    +  private static final String GORA_DEFAULT_JCACHE_PROVIDER_KEY = "gora.datastore.jcache.provider";
    +  private static final String JCACHE_READ_THROUGH_PROPERTY_KEY = "jcache.read.through.enable";
    +  private static final String JCACHE_WRITE_THROUGH_PROPERTY_KEY = "jcache.write.through.enable";
    +  private static final String JCACHE_STORE_BY_VALUE_PROPERTY_KEY = "jcache.store.by.value.enable";
    +  private static final String JCACHE_STATISTICS_PROPERTY_KEY = "jcache.statistics.enable";
    +  private static final String JCACHE_MANAGEMENT_PROPERTY_KEY = "jcache.management.enable";
    +  private static final String JCACHE_CACHE_NAMESPACE_PROPERTY_KEY = "jcache.cache.namespace";
    +  private static final String JCACHE_EVICTION_POLICY_PROPERTY_KEY = "jcache.eviction.policy";
    +  private static final String JCACHE_EVICTION_MAX_SIZE_POLICY_PROPERTY_KEY = "jcache.eviction.max.size.policy";
    +  private static final String JCACHE_EVICTION_SIZE_PROPERTY_KEY = "jcache.eviction.size";
    +  private static final String JCACHE_EXPIRE_POLICY_PROPERTY_KEY = "jcache.expire.policy";
    +  private static final String JCACHE_EXPIRE_POLICY_DURATION_PROPERTY_KEY = "jcache.expire.policy";
    +  private static final String JCACHE_ACCESSED_EXPIRY_IDENTIFIER = "ACCESSED";
    +  private static final String JCACHE_CREATED_EXPIRY_IDENTIFIER = "CREATED";
    +  private static final String JCACHE_MODIFIED_EXPIRY_IDENTIFIER = "MODIFIED";
    +  private static final String JCACHE_TOUCHED_EXPIRY_IDENTIFIER = "TOUCHED";
    +  private static final String HAZELCAST_CACHE_IN_MEMORY_FORMAT_PROPERTY_KEY = "jcache.cache.inmemory.format";
    +  private static final String HAZELCAST_CACHE_BINARY_IN_MEMORY_FORMAT_IDENTIFIER = "BINARY";
    +  private static final String HAZELCAST_CACHE_OBJECT_IN_MEMORY_FORMAT_IDENTIFIER = "OBJECT";
    +  private static final String HAZELCAST_CACHE_NATIVE_IN_MEMORY_FORMAT_IDENTIFIER = "NATIVE";
    +  private static final String JCACHE_AUTO_CREATE_CACHE_PROPERTY_KEY = "jcache.auto.create.cache";
    +  private static final Logger LOG = LoggerFactory.getLogger(JCacheStore.class);
    +  private ICache<K, T> cache;
    +  private CacheManager manager;
    +  private ConcurrentSkipListSet<K> cacheEntryList;
    +  private String goraCacheNamespace = GORA_DEFAULT_JCACHE_NAMESPACE;
    +  private DataStore<K, T> persistentDataStore;
    +  private CacheConfig<K, T> cacheConfig;
    +  private HazelcastInstance hazelcastInstance;
    +
    +  private static <T extends PersistentBase> T getPersistent(T persitent, String[] fields) {
    +    List<Schema.Field> otherFields = persitent.getSchema().getFields();
    +    String[] otherFieldStrings = new String[otherFields.size()];
    +    for (int i = 0; i < otherFields.size(); i++) {
    +      otherFieldStrings[i] = otherFields.get(i).name();
    +    }
    +    if (Arrays.equals(fields, otherFieldStrings)) {
    +      return persitent;
    +    }
    +    T clonedPersistent = AvroUtils.deepClonePersistent(persitent);
    +    clonedPersistent.clear();
    +    if (fields != null && fields.length > 0) {
    +      for (String field : fields) {
    +        Schema.Field otherField = persitent.getSchema().getField(field);
    +        int index = otherField.pos();
    +        clonedPersistent.put(index, persitent.get(index));
    +      }
    +    } else {
    +      for (String field : otherFieldStrings) {
    +        Schema.Field otherField = persitent.getSchema().getField(field);
    +        int index = otherField.pos();
    +        clonedPersistent.put(index, persitent.get(index));
    +      }
    +    }
    +    return clonedPersistent;
    +  }
    +
    +  @Override
    +  public void initialize(Class<K> keyClass, Class<T> persistentClass, Properties properties) {
    +    super.initialize(keyClass, persistentClass, properties);
    +    CachingProvider cachingProvider = Caching.getCachingProvider(
    +            properties.getProperty(GORA_DEFAULT_JCACHE_PROVIDER_KEY)
    +    );
    +    if (properties.getProperty(JCACHE_CACHE_NAMESPACE_PROPERTY_KEY) != null) {
    +      goraCacheNamespace = properties.getProperty(JCACHE_CACHE_NAMESPACE_PROPERTY_KEY);
    +    }
    +    try {
    +      this.persistentDataStore = DataStoreFactory.getDataStore(keyClass, persistentClass,
    +              new Configuration());
    +    } catch (GoraException ex) {
    +      LOG.error("Couldn't initialize persistent DataStore");
    +    }
    +    hazelcastInstance = Hazelcast.newHazelcastInstance();
    +    Properties providerProperties = new Properties();
    +    providerProperties.setProperty(HazelcastCachingProvider.HAZELCAST_INSTANCE_NAME,
    +            hazelcastInstance.getName());
    +    try {
    +      manager = cachingProvider.getCacheManager(new URI(goraCacheNamespace), null, providerProperties);
    +    } catch (URISyntaxException ex) {
    +      LOG.error("Couldn't initialize cache manager to bounded hazelcast instance");
    +      manager = cachingProvider.getCacheManager();
    +    }
    +    cacheEntryList = new ConcurrentSkipListSet<>();
    +    cacheConfig = new CacheConfig<K, T>();
    +    cacheConfig.setTypes(keyClass, persistentClass);
    +    if (properties.getProperty(JCACHE_READ_THROUGH_PROPERTY_KEY) != null) {
    +      cacheConfig.setReadThrough(Boolean.valueOf(properties.getProperty(JCACHE_READ_THROUGH_PROPERTY_KEY)));
    +    } else {
    +      cacheConfig.setReadThrough(true);
    +    }
    +    if (properties.getProperty(JCACHE_WRITE_THROUGH_PROPERTY_KEY) != null) {
    +      cacheConfig.setWriteThrough(Boolean.valueOf(properties.getProperty(JCACHE_WRITE_THROUGH_PROPERTY_KEY)));
    +    } else {
    +      cacheConfig.setWriteThrough(true);
    +    }
    +    if (properties.getProperty(JCACHE_STORE_BY_VALUE_PROPERTY_KEY) != null) {
    +      cacheConfig.setStoreByValue(Boolean.valueOf(properties.getProperty(JCACHE_STORE_BY_VALUE_PROPERTY_KEY)));
    +    }
    +    if (properties.getProperty(JCACHE_STATISTICS_PROPERTY_KEY) != null) {
    +      cacheConfig.setStatisticsEnabled(Boolean.valueOf(properties.getProperty(JCACHE_STATISTICS_PROPERTY_KEY)));
    +    }
    +    if (properties.getProperty(JCACHE_MANAGEMENT_PROPERTY_KEY) != null) {
    +      cacheConfig.setStatisticsEnabled(Boolean.valueOf(properties.getProperty(JCACHE_MANAGEMENT_PROPERTY_KEY)));
    +    }
    +    if (properties.getProperty(JCACHE_EVICTION_POLICY_PROPERTY_KEY) != null) {
    +      cacheConfig.getEvictionConfig()
    +              .setEvictionPolicy(EvictionPolicy.valueOf(properties.getProperty(JCACHE_EVICTION_POLICY_PROPERTY_KEY)));
    +    }
    +    if (properties.getProperty(JCACHE_EVICTION_MAX_SIZE_POLICY_PROPERTY_KEY) != null) {
    +      cacheConfig.getEvictionConfig()
    +              .setMaximumSizePolicy(EvictionConfig.MaxSizePolicy
    +                      .valueOf(properties.getProperty(JCACHE_EVICTION_MAX_SIZE_POLICY_PROPERTY_KEY)));
    +    }
    +    if (properties.getProperty(JCACHE_EVICTION_SIZE_PROPERTY_KEY) != null) {
    +      cacheConfig.getEvictionConfig()
    +              .setSize(Integer.valueOf(properties.getProperty(JCACHE_EVICTION_SIZE_PROPERTY_KEY)));
    +    }
    +    if (properties.getProperty(JCACHE_EXPIRE_POLICY_PROPERTY_KEY) != null) {
    +      String expiryPolicyIdentifier = properties.getProperty(JCACHE_EXPIRE_POLICY_PROPERTY_KEY);
    +      if (expiryPolicyIdentifier.equals(JCACHE_ACCESSED_EXPIRY_IDENTIFIER)) {
    +        cacheConfig.setExpiryPolicyFactory(FactoryBuilder.factoryOf(
    +                new AccessedExpiryPolicy(new Duration(TimeUnit.SECONDS,
    +                        Integer.valueOf(properties.getProperty(JCACHE_EXPIRE_POLICY_DURATION_PROPERTY_KEY))))
    +        ));
    +      } else if (expiryPolicyIdentifier.equals(JCACHE_CREATED_EXPIRY_IDENTIFIER)) {
    +        cacheConfig.setExpiryPolicyFactory(FactoryBuilder.factoryOf(
    +                new CreatedExpiryPolicy(new Duration(TimeUnit.SECONDS,
    +                        Integer.valueOf(properties.getProperty(JCACHE_EXPIRE_POLICY_DURATION_PROPERTY_KEY))))
    +        ));
    +      } else if (expiryPolicyIdentifier.equals(JCACHE_MODIFIED_EXPIRY_IDENTIFIER)) {
    +        cacheConfig.setExpiryPolicyFactory(FactoryBuilder.factoryOf(
    +                new ModifiedExpiryPolicy(new Duration(TimeUnit.SECONDS,
    +                        Integer.valueOf(properties.getProperty(JCACHE_EXPIRE_POLICY_DURATION_PROPERTY_KEY))))
    +        ));
    +      } else if (expiryPolicyIdentifier.equals(JCACHE_TOUCHED_EXPIRY_IDENTIFIER)) {
    +        cacheConfig.setExpiryPolicyFactory(FactoryBuilder.factoryOf(
    +                new TouchedExpiryPolicy(new Duration(TimeUnit.SECONDS,
    +                        Integer.valueOf(properties.getProperty(JCACHE_EXPIRE_POLICY_DURATION_PROPERTY_KEY))))
    +        ));
    +      }
    +    }
    +    if (properties.getProperty(HAZELCAST_CACHE_IN_MEMORY_FORMAT_PROPERTY_KEY) != null) {
    +      String inMemoryFormat = properties.getProperty(HAZELCAST_CACHE_IN_MEMORY_FORMAT_PROPERTY_KEY);
    +      if (inMemoryFormat.equals(HAZELCAST_CACHE_BINARY_IN_MEMORY_FORMAT_IDENTIFIER) ||
    +              inMemoryFormat.equals(HAZELCAST_CACHE_OBJECT_IN_MEMORY_FORMAT_IDENTIFIER) ||
    +              inMemoryFormat.equals(HAZELCAST_CACHE_NATIVE_IN_MEMORY_FORMAT_IDENTIFIER)) {
    +        cacheConfig.setInMemoryFormat(InMemoryFormat.valueOf(inMemoryFormat));
    +      }
    +    }
    +    cacheConfig.setCacheLoaderFactory(JCacheCacheFactoryBuilder
    +            .factoryOfCacheLoader(this.persistentDataStore));
    +    cacheConfig.setCacheWriterFactory(JCacheCacheFactoryBuilder
    +            .factoryOfCacheWriter(this.persistentDataStore));
    +    cacheConfig.addCacheEntryListenerConfiguration(
    +            new MutableCacheEntryListenerConfiguration<>(
    +                    JCacheCacheFactoryBuilder
    +                            .factoryOfEntryListener(new JCacheCacheEntryListener<K, T>(cacheEntryList)),
    +                    null, true, true
    +            )
    +    );
    +    if (properties.getProperty(JCACHE_AUTO_CREATE_CACHE_PROPERTY_KEY) != null) {
    +      Boolean createCache = Boolean.valueOf(properties.getProperty(JCACHE_AUTO_CREATE_CACHE_PROPERTY_KEY));
    +      if (createCache) {
    +        cache = manager.createCache(persistentClass.getSimpleName(),
    +                cacheConfig).unwrap(ICache.class);
    +      }
    +    } else {
    +      cache = manager.createCache(persistentClass.getSimpleName(),
    +              cacheConfig).unwrap(ICache.class);
    +    }
    +    LOG.info("JCache Gora datastore initialized successfully.");
    +  }
    +
    +  @Override
    +  public String getSchemaName() {
    +    return super.persistentClass.getSimpleName();
    +  }
    +
    +  @Override
    +  public void createSchema() {
    +    if (manager.getCache(super.getPersistentClass().getSimpleName(), keyClass, persistentClass) == null) {
    +      cacheEntryList.clear();
    +      cache = manager.createCache(persistentClass.getSimpleName(),
    +              cacheConfig).unwrap(ICache.class);
    +    }
    +    persistentDataStore.createSchema();
    +    LOG.info("Created schema on persistent store and initialized cache for persistent bean "
    +            + super.getPersistentClass().getSimpleName());
    +  }
    +
    +  @Override
    +  public void deleteSchema() {
    +    cacheEntryList.clear();
    +    manager.destroyCache(super.getPersistentClass().getSimpleName());
    +    persistentDataStore.deleteSchema();
    +    LOG.info("Deleted schema on persistent store and destroyed cache for persistent bean "
    +            + super.getPersistentClass().getSimpleName());
    +  }
    +
    +  @Override
    +  public boolean schemaExists() {
    +    return (manager.getCache(super.getPersistentClass().getSimpleName(), keyClass, persistentClass) != null);
    +  }
    +
    +  @Override
    +  public T get(K key, String[] fields) {
    +    T persitent = (T) cache.get(key);
    +    if (persitent == null) {
    +      return null;
    +    }
    +    return getPersistent(persitent, fields);
    +  }
    +
    +  @Override
    +  public T get(K key) {
    +    return cache.get(key);
    +  }
    +
    +  @Override
    +  public void put(K key, T val) {
    +    cache.put(key, val);
    +  }
    +
    +  @Override
    +  public boolean delete(K key) {
    +    return cache.remove(key);
    +  }
    +
    +  @Override
    +  public long deleteByQuery(Query<K, T> query) {
    +    try {
    +      long deletedRows = 0;
    +      Result<K, T> result = query.execute();
    +      String[] fields = getFieldsToQuery(query.getFields());
    +      boolean isAllFields = Arrays.equals(fields, getFields());
    +      while (result.next()) {
    +        if (isAllFields) {
    +          if (delete(result.getKey())) {
    +            deletedRows++;
    +          }
    +        } else {
    +          ArrayList<String> excludedFields = new ArrayList<>();
    +          for (String field : getFields()) {
    +            if (!Arrays.asList(fields).contains(field)) {
    +              excludedFields.add(field);
    +            }
    +          }
    +          T newClonedObj = getPersistent(result.get(),
    +                  excludedFields.toArray(new String[excludedFields.size()]));
    +          if (delete(result.getKey())) {
    +            put(result.getKey(), newClonedObj);
    +            deletedRows++;
    +          }
    +        }
    +      }
    +      LOG.info("JCache Gora datastore deleled " + deletedRows + " rows from Persistent datastore");
    --- End diff --
    
    Use parameterized logging
    http://www.slf4j.org/faq.html#logging_performance
    ```
     +      LOG.info("JCache Gora datastore deleled {} rows from Persistent datastore", deletedRows); 
    ```


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

[GitHub] gora issue #72: Gora 409

Posted by lewismc <gi...@git.apache.org>.
Github user lewismc commented on the issue:

    https://github.com/apache/gora/pull/72
  
    BTW, in order to resolve the above conflict just look into POM


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

[GitHub] gora pull request #72: Gora 409

Posted by lewismc <gi...@git.apache.org>.
Github user lewismc commented on a diff in the pull request:

    https://github.com/apache/gora/pull/72#discussion_r73616025
  
    --- Diff: gora-jcache/src/main/java/org/apache/gora/jcache/query/JCacheQuery.java ---
    @@ -0,0 +1,34 @@
    +/**
    + * 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.gora.jcache.query;
    +
    +import org.apache.gora.persistency.impl.PersistentBase;
    +import org.apache.gora.query.impl.QueryBase;
    +import org.apache.gora.store.DataStore;
    +
    +public class JCacheQuery<K, T extends PersistentBase> extends QueryBase<K, T> {
    +
    +  public JCacheQuery() {
    --- End diff --
    
    
    
    Please document this default constructor



---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

[GitHub] gora pull request #72: Gora 409

Posted by lewismc <gi...@git.apache.org>.
Github user lewismc commented on a diff in the pull request:

    https://github.com/apache/gora/pull/72#discussion_r73616206
  
    --- Diff: gora-jcache/src/main/java/org/apache/gora/jcache/store/JCacheCacheEntryListener.java ---
    @@ -0,0 +1,77 @@
    +/**
    + * 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.gora.jcache.store;
    +
    +import org.apache.gora.persistency.impl.PersistentBase;
    +import org.slf4j.Logger;
    +import org.slf4j.LoggerFactory;
    +
    +import javax.cache.event.CacheEntryExpiredListener;
    +import javax.cache.event.CacheEntryUpdatedListener;
    +import javax.cache.event.CacheEntryCreatedListener;
    +import javax.cache.event.CacheEntryRemovedListener;
    +import javax.cache.event.CacheEntryEvent;
    +import javax.cache.event.CacheEntryListenerException;
    +import java.util.concurrent.ConcurrentSkipListSet;
    +
    +public class JCacheCacheEntryListener<K, T extends PersistentBase>
    +        implements CacheEntryCreatedListener<K, T>,
    +        CacheEntryRemovedListener<K, T>, CacheEntryUpdatedListener<K, T>, CacheEntryExpiredListener<K, T> {
    +
    +  private static final Logger LOG = LoggerFactory.getLogger(JCacheCacheEntryListener.class);
    +  private ConcurrentSkipListSet<K> cacheEntryList;
    +
    +  public JCacheCacheEntryListener(ConcurrentSkipListSet cacheEntryList) {
    +    this.cacheEntryList = cacheEntryList;
    +  }
    +
    +  @Override
    +  public void onCreated(Iterable<CacheEntryEvent<? extends K, ? extends T>> cacheEntryEvents)
    --- End diff --
    
    Add documentation


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

[GitHub] gora pull request #72: Gora 409

Posted by lewismc <gi...@git.apache.org>.
Github user lewismc commented on a diff in the pull request:

    https://github.com/apache/gora/pull/72#discussion_r73615743
  
    --- Diff: gora-core/src/main/java/org/apache/gora/persistency/impl/PersistentBase.java ---
    @@ -181,10 +181,14 @@ public void setDirty(String field) {
         setDirty(getSchema().getField(field).pos());
       }
     
    -  private ByteBuffer getDirtyBytes() {
    +  public ByteBuffer getDirtyBytes() {
         return __g__dirty;
       }
     
    +  public void setDirtyBytes(ByteBuffer __g__dirty) {
    --- End diff --
    
    
    
    Can you document this please.



---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

[GitHub] gora issue #72: Gora 409

Posted by djkevincr <gi...@git.apache.org>.
Github user djkevincr commented on the issue:

    https://github.com/apache/gora/pull/72
  
    @lewismc  I didn't close this pr. Did I do something mistakenly? It says this pr successfully merged and closed I do have unit tests for map reduce and several improvements to previous ones. Shall I send another pr? 


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

[GitHub] gora pull request #72: Gora 409

Posted by lewismc <gi...@git.apache.org>.
Github user lewismc commented on a diff in the pull request:

    https://github.com/apache/gora/pull/72#discussion_r73615796
  
    --- Diff: gora-core/src/main/java/org/apache/gora/store/DataStoreFactory.java ---
    @@ -274,6 +276,39 @@ D getDataStore( Class<D> dataStoreClass, Class<K> keyClass,
         return createDataStore(c, keyClass, persistent, conf, createProps, null);
       }
     
    +
    +  /**
    +   * Instantiate <i>the default</i> {@link DataStore} wrapped over JCache datastore which provides caching
    +   * abstraction over any GORA persistence dataStore.
    +   * Uses default properties. Uses 'null' schema.
    +   *
    +   * Note:
    +   *    consider that default dataStore is always visible
    +   *
    +   * @param keyClass The key class.
    +   * @param persistent The value class.
    +   * @param conf {@link Configuration} to be used be the store.
    --- End diff --
    
    Please use the package name for Configuration or else Javadoc may not work.


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

[GitHub] gora issue #72: Gora 409

Posted by djkevincr <gi...@git.apache.org>.
Github user djkevincr commented on the issue:

    https://github.com/apache/gora/pull/72
  
    I will work on the gora-tutorial as well as some documentation next. Thank you.
    @lewismc Seems like recent PR is not merged yet. Please find it under [1]. 
    [1] https://github.com/apache/gora/pull/73


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

[GitHub] gora pull request #72: Gora 409

Posted by lewismc <gi...@git.apache.org>.
Github user lewismc commented on a diff in the pull request:

    https://github.com/apache/gora/pull/72#discussion_r73616520
  
    --- Diff: gora-jcache/src/main/java/org/apache/gora/jcache/store/JCacheCacheWriter.java ---
    @@ -0,0 +1,72 @@
    +/**
    + * 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.gora.jcache.store;
    +
    +import org.apache.gora.persistency.impl.PersistentBase;
    +import org.apache.gora.store.DataStore;
    +import org.slf4j.Logger;
    +import org.slf4j.LoggerFactory;
    +
    +import javax.cache.Cache;
    +import javax.cache.integration.CacheWriter;
    +import javax.cache.integration.CacheWriterException;
    +import java.util.Collection;
    +import java.util.Iterator;
    +
    +public class JCacheCacheWriter<K, T extends PersistentBase> implements CacheWriter<K, T> {
    +
    +  private static final Logger LOG = LoggerFactory.getLogger(JCacheCacheWriter.class);
    +  private DataStore<K, T> dataStore;
    +
    +  public JCacheCacheWriter(DataStore<K, T> dataStore) {
    --- End diff --
    
    Add documentation


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

[GitHub] gora pull request #72: Gora 409

Posted by lewismc <gi...@git.apache.org>.
Github user lewismc commented on a diff in the pull request:

    https://github.com/apache/gora/pull/72#discussion_r73616037
  
    --- Diff: gora-jcache/src/main/java/org/apache/gora/jcache/query/JCacheQuery.java ---
    @@ -0,0 +1,34 @@
    +/**
    + * 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.gora.jcache.query;
    +
    +import org.apache.gora.persistency.impl.PersistentBase;
    +import org.apache.gora.query.impl.QueryBase;
    +import org.apache.gora.store.DataStore;
    +
    +public class JCacheQuery<K, T extends PersistentBase> extends QueryBase<K, T> {
    +
    +  public JCacheQuery() {
    +    super(null);
    +  }
    +
    +  public JCacheQuery(DataStore<K, T> dataStore) {
    --- End diff --
    
    
    
    Please document this constructor



---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

[GitHub] gora pull request #72: Gora 409

Posted by lewismc <gi...@git.apache.org>.
Github user lewismc commented on a diff in the pull request:

    https://github.com/apache/gora/pull/72#discussion_r73616308
  
    --- Diff: gora-jcache/src/main/java/org/apache/gora/jcache/store/JCacheCacheEntryListenerFactory.java ---
    @@ -0,0 +1,56 @@
    +/**
    + * 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.gora.jcache.store;
    +
    +import org.apache.gora.persistency.impl.PersistentBase;
    +import org.slf4j.Logger;
    +import org.slf4j.LoggerFactory;
    +
    +import javax.cache.configuration.Factory;
    +
    +public class JCacheCacheEntryListenerFactory <K,T extends PersistentBase>
    +        implements Factory<JCacheCacheEntryListener<K, T>> {
    +
    +  private static final Logger LOG = LoggerFactory.getLogger(JCacheCacheEntryListenerFactory.class);
    +  public static final long serialVersionUID = 201305101634L;
    +  private transient JCacheCacheEntryListener<K, T> instance;
    +
    +  public JCacheCacheEntryListenerFactory(JCacheCacheEntryListener<K, T> instance) {
    +    LOG.info("JCache cache entry listener factory initialized successfully.");
    +    this.instance = instance;
    +  }
    +
    +  public JCacheCacheEntryListener<K, T> create() {
    +    return this.instance;
    +  }
    +
    +  public boolean equals(Object other) {
    --- End diff --
    
    Add documentation


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

[GitHub] gora pull request #72: Gora 409

Posted by lewismc <gi...@git.apache.org>.
Github user lewismc commented on a diff in the pull request:

    https://github.com/apache/gora/pull/72#discussion_r73616657
  
    --- Diff: gora-jcache/src/main/java/org/apache/gora/jcache/store/JCacheCacheWriterFactory.java ---
    @@ -0,0 +1,55 @@
    +/**
    + * 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.gora.jcache.store;
    +
    +import org.apache.gora.persistency.impl.PersistentBase;
    +import org.slf4j.Logger;
    +import org.slf4j.LoggerFactory;
    +
    +import javax.cache.configuration.Factory;
    +
    +public class JCacheCacheWriterFactory<K, T extends PersistentBase> implements Factory<JCacheCacheWriter<K,T>> {
    +
    +  private static final Logger LOG = LoggerFactory.getLogger(JCacheCacheWriterFactory.class);
    +  public static final long serialVersionUID = 201205101621L;
    +  private transient JCacheCacheWriter<K,T> instance;
    +
    +  public JCacheCacheWriterFactory(JCacheCacheWriter<K,T> instance) {
    --- End diff --
    
    Add documentation


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

[GitHub] gora pull request #72: Gora 409

Posted by lewismc <gi...@git.apache.org>.
Github user lewismc commented on a diff in the pull request:

    https://github.com/apache/gora/pull/72#discussion_r73616317
  
    --- Diff: gora-jcache/src/main/java/org/apache/gora/jcache/store/JCacheCacheEntryListenerFactory.java ---
    @@ -0,0 +1,56 @@
    +/**
    + * 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.gora.jcache.store;
    +
    +import org.apache.gora.persistency.impl.PersistentBase;
    +import org.slf4j.Logger;
    +import org.slf4j.LoggerFactory;
    +
    +import javax.cache.configuration.Factory;
    +
    +public class JCacheCacheEntryListenerFactory <K,T extends PersistentBase>
    +        implements Factory<JCacheCacheEntryListener<K, T>> {
    +
    +  private static final Logger LOG = LoggerFactory.getLogger(JCacheCacheEntryListenerFactory.class);
    +  public static final long serialVersionUID = 201305101634L;
    +  private transient JCacheCacheEntryListener<K, T> instance;
    +
    +  public JCacheCacheEntryListenerFactory(JCacheCacheEntryListener<K, T> instance) {
    +    LOG.info("JCache cache entry listener factory initialized successfully.");
    +    this.instance = instance;
    +  }
    +
    +  public JCacheCacheEntryListener<K, T> create() {
    +    return this.instance;
    +  }
    +
    +  public boolean equals(Object other) {
    +    if(this == other) {
    +      return true;
    +    } else if(other != null && this.getClass() == other.getClass()) {
    +      JCacheCacheEntryListenerFactory that = (JCacheCacheEntryListenerFactory)other;
    +      return this.instance.equals(that.instance);
    +    } else {
    +      return false;
    +    }
    +  }
    +
    +  public int hashCode() {
    --- End diff --
    
    Add documentation


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

[GitHub] gora issue #72: Gora 409

Posted by lewismc <gi...@git.apache.org>.
Github user lewismc commented on the issue:

    https://github.com/apache/gora/pull/72
  
    Brilliant thanks. This has also been merged.
    Please submit a PR for adding the JCache implementation to gora-tutorial. Thanks


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

[GitHub] gora pull request #72: Gora 409

Posted by lewismc <gi...@git.apache.org>.
Github user lewismc commented on a diff in the pull request:

    https://github.com/apache/gora/pull/72#discussion_r73615713
  
    --- Diff: gora-core/src/main/java/org/apache/gora/persistency/impl/DirtySetWrapper.java ---
    @@ -1,3 +1,21 @@
    +/**
    --- End diff --
    
    
    
    Thank you for adding license header here.



---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

[GitHub] gora pull request #72: Gora 409

Posted by lewismc <gi...@git.apache.org>.
Github user lewismc commented on a diff in the pull request:

    https://github.com/apache/gora/pull/72#discussion_r73616328
  
    --- Diff: gora-jcache/src/main/java/org/apache/gora/jcache/store/JCacheCacheFactoryBuilder.java ---
    @@ -0,0 +1,41 @@
    +/**
    + * 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.gora.jcache.store;
    +
    +import org.apache.gora.persistency.impl.PersistentBase;
    +import org.apache.gora.store.DataStore;
    +import javax.cache.configuration.Factory;
    +
    +public class JCacheCacheFactoryBuilder {
    --- End diff --
    
    Add documentation


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

[GitHub] gora pull request #72: Gora 409

Posted by lewismc <gi...@git.apache.org>.
Github user lewismc commented on a diff in the pull request:

    https://github.com/apache/gora/pull/72#discussion_r73616171
  
    --- Diff: gora-jcache/src/main/java/org/apache/gora/jcache/store/JCacheCacheEntryListener.java ---
    @@ -0,0 +1,77 @@
    +/**
    + * 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.gora.jcache.store;
    +
    +import org.apache.gora.persistency.impl.PersistentBase;
    +import org.slf4j.Logger;
    +import org.slf4j.LoggerFactory;
    +
    +import javax.cache.event.CacheEntryExpiredListener;
    +import javax.cache.event.CacheEntryUpdatedListener;
    +import javax.cache.event.CacheEntryCreatedListener;
    +import javax.cache.event.CacheEntryRemovedListener;
    +import javax.cache.event.CacheEntryEvent;
    +import javax.cache.event.CacheEntryListenerException;
    +import java.util.concurrent.ConcurrentSkipListSet;
    +
    +public class JCacheCacheEntryListener<K, T extends PersistentBase>
    --- End diff --
    
    Add documentation


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

[GitHub] gora pull request #72: Gora 409

Posted by lewismc <gi...@git.apache.org>.
Github user lewismc commented on a diff in the pull request:

    https://github.com/apache/gora/pull/72#discussion_r73616550
  
    --- Diff: gora-jcache/src/main/java/org/apache/gora/jcache/store/JCacheCacheWriter.java ---
    @@ -0,0 +1,72 @@
    +/**
    + * 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.gora.jcache.store;
    +
    +import org.apache.gora.persistency.impl.PersistentBase;
    +import org.apache.gora.store.DataStore;
    +import org.slf4j.Logger;
    +import org.slf4j.LoggerFactory;
    +
    +import javax.cache.Cache;
    +import javax.cache.integration.CacheWriter;
    +import javax.cache.integration.CacheWriterException;
    +import java.util.Collection;
    +import java.util.Iterator;
    +
    +public class JCacheCacheWriter<K, T extends PersistentBase> implements CacheWriter<K, T> {
    +
    +  private static final Logger LOG = LoggerFactory.getLogger(JCacheCacheWriter.class);
    +  private DataStore<K, T> dataStore;
    +
    +  public JCacheCacheWriter(DataStore<K, T> dataStore) {
    +    this.dataStore = dataStore;
    +  }
    +
    +  @Override
    +  public void write(Cache.Entry<? extends K,
    +          ? extends T> entry) throws CacheWriterException {
    +    dataStore.put(entry.getKey(), entry.getValue());
    +    LOG.info("Written data bean to persistent datastore on key " + entry.getKey().toString());
    +  }
    +
    +  @Override
    +  public void writeAll(Collection<Cache.Entry<? extends K,
    +          ? extends T>> entries) throws CacheWriterException {
    +    Iterator<Cache.Entry<? extends K, ? extends T>> iterator = entries.iterator();
    +    while (iterator.hasNext()) {
    +      write(iterator.next());
    +      iterator.remove();
    +    }
    +  }
    +
    +  @Override
    +  public void delete(Object key) throws CacheWriterException {
    +    dataStore.delete((K) key);
    +    LOG.info("Deleted data bean from persistent datastore on key " + key.toString());
    +  }
    +
    +  @Override
    +  public void deleteAll(Collection<?> keys) throws CacheWriterException {
    --- End diff --
    
    Add documentation


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

[GitHub] gora pull request #72: Gora 409

Posted by asfgit <gi...@git.apache.org>.
Github user asfgit closed the pull request at:

    https://github.com/apache/gora/pull/72


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

[GitHub] gora pull request #72: Gora 409

Posted by lewismc <gi...@git.apache.org>.
Github user lewismc commented on a diff in the pull request:

    https://github.com/apache/gora/pull/72#discussion_r73616677
  
    --- Diff: gora-jcache/src/main/java/org/apache/gora/jcache/store/JCacheCacheWriterFactory.java ---
    @@ -0,0 +1,55 @@
    +/**
    + * 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.gora.jcache.store;
    +
    +import org.apache.gora.persistency.impl.PersistentBase;
    +import org.slf4j.Logger;
    +import org.slf4j.LoggerFactory;
    +
    +import javax.cache.configuration.Factory;
    +
    +public class JCacheCacheWriterFactory<K, T extends PersistentBase> implements Factory<JCacheCacheWriter<K,T>> {
    +
    +  private static final Logger LOG = LoggerFactory.getLogger(JCacheCacheWriterFactory.class);
    +  public static final long serialVersionUID = 201205101621L;
    +  private transient JCacheCacheWriter<K,T> instance;
    +
    +  public JCacheCacheWriterFactory(JCacheCacheWriter<K,T> instance) {
    +    LOG.info("JCache cache writer factory initialized successfully.");
    +    this.instance = instance;
    +  }
    +
    +  public JCacheCacheWriter<K,T> create() {
    +    return (JCacheCacheWriter<K,T>)this.instance;
    +  }
    +
    +  public boolean equals(Object other) {
    --- End diff --
    
    Add documentation


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

[GitHub] gora issue #72: Gora 409

Posted by lewismc <gi...@git.apache.org>.
Github user lewismc commented on the issue:

    https://github.com/apache/gora/pull/72
  
    @djkevincr thats fine. The build is not stable. I merged this based upon your updates. Thanks for them.



---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

[GitHub] gora pull request #72: Gora 409

Posted by lewismc <gi...@git.apache.org>.
Github user lewismc commented on a diff in the pull request:

    https://github.com/apache/gora/pull/72#discussion_r73615603
  
    --- Diff: gora-compiler/src/main/velocity/org/apache/gora/compiler/templates/record.vm ---
    @@ -353,6 +353,33 @@ public class ${this.mangle($schema.getName())}#if ($schema.isError()) extends or
     	#end
       
       }
    +
    +  private static final org.apache.avro.io.DatumWriter
    +          DATUM_WRITER$ = new org.apache.avro.specific.SpecificDatumWriter(SCHEMA$);
    +  private static final org.apache.avro.io.DatumReader
    +          DATUM_READER$ = new org.apache.avro.specific.SpecificDatumReader(SCHEMA$);
    +
    +  @Override
    +  public void writeExternal(java.io.ObjectOutput out)
    --- End diff --
    
    Possible to document this method template?


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

[GitHub] gora pull request #72: Gora 409

Posted by lewismc <gi...@git.apache.org>.
Github user lewismc commented on a diff in the pull request:

    https://github.com/apache/gora/pull/72#discussion_r73615503
  
    --- Diff: gora-compiler/src/main/java/org/apache/gora/compiler/GoraCompiler.java ---
    @@ -273,4 +274,8 @@ private static Schema getRecordSchemaWithDirtySupport(Schema originalSchema, Map
         return newSchema;
       }
     
    +  public static long fingerprint64(Schema schema) {
    --- End diff --
    
    Can you please document this method


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

[GitHub] gora pull request #72: Gora 409

Posted by lewismc <gi...@git.apache.org>.
Github user lewismc commented on a diff in the pull request:

    https://github.com/apache/gora/pull/72#discussion_r73615862
  
    --- Diff: gora-core/src/main/java/org/apache/gora/store/DataStoreFactory.java ---
    @@ -381,6 +416,10 @@ private static String getDefaultDataStore(Properties properties) {
         return getProperty(properties, GORA_DEFAULT_DATASTORE_KEY);
       }
     
    +  private static String getDefaultCacheDataStore(Properties properties) {
    --- End diff --
    
    Add Javadoc here please.


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

[GitHub] gora pull request #72: Gora 409

Posted by lewismc <gi...@git.apache.org>.
Github user lewismc commented on a diff in the pull request:

    https://github.com/apache/gora/pull/72#discussion_r73615830
  
    --- Diff: gora-core/src/main/java/org/apache/gora/store/DataStoreFactory.java ---
    @@ -274,6 +276,39 @@ D getDataStore( Class<D> dataStoreClass, Class<K> keyClass,
         return createDataStore(c, keyClass, persistent, conf, createProps, null);
       }
     
    +
    +  /**
    +   * Instantiate <i>the default</i> {@link DataStore} wrapped over JCache datastore which provides caching
    +   * abstraction over any GORA persistence dataStore.
    +   * Uses default properties. Uses 'null' schema.
    +   *
    +   * Note:
    +   *    consider that default dataStore is always visible
    +   *
    +   * @param keyClass The key class.
    +   * @param persistent The value class.
    +   * @param conf {@link Configuration} to be used be the store.
    +   * @param isCacheEnabled caching enable
    +   * @return A new store instance.
    +   * @throws GoraException
    --- End diff --
    
    If what happens? Why would it throw a GoraException?


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

[GitHub] gora pull request #72: Gora 409

Posted by lewismc <gi...@git.apache.org>.
Github user lewismc commented on a diff in the pull request:

    https://github.com/apache/gora/pull/72#discussion_r73616514
  
    --- Diff: gora-jcache/src/main/java/org/apache/gora/jcache/store/JCacheCacheWriter.java ---
    @@ -0,0 +1,72 @@
    +/**
    + * 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.gora.jcache.store;
    +
    +import org.apache.gora.persistency.impl.PersistentBase;
    +import org.apache.gora.store.DataStore;
    +import org.slf4j.Logger;
    +import org.slf4j.LoggerFactory;
    +
    +import javax.cache.Cache;
    +import javax.cache.integration.CacheWriter;
    +import javax.cache.integration.CacheWriterException;
    +import java.util.Collection;
    +import java.util.Iterator;
    +
    +public class JCacheCacheWriter<K, T extends PersistentBase> implements CacheWriter<K, T> {
    --- End diff --
    
    Add documentation


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

[GitHub] gora pull request #72: Gora 409

Posted by lewismc <gi...@git.apache.org>.
Github user lewismc commented on a diff in the pull request:

    https://github.com/apache/gora/pull/72#discussion_r73615693
  
    --- Diff: gora-core/src/main/java/org/apache/gora/persistency/impl/DirtyListWrapper.java ---
    @@ -1,3 +1,21 @@
    +/**
    --- End diff --
    
    
    
    Thank you for adding license header here.



---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

[GitHub] gora pull request #72: Gora 409

Posted by lewismc <gi...@git.apache.org>.
Github user lewismc commented on a diff in the pull request:

    https://github.com/apache/gora/pull/72#discussion_r73616145
  
    --- Diff: gora-jcache/src/main/java/org/apache/gora/jcache/query/JCacheResult.java ---
    @@ -0,0 +1,69 @@
    +/**
    + * 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.gora.jcache.query;
    +
    +import java.io.IOException;
    +import java.util.Iterator;
    +import java.util.NavigableSet;
    +
    +import org.apache.gora.jcache.store.JCacheStore;
    +import org.apache.gora.persistency.impl.PersistentBase;
    +import org.apache.gora.query.Query;
    +import org.apache.gora.query.impl.ResultBase;
    +import org.apache.gora.store.DataStore;
    +
    +public class JCacheResult<K, T extends PersistentBase> extends ResultBase<K, T> {
    +
    +  private NavigableSet<K> cacheKeySet;
    +  private Iterator<K> iterator;
    +
    +  public JCacheResult(DataStore<K, T> dataStore, Query<K, T> query) {
    +    super(dataStore, query);
    +  }
    +
    +  public JCacheResult(DataStore<K, T> dataStore, Query<K, T> query, NavigableSet<K> cacheKeySet) {
    +    super(dataStore, query);
    +    this.cacheKeySet = cacheKeySet;
    +    this.iterator = cacheKeySet.iterator();
    +  }
    +
    +  public JCacheStore<K, T> getDataStore() {
    +    return (JCacheStore<K, T>) super.getDataStore();
    +  }
    +
    +  @Override
    +  public float getProgress() throws IOException {
    +    return 0;
    +  }
    +
    +  @Override
    +  public void close() throws IOException {
    --- End diff --
    
    Add documentation


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

[GitHub] gora pull request #72: Gora 409

Posted by lewismc <gi...@git.apache.org>.
Github user lewismc commented on a diff in the pull request:

    https://github.com/apache/gora/pull/72#discussion_r73616454
  
    --- Diff: gora-jcache/src/main/java/org/apache/gora/jcache/store/JCacheCacheLoader.java ---
    @@ -0,0 +1,65 @@
    +/**
    + * 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.gora.jcache.store;
    +
    +import org.apache.gora.persistency.impl.PersistentBase;
    +import org.apache.gora.store.DataStore;
    +import org.slf4j.Logger;
    +import org.slf4j.LoggerFactory;
    +
    +import javax.cache.integration.CacheLoader;
    +import javax.cache.integration.CacheLoaderException;
    +import java.util.HashMap;
    +import java.util.Map;
    +
    +
    +public class JCacheCacheLoader<K, T extends PersistentBase> implements CacheLoader<K, T> {
    +
    +  private static final Logger LOG = LoggerFactory.getLogger(JCacheCacheLoader.class);
    +  private DataStore<K, T> dataStore;
    +
    +  public JCacheCacheLoader(DataStore<K, T> dataStore) {
    --- End diff --
    
    Add documentation


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

[GitHub] gora pull request #72: Gora 409

Posted by lewismc <gi...@git.apache.org>.
Github user lewismc commented on a diff in the pull request:

    https://github.com/apache/gora/pull/72#discussion_r73615608
  
    --- Diff: gora-compiler/src/main/velocity/org/apache/gora/compiler/templates/record.vm ---
    @@ -353,6 +353,33 @@ public class ${this.mangle($schema.getName())}#if ($schema.isError()) extends or
     	#end
       
       }
    +
    +  private static final org.apache.avro.io.DatumWriter
    +          DATUM_WRITER$ = new org.apache.avro.specific.SpecificDatumWriter(SCHEMA$);
    +  private static final org.apache.avro.io.DatumReader
    +          DATUM_READER$ = new org.apache.avro.specific.SpecificDatumReader(SCHEMA$);
    +
    +  @Override
    +  public void writeExternal(java.io.ObjectOutput out)
    +          throws java.io.IOException {
    +    out.write(super.getDirtyBytes().array());
    +    DATUM_WRITER$.write
    +            (this, org.apache.avro.io.EncoderFactory.get()
    +                    .directBinaryEncoder((java.io.OutputStream) out,
    +                            null));
    +  }
    +
    +  @Override
    +  public void readExternal(java.io.ObjectInput in)
    --- End diff --
    
    Possible to document this method template?


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

[GitHub] gora pull request #72: Gora 409

Posted by lewismc <gi...@git.apache.org>.
Github user lewismc commented on a diff in the pull request:

    https://github.com/apache/gora/pull/72#discussion_r73615677
  
    --- Diff: gora-core/src/main/java/org/apache/gora/persistency/impl/DirtyIteratorWrapper.java ---
    @@ -1,3 +1,21 @@
    +/**
    --- End diff --
    
    
    
    Thank you for adding license header here.



---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

[GitHub] gora pull request #72: Gora 409

Posted by lewismc <gi...@git.apache.org>.
Github user lewismc commented on a diff in the pull request:

    https://github.com/apache/gora/pull/72#discussion_r73616444
  
    --- Diff: gora-jcache/src/main/java/org/apache/gora/jcache/store/JCacheCacheLoader.java ---
    @@ -0,0 +1,65 @@
    +/**
    + * 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.gora.jcache.store;
    +
    +import org.apache.gora.persistency.impl.PersistentBase;
    +import org.apache.gora.store.DataStore;
    +import org.slf4j.Logger;
    +import org.slf4j.LoggerFactory;
    +
    +import javax.cache.integration.CacheLoader;
    +import javax.cache.integration.CacheLoaderException;
    +import java.util.HashMap;
    +import java.util.Map;
    +
    +
    +public class JCacheCacheLoader<K, T extends PersistentBase> implements CacheLoader<K, T> {
    --- End diff --
    
    Add documentation


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

[GitHub] gora pull request #72: Gora 409

Posted by lewismc <gi...@git.apache.org>.
Github user lewismc commented on a diff in the pull request:

    https://github.com/apache/gora/pull/72#discussion_r73617092
  
    --- Diff: gora-jcache/src/test/java/org/apache/gora/jcache/store/JCacheGoraDataStoreTest.java ---
    @@ -0,0 +1,194 @@
    +/**
    + * 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.gora.jcache.store;
    +
    +import org.apache.gora.examples.WebPageDataCreator;
    +import org.apache.gora.examples.generated.Employee;
    +import org.apache.gora.examples.generated.WebPage;
    +import org.apache.gora.persistency.BeanFactory;
    +import org.apache.gora.persistency.impl.BeanFactoryImpl;
    +import org.apache.gora.query.Query;
    +import org.apache.gora.store.DataStore;
    +import org.apache.gora.store.DataStoreFactory;
    +import org.apache.gora.store.DataStoreTestBase;
    +import org.apache.gora.store.DataStoreTestUtil;
    +import org.apache.hadoop.conf.Configuration;
    +import org.junit.After;
    +import org.junit.Before;
    +import org.junit.Test;
    +import org.slf4j.Logger;
    +import org.slf4j.LoggerFactory;
    +
    +import java.io.IOException;
    +
    +import static org.apache.gora.examples.WebPageDataCreator.SORTED_URLS;
    +import static org.apache.gora.examples.WebPageDataCreator.URLS;
    +import static org.junit.Assert.assertNotNull;
    +import static org.junit.Assert.assertEquals;
    +import static org.junit.Assert.assertNull;
    +import static org.junit.Assert.assertTrue;
    +import static org.junit.Assume.assumeTrue;
    +
    +public class JCacheGoraDataStoreTest extends DataStoreTestBase {
    +
    +  private static final Logger LOG = LoggerFactory.getLogger(JCacheGoraDataStoreTest.class);
    +  private static final int NUM_KEYS = 4;
    +  private Configuration conf = new Configuration();
    +
    +  @Before
    +  public void setUp() throws Exception {
    +    super.setUp();
    +  }
    +
    +  @After
    +  public void tearDown() throws Exception {
    +    super.tearDown();
    +    //mandatory to clean up hazelcast instances
    --- End diff --
    
    Thank you for putting comments in here.


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

[GitHub] gora pull request #72: Gora 409

Posted by lewismc <gi...@git.apache.org>.
Github user lewismc commented on a diff in the pull request:

    https://github.com/apache/gora/pull/72#discussion_r73616479
  
    --- Diff: gora-jcache/src/main/java/org/apache/gora/jcache/store/JCacheCacheLoaderFactory.java ---
    @@ -0,0 +1,56 @@
    +/**
    + * 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.gora.jcache.store;
    +
    +import org.apache.gora.persistency.impl.PersistentBase;
    +import org.slf4j.Logger;
    +import org.slf4j.LoggerFactory;
    +
    +import javax.cache.configuration.Factory;
    +
    +public class JCacheCacheLoaderFactory<K, T extends PersistentBase>
    --- End diff --
    
    Add documentation


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

[GitHub] gora pull request #72: Gora 409

Posted by lewismc <gi...@git.apache.org>.
Github user lewismc commented on a diff in the pull request:

    https://github.com/apache/gora/pull/72#discussion_r73615651
  
    --- Diff: gora-core/src/main/java/org/apache/gora/persistency/impl/DirtyCollectionWrapper.java ---
    @@ -1,3 +1,21 @@
    +/**
    --- End diff --
    
    Thank you for adding license header here.


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

[GitHub] gora pull request #72: Gora 409

Posted by lewismc <gi...@git.apache.org>.
Github user lewismc commented on a diff in the pull request:

    https://github.com/apache/gora/pull/72#discussion_r73617056
  
    --- Diff: gora-jcache/src/main/java/org/apache/gora/jcache/store/JCacheStore.java ---
    @@ -0,0 +1,412 @@
    +/**
    + * 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.gora.jcache.store;
    +
    +import java.io.IOException;
    +import java.net.URI;
    +import java.net.URISyntaxException;
    +import java.util.List;
    +import java.util.Arrays;
    +import java.util.ArrayList;
    +import java.util.Properties;
    +import java.util.concurrent.ConcurrentSkipListSet;
    +import java.util.concurrent.TimeUnit;
    +
    +import com.hazelcast.cache.HazelcastCachingProvider;
    +import com.hazelcast.cache.ICache;
    +import com.hazelcast.config.CacheConfig;
    +import com.hazelcast.config.EvictionConfig;
    +import com.hazelcast.config.EvictionPolicy;
    +import com.hazelcast.config.InMemoryFormat;
    +import com.hazelcast.core.Hazelcast;
    +import com.hazelcast.core.HazelcastInstance;
    +import com.hazelcast.core.Member;
    +import com.hazelcast.core.Partition;
    +import org.apache.avro.Schema;
    +import org.apache.gora.jcache.query.JCacheQuery;
    +import org.apache.gora.jcache.query.JCacheResult;
    +import org.apache.gora.persistency.impl.PersistentBase;
    +import org.apache.gora.query.PartitionQuery;
    +import org.apache.gora.query.Query;
    +import org.apache.gora.query.Result;
    +import org.apache.gora.query.impl.PartitionQueryImpl;
    +import org.apache.gora.store.DataStore;
    +import org.apache.gora.store.DataStoreFactory;
    +import org.apache.gora.store.impl.DataStoreBase;
    +import org.apache.gora.util.AvroUtils;
    +import org.apache.gora.util.GoraException;
    +import org.apache.hadoop.conf.Configuration;
    +import org.slf4j.Logger;
    +import org.slf4j.LoggerFactory;
    +
    +import javax.cache.CacheManager;
    +import javax.cache.Caching;
    +import javax.cache.configuration.FactoryBuilder;
    +import javax.cache.configuration.MutableCacheEntryListenerConfiguration;
    +import javax.cache.expiry.AccessedExpiryPolicy;
    +import javax.cache.expiry.ModifiedExpiryPolicy;
    +import javax.cache.expiry.CreatedExpiryPolicy;
    +import javax.cache.expiry.TouchedExpiryPolicy;
    +import javax.cache.expiry.Duration;
    +import javax.cache.spi.CachingProvider;
    +
    +public class JCacheStore<K, T extends PersistentBase> extends DataStoreBase<K, T> {
    +
    +  private static final String GORA_DEFAULT_JCACHE_NAMESPACE = "gora.jcache.namespace";
    +  private static final String GORA_DEFAULT_JCACHE_PROVIDER_KEY = "gora.datastore.jcache.provider";
    +  private static final String JCACHE_READ_THROUGH_PROPERTY_KEY = "jcache.read.through.enable";
    +  private static final String JCACHE_WRITE_THROUGH_PROPERTY_KEY = "jcache.write.through.enable";
    +  private static final String JCACHE_STORE_BY_VALUE_PROPERTY_KEY = "jcache.store.by.value.enable";
    +  private static final String JCACHE_STATISTICS_PROPERTY_KEY = "jcache.statistics.enable";
    +  private static final String JCACHE_MANAGEMENT_PROPERTY_KEY = "jcache.management.enable";
    +  private static final String JCACHE_CACHE_NAMESPACE_PROPERTY_KEY = "jcache.cache.namespace";
    +  private static final String JCACHE_EVICTION_POLICY_PROPERTY_KEY = "jcache.eviction.policy";
    +  private static final String JCACHE_EVICTION_MAX_SIZE_POLICY_PROPERTY_KEY = "jcache.eviction.max.size.policy";
    +  private static final String JCACHE_EVICTION_SIZE_PROPERTY_KEY = "jcache.eviction.size";
    +  private static final String JCACHE_EXPIRE_POLICY_PROPERTY_KEY = "jcache.expire.policy";
    +  private static final String JCACHE_EXPIRE_POLICY_DURATION_PROPERTY_KEY = "jcache.expire.policy";
    +  private static final String JCACHE_ACCESSED_EXPIRY_IDENTIFIER = "ACCESSED";
    +  private static final String JCACHE_CREATED_EXPIRY_IDENTIFIER = "CREATED";
    +  private static final String JCACHE_MODIFIED_EXPIRY_IDENTIFIER = "MODIFIED";
    +  private static final String JCACHE_TOUCHED_EXPIRY_IDENTIFIER = "TOUCHED";
    +  private static final String HAZELCAST_CACHE_IN_MEMORY_FORMAT_PROPERTY_KEY = "jcache.cache.inmemory.format";
    +  private static final String HAZELCAST_CACHE_BINARY_IN_MEMORY_FORMAT_IDENTIFIER = "BINARY";
    +  private static final String HAZELCAST_CACHE_OBJECT_IN_MEMORY_FORMAT_IDENTIFIER = "OBJECT";
    +  private static final String HAZELCAST_CACHE_NATIVE_IN_MEMORY_FORMAT_IDENTIFIER = "NATIVE";
    +  private static final String JCACHE_AUTO_CREATE_CACHE_PROPERTY_KEY = "jcache.auto.create.cache";
    +  private static final Logger LOG = LoggerFactory.getLogger(JCacheStore.class);
    +  private ICache<K, T> cache;
    +  private CacheManager manager;
    +  private ConcurrentSkipListSet<K> cacheEntryList;
    +  private String goraCacheNamespace = GORA_DEFAULT_JCACHE_NAMESPACE;
    +  private DataStore<K, T> persistentDataStore;
    +  private CacheConfig<K, T> cacheConfig;
    +  private HazelcastInstance hazelcastInstance;
    +
    +  private static <T extends PersistentBase> T getPersistent(T persitent, String[] fields) {
    +    List<Schema.Field> otherFields = persitent.getSchema().getFields();
    +    String[] otherFieldStrings = new String[otherFields.size()];
    +    for (int i = 0; i < otherFields.size(); i++) {
    +      otherFieldStrings[i] = otherFields.get(i).name();
    +    }
    +    if (Arrays.equals(fields, otherFieldStrings)) {
    +      return persitent;
    +    }
    +    T clonedPersistent = AvroUtils.deepClonePersistent(persitent);
    +    clonedPersistent.clear();
    +    if (fields != null && fields.length > 0) {
    +      for (String field : fields) {
    +        Schema.Field otherField = persitent.getSchema().getField(field);
    +        int index = otherField.pos();
    +        clonedPersistent.put(index, persitent.get(index));
    +      }
    +    } else {
    +      for (String field : otherFieldStrings) {
    +        Schema.Field otherField = persitent.getSchema().getField(field);
    +        int index = otherField.pos();
    +        clonedPersistent.put(index, persitent.get(index));
    +      }
    +    }
    +    return clonedPersistent;
    +  }
    +
    +  @Override
    +  public void initialize(Class<K> keyClass, Class<T> persistentClass, Properties properties) {
    +    super.initialize(keyClass, persistentClass, properties);
    +    CachingProvider cachingProvider = Caching.getCachingProvider(
    +            properties.getProperty(GORA_DEFAULT_JCACHE_PROVIDER_KEY)
    +    );
    +    if (properties.getProperty(JCACHE_CACHE_NAMESPACE_PROPERTY_KEY) != null) {
    +      goraCacheNamespace = properties.getProperty(JCACHE_CACHE_NAMESPACE_PROPERTY_KEY);
    +    }
    +    try {
    +      this.persistentDataStore = DataStoreFactory.getDataStore(keyClass, persistentClass,
    +              new Configuration());
    +    } catch (GoraException ex) {
    +      LOG.error("Couldn't initialize persistent DataStore");
    +    }
    +    hazelcastInstance = Hazelcast.newHazelcastInstance();
    +    Properties providerProperties = new Properties();
    +    providerProperties.setProperty(HazelcastCachingProvider.HAZELCAST_INSTANCE_NAME,
    +            hazelcastInstance.getName());
    +    try {
    +      manager = cachingProvider.getCacheManager(new URI(goraCacheNamespace), null, providerProperties);
    +    } catch (URISyntaxException ex) {
    +      LOG.error("Couldn't initialize cache manager to bounded hazelcast instance");
    +      manager = cachingProvider.getCacheManager();
    +    }
    +    cacheEntryList = new ConcurrentSkipListSet<>();
    +    cacheConfig = new CacheConfig<K, T>();
    +    cacheConfig.setTypes(keyClass, persistentClass);
    +    if (properties.getProperty(JCACHE_READ_THROUGH_PROPERTY_KEY) != null) {
    +      cacheConfig.setReadThrough(Boolean.valueOf(properties.getProperty(JCACHE_READ_THROUGH_PROPERTY_KEY)));
    +    } else {
    +      cacheConfig.setReadThrough(true);
    +    }
    +    if (properties.getProperty(JCACHE_WRITE_THROUGH_PROPERTY_KEY) != null) {
    +      cacheConfig.setWriteThrough(Boolean.valueOf(properties.getProperty(JCACHE_WRITE_THROUGH_PROPERTY_KEY)));
    +    } else {
    +      cacheConfig.setWriteThrough(true);
    +    }
    +    if (properties.getProperty(JCACHE_STORE_BY_VALUE_PROPERTY_KEY) != null) {
    +      cacheConfig.setStoreByValue(Boolean.valueOf(properties.getProperty(JCACHE_STORE_BY_VALUE_PROPERTY_KEY)));
    +    }
    +    if (properties.getProperty(JCACHE_STATISTICS_PROPERTY_KEY) != null) {
    +      cacheConfig.setStatisticsEnabled(Boolean.valueOf(properties.getProperty(JCACHE_STATISTICS_PROPERTY_KEY)));
    +    }
    +    if (properties.getProperty(JCACHE_MANAGEMENT_PROPERTY_KEY) != null) {
    +      cacheConfig.setStatisticsEnabled(Boolean.valueOf(properties.getProperty(JCACHE_MANAGEMENT_PROPERTY_KEY)));
    +    }
    +    if (properties.getProperty(JCACHE_EVICTION_POLICY_PROPERTY_KEY) != null) {
    +      cacheConfig.getEvictionConfig()
    +              .setEvictionPolicy(EvictionPolicy.valueOf(properties.getProperty(JCACHE_EVICTION_POLICY_PROPERTY_KEY)));
    +    }
    +    if (properties.getProperty(JCACHE_EVICTION_MAX_SIZE_POLICY_PROPERTY_KEY) != null) {
    +      cacheConfig.getEvictionConfig()
    +              .setMaximumSizePolicy(EvictionConfig.MaxSizePolicy
    +                      .valueOf(properties.getProperty(JCACHE_EVICTION_MAX_SIZE_POLICY_PROPERTY_KEY)));
    +    }
    +    if (properties.getProperty(JCACHE_EVICTION_SIZE_PROPERTY_KEY) != null) {
    +      cacheConfig.getEvictionConfig()
    +              .setSize(Integer.valueOf(properties.getProperty(JCACHE_EVICTION_SIZE_PROPERTY_KEY)));
    +    }
    +    if (properties.getProperty(JCACHE_EXPIRE_POLICY_PROPERTY_KEY) != null) {
    +      String expiryPolicyIdentifier = properties.getProperty(JCACHE_EXPIRE_POLICY_PROPERTY_KEY);
    +      if (expiryPolicyIdentifier.equals(JCACHE_ACCESSED_EXPIRY_IDENTIFIER)) {
    +        cacheConfig.setExpiryPolicyFactory(FactoryBuilder.factoryOf(
    +                new AccessedExpiryPolicy(new Duration(TimeUnit.SECONDS,
    +                        Integer.valueOf(properties.getProperty(JCACHE_EXPIRE_POLICY_DURATION_PROPERTY_KEY))))
    +        ));
    +      } else if (expiryPolicyIdentifier.equals(JCACHE_CREATED_EXPIRY_IDENTIFIER)) {
    +        cacheConfig.setExpiryPolicyFactory(FactoryBuilder.factoryOf(
    +                new CreatedExpiryPolicy(new Duration(TimeUnit.SECONDS,
    +                        Integer.valueOf(properties.getProperty(JCACHE_EXPIRE_POLICY_DURATION_PROPERTY_KEY))))
    +        ));
    +      } else if (expiryPolicyIdentifier.equals(JCACHE_MODIFIED_EXPIRY_IDENTIFIER)) {
    +        cacheConfig.setExpiryPolicyFactory(FactoryBuilder.factoryOf(
    +                new ModifiedExpiryPolicy(new Duration(TimeUnit.SECONDS,
    +                        Integer.valueOf(properties.getProperty(JCACHE_EXPIRE_POLICY_DURATION_PROPERTY_KEY))))
    +        ));
    +      } else if (expiryPolicyIdentifier.equals(JCACHE_TOUCHED_EXPIRY_IDENTIFIER)) {
    +        cacheConfig.setExpiryPolicyFactory(FactoryBuilder.factoryOf(
    +                new TouchedExpiryPolicy(new Duration(TimeUnit.SECONDS,
    +                        Integer.valueOf(properties.getProperty(JCACHE_EXPIRE_POLICY_DURATION_PROPERTY_KEY))))
    +        ));
    +      }
    +    }
    +    if (properties.getProperty(HAZELCAST_CACHE_IN_MEMORY_FORMAT_PROPERTY_KEY) != null) {
    +      String inMemoryFormat = properties.getProperty(HAZELCAST_CACHE_IN_MEMORY_FORMAT_PROPERTY_KEY);
    +      if (inMemoryFormat.equals(HAZELCAST_CACHE_BINARY_IN_MEMORY_FORMAT_IDENTIFIER) ||
    +              inMemoryFormat.equals(HAZELCAST_CACHE_OBJECT_IN_MEMORY_FORMAT_IDENTIFIER) ||
    +              inMemoryFormat.equals(HAZELCAST_CACHE_NATIVE_IN_MEMORY_FORMAT_IDENTIFIER)) {
    +        cacheConfig.setInMemoryFormat(InMemoryFormat.valueOf(inMemoryFormat));
    +      }
    +    }
    +    cacheConfig.setCacheLoaderFactory(JCacheCacheFactoryBuilder
    +            .factoryOfCacheLoader(this.persistentDataStore));
    +    cacheConfig.setCacheWriterFactory(JCacheCacheFactoryBuilder
    +            .factoryOfCacheWriter(this.persistentDataStore));
    +    cacheConfig.addCacheEntryListenerConfiguration(
    +            new MutableCacheEntryListenerConfiguration<>(
    +                    JCacheCacheFactoryBuilder
    +                            .factoryOfEntryListener(new JCacheCacheEntryListener<K, T>(cacheEntryList)),
    +                    null, true, true
    +            )
    +    );
    +    if (properties.getProperty(JCACHE_AUTO_CREATE_CACHE_PROPERTY_KEY) != null) {
    +      Boolean createCache = Boolean.valueOf(properties.getProperty(JCACHE_AUTO_CREATE_CACHE_PROPERTY_KEY));
    +      if (createCache) {
    +        cache = manager.createCache(persistentClass.getSimpleName(),
    +                cacheConfig).unwrap(ICache.class);
    +      }
    +    } else {
    +      cache = manager.createCache(persistentClass.getSimpleName(),
    +              cacheConfig).unwrap(ICache.class);
    +    }
    +    LOG.info("JCache Gora datastore initialized successfully.");
    +  }
    +
    +  @Override
    +  public String getSchemaName() {
    +    return super.persistentClass.getSimpleName();
    +  }
    +
    +  @Override
    +  public void createSchema() {
    +    if (manager.getCache(super.getPersistentClass().getSimpleName(), keyClass, persistentClass) == null) {
    +      cacheEntryList.clear();
    +      cache = manager.createCache(persistentClass.getSimpleName(),
    +              cacheConfig).unwrap(ICache.class);
    +    }
    +    persistentDataStore.createSchema();
    +    LOG.info("Created schema on persistent store and initialized cache for persistent bean "
    +            + super.getPersistentClass().getSimpleName());
    +  }
    +
    +  @Override
    +  public void deleteSchema() {
    +    cacheEntryList.clear();
    +    manager.destroyCache(super.getPersistentClass().getSimpleName());
    +    persistentDataStore.deleteSchema();
    +    LOG.info("Deleted schema on persistent store and destroyed cache for persistent bean "
    +            + super.getPersistentClass().getSimpleName());
    +  }
    +
    +  @Override
    +  public boolean schemaExists() {
    +    return (manager.getCache(super.getPersistentClass().getSimpleName(), keyClass, persistentClass) != null);
    +  }
    +
    +  @Override
    +  public T get(K key, String[] fields) {
    +    T persitent = (T) cache.get(key);
    +    if (persitent == null) {
    +      return null;
    +    }
    +    return getPersistent(persitent, fields);
    +  }
    +
    +  @Override
    +  public T get(K key) {
    +    return cache.get(key);
    +  }
    +
    +  @Override
    +  public void put(K key, T val) {
    +    cache.put(key, val);
    +  }
    +
    +  @Override
    +  public boolean delete(K key) {
    +    return cache.remove(key);
    +  }
    +
    +  @Override
    +  public long deleteByQuery(Query<K, T> query) {
    +    try {
    +      long deletedRows = 0;
    +      Result<K, T> result = query.execute();
    +      String[] fields = getFieldsToQuery(query.getFields());
    +      boolean isAllFields = Arrays.equals(fields, getFields());
    +      while (result.next()) {
    +        if (isAllFields) {
    +          if (delete(result.getKey())) {
    +            deletedRows++;
    +          }
    +        } else {
    +          ArrayList<String> excludedFields = new ArrayList<>();
    +          for (String field : getFields()) {
    +            if (!Arrays.asList(fields).contains(field)) {
    +              excludedFields.add(field);
    +            }
    +          }
    +          T newClonedObj = getPersistent(result.get(),
    +                  excludedFields.toArray(new String[excludedFields.size()]));
    +          if (delete(result.getKey())) {
    +            put(result.getKey(), newClonedObj);
    +            deletedRows++;
    +          }
    +        }
    +      }
    +      LOG.info("JCache Gora datastore deleled " + deletedRows + " rows from Persistent datastore");
    +      return deletedRows;
    +    } catch (Exception e) {
    +      LOG.error("Exception occured while deleting entries from JCache Gora datastore. Hence returning 0");
    +      return 0;
    +    }
    +  }
    +
    +  @Override
    +  public Result<K, T> execute(Query<K, T> query) {
    +    K startKey = query.getStartKey();
    +    K endKey = query.getEndKey();
    +    if (startKey == null) {
    +      if (!cacheEntryList.isEmpty()) {
    +        startKey = (K) cacheEntryList.first();
    +      }
    +    }
    +    if (endKey == null) {
    +      if (!cacheEntryList.isEmpty()) {
    +        endKey = (K) cacheEntryList.last();
    +      }
    +    }
    +    query.setFields(getFieldsToQuery(query.getFields()));
    +    ConcurrentSkipListSet<K> cacheEntrySubList = null;
    +    try {
    +      cacheEntrySubList = (ConcurrentSkipListSet<K>) cacheEntryList.subSet(startKey, true, endKey, true);
    +    } catch (NullPointerException npe) {
    +      LOG.error("NPE occurred while executing the query for JCacheStore. Hence returning empty entry set.");
    +      return new JCacheResult<>(this, query, new ConcurrentSkipListSet<K>());
    +    }
    +    return new JCacheResult<>(this, query, cacheEntrySubList);
    +  }
    +
    +  @Override
    +  public Query<K, T> newQuery() {
    +    return new JCacheQuery<>(this);
    +  }
    +
    +  @Override
    +  public List<PartitionQuery<K, T>> getPartitions(Query<K, T> query) throws IOException {
    +    List<PartitionQuery<K, T>> partitions = new ArrayList<>();
    +    try {
    +      Member[] clusterMembers = new Member[hazelcastInstance.getCluster().getMembers().size()];
    +      this.hazelcastInstance.getCluster().getMembers().toArray(clusterMembers);
    +      for (Member member : clusterMembers) {
    +        JCacheResult<K, T> result = ((JCacheResult<K, T>) query.execute());
    +        ConcurrentSkipListSet<K> memberOwnedCacheEntries = new ConcurrentSkipListSet<>();
    +        while (result.next()) {
    +          K key = result.getKey();
    +          Partition partition = hazelcastInstance.getPartitionService().getPartition(key);
    +          if (partition.getOwner().getUuid().equals(member.getUuid())) {
    +            memberOwnedCacheEntries.add(key);
    +          }
    +        }
    +        PartitionQueryImpl<K, T> partition = new PartitionQueryImpl<>(
    +                query, memberOwnedCacheEntries.first(),
    +                memberOwnedCacheEntries.last(), member.getSocketAddress().getHostString());
    +        partitions.add(partition);
    +      }
    +    } catch (java.lang.Exception ex) {
    --- End diff --
    
    Is there no better exception to catch here?


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

[GitHub] gora pull request #72: Gora 409

Posted by lewismc <gi...@git.apache.org>.
Github user lewismc commented on a diff in the pull request:

    https://github.com/apache/gora/pull/72#discussion_r73616090
  
    --- Diff: gora-jcache/src/main/java/org/apache/gora/jcache/query/JCacheResult.java ---
    @@ -0,0 +1,69 @@
    +/**
    + * 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.gora.jcache.query;
    +
    +import java.io.IOException;
    +import java.util.Iterator;
    +import java.util.NavigableSet;
    +
    +import org.apache.gora.jcache.store.JCacheStore;
    +import org.apache.gora.persistency.impl.PersistentBase;
    +import org.apache.gora.query.Query;
    +import org.apache.gora.query.impl.ResultBase;
    +import org.apache.gora.store.DataStore;
    +
    +public class JCacheResult<K, T extends PersistentBase> extends ResultBase<K, T> {
    +
    +  private NavigableSet<K> cacheKeySet;
    +  private Iterator<K> iterator;
    +
    +  public JCacheResult(DataStore<K, T> dataStore, Query<K, T> query) {
    --- End diff --
    
    
    
    Add documentation



---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

[GitHub] gora pull request #72: Gora 409

Posted by lewismc <gi...@git.apache.org>.
Github user lewismc commented on a diff in the pull request:

    https://github.com/apache/gora/pull/72#discussion_r73616048
  
    --- Diff: gora-jcache/src/main/java/org/apache/gora/jcache/query/JCacheResult.java ---
    @@ -0,0 +1,69 @@
    +/**
    + * 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.gora.jcache.query;
    +
    +import java.io.IOException;
    +import java.util.Iterator;
    +import java.util.NavigableSet;
    +
    +import org.apache.gora.jcache.store.JCacheStore;
    +import org.apache.gora.persistency.impl.PersistentBase;
    +import org.apache.gora.query.Query;
    +import org.apache.gora.query.impl.ResultBase;
    +import org.apache.gora.store.DataStore;
    +
    +public class JCacheResult<K, T extends PersistentBase> extends ResultBase<K, T> {
    --- End diff --
    
    
    
    Please document this class



---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

[GitHub] gora issue #72: Gora 409

Posted by lewismc <gi...@git.apache.org>.
Github user lewismc commented on the issue:

    https://github.com/apache/gora/pull/72
  
    @djkevincr NICE WORK :)
    Some comments
     1. Please include Javadoc for all of your publicly available methods, Classes and also add package-info.java to every package you create.
     2. Please ALWAYS catch and log Exception's within your LOG message e.g. ```LOG.error("Log message here", e) 
     3. Please ALWAYS use parameterized logging e.g. ```LOG.info("Log {} and {} values", value1, value2)```
     4. My main question is, what is going on with the testing? Are you setting up the Hazelcast test infrastructure or how is the Mock testing working? This needs to be added to the test class and needs to be explained directly there. Also why are you not extending DataStoreTestBase?
     5. As far as I can see, you've not added this to the gora-tutorial meaning that no-one can run this entire module easily. Please make sure to add it to the gora-tutorial.
     6. Please add this to gora-goraci, we will wish to test it in the Cloud shortly and may even write Docker containers for all of this.
    
    If you can lease address the documentation and logging issues first then notify me that would be great.
    
    Again, great work. Nice to see this coming along.


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

[GitHub] gora pull request #72: Gora 409

Posted by lewismc <gi...@git.apache.org>.
Github user lewismc commented on a diff in the pull request:

    https://github.com/apache/gora/pull/72#discussion_r73616502
  
    --- Diff: gora-jcache/src/main/java/org/apache/gora/jcache/store/JCacheCacheLoaderFactory.java ---
    @@ -0,0 +1,56 @@
    +/**
    + * 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.gora.jcache.store;
    +
    +import org.apache.gora.persistency.impl.PersistentBase;
    +import org.slf4j.Logger;
    +import org.slf4j.LoggerFactory;
    +
    +import javax.cache.configuration.Factory;
    +
    +public class JCacheCacheLoaderFactory<K, T extends PersistentBase>
    +        implements Factory<JCacheCacheLoader<K,T>> {
    +
    +  private static final Logger LOG = LoggerFactory.getLogger(JCacheCacheLoaderFactory.class);
    +  public static final long serialVersionUID = 201305101626L;
    +  private transient JCacheCacheLoader<K, T> instance;
    +
    +  public JCacheCacheLoaderFactory(JCacheCacheLoader<K, T> instance) {
    +    LOG.info("JCache cache entry loader factory initialized successfully.");
    +    this.instance = instance;
    +  }
    +
    +  public JCacheCacheLoader<K, T> create() {
    +    return (JCacheCacheLoader<K, T>) this.instance;
    +  }
    +
    +  public boolean equals(Object other) {
    +    if (this == other) {
    +      return true;
    +    } else if (other != null && this.getClass() == other.getClass()) {
    +      return true;
    +    } else {
    +      return false;
    +    }
    +  }
    +
    +  public int hashCode() {
    --- End diff --
    
    Add documentation


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

[GitHub] gora issue #72: Gora 409

Posted by lewismc <gi...@git.apache.org>.
Github user lewismc commented on the issue:

    https://github.com/apache/gora/pull/72
  
    The entire pull request does not compile under Java 8
    When I change to Java7 it gives me the following
    ```
    [INFO] BUILD FAILURE
    [INFO] ------------------------------------------------------------------------
    [INFO] Total time: 07:41 min
    [INFO] Finished at: 2016-08-05T02:48:40-07:00
    [INFO] Final Memory: 54M/323M
    [INFO] ------------------------------------------------------------------------
    [ERROR] Failed to execute goal org.apache.maven.plugins:maven-compiler-plugin:3.1:compile (default-compile) on project gora-jcache: Compilation failure: Compilation failure:
    [ERROR] /usr/local/gora-dev/gora-jcache/src/main/java/org/apache/gora/jcache/query/JCacheQuery.java:[24,7] error: cannot access Writable
    [ERROR] class file for org.apache.hadoop.io.Writable not found
    [ERROR] /usr/local/gora-dev/gora-jcache/src/main/java/org/apache/gora/jcache/store/JCacheStore.java:[53,29] error: package org.apache.hadoop.conf does not exist
    [ERROR] /usr/local/gora-dev/gora-jcache/src/main/java/org/apache/gora/jcache/store/JCacheStore.java:[68,7] error: cannot access Configurable
    [ERROR] -> [Help 1]
    [ERROR]
    [ERROR] To see the full stack trace of the errors, re-run Maven with the -e switch.
    [ERROR] Re-run Maven using the -X switch to enable full debug logging.
    [ERROR]
    [ERROR] For more information about the errors and possible solutions, please read the following articles:
    [ERROR] [Help 1] http://cwiki.apache.org/confluence/display/MAVEN/MojoFailureException
    [ERROR]
    [ERROR] After correcting the problems, you can resume the build with the command
    [ERROR]   mvn <goals> -rf :gora-jcache
    ```


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

[GitHub] gora pull request #72: Gora 409

Posted by lewismc <gi...@git.apache.org>.
Github user lewismc commented on a diff in the pull request:

    https://github.com/apache/gora/pull/72#discussion_r73616117
  
    --- Diff: gora-jcache/src/main/java/org/apache/gora/jcache/query/JCacheResult.java ---
    @@ -0,0 +1,69 @@
    +/**
    + * 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.gora.jcache.query;
    +
    +import java.io.IOException;
    +import java.util.Iterator;
    +import java.util.NavigableSet;
    +
    +import org.apache.gora.jcache.store.JCacheStore;
    +import org.apache.gora.persistency.impl.PersistentBase;
    +import org.apache.gora.query.Query;
    +import org.apache.gora.query.impl.ResultBase;
    +import org.apache.gora.store.DataStore;
    +
    +public class JCacheResult<K, T extends PersistentBase> extends ResultBase<K, T> {
    +
    +  private NavigableSet<K> cacheKeySet;
    +  private Iterator<K> iterator;
    +
    +  public JCacheResult(DataStore<K, T> dataStore, Query<K, T> query) {
    +    super(dataStore, query);
    +  }
    +
    +  public JCacheResult(DataStore<K, T> dataStore, Query<K, T> query, NavigableSet<K> cacheKeySet) {
    +    super(dataStore, query);
    +    this.cacheKeySet = cacheKeySet;
    +    this.iterator = cacheKeySet.iterator();
    +  }
    +
    +  public JCacheStore<K, T> getDataStore() {
    +    return (JCacheStore<K, T>) super.getDataStore();
    +  }
    +
    +  @Override
    +  public float getProgress() throws IOException {
    --- End diff --
    
    Add documentation


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

[GitHub] gora pull request #72: Gora 409

Posted by lewismc <gi...@git.apache.org>.
Github user lewismc commented on a diff in the pull request:

    https://github.com/apache/gora/pull/72#discussion_r73616005
  
    --- Diff: gora-jcache/src/main/java/org/apache/gora/jcache/query/JCacheQuery.java ---
    @@ -0,0 +1,34 @@
    +/**
    + * 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.gora.jcache.query;
    +
    +import org.apache.gora.persistency.impl.PersistentBase;
    +import org.apache.gora.query.impl.QueryBase;
    +import org.apache.gora.store.DataStore;
    +
    +public class JCacheQuery<K, T extends PersistentBase> extends QueryBase<K, T> {
    --- End diff --
    
    Please document this class


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

[GitHub] gora pull request #72: Gora 409

Posted by lewismc <gi...@git.apache.org>.
Github user lewismc commented on a diff in the pull request:

    https://github.com/apache/gora/pull/72#discussion_r73616240
  
    --- Diff: gora-jcache/src/main/java/org/apache/gora/jcache/store/JCacheCacheEntryListener.java ---
    @@ -0,0 +1,77 @@
    +/**
    + * 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.gora.jcache.store;
    +
    +import org.apache.gora.persistency.impl.PersistentBase;
    +import org.slf4j.Logger;
    +import org.slf4j.LoggerFactory;
    +
    +import javax.cache.event.CacheEntryExpiredListener;
    +import javax.cache.event.CacheEntryUpdatedListener;
    +import javax.cache.event.CacheEntryCreatedListener;
    +import javax.cache.event.CacheEntryRemovedListener;
    +import javax.cache.event.CacheEntryEvent;
    +import javax.cache.event.CacheEntryListenerException;
    +import java.util.concurrent.ConcurrentSkipListSet;
    +
    +public class JCacheCacheEntryListener<K, T extends PersistentBase>
    +        implements CacheEntryCreatedListener<K, T>,
    +        CacheEntryRemovedListener<K, T>, CacheEntryUpdatedListener<K, T>, CacheEntryExpiredListener<K, T> {
    +
    +  private static final Logger LOG = LoggerFactory.getLogger(JCacheCacheEntryListener.class);
    +  private ConcurrentSkipListSet<K> cacheEntryList;
    +
    +  public JCacheCacheEntryListener(ConcurrentSkipListSet cacheEntryList) {
    +    this.cacheEntryList = cacheEntryList;
    +  }
    +
    +  @Override
    +  public void onCreated(Iterable<CacheEntryEvent<? extends K, ? extends T>> cacheEntryEvents)
    +          throws CacheEntryListenerException {
    +    for (CacheEntryEvent<? extends K, ? extends T> event : cacheEntryEvents) {
    +      cacheEntryList.add(event.getKey());
    +      LOG.info("Cache entry added on key " + event.getKey().toString());
    +    }
    +  }
    +
    +  @Override
    +  public void onRemoved(Iterable<CacheEntryEvent<? extends K, ? extends T>> cacheEntryEvents)
    +          throws CacheEntryListenerException {
    +    for (CacheEntryEvent<? extends K, ? extends T> event : cacheEntryEvents) {
    +      cacheEntryList.remove(event.getKey());
    +      LOG.info("Cache entry removed on key " + event.getKey().toString());
    +    }
    +  }
    +
    +  @Override
    +  public void onUpdated(Iterable<CacheEntryEvent<? extends K, ? extends T>> cacheEntryEvents)
    +          throws CacheEntryListenerException {
    +    for (CacheEntryEvent<? extends K, ? extends T> event : cacheEntryEvents) {
    +      LOG.info("Cache entry updated set on key " + event.getKey().toString());
    +    }
    +  }
    +
    +  @Override
    +  public void onExpired(Iterable<CacheEntryEvent<? extends K, ? extends T>> cacheEntryEvents)
    --- End diff --
    
    Add documentation


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

[GitHub] gora pull request #72: Gora 409

Posted by lewismc <gi...@git.apache.org>.
Github user lewismc commented on a diff in the pull request:

    https://github.com/apache/gora/pull/72#discussion_r73616535
  
    --- Diff: gora-jcache/src/main/java/org/apache/gora/jcache/store/JCacheCacheWriter.java ---
    @@ -0,0 +1,72 @@
    +/**
    + * 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.gora.jcache.store;
    +
    +import org.apache.gora.persistency.impl.PersistentBase;
    +import org.apache.gora.store.DataStore;
    +import org.slf4j.Logger;
    +import org.slf4j.LoggerFactory;
    +
    +import javax.cache.Cache;
    +import javax.cache.integration.CacheWriter;
    +import javax.cache.integration.CacheWriterException;
    +import java.util.Collection;
    +import java.util.Iterator;
    +
    +public class JCacheCacheWriter<K, T extends PersistentBase> implements CacheWriter<K, T> {
    +
    +  private static final Logger LOG = LoggerFactory.getLogger(JCacheCacheWriter.class);
    +  private DataStore<K, T> dataStore;
    +
    +  public JCacheCacheWriter(DataStore<K, T> dataStore) {
    +    this.dataStore = dataStore;
    +  }
    +
    +  @Override
    +  public void write(Cache.Entry<? extends K,
    +          ? extends T> entry) throws CacheWriterException {
    +    dataStore.put(entry.getKey(), entry.getValue());
    +    LOG.info("Written data bean to persistent datastore on key " + entry.getKey().toString());
    +  }
    +
    +  @Override
    +  public void writeAll(Collection<Cache.Entry<? extends K,
    --- End diff --
    
    Add documentation


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

[GitHub] gora pull request #72: Gora 409

Posted by lewismc <gi...@git.apache.org>.
Github user lewismc commented on a diff in the pull request:

    https://github.com/apache/gora/pull/72#discussion_r73616529
  
    --- Diff: gora-jcache/src/main/java/org/apache/gora/jcache/store/JCacheCacheWriter.java ---
    @@ -0,0 +1,72 @@
    +/**
    + * 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.gora.jcache.store;
    +
    +import org.apache.gora.persistency.impl.PersistentBase;
    +import org.apache.gora.store.DataStore;
    +import org.slf4j.Logger;
    +import org.slf4j.LoggerFactory;
    +
    +import javax.cache.Cache;
    +import javax.cache.integration.CacheWriter;
    +import javax.cache.integration.CacheWriterException;
    +import java.util.Collection;
    +import java.util.Iterator;
    +
    +public class JCacheCacheWriter<K, T extends PersistentBase> implements CacheWriter<K, T> {
    +
    +  private static final Logger LOG = LoggerFactory.getLogger(JCacheCacheWriter.class);
    +  private DataStore<K, T> dataStore;
    +
    +  public JCacheCacheWriter(DataStore<K, T> dataStore) {
    +    this.dataStore = dataStore;
    +  }
    +
    +  @Override
    +  public void write(Cache.Entry<? extends K,
    --- End diff --
    
    Add documentation


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

[GitHub] gora pull request #72: Gora 409

Posted by lewismc <gi...@git.apache.org>.
Github user lewismc commented on a diff in the pull request:

    https://github.com/apache/gora/pull/72#discussion_r73616671
  
    --- Diff: gora-jcache/src/main/java/org/apache/gora/jcache/store/JCacheCacheWriterFactory.java ---
    @@ -0,0 +1,55 @@
    +/**
    + * 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.gora.jcache.store;
    +
    +import org.apache.gora.persistency.impl.PersistentBase;
    +import org.slf4j.Logger;
    +import org.slf4j.LoggerFactory;
    +
    +import javax.cache.configuration.Factory;
    +
    +public class JCacheCacheWriterFactory<K, T extends PersistentBase> implements Factory<JCacheCacheWriter<K,T>> {
    +
    +  private static final Logger LOG = LoggerFactory.getLogger(JCacheCacheWriterFactory.class);
    +  public static final long serialVersionUID = 201205101621L;
    +  private transient JCacheCacheWriter<K,T> instance;
    +
    +  public JCacheCacheWriterFactory(JCacheCacheWriter<K,T> instance) {
    +    LOG.info("JCache cache writer factory initialized successfully.");
    +    this.instance = instance;
    +  }
    +
    +  public JCacheCacheWriter<K,T> create() {
    --- End diff --
    
    Add documentation


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

[GitHub] gora issue #72: Gora 409

Posted by lewismc <gi...@git.apache.org>.
Github user lewismc commented on the issue:

    https://github.com/apache/gora/pull/72
  
    OK



---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

[GitHub] gora issue #72: Gora 409

Posted by djkevincr <gi...@git.apache.org>.
Github user djkevincr commented on the issue:

    https://github.com/apache/gora/pull/72
  
    I think the build passed without any test failures see [1] Or Am I missing anything here?
    [1] https://builds.apache.org/job/gora-trunk/1674/


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

[GitHub] gora issue #72: Gora 409

Posted by lewismc <gi...@git.apache.org>.
Github user lewismc commented on the issue:

    https://github.com/apache/gora/pull/72
  
    @djkevincr why did you close this PR?


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

[GitHub] gora pull request #72: Gora 409

Posted by lewismc <gi...@git.apache.org>.
Github user lewismc commented on a diff in the pull request:

    https://github.com/apache/gora/pull/72#discussion_r73616180
  
    --- Diff: gora-jcache/src/main/java/org/apache/gora/jcache/store/JCacheCacheEntryListener.java ---
    @@ -0,0 +1,77 @@
    +/**
    + * 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.gora.jcache.store;
    +
    +import org.apache.gora.persistency.impl.PersistentBase;
    +import org.slf4j.Logger;
    +import org.slf4j.LoggerFactory;
    +
    +import javax.cache.event.CacheEntryExpiredListener;
    +import javax.cache.event.CacheEntryUpdatedListener;
    +import javax.cache.event.CacheEntryCreatedListener;
    +import javax.cache.event.CacheEntryRemovedListener;
    +import javax.cache.event.CacheEntryEvent;
    +import javax.cache.event.CacheEntryListenerException;
    +import java.util.concurrent.ConcurrentSkipListSet;
    +
    +public class JCacheCacheEntryListener<K, T extends PersistentBase>
    +        implements CacheEntryCreatedListener<K, T>,
    +        CacheEntryRemovedListener<K, T>, CacheEntryUpdatedListener<K, T>, CacheEntryExpiredListener<K, T> {
    +
    +  private static final Logger LOG = LoggerFactory.getLogger(JCacheCacheEntryListener.class);
    +  private ConcurrentSkipListSet<K> cacheEntryList;
    +
    +  public JCacheCacheEntryListener(ConcurrentSkipListSet cacheEntryList) {
    --- End diff --
    
    Add documentation


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

[GitHub] gora issue #72: Gora 409

Posted by lewismc <gi...@git.apache.org>.
Github user lewismc commented on the issue:

    https://github.com/apache/gora/pull/72
  
    Thats a ```mvn clean install```


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

[GitHub] gora pull request #72: Gora 409

Posted by lewismc <gi...@git.apache.org>.
Github user lewismc commented on a diff in the pull request:

    https://github.com/apache/gora/pull/72#discussion_r73616161
  
    --- Diff: gora-jcache/src/main/java/org/apache/gora/jcache/query/JCacheResult.java ---
    @@ -0,0 +1,69 @@
    +/**
    + * 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.gora.jcache.query;
    +
    +import java.io.IOException;
    +import java.util.Iterator;
    +import java.util.NavigableSet;
    +
    +import org.apache.gora.jcache.store.JCacheStore;
    +import org.apache.gora.persistency.impl.PersistentBase;
    +import org.apache.gora.query.Query;
    +import org.apache.gora.query.impl.ResultBase;
    +import org.apache.gora.store.DataStore;
    +
    +public class JCacheResult<K, T extends PersistentBase> extends ResultBase<K, T> {
    +
    +  private NavigableSet<K> cacheKeySet;
    +  private Iterator<K> iterator;
    +
    +  public JCacheResult(DataStore<K, T> dataStore, Query<K, T> query) {
    +    super(dataStore, query);
    +  }
    +
    +  public JCacheResult(DataStore<K, T> dataStore, Query<K, T> query, NavigableSet<K> cacheKeySet) {
    +    super(dataStore, query);
    +    this.cacheKeySet = cacheKeySet;
    +    this.iterator = cacheKeySet.iterator();
    +  }
    +
    +  public JCacheStore<K, T> getDataStore() {
    +    return (JCacheStore<K, T>) super.getDataStore();
    +  }
    +
    +  @Override
    +  public float getProgress() throws IOException {
    +    return 0;
    +  }
    +
    +  @Override
    +  public void close() throws IOException {
    +
    +  }
    +
    +  @Override
    +  protected boolean nextInner() throws IOException {
    --- End diff --
    
    Add documentation


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

[GitHub] gora issue #72: Gora 409

Posted by lewismc <gi...@git.apache.org>.
Github user lewismc commented on the issue:

    https://github.com/apache/gora/pull/72
  
    OK, no problems. I ask you to push code as often as possible in a pull request because there are some very competent coders who can peer review your code. This is a positive thing in your GSoC endeavor.
    Please push more frequently.
    Please let me know how you are doing and how your code is passing tests.
    We can easily verify this @ The Apache Software Foundation, we have builds. I can easily set up a branch just for your code and build your code via a branch every time you push updates.
    Lets begin to be serious about integration into the master branch of Gora. This is the main goal OK.
    Your work so far is excellent. We have a hurdle now, which is integration into a working, functional open source project.
    You are now part of the Gora community.


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

[GitHub] gora pull request #72: Gora 409

Posted by lewismc <gi...@git.apache.org>.
Github user lewismc commented on a diff in the pull request:

    https://github.com/apache/gora/pull/72#discussion_r73616426
  
    --- Diff: gora-jcache/src/main/java/org/apache/gora/jcache/store/JCacheCacheFactoryBuilder.java ---
    @@ -0,0 +1,41 @@
    +/**
    + * 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.gora.jcache.store;
    +
    +import org.apache.gora.persistency.impl.PersistentBase;
    +import org.apache.gora.store.DataStore;
    +import javax.cache.configuration.Factory;
    +
    +public class JCacheCacheFactoryBuilder {
    +
    +  public static <K, T extends PersistentBase> Factory<JCacheCacheLoader<K,T>>
    +  factoryOfCacheLoader(DataStore<K, T> dataStore) {
    +    return new JCacheCacheLoaderFactory<>(new JCacheCacheLoader<>(dataStore));
    +  }
    +
    +  public static <K, T extends PersistentBase> Factory<JCacheCacheWriter<K,T>>
    +  factoryOfCacheWriter(DataStore<K, T> dataStore) {
    +    return new JCacheCacheWriterFactory<>(new JCacheCacheWriter<>(dataStore));
    +  }
    +
    +  public static <K,T extends PersistentBase> Factory<JCacheCacheEntryListener<K, T>>
    --- End diff --
    
    Add documentation


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

[GitHub] gora pull request #72: Gora 409

Posted by lewismc <gi...@git.apache.org>.
Github user lewismc commented on a diff in the pull request:

    https://github.com/apache/gora/pull/72#discussion_r73616794
  
    --- Diff: gora-jcache/src/main/java/org/apache/gora/jcache/store/JCacheStore.java ---
    @@ -0,0 +1,412 @@
    +/**
    + * 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.gora.jcache.store;
    +
    +import java.io.IOException;
    +import java.net.URI;
    +import java.net.URISyntaxException;
    +import java.util.List;
    +import java.util.Arrays;
    +import java.util.ArrayList;
    +import java.util.Properties;
    +import java.util.concurrent.ConcurrentSkipListSet;
    +import java.util.concurrent.TimeUnit;
    +
    +import com.hazelcast.cache.HazelcastCachingProvider;
    +import com.hazelcast.cache.ICache;
    +import com.hazelcast.config.CacheConfig;
    +import com.hazelcast.config.EvictionConfig;
    +import com.hazelcast.config.EvictionPolicy;
    +import com.hazelcast.config.InMemoryFormat;
    +import com.hazelcast.core.Hazelcast;
    +import com.hazelcast.core.HazelcastInstance;
    +import com.hazelcast.core.Member;
    +import com.hazelcast.core.Partition;
    +import org.apache.avro.Schema;
    +import org.apache.gora.jcache.query.JCacheQuery;
    +import org.apache.gora.jcache.query.JCacheResult;
    +import org.apache.gora.persistency.impl.PersistentBase;
    +import org.apache.gora.query.PartitionQuery;
    +import org.apache.gora.query.Query;
    +import org.apache.gora.query.Result;
    +import org.apache.gora.query.impl.PartitionQueryImpl;
    +import org.apache.gora.store.DataStore;
    +import org.apache.gora.store.DataStoreFactory;
    +import org.apache.gora.store.impl.DataStoreBase;
    +import org.apache.gora.util.AvroUtils;
    +import org.apache.gora.util.GoraException;
    +import org.apache.hadoop.conf.Configuration;
    +import org.slf4j.Logger;
    +import org.slf4j.LoggerFactory;
    +
    +import javax.cache.CacheManager;
    +import javax.cache.Caching;
    +import javax.cache.configuration.FactoryBuilder;
    +import javax.cache.configuration.MutableCacheEntryListenerConfiguration;
    +import javax.cache.expiry.AccessedExpiryPolicy;
    +import javax.cache.expiry.ModifiedExpiryPolicy;
    +import javax.cache.expiry.CreatedExpiryPolicy;
    +import javax.cache.expiry.TouchedExpiryPolicy;
    +import javax.cache.expiry.Duration;
    +import javax.cache.spi.CachingProvider;
    +
    +public class JCacheStore<K, T extends PersistentBase> extends DataStoreBase<K, T> {
    +
    +  private static final String GORA_DEFAULT_JCACHE_NAMESPACE = "gora.jcache.namespace";
    +  private static final String GORA_DEFAULT_JCACHE_PROVIDER_KEY = "gora.datastore.jcache.provider";
    +  private static final String JCACHE_READ_THROUGH_PROPERTY_KEY = "jcache.read.through.enable";
    +  private static final String JCACHE_WRITE_THROUGH_PROPERTY_KEY = "jcache.write.through.enable";
    +  private static final String JCACHE_STORE_BY_VALUE_PROPERTY_KEY = "jcache.store.by.value.enable";
    +  private static final String JCACHE_STATISTICS_PROPERTY_KEY = "jcache.statistics.enable";
    +  private static final String JCACHE_MANAGEMENT_PROPERTY_KEY = "jcache.management.enable";
    +  private static final String JCACHE_CACHE_NAMESPACE_PROPERTY_KEY = "jcache.cache.namespace";
    +  private static final String JCACHE_EVICTION_POLICY_PROPERTY_KEY = "jcache.eviction.policy";
    +  private static final String JCACHE_EVICTION_MAX_SIZE_POLICY_PROPERTY_KEY = "jcache.eviction.max.size.policy";
    +  private static final String JCACHE_EVICTION_SIZE_PROPERTY_KEY = "jcache.eviction.size";
    +  private static final String JCACHE_EXPIRE_POLICY_PROPERTY_KEY = "jcache.expire.policy";
    +  private static final String JCACHE_EXPIRE_POLICY_DURATION_PROPERTY_KEY = "jcache.expire.policy";
    +  private static final String JCACHE_ACCESSED_EXPIRY_IDENTIFIER = "ACCESSED";
    +  private static final String JCACHE_CREATED_EXPIRY_IDENTIFIER = "CREATED";
    +  private static final String JCACHE_MODIFIED_EXPIRY_IDENTIFIER = "MODIFIED";
    +  private static final String JCACHE_TOUCHED_EXPIRY_IDENTIFIER = "TOUCHED";
    +  private static final String HAZELCAST_CACHE_IN_MEMORY_FORMAT_PROPERTY_KEY = "jcache.cache.inmemory.format";
    +  private static final String HAZELCAST_CACHE_BINARY_IN_MEMORY_FORMAT_IDENTIFIER = "BINARY";
    +  private static final String HAZELCAST_CACHE_OBJECT_IN_MEMORY_FORMAT_IDENTIFIER = "OBJECT";
    +  private static final String HAZELCAST_CACHE_NATIVE_IN_MEMORY_FORMAT_IDENTIFIER = "NATIVE";
    +  private static final String JCACHE_AUTO_CREATE_CACHE_PROPERTY_KEY = "jcache.auto.create.cache";
    +  private static final Logger LOG = LoggerFactory.getLogger(JCacheStore.class);
    +  private ICache<K, T> cache;
    +  private CacheManager manager;
    +  private ConcurrentSkipListSet<K> cacheEntryList;
    +  private String goraCacheNamespace = GORA_DEFAULT_JCACHE_NAMESPACE;
    +  private DataStore<K, T> persistentDataStore;
    +  private CacheConfig<K, T> cacheConfig;
    +  private HazelcastInstance hazelcastInstance;
    +
    +  private static <T extends PersistentBase> T getPersistent(T persitent, String[] fields) {
    +    List<Schema.Field> otherFields = persitent.getSchema().getFields();
    +    String[] otherFieldStrings = new String[otherFields.size()];
    +    for (int i = 0; i < otherFields.size(); i++) {
    +      otherFieldStrings[i] = otherFields.get(i).name();
    +    }
    +    if (Arrays.equals(fields, otherFieldStrings)) {
    +      return persitent;
    +    }
    +    T clonedPersistent = AvroUtils.deepClonePersistent(persitent);
    +    clonedPersistent.clear();
    +    if (fields != null && fields.length > 0) {
    +      for (String field : fields) {
    +        Schema.Field otherField = persitent.getSchema().getField(field);
    +        int index = otherField.pos();
    +        clonedPersistent.put(index, persitent.get(index));
    +      }
    +    } else {
    +      for (String field : otherFieldStrings) {
    +        Schema.Field otherField = persitent.getSchema().getField(field);
    +        int index = otherField.pos();
    +        clonedPersistent.put(index, persitent.get(index));
    +      }
    +    }
    +    return clonedPersistent;
    +  }
    +
    +  @Override
    +  public void initialize(Class<K> keyClass, Class<T> persistentClass, Properties properties) {
    +    super.initialize(keyClass, persistentClass, properties);
    +    CachingProvider cachingProvider = Caching.getCachingProvider(
    +            properties.getProperty(GORA_DEFAULT_JCACHE_PROVIDER_KEY)
    +    );
    +    if (properties.getProperty(JCACHE_CACHE_NAMESPACE_PROPERTY_KEY) != null) {
    +      goraCacheNamespace = properties.getProperty(JCACHE_CACHE_NAMESPACE_PROPERTY_KEY);
    +    }
    +    try {
    +      this.persistentDataStore = DataStoreFactory.getDataStore(keyClass, persistentClass,
    +              new Configuration());
    +    } catch (GoraException ex) {
    +      LOG.error("Couldn't initialize persistent DataStore");
    --- End diff --
    
    Also log the Exception itself
    ```
     +      LOG.error("Couldn't initialize persistent DataStore", ex); 
    ```


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---