While you probably could put a virtual table wrapper over the binlogs, you would want to wait for something like https://issues.apache.org/jira/browse/CASSANDRA-14629 to get in so you would not OOM. The current virtual table implementation requires you have the entire result set to be returned at once.
Chris On Mon, Mar 4, 2019 at 5:29 AM Sagar <sagarmeansoc...@gmail.com> wrote: > Hi Jonathan, > > I couldn't find much literature on Virtual tables apart from this ticket: > > https://issues.apache.org/jira/browse/CASSANDRA-7622 > > Any insights would be helpful. > > Thanks! > Sagar. > > On Sat, Mar 2, 2019 at 7:23 AM Jonathan Haddad <j...@jonhaddad.com> wrote: > > > Instead of logging to tables, putting a virtual table around the audit / > > query logs might be an option. Same with the commit log for cdc > > > > On Fri, Mar 1, 2019 at 5:25 PM Sagar <sagarmeansoc...@gmail.com> wrote: > > > > > Thanks all for the pointers. Really insightful. > > > > > > Subroto I think that’s part of the enterprise version but yeah even I > > have > > > seen it. Again not sure of the performance implications. > > > > > > Sagar. > > > > > > On Sat, 2 Mar 2019 at 5:15 AM, Subroto Barua > <sbarua...@yahoo.com.invalid > > > > > > wrote: > > > > > > > Datastax version has an option to store audit info to > > dse_audit.audit_log > > > > table; I do not know the performance impact since I use the file > option > > > > > > > > Subroto > > > > > > > > > On Mar 1, 2019, at 9:40 AM, Jeremiah D Jordan < > > > jeremiah.jor...@gmail.com> > > > > wrote: > > > > > > > > > > AFAIK the Full Query Logging binary format was already made more > > > general > > > > in order to support using that format for the audit logging. > > > > > > > > > > -Jeremiah > > > > > > > > > >> On Mar 1, 2019, at 11:38 AM, Joshua McKenzie < > jmcken...@apache.org> > > > > wrote: > > > > >> > > > > >> Is there a world in which a general purpose, side-channel file > > storage > > > > >> format for transient things like this (hints, batches, audit logs, > > > etc) > > > > >> could be useful as a first class citizen in the codebase? i.e. a > > world > > > > in > > > > >> which we refactored some of the hints-specific reader/writer code > to > > > be > > > > >> used for things like this if/when they come up? > > > > >> > > > > >>> On Thu, Feb 28, 2019 at 12:04 PM Jonathan Haddad < > > j...@jonhaddad.com > > > > <mailto:j...@jonhaddad.com>> wrote: > > > > >>> > > > > >>> Agreed with Dinesh and Josh. I would *never* put the audit log > > back > > > in > > > > >>> Cassandra. > > > > >>> > > > > >>> This is extendable, Sagar, so you're free to do as you want, but > > I'm > > > > very > > > > >>> opposed to putting a ticking time bomb in Cassandra proper. > > > > >>> > > > > >>> Jon > > > > >>> > > > > >>> > > > > >>> On Thu, Feb 28, 2019 at 8:38 AM Dinesh Joshi > > > > <djos...@icloud.com.invalid> > > > > >>> wrote: > > > > >>> > > > > >>>> I strongly echo Josh’s sentiment. Imagine losing audit entries > > > > because C* > > > > >>>> is overloaded? It’s fine if you don’t care about losing audit > > > entries. > > > > >>>> > > > > >>>> Dinesh > > > > >>>> > > > > >>>>> On Feb 28, 2019, at 6:41 AM, Joshua McKenzie < > > jmcken...@apache.org > > > > > > > > >>>> wrote: > > > > >>>>> > > > > >>>>> One of the things we've run into historically, on a *lot* of > > axes, > > > is > > > > >>>> that > > > > >>>>> "just put it in C*" for various functionality looks great from > a > > > user > > > > >>> and > > > > >>>>> usability perspective, and proves to be something of a > nightmare > > > from > > > > >>> an > > > > >>>>> admin / cluster behavior perspective. > > > > >>>>> > > > > >>>>> i.e. - cluster suffering so you're writing hints? Write them to > > C* > > > > >>> tables > > > > >>>>> and watch the cluster suffer more! :) > > > > >>>>> Same thing probably holds true for audit logging - at a time > > frame > > > > when > > > > >>>>> things are getting hairy w/a cluster, if you're writing that > > audit > > > > >>>> logging > > > > >>>>> into C* proper (and dealing with ser/deser, compaction > pressure, > > > > >>> flushing > > > > >>>>> pressure, etc) from that, there's a compounding effect of > > pressure > > > > and > > > > >>>> pain > > > > >>>>> on the cluster. > > > > >>>>> > > > > >>>>> So the TL;DR we as a project kind of philosophically have been > > > moving > > > > >>>>> towards (I think that's valid to say?) is: use C* for the > things > > > it's > > > > >>>>> absolutely great at, and try to side-channel other recovery > > > > operations > > > > >>> as > > > > >>>>> much as you can (see: file-based hints) to stay out of its way. > > > > >>>>> > > > > >>>>> Same thing held true w/design of CDC - I debated "materialize > in > > > > memory > > > > >>>> for > > > > >>>>> consumer to take over socket", and "keep the data in another C* > > > > table", > > > > >>>> but > > > > >>>>> the ramifications to perf and core I/O operations in C* the > > moment > > > > >>> things > > > > >>>>> start to go badly were significant enough that the route we > went > > > was > > > > >>> "do > > > > >>>> no > > > > >>>>> harm". For better or for worse, as there's obvious tradeoffs > > there. > > > > >>>>> > > > > >>>>>> On Thu, Feb 28, 2019 at 7:46 AM Sagar < > > sagarmeansoc...@gmail.com> > > > > >>>> wrote: > > > > >>>>>> > > > > >>>>>> Thanks all for the pointers. > > > > >>>>>> > > > > >>>>>> @Joseph, > > > > >>>>>> > > > > >>>>>> I have gone through the links shared by you. Also, I have been > > > > looking > > > > >>>> at > > > > >>>>>> the code base. > > > > >>>>>> > > > > >>>>>> I understand the fact that pushing the logs to ES or Solr is a > > lot > > > > >>>> easier > > > > >>>>>> to do. Having said that, the only reason I thought having > > > something > > > > >>> like > > > > >>>>>> this might help is, if I don't want to add more pieces and > still > > > > >>>> provide a > > > > >>>>>> central piece of audit logging within Cassandra itself and > still > > > be > > > > >>>>>> queryable. > > > > >>>>>> > > > > >>>>>> In terms of usages, one of them could definitely be CDC > related > > > use > > > > >>>> cases. > > > > >>>>>> With data being stored in tables and being queryable, it can > > > become > > > > a > > > > >>>> lot > > > > >>>>>> more easier to expose this data to external systems like Kafka > > > > >>> Connect, > > > > >>>>>> Debezium which have the ability to push data to Kafka for > > example. > > > > >>> Note > > > > >>>>>> that pushing data to Kafka is just an example, but what I mean > > is, > > > > if > > > > >>> we > > > > >>>>>> can have data in tables, then instead of everyone writing > custom > > > > >>> custom > > > > >>>>>> loggers, they can hook into this table info and take action. > > > > >>>>>> > > > > >>>>>> Regarding the infinite loop question, I have done some > analysis, > > > and > > > > >>> in > > > > >>>> my > > > > >>>>>> opinion, instead of tweaking the behaviour of Binlog and the > way > > > it > > > > >>>>>> functions currently, we can actually spin up another tailer > > thread > > > > to > > > > >>>> the > > > > >>>>>> same Chronicle Queue which can do the needful. This way the > > config > > > > >>>> options > > > > >>>>>> etc all remain the same(apart from the logger ofcourse). > > > > >>>>>> > > > > >>>>>> Let me know if any of it makes sense :D > > > > >>>>>> > > > > >>>>>> Thanks! > > > > >>>>>> Sagar. > > > > >>>>>> > > > > >>>>>> > > > > >>>>>> On Thu, Feb 28, 2019 at 1:09 AM Dinesh Joshi > > > > >>> <djos...@icloud.com.invalid > > > > >>>>> > > > > >>>>>> wrote: > > > > >>>>>> > > > > >>>>>>> > > > > >>>>>>> > > > > >>>>>>>> On Feb 27, 2019, at 10:41 AM, Joseph Lynch < > > > joe.e.ly...@gmail.com > > > > > > > > > >>>>>>> wrote: > > > > >>>>>>>> > > > > >>>>>>>> Vinay can confirm, but as far as I am aware we have no > current > > > > plans > > > > >>>> to > > > > >>>>>>>> implement audit logging to a table directly, but the > > > > implementation > > > > >>> is > > > > >>>>>>>> fully pluggable (like compaction, compression, etc ...). > Check > > > out > > > > >>> the > > > > >>>>>>> blog > > > > >>>>>>>> post [1] and documentation [2] Vinay wrote for more details, > > but > > > > the > > > > >>>>>>> short > > > > >>>>>>> > > > > >>>>>>> +1. I am still curious as to why you'd want to store audit > log > > > > >>> entries > > > > >>>>>>> back in Cassandra? Depending on the scale it can generate a > lot > > > of > > > > >>> load > > > > >>>>>> and > > > > >>>>>>> I think you'd end up in an infinite loop because as you're > > > > inserting > > > > >>>> the > > > > >>>>>>> audit log entry you'll generate a new one and so on unless > you > > > > black > > > > >>>> list > > > > >>>>>>> audits to that table / keyspace. > > > > >>>>>>> > > > > >>>>>>> Ideally you'd insert this data into ElasticSearch / Solr or > > some > > > > >>> other > > > > >>>>>>> place that can be then used for analytics or search. > > > > >>>>>>> > > > > >>>>>>> Dinesh > > > > >>>>>>> > > > > --------------------------------------------------------------------- > > > > >>>>>>> To unsubscribe, e-mail: dev-unsubscr...@cassandra.apache.org > > > > >>>>>>> For additional commands, e-mail: > dev-h...@cassandra.apache.org > > > > >>>>>>> > > > > >>>>>>> > > > > >>>>>> > > > > >>>> > > > > >>>> > > > > >>>> > > > --------------------------------------------------------------------- > > > > >>>> To unsubscribe, e-mail: dev-unsubscr...@cassandra.apache.org > > > > >>>> For additional commands, e-mail: dev-h...@cassandra.apache.org > > > > >>>> > > > > >>>> > > > > >>> > > > > >>> -- > > > > >>> Jon Haddad > > > > >>> > > > > > > > > > > https://urldefense.proofpoint.com/v2/url?u=http-3A__www.rustyrazorblade.com&d=DwIFaQ&c=adz96Xi0w1RHqtPMowiL2g&r=CNZK3RiJDLqhsZDG6FQGnXn8WyPRCQhp4x_uBICNC0g&m=vyXA1unA3gpHGCpKOfRurmET3jOHaV2bjs1mHVVsb2U&s=EDg90XhABktX19m4FaDHKIjFaU2YAHbXjeEGk7Jx6dk&e= > > > > < > > > > > > > > > > https://urldefense.proofpoint.com/v2/url?u=http-3A__www.rustyrazorblade.com&d=DwIFaQ&c=adz96Xi0w1RHqtPMowiL2g&r=CNZK3RiJDLqhsZDG6FQGnXn8WyPRCQhp4x_uBICNC0g&m=vyXA1unA3gpHGCpKOfRurmET3jOHaV2bjs1mHVVsb2U&s=EDg90XhABktX19m4FaDHKIjFaU2YAHbXjeEGk7Jx6dk&e= > > > > > > > > > >>> twitter: rustyrazorblade > > > > > > > > > > > > > > > > > --------------------------------------------------------------------- > > > > To unsubscribe, e-mail: dev-unsubscr...@cassandra.apache.org > > > > For additional commands, e-mail: dev-h...@cassandra.apache.org > > > > > > > > > > > > > -- > > Jon Haddad > > http://www.rustyrazorblade.com > > twitter: rustyrazorblade > > >