Re: Review Request 43053: allow warning instead of fail in case of invalid num of partitions in the checkpoint partition

2016-02-05 Thread Boris Shkolnik

---
This is an automatically generated e-mail. To reply, visit:
https://reviews.apache.org/r/43053/
---

(Updated Feb. 5, 2016, 9:48 p.m.)


Review request for samza.


Bugs: SAMZA-864
https://issues.apache.org/jira/browse/SAMZA-864


Repository: samza


Description
---

We have a validation code that verifies that checkpoint topic has the right 
number of partitions (1).
But, in some environments, it is difficult to repair or delete the invalid 
topic. 
This config will allow to by pass this validation (it will issue a warning 
only) and to continue with a checkpoint topic with an incorrect number of 
partitions. 
The checkpoints are written into partion 0.


Diffs
-

  samza-core/src/main/scala/org/apache/samza/config/JobConfig.scala 
1a8adae4d30fa198c90e8c177c7f17269c5953cd 
  
samza-kafka/src/main/scala/org/apache/samza/checkpoint/kafka/KafkaCheckpointManager.scala
 787de1f62479a098bf251f072fca03bbf92f7c6d 
  
samza-kafka/src/main/scala/org/apache/samza/checkpoint/kafka/KafkaCheckpointManagerFactory.scala
 7db894091284794b7f5fac164eb55b5d78184a36 
  
samza-kafka/src/main/scala/org/apache/samza/migration/KafkaCheckpointMigration.scala
 c6b1fe4bf3c3601502e014d582d86f8ea0850b20 
  samza-kafka/src/main/scala/org/apache/samza/util/KafkaUtil.scala 
f4311d1cda7c66c66544c5a3ac94a17cae62863a 
  
samza-kafka/src/test/scala/org/apache/samza/checkpoint/kafka/TestKafkaCheckpointManager.scala
 af4051b28df5eeaeaee527a24907a8e66441f743 

Diff: https://reviews.apache.org/r/43053/diff/


Testing
---


Thanks,

Boris Shkolnik



Help to vote on Samza talks in HadoopSummit

2016-02-05 Thread Yi Pan
Hi, all Samza lovers,

We recently submit a talk on Samza to Hadoop Summit this year. If you are
interested in seeing this talk in the conference, please help to vote on
the talk:
https://hadoopsummit.uservoice.com/forums/344973-iot-and-streaming/suggestions/11665164-lambda-less-stream-processing-scale-in-linkedin

Feedbacks and comments are also greatly appreciated!

Thanks!

-Yi


Review Request 43244: support elasticsearch version 2.x

2016-02-05 Thread Jiri Humpolicek

---
This is an automatically generated e-mail. To reply, visit:
https://reviews.apache.org/r/43244/
---

Review request for samza.


Repository: samza


Description
---

Samza elasticsearch system doesn't support elastic search 2.x because of 
breaking changes in java API: 
https://www.elastic.co/guide/en/elasticsearch/reference/current/breaking_20_java_api_changes.html
 . It will be nice to support that.


Diffs
-

  gradle/dependency-versions.gradle 52e25aa 
  
samza-elasticsearch/src/main/java/org/apache/samza/system/elasticsearch/client/NodeClientFactory.java
 0ee9e3f 
  
samza-elasticsearch/src/main/java/org/apache/samza/system/elasticsearch/client/TransportClientFactory.java
 7f8f3f3 
  
samza-elasticsearch/src/test/java/org/apache/samza/system/elasticsearch/indexrequest/DefaultIndexRequestFactoryTest.java
 61c3e7d 

Diff: https://reviews.apache.org/r/43244/diff/


Testing
---

It works for me with elastic search 2.1


Thanks,

Jiri Humpolicek



Re: ChangeLog Question for TTL rocksDB stores

2016-02-05 Thread Tao Feng
Hi David,

My understanding is that Samza changelog could still be logging those
entries deleted by RocksDB TTL. You could refer to SAMZA-677 for more
information.

Thanks,
-Tao

On Wed, Feb 3, 2016 at 5:46 PM, David Garcia 
wrote:

