Navina,

It's not my code changes. I started off fresh on a different but similar
RHEL 6.6 box.

Cloned and built latest from master and changed only gradle/dependencies
jackson 1.9.3 and yarn 2.7.1 (as I used to do since day 1 to get jobs
running on my cluster).

All my KV store jobs fail with same error

org.apache.samza.SamzaException: Error opening RocksDB store
xxxx-store/Partition_6, received the following exception from RocksDB
org.rocksdb.RocksDBException: Invalid argument: Compression type Snappy is
not linked with the binary.
        at
org.apache.samza.storage.kv.RocksDbKeyValueStore$.openDB(RocksDbKeyValueStore.scala:81)

Edi

On Wed, Feb 10, 2016 at 9:41 AM, Edi Bice <edi.b...@gmail.com> wrote:

> Navina,
>
> I created a pull request some time back and it's still probably the
> easiest way to see what I changed. Let me know if you still prefer a patch
> file attachment.
>
> https://github.com/apache/samza/pull/6
>
> Thanks,
>
> Edi
>
> On Tue, Feb 9, 2016 at 6:09 PM, Navina Ramesh <
> nram...@linkedin.com.invalid> wrote:
>
>> Thanks, Edi.
>> Not sure why we are running into this problem yet again. Looks like it has
>> to be a combination of OS and java options to reproduce this, related to
>> SAMZA-836.
>> Can you please attach the patch that you were trying with the latest repo?
>> it could be a different code path in the library that is causing this
>> issue.
>>
>> Navina
>>
>> On Tue, Feb 9, 2016 at 2:06 PM, Edi Bice <edi.b...@gmail.com> wrote:
>>
>> > Hi Navina,
>> >
>> > I did indeed came across the long thread dealing with this. And yes,
>> things
>> > were working fine after the rocksdb upgrade to 3.13.1 which I'm still
>> > using. Not sure what I changed that's causing this. I did merge latest
>> from
>> > upstream into my repo as well as exposed some RocksDb config though the
>> > failing jobs are not using.
>> >
>> > I'm using Red Hat Enterprise Linux 6.6 (gcc 4.4) and Oracle Java 8.
>> >
>> > Did open a JIRA:
>> >
>> > https://issues.apache.org/jira/browse/SAMZA-870
>> >
>> > Edi
>> >
>> > On Mon, Feb 8, 2016 at 1:05 PM, Navina Ramesh
>> <nram...@linkedin.com.invalid
>> > >
>> > wrote:
>> >
>> > > Hi Edi,
>> > > I have seen this issue only before the rocksdb upgrade when the build
>> was
>> > > running on Linux boxes.
>> > >
>> > > What OS are you running on? And can you confirm the rocksdb version?
>> > >
>> > > If you are running on Linux with the correct version of rocksdb,
>> please
>> > > open a JIRA ticket. I will try to reproduce on my end.
>> > >
>> > > Thanks!
>> > > Navina
>> > >
>> > > On Mon, Feb 8, 2016 at 6:33 AM, Edi Bice <edi.b...@gmail.com> wrote:
>> > >
>> > > > I'm running the latest master with a small patch of mine allowing
>> for
>> > > > additional RocksDb config options. I deleted ~/.m2 and gradle cache
>> to
>> > > > avoid some issues I was running into. After rebuilding Samza, I'm
>> > getting
>> > > > the following exception on all my jobs:
>> > > >
>> > > > org.apache.samza.SamzaException: Error opening RocksDB store
>> > > > xxx/Partition_6, received the following exception from RocksDB
>> > > > org.rocksdb.RocksDBException: Invalid argument: Compression type
>> Snappy
>> > > is
>> > > > not linked with the binary.
>> > > >         at
>> > > >
>> > > >
>> > >
>> >
>> org.apache.samza.storage.kv.RocksDbKeyValueStore$.openDB(RocksDbKeyValueStore.scala:81)
>> > > >         at
>> > > >
>> > > >
>> > >
>> >
>> org.apache.samza.storage.kv.RocksDbKeyValueStore.db$lzycompute(RocksDbKeyValueStore.scala:103)
>> > > >         at
>> > > >
>> > > >
>> > >
>> >
>> org.apache.samza.storage.kv.RocksDbKeyValueStore.db(RocksDbKeyValueStore.scala:103)
>> > > >         at
>> > > >
>> > > >
>> > >
>> >
>> org.apache.samza.storage.kv.RocksDbKeyValueStore.flush(RocksDbKeyValueStore.scala:195)
>> > > >         at
>> > > > org.apache.samza.storage.kv.LoggedStore.flush(LoggedStore.scala:107)
>> > > >         at
>> > > >
>> > > >
>> > >
>> >
>> org.apache.samza.storage.kv.SerializedKeyValueStore.flush(SerializedKeyValueStore.scala:117)
>> > > >         at
>> > > > org.apache.samza.storage.kv.CachedStore.flush(CachedStore.scala:194)
>> > > >         at
>> > > >
>> > > >
>> > >
>> >
>> org.apache.samza.storage.kv.NullSafeKeyValueStore.flush(NullSafeKeyValueStore.scala:82)
>> > > >         at
>> > > >
>> > > >
>> > >
>> >
>> org.apache.samza.storage.kv.KeyValueStorageEngine.flush(KeyValueStorageEngine.scala:123)
>> > > >         at
>> > > >
>> > > >
>> > >
>> >
>> org.apache.samza.storage.kv.KeyValueStorageEngine.close(KeyValueStorageEngine.scala:135)
>> > > >         at
>> > > >
>> > > >
>> > >
>> >
>> org.apache.samza.storage.kv.KeyValueStorageEngine.stop(KeyValueStorageEngine.scala:129)
>> > > >         at
>> > > >
>> > > >
>> > >
>> >
>> org.apache.samza.storage.TaskStorageManager$$anonfun$stopStores$2.apply(TaskStorageManager.scala:193)
>> > > >         at
>> > > >
>> > > >
>> > >
>> >
>> org.apache.samza.storage.TaskStorageManager$$anonfun$stopStores$2.apply(TaskStorageManager.scala:193)
>> > > >         at
>> scala.collection.Iterator$class.foreach(Iterator.scala:727)
>> > > >         at
>> > scala.collection.AbstractIterator.foreach(Iterator.scala:1157)
>> > > >         at
>> > > >
>> >
>> scala.collection.MapLike$DefaultValuesIterable.foreach(MapLike.scala:206)
>> > > >         at
>> > > >
>> > > >
>> > >
>> >
>> org.apache.samza.storage.TaskStorageManager.stopStores(TaskStorageManager.scala:193)
>> > > >         at
>> > > >
>> > > >
>> > >
>> >
>> org.apache.samza.storage.TaskStorageManager.stop(TaskStorageManager.scala:197)
>> > > >         at
>> > > >
>> > > >
>> > >
>> >
>> org.apache.samza.container.TaskInstance.shutdownStores(TaskInstance.scala:203)
>> > > >         at
>> > > >
>> > > >
>> > >
>> >
>> org.apache.samza.container.SamzaContainer$$anonfun$shutdownStores$2.apply(SamzaContainer.scala:689)
>> > > >         at
>> > > >
>> > > >
>> > >
>> >
>> org.apache.samza.container.SamzaContainer$$anonfun$shutdownStores$2.apply(SamzaContainer.scala:689)
>> > > >         at
>> scala.collection.Iterator$class.foreach(Iterator.scala:727)
>> > > >         at
>> > scala.collection.AbstractIterator.foreach(Iterator.scala:1157)
>> > > >         at
>> > > >
>> >
>> scala.collection.MapLike$DefaultValuesIterable.foreach(MapLike.scala:206)
>> > > >         at
>> > > >
>> > > >
>> > >
>> >
>> org.apache.samza.container.SamzaContainer.shutdownStores(SamzaContainer.scala:689)
>> > > >         at
>> > > >
>> org.apache.samza.container.SamzaContainer.run(SamzaContainer.scala:568)
>> > > >         at
>> > > >
>> > > >
>> > >
>> >
>> org.apache.samza.container.SamzaContainer$.safeMain(SamzaContainer.scala:93)
>> > > >         at
>> > > >
>> > org.apache.samza.container.SamzaContainer$.main(SamzaContainer.scala:67)
>> > > >         at
>> > > > org.apache.samza.container.SamzaContainer.main(SamzaContainer.scala)
>> > > >
>> > >
>> > >
>> > >
>> > > --
>> > > Navina R.
>> > >
>> >
>>
>>
>>
>> --
>> Navina R.
>>
>
>

Reply via email to