Hello 1) yes its local only. The driver by default does connect to each host > though so its pretty trivial to have a load balancing policy that you can > direct to specific hosts (this should probably be in the driver so people > dont have to keep reimplementing it). >
The capability to target a specific host was added to the java driver (and others) recently in anticipation of Virtual Tables in version 3.6.0+ via Statement.setHost [1]. This will bypass the load balancing policy completely and send the request directly to that that Host (assuming it's connected). The drivers also parse virtual table metadata as well. [1]: https://docs.datastax.com/en/drivers/java/3.6/com/datastax/driver/core/Statement.html#setHost-com.datastax.driver.core.Host- Thanks! Andy On Mon, Mar 25, 2019 at 11:29 AM Sagar <sagarmeansoc...@gmail.com> wrote: > Thanks Chris. I got caught up with a few things and couldn't reply back. > So, I re-looked this again and I think virtual tables can be used for audit > logging. Considering that they don't have any replication - so we won't be > clogging the network with replication IO. > > In terms of storage, from what I understood, virtual tables don't have any > associated SSTables. So, is data stored only in Memtables? Can you please > shed some light on storage and the retention because of this? > > Lastly, the driver changes, I agree, we should make the driver be able to > contact to specific hosts with the correct LBP. If we do go this route, I > can start taking a look at it. > > Thanks! > Sagar. > > On Wed, Mar 6, 2019 at 10:42 PM Chris Lohfink <clohfin...@gmail.com> > wrote: > > > 1) yes its local only. The driver by default does connect to each host > > though so its pretty trivial to have a load balancing policy that you can > > direct to specific hosts (this should probably be in the driver so people > > dont have to keep reimplementing it). > > > > 2) yes, easiest way is to setup a whitelist load balancing policy like in > > cqlsh but like above. Best option is a custom LBP + StatementWrapper that > > holds the host target which can direct individual queries to specific > hosts > > > > 3) yes, cqlsh makes a connection to local C* instance with whitelist > policy > > so it only queries that one node. > > > > Chris > > > > On Wed, Mar 6, 2019 at 9:43 AM Sagar <sagarmeansoc...@gmail.com> wrote: > > > > > So, I went through the ticket for the creation of Virtual Tables(must > say > > > it was quite a long ticket spanning across 4 years). > > > > > > I see that there are a few tables created in the db.virtual package. > > These > > > appear to be metrics related tables. > > > > > > Couple of questions here: > > > > > > 1) Do all the tables pertain only data locally? What I mean is that in > a > > > cluster, each node will have its own ThreadPoolsTable pertaining to > > thread > > > pools on that node? Is that assumption correct? > > > 2) In terms of querying, again can we query only locally? I saw a lot > of > > > discussion on the ticket for where node = 1.2.3.4. I guess that isn't > > > supported? So. for any user to query for metrics of a given node, he > will > > > have to login and query on that node. > > > 3) Looks like these metrics are queryable via cqlsh? Is that statement > > > correct? > > > > > > Thanks! > > > Sagar. > > > > > > On Tue, Mar 5, 2019 at 7:30 AM Sagar <sagarmeansoc...@gmail.com> > wrote: > > > > > > > Right, Thanks Jonathan and Chris. > > > > > > > > Mean while, I would go through the 2 jira items to try and understand > > > > about virtual tables. > > > > > > > > Thanks! > > > > Sagar. > > > > > > > > On Tue, Mar 5, 2019 at 1:14 AM Jonathan Haddad <j...@jonhaddad.com> > > > wrote: > > > > > > > >> Sagar, > > > >> > > > >> There isn't going to be much in the way of docs, since it's brand > new > > > and > > > >> not really a public facing thing yet. As Chris pointed out, there's > > > other > > > >> work that would need to be done to work on virtual tables for large > > > >> datasets. > > > >> > > > >> Jon > > > >> > > > >> On Mon, Mar 4, 2019 at 6:42 AM Chris Lohfink <clohfin...@gmail.com> > > > >> wrote: > > > >> > > > >> > While you probably could put a virtual table wrapper over the > > binlogs, > > > >> you > > > >> > would want to wait for something like > > > >> > > https://urldefense.proofpoint.com/v2/url?u=https-3A__issues.apache.org_jira_browse_CASSANDRA-2D14629&d=DwIFaQ&c=adz96Xi0w1RHqtPMowiL2g&r=VHsWqsWT2MoX5jRZ0xZfdAWZxBsrn5KzowynGYCJaXE&m=Hm3Dmun0qF2plgG5tk2ihs3UcucypmoQY1YEE2vpsuE&s=MMYt6iqyvTZ5IilHTp0BhXswf-zCSN-xjXqIbC0IV_I&e= > 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://urldefense.proofpoint.com/v2/url?u=https-3A__issues.apache.org_jira_browse_CASSANDRA-2D7622&d=DwIFaQ&c=adz96Xi0w1RHqtPMowiL2g&r=VHsWqsWT2MoX5jRZ0xZfdAWZxBsrn5KzowynGYCJaXE&m=Hm3Dmun0qF2plgG5tk2ihs3UcucypmoQY1YEE2vpsuE&s=gMx2_o1_2qTFpyS4Lc_mO0wPNXKRH-gn8vO2bpJr2-o&e= > > > >> > > > > > >> > > 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 > > > >> > > > > https://urldefense.proofpoint.com/v2/url?u=http-3A__www.rustyrazorblade.com&d=DwIFaQ&c=adz96Xi0w1RHqtPMowiL2g&r=VHsWqsWT2MoX5jRZ0xZfdAWZxBsrn5KzowynGYCJaXE&m=Hm3Dmun0qF2plgG5tk2ihs3UcucypmoQY1YEE2vpsuE&s=2McQRIC_i0mUwuhRKH3M0fWYXD78djaxqePdOqpgah8&e= > > > >> > > > twitter: rustyrazorblade > > > >> > > > > > > >> > > > > > >> > > > > >> > > > >> > > > >> -- > > > >> Jon Haddad > > > >> > https://urldefense.proofpoint.com/v2/url?u=http-3A__www.rustyrazorblade.com&d=DwIFaQ&c=adz96Xi0w1RHqtPMowiL2g&r=VHsWqsWT2MoX5jRZ0xZfdAWZxBsrn5KzowynGYCJaXE&m=Hm3Dmun0qF2plgG5tk2ihs3UcucypmoQY1YEE2vpsuE&s=2McQRIC_i0mUwuhRKH3M0fWYXD78djaxqePdOqpgah8&e= > > > >> twitter: rustyrazorblade > > > >> > > > > > > > > > >