> Boris, thank you for the clarification.  But just to make sure I
> understand, is it correct to say that entries deleted by the TTL-policy in
> rocksDB will NOT be logged in the change-log?  My job processes a lot of
> data and saves a large portion of it to RocksDB (for reference later…but
> subject to a retention policy).  I need to ensure that rocksDB doesn’t
> grow uncontrollably.  If the TTL isn’t reflected in the changelog, then
> it’s quite possible that job restart will push too many messages into
> rocksDB.  Thx again for the help!
>
> -David
>
> On 2/3/16, 7:35 PM, "Boris Shkolnik"  wrote:
>
> >As Jacob mentioned there is not direct relationship between the rocksdb
> >tts
> >(internal to rocksdb) and changelog (done by Samza).
> >The problem may arise if the store is restored from the changelog, since
> >the log will have the expired entries, and they will be entered with the
> >NEW date (and as Yi mentioned, there is no TTL on kafka based changelogs
> >now).
> >But since it is not an error per se, SAMZA-862
> > has changed this
> message
> >to be a warning instead of an error.
> >
> >On Thu, Jan 28, 2016 at 11:51 AM, Yi Pan  wrote:
> >
> >> Hi, David,
> >>
> >> The "compaction" referred to together w/ TTL is referring to RocksDb's
> >> compaction, not the Kafka-based changelog topic. Currently, TTL is not
> >> applied to Kafka-based changelog topic. SAMZA-677 is opened for this.
> >>
> >> -Yi
> >>
> >> On Thu, Jan 28, 2016 at 11:36 AM, David Garcia
> >>  >> > wrote:
> >>
> >> > Ok, that makes sense.  I had assumed that the changelog was supported
> >> > because the docs mention that TTL is enforced upon ³compaction² (I had
> >> > assumed compaction of the DB changelog).  Which topic does the TTL
> >>policy
> >> > listen for the compaction of (since compaction policies of topics can
> >> > differ)?
> >> >
> >> > -David
> >> >
> >> > On 1/27/16, 8:46 PM, "Jacob Maes"  wrote:
> >> >
> >> > >Here's my understanding. The others can correct me if I'm mistaken.
> >> > >
> >> > >Samza provides the changelog functionality by intercepting RocksDB
> >>"put"
> >> > >and "delete" operations. However, TTL is managed by RocksDB
> >>internally
> >> and
> >> > >there aren't any hooks exposed in the RocksDB JNI. So there are 2
> >> problems
> >> > >that arise with TTL and change logging:
> >> > >1. Samza doesn't know when an entry expires, so it can't delete the
> >> > >expired
> >> > >entry from the changelog.
> >> > >2. The changelog currently has no concept of entry age/timestamp, so
> >> when
> >> > >the changelog is restored, it's unknown whether some subset (or all)
> >>of
> >> > >the
> >> > >entries should be immediately expired.
> >> > >
> >> > >These issues aren't insurmountable, but they weren't pursued for the
> >> > >initial implementation. Perhaps because there was a shortage of use
> >> cases
> >> > >that needed both TTL and changelogging, but I'm not sure.
> >> > >
> >> > >-Jake
> >> > >
> >> > >On Wed, Jan 27, 2016 at 6:19 PM, David Garcia
> >> > >
> >> > >wrote:
> >> > >
> >> > >> So, I saw this very scary message:
> >> > >>
> >> > >>
> >> > >> ERROR - e.kv.RocksDbKeyValueStore$ - sessionJoinStore is a TTL
> >>based
> >> > >> store, changelog is not supported for TTL based stores, use at your
> >> own
> >> > >> discretion
> >> > >>
> >> > >>
> >> > >>
> >> > >>
> >> > >> A few of questions:
> >> > >>
> >> > >> 1.) Does this mean that this store is NOT backed by the changelog?
> >> > >>
> >> > >> 2.) Provided that the store IS backed by a change log, do the TTL
> >> > >> expirations commit removals from the changelog (I.e.
> >> Nulls)...presumably
> >> > >> upon compaction
> >> > >>
> >> > >> 3.) Can I please get a bit more detail on how TTL affects a
> >>changelog
> >> > >> store?
> >> > >>
> >> > >>
> >> > >> -David
> >> > >>
> >> >
> >> >
> >>
>
>


Re: Review Request 43053: allow warning instead of fail in case of invalid num of partitions in the checkpoint partition

2016-02-05 Thread Boris Shkolnik


> On Feb. 3, 2016, 2:36 a.m., Jagadish Venkatraman wrote:
> > samza-kafka/src/test/scala/org/apache/samza/checkpoint/kafka/TestKafkaCheckpointManager.scala,
> >  line 84
> > 
> >
> > nit: failOnTopicValidation.

tnx, fixed.


- Boris


---
This is an automatically generated e-mail. To reply, visit:
https://reviews.apache.org/r/43053/#review117554
---


On Feb. 3, 2016, 1:41 a.m., Boris Shkolnik wrote:
> 
> ---
> This is an automatically generated e-mail. To reply, visit:
> https://reviews.apache.org/r/43053/
> ---
> 
> (Updated Feb. 3, 2016, 1:41 a.m.)
> 
> 
> Review request for samza.
> 
> 
> Repository: samza
> 
> 
> Description
> ---
> 
> We have a validation code that verifies that checkpoint topic has the right 
> number of partitions (1).
> But, in some environments, it is difficult to repair or delete the invalid 
> topic. 
> This config will allow to by pass this validation (it will issue a warning 
> only) and to continue with a checkpoint topic with an incorrect number of 
> partitions. 
> The checkpoints are written into partion 0.
> 
> 
> Diffs
> -
> 
>   samza-core/src/main/scala/org/apache/samza/config/JobConfig.scala 
> 1a8adae4d30fa198c90e8c177c7f17269c5953cd 
>   
> samza-kafka/src/main/scala/org/apache/samza/checkpoint/kafka/KafkaCheckpointManager.scala
>  787de1f62479a098bf251f072fca03bbf92f7c6d 
>   
> samza-kafka/src/main/scala/org/apache/samza/checkpoint/kafka/KafkaCheckpointManagerFactory.scala
>  7db894091284794b7f5fac164eb55b5d78184a36 
>   
> samza-kafka/src/main/scala/org/apache/samza/migration/KafkaCheckpointMigration.scala
>  c6b1fe4bf3c3601502e014d582d86f8ea0850b20 
>   samza-kafka/src/main/scala/org/apache/samza/util/KafkaUtil.scala 
> f4311d1cda7c66c66544c5a3ac94a17cae62863a 
>   
> samza-kafka/src/test/scala/org/apache/samza/checkpoint/kafka/TestKafkaCheckpointManager.scala
>  af4051b28df5eeaeaee527a24907a8e66441f743 
> 
> Diff: https://reviews.apache.org/r/43053/diff/
> 
> 
> Testing
> ---
> 
> 
> Thanks,
> 
> Boris Shkolnik
> 
>



Re: Review Request 43074: SAMZA-866 Refactor and fix Container allocation logic.

2016-02-05 Thread Navina Ramesh

---
This is an automatically generated e-mail. To reply, visit:
https://reviews.apache.org/r/43074/#review118073
---


Ship it!





samza-yarn/src/main/java/org/apache/samza/job/yarn/ContainerAllocator.java 
(line 54)


Minor nit:
I think the max character line limit is 120 character. Ideally, it should 
be :
```java
while (!containerRequestState.getRequestsQueue() &&
   allocatedContainers != null &&
   allocatedContainers.size() > 0) {
   ...
}
```


- Navina Ramesh


On Feb. 3, 2016, 8:04 p.m., Jake Maes wrote:
> 
> ---
> This is an automatically generated e-mail. To reply, visit:
> https://reviews.apache.org/r/43074/
> ---
> 
> (Updated Feb. 3, 2016, 8:04 p.m.)
> 
> 
> Review request for samza, Navina Ramesh and Yi Pan (Data Infrastructure).
> 
> 
> Repository: samza
> 
> 
> Description
> ---
> 
> Bug:  
> ContainerAllocator and HostAwareContainerAllocator have different try-catch 
> logic in their run() methods. The HostAwareContainerAllocator try-catch is 
> outside the while loop, which would cause it to stop allocating containers in 
> the event of an exception. ContainerAllocator correctly has the try-catch 
> inside the loop
>   
> Fix:  
> * Refactor the loop and try-catch to a common run() method in 
> AbstractContainerAllocator  
> * Change the log type to WARN for the general exception case  
> * Refactor some duplicate code in ContainerRequestState
> 
> 
> Diffs
> -
> 
>   
> samza-yarn/src/main/java/org/apache/samza/job/yarn/AbstractContainerAllocator.java
>  9ee2daccc3c44202308637207a084def81b49c09 
>   samza-yarn/src/main/java/org/apache/samza/job/yarn/ContainerAllocator.java 
> 7c57a866114aabf76a36d3b7ca4c5810628e0c77 
>   
> samza-yarn/src/main/java/org/apache/samza/job/yarn/ContainerRequestState.java 
> ab3061eae2cfc2da6681ce2034492b165d0d8b96 
>   
> samza-yarn/src/main/java/org/apache/samza/job/yarn/HostAwareContainerAllocator.java
>  ff22dbfe5537ba2a3c55c4e6063680ff4c9e55f4 
> 
> Diff: https://reviews.apache.org/r/43074/diff/
> 
> 
> Testing
> ---
> 
> Unit tests still pass.
> 
> 
> Thanks,
> 
> Jake Maes
> 
>



Re: Need Help with Samza - Changelog System

2016-02-05 Thread Jagadish Venkatraman
Another pointer is to run the job with DEBUG level turned on and share the
logs with us.

On Fri, Feb 5, 2016 at 5:05 PM, Jagadish Venkatraman  wrote:

> Hey Jason,
>
> Can you share the entire container log? It will be useful to find out what
> went wrong.
> If this is a non-yarn, it will be also useful if you share the JobRunner
> logs.
>
> Thanks,
> Jagadish
>
> On Fri, Feb 5, 2016 at 4:33 PM, Jason Erickson 
> wrote:
>
>> Is the Kafka producer configuration different than the Samza configuration
>> of the Samza task that references the store? If not, here is are those
>> configuration values.  The changelog in question is
>> resourceStore-changelog.
>>
>> # Job
>> job.factory.class=org.apache.samza.job.local.ThreadJobFactory
>> job.name=resource_normalizer
>>
>> job.coordinator.system=kafka
>>
>> # Task
>>
>> task.class=com.foo.blazer.resource.normalizer.samza.ResourceNormalizerSamzaTask
>> task.inputs=kafka.com.foo.iam.indexing.resource.mutation
>> task.window.ms=1
>>
>> task.checkpoint.factory=org.apache.samza.checkpoint.kafka.KafkaCheckpointManagerFactory
>> task.checkpoint.system=kafka
>> # Normally, this would be 3, but we have only one broker.
>> task.checkpoint.replication.factor=3
>> task.checkpoint.skip-migration=true
>>
>> # Serializers
>>
>> serializers.registry.byte.class=org.apache.samza.serializers.ByteSerdeFactory
>>
>> serializers.registry.entity.class=com.foo.blazer.resource.normalizer.serde.ResourceEventEntitySerdeFactory
>>
>> serializers.registry.string.class=org.apache.samza.serializers.StringSerdeFactory
>>
>> serializers.registry.int.class=org.apache.samza.serializers.IntegerSerdeFactory
>>
>> serializers.registry.metrics.class=org.apache.samza.serializers.MetricsSnapshotSerdeFactory
>>
>> # Systems
>>
>> systems.kafka.samza.factory=org.apache.samza.system.kafka.KafkaSystemFactory
>> systems.kafka.samza.msg.serde=byte
>> systems.kafka.samza.offset.default=oldest
>> systems.kafka.consumer.zookeeper.connect=${ZK_NODES}/${ZK_ROOT}
>> systems.kafka.consumer.auto.offset.reset=smallest
>> systems.kafka.producer.bootstrap.servers=${KAFKA_NODES}
>> systems.kafka.producer.max.request.size=52428800
>> systems.kafka.streams.metrics.samza.msg.serde=metrics
>>
>> # Metrics
>>
>> metrics.reporter.snapshot.class=org.apache.samza.metrics.reporter.MetricsSnapshotReporterFactor
>> metrics.reporter.snapshot.stream=kafka.metrics
>>
>> metrics.reporter.jmx.class=org.apache.samza.metrics.reporter.JmxReporterFactory
>> metrics.reporters=snapshot,jmx
>>
>> # Stores
>>
>> stores.resourceStore.factory=org.apache.samza.storage.kv.RocksDbKeyValueStorageEngineFactory
>> stores.resourceStore.changelog=kafka.resourceStore-changelog
>> stores.resourceStore.changelog.replication.factor=3
>>
>> stores.resourceStore.key.serde=string
>> stores.resourceStore.msg.serde=entity
>>
>>
>> stores.deletedFlagStore.factory=org.apache.samza.storage.kv.RocksDbKeyValueStorageEngineFactory
>> stores.deletedFlagStore.changelog=kafka.deletedFlagStore-changelog
>> stores.deletedFlagStore.changelog.replication.factor=3
>>
>> stores.deletedFlagStore.key.serde=string
>> stores.deletedFlagStore.msg.serde=int
>>
>>
>>
>> We do not get a stack trace from the Samza task itself, it just never
>> seems
>> to fully start.  However if we use kafka-console-consumer to try to
>> examine
>> the changelog we get this:
>>
>> 2016-02-02 22:10:11,252] ERROR Error processing message, terminating
>> consumer process:  (kafka.tools.ConsoleConsumer$)
>> kafka.common.MessageSizeTooLargeException: Found a message larger than the
>> maximum fetch size of this consumer on topic resourceStore-changelog
>> partition 7 at fetch offset 0. Increase the fetch size, or decrease the
>> maximum message size the broker will allow.
>> at kafka.consumer.ConsumerIterator.makeNext(ConsumerIterator.scala:90)
>> at kafka.consumer.ConsumerIterator.makeNext(ConsumerIterator.scala:33)
>> at
>> kafka.utils.IteratorTemplate.maybeComputeNext(IteratorTemplate.scala:66)
>> at kafka.utils.IteratorTemplate.hasNext(IteratorTemplate.scala:58)
>> at kafka.consumer.OldConsumer.receive(BaseConsumer.scala:79)
>> at kafka.tools.ConsoleConsumer$.process(ConsoleConsumer.scala:110)
>> at kafka.tools.ConsoleConsumer$.run(ConsoleConsumer.scala:69)
>> at kafka.tools.ConsoleConsumer$.main(ConsoleConsumer.scala:47)
>> at kafka.tools.ConsoleConsumer.main(ConsoleConsumer.scala)
>>
>>
>> On Fri, Feb 5, 2016 at 3:42 PM, Ramesh Bhojan 
>> wrote:
>>
>> > Jason,
>> > Can we please share more information about the exact stack trace and the
>> > job configuration, especially the Kafka producer configuration for the
>> > changelog system, as requested by Yi Pan?
>> >
>> > Regards,
>> > Ramesh
>> >
>> > On Thu, Feb 4, 2016 at 11:56 AM, Ramesh Bhojan <
>> rbhojan.soc...@gmail.com>
>> > wrote:
>> >
>> >> Dear team @ Samza,
>> >> I would really appreciate some help with the following question posted
>> in
>> >> Stack Overflow :
>> >>

Re: Need Help with Samza - Changelog System

2016-02-05 Thread Jagadish Venkatraman
Hey Jason,

Can you share the entire container log? It will be useful to find out what
went wrong.
If this is a non-yarn, it will be also useful if you share the JobRunner
logs.

Thanks,
Jagadish

On Fri, Feb 5, 2016 at 4:33 PM, Jason Erickson  wrote:

> Is the Kafka producer configuration different than the Samza configuration
> of the Samza task that references the store? If not, here is are those
> configuration values.  The changelog in question is
> resourceStore-changelog.
>
> # Job
> job.factory.class=org.apache.samza.job.local.ThreadJobFactory
> job.name=resource_normalizer
>
> job.coordinator.system=kafka
>
> # Task
>
> task.class=com.foo.blazer.resource.normalizer.samza.ResourceNormalizerSamzaTask
> task.inputs=kafka.com.foo.iam.indexing.resource.mutation
> task.window.ms=1
>
> task.checkpoint.factory=org.apache.samza.checkpoint.kafka.KafkaCheckpointManagerFactory
> task.checkpoint.system=kafka
> # Normally, this would be 3, but we have only one broker.
> task.checkpoint.replication.factor=3
> task.checkpoint.skip-migration=true
>
> # Serializers
>
> serializers.registry.byte.class=org.apache.samza.serializers.ByteSerdeFactory
>
> serializers.registry.entity.class=com.foo.blazer.resource.normalizer.serde.ResourceEventEntitySerdeFactory
>
> serializers.registry.string.class=org.apache.samza.serializers.StringSerdeFactory
>
> serializers.registry.int.class=org.apache.samza.serializers.IntegerSerdeFactory
>
> serializers.registry.metrics.class=org.apache.samza.serializers.MetricsSnapshotSerdeFactory
>
> # Systems
>
> systems.kafka.samza.factory=org.apache.samza.system.kafka.KafkaSystemFactory
> systems.kafka.samza.msg.serde=byte
> systems.kafka.samza.offset.default=oldest
> systems.kafka.consumer.zookeeper.connect=${ZK_NODES}/${ZK_ROOT}
> systems.kafka.consumer.auto.offset.reset=smallest
> systems.kafka.producer.bootstrap.servers=${KAFKA_NODES}
> systems.kafka.producer.max.request.size=52428800
> systems.kafka.streams.metrics.samza.msg.serde=metrics
>
> # Metrics
>
> metrics.reporter.snapshot.class=org.apache.samza.metrics.reporter.MetricsSnapshotReporterFactor
> metrics.reporter.snapshot.stream=kafka.metrics
>
> metrics.reporter.jmx.class=org.apache.samza.metrics.reporter.JmxReporterFactory
> metrics.reporters=snapshot,jmx
>
> # Stores
>
> stores.resourceStore.factory=org.apache.samza.storage.kv.RocksDbKeyValueStorageEngineFactory
> stores.resourceStore.changelog=kafka.resourceStore-changelog
> stores.resourceStore.changelog.replication.factor=3
>
> stores.resourceStore.key.serde=string
> stores.resourceStore.msg.serde=entity
>
>
> stores.deletedFlagStore.factory=org.apache.samza.storage.kv.RocksDbKeyValueStorageEngineFactory
> stores.deletedFlagStore.changelog=kafka.deletedFlagStore-changelog
> stores.deletedFlagStore.changelog.replication.factor=3
>
> stores.deletedFlagStore.key.serde=string
> stores.deletedFlagStore.msg.serde=int
>
>
>
> We do not get a stack trace from the Samza task itself, it just never seems
> to fully start.  However if we use kafka-console-consumer to try to examine
> the changelog we get this:
>
> 2016-02-02 22:10:11,252] ERROR Error processing message, terminating
> consumer process:  (kafka.tools.ConsoleConsumer$)
> kafka.common.MessageSizeTooLargeException: Found a message larger than the
> maximum fetch size of this consumer on topic resourceStore-changelog
> partition 7 at fetch offset 0. Increase the fetch size, or decrease the
> maximum message size the broker will allow.
> at kafka.consumer.ConsumerIterator.makeNext(ConsumerIterator.scala:90)
> at kafka.consumer.ConsumerIterator.makeNext(ConsumerIterator.scala:33)
> at kafka.utils.IteratorTemplate.maybeComputeNext(IteratorTemplate.scala:66)
> at kafka.utils.IteratorTemplate.hasNext(IteratorTemplate.scala:58)
> at kafka.consumer.OldConsumer.receive(BaseConsumer.scala:79)
> at kafka.tools.ConsoleConsumer$.process(ConsoleConsumer.scala:110)
> at kafka.tools.ConsoleConsumer$.run(ConsoleConsumer.scala:69)
> at kafka.tools.ConsoleConsumer$.main(ConsoleConsumer.scala:47)
> at kafka.tools.ConsoleConsumer.main(ConsoleConsumer.scala)
>
>
> On Fri, Feb 5, 2016 at 3:42 PM, Ramesh Bhojan 
> wrote:
>
> > Jason,
> > Can we please share more information about the exact stack trace and the
> > job configuration, especially the Kafka producer configuration for the
> > changelog system, as requested by Yi Pan?
> >
> > Regards,
> > Ramesh
> >
> > On Thu, Feb 4, 2016 at 11:56 AM, Ramesh Bhojan  >
> > wrote:
> >
> >> Dear team @ Samza,
> >> I would really appreciate some help with the following question posted
> in
> >> Stack Overflow :
> >>
> >>
> >>
> http://stackoverflow.com/questions/35168641/is-there-a-configuration-setting-to-allow-large-values-in-my-samza-store
> >>
> >> Thanks,
> >> Ramesh
> >>
> >
> >
>
>
> --
> Thanks,
>
> Jason Erickson
>



-- 
Jagadish V,
Graduate Student,
Department of Computer Science,
Stanford University