diff --git a/Cargo.toml b/Cargo.toml index 8d57b74..0ff6cb1 100644 --- a/Cargo.toml +++ b/Cargo.toml @@ -158,7 +158,7 @@ bnum = "0.13.0" clickhouse-macros = { version = "0.3.0", path = "macros" } criterion = "0.6" serde = { version = "1.0.106", features = ["derive"] } -tokio = { version = "1.0.1", features = ["full", "test-util"] } +tokio = { version = "1.0.1", features = ["full", "test-util", "io-util"] } hyper = { version = "1.1", features = ["server"] } indexmap = { version = "2.10.0", features = ["serde"] } linked-hash-map = { version = "0.5.6", features = ["serde_impl"] } diff --git a/src/insert_formatted.rs b/src/insert_formatted.rs new file mode 100644 index 0000000..c983eca --- /dev/null +++ b/src/insert_formatted.rs @@ -0,0 +1,656 @@ +use crate::headers::{with_authentication, with_request_headers}; +use crate::{ + Client, Compression, + error::{Error, Result}, + request_body::{ChunkSender, RequestBody}, + response::Response, + settings, +}; +use bytes::{Bytes, BytesMut}; +use hyper::{self, Request}; +use std::ops::ControlFlow; +use std::task::{Context, Poll, ready}; +use std::{cmp, future::Future, io, mem, panic, pin::Pin, time::Duration}; +use tokio::io::AsyncWrite; +use tokio::{ + task::JoinHandle, + time::{Instant, Sleep}, +}; +use url::Url; + +#[cfg(feature = "lz4")] +pub use compression::CompressedData; + +// The desired max frame size. +const BUFFER_SIZE: usize = 256 * 1024; + +/// Performs one `INSERT`, sending pre-formatted data. +/// +/// The [`InsertFormatted::end`] method must be called to finalize the `INSERT`. +/// Otherwise, the whole `INSERT` will be aborted. +/// +/// Rows are sent progressively to spread network load. +/// +/// # Note: Not Validated +/// Unlike [`Insert`][crate::insert::Insert] and [`Inserter`][crate::inserter::Inserter], +/// this does not perform any validation on the submitted data. +/// +/// Only the use of self-describing formats (e.g. CSV, TabSeparated, JSON) is recommended. +/// +/// See the [list of supported formats](https://clickhouse.com/docs/interfaces/formats) +/// for details. +#[must_use] +pub struct InsertFormatted { + state: InsertState, + #[cfg(feature = "lz4")] + compression: Compression, + send_timeout: Option, + end_timeout: Option, + // Use boxed `Sleep` to reuse a timer entry, it improves performance. + // Also, `tokio::time::timeout()` significantly increases a future's size. + sleep: Pin>, +} + +struct Timeout { + duration: Duration, + is_set: bool, +} + +enum InsertState { + NotStarted { + client: Box, + sql: String, + }, + Active { + sender: ChunkSender, + handle: JoinHandle>, + }, + Terminated { + handle: JoinHandle>, + }, + Completed, +} + +impl InsertState { + #[inline(always)] + fn is_not_started(&self) -> bool { + matches!(self, Self::NotStarted { .. }) + } + + fn sender(&mut self) -> Option<&mut ChunkSender> { + match self { + InsertState::Active { sender, .. } => Some(sender), + _ => None, + } + } + + fn handle(&mut self) -> Option<&mut JoinHandle>> { + match self { + InsertState::Active { handle, .. } | InsertState::Terminated { handle } => Some(handle), + _ => None, + } + } + + fn client_with_sql(&self) -> Option<(&Client, &str)> { + match self { + InsertState::NotStarted { client, sql } => Some((client, sql)), + _ => None, + } + } + + #[inline] + fn expect_client_mut(&mut self) -> &mut Client { + let Self::NotStarted { client, .. } = self else { + panic!("cannot modify client options while an insert is in-progress") + }; + + client + } + + fn terminated(&mut self) { + match mem::replace(self, InsertState::Completed) { + InsertState::NotStarted { .. } | InsertState::Completed => (), + InsertState::Active { handle, .. } => { + *self = InsertState::Terminated { handle }; + } + InsertState::Terminated { handle } => { + *self = InsertState::Terminated { handle }; + } + } + } +} + +impl InsertFormatted { + pub(crate) fn new(client: &Client, sql: String) -> Self { + Self { + state: InsertState::NotStarted { + client: Box::new(client.clone()), + sql, + }, + #[cfg(feature = "lz4")] + compression: client.compression, + send_timeout: None, + end_timeout: None, + sleep: Box::pin(tokio::time::sleep(Duration::new(0, 0))), + } + } + + /// Sets timeouts for different operations. + /// + /// `send_timeout` restricts time on sending a data chunk to a socket. + /// `None` disables the timeout, it's a default. + /// It's roughly equivalent to `tokio::time::timeout(insert.write(...))`. + /// + /// `end_timeout` restricts time on waiting for a response from the CH + /// server. Thus, it includes all work needed to handle `INSERT` by the + /// CH server, e.g. handling all materialized views and so on. + /// `None` disables the timeout, it's a default. + /// It's roughly equivalent to `tokio::time::timeout(insert.end(...))`. + /// + /// These timeouts are much more performant (~x10) than wrapping `write()` + /// and `end()` calls into `tokio::time::timeout()`. + pub fn with_timeouts( + mut self, + send_timeout: Option, + end_timeout: Option, + ) -> Self { + self.set_timeouts(send_timeout, end_timeout); + self + } + + /// Configure the [roles] to use when executing `INSERT` statements. + /// + /// Overrides any roles previously set by this method, [`InsertFormatted::with_option`], + /// [`Client::with_roles`] or [`Client::with_option`]. + /// + /// An empty iterator may be passed to clear the set roles. + /// + /// [roles]: https://clickhouse.com/docs/operations/access-rights#role-management + /// + /// # Panics + /// If called after the request is started, e.g., after [`InsertFormatted::send`]. + pub fn with_roles(mut self, roles: impl IntoIterator>) -> Self { + self.state.expect_client_mut().set_roles(roles); + self + } + + /// Clear any explicit [roles] previously set on this `Insert` or inherited from [`Client`]. + /// + /// Overrides any roles previously set by [`InsertFormatted::with_roles`], [`InsertFormatted::with_option`], + /// [`Client::with_roles`] or [`Client::with_option`]. + /// + /// [roles]: https://clickhouse.com/docs/operations/access-rights#role-management + /// + /// # Panics + /// If called after the request is started, e.g., after [`InsertFormatted::send`]. + pub fn with_default_roles(mut self) -> Self { + self.state.expect_client_mut().clear_roles(); + self + } + + /// Similar to [`Client::with_option`], but for this particular INSERT + /// statement only. + /// + /// # Panics + /// If called after the request is started, e.g., after [`InsertFormatted::send`]. + #[track_caller] + pub fn with_option(mut self, name: impl Into, value: impl Into) -> Self { + self.state.expect_client_mut().add_option(name, value); + self + } + + pub(crate) fn set_timeouts( + &mut self, + send_timeout: Option, + end_timeout: Option, + ) { + self.send_timeout = Timeout::new_opt(send_timeout); + self.end_timeout = Timeout::new_opt(end_timeout); + } + + /// Wrap this `InsertFormatted` with a buffer of a default size. + /// + /// The returned type also implements [`AsyncWrite`]. + /// + /// To set the capacity, use [`Self::buffered_with_capacity()`]. + pub fn buffered(self) -> BufInsertFormatted { + self.buffered_with_capacity(BUFFER_SIZE) + } + + /// Wrap this `InsertFormatted` with a buffer of a given size. + /// + /// The returned type also implements [`AsyncWrite`]. + pub fn buffered_with_capacity(self, capacity: usize) -> BufInsertFormatted { + BufInsertFormatted::new(self, capacity) + } + + /// Send a chunk of data. + /// + /// If compression is enabled, the data is compressed first. + /// + /// To pre-compress the data, use [`Self::send_compressed()`] instead. + /// + /// # Note: Unbuffered + /// This immediately compresses and queues the data to be sent on the connection + /// without waiting for more chunks. For best performance, chunks should not be too small. + /// + /// Use [`Self::buffered()`] for a buffered implementation which also implements [`AsyncWrite`]. + pub async fn send(&mut self, data: Bytes) -> Result<()> { + #[cfg(feature = "lz4")] + let data = if self.compression.is_lz4() { + CompressedData::from_slice(&data).0 + } else { + data + }; + + self.send_inner(data).await + } + + async fn send_inner(&mut self, mut data: Bytes) -> Result<()> { + std::future::poll_fn(move |cx| { + loop { + ready!(self.poll_ready(cx))?; + + // Potentially cheaper than cloning `data` which touches the refcount + match self.try_send(mem::take(&mut data)) { + ControlFlow::Break(res) => return Poll::Ready(res), + ControlFlow::Continue(unsent) => { + data = unsent; + } + } + } + }) + .await + } + + #[inline] + pub(crate) fn poll_ready(&mut self, cx: &mut Context<'_>) -> Poll> { + if self.state.is_not_started() { + self.init_request()?; + } + + let Some(sender) = self.state.sender() else { + return Poll::Ready(Err(Error::Network("channel closed".into()))); + }; + + match sender.poll_ready(cx) { + Poll::Ready(true) => { + Timeout::reset_opt(self.send_timeout.as_mut()); + Poll::Ready(Ok(())) + } + Poll::Ready(false) => Poll::Ready(Err(Error::Network("channel closed".into()))), + Poll::Pending => { + ready!(Timeout::poll_opt( + self.send_timeout.as_mut(), + self.sleep.as_mut(), + cx + )); + self.abort(); + Poll::Ready(Err(Error::TimedOut)) + } + } + } + + #[inline(always)] + pub(crate) fn try_send(&mut self, bytes: Bytes) -> ControlFlow, Bytes> { + let Some(sender) = self.state.sender() else { + return ControlFlow::Break(Err(Error::Network("channel closed".into()))); + }; + + sender + .try_send(bytes) + .map_break(|res| res.map_err(|e| Error::Network(e.into()))) + } + + /// Ends `INSERT`, the server starts processing the data. + /// + /// Succeeds if the server returns 200, that means the `INSERT` was handled + /// successfully, including all materialized views and quorum writes. + /// + /// NOTE: If this isn't called, the whole `INSERT` is aborted. + pub async fn end(mut self) -> Result<()> { + std::future::poll_fn(|cx| self.poll_end(cx)).await + } + + pub(crate) fn poll_end(&mut self, cx: &mut Context<'_>) -> Poll> { + self.state.terminated(); + self.poll_wait_handle(cx) + } + + fn poll_wait_handle(&mut self, cx: &mut Context<'_>) -> Poll> { + let Some(handle) = self.state.handle() else { + return Poll::Ready(Ok(())); + }; + + let Poll::Ready(res) = Pin::new(&mut *handle).poll(cx) else { + ready!(Timeout::poll_opt( + self.end_timeout.as_mut(), + self.sleep.as_mut(), + cx + )); + + // We can do nothing useful here, so just shut down the background task. + handle.abort(); + return Poll::Ready(Err(Error::TimedOut)); + }; + + let res = match res { + Ok(res) => res, + Err(err) if err.is_panic() => panic::resume_unwind(err.into_panic()), + Err(err) => Err(Error::Custom(format!("unexpected error: {err}"))), + }; + + self.state = InsertState::Completed; + + Poll::Ready(res) + } + + #[cold] + #[track_caller] + #[inline(never)] + fn init_request(&mut self) -> Result<()> { + debug_assert!(matches!(self.state, InsertState::NotStarted { .. })); + let (client, sql) = self.state.client_with_sql().unwrap(); // checked above + + let mut url = Url::parse(&client.url).map_err(|err| Error::InvalidParams(err.into()))?; + let mut pairs = url.query_pairs_mut(); + pairs.clear(); + + if let Some(database) = &client.database { + pairs.append_pair(settings::DATABASE, database); + } + + pairs.append_pair(settings::QUERY, sql); + + if client.compression.is_lz4() { + pairs.append_pair(settings::DECOMPRESS, "1"); + } + + for (name, value) in &client.options { + pairs.append_pair(name, value); + } + + drop(pairs); + + let mut builder = Request::post(url.as_str()); + builder = with_request_headers(builder, &client.headers, &client.products_info); + builder = with_authentication(builder, &client.authentication); + + let (sender, body) = RequestBody::chunked(); + + let request = builder + .body(body) + .map_err(|err| Error::InvalidParams(Box::new(err)))?; + + let future = client.http.request(request); + // TODO: introduce `Executor` to allow bookkeeping of spawned tasks. + let handle = + tokio::spawn(async move { Response::new(future, Compression::None).finish().await }); + + self.state = InsertState::Active { handle, sender }; + Ok(()) + } + + fn abort(&mut self) { + if let Some(sender) = self.state.sender() { + sender.abort(); + } + } +} + +impl Drop for InsertFormatted { + fn drop(&mut self) { + self.abort(); + } +} + +/// A wrapper around [`InsertFormatted`] which buffers writes. +pub struct BufInsertFormatted { + insert: InsertFormatted, + buffer: BytesMut, + capacity: usize, +} + +impl BufInsertFormatted { + fn new(insert: InsertFormatted, capacity: usize) -> Self { + Self { + insert, + buffer: BytesMut::with_capacity(capacity), + capacity, + } + } + + /// Return the number of buffered bytes. + #[inline(always)] + pub fn buf_len(&self) -> usize { + self.buffer.len() + } + + /// Return the current capacity of the buffer. + /// + /// Note: Size is Not Constant + /// This may be smaller than the original capacity if part of the buffer + /// is still being used by the connection. + /// + /// This may be larger if a call to [`Self::write_buffered()`] caused the buffer to expand. + #[inline(always)] + pub fn capacity(&self) -> usize { + self.buffer.capacity() + } + + /// Write data to the buffer without waiting for it to be flushed. + /// + /// May cause the buffer to resize to fit the data. + #[inline(always)] + pub fn write_buffered(&mut self, data: &[u8]) { + self.buffer.extend_from_slice(data); + } + + // `#[inline]` is *supposed* to work + // https://doc.rust-lang.org/reference/attributes/codegen.html#r-attributes.codegen.inline.async + // but it's apparently not implemented yet: https://github.com/rust-lang/rust/pull/149245 + #[inline(always)] + pub async fn write_all(&mut self, mut data: &[u8]) -> Result<()> { + std::future::poll_fn(|cx| { + while !data.is_empty() { + let written = ready!(self.poll_write_inner(data, cx))?; + data = &data[written..]; + } + + Poll::Ready(Ok(())) + }) + .await + } + + // `poll_write` but it returns `crate::Result` instead of `io::Result` + #[inline(always)] + fn poll_write_inner(&mut self, data: &[u8], cx: &mut Context<'_>) -> Poll> { + if self.insert.state.is_not_started() { + // We don't want to wait for the buffer to be full before we start the request, + // in the event of an error. + self.insert.init_request()?; + } + + if self.buffer.len() >= self.capacity { + ready!(self.poll_flush_inner(cx))?; + debug_assert!(self.buffer.is_empty()); + } + + if self.buffer.capacity() == 0 { + self.buffer.reserve(self.capacity); + } + + let write_len = cmp::min(self.buffer.capacity(), data.len()); + + self.buffer.extend_from_slice(&data[..write_len]); + Poll::Ready(Ok(write_len)) + } + + /// Flush the buffer to the server as a single chunk. + /// + /// If [compression is enabled][Client::with_compression], the full buffer will be compressed. + #[inline(always)] + pub async fn flush(&mut self) -> Result<()> { + std::future::poll_fn(|cx| self.poll_flush_inner(cx)).await + } + + #[inline(always)] + fn poll_flush_inner(&mut self, cx: &mut Context<'_>) -> Poll> { + if self.buffer.is_empty() { + return Poll::Ready(Ok(())); + } + + ready!(self.insert.poll_ready(cx))?; + + let data = self.buffer.split().freeze(); + + #[cfg(feature = "lz4")] + let data = if self.insert.compression.is_lz4() { + CompressedData::from(data).0 + } else { + data + }; + + let ControlFlow::Break(res) = self.insert.try_send(data) else { + unreachable!("BUG: we just checked that `ChunkSender` was ready") + }; + + Poll::Ready(res) + } + + /// Flushes the buffer, then calls [`InsertFormatted::end()`]. + /// + /// Cancel-safe. + #[inline(always)] + pub async fn end(&mut self) -> Result<()> { + std::future::poll_fn(|cx| self.poll_end(cx)).await + } + + #[inline(always)] + fn poll_end(&mut self, cx: &mut Context<'_>) -> Poll> { + if !self.buffer.is_empty() { + ready!(self.poll_flush_inner(cx))?; + debug_assert!(self.buffer.is_empty()); + } + + self.insert.poll_end(cx) + } +} + +impl AsyncWrite for BufInsertFormatted { + #[inline(always)] + fn poll_write( + mut self: Pin<&mut Self>, + cx: &mut Context<'_>, + buf: &[u8], + ) -> Poll> { + self.poll_write_inner(buf, cx).map_err(Into::into) + } + + #[inline(always)] + fn poll_flush( + mut self: Pin<&mut Self>, + cx: &mut Context<'_>, + ) -> Poll> { + self.poll_flush_inner(cx).map_err(Into::into) + } + + #[inline(always)] + fn poll_shutdown( + mut self: Pin<&mut Self>, + cx: &mut Context<'_>, + ) -> Poll> { + self.poll_end(cx).map_err(Into::into) + } +} + +impl Timeout { + fn new_opt(duration: Option) -> Option { + duration.map(|duration| Self { + duration, + is_set: false, + }) + } + + /// Returns `Poll::Pending` if `None`. + #[inline(always)] + fn poll_opt(this: Option<&mut Self>, sleep: Pin<&mut Sleep>, cx: &mut Context<'_>) -> Poll<()> { + if let Some(this) = this { + this.poll(sleep, cx) + } else { + Poll::Pending + } + } + + #[inline] + fn poll(&mut self, mut sleep: Pin<&mut Sleep>, cx: &mut Context<'_>) -> Poll<()> { + if !self.is_set + && let Some(deadline) = Instant::now().checked_add(self.duration) + { + sleep.as_mut().reset(deadline); + self.is_set = true; + } + + ready!(sleep.as_mut().poll(cx)); + self.is_set = false; + + Poll::Ready(()) + } + + #[inline(always)] + fn reset_opt(this: Option<&mut Self>) { + if let Some(this) = this { + this.is_set = false; + } + } +} + +// Just so I don't have to repeat this feature flag a hundred times. +#[cfg(feature = "lz4")] +mod compression { + use crate::error::{Error, Result}; + use crate::insert_formatted::InsertFormatted; + use bytes::Bytes; + + /// A chunk of pre-compressed data. + #[cfg_attr(docsrs, doc(cfg(feature = "lz4")))] + pub struct CompressedData(pub(crate) Bytes); + + impl CompressedData { + /// Compress a slice of bytes. + #[inline(always)] + pub fn from_slice(slice: &[u8]) -> Self { + Self( + crate::compression::lz4::compress(slice) + .expect("BUG: `lz4::compress()` should not error"), + ) + } + } + + impl From for CompressedData + where + T: AsRef<[u8]>, + { + #[inline(always)] + fn from(value: T) -> Self { + Self::from_slice(value.as_ref()) + } + } + + impl InsertFormatted { + /// Send a chunk of pre-compressed data. + /// + /// # Errors + /// In addition to network errors, this will return [`Error::Compression`] if the + /// [`Client`][crate::Client] does not have compression enabled. + pub async fn send_compressed(&mut self, data: CompressedData) -> Result<()> { + if !self.compression.is_lz4() { + return Err(Error::Compression( + "attempting to send compressed data, but compression is not enabled".into(), + )); + } + + self.send_inner(data.0).await + } + } +} diff --git a/src/lib.rs b/src/lib.rs index a9c5282..bc42360 100644 --- a/src/lib.rs +++ b/src/lib.rs @@ -19,6 +19,7 @@ use tokio::sync::RwLock; pub mod error; pub mod insert; +pub mod insert_formatted; #[cfg(feature = "inserter")] pub mod inserter; pub mod query; @@ -415,6 +416,29 @@ impl Client { inserter::Inserter::new(self, table) } + /// Start an `INSERT` statement sending pre-formatted data. + /// + /// `sql` should be an `INSERT INTO ... FORMAT ` statement. + /// Any other type of statement may produce incorrect results. + /// + /// The statement is not issued until the first call to + /// [`.send()`][insert_formatted::InsertFormatted::send]. + /// + /// # Note: Not Validated + /// Unlike [`Insert`][insert::Insert] and [`Inserter`][inserter::Inserter], + /// this does not perform any validation on the submitted data. + /// + /// Only the use of self-describing formats (e.g. CSV, TabSeparated, JSON) is recommended. + /// + /// See the [list of supported formats](https://clickhouse.com/docs/interfaces/formats) + /// for details. + pub fn insert_formatted_with( + &self, + sql: impl Into, + ) -> insert_formatted::InsertFormatted { + insert_formatted::InsertFormatted::new(self, sql.into()) + } + /// Starts a new SELECT/DDL query. pub fn query(&self, query: &str) -> query::Query { query::Query::new(self, query) diff --git a/src/request_body.rs b/src/request_body.rs index 1c4425c..4e4bfe3 100644 --- a/src/request_body.rs +++ b/src/request_body.rs @@ -1,15 +1,14 @@ +use bytes::Bytes; +use futures_channel::mpsc; +use futures_util::{SinkExt, Stream}; +use hyper::body::{Body, Frame, SizeHint}; +use std::ops::ControlFlow; use std::{ error::Error as StdError, mem, pin::Pin, task::{Context, Poll}, }; - -use bytes::Bytes; -use futures_channel::mpsc; -use futures_util::{SinkExt, Stream}; -use hyper::body::{Body, Frame, SizeHint}; - // === RequestBody === pub struct RequestBody(Inner); @@ -82,6 +81,32 @@ impl ChunkSender { self.0.send(Message::Chunk(chunk)).await.is_ok() } + #[inline(always)] + pub(crate) fn poll_ready(&mut self, cx: &mut Context<'_>) -> Poll { + self.0.poll_ready(cx).map(|res| res.is_ok()) + } + + #[inline(always)] + pub(crate) fn try_send( + &mut self, + chunk: Bytes, + ) -> ControlFlow, Bytes> { + self.0.try_send(Message::Chunk(chunk)).map_or_else( + |e| { + if e.is_full() { + let Message::Chunk(bytes) = e.into_inner() else { + unreachable!() + }; + + ControlFlow::Continue(bytes) + } else { + ControlFlow::Break(Err("channel closed")) + } + }, + |()| ControlFlow::Break(Ok(())), + ) + } + pub(crate) fn abort(&self) { // `clone()` allows to send even if the channel is full. let _ = self.0.clone().try_send(Message::Abort); diff --git a/tests/it/cursor_error.rs b/tests/it/cursor_error.rs index d58128f..911f9df 100644 --- a/tests/it/cursor_error.rs +++ b/tests/it/cursor_error.rs @@ -41,7 +41,13 @@ async fn max_execution_time(mut client: Client, wait_end_of_query: bool) -> u8 { Err(err) => break err, } }; - assert!(err.to_string().contains("TIMEOUT_EXCEEDED")); + + let err_s = err.to_string(); + assert!( + err_s.contains("TIMEOUT_EXCEEDED"), + "expected TIMEOUT_EXCEEDED in error string, got {err_s:?}; original: {err:?}" + ); + i } @@ -96,5 +102,10 @@ async fn deferred_lz4() { }; assert_ne!(i, 0); // we're interested only in errors during processing - assert!(err.to_string().contains("TIMEOUT_EXCEEDED")); + + let err_s = err.to_string(); + assert!( + err_s.contains("TIMEOUT_EXCEEDED"), + "expected TIMEOUT_EXCEEDED in error string, got {err_s:?}; original: {err:?}" + ); } diff --git a/tests/it/fixtures/nyc-taxi_trips_0_head_1000.tsv b/tests/it/fixtures/nyc-taxi_trips_0_head_1000.tsv new file mode 100644 index 0000000..9d1effb --- /dev/null +++ b/tests/it/fixtures/nyc-taxi_trips_0_head_1000.tsv @@ -0,0 +1,1001 @@ +trip_id vendor_id pickup_date pickup_datetime dropoff_date dropoff_datetime store_and_fwd_flag rate_code_id pickup_longitude pickup_latitude dropoff_longitude dropoff_latitude passenger_count trip_distance fare_amount extra mta_tax tip_amount tolls_amount ehail_fee improvement_surcharge total_amount payment_type trip_type pickup dropoff cab_type pickup_nyct2010_gid pickup_ctlabel pickup_borocode pickup_ct2010 pickup_boroct2010 pickup_cdeligibil pickup_ntacode pickup_ntaname pickup_puma dropoff_nyct2010_gid dropoff_ctlabel dropoff_borocode dropoff_ct2010 dropoff_boroct2010 dropoff_cdeligibil dropoff_ntacode dropoff_ntaname dropoff_puma +1199999902 2 2015-07-07 2015-07-07 19:45:07 2015-07-07 2015-07-07 20:05:24 0 1 -73.95954895019531 40.75904846191406 -73.99234008789062 40.749141693115234 2 2.59 14.5 1 0.5 3.26 0 0 0.3 19.56 CSH 0 @ � yellow -36 106.02 1 Manhattan 1010602 I MN31 Lenox Hill-Roosevelt Island 3805 131 101 1 Manhattan 1010100 I MN17 Midtown-Midtown South 3807 +1199999919 1 2015-07-07 2015-07-07 20:26:29 2015-07-07 2015-07-07 20:33:21 0 1 -73.78194427490234 40.644710540771484 -73.78457641601562 40.666263580322266 1 2.4 9 0.5 0.5 0 0 0 0.3 10.3 CRE 0 2  yellow 8 716 4 Queens 4071600 I QN98 Airport 4105 7 306 4 Queens 4030600 I QN03 Springfield Gardens South-Brookville 4105 +1199999944 2 2015-07-07 2015-07-07 21:25:09 2015-07-07 2015-07-07 21:49:58 0 1 -74.00820922851562 40.721553802490234 -73.96749114990234 40.77031326293945 1 5.13 20 0.5 0.5 3 0 0 0.3 24.3 CSH 0 1 7 yellow 15 33 1 Manhattan 1003300 I MN24 SoHo-TriBeCa-Civic Center-Little Italy 3810 228 122 1 Manhattan 1012200 I MN40 Upper East Side-Carnegie Hill 3805 +1199999969 1 2015-07-07 2015-07-07 22:55:31 2015-07-07 2015-07-07 23:02:57 0 1 -74.00052642822266 40.73436737060547 -73.99207305908203 40.74930191040039 1 1.2 7 0.5 0.5 1.65 0 0 0.3 9.95 CSH 0 I � yellow -94 71 1 Manhattan 1007100 I MN23 West Village 3810 131 101 1 Manhattan 1010100 I MN17 Midtown-Midtown South 3807 +1199999990 2 2015-07-08 2015-07-08 00:04:15 2015-07-08 2015-07-08 00:12:58 0 1 -73.98857879638672 40.718753814697266 -73.98199462890625 40.74325942993164 5 2.17 8.5 0.5 0.5 0 0 0 0.3 9.8 CSH 0 H p yellow -61 18 1 Manhattan 1001800 E MN27 Chinatown 3809 130 72 1 Manhattan 1007200 I MN20 Murray Hill-Kips Bay 3808 +1200000004 1 2015-07-08 2015-07-08 01:27:45 2015-07-08 2015-07-08 01:52:15 0 1 -74.00508880615234 40.75154495239258 -73.96592712402344 40.712791442871094 2 5.4 20.5 0.5 0.5 4.35 0 0 0.3 26.15 CSH 0 F V yellow 50 99 1 Manhattan 1009900 I MN13 Hudson Yards-Chelsea-Flatiron-Union Square 3807 68 549 3 Brooklyn 3054900 E BK73 North Side-South Side 4001 +1200000035 2 2015-07-08 2015-07-08 07:37:10 2015-07-08 2015-07-08 07:47:08 0 1 -73.96324920654297 40.75680923461914 -73.96163940429688 40.77384948730469 1 1.6600000000000001 8.5 0 0.5 0 0 0 0.3 9.3 CRE 0 ) 7 yellow 114 106.01 1 Manhattan 1010601 I MN19 Turtle Bay-East Midtown 3808 18 128 1 Manhattan 1012800 I MN40 Upper East Side-Carnegie Hill 3805 +1200000045 2 2015-07-08 2015-07-08 08:00:56 2015-07-08 2015-07-08 08:09:33 0 1 -73.97911071777344 40.7818603515625 -73.95894622802734 40.780948638916016 1 1.6600000000000001 8 0 0.5 1.76 0 0 0.3 10.56 CSH 0 9 6 yellow 63 161 1 Manhattan 1016100 I MN12 Upper West Side 3806 74 150.02 1 Manhattan 1015002 I MN40 Upper East Side-Carnegie Hill 3805 +1200000050 1 2015-07-08 2015-07-08 08:34:28 2015-07-08 2015-07-08 08:47:00 0 1 -73.98241424560547 40.73949432373047 -73.98977661132812 40.73023986816406 1 1.1 9.5 0 0.5 0 0 0 0.3 10.3 CRE 0 7 y yellow -11 66 1 Manhattan 1006600 I MN20 Murray Hill-Kips Bay 3808 200 42 1 Manhattan 1004200 I MN22 East Village 3809 +1200000067 2 2015-07-08 2015-07-08 09:21:48 2015-07-08 2015-07-08 09:51:11 0 1 -73.98890686035156 40.688629150390625 -73.99678039550781 40.751121520996094 1 5.14 22.5 0 0.5 4.66 0 0 0.3 27.96 CSH 0 % h yellow 90 43 3 Brooklyn 3004300 I BK38 DUMBO-Vinegar Hill-Downtown Brooklyn-Boerum Hill 4004 103 103 1 Manhattan 1010300 I MN13 Hudson Yards-Chelsea-Flatiron-Union Square 3807 +1200000133 2 2015-07-08 2015-07-08 13:24:02 2015-07-08 2015-07-08 14:08:10 0 1 -73.87342071533203 40.773902893066406 -73.98394775390625 40.75723648071289 5 10.79 42 0 0.5 0 5.54 0 0.3 48.34 CRE 0 8 0 yellow 48 331 4 Queens 4033100 I QN98 Airport 4102 133 119 1 Manhattan 1011900 E MN17 Midtown-Midtown South 3807 +1200000153 2 2015-07-08 2015-07-08 14:17:06 2015-07-08 2015-07-08 14:30:07 0 1 -73.9917984008789 40.726112365722656 -73.9957046508789 40.72648620605469 2 1.27 9.5 0 0.5 0 0 0 0.3 10.3 CRE 0 y  yellow -62 36.02 1 Manhattan 1003602 I MN22 East Village 3809 191 55.02 1 Manhattan 1005502 I MN23 West Village 3810 +1200000155 1 2015-07-08 2015-07-08 14:19:59 2015-07-08 2015-07-08 14:24:23 0 1 -74.00286865234375 40.74956130981445 -73.99490356445312 40.760406494140625 1 0.9 5.5 0 0.5 1.2 0 0 0.3 7.5 CSH 0 F H yellow 50 99 1 Manhattan 1009900 I MN13 Hudson Yards-Chelsea-Flatiron-Union Square 3807 139 121 1 Manhattan 1012100 E MN15 Clinton 3807 +1200000185 2 2015-07-13 2015-07-13 07:50:20 2015-07-13 2015-07-13 07:54:14 0 1 -73.991943359375 40.76433181762695 -73.98014068603516 40.76539993286133 2 0.89 5 0 0.5 1.16 0 0 0.3 6.96 CSH 0 H c yellow -118 127 1 Manhattan 1012700 I MN15 Clinton 3807 4 137 1 Manhattan 1013700 I MN17 Midtown-Midtown South 3807 +1200000198 1 2015-07-13 2015-07-13 08:40:34 2015-07-13 2015-07-13 08:46:20 0 1 -73.98928833007812 40.75246810913086 -73.99266815185547 40.739051818847656 3 1.2 6.5 0 0.5 1.45 0 0 0.3 8.75 CSH 0  4 yellow -124 109 1 Manhattan 1010900 I MN17 Midtown-Midtown South 3807 142 54 1 Manhattan 1005400 I MN13 Hudson Yards-Chelsea-Flatiron-Union Square 3807 +1200000269 1 2015-07-13 2015-07-13 11:58:14 2015-07-13 2015-07-13 12:04:25 0 1 -73.97785949707031 40.788822174072266 -73.95704650878906 40.778324127197266 4 1.3 7 0 0.5 0 0 0 0.3 7.8 CRE 0 9 6 yellow -90 171 1 Manhattan 1017100 I MN12 Upper West Side 3806 24 140 1 Manhattan 1014000 I MN40 Upper East Side-Carnegie Hill 3805 +1200000270 1 2015-07-13 2015-07-13 12:09:27 2015-07-13 2015-07-13 12:46:00 0 1 -73.87297058105469 40.774009704589844 -74.00678253173828 40.70582580566406 1 12 36 0 0.5 7 0 0 0.3 43.8 CSH 0 8 � yellow 48 331 4 Queens 4033100 I QN98 Airport 4102 255 7 1 Manhattan 1000700 I MN25 Battery Park City-Lower Manhattan 3810 +1200000281 1 2015-07-13 2015-07-13 12:53:20 2015-07-13 2015-07-13 13:45:06 0 2 -73.99722290039062 40.72294616699219 -73.7848892211914 40.64315414428711 2 21.5 52 0 0.5 11.65 5.54 0 0.3 69.99 CSH 0 D 2 yellow -67 43 1 Manhattan 1004300 I MN24 SoHo-TriBeCa-Civic Center-Little Italy 3810 8 716 4 Queens 4071600 I QN98 Airport 4105 +1200000346 2 2015-07-13 2015-07-13 17:13:52 2015-07-13 2015-07-13 17:29:40 0 1 -73.96246337890625 40.77897644042969 -73.97981262207031 40.75590515136719 1 1.88 11.5 1 0.5 0 0 0 0.3 13.3 CRE 0 C a yellow 43 143 1 Manhattan 1014300 I MN99 park-cemetery-etc-Manhattan 3806 13 96 1 Manhattan 1009600 E MN17 Midtown-Midtown South 3807 +1200000366 2 2015-07-13 2015-07-13 18:32:48 2015-07-13 2015-07-13 18:53:14 0 1 -74.00650787353516 40.71184158325195 -73.97330474853516 40.76386642456055 5 6.27 21.5 1 0.5 4.66 0 0 0.3 27.96 CSH 0 \t c yellow -101 15.01 1 Manhattan 1001501 I MN25 Battery Park City-Lower Manhattan 3810 19 112.02 1 Manhattan 1011202 I MN17 Midtown-Midtown South 3807 +1200000368 2 2015-07-13 2015-07-13 18:33:38 2015-07-13 2015-07-13 18:45:21 0 1 -73.95435333251953 40.76413345336914 -73.972412109375 40.76524353027344 1 1.25 9 1 0.5 3.24 0 0 0.3 14.04 CSH 0 @ C yellow -34 116 1 Manhattan 1011600 I MN31 Lenox Hill-Roosevelt Island 3805 43 143 1 Manhattan 1014300 I MN99 park-cemetery-etc-Manhattan 3806 +1200000427 2 2015-07-13 2015-07-13 21:41:11 2015-07-13 2015-07-13 21:45:27 0 1 -73.99327087402344 40.7332878112793 -73.98324584960938 40.73789596557617 1 0.91 5 0.5 0.5 0 0 0 0.3 6.3 CRE 0   yellow -55 61 1 Manhattan 1006100 I MN23 West Village 3810 154 64 1 Manhattan 1006400 I MN21 Gramercy 3808 +1200000454 2 2015-07-13 2015-07-13 22:57:34 2015-07-13 2015-07-13 23:10:06 0 1 -74.00924682617188 40.72603988647461 -73.9782485961914 40.76253128051758 1 3.18 12 0.5 0.5 0 0 0 0.3 13.3 CSH 0 % a yellow 13 37 1 Manhattan 1003700 I MN24 SoHo-TriBeCa-Civic Center-Little Italy 3810 17 104 1 Manhattan 1010400 I MN17 Midtown-Midtown South 3807 +1200000486 1 2015-07-14 2015-07-14 05:06:17 2015-07-14 2015-07-14 05:24:50 0 1 -73.9811019897461 40.758785247802734 -73.90910339355469 40.74311828613281 1 4.8 18 0.5 0.5 0 0 0 0.3 19.3 CRE 0 a ` yellow 13 96 1 Manhattan 1009600 E MN17 Midtown-Midtown South 3807 25 251 4 Queens 4025100 E QN63 Woodside 4109 +1200000498 1 2015-07-14 2015-07-14 06:53:17 2015-07-14 2015-07-14 06:58:18 0 1 -73.98514556884766 40.74502182006836 -73.97493743896484 40.758750915527344 1 1 5.5 0 0.5 1.25 0 0 0.3 7.55 CSH 0 d a yellow -125 74 1 Manhattan 1007400 I MN17 Midtown-Midtown South 3807 16 102 1 Manhattan 1010200 I MN17 Midtown-Midtown South 3807 +1200000500 1 2015-07-14 2015-07-14 06:54:46 2015-07-14 2015-07-14 07:13:43 0 1 -73.98703002929688 40.768821716308594 -73.97984313964844 40.71781921386719 1 4.8 17.5 0 0.5 0 0 0 0.3 18.3 CRE 0 H 2 yellow -54 139 1 Manhattan 1013900 I MN15 Clinton 3807 89 10.02 1 Manhattan 1001002 E MN28 Lower East Side 3809 +1200000517 2 2015-07-14 2015-07-14 07:57:15 2015-07-14 2015-07-14 08:07:42 0 1 -73.97943878173828 40.74374008178711 -73.98390197753906 40.75490188598633 1 1.11 8 0 0.5 1.76 0 0 0.3 10.56 CSH 0 7 a yellow -10 70 1 Manhattan 1007000 I MN20 Murray Hill-Kips Bay 3808 13 96 1 Manhattan 1009600 E MN17 Midtown-Midtown South 3807 +1200000541 1 2015-07-14 2015-07-14 09:10:00 2015-07-14 2015-07-14 09:10:56 0 1 -73.97811126708984 40.749061584472656 -73.98039245605469 40.74677658081055 1 0.2 3 0 0.5 0 0 0 0.3 3.8 CRE 0 p p yellow -123 80 1 Manhattan 1008000 I MN20 Murray Hill-Kips Bay 3808 130 72 1 Manhattan 1007200 I MN20 Murray Hill-Kips Bay 3808 +1200000588 1 2015-07-14 2015-07-14 11:47:37 2015-07-14 2015-07-14 12:21:55 0 1 -74.00375366210938 40.70683670043945 -73.97721099853516 40.76264572143555 1 6.1 27.5 0 0.5 4 0 0 0.3 32.3 CSH 0 � a yellow 12 15.02 1 Manhattan 1001502 I MN25 Battery Park City-Lower Manhattan 3810 17 104 1 Manhattan 1010400 I MN17 Midtown-Midtown South 3807 +1200000599 2 2015-07-14 2015-07-14 12:35:24 2015-07-14 2015-07-14 12:40:50 0 1 -74.0030288696289 40.723533630371094 -74.00811767578125 40.72238540649414 1 0.73 5.5 0 0.5 0 0 0 0.3 6.3 CRE 0  1 yellow 16 47 1 Manhattan 1004700 I MN24 SoHo-TriBeCa-Civic Center-Little Italy 3810 15 33 1 Manhattan 1003300 I MN24 SoHo-TriBeCa-Civic Center-Little Italy 3810 +1200000608 1 2015-07-14 2015-07-14 12:58:19 2015-07-14 2015-07-14 13:14:44 0 1 -73.99512481689453 40.734012603759766 -73.98261260986328 40.7357063293457 1 1.3 11 0 0.5 0 0 0 0.3 11.8 CRE 0   yellow -122 63 1 Manhattan 1006300 I MN23 West Village 3810 154 64 1 Manhattan 1006400 I MN21 Gramercy 3808 +1200000625 2 2015-07-14 2015-07-14 13:59:11 2015-07-14 2015-07-14 14:05:48 0 1 -73.99552154541016 40.73931884765625 -73.9981918334961 40.74517059326172 5 0.61 6 0 0.5 1.7 0 0 0.3 8.5 CSH 0 � � yellow -115 81 1 Manhattan 1008100 I MN13 Hudson Yards-Chelsea-Flatiron-Union Square 3807 132 91 1 Manhattan 1009100 I MN13 Hudson Yards-Chelsea-Flatiron-Union Square 3807 +1200000642 1 2015-07-14 2015-07-14 14:46:47 2015-07-14 2015-07-14 15:08:18 0 1 -73.97401428222656 40.76260757446289 -74.0013656616211 40.71952819824219 1 3.5 15.5 0 0.5 0 0 0 0.3 16.3 CRE 0 c D yellow 19 112.02 1 Manhattan 1011202 I MN17 Midtown-Midtown South 3807 124 45 1 Manhattan 1004500 I MN24 SoHo-TriBeCa-Civic Center-Little Italy 3810 +1200000684 2 2015-07-14 2015-07-14 17:30:40 2015-07-14 2015-07-14 17:52:02 0 1 -73.98577117919922 40.76331329345703 -73.98336791992188 40.75286865234375 1 1.05 13 1 0.5 3.7 0 0 0.3 18.5 CSH 0 H d yellow -119 133 1 Manhattan 1013300 I MN15 Clinton 3807 209 84 1 Manhattan 1008400 I MN17 Midtown-Midtown South 3807 +1200000701 2 2015-07-14 2015-07-14 18:19:13 2015-07-14 2015-07-14 18:50:05 0 1 -73.98551177978516 40.7595329284668 -73.98565673828125 40.73176574707031 2 2.89 19.5 1 0.5 0 0 0 0.3 21.3 CRE 0 0 y yellow -122 125 1 Manhattan 1012500 I MN17 Midtown-Midtown South 3807 93 40 1 Manhattan 1004000 I MN22 East Village 3809 +1200000709 2 2015-07-14 2015-07-14 18:42:13 2015-07-14 2015-07-14 19:11:22 0 1 -73.95545196533203 40.77942657470703 -74.00028228759766 40.69282531738281 1 9.54 29.5 1 0.5 0 0 0 0.3 31.3 CRE 0 6 3 yellow 28 148.01 1 Manhattan 1014801 I MN40 Upper East Side-Carnegie Hill 3805 141 7 3 Brooklyn 3000700 I BK09 Brooklyn Heights-Cobble Hill 4004 +1200000750 2 2015-07-14 2015-07-14 20:33:10 2015-07-14 2015-07-14 20:39:21 0 1 -73.97503662109375 40.76156997680664 -73.9826431274414 40.77172088623047 5 1.65 7 0.5 0.5 0 0 0 0.3 8.3 CRE 0 a B yellow 17 104 1 Manhattan 1010400 I MN17 Midtown-Midtown South 3807 67 149 1 Manhattan 1014900 I MN14 Lincoln Square 3806 +1200000769 2 2015-07-14 2015-07-14 21:25:16 2015-07-14 2015-07-14 22:03:56 0 1 -73.97212982177734 40.74980163574219 -73.93721771240234 40.643089294433594 1 11.29 36 0.5 0.5 7.46 0 0 0.3 44.76 CSH 0 3 q yellow -105 88 1 Manhattan 1008800 I MN19 Turtle Bay-East Midtown 3808 180 834 3 Brooklyn 3083400 I BK91 East Flatbush-Farragut 4010 +1200000794 2 2015-07-14 2015-07-14 22:39:09 2015-07-14 2015-07-14 22:54:44 0 1 -73.94503021240234 40.808250427246094 -74.00447082519531 40.74864959716797 1 6.93 21.5 0.5 0.5 4.56 0 0 0.3 27.36 CSH 0 A F yellow 57 200 1 Manhattan 1020000 E MN11 Central Harlem South 3803 50 99 1 Manhattan 1009900 I MN13 Hudson Yards-Chelsea-Flatiron-Union Square 3807 +1200000813 1 2015-07-14 2015-07-14 23:45:53 2015-07-15 2015-07-15 00:08:51 0 1 -73.8725814819336 40.7741584777832 -73.73307800292969 40.65583419799805 1 13.6 38 0.5 0.5 0 0 0 0.3 39.3 CRE 0 8  yellow 48 331 4 Queens 4033100 I QN98 Airport 4102 220 660 4 Queens 4066000 I QN05 Rosedale 4105 +1200000815 1 2015-07-14 2015-07-14 23:52:57 2015-07-15 2015-07-15 00:03:38 0 1 -73.96569061279297 40.76266860961914 -73.97020721435547 40.79657745361328 1 3.1 11.5 0.5 0.5 2.55 0 0 0.3 15.35 CSH 0 A Q yellow -35 110 1 Manhattan 1011000 I MN31 Lenox Hill-Roosevelt Island 3805 169 187 1 Manhattan 1018700 I MN12 Upper West Side 3806 +1200000847 1 2015-07-15 2015-07-15 07:22:27 2015-07-15 2015-07-15 07:27:51 0 1 -73.87479400634766 40.7606201171875 -73.87293243408203 40.77428436279297 1 1.4 6.5 0 0.5 0 0 0 0.3 7.3 CRE 0 p 8 yellow 122 353 4 Queens 4035300 E QN27 East Elmhurst 4102 48 331 4 Queens 4033100 I QN98 Airport 4102 +1200000851 1 2015-07-15 2015-07-15 07:32:20 2015-07-15 2015-07-15 07:35:51 0 1 -74.00594329833984 40.74515151977539 -73.99862670898438 40.74479293823242 1 0.4 4.5 0 0.5 1.3 0 0 0.3 6.6 CSH 0 h � yellow 12 89 1 Manhattan 1008900 I MN13 Hudson Yards-Chelsea-Flatiron-Union Square 3807 132 91 1 Manhattan 1009100 I MN13 Hudson Yards-Chelsea-Flatiron-Union Square 3807 +1200000867 2 2015-07-15 2015-07-15 08:18:19 2015-07-15 2015-07-15 08:24:16 0 1 -73.99292755126953 40.7432975769043 -73.98216247558594 40.748748779296875 1 0.94 6 0 0.5 0 0 0 0.3 6.8 CRE 0 � p yellow -124 91 1 Manhattan 1009100 I MN13 Hudson Yards-Chelsea-Flatiron-Union Square 3807 93 82 1 Manhattan 1008200 I MN20 Murray Hill-Kips Bay 3808 +1200000868 1 2015-07-15 2015-07-15 08:19:37 2015-07-15 2015-07-15 08:31:48 0 1 -73.98114776611328 40.73424530029297 -73.94168090820312 40.79450607299805 1 5.4 17 0 0.5 0 0 0 0.3 17.8 CRE 0  t yellow 52 48 1 Manhattan 1004800 I MN21 Gramercy 3808 36 180 1 Manhattan 1018000 E MN34 East Harlem North 3804 +1200000895 1 2015-07-15 2015-07-15 09:28:32 2015-07-15 2015-07-15 09:46:37 0 1 -73.99432373046875 40.750892639160156 -73.96812438964844 40.75959014892578 1 2.2 12.5 0 0.5 2.65 0 0 0.3 15.95 CSH 0 � b yellow -125 101 1 Manhattan 1010100 I MN17 Midtown-Midtown South 3807 15 100 1 Manhattan 1010000 I MN19 Turtle Bay-East Midtown 3808 +1200000906 2 2015-07-15 2015-07-15 09:49:22 2015-07-15 2015-07-15 10:03:46 0 1 -73.98651885986328 40.72243881225586 -73.97570037841797 40.75484848022461 1 2.83 12 0 0.5 2.56 0 0 0.3 15.36 CSH 0 y a yellow -94 30.02 1 Manhattan 1003002 E MN22 East Village 3809 225 94 1 Manhattan 1009400 E MN17 Midtown-Midtown South 3807 +1200000924 1 2015-07-15 2015-07-15 07:08:39 2015-07-15 2015-07-15 07:13:03 0 1 -73.97537231445312 40.74561309814453 -73.9797134399414 40.75272750854492 1 0.7 5 0 0.5 0 0 0 0.3 5.8 CRE 0 p p yellow -96 78 1 Manhattan 1007800 I MN20 Murray Hill-Kips Bay 3808 93 82 1 Manhattan 1008200 I MN20 Murray Hill-Kips Bay 3808 +1200000934 2 2015-07-15 2015-07-15 11:11:35 2015-07-15 2015-07-15 11:21:28 0 1 -73.98818969726562 40.7488899230957 -73.97728729248047 40.76166915893555 3 1.13 8 0 0.5 0 0 0 0.3 8.8 CRE 0 d a yellow -48 76 1 Manhattan 1007600 I MN17 Midtown-Midtown South 3807 17 104 1 Manhattan 1010400 I MN17 Midtown-Midtown South 3807 +1200000938 2 2015-07-15 2015-07-15 11:13:46 2015-07-15 2015-07-15 11:26:02 0 1 -73.9763412475586 40.75197982788086 -73.9902572631836 40.74367141723633 5 1.2 9 0 0.5 0 0 0 0.3 9.8 CRE 0 b 4 yellow -30 92 1 Manhattan 1009200 I MN19 Turtle Bay-East Midtown 3808 133 58 1 Manhattan 1005800 I MN13 Hudson Yards-Chelsea-Flatiron-Union Square 3807 +1200000948 2 2015-07-15 2015-07-15 11:36:54 2015-07-15 2015-07-15 11:44:23 0 1 -73.9605484008789 40.76890182495117 -73.96851348876953 40.764747619628906 2 0.69 6 0 0.5 0 0 0 0.3 6.8 CRE 0 A 7 yellow -73 126 1 Manhattan 1012600 I MN31 Lenox Hill-Roosevelt Island 3805 227 114.01 1 Manhattan 1011401 I MN40 Upper East Side-Carnegie Hill 3805 +1200000990 1 2015-07-15 2015-07-15 13:49:34 2015-07-15 2015-07-15 14:09:47 0 1 -73.98294067382812 40.760841369628906 -73.95992279052734 40.80646896362305 1 3.9 16.5 0 0.5 3.45 0 0 0.3 20.75 CSH 0 0 f yellow -122 125 1 Manhattan 1012500 I MN17 Midtown-Midtown South 3807 37 201.01 1 Manhattan 1020101 I MN09 Morningside Heights 3802 +1200001001 2 2015-07-15 2015-07-15 14:16:51 2015-07-15 2015-07-15 14:42:50 0 1 -73.96964263916016 40.76128387451172 -74.00507354736328 40.706546783447266 1 5.47 22 0 0.5 0 0 0 0.3 22.8 CRE 0 b � yellow 20 112.03 1 Manhattan 1011203 I MN19 Turtle Bay-East Midtown 3808 12 15.02 1 Manhattan 1001502 I MN25 Battery Park City-Lower Manhattan 3810 +1200001007 2 2015-07-15 2015-07-15 14:36:43 2015-07-15 2015-07-15 14:53:16 0 1 -73.95915985107422 40.7793083190918 -73.95954895019531 40.77932357788086 2 1.8900000000000001 11.5 0 0.5 2.46 0 0 0.3 14.76 CSH 0 6 6 yellow 75 150.01 1 Manhattan 1015001 I MN40 Upper East Side-Carnegie Hill 3805 25 142 1 Manhattan 1014200 I MN40 Upper East Side-Carnegie Hill 3805 +1200001051 1 2015-07-15 2015-07-15 16:46:31 2015-07-15 2015-07-15 17:01:53 0 1 -74.00138854980469 40.75059509277344 -73.984375 40.73303985595703 1 1.7 11.5 1 0.5 0 0 0 0.3 13.3 CRE 0 h  yellow -18 97 1 Manhattan 1009700 E MN13 Hudson Yards-Chelsea-Flatiron-Union Square 3807 52 48 1 Manhattan 1004800 I MN21 Gramercy 3808 +1200001058 1 2015-07-15 2015-07-15 17:15:37 2015-07-15 2015-07-15 18:06:43 0 2 -73.79020690917969 40.64499282836914 -73.98121643066406 40.77085494995117 1 20.4 52 0 0.5 0 0 0 0.3 52.8 CRE 0 2 B yellow 8 716 4 Queens 4071600 I QN98 Airport 4105 67 149 1 Manhattan 1014900 I MN14 Lincoln Square 3806 +1200001062 1 2015-07-15 2015-07-15 17:24:28 2015-07-15 2015-07-15 17:40:58 0 1 -73.97298431396484 40.75374221801758 -73.98475646972656 40.73737716674805 2 1.5 11.5 1 0.5 2.66 0 0 0.3 15.96 CSH 0 b  yellow -30 92 1 Manhattan 1009200 I MN19 Turtle Bay-East Midtown 3808 244 50 1 Manhattan 1005000 I MN21 Gramercy 3808 +1200001063 2 2015-07-15 2015-07-15 17:27:16 2015-07-15 2015-07-15 17:39:47 0 1 -73.9993896484375 40.740596771240234 -73.97940826416016 40.741573333740234 2 1.42 9.5 1 0.5 2.82 0 0 0.3 14.12 CSH 0 � 7 yellow -115 81 1 Manhattan 1008100 I MN13 Hudson Yards-Chelsea-Flatiron-Union Square 3807 245 66 1 Manhattan 1006600 I MN20 Murray Hill-Kips Bay 3808 +1200001068 2 2015-07-15 2015-07-15 17:58:09 2015-07-15 2015-07-15 18:08:00 0 1 -73.98013305664062 40.76485061645508 -73.9911880493164 40.75035858154297 4 1.17 8 1 0.5 0 0 0 0.3 9.8 CRE 0 c � yellow 4 137 1 Manhattan 1013700 I MN17 Midtown-Midtown South 3807 131 101 1 Manhattan 1010100 I MN17 Midtown-Midtown South 3807 +1200001069 2 2015-07-15 2015-07-15 17:59:40 2015-07-15 2015-07-15 18:11:56 0 1 -73.99163055419922 40.75006103515625 -73.98931121826172 40.77682113647461 1 2.61 11 1 0.5 0 0 0 0.3 12.8 CRE 0 � C yellow -125 101 1 Manhattan 1010100 I MN17 Midtown-Midtown South 3807 205 151 1 Manhattan 1015100 E MN14 Lincoln Square 3806 +1200001083 2 2015-07-15 2015-07-15 18:41:56 2015-07-15 2015-07-15 18:44:25 0 1 -73.98201751708984 40.77212905883789 -73.98198699951172 40.77790832519531 5 0.44 4 1 0.5 1.16 0 0 0.3 6.96 CSH 0 B B yellow 67 149 1 Manhattan 1014900 I MN14 Lincoln Square 3806 21 157 1 Manhattan 1015700 I MN14 Lincoln Square 3806 +1200001135 2 2015-07-15 2015-07-15 20:41:53 2015-07-15 2015-07-15 21:14:21 0 1 -73.7828598022461 40.64444351196289 -73.96597290039062 40.60808563232422 1 17.16 48.5 0.5 0.5 0 0 0 0.3 49.8 CRE 0 2 # yellow 8 716 4 Queens 4071600 I QN98 Airport 4105 205 420 3 Brooklyn 3042000 E BK25 Homecrest 4016 +1200001139 1 2015-07-15 2015-07-15 20:46:14 2015-07-15 2015-07-15 20:52:05 0 1 -74.00511169433594 40.74088668823242 -73.99418640136719 40.75421905517578 1 1.5 7 0.5 0.5 0 0 0 0.3 8.3 CRE 0 h h yellow 75 83 1 Manhattan 1008300 E MN13 Hudson Yards-Chelsea-Flatiron-Union Square 3807 162 111 1 Manhattan 1011100 I MN13 Hudson Yards-Chelsea-Flatiron-Union Square 3807 +1200001143 2 2015-07-15 2015-07-15 21:02:39 2015-07-15 2015-07-15 21:08:42 0 1 -73.99290466308594 40.730133056640625 -73.98881530761719 40.72300338745117 1 0.78 5.5 0.5 0.5 1.36 0 0 0.3 8.16 CSH 0  y yellow -64 57 1 Manhattan 1005700 I MN23 West Village 3810 194 36.02 1 Manhattan 1003602 I MN22 East Village 3809 +1200001163 2 2015-07-15 2015-07-15 21:53:07 2015-07-15 2015-07-15 22:02:26 0 1 -73.98802185058594 40.73379898071289 -73.9884033203125 40.72002029418945 1 1.34 8 0.5 0.5 0 0 0 0.3 9.3 CSH 0  H yellow -12 50 1 Manhattan 1005000 I MN21 Gramercy 3808 195 18 1 Manhattan 1001800 E MN27 Chinatown 3809 +1200001171 2 2015-07-15 2015-07-15 22:20:25 2015-07-15 2015-07-15 22:27:19 0 1 -74.00592803955078 40.74126052856445 -73.99138641357422 40.744808197021484 1 1.23 7 0.5 0.5 1.66 0 0 0.3 9.96 CSH 0 h 4 yellow 75 83 1 Manhattan 1008300 E MN13 Hudson Yards-Chelsea-Flatiron-Union Square 3807 133 58 1 Manhattan 1005800 I MN13 Hudson Yards-Chelsea-Flatiron-Union Square 3807 +1200001178 1 2015-07-15 2015-07-15 22:31:40 2015-07-15 2015-07-15 22:49:22 0 1 -73.99224853515625 40.72509765625 -73.95452117919922 40.7808723449707 1 4.5 16.5 0.5 0.5 3.55 0 0 0.3 21.35 CSH 0 y 6 yellow -62 36.02 1 Manhattan 1003602 I MN22 East Village 3809 29 148.02 1 Manhattan 1014802 I MN40 Upper East Side-Carnegie Hill 3805 +1200001211 1 2015-07-15 2015-07-15 23:36:49 2015-07-16 2015-07-16 00:12:13 0 2 -73.78954315185547 40.64349365234375 -73.99507904052734 40.7326774597168 1 19.9 52 0 0.5 20 0 0 0.3 72.8 CSH 0 2  yellow 8 716 4 Queens 4071600 I QN98 Airport 4105 163 59 1 Manhattan 1005900 I MN23 West Village 3810 +1200001213 2 2015-07-15 2015-07-15 23:43:54 2015-07-15 2015-07-15 23:52:21 0 1 -73.99329376220703 40.7244987487793 -73.97918701171875 40.728240966796875 1 1.35 7.5 0.5 0.5 1.76 0 0 0.3 10.56 CSH 0  y yellow -65 55.02 1 Manhattan 1005502 I MN23 West Village 3810 98 34 1 Manhattan 1003400 I MN22 East Village 3809 +1200001242 2 2015-07-16 2015-07-16 02:20:42 2015-07-16 2015-07-16 02:30:07 0 1 -73.96798706054688 40.80012130737305 -73.93901062011719 40.80223083496094 5 2.13 9.5 0.5 0.5 0 0 0 0.3 10.8 CSH 0 Q t yellow 112 191 1 Manhattan 1019100 I MN12 Upper West Side 3806 233 196 1 Manhattan 1019600 E MN34 East Harlem North 3804 +1200001264 2 2015-07-16 2015-07-16 07:05:20 2015-07-16 2015-07-16 07:12:31 0 1 -73.98945617675781 40.74746322631836 -73.97411346435547 40.7607307434082 6 1.44 7.5 0 0.5 0 0 0 0.3 8.3 CRE 0 d a yellow -48 76 1 Manhattan 1007600 I MN17 Midtown-Midtown South 3807 16 102 1 Manhattan 1010200 I MN17 Midtown-Midtown South 3807 +1200001271 2 2015-07-16 2015-07-16 08:01:57 2015-07-16 2015-07-16 08:16:32 0 1 -73.97805786132812 40.78641128540039 -73.97883605957031 40.754798889160156 1 2.64 12 0 0.5 2 0 0 0.3 14.8 CSH 0 9 a yellow -90 171 1 Manhattan 1017100 I MN12 Upper West Side 3806 225 94 1 Manhattan 1009400 E MN17 Midtown-Midtown South 3807 +1200001275 1 2015-07-16 2015-07-16 08:15:59 2015-07-16 2015-07-16 08:32:23 0 1 -73.94477081298828 40.77928924560547 -73.9732437133789 40.79289245605469 1 2.3 12.5 0 0.5 2.66 0 0 0.3 15.96 CSH 0 b 8 yellow 1 152 1 Manhattan 1015200 I MN32 Yorkville 3805 167 179 1 Manhattan 1017900 I MN12 Upper West Side 3806 +1200001285 1 2015-07-16 2015-07-16 08:45:03 2015-07-16 2015-07-16 08:57:14 0 1 -73.9524154663086 40.77303695678711 -73.9616928100586 40.757652282714844 1 1.4 9 0 0.5 0 0 0 0.3 9.8 CRE 0 c ) yellow 0 138 1 Manhattan 1013800 I MN32 Yorkville 3805 114 106.01 1 Manhattan 1010601 I MN19 Turtle Bay-East Midtown 3808 +1200001290 1 2015-07-16 2015-07-16 08:55:47 2015-07-16 2015-07-16 09:39:06 0 1 -73.86864471435547 40.77288055419922 -73.98999786376953 40.71857452392578 2 14.7 43.5 0 0.5 8 5.54 0 0.3 57.84 CSH 0 8 H yellow 48 331 4 Queens 4033100 I QN98 Airport 4102 195 18 1 Manhattan 1001800 E MN27 Chinatown 3809 +1200001292 2 2015-07-16 2015-07-16 09:02:50 2015-07-16 2015-07-16 09:16:30 0 1 -73.97405242919922 40.759883880615234 -73.98989868164062 40.74089813232422 1 1.6800000000000002 10 0 0.5 1 0 0 0.3 11.8 CSH 0 a 4 yellow 16 102 1 Manhattan 1010200 I MN17 Midtown-Midtown South 3807 166 56 1 Manhattan 1005600 I MN13 Hudson Yards-Chelsea-Flatiron-Union Square 3807 +1200001303 1 2015-07-16 2015-07-16 09:32:42 2015-07-16 2015-07-16 09:45:56 0 1 -73.96053314208984 40.769989013671875 -73.96854400634766 40.75539016723633 1 1.1 9.5 0 0.5 0 0 0 0.3 10.3 CRE 0 7 ) yellow 18 128 1 Manhattan 1012800 I MN40 Upper East Side-Carnegie Hill 3805 14 98 1 Manhattan 1009800 I MN19 Turtle Bay-East Midtown 3808 +1200001314 2 2015-07-16 2015-07-16 09:52:38 2015-07-16 2015-07-16 09:58:54 0 1 -73.91786193847656 40.746578216552734 -73.94788360595703 40.7422981262207 3 1.6600000000000001 7.5 0 0.5 1 0 0 0.3 9.3 CSH 0 & E yellow 69 169 4 Queens 4016900 I QN31 Hunters Point-Sunnyside-West Maspeth 4109 137 19 4 Queens 4001900 I QN31 Hunters Point-Sunnyside-West Maspeth 4109 +1200001318 2 2015-07-16 2015-07-16 10:04:07 2015-07-16 2015-07-16 10:53:34 0 2 -73.78036499023438 40.64537048339844 -73.98020935058594 40.75563430786133 1 17.41 52 0 0.5 5 0 0 0.3 57.8 CSH 0 2 a yellow 8 716 4 Queens 4071600 I QN98 Airport 4105 13 96 1 Manhattan 1009600 E MN17 Midtown-Midtown South 3807 +1200001355 2 2015-07-16 2015-07-16 12:00:47 2015-07-16 2015-07-16 12:35:00 0 1 -73.96289825439453 40.77434539794922 -73.96875762939453 40.69236373901367 1 9.42 33 0 0.5 8.45 0 0 0.3 42.25 CSH 0 7 I yellow 23 130 1 Manhattan 1013000 I MN40 Upper East Side-Carnegie Hill 3805 107 195 3 Brooklyn 3019500 E BK69 Clinton Hill 4004 +1200001359 1 2015-07-16 2015-07-16 12:09:33 2015-07-16 2015-07-16 12:18:40 0 1 -73.9948959350586 40.739898681640625 -74.00540924072266 40.72755813598633 1 1.2 8 0 0.5 0 0 0 0.3 8.8 CRE 0 4 % yellow -114 54 1 Manhattan 1005400 I MN13 Hudson Yards-Chelsea-Flatiron-Union Square 3807 13 37 1 Manhattan 1003700 I MN24 SoHo-TriBeCa-Civic Center-Little Italy 3810 +1200001390 2 2015-07-16 2015-07-16 14:03:50 2015-07-16 2015-07-16 14:34:02 0 1 -73.998291015625 40.724456787109375 -73.98194122314453 40.75367736816406 5 2.63 19 0 0.5 2.5 0 0 0.3 22.3 CSH 0  d yellow 35 49 1 Manhattan 1004900 I MN24 SoHo-TriBeCa-Civic Center-Little Italy 3810 209 84 1 Manhattan 1008400 I MN17 Midtown-Midtown South 3807 +1200001430 2 2015-07-16 2015-07-16 16:28:52 2015-07-16 2015-07-16 16:35:29 0 1 -73.97129821777344 40.76396942138672 -73.95883178710938 40.763858795166016 2 0.86 6 1 0.5 1.56 0 0 0.3 9.36 CSH 0 7 @ yellow -29 114.01 1 Manhattan 1011401 I MN40 Upper East Side-Carnegie Hill 3805 222 116 1 Manhattan 1011600 I MN31 Lenox Hill-Roosevelt Island 3805 +1200001484 2 2015-07-16 2015-07-16 15:00:14 2015-07-16 2015-07-16 16:00:28 0 1 -73.97631072998047 40.751190185546875 -73.97396087646484 40.574989318847656 1 18.78 58 0 0.5 0 0 0 0.3 58.8 CRE 0 p U yellow -123 80 1 Manhattan 1008000 I MN20 Murray Hill-Kips Bay 3808 250 354 3 Brooklyn 3035400 E BK23 West Brighton 4018 +1200001489 1 2015-07-16 2015-07-16 19:11:42 2015-07-16 2015-07-16 19:37:12 0 1 -73.98075866699219 40.76285934448242 -73.99028778076172 40.737125396728516 1 2.1 16 1 0.5 3.55 0 0 0.3 21.35 CSH 0 0 4 yellow 3 131 1 Manhattan 1013100 I MN17 Midtown-Midtown South 3807 245 52 1 Manhattan 1005200 I MN13 Hudson Yards-Chelsea-Flatiron-Union Square 3807 +1200001495 2 2015-07-16 2015-07-16 19:23:21 2015-07-16 2015-07-16 19:36:25 0 1 -73.9831771850586 40.73054122924805 -73.9567642211914 40.76669692993164 2 2.87 12 1 0.5 2.76 0 0 0.3 16.56 CSH 0 y @ yellow 93 40 1 Manhattan 1004000 I MN22 East Village 3809 160 124 1 Manhattan 1012400 I MN31 Lenox Hill-Roosevelt Island 3805 +1200001520 2 2015-07-16 2015-07-16 20:14:32 2015-07-16 2015-07-16 20:36:43 0 1 -73.98934173583984 40.73667907714844 -73.98529815673828 40.77809143066406 6 3.67 16.5 0.5 0.5 3.56 0 0 0.3 21.36 CSH 0 4 C yellow -11 52 1 Manhattan 1005200 I MN13 Hudson Yards-Chelsea-Flatiron-Union Square 3807 206 155 1 Manhattan 1015500 I MN14 Lincoln Square 3806 +1200001536 2 2015-07-16 2015-07-16 20:49:20 2015-07-16 2015-07-16 21:05:34 0 1 -73.98979949951172 40.735595703125 -73.9762191772461 40.77653884887695 1 3.63 14.5 0.5 0.5 0 0 0 0.3 15.8 CRE 0 4 B yellow -11 52 1 Manhattan 1005200 I MN13 Hudson Yards-Chelsea-Flatiron-Union Square 3807 21 157 1 Manhattan 1015700 I MN14 Lincoln Square 3806 +1200001538 2 2015-07-16 2015-07-16 20:51:36 2015-07-16 2015-07-16 21:21:45 0 1 -73.9732437133789 40.7953987121582 -73.88165283203125 40.76744079589844 1 9 30.5 0.5 0.5 9.34 5.54 0 0.3 46.68 CSH 0 8 ) yellow -88 183 1 Manhattan 1018300 I MN12 Upper West Side 3806 117 329 4 Queens 4032900 I QN28 Jackson Heights 4102 +1200001559 2 2015-07-16 2015-07-16 21:50:44 2015-07-16 2015-07-16 21:58:39 0 1 -73.98175811767578 40.75697326660156 -73.9660415649414 40.76626205444336 1 1.42 7.5 0.5 0.5 2.2 0 0 0.3 11 CSH 0 a 7 yellow 13 96 1 Manhattan 1009600 E MN17 Midtown-Midtown South 3807 17 120 1 Manhattan 1012000 I MN40 Upper East Side-Carnegie Hill 3805 +1200001560 1 2015-07-16 2015-07-16 21:51:17 2015-07-16 2015-07-16 21:54:39 0 1 -73.97759246826172 40.74642562866211 -73.97088623046875 40.75566101074219 1 0.7 4.5 0.5 0.5 1.15 0 0 0.3 6.95 CSH 0 p b yellow -96 78 1 Manhattan 1007800 I MN20 Murray Hill-Kips Bay 3808 15 100 1 Manhattan 1010000 I MN19 Turtle Bay-East Midtown 3808 +1200001601 1 2015-07-16 2015-07-16 23:38:37 2015-07-16 2015-07-16 23:40:28 0 1 -73.97822570800781 40.6712646484375 -73.97895050048828 40.67161178588867 1 0 3 0.5 0.5 0 0 0 0.3 4.3 CRE 0 � � yellow 93 155 3 Brooklyn 3015500 I BK37 Park Slope-Gowanus 4005 93 155 3 Brooklyn 3015500 I BK37 Park Slope-Gowanus 4005 +1200001610 1 2015-07-16 2015-07-16 23:53:07 2015-07-16 2015-07-16 23:58:11 0 1 -74.02296447753906 40.62881088256836 -74.02716827392578 40.617393493652344 1 1 6 0.5 0.5 0 0 0 0.3 7.3 CRE 0   yellow 91 134 3 Brooklyn 3013400 I BK31 Bay Ridge 4013 152 160 3 Brooklyn 3016000 E BK31 Bay Ridge 4013 +1200001629 2 2015-07-17 2015-07-17 00:42:52 2015-07-17 2015-07-17 00:56:34 0 1 -73.96597290039062 40.76206588745117 -73.9844741821289 40.739871978759766 1 1.96 11 0.5 0.5 2.46 0 0 0.3 14.76 CSH 0 A  yellow -35 110 1 Manhattan 1011000 I MN31 Lenox Hill-Roosevelt Island 3805 11 68 1 Manhattan 1006800 I MN21 Gramercy 3808 +1200001640 2 2015-07-17 2015-07-17 01:37:49 2015-07-17 2015-07-17 01:50:20 0 1 -73.98298645019531 40.76765823364258 -73.99949645996094 40.728511810302734 1 3.61 13 0.5 0.5 2 0 0 0.3 16.3 CSH 0 B  yellow 66 145 1 Manhattan 1014500 I MN14 Lincoln Square 3806 37 65 1 Manhattan 1006500 I MN23 West Village 3810 +1200001642 2 2015-07-17 2015-07-17 01:55:45 2015-07-17 2015-07-17 02:00:22 0 1 -73.98216247558594 40.73112869262695 -73.97502136230469 40.72037887573242 1 1.12 5.5 0.5 0.5 2.2 0 0 0.3 9 CSH 0 y  yellow 98 34 1 Manhattan 1003400 I MN22 East Village 3809 143 20 1 Manhattan 1002000 E MN28 Lower East Side 3809 +1200001646 1 2015-07-17 2015-07-17 02:23:58 2015-07-17 2015-07-17 02:38:46 0 1 -74.00428771972656 40.752567291259766 -73.98552703857422 40.6976203918457 1 4.3 16 0.5 0.5 0 0 0 0.3 17.3 CRE 0 F e yellow 50 99 1 Manhattan 1009900 I MN13 Hudson Yards-Chelsea-Flatiron-Union Square 3807 103 13 3 Brooklyn 3001300 I BK38 DUMBO-Vinegar Hill-Downtown Brooklyn-Boerum Hill 4004 +1200001676 2 2015-07-17 2015-07-17 07:29:21 2015-07-17 2015-07-17 07:38:40 0 1 -73.94969940185547 40.78077697753906 -73.97920989990234 40.771793365478516 5 2.67 10.5 0 0.5 2.26 0 0 0.3 13.56 CSH 0 c B yellow 31 154 1 Manhattan 1015400 I MN32 Yorkville 3805 67 149 1 Manhattan 1014900 I MN14 Lincoln Square 3806 +1200001694 1 2015-07-16 2015-07-16 21:38:23 2015-07-16 2015-07-16 21:44:05 0 1 -73.96829986572266 40.755008697509766 -73.98112487792969 40.737327575683594 1 1.4 6.5 0.5 0.5 1.56 0 0 0.3 9.36 CSH 0 )  yellow 14 98 1 Manhattan 1009800 I MN19 Turtle Bay-East Midtown 3808 154 64 1 Manhattan 1006400 I MN21 Gramercy 3808 +1200001739 1 2015-07-17 2015-07-17 11:51:48 2015-07-17 2015-07-17 12:18:22 0 1 -74.00737762451172 40.71173858642578 -73.94960021972656 40.79367446899414 1 8.1 27 0 0.5 0 0 0 0.3 27.8 CRE 0 1 u yellow -102 31 1 Manhattan 1003100 I MN24 SoHo-TriBeCa-Civic Center-Little Italy 3810 120 174.01 1 Manhattan 1017401 E MN33 East Harlem South 3804 +1200001788 2 2015-07-17 2015-07-17 14:52:52 2015-07-17 2015-07-17 15:30:05 0 1 -73.97547912597656 40.7517204284668 -74.00247192382812 40.75526809692383 5 2.26 22 0 0.5 4.56 0 0 0.3 27.36 CSH 0 b F yellow -30 92 1 Manhattan 1009200 I MN19 Turtle Bay-East Midtown 3808 50 99 1 Manhattan 1009900 I MN13 Hudson Yards-Chelsea-Flatiron-Union Square 3807 +1200001795 1 2015-07-17 2015-07-17 15:16:40 2015-07-17 2015-07-17 15:34:46 0 1 -74.00824737548828 40.7119026184082 -74.01537322998047 40.70881652832031 3 1 11.5 0 0.5 0 0 0 0.3 12.3 CRE 0 1  yellow 47 21 1 Manhattan 1002100 I MN24 SoHo-TriBeCa-Civic Center-Little Italy 3810 54 317.04 1 Manhattan 1031704 I MN25 Battery Park City-Lower Manhattan 3810 +1200001821 2 2015-07-17 2015-07-17 17:15:36 2015-07-17 2015-07-17 17:27:47 0 1 -74.00386047363281 40.73830795288086 -73.99574279785156 40.72505569458008 1 1.37 9 1 0.5 1 0 0 0.3 11.8 CSH 0 I D yellow 78 77 1 Manhattan 1007700 I MN23 West Village 3810 189 43 1 Manhattan 1004300 I MN24 SoHo-TriBeCa-Civic Center-Little Italy 3810 +1200001856 2 2015-07-17 2015-07-17 19:19:51 2015-07-17 2015-07-17 19:30:14 0 1 -73.94436645507812 40.7116813659668 -73.94243621826172 40.700355529785156 1 1.15 8 1 0.5 0 0 0 0.3 9.8 CRE 0 � 7 yellow -17 495 3 Brooklyn 3049500 I BK90 East Williamsburg 4001 163 285.01 3 Brooklyn 3028501 I BK78 Bushwick South 4002 +1200001861 2 2015-07-17 2015-07-17 19:27:18 2015-07-17 2015-07-17 19:40:16 0 1 -73.95530700683594 40.779258728027344 -73.97895050048828 40.789398193359375 1 1.5 10 1 0.5 2 0 0 0.3 13.8 CSH 0 6 8 yellow 28 148.01 1 Manhattan 1014801 I MN40 Upper East Side-Carnegie Hill 3805 138 175 1 Manhattan 1017500 I MN12 Upper West Side 3806 +1200001868 2 2015-07-17 2015-07-17 19:39:12 2015-07-17 2015-07-17 19:46:10 0 1 -73.95748138427734 40.77399826049805 -73.9762191772461 40.785865783691406 1 1.47 7 1 0.5 1.76 0 0 0.3 10.56 CSH 0 A 9 yellow -38 134 1 Manhattan 1013400 I MN31 Lenox Hill-Roosevelt Island 3805 136 169 1 Manhattan 1016900 I MN12 Upper West Side 3806 +1200001875 1 2015-07-17 2015-07-17 20:00:24 2015-07-17 2015-07-17 20:25:37 0 1 -73.9810562133789 40.761043548583984 -73.98876953125 40.73145294189453 1 2.6 17 0.5 0.5 0 0 0 0.3 18.3 CRE 0 0 y yellow 3 131 1 Manhattan 1013100 I MN17 Midtown-Midtown South 3807 200 42 1 Manhattan 1004200 I MN22 East Village 3809 +1200001877 1 2015-07-17 2015-07-17 20:03:20 2015-07-17 2015-07-17 20:19:33 0 1 -73.9791488647461 40.74711608886719 -73.99269104003906 40.72142028808594 1 2.1 12 0.5 0.5 2.65 0 0 0.3 15.95 CSH 0 p H yellow -123 80 1 Manhattan 1008000 I MN20 Murray Hill-Kips Bay 3808 193 36.01 1 Manhattan 1003601 E MN27 Chinatown 3809 +1200001903 2 2015-07-17 2015-07-17 21:26:21 2015-07-17 2015-07-17 21:35:01 0 1 -73.96847534179688 40.75965118408203 -73.98286437988281 40.77116012573242 2 1.4 8 0.5 0.5 0 0 0 0.3 9.3 CRE 0 b B yellow 15 100 1 Manhattan 1010000 I MN19 Turtle Bay-East Midtown 3808 67 149 1 Manhattan 1014900 I MN14 Lincoln Square 3806 +1200001948 2 2015-07-17 2015-07-17 23:14:28 2015-07-17 2015-07-17 23:24:43 0 1 -74.00115203857422 40.74171829223633 -74.00859069824219 40.73329162597656 3 0.96 8 0.5 0.5 2.32 0 0 0.3 11.62 CSH 0 h X yellow 75 83 1 Manhattan 1008300 E MN13 Hudson Yards-Chelsea-Flatiron-Union Square 3807 150 75 1 Manhattan 1007500 I MN23 West Village 3810 +1200002006 2 2015-07-18 2015-07-18 02:36:17 2015-07-18 2015-07-18 02:58:36 0 1 -73.97046661376953 40.793758392333984 -74.00756072998047 40.70486831665039 1 8.47 26 0.5 0.5 0 0 0 0.3 27.3 CRE 0 8 � yellow 24 181 1 Manhattan 1018100 I MN12 Upper West Side 3806 255 7 1 Manhattan 1000700 I MN25 Battery Park City-Lower Manhattan 3810 +1200002010 1 2015-07-18 2015-07-18 03:02:13 2015-07-18 2015-07-18 03:10:25 0 1 -74.00401306152344 40.743038177490234 -73.98571014404297 40.7438850402832 1 1.4 8 0.5 0.5 2.75 0 0 0.3 12.05 CSH 0 h 4 yellow 75 83 1 Manhattan 1008300 E MN13 Hudson Yards-Chelsea-Flatiron-Union Square 3807 166 56 1 Manhattan 1005600 I MN13 Hudson Yards-Chelsea-Flatiron-Union Square 3807 +1200002017 1 2015-07-18 2015-07-18 03:31:33 2015-07-18 2015-07-18 03:34:11 0 1 -73.93047332763672 40.69890594482422 -73.93529510498047 40.69718933105469 1 0.3 4 0.5 0.5 0 0 0 0.3 5.3 CRE 0 7 7 yellow 65 425 3 Brooklyn 3042500 E BK78 Bushwick South 4002 20 389 3 Brooklyn 3038900 E BK78 Bushwick South 4002 +1200002028 2 2015-07-18 2015-07-18 04:19:32 2015-07-18 2015-07-18 04:25:19 0 1 -74.00269317626953 40.749778747558594 -73.97994995117188 40.77613067626953 1 2.35 8.5 0.5 0.5 1.96 0 0 0.3 11.76 CSH 0 F B yellow 50 99 1 Manhattan 1009900 I MN13 Hudson Yards-Chelsea-Flatiron-Union Square 3807 30 153 1 Manhattan 1015300 I MN14 Lincoln Square 3806 +1200002040 2 2015-07-18 2015-07-18 06:57:19 2015-07-18 2015-07-18 07:04:07 0 1 -73.95987701416016 40.77362823486328 -73.99073028564453 40.75083923339844 2 2.83 9.5 0 0.5 2.06 0 0 0.3 12.36 CSH 0 7 � yellow 18 128 1 Manhattan 1012800 I MN40 Upper East Side-Carnegie Hill 3805 131 101 1 Manhattan 1010100 I MN17 Midtown-Midtown South 3807 +1200002105 2 2015-07-18 2015-07-18 11:16:46 2015-07-18 2015-07-18 11:21:47 0 1 -73.98150634765625 40.78093338012695 -73.98328399658203 40.77672576904297 1 0.52 5 0 0.5 0 0 0 0.3 5.8 CRE 0 9 C yellow -101 163 1 Manhattan 1016300 I MN12 Upper West Side 3806 206 155 1 Manhattan 1015500 I MN14 Lincoln Square 3806 +1200002108 2 2015-07-18 2015-07-18 11:26:00 2015-07-18 2015-07-18 11:56:54 0 1 -73.77677154541016 40.64555740356445 -73.93250274658203 40.763648986816406 4 15.47 43 0 0.5 0 0 0 0.3 43.8 CRE 0 2  yellow 8 716 4 Queens 4071600 I QN98 Airport 4105 80 45 4 Queens 4004500 I QN70 Astoria 4101 +1200002116 1 2015-07-18 2015-07-18 11:48:54 2015-07-18 2015-07-18 11:56:11 0 1 -73.9764633178711 40.75680160522461 -73.95606231689453 40.78461456298828 1 2.1 8.5 0 0.5 1.5 0 0 0.3 10.8 CSH 0 a 6 yellow -31 94 1 Manhattan 1009400 E MN17 Midtown-Midtown South 3807 34 160.01 1 Manhattan 1016001 I MN40 Upper East Side-Carnegie Hill 3805 +1200002126 1 2015-07-18 2015-07-18 12:22:03 2015-07-18 2015-07-18 12:33:40 0 1 -73.99137878417969 40.727294921875 -73.97683715820312 40.75163650512695 1 1.9 10 0 0.5 2.15 0 0 0.3 12.95 CSH 0 y p yellow -66 38 1 Manhattan 1003800 I MN22 East Village 3809 133 80 1 Manhattan 1008000 I MN20 Murray Hill-Kips Bay 3808 +1200002193 2 2015-07-18 2015-07-18 16:46:39 2015-07-18 2015-07-18 16:55:08 0 1 -73.99213409423828 40.75910949707031 -73.9886703491211 40.73866653442383 1 2 8.5 0 0.5 1.86 0 0 0.3 11.16 CSH 0 H 4 yellow -117 121 1 Manhattan 1012100 E MN15 Clinton 3807 245 52 1 Manhattan 1005200 I MN13 Hudson Yards-Chelsea-Flatiron-Union Square 3807 +1200002213 2 2015-07-18 2015-07-18 17:57:16 2015-07-18 2015-07-18 18:04:24 0 1 -73.99067687988281 40.72894287109375 -73.9975814819336 40.71684646606445 4 1.31 7 0 0.5 0 0 0 0.3 7.8 CRE 0 y D yellow -56 42 1 Manhattan 1004200 I MN22 East Village 3809 196 41 1 Manhattan 1004100 E MN24 SoHo-TriBeCa-Civic Center-Little Italy 3810 +1200002225 2 2015-07-18 2015-07-18 18:38:28 2015-07-18 2015-07-18 18:49:06 0 1 -73.98584747314453 40.75408935546875 -74.00125885009766 40.76259994506836 1 1.24 8.5 0 0.5 0 0 0 0.3 9.3 CRE 0  P yellow 21 113 1 Manhattan 1011300 I MN17 Midtown-Midtown South 3807 140 129 1 Manhattan 1012900 I MN15 Clinton 3807 +1200002230 2 2015-07-18 2015-07-18 18:45:04 2015-07-18 2015-07-18 19:21:01 0 2 -73.7892074584961 40.641788482666016 -73.952880859375 40.76900863647461 5 19.21 52 0 0.5 11.67 5.54 0 0.3 70.01 CSH 0 2 @ yellow 8 716 4 Queens 4071600 I QN98 Airport 4105 161 132 1 Manhattan 1013200 I MN31 Lenox Hill-Roosevelt Island 3805 +1200002247 2 2015-07-18 2015-07-18 19:57:09 2015-07-18 2015-07-18 20:09:19 0 1 -73.99668884277344 40.723228454589844 -73.97313690185547 40.75627136230469 6 2.7199999999999998 11 0 0.5 0 0 0 0.3 11.8 CSH 0 D b yellow -67 43 1 Manhattan 1004300 I MN24 SoHo-TriBeCa-Civic Center-Little Italy 3810 15 100 1 Manhattan 1010000 I MN19 Turtle Bay-East Midtown 3808 +1200002267 1 2015-07-18 2015-07-18 20:41:41 2015-07-18 2015-07-18 20:44:29 0 1 -73.97386169433594 40.763938903808594 -73.9674072265625 40.7667236328125 1 0.5 4 0.5 0.5 0 0 0 0.3 5.3 CRE 0 c 7 yellow 18 112.01 1 Manhattan 1011201 I MN17 Midtown-Midtown South 3807 228 122 1 Manhattan 1012200 I MN40 Upper East Side-Carnegie Hill 3805 +1200002277 1 2015-07-18 2015-07-18 20:55:28 2015-07-18 2015-07-18 21:02:38 0 1 -73.98158264160156 40.747196197509766 -73.99498748779297 40.729000091552734 1 1.5 7 0.5 0.5 2.05 0 0 0.3 10.35 CSH 0 d  yellow -125 74 1 Manhattan 1007400 I MN17 Midtown-Midtown South 3807 163 59 1 Manhattan 1005900 I MN23 West Village 3810 +1200002279 1 2015-07-18 2015-07-18 20:57:15 2015-07-18 2015-07-18 21:00:08 0 1 -73.94102478027344 40.798282623291016 -73.93235778808594 40.79478454589844 1 0.5 4 0.5 0.5 1.05 0 0 0.3 6.35 CSH 0 t t yellow -26 182 1 Manhattan 1018200 E MN34 East Harlem North 3804 17 178 1 Manhattan 1017800 E MN34 East Harlem North 3804 +1200002280 1 2015-07-18 2015-07-18 21:03:10 2015-07-18 2015-07-18 21:16:16 0 1 -74.013916015625 40.717716217041016 -73.9810791015625 40.703250885009766 1 2.7 11.5 0.5 0.5 1 0 0 0.3 13.8 CSH 0  f yellow 52 317.03 1 Manhattan 1031703 I MN25 Battery Park City-Lower Manhattan 3810 248 21 3 Brooklyn 3002100 I BK38 DUMBO-Vinegar Hill-Downtown Brooklyn-Boerum Hill 4004 +1200002290 2 2015-07-18 2015-07-18 21:41:16 2015-07-18 2015-07-18 22:06:06 0 1 -73.9897689819336 40.72573471069336 -73.93791961669922 40.80068588256836 4 4.64 18.5 0.5 0.5 0 0 0 0.3 19.8 CRE 0 y t yellow -66 38 1 Manhattan 1003800 I MN22 East Village 3809 232 194 1 Manhattan 1019400 E MN34 East Harlem North 3804 +1200002291 2 2015-07-18 2015-07-18 21:42:11 2015-07-18 2015-07-18 21:50:59 0 1 -73.99507904052734 40.724876403808594 -73.97960662841797 40.72365951538086 1 1.11 7 0.5 0.5 1 0 0 0.3 9.3 CSH 0 D  yellow -67 43 1 Manhattan 1004300 I MN24 SoHo-TriBeCa-Civic Center-Little Italy 3810 148 26.02 1 Manhattan 1002602 E MN28 Lower East Side 3809 +1200002297 1 2015-07-18 2015-07-18 21:53:06 2015-07-18 2015-07-18 22:02:37 0 1 -73.98822784423828 40.729679107666016 -74.00459289550781 40.73464584350586 1 1 7.5 0.5 0.5 1.75 0 0 0.3 10.55 CSH 0 y I yellow 93 40 1 Manhattan 1004000 I MN22 East Village 3809 163 73 1 Manhattan 1007300 I MN23 West Village 3810 +1200002298 2 2015-07-18 2015-07-18 22:04:49 2015-07-18 2015-07-18 22:06:01 0 1 -73.97142028808594 40.750831604003906 -73.97337341308594 40.748191833496094 1 0.23 3 0.5 0.5 0 0 0 0.3 4.3 CRE 0 3 3 yellow -105 88 1 Manhattan 1008800 I MN19 Turtle Bay-East Midtown 3808 151 88 1 Manhattan 1008800 I MN19 Turtle Bay-East Midtown 3808 +1200002354 1 2015-07-19 2015-07-19 00:39:49 2015-07-19 2015-07-19 00:43:29 0 1 -73.98346710205078 40.73889923095703 -73.98664855957031 40.733055114746094 1 0.5 4.5 0.5 0.5 1 0 0 0.3 6.8 CSH 0  y yellow 11 68 1 Manhattan 1006800 I MN21 Gramercy 3808 93 40 1 Manhattan 1004000 I MN22 East Village 3809 +1200002402 2 2015-07-19 2015-07-19 03:12:58 2015-07-19 2015-07-19 03:41:58 0 1 -73.9828109741211 40.73136901855469 -73.90953826904297 40.76525115966797 2 10.11 33 0.5 0.5 6.86 0 0 0.3 41.16 CSH 0 y  yellow 93 40 1 Manhattan 1004000 I MN22 East Village 3809 69 143 4 Queens 4014300 E QN70 Astoria 4101 +1200002404 2 2015-07-19 2015-07-19 03:17:58 2015-07-19 2015-07-19 03:31:43 0 1 -73.98941040039062 40.72635269165039 -73.97490692138672 40.68038558959961 3 3.59 14 0.5 0.5 0 0 0 0.3 15.3 CRE 0 y � yellow -66 38 1 Manhattan 1003800 I MN22 East Village 3809 40 129.02 3 Brooklyn 3012902 I BK37 Park Slope-Gowanus 4005 +1200002424 1 2015-07-19 2015-07-19 06:29:03 2015-07-19 2015-07-19 06:31:01 0 1 -73.98233032226562 40.73149108886719 -73.97478485107422 40.74196243286133 1 0.8 4.5 0 0.5 0 0 0 0.3 5.3 CRE 0  7 yellow 52 48 1 Manhattan 1004800 I MN21 Gramercy 3808 85 62 1 Manhattan 1006200 I MN20 Murray Hill-Kips Bay 3808 +1200002426 1 2015-07-19 2015-07-19 06:03:04 2015-07-19 2015-07-19 06:35:06 0 1 -73.9957275390625 40.7647819519043 -73.92303466796875 40.86890411376953 1 11.2 37 0 0.5 4 0 0 0.3 41.8 CSH 0 P \' yellow -116 129 1 Manhattan 1012900 I MN15 Clinton 3807 101 295 1 Manhattan 1029500 E MN01 Marble Hill-Inwood 3801 +1200002440 2 2015-07-19 2015-07-19 09:25:56 2015-07-19 2015-07-19 09:44:36 0 1 -73.87449645996094 40.77394104003906 -73.97428894042969 40.73699951171875 1 9.23 27 0 0.5 6 5.54 0 0.3 39.34 CSH 0 8 7 yellow 48 331 4 Queens 4033100 I QN98 Airport 4102 85 62 1 Manhattan 1006200 I MN20 Murray Hill-Kips Bay 3808 +1200002444 1 2015-07-19 2015-07-19 09:42:06 2015-07-19 2015-07-19 09:47:06 0 1 -73.95204162597656 40.798702239990234 -73.94396209716797 40.81477355957031 1 1.3 6.5 0 0.5 1.45 0 0 0.3 8.75 CSH 0 A B yellow 122 186 1 Manhattan 1018600 E MN11 Central Harlem South 3803 62 228 1 Manhattan 1022800 E MN03 Central Harlem North-Polo Grounds 3803 +1200002464 1 2015-07-19 2015-07-19 10:56:01 2015-07-19 2015-07-19 11:11:49 0 1 -73.98196411132812 40.748111724853516 -73.98190307617188 40.77851867675781 2 2.5 12.5 0 0.5 2 0 0 0.3 15.3 CSH 0 d C yellow -125 74 1 Manhattan 1007400 I MN17 Midtown-Midtown South 3807 154 159 1 Manhattan 1015900 I MN14 Lincoln Square 3806 +1200002467 1 2015-07-19 2015-07-19 10:59:47 2015-07-19 2015-07-19 11:17:15 0 1 -73.98756408691406 40.71848678588867 -73.99395751953125 40.7614860534668 1 4.1 16.5 0 0.5 3.45 0 0 0.3 20.75 CSH 0 H H yellow -61 18 1 Manhattan 1001800 E MN27 Chinatown 3809 139 121 1 Manhattan 1012100 E MN15 Clinton 3807 +1200002477 1 2015-07-19 2015-07-19 11:48:00 2015-07-19 2015-07-19 12:00:35 0 1 -73.95216369628906 40.76914978027344 -73.98869323730469 40.774410247802734 1 2.4 11 0 0.5 2.35 0 0 0.3 14.15 CSH 0 @ C yellow -95 132 1 Manhattan 1013200 I MN31 Lenox Hill-Roosevelt Island 3805 205 151 1 Manhattan 1015100 E MN14 Lincoln Square 3806 +1200002491 1 2015-07-19 2015-07-19 12:20:46 2015-07-19 2015-07-19 12:24:12 0 1 -74.01510620117188 40.71125030517578 -74.01625061035156 40.704742431640625 1 0.5 4.5 0 0.5 0 0 0 0.3 5.3 CRE 0   yellow 54 317.04 1 Manhattan 1031704 I MN25 Battery Park City-Lower Manhattan 3810 44 319 1 Manhattan 1031900 I MN99 park-cemetery-etc-Manhattan 3810 +1200002503 2 2015-07-19 2015-07-19 13:05:42 2015-07-19 2015-07-19 13:10:12 0 1 -73.96560668945312 40.76851272583008 -73.95579528808594 40.76601791381836 1 0.69 5 0 0.5 0 0 0 0.3 5.8 CRE 0 7 @ yellow 17 120 1 Manhattan 1012000 I MN40 Upper East Side-Carnegie Hill 3805 160 124 1 Manhattan 1012400 I MN31 Lenox Hill-Roosevelt Island 3805 +1200002505 2 2015-07-19 2015-07-19 13:09:51 2015-07-19 2015-07-19 13:21:18 0 1 -73.97736358642578 40.764591217041016 -73.9914321899414 40.74994659423828 2 1.63 9.5 0 0.5 3.09 0 0 0.3 13.39 CSH 0 c � yellow 4 137 1 Manhattan 1013700 I MN17 Midtown-Midtown South 3807 131 101 1 Manhattan 1010100 I MN17 Midtown-Midtown South 3807 +1200002526 2 2015-07-19 2015-07-19 13:47:26 2015-07-19 2015-07-19 13:59:51 0 1 -73.98465728759766 40.779659271240234 -73.97065734863281 40.76333999633789 1 1.74 10 0 0.5 2.16 0 0 0.3 12.96 CSH 0 C 7 yellow -102 159 1 Manhattan 1015900 I MN14 Lincoln Square 3806 227 114.01 1 Manhattan 1011401 I MN40 Upper East Side-Carnegie Hill 3805 +1200002544 2 2015-07-19 2015-07-19 14:53:42 2015-07-19 2015-07-19 14:57:58 0 1 -74.00567626953125 40.74034881591797 -73.99463653564453 40.745750427246094 3 0.97 5.5 0 0.5 0 0 0 0.3 6.3 CRE 0 X � yellow 79 79 1 Manhattan 1007900 I MN23 West Village 3810 132 91 1 Manhattan 1009100 I MN13 Hudson Yards-Chelsea-Flatiron-Union Square 3807 +1200002546 2 2015-07-19 2015-07-19 15:04:36 2015-07-19 2015-07-19 15:30:57 0 1 -73.97882843017578 40.76120376586914 -73.8618392944336 40.76778793334961 6 10.88 32.5 0 0.5 9.71 5.54 0 0.3 48.55 CSH 0 a 8 yellow 17 104 1 Manhattan 1010400 I MN17 Midtown-Midtown South 3807 48 331 4 Queens 4033100 I QN98 Airport 4102 +1200002551 1 2015-07-19 2015-07-19 15:31:00 2015-07-19 2015-07-19 15:45:26 0 1 -74.01486206054688 40.71369552612305 -73.98373413085938 40.73097610473633 1 5.2 17 0 0.5 0 0 0 0.3 17.8 CSH 0  y yellow 52 317.03 1 Manhattan 1031703 I MN25 Battery Park City-Lower Manhattan 3810 93 40 1 Manhattan 1004000 I MN22 East Village 3809 +1200002553 2 2015-07-19 2015-07-19 15:40:02 2015-07-19 2015-07-19 15:49:25 0 1 -73.95724487304688 40.76198196411133 -73.98087310791016 40.7213134765625 1 3.7199999999999998 12.5 0 0.5 0 0 0 0.3 13.3 CRE 0 @  yellow -34 116 1 Manhattan 1011600 I MN31 Lenox Hill-Roosevelt Island 3805 9 22.02 1 Manhattan 1002202 I MN28 Lower East Side 3809 +1200002577 1 2015-07-19 2015-07-19 17:01:31 2015-07-19 2015-07-19 17:20:23 0 1 -73.96908569335938 40.75529861450195 -74.0043716430664 40.730865478515625 2 3.2 14.5 0 0.5 2 0 0 0.3 17.3 CSH 0 ) I yellow 14 98 1 Manhattan 1009800 I MN19 Turtle Bay-East Midtown 3808 36 67 1 Manhattan 1006700 I MN23 West Village 3810 +1200002591 2 2015-07-19 2015-07-19 17:49:58 2015-07-19 2015-07-19 17:54:39 0 1 -73.97291564941406 40.79526138305664 -73.97631072998047 40.788536071777344 1 0.83 5.5 0 0.5 0 0 0 0.3 6.3 CRE 0 8 8 yellow -88 183 1 Manhattan 1018300 I MN12 Upper West Side 3806 138 175 1 Manhattan 1017500 I MN12 Upper West Side 3806 +1200002616 2 2015-07-19 2015-07-19 19:03:17 2015-07-19 2015-07-19 19:23:38 0 1 -73.96902465820312 40.80094528198242 -73.87104034423828 40.774139404296875 5 8.26 25.5 0 0.5 6.37 5.54 0 0.3 38.21 CSH 0 Q 8 yellow 112 191 1 Manhattan 1019100 I MN12 Upper West Side 3806 48 331 4 Queens 4033100 I QN98 Airport 4102 +1200002637 1 2015-07-19 2015-07-19 20:15:48 2015-07-19 2015-07-19 20:28:13 0 1 -74.01289367675781 40.706512451171875 -73.98177337646484 40.74714279174805 2 5.5 17.5 0.5 0.5 2 0 0 0.3 20.8 CSH 0 a d yellow 53 13 1 Manhattan 1001300 I MN25 Battery Park City-Lower Manhattan 3810 131 74 1 Manhattan 1007400 I MN17 Midtown-Midtown South 3807 +1200002640 2 2015-07-19 2015-07-19 20:19:27 2015-07-19 2015-07-19 20:29:15 0 1 -73.99398803710938 40.74126052856445 -73.99223327636719 40.724430084228516 5 1.7 8.5 0.5 0.5 1.96 0 0 0.3 11.76 CSH 0 4 y yellow -114 54 1 Manhattan 1005400 I MN13 Hudson Yards-Chelsea-Flatiron-Union Square 3807 194 36.02 1 Manhattan 1003602 I MN22 East Village 3809 +1200002658 2 2015-07-19 2015-07-19 20:56:46 2015-07-19 2015-07-19 21:05:52 0 1 -73.97624969482422 40.76591873168945 -73.97581481933594 40.78912353515625 1 1.95 9 0.5 0.5 0 0 0 0.3 10.3 CRE 0 C 8 yellow 43 143 1 Manhattan 1014300 I MN99 park-cemetery-etc-Manhattan 3806 138 175 1 Manhattan 1017500 I MN12 Upper West Side 3806 +1200002697 1 2015-07-19 2015-07-19 23:19:21 2015-07-19 2015-07-19 23:21:52 0 1 -73.98899841308594 40.75801467895508 -73.99749755859375 40.75691604614258 1 0.7 4 0.5 0.5 1.05 0 0 0.3 6.35 CSH 0 0 F yellow -123 119 1 Manhattan 1011900 E MN17 Midtown-Midtown South 3807 50 99 1 Manhattan 1009900 I MN13 Hudson Yards-Chelsea-Flatiron-Union Square 3807 +1200002708 2 2015-07-20 2015-07-20 00:23:00 2015-07-20 2015-07-20 00:46:51 0 2 -73.77685546875 40.64600372314453 -73.9712905883789 40.753536224365234 1 17.84 52 0 0.5 5 5.54 0 0.3 63.34 CSH 0 2 3 yellow 8 716 4 Queens 4071600 I QN98 Airport 4105 152 90 1 Manhattan 1009000 I MN19 Turtle Bay-East Midtown 3808 +1200002716 1 2015-07-19 2015-07-19 17:29:17 2015-07-19 2015-07-19 17:56:38 0 1 -73.9725341796875 40.780975341796875 -73.99459075927734 40.745487213134766 1 2.8 18 0 0.5 0 0 0 0.3 18.8 CRE 0 9 � yellow 22 165 1 Manhattan 1016500 I MN12 Upper West Side 3806 132 91 1 Manhattan 1009100 I MN13 Hudson Yards-Chelsea-Flatiron-Union Square 3807 +1200002719 2 2015-07-20 2015-07-20 03:28:48 2015-07-20 2015-07-20 03:33:36 0 1 -73.87342834472656 40.750911712646484 -73.89080047607422 40.74857711791992 1 1 5.5 0.5 0.5 0 0 0 0.3 6.8 CSH 0 ) ) yellow 103 273 4 Queens 4027300 E QN28 Jackson Heights 4102 113 287 4 Queens 4028700 E QN28 Jackson Heights 4102 +1200002733 2 2015-07-20 2015-07-20 06:37:16 2015-07-20 2015-07-20 06:46:17 0 1 -74.01148223876953 40.71567153930664 -73.97930908203125 40.761478424072266 1 3.81 12 0 0.5 2.56 0 0 0.3 15.36 CSH 0 1 0 yellow 47 21 1 Manhattan 1002100 I MN24 SoHo-TriBeCa-Civic Center-Little Italy 3810 3 131 1 Manhattan 1013100 I MN17 Midtown-Midtown South 3807 +1200002749 1 2015-07-20 2015-07-20 07:31:27 2015-07-20 2015-07-20 07:43:47 0 1 -73.9913558959961 40.75007247924805 -73.97430419921875 40.76243591308594 1 1.5 9.5 0 0.5 0 0 0 0.3 10.3 CRE 0 � c yellow -125 101 1 Manhattan 1010100 I MN17 Midtown-Midtown South 3807 19 112.02 1 Manhattan 1011202 I MN17 Midtown-Midtown South 3807 +1200002790 1 2015-07-20 2015-07-20 09:50:50 2015-07-20 2015-07-20 09:58:12 0 1 -73.97554016113281 40.75223159790039 -73.98535919189453 40.7430534362793 1 0.9 6.5 0 0.5 1.45 0 0 0.3 8.75 CSH 0 b 4 yellow -30 92 1 Manhattan 1009200 I MN19 Turtle Bay-East Midtown 3808 166 56 1 Manhattan 1005600 I MN13 Hudson Yards-Chelsea-Flatiron-Union Square 3807 +1200002794 2 2015-07-20 2015-07-20 10:04:19 2015-07-20 2015-07-20 10:31:27 0 1 -73.97110748291016 40.76374053955078 -73.9970474243164 40.696861267089844 2 7.9399999999999995 26.5 0 0.5 2.5 0 0 0.3 29.8 CSH 0 7 3 yellow -29 114.01 1 Manhattan 1011401 I MN40 Upper East Side-Carnegie Hill 3805 116 3.01 3 Brooklyn 3000301 I BK09 Brooklyn Heights-Cobble Hill 4004 +1200002803 1 2015-07-20 2015-07-20 10:26:02 2015-07-20 2015-07-20 10:35:20 0 1 -73.9686050415039 40.77027893066406 -73.97756958007812 40.764408111572266 1 0.8 7.5 0 0.5 0 0 0 0.3 8.3 CRE 0 7 c yellow -28 122 1 Manhattan 1012200 I MN40 Upper East Side-Carnegie Hill 3805 4 137 1 Manhattan 1013700 I MN17 Midtown-Midtown South 3807 +1200002806 2 2015-07-20 2015-07-20 10:42:47 2015-07-20 2015-07-20 10:49:05 0 1 -73.96440124511719 40.770198822021484 -73.96778106689453 40.76258850097656 2 0.77 6 0 0.5 0 0 0 0.3 6.8 CRE 0 7 7 yellow 18 128 1 Manhattan 1012800 I MN40 Upper East Side-Carnegie Hill 3805 22 114.02 1 Manhattan 1011402 I MN40 Upper East Side-Carnegie Hill 3805 +1200002832 2 2015-07-20 2015-07-20 11:53:29 2015-07-20 2015-07-20 12:18:16 0 1 -73.97740936279297 40.76416015625 -73.99762725830078 40.72184753417969 1 3.68 17 0 0.5 0 0 0 0.3 17.8 CRE 0 c D yellow 4 137 1 Manhattan 1013700 I MN17 Midtown-Midtown South 3807 124 45 1 Manhattan 1004500 I MN24 SoHo-TriBeCa-Civic Center-Little Italy 3810 +1200002879 2 2015-07-20 2015-07-20 14:14:33 2015-07-20 2015-07-20 14:40:17 0 1 -73.97686767578125 40.79069137573242 -73.87310028076172 40.774471282958984 5 9.29 29.5 0 0.5 8.96 5.54 0 0.3 44.8 CSH 0 8 8 yellow -118 175 1 Manhattan 1017500 I MN12 Upper West Side 3806 48 331 4 Queens 4033100 I QN98 Airport 4102 +1200002892 1 2015-07-20 2015-07-20 14:51:36 2015-07-20 2015-07-20 15:37:57 0 2 -73.9853286743164 40.746978759765625 -73.7823486328125 40.648807525634766 1 16.9 52 0 0.5 0 5.54 0 0.3 58.34 CRE 0 d 2 yellow -125 74 1 Manhattan 1007400 I MN17 Midtown-Midtown South 3807 8 716 4 Queens 4071600 I QN98 Airport 4105 +1200002899 2 2015-07-20 2015-07-20 15:07:02 2015-07-20 2015-07-20 15:33:26 0 1 -73.99408721923828 40.751258850097656 -73.96881866455078 40.764404296875 6 2.34 16.5 0 0.5 0 0 0 0.3 17.3 CRE 0 � 7 yellow -125 101 1 Manhattan 1010100 I MN17 Midtown-Midtown South 3807 227 114.01 1 Manhattan 1011401 I MN40 Upper East Side-Carnegie Hill 3805 +1200002917 2 2015-07-20 2015-07-20 16:41:34 2015-07-20 2015-07-20 17:19:14 0 1 -73.77670288085938 40.64539337158203 -73.93128967285156 40.694454193115234 1 11.1 36.5 1 0.5 4 0 0 0.3 42.3 CSH 0 2 % yellow 8 716 4 Queens 4071600 I QN98 Airport 4105 43 289 3 Brooklyn 3028900 E BK35 Stuyvesant Heights 4003 +1200002942 2 2015-07-20 2015-07-20 17:57:40 2015-07-20 2015-07-20 18:16:47 0 1 -73.98847198486328 40.74593734741211 -73.989013671875 40.758174896240234 1 1.35 12.5 1 0.5 2 0 0 0.3 16.3 CSH 0 d 0 yellow -48 76 1 Manhattan 1007600 I MN17 Midtown-Midtown South 3807 133 119 1 Manhattan 1011900 E MN17 Midtown-Midtown South 3807 +1200002975 2 2015-07-20 2015-07-20 19:02:12 2015-07-20 2015-07-20 19:17:33 0 1 -73.96691131591797 40.79378128051758 -73.93804931640625 40.81768035888672 1 2.81 13 1 0.5 2.96 0 0 0.3 17.76 CSH 0 Q B yellow 25 185 1 Manhattan 1018500 I MN12 Upper West Side 3806 51 214 1 Manhattan 1021400 E MN03 Central Harlem North-Polo Grounds 3803 +1200002978 1 2015-07-20 2015-07-20 19:10:25 2015-07-20 2015-07-20 19:24:25 0 1 -74.00444793701172 40.707550048828125 -73.94417572021484 40.71162796020508 2 3.7 14 1 0.5 3.15 0 0 0.3 18.95 CSH 0 � � yellow 12 15.02 1 Manhattan 1001502 I MN25 Battery Park City-Lower Manhattan 3810 239 495 3 Brooklyn 3049500 I BK90 East Williamsburg 4001 +1200002985 2 2015-07-20 2015-07-20 19:42:04 2015-07-20 2015-07-20 19:51:28 0 1 -73.99142456054688 40.749515533447266 -73.97360229492188 40.750518798828125 3 1.48 8 1 0.5 2.45 0 0 0.3 12.25 CSH 0 � 3 yellow -125 101 1 Manhattan 1010100 I MN17 Midtown-Midtown South 3807 151 88 1 Manhattan 1008800 I MN19 Turtle Bay-East Midtown 3808 +1200003016 1 2015-07-20 2015-07-20 20:49:23 2015-07-20 2015-07-20 21:15:00 0 1 -73.97926330566406 40.75225830078125 -73.98099517822266 40.67177963256836 1 8.4 27.5 0.5 0.5 4 0 0 0.3 32.8 CSH 0 p � yellow 93 82 1 Manhattan 1008200 I MN20 Murray Hill-Kips Bay 3808 126 135 3 Brooklyn 3013500 I BK37 Park Slope-Gowanus 4005 +1200003028 1 2015-07-20 2015-07-20 21:31:15 2015-07-20 2015-07-20 21:35:28 0 1 -73.97804260253906 40.752315521240234 -73.99005889892578 40.75703048706055 1 0.7 5 0.5 0.5 0 0 0 0.3 6.3 CSH 0 b H yellow -30 92 1 Manhattan 1009200 I MN19 Turtle Bay-East Midtown 3808 163 115 1 Manhattan 1011500 I MN15 Clinton 3807 +1200003113 1 2015-07-21 2015-07-21 07:40:21 2015-07-21 2015-07-21 07:48:05 0 1 -73.99527740478516 40.74955749511719 -73.9819107055664 40.745723724365234 1 0.8 6.5 0 0.5 1.8 0 0 0.3 9.1 CSH 0 � p yellow -125 95 1 Manhattan 1009500 I MN17 Midtown-Midtown South 3807 130 72 1 Manhattan 1007200 I MN20 Murray Hill-Kips Bay 3808 +1200003128 1 2015-07-21 2015-07-21 08:41:15 2015-07-21 2015-07-21 08:49:02 0 1 -73.95735931396484 40.782630920410156 -73.976318359375 40.787967681884766 1 1.7 8 0 0.5 2.2 0 0 0.3 11 CSH 0 6 9 yellow 74 150.02 1 Manhattan 1015002 I MN40 Upper East Side-Carnegie Hill 3805 166 171 1 Manhattan 1017100 I MN12 Upper West Side 3806 +1200003130 1 2015-07-21 2015-07-21 08:46:30 2015-07-21 2015-07-21 09:20:20 0 1 -73.95117950439453 40.78268051147461 -74.01029205322266 40.709869384765625 1 7.7 29.5 0 0.5 7.55 0 0 0.3 37.85 CSH 0 6 a yellow -103 158.01 1 Manhattan 1015801 I MN40 Upper East Side-Carnegie Hill 3805 53 13 1 Manhattan 1001300 I MN25 Battery Park City-Lower Manhattan 3810 +1200003149 2 2015-07-21 2015-07-21 09:57:45 2015-07-21 2015-07-21 10:33:34 0 1 -74.00782775878906 40.70505142211914 -73.98185729980469 40.762699127197266 1 6.88 29.5 0 0.5 4 0 0 0.3 34.3 CSH 0 � 0 yellow -1 7 1 Manhattan 1000700 I MN25 Battery Park City-Lower Manhattan 3810 3 131 1 Manhattan 1013100 I MN17 Midtown-Midtown South 3807 +1200003215 1 2015-07-21 2015-07-21 13:31:22 2015-07-21 2015-07-21 13:45:46 0 1 -73.99177551269531 40.749698638916016 -73.98206329345703 40.75236511230469 1 0.9 10 0 0.5 2.15 0 0 0.3 12.95 CSH 0 � d yellow -125 101 1 Manhattan 1010100 I MN17 Midtown-Midtown South 3807 209 84 1 Manhattan 1008400 I MN17 Midtown-Midtown South 3807 +1200003220 2 2015-07-21 2015-07-21 13:40:49 2015-07-21 2015-07-21 13:48:34 0 1 -73.99154663085938 40.754520416259766 -73.98380279541016 40.75114822387695 4 0.48 6 0 0.5 1.36 0 0 0.3 8.16 CSH 0  d yellow -124 109 1 Manhattan 1010900 I MN17 Midtown-Midtown South 3807 209 84 1 Manhattan 1008400 I MN17 Midtown-Midtown South 3807 +1200003229 2 2015-07-21 2015-07-21 14:20:23 2015-07-21 2015-07-21 14:34:56 0 1 -73.97599029541016 40.76492691040039 -73.97969055175781 40.745948791503906 1 1.78 11 0 0.5 0 0 0 0.3 11.8 CRE 0 c p yellow 18 112.01 1 Manhattan 1011201 I MN17 Midtown-Midtown South 3807 130 72 1 Manhattan 1007200 I MN20 Murray Hill-Kips Bay 3808 +1200003233 1 2015-07-21 2015-07-21 14:36:24 2015-07-21 2015-07-21 14:43:25 0 1 -73.96761322021484 40.766380310058594 -73.9524154663086 40.781524658203125 1 1.5 7.5 0 0.5 1.65 0 0 0.3 9.95 CSH 0 7 6 yellow -28 122 1 Manhattan 1012200 I MN40 Upper East Side-Carnegie Hill 3805 29 148.02 1 Manhattan 1014802 I MN40 Upper East Side-Carnegie Hill 3805 +1200003235 1 2015-07-21 2015-07-21 14:39:17 2015-07-21 2015-07-21 14:49:28 0 1 -73.95198822021484 40.77766418457031 -73.9609603881836 40.76545715332031 1 0.9 8 0 0.5 1.75 0 0 0.3 10.55 CSH 0 c A yellow 26 146.01 1 Manhattan 1014601 I MN32 Yorkville 3805 182 118 1 Manhattan 1011800 I MN31 Lenox Hill-Roosevelt Island 3805 +1200003241 2 2015-07-21 2015-07-21 14:48:17 2015-07-21 2015-07-21 14:50:25 0 1 -73.96395874023438 40.76517105102539 -73.96119689941406 40.77006530761719 1 0.39 3.5 0 0.5 1.08 0 0 0.3 5.38 CSH 0 7 7 yellow 17 120 1 Manhattan 1012000 I MN40 Upper East Side-Carnegie Hill 3805 18 128 1 Manhattan 1012800 I MN40 Upper East Side-Carnegie Hill 3805 +1200003242 1 2015-07-21 2015-07-21 14:49:29 2015-07-21 2015-07-21 15:55:04 0 2 -73.78199768066406 40.6447868347168 -73.965087890625 40.8096809387207 1 19.4 52 0 0.5 0 5.54 0 0.3 58.34 CRE 0 2 f yellow 8 716 4 Queens 4071600 I QN98 Airport 4105 115 205 1 Manhattan 1020500 I MN09 Morningside Heights 3802 +1200003246 2 2015-07-21 2015-07-21 15:02:27 2015-07-21 2015-07-21 16:10:42 0 2 -73.78488159179688 40.6485710144043 -73.97480010986328 40.75095748901367 1 16.27 52 0 0.5 11.67 5.54 0 0.3 70.01 CSH 0 2 b yellow 8 716 4 Queens 4071600 I QN98 Airport 4105 226 92 1 Manhattan 1009200 I MN19 Turtle Bay-East Midtown 3808 +1200003287 1 2015-07-21 2015-07-21 17:46:25 2015-07-21 2015-07-21 17:52:37 0 1 -73.99249267578125 40.768516540527344 -73.97818756103516 40.783939361572266 1 1.4 7 1 0.5 0 0 0 0.3 8.8 CSH 0 P 9 yellow -121 135 1 Manhattan 1013500 E MN15 Clinton 3807 153 167 1 Manhattan 1016700 I MN12 Upper West Side 3806 +1200003297 1 2015-07-21 2015-07-21 18:15:46 2015-07-21 2015-07-21 18:46:36 0 1 -73.98818969726562 40.72279357910156 -73.97760772705078 40.75507736206055 2 2.8 19 1 0.5 4.15 0 0 0.3 24.95 CSH 0 H a yellow -95 30.01 1 Manhattan 1003001 E MN27 Chinatown 3809 225 94 1 Manhattan 1009400 E MN17 Midtown-Midtown South 3807 +1200003318 2 2015-07-21 2015-07-21 19:18:15 2015-07-21 2015-07-21 19:30:04 0 1 -73.99254608154297 40.69416046142578 -73.97461700439453 40.687034606933594 1 1.81 9.5 1 0.5 0 0 0 0.3 11.3 CRE 0 3 � yellow -127 5.02 3 Brooklyn 3000502 I BK09 Brooklyn Heights-Cobble Hill 4004 4 35 3 Brooklyn 3003500 I BK68 Fort Greene 4004 +1200003341 1 2015-07-21 2015-07-21 20:24:48 2015-07-21 2015-07-21 20:30:57 0 1 -73.99109649658203 40.723472595214844 -73.9897689819336 40.71890640258789 1 0.5 5.5 0.5 0.5 1 0 0 0.3 7.8 CSH 0 H H yellow -63 36.01 1 Manhattan 1003601 E MN27 Chinatown 3809 195 18 1 Manhattan 1001800 E MN27 Chinatown 3809 +1200003342 1 2015-07-21 2015-07-21 20:25:20 2015-07-21 2015-07-21 20:47:27 0 1 -73.96149444580078 40.78028106689453 -73.93138122558594 40.760318756103516 1 4.6 18.5 1 0.5 4.05 0 0 0.3 24.35 CSH 0 C  yellow 43 143 1 Manhattan 1014300 I MN99 park-cemetery-etc-Manhattan 3806 2 47 4 Queens 4004700 E QN70 Astoria 4101 +1200003369 1 2015-07-21 2015-07-21 21:50:35 2015-07-21 2015-07-21 21:57:43 0 1 -73.95735931396484 40.78255081176758 -73.97215270996094 40.79900360107422 1 1.8 8 0.5 0.5 0 0 0 0.3 9.3 NOC 0 6 Q yellow 74 150.02 1 Manhattan 1015002 I MN40 Upper East Side-Carnegie Hill 3805 169 187 1 Manhattan 1018700 I MN12 Upper West Side 3806 +1200003375 2 2015-07-21 2015-07-21 22:03:26 2015-07-21 2015-07-21 22:13:26 0 1 -73.99152374267578 40.726802825927734 -74.005615234375 40.74515151977539 3 2.06 9.5 0.5 0.5 2.16 0 0 0.3 12.96 CSH 0 y h yellow -66 38 1 Manhattan 1003800 I MN22 East Village 3809 12 89 1 Manhattan 1008900 I MN13 Hudson Yards-Chelsea-Flatiron-Union Square 3807 +1200003406 2 2015-07-21 2015-07-21 23:58:20 2015-07-22 2015-07-22 00:04:16 0 1 -73.95866394042969 40.78396987915039 -73.97893524169922 40.777408599853516 3 1.71 7.5 0.5 0.5 1.76 0 0 0.3 10.56 CSH 0 C B yellow 43 143 1 Manhattan 1014300 I MN99 park-cemetery-etc-Manhattan 3806 21 157 1 Manhattan 1015700 I MN14 Lincoln Square 3806 +1200003410 2 2015-07-22 2015-07-22 00:08:00 2015-07-22 2015-07-22 00:16:16 0 1 -73.9800033569336 40.755496978759766 -73.98690032958984 40.73344039916992 1 1.79 8 0.5 0.5 1.86 0 0 0.3 11.16 CSH 0 a  yellow 13 96 1 Manhattan 1009600 E MN17 Midtown-Midtown South 3807 52 48 1 Manhattan 1004800 I MN21 Gramercy 3808 +1200003411 2 2015-07-22 2015-07-22 00:09:22 2015-07-22 2015-07-22 00:32:59 0 1 -73.98543548583984 40.76343536376953 -73.89051818847656 40.75239944458008 5 5.76 20.5 0.5 0.5 0 0 0 0.3 21.8 CRE 0 H ) yellow -119 133 1 Manhattan 1013300 I MN15 Clinton 3807 114 289 4 Queens 4028900 E QN28 Jackson Heights 4102 +1200003432 1 2015-07-22 2015-07-22 02:22:52 2015-07-22 2015-07-22 02:44:14 0 1 -74.01023864746094 40.72011947631836 -73.88407897949219 40.74775314331055 4 10.1 29.5 0.5 0.5 0 0 0 0.3 30.8 CRE 0 1 ) yellow 14 39 1 Manhattan 1003900 I MN24 SoHo-TriBeCa-Civic Center-Little Italy 3810 111 283 4 Queens 4028300 E QN28 Jackson Heights 4102 +1200003467 2 2015-07-22 2015-07-22 08:20:21 2015-07-22 2015-07-22 08:37:29 0 1 -73.95368957519531 40.766876220703125 -73.97720336914062 40.75212860107422 5 2.32 12.5 0 0.5 1.5 0 0 0.3 14.8 CSH 0 @ b yellow -96 124 1 Manhattan 1012400 I MN31 Lenox Hill-Roosevelt Island 3805 226 92 1 Manhattan 1009200 I MN19 Turtle Bay-East Midtown 3808 +1200003473 1 2015-07-22 2015-07-22 08:41:40 2015-07-22 2015-07-22 08:53:38 0 1 -73.96144104003906 40.78022003173828 -73.98160552978516 40.76565933227539 1 2.3 10.5 0 0.5 2.25 0 0 0.3 13.55 CSH 0 C c yellow 43 143 1 Manhattan 1014300 I MN99 park-cemetery-etc-Manhattan 3806 4 137 1 Manhattan 1013700 I MN17 Midtown-Midtown South 3807 +1200003530 2 2015-07-22 2015-07-22 12:07:16 2015-07-22 2015-07-22 12:34:08 0 1 -73.98963928222656 40.74148178100586 -73.97283172607422 40.76258850097656 2 2.31 17 0 0.5 3.56 0 0 0.3 21.36 CSH 0 4 c yellow -123 58 1 Manhattan 1005800 I MN13 Hudson Yards-Chelsea-Flatiron-Union Square 3807 19 112.02 1 Manhattan 1011202 I MN17 Midtown-Midtown South 3807 +1200003555 2 2015-07-22 2015-07-22 10:19:32 2015-07-22 2015-07-22 10:29:12 0 1 -74.01506042480469 40.71833801269531 -74.00167846679688 40.73051834106445 1 1.5 8.5 0 0.5 0 0 0 0.3 9.3 CRE 0   yellow 52 317.03 1 Manhattan 1031703 I MN25 Battery Park City-Lower Manhattan 3810 37 65 1 Manhattan 1006500 I MN23 West Village 3810 +1200003561 2 2015-07-22 2015-07-22 14:02:39 2015-07-22 2015-07-22 14:32:01 0 1 -74.01011657714844 40.71443176269531 -73.97512817382812 40.74882125854492 2 4.61 21 0 0.5 0 0 0 0.3 21.8 CSH 0 1 3 yellow 47 21 1 Manhattan 1002100 I MN24 SoHo-TriBeCa-Civic Center-Little Italy 3810 151 88 1 Manhattan 1008800 I MN19 Turtle Bay-East Midtown 3808 +1200003568 2 2015-07-22 2015-07-22 14:32:31 2015-07-22 2015-07-22 14:56:11 0 1 -73.99137878417969 40.775535583496094 -73.99847412109375 40.734947204589844 1 3.69 16.5 0 0.5 0 0 0 0.3 17.3 CRE 0 C  yellow -51 151 1 Manhattan 1015100 E MN14 Lincoln Square 3806 134 63 1 Manhattan 1006300 I MN23 West Village 3810 +1200003605 2 2015-07-22 2015-07-22 16:51:32 2015-07-22 2015-07-22 16:51:35 0 1 0 0 -74.00302124023438 40.72706985473633 1 0 2.5 1 0.5 0 0 0 0.3 4.3 CSH 0 % yellow 0 0 0 0000000 0000 0 13 37 1 Manhattan 1003700 I MN24 SoHo-TriBeCa-Civic Center-Little Italy 3810 +1200003606 2 2015-07-22 2015-07-22 16:54:29 2015-07-22 2015-07-22 17:00:35 0 1 -73.9662094116211 40.75816345214844 -73.97374725341797 40.751163482666016 2 0.74 5.5 1 0.5 1.46 0 0 0.3 8.76 CSH 0 ) 3 yellow -101 108 1 Manhattan 1010800 I MN19 Turtle Bay-East Midtown 3808 151 88 1 Manhattan 1008800 I MN19 Turtle Bay-East Midtown 3808 +1200003614 2 2015-07-22 2015-07-22 17:32:33 2015-07-22 2015-07-22 17:45:30 0 1 -73.97016143798828 40.76519775390625 -73.9755859375 40.78935623168945 1 2.46 11 1 0.5 0 0 0 0.3 12.8 CRE 0 7 8 yellow -29 114.01 1 Manhattan 1011401 I MN40 Upper East Side-Carnegie Hill 3805 138 175 1 Manhattan 1017500 I MN12 Upper West Side 3806 +1200003626 1 2015-07-22 2015-07-22 18:16:14 2015-07-22 2015-07-22 18:27:17 0 1 -74.00762176513672 40.740787506103516 -73.98898315429688 40.730712890625 1 1.7 9 1 0.5 2.15 0 0 0.3 12.95 CSH 0 X y yellow 79 79 1 Manhattan 1007900 I MN23 West Village 3810 93 40 1 Manhattan 1004000 I MN22 East Village 3809 +1200003632 1 2015-07-22 2015-07-22 18:29:20 2015-07-22 2015-07-22 18:51:36 0 1 -73.9728775024414 40.7548713684082 -73.98975372314453 40.74165344238281 1 1.8 14.5 1 0.5 0 0 0 0.3 16.3 CRE 0 b 4 yellow -30 92 1 Manhattan 1009200 I MN19 Turtle Bay-East Midtown 3808 133 58 1 Manhattan 1005800 I MN13 Hudson Yards-Chelsea-Flatiron-Union Square 3807 +1200003641 1 2015-07-22 2015-07-22 18:51:36 2015-07-22 2015-07-22 18:56:33 0 1 -73.97515106201172 40.745933532714844 -73.98226928710938 40.736175537109375 1 0.7 5 1 0.5 1.35 0 0 0.3 8.15 CSH 0 p  yellow -96 78 1 Manhattan 1007800 I MN20 Murray Hill-Kips Bay 3808 154 64 1 Manhattan 1006400 I MN21 Gramercy 3808 +1200003677 2 2015-07-22 2015-07-22 20:38:34 2015-07-22 2015-07-22 20:43:11 0 1 -73.94550323486328 40.778656005859375 -73.95079040527344 40.78239440917969 2 0.64 5 0.5 0.5 0 0 0 0.3 6.3 CRE 0 b c yellow 1 152 1 Manhattan 1015200 I MN32 Yorkville 3805 31 154 1 Manhattan 1015400 I MN32 Yorkville 3805 +1200003703 2 2015-07-22 2015-07-22 21:25:56 2015-07-22 2015-07-22 21:33:32 0 1 -73.9718246459961 40.75403594970703 -73.9711685180664 40.76536178588867 1 0.97 6.5 0.5 0.5 1.95 0 0 0.3 9.75 CSH 0 3 7 yellow -104 90 1 Manhattan 1009000 I MN19 Turtle Bay-East Midtown 3808 227 114.01 1 Manhattan 1011401 I MN40 Upper East Side-Carnegie Hill 3805 +1200003706 1 2015-07-22 2015-07-22 21:32:59 2015-07-22 2015-07-22 21:47:43 0 1 -73.99124908447266 40.750328063964844 -73.94975280761719 40.72722244262695 1 4.1 14.5 0.5 0.5 4.25 5.54 0 0.3 25.59 CSH 0 �  yellow -125 101 1 Manhattan 1010100 I MN17 Midtown-Midtown South 3807 23 573 3 Brooklyn 3057300 I BK76 Greenpoint 4001 +1200003708 2 2015-07-22 2015-07-22 21:34:38 2015-07-22 2015-07-22 21:49:47 0 1 -73.98158264160156 40.74092483520508 -73.97721099853516 40.766212463378906 1 2.31 12 0.5 0.5 1 0 0 0.3 14.3 CSH 0 7 C yellow -11 66 1 Manhattan 1006600 I MN20 Murray Hill-Kips Bay 3808 43 143 1 Manhattan 1014300 I MN99 park-cemetery-etc-Manhattan 3806 +1200003718 2 2015-07-22 2015-07-22 22:02:26 2015-07-22 2015-07-22 22:21:15 0 1 -73.9714584350586 40.7635612487793 -73.98995208740234 40.73459243774414 5 2.62 14 0.5 0.5 0 0 0 0.3 15.3 CRE 0 c 4 yellow 19 112.02 1 Manhattan 1011202 I MN17 Midtown-Midtown South 3807 245 52 1 Manhattan 1005200 I MN13 Hudson Yards-Chelsea-Flatiron-Union Square 3807 +1200003720 1 2015-07-22 2015-07-22 22:06:47 2015-07-22 2015-07-22 22:18:12 0 1 -74.0014877319336 40.73609161376953 -73.99698638916016 40.72123718261719 1 1.4 9 0.5 0.5 1.5 0 0 0.3 11.8 CSH 0 I D yellow -94 71 1 Manhattan 1007100 I MN23 West Village 3810 196 41 1 Manhattan 1004100 E MN24 SoHo-TriBeCa-Civic Center-Little Italy 3810 +1200003724 2 2015-07-22 2015-07-22 22:22:08 2015-07-22 2015-07-22 22:28:59 0 1 -73.99690246582031 40.72256088256836 -73.98574829101562 40.74079132080078 1 1.41 7 0.5 0.5 0 0 0 0.3 8.3 CRE 0 D  yellow -67 43 1 Manhattan 1004300 I MN24 SoHo-TriBeCa-Civic Center-Little Italy 3810 11 68 1 Manhattan 1006800 I MN21 Gramercy 3808 +1200003748 1 2015-07-22 2015-07-22 23:27:27 2015-07-22 2015-07-22 23:37:43 0 1 -73.98017883300781 40.76049041748047 -73.96441650390625 40.79191970825195 1 2.6 10.5 0.5 0.5 0 0 0 0.3 11.8 CRE 0 0 C yellow 3 131 1 Manhattan 1013100 I MN17 Midtown-Midtown South 3807 43 143 1 Manhattan 1014300 I MN99 park-cemetery-etc-Manhattan 3806 +1200003759 2 2015-07-23 2015-07-23 00:16:55 2015-07-23 2015-07-23 00:19:58 0 1 -73.96614074707031 40.76200866699219 -73.95536041259766 40.76876449584961 1 0.95 5 0.5 0.5 1.3 0 0 0.3 7.6 CSH 0 A A yellow -35 110 1 Manhattan 1011000 I MN31 Lenox Hill-Roosevelt Island 3805 183 126 1 Manhattan 1012600 I MN31 Lenox Hill-Roosevelt Island 3805 +1200003779 2 2015-07-23 2015-07-23 01:39:20 2015-07-23 2015-07-23 01:47:29 0 1 -73.98261260986328 40.76749038696289 -73.96272277832031 40.772621154785156 5 1.67 8.5 0.5 0.5 0 0 0 0.3 9.8 CRE 0 H 7 yellow -54 139 1 Manhattan 1013900 I MN15 Clinton 3807 23 130 1 Manhattan 1013000 I MN40 Upper East Side-Carnegie Hill 3805 +1200003786 1 2015-07-23 2015-07-23 02:56:02 2015-07-23 2015-07-23 03:11:24 0 1 -73.99235534667969 40.74070358276367 -73.93535614013672 40.7960205078125 1 6.1 19 0.5 0.5 3 0 0 0.3 23.3 CSH 0 4 t yellow -114 54 1 Manhattan 1005400 I MN13 Hudson Yards-Chelsea-Flatiron-Union Square 3807 17 178 1 Manhattan 1017800 E MN34 East Harlem North 3804 +1200003793 2 2015-07-23 2015-07-23 03:30:13 2015-07-23 2015-07-23 03:37:52 0 1 -73.98480224609375 40.7366943359375 -74.0037841796875 40.73723220825195 5 1.37 7.5 0.5 0.5 2.64 0 0 0.3 11.44 CSH 0  I yellow -12 50 1 Manhattan 1005000 I MN21 Gramercy 3808 78 77 1 Manhattan 1007700 I MN23 West Village 3810 +1200003804 2 2015-07-23 2015-07-23 06:20:36 2015-07-23 2015-07-23 06:24:48 0 1 -73.99400329589844 40.75135040283203 -74.00511169433594 40.74177169799805 3 1.01 5.5 0 0.5 0 0 0 0.3 6.3 CRE 0 � h yellow -125 101 1 Manhattan 1010100 I MN17 Midtown-Midtown South 3807 75 83 1 Manhattan 1008300 E MN13 Hudson Yards-Chelsea-Flatiron-Union Square 3807 +1200003822 2 2015-07-23 2015-07-23 07:39:16 2015-07-23 2015-07-23 07:50:19 0 1 -73.97563171386719 40.78190994262695 -73.97418212890625 40.760318756103516 5 2.08 9.5 0 0.5 2.06 0 0 0.3 12.36 CSH 0 9 a yellow 22 165 1 Manhattan 1016500 I MN12 Upper West Side 3806 16 102 1 Manhattan 1010200 I MN17 Midtown-Midtown South 3807 +1200003866 2 2015-07-23 2015-07-23 10:04:24 2015-07-23 2015-07-23 10:15:58 0 1 -73.98734283447266 40.75838088989258 -73.99677276611328 40.752811431884766 1 0.84 8.5 0 0.5 1.86 0 0 0.3 11.16 CSH 0 0 h yellow -123 119 1 Manhattan 1011900 E MN17 Midtown-Midtown South 3807 103 103 1 Manhattan 1010300 I MN13 Hudson Yards-Chelsea-Flatiron-Union Square 3807 +1200003923 2 2015-07-23 2015-07-23 13:29:00 2015-07-23 2015-07-23 13:55:42 0 1 -73.98343658447266 40.755279541015625 -73.94219970703125 40.7488899230957 1 3.21 19 0 0.5 4.95 0 0 0.3 24.75 CSH 0 a E yellow 13 96 1 Manhattan 1009600 E MN17 Midtown-Midtown South 3807 137 19 4 Queens 4001900 I QN31 Hunters Point-Sunnyside-West Maspeth 4109 +1200003950 2 2015-07-23 2015-07-23 15:29:44 2015-07-23 2015-07-23 15:37:58 0 1 -73.99921417236328 40.73341751098633 -73.99887084960938 40.724857330322266 1 0.92 7 0 0.5 1.56 0 0 0.3 9.36 CSH 0   yellow -122 63 1 Manhattan 1006300 I MN23 West Village 3810 35 49 1 Manhattan 1004900 I MN24 SoHo-TriBeCa-Civic Center-Little Italy 3810 +1200003954 2 2015-07-23 2015-07-23 15:41:59 2015-07-23 2015-07-23 16:06:59 0 1 -73.98619842529297 40.77743911743164 -73.93900299072266 40.80482864379883 1 4.54 18.5 0 0.5 4.82 0 0 0.3 24.12 CSH 0 C t yellow -50 155 1 Manhattan 1015500 I MN14 Lincoln Square 3806 233 196 1 Manhattan 1019600 E MN34 East Harlem North 3804 +1200003981 2 2015-07-23 2015-07-23 17:42:05 2015-07-23 2015-07-23 18:00:44 0 1 -73.98290252685547 40.75143051147461 -73.97730255126953 40.76380157470703 1 1.24 12 1 0.5 0 0 0 0.3 13.8 CRE 0 d c yellow -47 84 1 Manhattan 1008400 I MN17 Midtown-Midtown South 3807 18 112.01 1 Manhattan 1011201 I MN17 Midtown-Midtown South 3807 +1200003984 2 2015-07-23 2015-07-23 17:52:08 2015-07-23 2015-07-23 18:05:18 0 1 -73.97830200195312 40.7624397277832 -73.97518920898438 40.78744888305664 1 2 10.5 1 0.5 3.69 0 0 0.3 15.99 CSH 0 a 9 yellow 17 104 1 Manhattan 1010400 I MN17 Midtown-Midtown South 3807 166 171 1 Manhattan 1017100 I MN12 Upper West Side 3806 +1200003988 2 2015-07-23 2015-07-23 18:05:57 2015-07-23 2015-07-23 18:22:57 0 1 -74.01151275634766 40.70991134643555 -74.00670623779297 40.730140686035156 5 1.6099999999999999 11.5 1 0.5 0 0 0 0.3 13.3 CRE 0 a I yellow 53 13 1 Manhattan 1001300 I MN25 Battery Park City-Lower Manhattan 3810 36 67 1 Manhattan 1006700 I MN23 West Village 3810 +1200003991 1 2015-07-23 2015-07-23 18:15:32 2015-07-23 2015-07-23 18:15:36 0 1 -73.9810562133789 40.68294143676758 -73.98106384277344 40.68293762207031 1 0 2.5 1 0.5 0 0 0 0.3 4.3 NOC 0 % % yellow 82 127 3 Brooklyn 3012700 E BK38 DUMBO-Vinegar Hill-Downtown Brooklyn-Boerum Hill 4004 82 127 3 Brooklyn 3012700 E BK38 DUMBO-Vinegar Hill-Downtown Brooklyn-Boerum Hill 4004 +1200003998 2 2015-07-23 2015-07-23 18:47:10 2015-07-23 2015-07-23 18:57:17 0 1 -73.99275970458984 40.744232177734375 -74.00787353515625 40.751548767089844 1 1.06 8 1 0.5 0 0 0 0.3 9.8 CRE 0 � F yellow -124 91 1 Manhattan 1009100 I MN13 Hudson Yards-Chelsea-Flatiron-Union Square 3807 50 99 1 Manhattan 1009900 I MN13 Hudson Yards-Chelsea-Flatiron-Union Square 3807 +1200004026 2 2015-07-23 2015-07-23 19:19:28 2015-07-23 2015-07-23 19:39:11 0 1 -73.87580871582031 40.7735481262207 -73.96195983886719 40.76302719116211 1 9.3 27 1 0.5 6.87 5.54 0 0.3 41.21 CSH 0 8 A yellow 48 331 4 Queens 4033100 I QN98 Airport 4102 221 110 1 Manhattan 1011000 I MN31 Lenox Hill-Roosevelt Island 3805 +1200004035 1 2015-07-23 2015-07-23 19:35:40 2015-07-23 2015-07-23 19:41:05 0 1 -73.96012878417969 40.77054214477539 -73.96876525878906 40.76457214355469 1 0.9 6 1 0.5 1 0 0 0.3 8.8 CSH 0 7 7 yellow 18 128 1 Manhattan 1012800 I MN40 Upper East Side-Carnegie Hill 3805 227 114.01 1 Manhattan 1011401 I MN40 Upper East Side-Carnegie Hill 3805 +1200004074 2 2015-07-23 2015-07-23 21:27:41 2015-07-23 2015-07-23 21:48:59 0 1 -73.95592498779297 40.764015197753906 -73.99085998535156 40.75089645385742 2 2.87 16 0.5 0.5 3.46 0 0 0.3 20.76 CSH 0 @ � yellow -34 116 1 Manhattan 1011600 I MN31 Lenox Hill-Roosevelt Island 3805 131 101 1 Manhattan 1010100 I MN17 Midtown-Midtown South 3807 +1200004123 2 2015-07-23 2015-07-23 23:21:02 2015-07-23 2015-07-23 23:32:28 0 1 -74.00236511230469 40.73349380493164 -73.999267578125 40.7443962097168 1 1.44 9 0.5 0.5 2.06 0 0 0.3 12.36 CSH 0 I h yellow -94 71 1 Manhattan 1007100 I MN23 West Village 3810 12 89 1 Manhattan 1008900 I MN13 Hudson Yards-Chelsea-Flatiron-Union Square 3807 +1200004126 2 2015-07-23 2015-07-23 19:56:35 2015-07-23 2015-07-23 20:00:23 0 1 -73.98133850097656 40.73690414428711 -73.98834991455078 40.72755813598633 2 0.77 4.5 1 0.5 0 0 0 0.3 6.3 CRE 0  y yellow -102 64 1 Manhattan 1006400 I MN21 Gramercy 3808 190 38 1 Manhattan 1003800 I MN22 East Village 3809 +1200004130 1 2015-07-23 2015-07-23 23:35:51 2015-07-23 2015-07-23 23:46:18 0 1 -73.95025634765625 40.779815673828125 -73.97367858886719 40.75056457519531 1 2.4 10 0.5 0.5 0 0 0 0.3 11.3 CRE 0 c 3 yellow 31 154 1 Manhattan 1015400 I MN32 Yorkville 3805 151 88 1 Manhattan 1008800 I MN19 Turtle Bay-East Midtown 3808 +1200004163 2 2015-07-24 2015-07-24 00:51:05 2015-07-24 2015-07-24 00:59:30 0 1 -74.00415802001953 40.75182342529297 -73.98495483398438 40.7617073059082 1 1.5699999999999998 6.5 0.5 0.5 0 0 0 0.3 7.8 CRE 0 F 0 yellow 50 99 1 Manhattan 1009900 I MN13 Hudson Yards-Chelsea-Flatiron-Union Square 3807 134 125 1 Manhattan 1012500 I MN17 Midtown-Midtown South 3807 +1200004183 1 2015-07-24 2015-07-24 02:18:27 2015-07-24 2015-07-24 02:24:54 0 1 -73.98854064941406 40.72894287109375 -74.00288391113281 40.715572357177734 1 1.5 7 0.5 0.5 0 0 0 0.3 8.3 CSH 0 y 1 yellow -66 38 1 Manhattan 1003800 I MN22 East Village 3809 154 31 1 Manhattan 1003100 I MN24 SoHo-TriBeCa-Civic Center-Little Italy 3810 +1200004186 1 2015-07-24 2015-07-24 03:06:37 2015-07-24 2015-07-24 03:26:31 0 1 -74.00397491455078 40.74170684814453 -73.94173431396484 40.83821487426758 1 8.5 25.5 0.5 0.5 5.35 0 0 0.3 32.15 CSH 0 h D yellow 75 83 1 Manhattan 1008300 E MN13 Hudson Yards-Chelsea-Flatiron-Union Square 3807 142 245 1 Manhattan 1024500 E MN36 Washington Heights South 3801 +1200004210 1 2015-07-24 2015-07-24 07:34:09 2015-07-24 2015-07-24 07:38:25 0 1 -73.98292541503906 40.7420768737793 -73.99256896972656 40.74308395385742 1 0.6 5 0 0.5 1.15 0 0 0.3 6.95 CSH 0  4 yellow 11 68 1 Manhattan 1006800 I MN21 Gramercy 3808 133 58 1 Manhattan 1005800 I MN13 Hudson Yards-Chelsea-Flatiron-Union Square 3807 +1200004212 2 2015-07-24 2015-07-24 07:46:29 2015-07-24 2015-07-24 07:52:20 0 1 -73.99211883544922 40.73796463012695 -74.00130462646484 40.726356506347656 1 1.11 6 0 0.5 1.36 0 0 0.3 8.16 CSH 0 4  yellow -11 52 1 Manhattan 1005200 I MN13 Hudson Yards-Chelsea-Flatiron-Union Square 3807 35 49 1 Manhattan 1004900 I MN24 SoHo-TriBeCa-Civic Center-Little Italy 3810 +1200004221 1 2015-07-24 2015-07-24 08:16:57 2015-07-24 2015-07-24 08:39:11 0 1 -73.95540618896484 40.78306579589844 -74.00123596191406 40.7466926574707 1 4.2 18 0 0.5 3.75 0 0 0.3 22.55 CSH 0 6 h yellow 74 150.02 1 Manhattan 1015002 I MN40 Upper East Side-Carnegie Hill 3805 113 93 1 Manhattan 1009300 I MN13 Hudson Yards-Chelsea-Flatiron-Union Square 3807 +1200004268 1 2015-07-24 2015-07-24 09:58:52 2015-07-24 2015-07-24 10:17:00 0 1 -73.97689819335938 40.76100158691406 -73.99317169189453 40.7432746887207 1 1.8 12 0 0.5 0 0 0 0.3 12.8 CRE 0 a � yellow 17 104 1 Manhattan 1010400 I MN17 Midtown-Midtown South 3807 132 91 1 Manhattan 1009100 I MN13 Hudson Yards-Chelsea-Flatiron-Union Square 3807 +1200004286 2 2015-07-24 2015-07-24 11:04:25 2015-07-24 2015-07-24 11:11:17 0 1 -73.95945739746094 40.78282165527344 -73.94830322265625 40.782493591308594 2 0.83 5.5 0 0.5 0 0 0 0.3 6.3 CSH 0 6 c yellow 74 150.02 1 Manhattan 1015002 I MN40 Upper East Side-Carnegie Hill 3805 31 154 1 Manhattan 1015400 I MN32 Yorkville 3805 +1200004292 2 2015-07-24 2015-07-24 11:11:37 2015-07-24 2015-07-24 11:26:50 0 1 -73.98443603515625 40.74537658691406 -73.97258758544922 40.76140213012695 1 1.33 10.5 0 0.5 2.82 0 0 0.3 14.12 CSH 0 d a yellow -125 74 1 Manhattan 1007400 I MN17 Midtown-Midtown South 3807 16 102 1 Manhattan 1010200 I MN17 Midtown-Midtown South 3807 +1200004310 2 2015-07-24 2015-07-24 12:36:42 2015-07-24 2015-07-24 12:46:47 0 1 -73.9936752319336 40.759178161621094 -73.98106384277344 40.776153564453125 5 1.43 8.5 0 0.5 0 0 0 0.3 9.3 CRE 0 H B yellow -117 121 1 Manhattan 1012100 E MN15 Clinton 3807 30 153 1 Manhattan 1015300 I MN14 Lincoln Square 3806 +1200004316 1 2015-07-24 2015-07-24 13:06:21 2015-07-24 2015-07-24 13:34:41 0 1 -73.99018096923828 40.751853942871094 -73.99913787841797 40.735755920410156 1 3.8 18.5 0 0.5 0 0 0 0.3 19.3 CRE 0  I yellow -124 109 1 Manhattan 1010900 I MN17 Midtown-Midtown South 3807 162 71 1 Manhattan 1007100 I MN23 West Village 3810 +1200004346 1 2015-07-24 2015-07-24 15:08:02 2015-07-24 2015-07-24 15:37:04 0 1 -73.7891845703125 40.64180374145508 -73.83966827392578 40.71651077270508 1 7.6 26 0 0.5 5.36 0 0 0.3 32.16 CSH 0 2 � yellow 8 716 4 Queens 4071600 I QN98 Airport 4105 166 769.01 4 Queens 4076901 I QN17 Forest Hills 4108 +1200004353 1 2015-07-24 2015-07-24 15:39:43 2015-07-24 2015-07-24 16:02:48 0 1 -73.99127197265625 40.73642349243164 -74.01726531982422 40.71104431152344 1 3.4 16.5 0 0.5 3.45 0 0 0.3 20.75 CSH 0 4  yellow -11 52 1 Manhattan 1005200 I MN13 Hudson Yards-Chelsea-Flatiron-Union Square 3807 54 317.04 1 Manhattan 1031704 I MN25 Battery Park City-Lower Manhattan 3810 +1200004357 2 2015-07-24 2015-07-24 15:50:34 2015-07-24 2015-07-24 16:05:00 0 1 -73.98625946044922 40.74345779418945 -73.97221374511719 40.756839752197266 5 1.51 10.5 0 0.5 1 0 0 0.3 12.3 CSH 0 4 b yellow -90 56 1 Manhattan 1005600 I MN13 Hudson Yards-Chelsea-Flatiron-Union Square 3807 15 100 1 Manhattan 1010000 I MN19 Turtle Bay-East Midtown 3808 +1200004372 2 2015-07-24 2015-07-24 17:28:49 2015-07-24 2015-07-24 17:36:06 0 1 -73.95513916015625 40.78352355957031 -73.9548568725586 40.78325653076172 5 1.3900000000000001 7 1 0.5 2.2 0 0 0.3 11 CSH 0 6 6 yellow 34 160.01 1 Manhattan 1016001 I MN40 Upper East Side-Carnegie Hill 3805 29 148.02 1 Manhattan 1014802 I MN40 Upper East Side-Carnegie Hill 3805 +1200004388 1 2015-07-24 2015-07-24 18:17:50 2015-07-24 2015-07-24 18:31:56 0 1 -73.97760009765625 40.773826599121094 -73.94709777832031 40.81417465209961 1 3.3 13 0.5 0.5 2 0 0 0.3 16.3 CSH 0 C B yellow 43 143 1 Manhattan 1014300 I MN99 park-cemetery-etc-Manhattan 3806 155 226 1 Manhattan 1022600 E MN03 Central Harlem North-Polo Grounds 3803 +1200004409 2 2015-07-24 2015-07-24 19:14:30 2015-07-24 2015-07-24 19:26:10 0 1 -73.97039031982422 40.76205062866211 -73.98239135742188 40.74821853637695 1 1.33 8.5 1 0.5 0 0 0 0.3 10.3 CRE 0 b d yellow 20 112.03 1 Manhattan 1011203 I MN19 Turtle Bay-East Midtown 3808 131 74 1 Manhattan 1007400 I MN17 Midtown-Midtown South 3807 +1200004441 2 2015-07-24 2015-07-24 20:25:12 2015-07-24 2015-07-24 20:38:53 0 1 -74.00778198242188 40.7517204284668 -73.96247863769531 40.76667022705078 4 3.63 13 0.5 0.5 2.86 0 0 0.3 17.16 CSH 0 F A yellow 50 99 1 Manhattan 1009900 I MN13 Hudson Yards-Chelsea-Flatiron-Union Square 3807 182 118 1 Manhattan 1011800 I MN31 Lenox Hill-Roosevelt Island 3805 +1200004453 2 2015-07-24 2015-07-24 20:45:42 2015-07-24 2015-07-24 20:53:12 0 1 -73.99583435058594 40.739471435546875 -74.004150390625 40.72978210449219 1 1.02 6.5 0.5 0.5 1.95 0 0 0.3 9.75 CSH 0 � I yellow -115 81 1 Manhattan 1008100 I MN13 Hudson Yards-Chelsea-Flatiron-Union Square 3807 36 67 1 Manhattan 1006700 I MN23 West Village 3810 +1200004465 2 2015-07-24 2015-07-24 21:15:13 2015-07-24 2015-07-24 21:20:16 0 1 -73.97238159179688 40.74173355102539 -73.96009826660156 40.7620964050293 5 1.77 7.5 0.5 0.5 1.76 0 0 0.3 10.56 CSH 0 7 @ yellow 85 62 1 Manhattan 1006200 I MN20 Murray Hill-Kips Bay 3808 220 106.02 1 Manhattan 1010602 I MN31 Lenox Hill-Roosevelt Island 3805 +1200004469 2 2015-07-24 2015-07-24 21:26:20 2015-07-24 2015-07-24 21:45:27 0 1 -73.94972229003906 40.79639434814453 -73.9546890258789 40.8271484375 3 3.3 15 0.5 0.5 5 0 0 0.3 21.3 CSH 0 C  yellow 43 143 1 Manhattan 1014300 I MN99 park-cemetery-etc-Manhattan 3806 42 229 1 Manhattan 1022900 E MN04 Hamilton Heights 3802 +1200004479 2 2015-07-24 2015-07-24 22:02:22 2015-07-24 2015-07-24 22:11:53 0 1 -74.00267028808594 40.73368835449219 -73.9869155883789 40.7508544921875 3 1.65 8 0.5 0.5 0 0 0 0.3 9.3 CRE 0 I d yellow -93 73 1 Manhattan 1007300 I MN23 West Village 3810 209 84 1 Manhattan 1008400 I MN17 Midtown-Midtown South 3807 +1200004497 1 2015-07-24 2015-07-24 23:01:56 2015-07-24 2015-07-24 23:04:44 0 1 -73.95661163330078 40.766963958740234 -73.94612884521484 40.77885055541992 1 1 5 0.5 0.5 1.25 0 0 0.3 7.55 CSH 0 A b yellow -73 126 1 Manhattan 1012600 I MN31 Lenox Hill-Roosevelt Island 3805 1 152 1 Manhattan 1015200 I MN32 Yorkville 3805 +1200004513 2 2015-07-24 2015-07-24 23:45:46 2015-07-24 2015-07-24 23:51:38 0 1 -73.98834991455078 40.76151657104492 -73.99449157714844 40.76335144042969 5 0.42 5.5 0.5 0.5 1 0 0 0.3 7.8 CSH 0 H P yellow -118 127 1 Manhattan 1012700 I MN15 Clinton 3807 140 129 1 Manhattan 1012900 I MN15 Clinton 3807 +1200004520 2 2015-07-25 2015-07-25 00:00:48 2015-07-25 2015-07-25 00:06:19 0 1 -73.99131774902344 40.72701644897461 -73.97796630859375 40.72136688232422 5 0.8 5.5 0.5 0.5 1.36 0 0 0.3 8.16 CSH 0 y  yellow -66 38 1 Manhattan 1003800 I MN22 East Village 3809 147 26.01 1 Manhattan 1002601 E MN28 Lower East Side 3809 +1200004528 2 2015-07-25 2015-07-25 00:17:37 2015-07-25 2015-07-25 00:22:29 0 1 -73.96495056152344 40.759647369384766 -73.97218322753906 40.75696563720703 6 0.58 4.5 0.5 0.5 0 0 0 0.3 5.8 CRE 0 ) b yellow -101 108 1 Manhattan 1010800 I MN19 Turtle Bay-East Midtown 3808 15 100 1 Manhattan 1010000 I MN19 Turtle Bay-East Midtown 3808 +1200004612 2 2015-07-25 2015-07-25 08:26:16 2015-07-25 2015-07-25 08:29:04 0 1 -73.93428039550781 40.76261901855469 -73.94242095947266 40.754119873046875 1 0.72 4.5 0 0.5 0 0 0 0.3 5.3 CRE 0 � � yellow -117 43 4 Queens 4004300 E QN68 Queensbridge-Ravenswood-Long Island City 4101 236 25 4 Queens 4002500 E QN68 Queensbridge-Ravenswood-Long Island City 4101 +1200004624 2 2015-07-25 2015-07-25 09:37:12 2015-07-25 2015-07-25 09:43:20 0 1 -73.97029876708984 40.76820755004883 -73.97259521484375 40.756675720214844 3 1.05 6.5 0 0.5 0 0 0 0.3 7.3 CRE 0 C b yellow 43 143 1 Manhattan 1014300 I MN99 park-cemetery-etc-Manhattan 3806 15 100 1 Manhattan 1010000 I MN19 Turtle Bay-East Midtown 3808 +1200004691 1 2015-07-25 2015-07-25 14:04:37 2015-07-25 2015-07-25 14:09:48 0 1 -73.95365142822266 40.767208099365234 -73.96663665771484 40.76969909667969 1 0.8 5.5 0 0.5 0 0 0 0.3 6.3 CRE 0 @ 7 yellow -96 124 1 Manhattan 1012400 I MN31 Lenox Hill-Roosevelt Island 3805 228 122 1 Manhattan 1012200 I MN40 Upper East Side-Carnegie Hill 3805 +1200004721 2 2015-07-25 2015-07-25 15:29:17 2015-07-25 2015-07-25 15:38:23 0 1 -73.97673034667969 40.74357223510742 -73.98824310302734 40.72018051147461 1 2 9 0 0.5 1.96 0 0 0.3 11.76 CSH 0 7 H yellow -10 70 1 Manhattan 1007000 I MN20 Murray Hill-Kips Bay 3808 161 30.01 1 Manhattan 1003001 E MN27 Chinatown 3809 +1200004726 2 2015-07-25 2015-07-25 15:53:51 2015-07-25 2015-07-25 16:08:47 0 1 -73.97344207763672 40.74383544921875 -73.97173309326172 40.764427185058594 1 1.8599999999999999 11 0 0.5 0 0 0 0.3 11.8 CRE 0 3 7 yellow -29 86.01 1 Manhattan 1008601 I MN19 Turtle Bay-East Midtown 3808 227 114.01 1 Manhattan 1011401 I MN40 Upper East Side-Carnegie Hill 3805 +1200004740 2 2015-07-25 2015-07-25 17:05:43 2015-07-25 2015-07-25 17:25:19 0 1 -73.99030303955078 40.740169525146484 -73.98174285888672 40.76203918457031 1 2.39 13.5 0 0.5 2.86 0 0 0.3 17.16 CSH 0 4 0 yellow -11 52 1 Manhattan 1005200 I MN13 Hudson Yards-Chelsea-Flatiron-Union Square 3807 3 131 1 Manhattan 1013100 I MN17 Midtown-Midtown South 3807 +1200004744 1 2015-07-25 2015-07-25 17:10:52 2015-07-25 2015-07-25 17:17:56 0 1 -73.98345947265625 40.78129577636719 -73.96717834472656 40.793907165527344 1 1.7 7.5 0 0.5 1.65 0 0 0.3 9.95 CSH 0 9 Q yellow -101 163 1 Manhattan 1016300 I MN12 Upper West Side 3806 25 185 1 Manhattan 1018500 I MN12 Upper West Side 3806 +1200004758 1 2015-07-25 2015-07-25 17:37:50 2015-07-25 2015-07-25 17:45:46 0 1 -73.98460388183594 40.7493896484375 -73.97438049316406 40.756961822509766 4 1.4 7.5 0 0.5 0 0 0 0.3 8.3 CRE 0 d a yellow -47 84 1 Manhattan 1008400 I MN17 Midtown-Midtown South 3807 16 102 1 Manhattan 1010200 I MN17 Midtown-Midtown South 3807 +1200004762 2 2015-07-25 2015-07-25 17:50:29 2015-07-25 2015-07-25 18:08:47 0 1 -74.0123291015625 40.7198486328125 -73.99224090576172 40.77057647705078 1 3.83 16.5 0 0.5 3 0 0 0.3 20.3 CSH 0 1 P yellow 14 39 1 Manhattan 1003900 I MN24 SoHo-TriBeCa-Civic Center-Little Italy 3810 135 135 1 Manhattan 1013500 E MN15 Clinton 3807 +1200004804 2 2015-07-25 2015-07-25 19:48:32 2015-07-25 2015-07-25 20:14:39 0 1 -74.00415802001953 40.72222900390625 -73.94828033447266 40.772300720214844 5 5.47 21.5 0 0.5 0 0 0 0.3 22.3 CRE 0  b yellow 16 47 1 Manhattan 1004700 I MN24 SoHo-TriBeCa-Civic Center-Little Italy 3810 255 136 1 Manhattan 1013600 I MN32 Yorkville 3805 +1200004813 2 2015-07-25 2015-07-25 20:21:08 2015-07-25 2015-07-25 20:33:37 0 1 -73.98052978515625 40.74311828613281 -73.9477310180664 40.78334045410156 1 3.7 13 0.5 0.5 0 0 0 0.3 14.3 CSH 0 p c yellow -126 72 1 Manhattan 1007200 I MN20 Murray Hill-Kips Bay 3808 32 156.01 1 Manhattan 1015601 I MN32 Yorkville 3805 +1200004816 1 2015-07-25 2015-07-25 20:50:40 2015-07-25 2015-07-25 21:03:14 0 1 -73.98804473876953 40.72001266479492 -74.00265502929688 40.7235221862793 1 1.2 9 0.5 0.5 2.05 0 0 0.3 12.35 CSH 0 H  yellow -95 30.01 1 Manhattan 1003001 E MN27 Chinatown 3809 16 47 1 Manhattan 1004700 I MN24 SoHo-TriBeCa-Civic Center-Little Italy 3810 +1200004867 2 2015-07-25 2015-07-25 23:21:01 2015-07-25 2015-07-25 23:42:31 0 1 -74.01311492919922 40.716251373291016 -74.01315307617188 40.67745590209961 3 4.41 17.5 0.5 0.5 1 0 0 0.3 19.8 CSH 0 1 � yellow 47 21 1 Manhattan 1002100 I MN24 SoHo-TriBeCa-Civic Center-Little Italy 3810 108 53 3 Brooklyn 3005300 E BK33 Carroll Gardens-Columbia Street-Red Hook 4005 +1200004869 2 2015-07-25 2015-07-25 23:26:55 2015-07-25 2015-07-25 23:30:20 0 1 -73.98760986328125 40.75522994995117 -73.98861694335938 40.74623107910156 1 0.8 4.5 0.5 0.5 1 0 0 0.3 6.8 CSH 0  d yellow 21 113 1 Manhattan 1011300 I MN17 Midtown-Midtown South 3807 208 76 1 Manhattan 1007600 I MN17 Midtown-Midtown South 3807 +1200004936 1 2015-07-26 2015-07-26 04:39:10 2015-07-26 2015-07-26 04:44:16 0 1 -73.97681427001953 40.751956939697266 -73.98353576660156 40.75679397583008 2 0.9 6 0.5 0.5 1.46 0 0 0.3 8.76 CSH 0 b 0 yellow -30 92 1 Manhattan 1009200 I MN19 Turtle Bay-East Midtown 3808 133 119 1 Manhattan 1011900 E MN17 Midtown-Midtown South 3807 +1200004956 1 2015-07-26 2015-07-26 00:56:21 2015-07-26 2015-07-26 01:08:42 0 1 -73.99081420898438 40.72449493408203 -73.98673248291016 40.75135040283203 1 2.5 11 0.5 0.5 2.45 0 0 0.3 14.75 CSH 0 y  yellow -62 36.02 1 Manhattan 1003602 I MN22 East Village 3809 132 109 1 Manhattan 1010900 I MN17 Midtown-Midtown South 3807 +1200004961 1 2015-07-26 2015-07-26 09:08:29 2015-07-26 2015-07-26 09:15:12 0 1 -73.97577667236328 40.749088287353516 -73.98339080810547 40.76041793823242 1 1.2 6.5 0 0.5 0 0 0 0.3 7.3 CRE 0 p 0 yellow -123 80 1 Manhattan 1008000 I MN20 Murray Hill-Kips Bay 3808 134 125 1 Manhattan 1012500 I MN17 Midtown-Midtown South 3807 +1200004986 1 2015-07-26 2015-07-26 11:21:40 2015-07-26 2015-07-26 11:26:24 0 1 -73.96859741210938 40.7588005065918 -73.95529174804688 40.778438568115234 1 1.6 6.5 0 0.5 1.45 0 0 0.3 8.75 CSH 0 b 6 yellow 15 100 1 Manhattan 1010000 I MN19 Turtle Bay-East Midtown 3808 28 148.01 1 Manhattan 1014801 I MN40 Upper East Side-Carnegie Hill 3805 +1200005049 2 2015-07-26 2015-07-26 14:33:45 2015-07-26 2015-07-26 14:47:10 0 1 -73.99568176269531 40.730751037597656 -73.9919204711914 40.75654983520508 3 2.46 11 0 0.5 0 0 0 0.3 11.8 CRE 0  H yellow 37 65 1 Manhattan 1006500 I MN23 West Village 3810 163 115 1 Manhattan 1011500 I MN15 Clinton 3807 +1200005056 1 2015-07-26 2015-07-26 15:20:12 2015-07-26 2015-07-26 15:27:05 0 1 -73.97978210449219 40.74383544921875 -73.9903793334961 40.7291259765625 3 1.3 7 0 0.5 1 0 0 0.3 8.8 CSH 0 p y yellow -126 72 1 Manhattan 1007200 I MN20 Murray Hill-Kips Bay 3808 200 42 1 Manhattan 1004200 I MN22 East Village 3809 +1200005093 1 2015-07-26 2015-07-26 17:46:10 2015-07-26 2015-07-26 17:56:57 0 1 -73.9849853515625 40.74721145629883 -74.00178527832031 40.75584411621094 2 1.2 8 0 0.5 2.2 0 0 0.3 11 CSH 0 d F yellow -125 74 1 Manhattan 1007400 I MN17 Midtown-Midtown South 3807 50 99 1 Manhattan 1009900 I MN13 Hudson Yards-Chelsea-Flatiron-Union Square 3807 +1200005096 1 2015-07-26 2015-07-26 18:09:02 2015-07-26 2015-07-26 18:27:29 0 1 -73.97338104248047 40.763553619384766 -73.99188995361328 40.72616958618164 1 3.1 13.5 0 0.5 0 0 0 0.3 14.3 CRE 0 c  yellow 19 112.02 1 Manhattan 1011202 I MN17 Midtown-Midtown South 3807 191 55.02 1 Manhattan 1005502 I MN23 West Village 3810 +1200005115 2 2015-07-26 2015-07-26 15:04:01 2015-07-26 2015-07-26 15:15:03 0 1 -73.95926666259766 40.77436828613281 -73.98670959472656 40.74863815307617 6 2.76 11 0 0.5 2.36 0 0 0.3 14.16 CSH 0 6 d yellow 24 140 1 Manhattan 1014000 I MN40 Upper East Side-Carnegie Hill 3805 208 76 1 Manhattan 1007600 I MN17 Midtown-Midtown South 3807 +1200005116 2 2015-07-26 2015-07-26 15:12:53 2015-07-26 2015-07-26 15:32:18 0 1 -73.78824615478516 40.64162826538086 -73.75337219238281 40.673038482666016 1 6.05 19.5 0 0.5 0 0 0 0.3 20.3 CRE 0 2 9 yellow 8 716 4 Queens 4071600 I QN98 Airport 4105 72 358 4 Queens 4035800 I QN66 Laurelton 4105 +1200005117 2 2015-07-26 2015-07-26 19:27:04 2015-07-26 2015-07-26 19:30:02 0 1 -73.99776458740234 40.736202239990234 -73.9972915649414 40.74165725708008 2 0.49 4 0 0.5 0 0 0 0.3 4.8 CRE 0 I � yellow -94 71 1 Manhattan 1007100 I MN23 West Village 3810 143 87 1 Manhattan 1008700 I MN13 Hudson Yards-Chelsea-Flatiron-Union Square 3807 +1200005121 1 2015-07-26 2015-07-26 19:28:20 2015-07-26 2015-07-26 19:48:59 0 1 -74.00373077392578 40.72998809814453 -73.96360778808594 40.70951461791992 1 3.2 16 0 0.5 3.35 0 0 0.3 20.15 CSH 0 I V yellow 36 67 1 Manhattan 1006700 I MN23 West Village 3810 68 549 3 Brooklyn 3054900 E BK73 North Side-South Side 4001 +1200005225 2 2015-07-27 2015-07-27 06:39:41 2015-07-27 2015-07-27 06:46:12 0 1 -73.95479583740234 40.78349685668945 -73.97821044921875 40.756103515625 1 2.5300000000000002 9 0 0.5 0 0 0 0.3 9.8 CRE 0 6 a yellow 34 160.01 1 Manhattan 1016001 I MN40 Upper East Side-Carnegie Hill 3805 225 94 1 Manhattan 1009400 E MN17 Midtown-Midtown South 3807 +1200005252 2 2015-07-27 2015-07-27 08:27:06 2015-07-27 2015-07-27 08:56:59 0 1 -74.00621032714844 40.734066009521484 -73.97250366210938 40.76411437988281 1 3.29 20 0 0.5 2 0 0 0.3 22.8 CSH 0 I 7 yellow -93 73 1 Manhattan 1007300 I MN23 West Village 3810 227 114.01 1 Manhattan 1011401 I MN40 Upper East Side-Carnegie Hill 3805 +1200005253 1 2015-07-27 2015-07-27 08:36:30 2015-07-27 2015-07-27 08:43:56 0 1 -73.9828872680664 40.762699127197266 -73.99208068847656 40.752220153808594 1 0.9 6.5 0 0.5 0 0 0 0.3 7.3 CRE 0 0  yellow 3 131 1 Manhattan 1013100 I MN17 Midtown-Midtown South 3807 132 109 1 Manhattan 1010900 I MN17 Midtown-Midtown South 3807 +1200005257 2 2015-07-27 2015-07-27 08:49:30 2015-07-27 2015-07-27 09:09:31 0 1 -73.97981262207031 40.73960494995117 -73.98223114013672 40.75162887573242 5 1.29 13 0 0.5 2.76 0 0 0.3 16.56 CSH 0 7 p yellow -11 66 1 Manhattan 1006600 I MN20 Murray Hill-Kips Bay 3808 93 82 1 Manhattan 1008200 I MN20 Murray Hill-Kips Bay 3808 +1200005307 1 2015-07-27 2015-07-27 11:30:50 2015-07-27 2015-07-27 11:57:32 0 1 -73.95108795166016 40.78593063354492 -73.97761535644531 40.75431442260742 1 2.7 18 0 0.5 4.7 0 0 0.3 23.5 CSH 0 6 a yellow -103 158.01 1 Manhattan 1015801 I MN40 Upper East Side-Carnegie Hill 3805 225 94 1 Manhattan 1009400 E MN17 Midtown-Midtown South 3807 +1200005323 1 2015-07-27 2015-07-27 12:39:46 2015-07-27 2015-07-27 12:56:43 0 1 -73.98226165771484 40.76258087158203 -73.97428894042969 40.746795654296875 1 1.6 11.5 0 0.5 2.45 0 0 0.3 14.75 CSH 0 0 p yellow 3 131 1 Manhattan 1013100 I MN17 Midtown-Midtown South 3807 160 78 1 Manhattan 1007800 I MN20 Murray Hill-Kips Bay 3808 +1200005346 1 2015-07-27 2015-07-27 13:52:50 2015-07-27 2015-07-27 14:07:46 0 1 -73.99342346191406 40.75031661987305 -73.98067474365234 40.765663146972656 1 1.4 11 0 0.5 0 0 0 0.3 11.8 CRE 0 � c yellow -125 101 1 Manhattan 1010100 I MN17 Midtown-Midtown South 3807 4 137 1 Manhattan 1013700 I MN17 Midtown-Midtown South 3807 +1200005350 1 2015-07-27 2015-07-27 14:09:32 2015-07-27 2015-07-27 14:52:32 0 1 -73.97758483886719 40.76474380493164 -74.00617980957031 40.7108154296875 1 7.6 31.5 0 0.5 0 0 0 0.3 32.3 CSH 0 c \t yellow 4 137 1 Manhattan 1013700 I MN17 Midtown-Midtown South 3807 155 15.01 1 Manhattan 1001501 I MN25 Battery Park City-Lower Manhattan 3810 +1200005364 2 2015-07-27 2015-07-27 14:57:54 2015-07-27 2015-07-27 15:38:09 0 1 -73.99381256103516 40.74650573730469 -73.8386001586914 40.72012710571289 5 9.83 34.5 0 0.5 12.25 5.54 0 0.3 53.09 CSH 0 � � yellow -125 95 1 Manhattan 1009500 I MN17 Midtown-Midtown South 3807 112 757.02 4 Queens 4075702 I QN17 Forest Hills 4108 +1200005375 1 2015-07-27 2015-07-27 15:56:14 2015-07-27 2015-07-27 16:10:11 0 1 -73.97189331054688 40.74580383300781 -73.96363830566406 40.77202224731445 1 2.4 11 0 0.5 2.35 0 0 0.3 14.15 CSH 0 3 7 yellow -29 86.01 1 Manhattan 1008601 I MN19 Turtle Bay-East Midtown 3808 23 130 1 Manhattan 1013000 I MN40 Upper East Side-Carnegie Hill 3805 +1200005397 1 2015-07-27 2015-07-27 17:06:01 2015-07-27 2015-07-27 17:12:00 0 1 -73.94676208496094 40.791412353515625 -73.951171875 40.77925491333008 2 1 6 1 0.5 0 0 0 0.3 7.8 CRE 0 u c yellow 120 166 1 Manhattan 1016600 E MN33 East Harlem South 3804 27 146.02 1 Manhattan 1014602 I MN32 Yorkville 3805 +1200005409 2 2015-07-27 2015-07-27 17:42:23 2015-07-27 2015-07-27 18:27:35 0 2 -73.78877258300781 40.641761779785156 -73.95165252685547 40.78036117553711 1 19.07 52 0 0.5 11.67 5.54 0 0.3 70.01 CSH 0 2 c yellow 8 716 4 Queens 4071600 I QN98 Airport 4105 27 146.02 1 Manhattan 1014602 I MN32 Yorkville 3805 +1200005417 2 2015-07-27 2015-07-27 18:05:14 2015-07-27 2015-07-27 18:24:31 0 1 -73.97216796875 40.760101318359375 -73.99280548095703 40.742645263671875 1 2.03 13 1 0.5 0 0 0 0.3 14.8 CRE 0 a 4 yellow 16 102 1 Manhattan 1010200 I MN17 Midtown-Midtown South 3807 133 58 1 Manhattan 1005800 I MN13 Hudson Yards-Chelsea-Flatiron-Union Square 3807 +1200005431 1 2015-07-27 2015-07-27 18:40:31 2015-07-27 2015-07-27 18:54:50 0 1 -73.98480224609375 40.753963470458984 -73.96676635742188 40.76638412475586 1 1.6 10.5 1 0.5 2.25 0 0 0.3 14.55 CSH 0  7 yellow 21 113 1 Manhattan 1011300 I MN17 Midtown-Midtown South 3807 17 120 1 Manhattan 1012000 I MN40 Upper East Side-Carnegie Hill 3805 +1200005439 2 2015-07-27 2015-07-27 19:10:22 2015-07-27 2015-07-27 19:20:48 0 1 -73.97564697265625 40.75852584838867 -73.9775619506836 40.772743225097656 1 1.51 8.5 1 0.5 3.09 0 0 0.3 13.39 CSH 0 a C yellow 16 102 1 Manhattan 1010200 I MN17 Midtown-Midtown South 3807 43 143 1 Manhattan 1014300 I MN99 park-cemetery-etc-Manhattan 3806 +1200005459 1 2015-07-27 2015-07-27 20:06:14 2015-07-27 2015-07-27 20:18:35 0 1 -74.00170135498047 40.75568771362305 -73.94598388671875 40.83543014526367 1 6.7 20 0 0.5 3 0 0 0.3 23.8 CSH 0 F D yellow 50 99 1 Manhattan 1009900 I MN13 Hudson Yards-Chelsea-Flatiron-Union Square 3807 124 247 1 Manhattan 1024700 E MN36 Washington Heights South 3801 +1200005477 1 2015-07-27 2015-07-27 21:02:51 2015-07-27 2015-07-27 21:12:08 0 1 -73.9808120727539 40.74305725097656 -73.98681640625 40.72236251831055 3 1.9 8.5 0.5 0.5 1.95 0 0 0.3 11.75 CSH 0 p H yellow -126 72 1 Manhattan 1007200 I MN20 Murray Hill-Kips Bay 3808 161 30.01 1 Manhattan 1003001 E MN27 Chinatown 3809 +1200005493 2 2015-07-27 2015-07-27 21:38:49 2015-07-27 2015-07-27 21:41:11 0 1 -73.9765853881836 40.79069519042969 -73.98149871826172 40.78459167480469 4 0.52 4 0.5 0.5 1.06 0 0 0.3 6.36 CSH 0 8 9 yellow -118 175 1 Manhattan 1017500 I MN12 Upper West Side 3806 153 167 1 Manhattan 1016700 I MN12 Upper West Side 3806 +1200005501 1 2015-07-27 2015-07-27 21:54:58 2015-07-27 2015-07-27 22:01:31 0 1 -73.97321319580078 40.74848175048828 -73.99017333984375 40.729156494140625 1 1.7 7.5 0.5 0.5 1 0 0 0.3 9.8 CSH 0 3 y yellow -105 88 1 Manhattan 1008800 I MN19 Turtle Bay-East Midtown 3808 190 38 1 Manhattan 1003800 I MN22 East Village 3809 +1200005523 2 2015-07-27 2015-07-27 23:01:26 2015-07-27 2015-07-27 23:07:35 0 1 -74.00537109375 40.73981475830078 -74.0062255859375 40.72032165527344 1 1.43 6.5 0.5 0.5 1.56 0 0 0.3 9.36 CSH 0 X 1 yellow 79 79 1 Manhattan 1007900 I MN23 West Village 3810 15 33 1 Manhattan 1003300 I MN24 SoHo-TriBeCa-Civic Center-Little Italy 3810 +1200005527 2 2015-07-27 2015-07-27 23:16:52 2015-07-27 2015-07-27 23:25:36 0 1 -73.97349548339844 40.794921875 -73.98767852783203 40.77134704589844 1 1.9300000000000002 9 0.5 0.5 2.06 0 0 0.3 12.36 CSH 0 8 C yellow -88 183 1 Manhattan 1018300 I MN12 Upper West Side 3806 2 147 1 Manhattan 1014700 I MN14 Lincoln Square 3806 +1200005545 1 2015-07-28 2015-07-28 00:58:37 2015-07-28 2015-07-28 01:02:47 0 1 -73.96969604492188 40.76272964477539 -73.95670318603516 40.77146911621094 1 1.1 6 0.5 0.5 0 0 0 0.3 7.3 CRE 0 b A yellow 20 112.03 1 Manhattan 1011203 I MN19 Turtle Bay-East Midtown 3808 218 134 1 Manhattan 1013400 I MN31 Lenox Hill-Roosevelt Island 3805 +1200005592 1 2015-07-28 2015-07-28 08:04:28 2015-07-28 2015-07-28 08:13:55 0 1 -73.97981262207031 40.776065826416016 -73.9725570678711 40.759761810302734 1 1.8 9 0 0.5 1.2 0 0 0.3 11 CSH 0 B a yellow 30 153 1 Manhattan 1015300 I MN14 Lincoln Square 3806 16 102 1 Manhattan 1010200 I MN17 Midtown-Midtown South 3807 +1200005623 1 2015-07-28 2015-07-28 10:06:27 2015-07-28 2015-07-28 10:18:23 0 1 -73.97676086425781 40.78847885131836 -73.98551177978516 40.76774597167969 1 1.8 10 0 0.5 2.7 0 0 0.3 13.5 CSH 0 8 H yellow -118 175 1 Manhattan 1017500 I MN12 Upper West Side 3806 202 139 1 Manhattan 1013900 I MN15 Clinton 3807 +1200005644 1 2015-07-28 2015-07-28 11:06:45 2015-07-28 2015-07-28 11:17:26 0 1 -73.98307800292969 40.76338577270508 -74.00325775146484 40.756378173828125 3 1.5 9 0 0.5 0 0 0 0.3 9.8 CRE 0 0 F yellow 3 131 1 Manhattan 1013100 I MN17 Midtown-Midtown South 3807 50 99 1 Manhattan 1009900 I MN13 Hudson Yards-Chelsea-Flatiron-Union Square 3807 +1200005654 2 2015-07-28 2015-07-28 11:42:41 2015-07-28 2015-07-28 11:49:43 0 1 -73.97913360595703 40.76237106323242 -73.99015045166016 40.76974105834961 1 0.98 6.5 0 0.5 1.46 0 0 0.3 8.76 CSH 0 0 P yellow 3 131 1 Manhattan 1013100 I MN17 Midtown-Midtown South 3807 135 135 1 Manhattan 1013500 E MN15 Clinton 3807 +1200005670 2 2015-07-28 2015-07-28 12:31:55 2015-07-28 2015-07-28 12:42:19 0 1 -74.00469207763672 40.70750045776367 -73.97176361083984 40.74625015258789 6 4.14 14 0 0.5 0 0 0 0.3 14.8 CRE 0 � p yellow 12 15.02 1 Manhattan 1001502 I MN25 Battery Park City-Lower Manhattan 3810 160 78 1 Manhattan 1007800 I MN20 Murray Hill-Kips Bay 3808 +1200005694 2 2015-07-28 2015-07-28 14:01:47 2015-07-28 2015-07-28 14:12:21 0 1 -73.97589111328125 40.75489044189453 -73.96794891357422 40.76554870605469 1 0.83 8 0 0.5 2.2 0 0 0.3 11 CSH 0 a 7 yellow -31 94 1 Manhattan 1009400 E MN17 Midtown-Midtown South 3807 228 122 1 Manhattan 1012200 I MN40 Upper East Side-Carnegie Hill 3805 +1200005717 2 2015-07-28 2015-07-28 15:19:21 2015-07-28 2015-07-28 15:31:18 0 1 -73.98280334472656 40.7690544128418 -73.97312927246094 40.779991149902344 5 1.9100000000000001 10 0 0.5 0 0 0 0.3 10.8 CRE 0 B C yellow 66 145 1 Manhattan 1014500 I MN14 Lincoln Square 3806 43 143 1 Manhattan 1014300 I MN99 park-cemetery-etc-Manhattan 3806 +1200005732 1 2015-07-28 2015-07-28 16:12:38 2015-07-28 2015-07-28 16:34:19 0 1 -73.92538452148438 40.7435302734375 -73.86176300048828 40.735389709472656 1 4.8 19 1 0.5 10 0 0 0.3 30.8 CSH 0 & V yellow 15 179 4 Queens 4017900 E QN31 Hunters Point-Sunnyside-West Maspeth 4109 195 455 4 Queens 4045500 E QN25 Corona 4107 +1200005738 2 2015-07-28 2015-07-28 16:40:42 2015-07-28 2015-07-28 16:54:23 0 1 -73.98468017578125 40.72977828979492 -73.97615814208984 40.7287483215332 1 2.2800000000000002 10 1 0.5 0 0 0 0.3 11.8 CRE 0 y $ yellow 93 40 1 Manhattan 1004000 I MN22 East Village 3809 110 44 1 Manhattan 1004400 I MN50 Stuyvesant Town-Cooper Village 3808 +1200005873 2 2015-07-28 2015-07-28 22:39:51 2015-07-28 2015-07-28 23:11:23 0 1 -73.92137908935547 40.7437858581543 -73.78206634521484 40.70505905151367 6 10.68 33.5 0.5 0.5 0 0 0 0.3 34.8 CRE 0 &  yellow 18 183 4 Queens 4018300 I QN31 Hunters Point-Sunnyside-West Maspeth 4109 11 440 4 Queens 4044000 E QN01 South Jamaica 4112 +1200005894 1 2015-07-29 2015-07-29 00:14:14 2015-07-29 2015-07-29 00:18:07 0 1 -73.98120880126953 40.75035858154297 -73.98229217529297 40.76122283935547 1 1 5 0.5 0.5 1.26 0 0 0.3 7.56 CSH 0 p 0 yellow 93 82 1 Manhattan 1008200 I MN20 Murray Hill-Kips Bay 3808 3 131 1 Manhattan 1013100 I MN17 Midtown-Midtown South 3807 +1200005902 2 2015-07-29 2015-07-29 01:02:49 2015-07-29 2015-07-29 01:07:20 0 1 -73.97600555419922 40.75516891479492 -73.9671401977539 40.76372528076172 5 0.96 5.5 0.5 0.5 0 0 0 0.3 6.8 CRE 0 a 7 yellow -31 94 1 Manhattan 1009400 E MN17 Midtown-Midtown South 3807 22 114.02 1 Manhattan 1011402 I MN40 Upper East Side-Carnegie Hill 3805 +1200005908 1 2015-07-29 2015-07-29 01:40:34 2015-07-29 2015-07-29 02:17:43 0 1 -73.98140716552734 40.752410888671875 -74.14974975585938 40.62253189086914 1 20 55.5 0.5 0.5 18.35 16.62 0 0.3 91.77 CSH 0 p V yellow 93 82 1 Manhattan 1008200 I MN20 Murray Hill-Kips Bay 3808 31 303.01 5 Staten Island 5030301 I SI12 Mariner\'s Harbor-Arlington-Port Ivory-Graniteville 3903 +1200005915 2 2015-07-29 2015-07-29 03:58:18 2015-07-29 2015-07-29 04:04:39 0 1 -73.97283935546875 40.793243408203125 -73.9460678100586 40.83540344238281 5 3.66 12 0.5 0.5 2.66 0 0 0.3 15.96 CSH 0 8 D yellow -89 179 1 Manhattan 1017900 I MN12 Upper West Side 3806 124 247 1 Manhattan 1024700 E MN36 Washington Heights South 3801 +1200005933 2 2015-07-29 2015-07-29 07:08:39 2015-07-29 2015-07-29 07:20:06 0 1 -73.94747161865234 40.77119445800781 -73.9767074584961 40.764163970947266 5 2.2800000000000002 10.5 0 0.5 1 0 0 0.3 12.3 CSH 0 b c yellow -1 136 1 Manhattan 1013600 I MN32 Yorkville 3805 18 112.01 1 Manhattan 1011201 I MN17 Midtown-Midtown South 3807 +1200005947 1 2015-07-29 2015-07-29 07:50:58 2015-07-29 2015-07-29 08:04:50 0 1 -73.9941635131836 40.751041412353516 -73.99542999267578 40.72623825073242 1 2.6 11.5 0 0.5 0 0 0 0.3 12.3 CRE 0 �  yellow -125 101 1 Manhattan 1010100 I MN17 Midtown-Midtown South 3807 191 55.02 1 Manhattan 1005502 I MN23 West Village 3810 +1200005959 2 2015-07-29 2015-07-29 08:16:34 2015-07-29 2015-07-29 08:33:43 0 1 -73.99761199951172 40.720821380615234 -73.97298431396484 40.75828552246094 1 3.01 13.5 0 0.5 1 0 0 0.3 15.3 CSH 0 D b yellow -60 41 1 Manhattan 1004100 E MN24 SoHo-TriBeCa-Civic Center-Little Italy 3810 15 100 1 Manhattan 1010000 I MN19 Turtle Bay-East Midtown 3808 +1200005974 2 2015-07-29 2015-07-29 09:17:10 2015-07-29 2015-07-29 09:29:26 0 1 -74.00839233398438 40.73527908325195 -73.9899673461914 40.74187088012695 5 1.45 9 0 0.5 1.96 0 0 0.3 11.76 CSH 0 X 4 yellow -106 75 1 Manhattan 1007500 I MN23 West Village 3810 133 58 1 Manhattan 1005800 I MN13 Hudson Yards-Chelsea-Flatiron-Union Square 3807 +1200005983 2 2015-07-29 2015-07-29 09:44:41 2015-07-29 2015-07-29 10:10:31 0 1 -73.97784423828125 40.773494720458984 -73.9777603149414 40.75950622558594 1 1.8 16 0 0.5 0 0 0 0.3 16.8 CRE 0 C a yellow 43 143 1 Manhattan 1014300 I MN99 park-cemetery-etc-Manhattan 3806 17 104 1 Manhattan 1010400 I MN17 Midtown-Midtown South 3807 +1200006011 1 2015-07-29 2015-07-29 11:29:03 2015-07-29 2015-07-29 11:41:43 0 1 -73.97634887695312 40.75967025756836 -73.99529266357422 40.743892669677734 1 1.7 9.5 0 0.5 0 0 0 0.3 10.3 CSH 0 a � yellow 16 102 1 Manhattan 1010200 I MN17 Midtown-Midtown South 3807 132 91 1 Manhattan 1009100 I MN13 Hudson Yards-Chelsea-Flatiron-Union Square 3807 +1200006016 1 2015-07-29 2015-07-29 11:42:25 2015-07-29 2015-07-29 12:00:04 0 1 -73.9685287475586 40.761680603027344 -73.98332214355469 40.754661560058594 1 1.1 11.5 0 0.5 0 0 0 0.3 12.3 CRE 0 b a yellow 20 112.03 1 Manhattan 1011203 I MN19 Turtle Bay-East Midtown 3808 13 96 1 Manhattan 1009600 E MN17 Midtown-Midtown South 3807 +1200006033 1 2015-07-29 2015-07-29 12:24:17 2015-07-29 2015-07-29 12:32:25 0 1 -73.9569091796875 40.78612518310547 -73.95484161376953 40.773529052734375 1 1.2 7 0 0.5 0 0 0 0.3 7.8 CRE 0 6 c yellow 34 160.01 1 Manhattan 1016001 I MN40 Upper East Side-Carnegie Hill 3805 0 138 1 Manhattan 1013800 I MN32 Yorkville 3805 +1200006047 2 2015-07-29 2015-07-29 13:00:31 2015-07-29 2015-07-29 13:07:35 0 1 -73.97667694091797 40.76594543457031 -73.97296905517578 40.78022384643555 1 1.26 7 0 0.5 0 0 0 0.3 7.8 CRE 0 C C yellow 43 143 1 Manhattan 1014300 I MN99 park-cemetery-etc-Manhattan 3806 43 143 1 Manhattan 1014300 I MN99 park-cemetery-etc-Manhattan 3806 +1200006057 2 2015-07-29 2015-07-29 13:15:42 2015-07-29 2015-07-29 13:26:25 0 1 -73.98502349853516 40.7419548034668 -73.99201965332031 40.74918746948242 4 0.8 7.5 0 0.5 2.49 0 0 0.3 10.79 CSH 0  � yellow 11 68 1 Manhattan 1006800 I MN21 Gramercy 3808 131 101 1 Manhattan 1010100 I MN17 Midtown-Midtown South 3807 +1200006069 1 2015-07-29 2015-07-29 14:17:27 2015-07-29 2015-07-29 14:30:34 0 1 -73.97695922851562 40.7613410949707 -73.96929931640625 40.76670455932617 1 0.8 9 0 0.5 2.45 0 0 0.3 12.25 CSH 0 a 7 yellow 17 104 1 Manhattan 1010400 I MN17 Midtown-Midtown South 3807 228 122 1 Manhattan 1012200 I MN40 Upper East Side-Carnegie Hill 3805 +1200006074 2 2015-07-29 2015-07-29 14:38:46 2015-07-29 2015-07-29 14:43:57 0 1 -73.98220825195312 40.77484130859375 -73.98319244384766 40.763877868652344 2 0.77 5.5 0 0.5 0 0 0 0.3 6.3 CRE 0 B 0 yellow 30 153 1 Manhattan 1015300 I MN14 Lincoln Square 3806 3 131 1 Manhattan 1013100 I MN17 Midtown-Midtown South 3807 +1200006077 1 2015-07-29 2015-07-29 14:44:00 2015-07-29 2015-07-29 14:57:45 0 1 -73.9673080444336 40.76920700073242 -73.97575378417969 40.7559814453125 1 1.2 9.5 0 0.5 0 0 0 0.3 10.3 CRE 0 7 a yellow -28 122 1 Manhattan 1012200 I MN40 Upper East Side-Carnegie Hill 3805 225 94 1 Manhattan 1009400 E MN17 Midtown-Midtown South 3807 +1200006099 1 2015-07-29 2015-07-29 15:42:27 2015-07-29 2015-07-29 15:51:26 0 1 -73.99339294433594 40.747188568115234 -74.00535583496094 40.7276496887207 1 1.5 8.5 0 0.5 2.3 0 0 0.3 11.6 CSH 0 � % yellow -125 95 1 Manhattan 1009500 I MN17 Midtown-Midtown South 3807 13 37 1 Manhattan 1003700 I MN24 SoHo-TriBeCa-Civic Center-Little Italy 3810 +1200006157 2 2015-07-29 2015-07-29 18:39:25 2015-07-29 2015-07-29 18:56:00 0 1 -73.9563217163086 40.74445724487305 -73.86500549316406 40.77044677734375 6 8.13 24 1 0.5 5 0 0 0.3 30.8 CSH 0 E 8 yellow 33 1 4 Queens 4000100 I QN31 Hunters Point-Sunnyside-West Maspeth 4109 48 331 4 Queens 4033100 I QN98 Airport 4102 +1200006163 1 2015-07-29 2015-07-29 18:58:24 2015-07-29 2015-07-29 19:08:33 0 1 -73.98808288574219 40.71888732910156 -73.97981262207031 40.732765197753906 1 1.2 8 1 0.5 1.96 0 0 0.3 11.76 CSH 0 H $ yellow -61 18 1 Manhattan 1001800 E MN27 Chinatown 3809 110 44 1 Manhattan 1004400 I MN50 Stuyvesant Town-Cooper Village 3808 +1200006184 1 2015-07-29 2015-07-29 19:26:44 2015-07-29 2015-07-29 19:34:19 0 1 -73.98413848876953 40.721092224121094 -74.0114517211914 40.704368591308594 2 3.2 11.5 1 0.5 2.65 0 0 0.3 15.95 CSH 0 H � yellow -95 30.01 1 Manhattan 1003001 E MN27 Chinatown 3809 0 9 1 Manhattan 1000900 I MN25 Battery Park City-Lower Manhattan 3810 +1200006209 2 2015-07-29 2015-07-29 20:28:33 2015-07-29 2015-07-29 20:37:25 0 1 -73.95558166503906 40.77960968017578 -73.97753143310547 40.7889518737793 2 1.3900000000000001 8 0.5 0.5 1.86 0 0 0.3 11.16 CSH 0 6 8 yellow 29 148.02 1 Manhattan 1014802 I MN40 Upper East Side-Carnegie Hill 3805 138 175 1 Manhattan 1017500 I MN12 Upper West Side 3806 +1200006218 2 2015-07-29 2015-07-29 20:55:11 2015-07-29 2015-07-29 21:10:22 0 1 -73.97083282470703 40.75865173339844 -73.99455261230469 40.72602081298828 1 2.73 12.5 0.5 0.5 2.76 0 0 0.3 16.56 CSH 0 b  yellow 15 100 1 Manhattan 1010000 I MN19 Turtle Bay-East Midtown 3808 191 55.02 1 Manhattan 1005502 I MN23 West Village 3810 +1200006229 1 2015-07-29 2015-07-29 21:29:55 2015-07-29 2015-07-29 21:37:28 0 1 -73.98661041259766 40.76160430908203 -73.97483825683594 40.76154708862305 2 0.9 6.5 0.5 0.5 0 0 0 0.3 7.8 CRE 0 H a yellow -118 127 1 Manhattan 1012700 I MN15 Clinton 3807 16 102 1 Manhattan 1010200 I MN17 Midtown-Midtown South 3807 +1200006235 1 2015-07-29 2015-07-29 21:38:29 2015-07-29 2015-07-29 21:49:44 0 1 -73.9766616821289 40.76486587524414 -73.9736328125 40.79133987426758 1 2.4 10.5 0.5 0.5 3 0 0 0.3 14.8 CSH 0 c 8 yellow 18 112.01 1 Manhattan 1011201 I MN17 Midtown-Midtown South 3807 167 179 1 Manhattan 1017900 I MN12 Upper West Side 3806 +1200006252 2 2015-07-29 2015-07-29 22:11:56 2015-07-29 2015-07-29 22:26:18 0 1 -74.00281524658203 40.73372268676758 -73.98397064208984 40.76521682739258 1 2.71 12 0.5 0.5 2.66 0 0 0.3 15.96 CSH 0 I H yellow -93 73 1 Manhattan 1007300 I MN23 West Village 3810 202 139 1 Manhattan 1013900 I MN15 Clinton 3807 +1200006280 2 2015-07-29 2015-07-29 23:25:41 2015-07-29 2015-07-29 23:30:50 0 1 -73.98812103271484 40.74348068237305 -73.97834777832031 40.74993896484375 5 0.94 6 0.5 0.5 2.19 0 0 0.3 9.49 CSH 0 4 p yellow -90 56 1 Manhattan 1005600 I MN13 Hudson Yards-Chelsea-Flatiron-Union Square 3807 133 80 1 Manhattan 1008000 I MN20 Murray Hill-Kips Bay 3808 +1200006306 2 2015-07-30 2015-07-30 01:19:35 2015-07-30 2015-07-30 01:30:20 0 1 -73.98906707763672 40.72661209106445 -74.01464080810547 40.709468841552734 1 4.46 15 0.5 0.5 0 0 0 0.3 16.3 CRE 0 y a yellow -66 38 1 Manhattan 1003800 I MN22 East Village 3809 53 13 1 Manhattan 1001300 I MN25 Battery Park City-Lower Manhattan 3810 +1200006323 2 2015-07-30 2015-07-30 03:55:01 2015-07-30 2015-07-30 04:02:33 0 1 -73.98578643798828 40.753570556640625 -73.99324798583984 40.7294807434082 1 2.46 9.5 0.5 0.5 1 0 0 0.3 11.8 CSH 0   yellow 21 113 1 Manhattan 1011300 I MN17 Midtown-Midtown South 3807 192 57 1 Manhattan 1005700 I MN23 West Village 3810 +1200006335 2 2015-07-30 2015-07-30 06:37:25 2015-07-30 2015-07-30 06:43:37 0 1 -73.97126007080078 40.75587844848633 -73.9594955444336 40.77351760864258 1 1.51 7 0 0.5 1.95 0 0 0.3 9.75 CSH 0 b 6 yellow 15 100 1 Manhattan 1010000 I MN19 Turtle Bay-East Midtown 3808 24 140 1 Manhattan 1014000 I MN40 Upper East Side-Carnegie Hill 3805 +1200006363 1 2015-07-30 2015-07-30 07:56:20 2015-07-30 2015-07-30 08:22:07 0 1 -73.94933319091797 40.77703857421875 -73.97990417480469 40.755611419677734 1 2.8 17 0 0.5 3.55 0 0 0.3 21.35 CSH 0 c a yellow 27 146.02 1 Manhattan 1014602 I MN32 Yorkville 3805 13 96 1 Manhattan 1009600 E MN17 Midtown-Midtown South 3807 +1200006395 1 2015-07-30 2015-07-30 09:09:25 2015-07-30 2015-07-30 09:23:53 0 1 -73.96145629882812 40.768951416015625 -73.98268127441406 40.76246643066406 1 1.9 11 0 0.5 0 0 0 0.3 11.8 CRE 0 7 0 yellow 18 128 1 Manhattan 1012800 I MN40 Upper East Side-Carnegie Hill 3805 3 131 1 Manhattan 1013100 I MN17 Midtown-Midtown South 3807 +1200006397 1 2015-07-30 2015-07-30 09:13:05 2015-07-30 2015-07-30 09:44:50 0 1 -73.9781723022461 40.752376556396484 -74.0147705078125 40.71396255493164 1 7 27.5 0 0.5 2 0 0 0.3 30.3 CSH 0 b  yellow -30 92 1 Manhattan 1009200 I MN19 Turtle Bay-East Midtown 3808 52 317.03 1 Manhattan 1031703 I MN25 Battery Park City-Lower Manhattan 3810 +1200006413 2 2015-07-30 2015-07-30 10:07:12 2015-07-30 2015-07-30 10:19:45 0 1 -73.95320129394531 40.7717399597168 -73.97758483886719 40.738121032714844 1 3.23 12.5 0 0.5 2.66 0 0 0.3 15.96 CSH 0 A 7 yellow -38 134 1 Manhattan 1013400 I MN31 Lenox Hill-Roosevelt Island 3805 85 62 1 Manhattan 1006200 I MN20 Murray Hill-Kips Bay 3808 +1200006433 2 2015-07-30 2015-07-30 11:10:44 2015-07-30 2015-07-30 11:59:35 0 1 -73.99678802490234 40.74745178222656 -73.94703674316406 40.792518615722656 2 4.73 30 0 0.5 6.16 0 0 0.3 36.96 CSH 0 � u yellow -125 95 1 Manhattan 1009500 I MN17 Midtown-Midtown South 3807 35 172 1 Manhattan 1017200 E MN33 East Harlem South 3804 +1200006507 2 2015-07-30 2015-07-30 14:52:17 2015-07-30 2015-07-30 15:13:18 0 1 -73.98333740234375 40.769935607910156 -73.97623443603516 40.76112747192383 6 1 13 0 0.5 0 0 0 0.3 13.8 CRE 0 B a yellow 66 145 1 Manhattan 1014500 I MN14 Lincoln Square 3806 17 104 1 Manhattan 1010400 I MN17 Midtown-Midtown South 3807 +1200006543 2 2015-07-30 2015-07-30 17:36:15 2015-07-30 2015-07-30 17:46:27 0 1 -73.9725570678711 40.78105926513672 -73.97937774658203 40.764801025390625 1 1.5699999999999998 8.5 1 0.5 0 0 0 0.3 10.3 CRE 0 9 c yellow 22 165 1 Manhattan 1016500 I MN12 Upper West Side 3806 4 137 1 Manhattan 1013700 I MN17 Midtown-Midtown South 3807 +1200006559 1 2015-07-30 2015-07-30 18:38:28 2015-07-30 2015-07-30 19:03:10 0 1 -74.00841522216797 40.73929977416992 -73.98599243164062 40.77793884277344 1 3.3 16.5 1 0.5 3 0 0 0.3 21.3 CSH 0 X C yellow 79 79 1 Manhattan 1007900 I MN23 West Village 3810 206 155 1 Manhattan 1015500 I MN14 Lincoln Square 3806 +1200006589 1 2015-07-30 2015-07-30 20:11:16 2015-07-30 2015-07-30 20:32:08 0 1 -73.97032928466797 40.76479721069336 -73.99247741699219 40.751766204833984 1 1.9 14 0.5 0.5 0 0 0 0.3 15.3 CRE 0 7 � yellow -29 114.01 1 Manhattan 1011401 I MN40 Upper East Side-Carnegie Hill 3805 131 101 1 Manhattan 1010100 I MN17 Midtown-Midtown South 3807 +1200006665 2 2015-07-31 2015-07-31 00:11:45 2015-07-31 2015-07-31 00:25:15 0 1 -73.9854507446289 40.753231048583984 -73.99014282226562 40.7264289855957 1 2.35 10.5 0.5 0.5 2.36 0 0 0.3 14.16 CSH 0  y yellow 21 113 1 Manhattan 1011300 I MN17 Midtown-Midtown South 3807 190 38 1 Manhattan 1003800 I MN22 East Village 3809 +1200006677 1 2015-07-31 2015-07-31 00:47:40 2015-07-31 2015-07-31 00:59:37 0 1 -73.78215789794922 40.64474868774414 -73.86817169189453 40.67927932739258 1 7.3 21.5 0.5 0.5 0.02 0 0 0.3 22.82 CSH 0 2 c yellow 8 716 4 Queens 4071600 I QN98 Airport 4105 62 1188 3 Brooklyn 3118800 E BK83 Cypress Hills-City Line 4008 +1200006689 2 2015-07-31 2015-07-31 01:38:02 2015-07-31 2015-07-31 01:46:29 0 1 -73.97042083740234 40.756248474121094 -73.99324798583984 40.73408889770508 1 2.38 9 0.5 0.5 2.06 0 0 0.3 12.36 CSH 0 )  yellow 14 98 1 Manhattan 1009800 I MN19 Turtle Bay-East Midtown 3808 201 61 1 Manhattan 1006100 I MN23 West Village 3810 +1200006706 1 2015-07-31 2015-07-31 04:13:04 2015-07-31 2015-07-31 04:25:29 0 1 -73.98286437988281 40.761924743652344 -73.98268127441406 40.72341537475586 1 3.6 13 0.5 0.5 2.85 0 0 0.3 17.15 CSH 0 0 y yellow 3 131 1 Manhattan 1013100 I MN17 Midtown-Midtown South 3807 10 32 1 Manhattan 1003200 I MN22 East Village 3809 +1200006716 1 2015-07-31 2015-07-31 06:12:57 2015-07-31 2015-07-31 06:20:01 0 1 -73.958740234375 40.76412582397461 -73.97467803955078 40.75167465209961 1 1.6 7.5 0 0.5 0 0 0 0.3 8.3 CRE 0 A b yellow -74 118 1 Manhattan 1011800 I MN31 Lenox Hill-Roosevelt Island 3805 226 92 1 Manhattan 1009200 I MN19 Turtle Bay-East Midtown 3808 +1200006775 2 2015-07-31 2015-07-31 10:19:59 2015-07-31 2015-07-31 10:22:40 0 1 -73.95901489257812 40.81492614746094 -73.96833801269531 40.799781799316406 1 1.19 5.5 0 0.5 0.5 0 0 0.3 6.8 CSH 0 f Q yellow 45 211 1 Manhattan 1021100 E MN09 Morningside Heights 3802 112 191 1 Manhattan 1019100 I MN12 Upper West Side 3806 +1200006809 2 2015-07-31 2015-07-31 12:16:41 2015-07-31 2015-07-31 12:31:34 0 1 -73.9876937866211 40.747711181640625 -73.97814178466797 40.761009216308594 1 1.47 10.5 0 0.5 0 0 0 0.3 11.3 CRE 0 d a yellow -48 76 1 Manhattan 1007600 I MN17 Midtown-Midtown South 3807 17 104 1 Manhattan 1010400 I MN17 Midtown-Midtown South 3807 +1200006814 1 2015-07-31 2015-07-31 12:38:40 2015-07-31 2015-07-31 12:52:25 1 1 -73.949462890625 40.77246856689453 -73.97256469726562 40.743839263916016 1 2.5 12.5 0 0.5 0 0 0 0.3 13.3 CRE 0 b 3 yellow -1 136 1 Manhattan 1013600 I MN32 Yorkville 3805 227 86.01 1 Manhattan 1008601 I MN19 Turtle Bay-East Midtown 3808 +1200006821 1 2015-07-31 2015-07-31 12:56:03 2015-07-31 2015-07-31 13:05:02 0 1 -74.00324249267578 40.74739456176758 -73.99504852294922 40.7498893737793 1 0.6 7 0 0.5 0 0 0 0.3 7.8 CRE 0 h � yellow 113 93 1 Manhattan 1009300 I MN13 Hudson Yards-Chelsea-Flatiron-Union Square 3807 131 101 1 Manhattan 1010100 I MN17 Midtown-Midtown South 3807 +1200006822 1 2015-07-31 2015-07-31 12:56:59 2015-07-31 2015-07-31 13:00:11 0 1 -73.98331451416016 40.73076629638672 -73.99066162109375 40.73392105102539 1 0.4 4 0 0.5 0 0 0 0.3 4.8 CRE 0 y  yellow 93 40 1 Manhattan 1004000 I MN22 East Village 3809 201 61 1 Manhattan 1006100 I MN23 West Village 3810 +1200006823 2 2015-07-31 2015-07-31 12:57:41 2015-07-31 2015-07-31 13:04:37 0 1 -73.9660873413086 40.80512619018555 -73.97334289550781 40.78490447998047 4 1.77 7.5 0 0.5 1.66 0 0 0.3 9.96 CSH 0 f 9 yellow 47 199 1 Manhattan 1019900 I MN09 Morningside Heights 3802 136 169 1 Manhattan 1016900 I MN12 Upper West Side 3806 +1200006832 1 2015-07-31 2015-07-31 13:37:41 2015-07-31 2015-07-31 13:51:38 0 1 -73.99606323242188 40.73804473876953 -73.98711395263672 40.75064468383789 1 1.1 10 0 0.5 1 0 0 0.3 11.8 CSH 0 4 d yellow -114 54 1 Manhattan 1005400 I MN13 Hudson Yards-Chelsea-Flatiron-Union Square 3807 209 84 1 Manhattan 1008400 I MN17 Midtown-Midtown South 3807 +1200006839 1 2015-07-31 2015-07-31 14:05:17 2015-07-31 2015-07-31 14:07:42 0 1 -73.9672622680664 40.793277740478516 -73.97355651855469 40.78467559814453 1 0.6 4 0 0.5 0 0 0 0.3 4.8 CRE 0 8 9 yellow 24 181 1 Manhattan 1018100 I MN12 Upper West Side 3806 136 169 1 Manhattan 1016900 I MN12 Upper West Side 3806 +1200006848 1 2015-07-31 2015-07-31 14:41:43 2015-07-31 2015-07-31 15:03:57 0 1 -73.9712142944336 40.78761672973633 -73.98942565917969 40.76267623901367 1 1.9 15 0 0.5 3.15 0 0 0.3 18.95 CSH 0 8 H yellow -119 173 1 Manhattan 1017300 I MN12 Upper West Side 3806 138 127 1 Manhattan 1012700 I MN15 Clinton 3807 +1200006913 1 2015-07-31 2015-07-31 18:25:21 2015-07-31 2015-07-31 18:36:01 0 1 -73.96147918701172 40.801334381103516 -73.95127868652344 40.81019592285156 1 1 8 1 0.5 0 0 0 0.3 9.8 CRE 0 $ A yellow 110 193 1 Manhattan 1019300 E MN09 Morningside Heights 3802 47 257 1 Manhattan 1025700 E MN11 Central Harlem South 3803 +1200006921 2 2015-07-31 2015-07-31 18:37:04 2015-07-31 2015-07-31 18:39:40 0 1 -73.9651107788086 40.77225875854492 -73.95811462402344 40.7818489074707 1 0.77 4.5 1 0.5 0 0 0 0.3 6.3 CRE 0 7 6 yellow 23 130 1 Manhattan 1013000 I MN40 Upper East Side-Carnegie Hill 3805 74 150.02 1 Manhattan 1015002 I MN40 Upper East Side-Carnegie Hill 3805 +1200006938 1 2015-07-31 2015-07-31 19:27:21 2015-07-31 2015-07-31 19:35:06 0 1 -73.9453125 40.778778076171875 -73.96776580810547 40.76277542114258 1 2 8.5 1 0.5 0 0 0 0.3 10.3 CRE 0 b 7 yellow 1 152 1 Manhattan 1015200 I MN32 Yorkville 3805 22 114.02 1 Manhattan 1011402 I MN40 Upper East Side-Carnegie Hill 3805 +1200006953 2 2015-07-31 2015-07-31 20:05:44 2015-07-31 2015-07-31 20:13:10 0 1 -73.95556640625 40.804298400878906 -73.94181060791016 40.798728942871094 6 0.98 6.5 0.5 0.5 0 0 0 0.3 7.8 CRE 0 A t yellow 38 201.02 1 Manhattan 1020102 E MN11 Central Harlem South 3803 230 182 1 Manhattan 1018200 E MN34 East Harlem North 3804 +1200006968 2 2015-07-31 2015-07-31 21:00:34 2015-07-31 2015-07-31 21:03:49 0 1 -73.9594497680664 40.77146911621094 -73.95362091064453 40.7828254699707 5 1.02 5 0.5 0.5 1.26 0 0 0.3 7.56 CSH 0 7 6 yellow 18 128 1 Manhattan 1012800 I MN40 Upper East Side-Carnegie Hill 3805 29 148.02 1 Manhattan 1014802 I MN40 Upper East Side-Carnegie Hill 3805 +1200006969 1 2015-07-31 2015-07-31 21:04:02 2015-07-31 2015-07-31 21:27:54 0 1 -73.97811126708984 40.72514724731445 -73.95539855957031 40.73725509643555 1 6.4 22 0.5 0.5 3.5 0 0 0.3 26.8 CSH 0   yellow -108 26.02 1 Manhattan 1002602 E MN28 Lower East Side 3809 165 579 3 Brooklyn 3057900 E BK76 Greenpoint 4001 +1200007001 2 2015-07-31 2015-07-31 22:13:55 2015-07-31 2015-07-31 22:34:02 0 1 -73.99978637695312 40.734371185302734 -73.94956970214844 40.71535110473633 1 4.27 17.5 0.5 0.5 3.76 0 0 0.3 22.56 CSH 0 I � yellow -94 71 1 Manhattan 1007100 I MN23 West Village 3810 66 501 3 Brooklyn 3050100 I BK90 East Williamsburg 4001 +1200007024 2 2015-07-31 2015-07-31 23:18:26 2015-07-31 2015-07-31 23:28:58 0 1 -73.98070526123047 40.730506896972656 -73.98835754394531 40.71869659423828 1 1.03 8 0.5 0.5 0 0 0 0.3 9.3 CRE 0 y H yellow 98 34 1 Manhattan 1003400 I MN22 East Village 3809 195 18 1 Manhattan 1001800 E MN27 Chinatown 3809 +1200007063 1 2015-07-29 2015-07-29 17:24:49 2015-07-29 2015-07-29 17:25:25 0 5 -73.82606506347656 40.92140197753906 -73.82606506347656 40.92140197753906 2 0 88 0 0 15 5.54 0 0.3 108.84 CSH 0 yellow 0 0 0 0000000 0000 0 0 0 0 0000000 0000 0 +1200007064 1 2015-07-15 2015-07-15 00:52:59 2015-07-15 2015-07-15 01:04:41 0 1 0 0 0 0 1 2.6 10.5 0.5 0.5 2.95 0 0 0.3 14.75 CSH 0 yellow 0 0 0 0000000 0000 0 0 0 0 0000000 0000 0 +1200007079 1 2015-07-10 2015-07-10 20:55:55 2015-07-10 2015-07-10 20:59:49 0 1 0 0 0 0 2 0.6 5 0.5 0.5 0 0 0 0.3 6.3 CRE 0 yellow 0 0 0 0000000 0000 0 0 0 0 0000000 0000 0 +1200007132 2 2015-07-08 2015-07-08 19:00:52 2015-07-08 2015-07-08 19:05:15 0 1 0 0 0 0 1 0.66 5 1 0.5 1.7 0 0 0.3 8.5 CSH 0 yellow 0 0 0 0000000 0000 0 0 0 0 0000000 0000 0 +1200007157 1 2015-07-11 2015-07-11 18:26:04 2015-07-11 2015-07-11 18:26:04 0 1 -73.92134094238281 40.69123458862305 0 0 2 0 15 0 0.5 0 0 0 0.3 15.8 CRE 0 7 yellow -64 397 3 Brooklyn 3039700 E BK78 Bushwick South 4002 0 0 0 0000000 0000 0 +1200007186 1 2015-07-20 2015-07-20 05:54:34 2015-07-20 2015-07-20 05:57:00 0 1 0 0 0 0 1 0.8 4.5 0.5 0.5 1 0 0 0.3 6.8 CSH 0 yellow 0 0 0 0000000 0000 0 0 0 0 0000000 0000 0 +1200007200 2 2015-07-03 2015-07-03 23:42:03 2015-07-04 2015-07-04 00:11:43 0 1 0 0 0 0 5 7.97 28.5 0.5 0.5 5.96 0 0 0.3 35.76 CSH 0 yellow 0 0 0 0000000 0000 0 0 0 0 0000000 0000 0 +1200007228 1 2015-07-04 2015-07-04 20:40:03 2015-07-04 2015-07-04 20:45:19 0 1 0 0 0 0 2 1.3 6 0.5 0.5 0 0 0 0.3 7.3 CRE 0 yellow 0 0 0 0000000 0000 0 0 0 0 0000000 0000 0 +1200007233 2 2015-07-01 2015-07-01 00:06:29 2015-07-01 2015-07-01 00:18:55 0 1 -73.98421478271484 40.725257873535156 -73.97735595703125 40.683837890625 1 3.7199999999999998 13 0.5 0.5 2 0 0 0.3 16.3 CSH 0 y � yellow 10 32 1 Manhattan 1003200 I MN22 East Village 3809 39 129.01 3 Brooklyn 3012901 I BK37 Park Slope-Gowanus 4005 +1200007290 1 2015-07-01 2015-07-01 07:39:27 2015-07-01 2015-07-01 07:48:36 0 1 -73.97777557373047 40.74229049682617 -74.00455474853516 40.74070358276367 1 1.9 9 0 0.5 1 0 0 0.3 10.8 CSH 0 7 h yellow -10 70 1 Manhattan 1007000 I MN20 Murray Hill-Kips Bay 3808 75 83 1 Manhattan 1008300 E MN13 Hudson Yards-Chelsea-Flatiron-Union Square 3807 +1200007304 2 2015-07-01 2015-07-01 08:21:54 2015-07-01 2015-07-01 09:17:58 0 2 -73.79147338867188 40.64535140991211 -74.01126861572266 40.70222473144531 5 20.81 52 0 0.5 17.5 5.54 0 0.3 75.84 CSH 0 2 � yellow 8 716 4 Queens 4071600 I QN98 Airport 4105 0 9 1 Manhattan 1000900 I MN25 Battery Park City-Lower Manhattan 3810 +1200007310 2 2015-07-01 2015-07-01 08:39:14 2015-07-01 2015-07-01 08:51:11 0 1 -73.97483825683594 40.7650032043457 -73.99361419677734 40.74672317504883 1 2.05 10 0 0.5 0 0 0 0.3 10.8 CRE 0 c � yellow 18 112.01 1 Manhattan 1011201 I MN17 Midtown-Midtown South 3807 131 95 1 Manhattan 1009500 I MN17 Midtown-Midtown South 3807 +1200007318 1 2015-07-01 2015-07-01 08:47:24 2015-07-01 2015-07-01 08:54:34 0 1 -73.965576171875 40.76527404785156 -73.95428466796875 40.76737976074219 1 0.8 6.5 0 0.5 0 0 0 0.3 7.3 CSH 0 7 @ yellow 17 120 1 Manhattan 1012000 I MN40 Upper East Side-Carnegie Hill 3805 160 124 1 Manhattan 1012400 I MN31 Lenox Hill-Roosevelt Island 3805 +1200007340 2 2015-07-01 2015-07-01 09:41:15 2015-07-01 2015-07-01 09:48:05 0 1 -73.96173858642578 40.77983093261719 -73.9538803100586 40.79066848754883 6 1.23 6.5 0 0.5 0 0 0 0.3 7.3 CRE 0 C C yellow 43 143 1 Manhattan 1014300 I MN99 park-cemetery-etc-Manhattan 3806 43 143 1 Manhattan 1014300 I MN99 park-cemetery-etc-Manhattan 3806 +1200007344 1 2015-07-01 2015-07-01 09:31:32 2015-07-01 2015-07-01 09:52:10 0 1 -73.91536712646484 40.76346969604492 -73.98749542236328 40.74155044555664 1 3.8 17.5 0 0.5 3.66 0 0 0.3 21.96 CSH 0  4 yellow 105 155 4 Queens 4015500 I QN70 Astoria 4101 166 56 1 Manhattan 1005600 I MN13 Hudson Yards-Chelsea-Flatiron-Union Square 3807 +1200007366 2 2015-07-01 2015-07-01 10:45:30 2015-07-01 2015-07-01 10:50:40 0 1 -73.97015380859375 40.794219970703125 -73.95894622802734 40.810150146484375 1 1.29 6 0 0.5 0 0 0 0.3 6.8 CRE 0 8 f yellow 24 181 1 Manhattan 1018100 I MN12 Upper West Side 3806 140 203 1 Manhattan 1020300 E MN09 Morningside Heights 3802 +1200007370 1 2015-07-01 2015-07-01 10:48:01 2015-07-01 2015-07-01 10:57:46 0 1 -73.96954345703125 40.758026123046875 -73.95506286621094 40.7656135559082 1 1.2 8 0 0.5 1.75 0 0 0.3 10.55 CSH 0 b @ yellow 15 100 1 Manhattan 1010000 I MN19 Turtle Bay-East Midtown 3808 160 124 1 Manhattan 1012400 I MN31 Lenox Hill-Roosevelt Island 3805 +1200007408 2 2015-07-01 2015-07-01 13:35:14 2015-07-01 2015-07-01 13:43:02 0 1 -73.9938735961914 40.736053466796875 -73.99730682373047 40.724735260009766 1 1 6.5 0 0.5 1 0 0 0.3 8.3 CSH 0  D yellow -122 63 1 Manhattan 1006300 I MN23 West Village 3810 189 43 1 Manhattan 1004300 I MN24 SoHo-TriBeCa-Civic Center-Little Italy 3810 +1200007462 1 2015-07-01 2015-07-01 16:39:59 2015-07-01 2015-07-01 16:45:22 0 1 -73.96697998046875 40.76973342895508 -73.95291900634766 40.772701263427734 1 1 6 1 0.5 0 0 0 0.3 7.8 CRE 0 7 c yellow -28 122 1 Manhattan 1012200 I MN40 Upper East Side-Carnegie Hill 3805 0 138 1 Manhattan 1013800 I MN32 Yorkville 3805 +1200007482 1 2015-07-01 2015-07-01 17:49:44 2015-07-01 2015-07-01 17:57:45 0 1 -73.95664978027344 40.77621078491211 -73.94552612304688 40.80124282836914 1 2.1 8.5 1 0.5 1.2 0 0 0.3 11.5 CSH 0 6 t yellow 24 140 1 Manhattan 1014000 I MN40 Upper East Side-Carnegie Hill 3805 35 184 1 Manhattan 1018400 E MN34 East Harlem North 3804 +1200007510 1 2015-07-01 2015-07-01 19:20:19 2015-07-01 2015-07-01 19:24:22 0 1 -73.99736785888672 40.7256965637207 -74.00257110595703 40.73366928100586 1 0.8 5 1 0.5 1 0 0 0.3 7.8 CSH 0  I yellow -97 55.01 1 Manhattan 1005501 I MN23 West Village 3810 163 73 1 Manhattan 1007300 I MN23 West Village 3810 +1200007532 2 2015-07-01 2015-07-01 20:03:26 2015-07-01 2015-07-01 20:14:47 0 1 -73.98169708251953 40.76839828491211 -73.96312713623047 40.76631164550781 1 1.37 9 0.5 0.5 2.06 0 0 0.3 12.36 CSH 0 B 7 yellow 66 145 1 Manhattan 1014500 I MN14 Lincoln Square 3806 17 120 1 Manhattan 1012000 I MN40 Upper East Side-Carnegie Hill 3805 +1200007537 1 2015-07-01 2015-07-01 20:16:01 2015-07-01 2015-07-01 20:26:14 0 1 -73.99237060546875 40.7591552734375 -73.97034454345703 40.79432678222656 1 2.8 10.5 0.5 0.5 3.2 0 0 0.3 15 CSH 0 H 8 yellow -117 121 1 Manhattan 1012100 E MN15 Clinton 3807 168 183 1 Manhattan 1018300 I MN12 Upper West Side 3806 +1200007563 1 2015-07-01 2015-07-01 21:29:35 2015-07-01 2015-07-01 21:53:39 0 1 -73.97285461425781 40.75322341918945 -73.84723663330078 40.724796295166016 1 8.4 26.5 0.5 0.5 6.95 0 0 0.3 34.75 CSH 0 b � yellow -30 92 1 Manhattan 1009200 I MN19 Turtle Bay-East Midtown 3808 76 713.03 4 Queens 4071303 I QN17 Forest Hills 4108 +1200007566 2 2015-07-01 2015-07-01 21:43:31 2015-07-01 2015-07-01 21:53:51 0 1 -73.97994995117188 40.746185302734375 -74.00813293457031 40.73805618286133 1 2.19 9.5 0.5 0.5 2 0 0 0.3 12.8 CSH 0 p X yellow -126 72 1 Manhattan 1007200 I MN20 Murray Hill-Kips Bay 3808 79 79 1 Manhattan 1007900 I MN23 West Village 3810 +1200007570 2 2015-07-01 2015-07-01 21:51:05 2015-07-01 2015-07-01 22:02:32 0 1 -74.00251007080078 40.72924041748047 -74.0028076171875 40.76053237915039 3 2.91 11.5 0.5 0.5 1 0 0 0.3 13.8 CSH 0 I F yellow 36 67 1 Manhattan 1006700 I MN23 West Village 3810 51 117 1 Manhattan 1011700 I MN13 Hudson Yards-Chelsea-Flatiron-Union Square 3807 +1200007572 2 2015-07-01 2015-07-01 21:52:17 2015-07-01 2015-07-01 22:08:25 0 1 -73.99556732177734 40.72484588623047 -73.98789978027344 40.77007293701172 1 4.13 15 0.5 0.5 3.26 0 0 0.3 19.56 CSH 0 D C yellow -67 43 1 Manhattan 1004300 I MN24 SoHo-TriBeCa-Civic Center-Little Italy 3810 2 147 1 Manhattan 1014700 I MN14 Lincoln Square 3806 +1200007581 2 2015-07-01 2015-07-01 22:20:27 2015-07-01 2015-07-01 22:27:17 0 1 -73.96570587158203 40.75446701049805 -73.95352172851562 40.78074645996094 5 2.26 8.5 0.5 0.5 1.5 0 0 0.3 11.3 CSH 0 ) 6 yellow 104 86.03 1 Manhattan 1008603 I MN19 Turtle Bay-East Midtown 3808 29 148.02 1 Manhattan 1014802 I MN40 Upper East Side-Carnegie Hill 3805 +1200007599 1 2015-07-01 2015-07-01 23:26:00 2015-07-01 2015-07-01 23:44:28 0 1 -73.9557876586914 40.714115142822266 -73.95341491699219 40.66645812988281 2 4.2 15.5 0.5 0.5 0 0 0 0.3 16.8 CRE 0 U b yellow 113 519 3 Brooklyn 3051900 I BK73 North Side-South Side 4001 19 321 3 Brooklyn 3032100 E BK63 Crown Heights South 4011 +1200007625 2 2015-07-02 2015-07-02 01:10:24 2015-07-02 2015-07-02 01:18:37 0 1 -73.98648834228516 40.74068832397461 -73.99443817138672 40.756526947021484 2 1.6800000000000002 8 0.5 0.5 1.86 0 0 0.3 11.16 CSH 0 4 H yellow -90 56 1 Manhattan 1005600 I MN13 Hudson Yards-Chelsea-Flatiron-Union Square 3807 163 115 1 Manhattan 1011500 I MN15 Clinton 3807 +1200007641 1 2015-07-02 2015-07-02 06:24:16 2015-07-02 2015-07-02 06:38:03 0 1 -73.98951721191406 40.77301025390625 -73.97499084472656 40.751014709472656 1 2.4 12 0 0.5 0 0 0 0.3 12.8 CRE 0 C b yellow -51 151 1 Manhattan 1015100 E MN14 Lincoln Square 3806 226 92 1 Manhattan 1009200 I MN19 Turtle Bay-East Midtown 3808 +1200007653 1 2015-07-02 2015-07-02 07:50:28 2015-07-02 2015-07-02 08:00:23 0 1 -74.00056457519531 40.732200622558594 -74.00395965576172 40.75264358520508 1 1.9 9 0 0.5 2.45 0 0 0.3 12.25 CSH 0  F yellow 37 65 1 Manhattan 1006500 I MN23 West Village 3810 50 99 1 Manhattan 1009900 I MN13 Hudson Yards-Chelsea-Flatiron-Union Square 3807 +1200007654 2 2015-07-02 2015-07-02 07:50:45 2015-07-02 2015-07-02 08:00:15 0 1 -73.96041107177734 40.7974853515625 -73.93844604492188 40.817291259765625 1 2.15 9.5 0 0.5 0 0 0 0.3 10.3 CRE 0 C B yellow 43 143 1 Manhattan 1014300 I MN99 park-cemetery-etc-Manhattan 3806 51 214 1 Manhattan 1021400 E MN03 Central Harlem North-Polo Grounds 3803 +1200007722 1 2015-07-02 2015-07-02 10:45:06 2015-07-02 2015-07-02 10:56:40 0 1 -74.01493835449219 40.714317321777344 -73.99494934082031 40.7429313659668 1 3.1 12 0 0.5 0 0 0 0.3 12.8 CRE 0  � yellow 52 317.03 1 Manhattan 1031703 I MN25 Battery Park City-Lower Manhattan 3810 143 87 1 Manhattan 1008700 I MN13 Hudson Yards-Chelsea-Flatiron-Union Square 3807 +1200007741 1 2015-07-02 2015-07-02 12:32:11 2015-07-02 2015-07-02 13:07:14 0 1 -73.97283172607422 40.78071594238281 -73.99794006347656 40.72407913208008 3 4.8 23.5 0 0.5 0 0 0 0.3 24.3 CRE 0 9  yellow 22 165 1 Manhattan 1016500 I MN12 Upper West Side 3806 35 49 1 Manhattan 1004900 I MN24 SoHo-TriBeCa-Civic Center-Little Italy 3810 +1200007746 1 2015-07-02 2015-07-02 12:55:23 2015-07-02 2015-07-02 13:08:00 0 1 -73.9853286743164 40.7445182800293 -74.00079345703125 40.74730682373047 1 0.9 9 0 0.5 1.96 0 0 0.3 11.76 CSH 0 d h yellow -125 74 1 Manhattan 1007400 I MN17 Midtown-Midtown South 3807 113 93 1 Manhattan 1009300 I MN13 Hudson Yards-Chelsea-Flatiron-Union Square 3807 +1200007766 2 2015-07-02 2015-07-02 14:31:56 2015-07-02 2015-07-02 14:38:18 0 1 -73.95555877685547 40.7825813293457 -73.96797943115234 40.7654914855957 5 1.3 7 0 0.5 0 0 0 0.3 7.8 CRE 0 6 7 yellow 74 150.02 1 Manhattan 1015002 I MN40 Upper East Side-Carnegie Hill 3805 228 122 1 Manhattan 1012200 I MN40 Upper East Side-Carnegie Hill 3805 +1200007777 2 2015-07-02 2015-07-02 15:07:55 2015-07-02 2015-07-02 15:13:41 0 1 -73.96427917480469 40.761451721191406 -73.96199035644531 40.7677001953125 1 0.55 5.5 0 0.5 0 0 0 0.3 6.3 CRE 0 A A yellow -35 110 1 Manhattan 1011000 I MN31 Lenox Hill-Roosevelt Island 3805 182 118 1 Manhattan 1011800 I MN31 Lenox Hill-Roosevelt Island 3805 +1200007799 2 2015-07-02 2015-07-02 16:17:52 2015-07-02 2015-07-02 16:50:17 0 1 -73.86377716064453 40.76997375488281 -73.95311737060547 40.77611541748047 1 8.68 29 1 0.5 0 5.54 0 0.3 36.34 CSH 0 8 c yellow 48 331 4 Queens 4033100 I QN98 Airport 4102 0 138 1 Manhattan 1013800 I MN32 Yorkville 3805 +1200007809 1 2015-07-02 2015-07-02 16:58:20 2015-07-02 2015-07-02 17:17:45 0 1 -73.98918151855469 40.74851989746094 -73.95587158203125 40.77040481567383 1 3 14 1 0.5 2 0 0 0.3 17.8 CSH 0 � A yellow -125 101 1 Manhattan 1010100 I MN17 Midtown-Midtown South 3807 218 134 1 Manhattan 1013400 I MN31 Lenox Hill-Roosevelt Island 3805 +1200007813 1 2015-07-02 2015-07-02 17:07:30 2015-07-02 2015-07-02 17:17:37 0 1 -73.95809173583984 40.776004791259766 -73.96031188964844 40.760650634765625 1 1.5 8.5 1 0.5 2.55 0 0 0.3 12.85 CSH 0 6 @ yellow 24 140 1 Manhattan 1014000 I MN40 Upper East Side-Carnegie Hill 3805 220 106.02 1 Manhattan 1010602 I MN31 Lenox Hill-Roosevelt Island 3805 +1200007817 1 2015-07-02 2015-07-02 17:12:46 2015-07-02 2015-07-02 17:20:55 0 1 -73.98286437988281 40.722774505615234 -73.98986053466797 40.73698043823242 1 1.4 7.5 1 0.5 1.85 0 0 0.3 11.15 CSH 0  4 yellow -109 26.01 1 Manhattan 1002601 E MN28 Lower East Side 3809 245 52 1 Manhattan 1005200 I MN13 Hudson Yards-Chelsea-Flatiron-Union Square 3807 +1200007836 2 2015-07-02 2015-07-02 18:07:25 2015-07-02 2015-07-02 18:24:55 0 1 -73.9915542602539 40.742740631103516 -74.01760864257812 40.70839309692383 1 3.85 15.5 1 0.5 3.46 0 0 0.3 20.76 CSH 0 4  yellow -123 58 1 Manhattan 1005800 I MN13 Hudson Yards-Chelsea-Flatiron-Union Square 3807 54 317.04 1 Manhattan 1031704 I MN25 Battery Park City-Lower Manhattan 3810 +1200007929 1 2015-07-02 2015-07-02 22:25:38 2015-07-02 2015-07-02 22:28:46 0 1 -73.98924255371094 40.76292419433594 -73.9947280883789 40.755435943603516 1 0.6 4.5 0.5 0.5 1.15 0 0 0.3 6.95 CSH 0 H h yellow -118 127 1 Manhattan 1012700 I MN15 Clinton 3807 162 111 1 Manhattan 1011100 I MN13 Hudson Yards-Chelsea-Flatiron-Union Square 3807 +1200007955 1 2015-07-02 2015-07-02 23:28:15 2015-07-02 2015-07-02 23:47:20 0 1 -73.9930648803711 40.74138641357422 -74.00785827636719 40.737548828125 1 4.3 16.5 0.5 0.5 3.55 0 0 0.3 21.35 CSH 0 4 X yellow -123 58 1 Manhattan 1005800 I MN13 Hudson Yards-Chelsea-Flatiron-Union Square 3807 79 79 1 Manhattan 1007900 I MN23 West Village 3810 +1200008001 2 2015-07-03 2015-07-03 02:40:19 2015-07-03 2015-07-03 02:57:29 0 1 -74.00286865234375 40.73345947265625 -73.95269012451172 40.72686004638672 1 5.19 18.5 0.5 0.5 0 0 0 0.3 19.8 CRE 0 I  yellow -93 73 1 Manhattan 1007300 I MN23 West Village 3810 245 561 3 Brooklyn 3056100 I BK76 Greenpoint 4001 +1200008016 1 2015-07-03 2015-07-03 04:46:33 2015-07-03 2015-07-03 04:51:45 0 1 -73.99105834960938 40.72789764404297 -73.99797058105469 40.743431091308594 1 1.5 7 0.5 0.5 0 0 0 0.3 8.3 CRE 0 y � yellow -66 38 1 Manhattan 1003800 I MN22 East Village 3809 143 87 1 Manhattan 1008700 I MN13 Hudson Yards-Chelsea-Flatiron-Union Square 3807 +1200008052 1 2015-07-03 2015-07-03 09:51:15 2015-07-03 2015-07-03 09:56:45 0 1 -73.95743560791016 40.768394470214844 -73.96751403808594 40.76321792602539 1 0.9 6 0 0.5 1.35 0 0 0.3 8.15 CSH 0 A 7 yellow -73 126 1 Manhattan 1012600 I MN31 Lenox Hill-Roosevelt Island 3805 22 114.02 1 Manhattan 1011402 I MN40 Upper East Side-Carnegie Hill 3805 +1200008065 2 2015-07-03 2015-07-03 10:40:53 2015-07-03 2015-07-03 11:01:09 0 1 -74.00102233886719 40.72589111328125 -73.96318817138672 40.7748908996582 1 4.43 16.5 0 0.5 0 0 0 0.3 17.3 CRE 0  7 yellow 35 49 1 Manhattan 1004900 I MN24 SoHo-TriBeCa-Civic Center-Little Italy 3810 23 130 1 Manhattan 1013000 I MN40 Upper East Side-Carnegie Hill 3805 +1200008095 2 2015-07-03 2015-07-03 12:54:39 2015-07-03 2015-07-03 13:08:55 0 1 -73.98137664794922 40.741310119628906 -73.99221801757812 40.74925994873047 2 0.96 10 0 0.5 1 0 0 0.3 11.8 CSH 0  � yellow 11 68 1 Manhattan 1006800 I MN21 Gramercy 3808 131 101 1 Manhattan 1010100 I MN17 Midtown-Midtown South 3807 +1200008117 2 2015-07-03 2015-07-03 14:11:34 2015-07-03 2015-07-03 14:21:31 0 1 -73.96913146972656 40.793636322021484 -73.98839569091797 40.77486801147461 1 1.87 9.5 0 0.5 2.06 0 0 0.3 12.36 CSH 0 8 C yellow 24 181 1 Manhattan 1018100 I MN12 Upper West Side 3806 205 151 1 Manhattan 1015100 E MN14 Lincoln Square 3806 +1200008130 2 2015-07-03 2015-07-03 15:27:30 2015-07-03 2015-07-03 15:46:12 0 1 -74.00418853759766 40.707496643066406 -73.9798355102539 40.68232345581055 5 3.49 15 0 0.5 0 0 0 0.3 15.8 CRE 0 � % yellow 12 15.02 1 Manhattan 1001502 I MN25 Battery Park City-Lower Manhattan 3810 82 127 3 Brooklyn 3012700 E BK38 DUMBO-Vinegar Hill-Downtown Brooklyn-Boerum Hill 4004 +1200008135 1 2015-07-03 2015-07-03 15:50:12 2015-07-03 2015-07-03 15:57:41 0 1 -73.99407958984375 40.754859924316406 -73.9866943359375 40.7453498840332 1 1.1 6.5 1 0.5 0 0 0 0.3 8.3 CRE 0 h d yellow -94 111 1 Manhattan 1011100 I MN13 Hudson Yards-Chelsea-Flatiron-Union Square 3807 131 74 1 Manhattan 1007400 I MN17 Midtown-Midtown South 3807 +1200008140 1 2015-07-03 2015-07-03 15:55:02 2015-07-03 2015-07-03 15:59:51 1 1 -73.99469757080078 40.7399787902832 -73.99868774414062 40.733978271484375 1 0.9 5.5 1 0.5 1 0 0 0.3 8.3 CSH 0 4  yellow -114 54 1 Manhattan 1005400 I MN13 Hudson Yards-Chelsea-Flatiron-Union Square 3807 134 63 1 Manhattan 1006300 I MN23 West Village 3810 +1200008209 1 2015-07-03 2015-07-03 19:01:17 2015-07-03 2015-07-03 19:05:07 0 1 -73.97836303710938 40.77800369262695 -73.98622131347656 40.778804779052734 1 0.6 4.5 0 0.5 1.55 0 0 0.3 6.85 CSH 0 B C yellow 21 157 1 Manhattan 1015700 I MN14 Lincoln Square 3806 154 159 1 Manhattan 1015900 I MN14 Lincoln Square 3806 +1200008220 1 2015-07-03 2015-07-03 19:35:49 2015-07-03 2015-07-03 19:45:00 0 1 -73.98312377929688 40.76139831542969 -74.00455474853516 40.74203872680664 1 1.8 8.5 1 0.5 0 0 0 0.3 10.3 CRE 0 0 h yellow 3 131 1 Manhattan 1013100 I MN17 Midtown-Midtown South 3807 75 83 1 Manhattan 1008300 E MN13 Hudson Yards-Chelsea-Flatiron-Union Square 3807 +1200008235 2 2015-07-03 2015-07-03 20:38:05 2015-07-03 2015-07-03 20:44:06 0 1 -73.99494934082031 40.75536346435547 -73.98418426513672 40.7750358581543 2 1.7 7 0.5 0.5 1 0 0 0.3 9.3 CSH 0 h B yellow -94 111 1 Manhattan 1011100 I MN13 Hudson Yards-Chelsea-Flatiron-Union Square 3807 30 153 1 Manhattan 1015300 I MN14 Lincoln Square 3806 +1200008247 1 2015-07-03 2015-07-03 21:33:55 2015-07-03 2015-07-03 21:43:59 0 1 -74.00592041015625 40.740150451660156 -74.00220489501953 40.72987747192383 1 0.8 7 0.5 0.5 0 0 0 0.3 8.3 CRE 0 X I yellow 79 79 1 Manhattan 1007900 I MN23 West Village 3810 36 67 1 Manhattan 1006700 I MN23 West Village 3810 +1200008259 2 2015-07-03 2015-07-03 21:59:36 2015-07-03 2015-07-03 22:03:00 0 1 -73.98870086669922 40.77421188354492 -73.99566650390625 40.76483154296875 1 0.74 5 0.5 0.5 0.7 0 0 0.3 7 CSH 0 C P yellow -51 151 1 Manhattan 1015100 E MN14 Lincoln Square 3806 140 129 1 Manhattan 1012900 I MN15 Clinton 3807 +1200008261 1 2015-07-03 2015-07-03 22:03:59 2015-07-03 2015-07-03 22:18:28 0 1 -73.99031066894531 40.761539459228516 -73.95673370361328 40.77122116088867 1 2.8 12.5 0.5 0.5 1 0 0 0.3 14.8 CSH 0 H A yellow -118 127 1 Manhattan 1012700 I MN15 Clinton 3807 218 134 1 Manhattan 1013400 I MN31 Lenox Hill-Roosevelt Island 3805 +1200008306 2 2015-07-04 2015-07-04 01:21:13 2015-07-04 2015-07-04 01:29:52 0 1 -73.98100280761719 40.74177932739258 -73.99125671386719 40.75878143310547 1 1.81 8.5 0.5 0.5 1.96 0 0 0.3 11.76 CSH 0  H yellow 11 68 1 Manhattan 1006800 I MN21 Gramercy 3808 139 121 1 Manhattan 1012100 E MN15 Clinton 3807 +1200008326 2 2015-07-04 2015-07-04 02:47:44 2015-07-04 2015-07-04 02:51:48 0 1 -73.9819107055664 40.77290725708008 -73.96826934814453 40.76125717163086 5 1.44 6 0.5 0.5 0 0 0 0.3 7.3 CRE 0 B b yellow 67 149 1 Manhattan 1014900 I MN14 Lincoln Square 3806 20 112.03 1 Manhattan 1011203 I MN19 Turtle Bay-East Midtown 3808 +1200008356 2 2015-07-04 2015-07-04 08:40:12 2015-07-04 2015-07-04 08:46:58 0 1 -73.98173522949219 40.778968811035156 -73.98127746582031 40.76189041137695 1 1.31 7 0 0.5 0 0 0 0.3 7.8 CRE 0 C 0 yellow -102 159 1 Manhattan 1015900 I MN14 Lincoln Square 3806 3 131 1 Manhattan 1013100 I MN17 Midtown-Midtown South 3807 +1200008391 2 2015-07-04 2015-07-04 12:38:58 2015-07-04 2015-07-04 12:47:30 0 1 -73.98223876953125 40.75748825073242 -73.97215270996094 40.76570129394531 1 1.47 7.5 0 0.5 0 0 0 0.3 8.3 CRE 0 0 C yellow -122 125 1 Manhattan 1012500 I MN17 Midtown-Midtown South 3807 43 143 1 Manhattan 1014300 I MN99 park-cemetery-etc-Manhattan 3806 +1200008409 2 2015-07-04 2015-07-04 13:58:57 2015-07-04 2015-07-04 14:02:48 0 1 -73.97749328613281 40.78422927856445 -73.966796875 40.79901885986328 1 1.2 5.5 0 0.5 1.26 0 0 0.3 7.56 CSH 0 9 Q yellow 22 165 1 Manhattan 1016500 I MN12 Upper West Side 3806 112 191 1 Manhattan 1019100 I MN12 Upper West Side 3806 +1200008415 1 2015-07-04 2015-07-04 14:23:05 2015-07-04 2015-07-04 14:32:21 0 1 -73.97215270996094 40.7947883605957 -73.9823226928711 40.772159576416016 1 1.6 9 0 0.5 0 0 0 0.3 9.8 CRE 0 8 B yellow -88 183 1 Manhattan 1018300 I MN12 Upper West Side 3806 67 149 1 Manhattan 1014900 I MN14 Lincoln Square 3806 +1200008443 1 2015-07-04 2015-07-04 18:20:18 2015-07-04 2015-07-04 18:26:00 0 1 -74.00305938720703 40.731563568115234 -73.99268341064453 40.72397232055664 2 0.8 5.5 0 0.5 1.25 0 0 0.3 7.55 CSH 0 I D yellow 36 67 1 Manhattan 1006700 I MN23 West Village 3810 189 43 1 Manhattan 1004300 I MN24 SoHo-TriBeCa-Civic Center-Little Italy 3810 +1200008466 2 2015-07-04 2015-07-04 19:54:23 2015-07-04 2015-07-04 20:16:41 0 1 -73.97782897949219 40.75788116455078 -74.01721954345703 40.70861053466797 1 4.85 18.5 0 0.5 0 0 0 0.3 19.3 CRE 0 a  yellow 13 96 1 Manhattan 1009600 E MN17 Midtown-Midtown South 3807 54 317.04 1 Manhattan 1031704 I MN25 Battery Park City-Lower Manhattan 3810 +1200008483 2 2015-07-04 2015-07-04 21:01:32 2015-07-04 2015-07-04 21:08:57 0 1 -73.94486999511719 40.788963317871094 -73.96826934814453 40.7919807434082 6 1.6400000000000001 8 0.5 0.5 0 0 0 0.3 9.3 CRE 0 u 8 yellow 107 164 1 Manhattan 1016400 E MN33 East Harlem South 3804 24 181 1 Manhattan 1018100 I MN12 Upper West Side 3806 +1200008484 1 2015-07-04 2015-07-04 21:02:29 2015-07-04 2015-07-04 21:06:32 0 1 -73.96756744384766 40.7601432800293 -73.9487075805664 40.77489471435547 1 1.8 7 0.5 0.5 0 0 0 0.3 8.3 CRE 0 b b yellow 20 112.03 1 Manhattan 1011203 I MN19 Turtle Bay-East Midtown 3808 152 144.01 1 Manhattan 1014401 I MN32 Yorkville 3805 +1200008498 2 2015-07-04 2015-07-04 22:41:07 2015-07-04 2015-07-04 22:52:11 0 1 -74.00228118896484 40.74000930786133 -73.98193359375 40.76966094970703 1 2.33 10.5 0.5 0.5 0 0 0 0.3 11.8 CRE 0 � B yellow -115 81 1 Manhattan 1008100 I MN13 Hudson Yards-Chelsea-Flatiron-Union Square 3807 66 145 1 Manhattan 1014500 I MN14 Lincoln Square 3806 +1200008508 2 2015-07-04 2015-07-04 23:26:08 2015-07-04 2015-07-04 23:36:15 0 1 -74.0091552734375 40.729820251464844 -74.00445556640625 40.721187591552734 1 1.03 8 0.5 0.5 0 0 0 0.3 9.3 CRE 0 X 1 yellow -107 69 1 Manhattan 1006900 I MN23 West Village 3810 15 33 1 Manhattan 1003300 I MN24 SoHo-TriBeCa-Civic Center-Little Italy 3810 +1200008515 2 2015-07-04 2015-07-04 23:47:09 2015-07-04 2015-07-04 23:50:43 0 1 -74.00660705566406 40.73181915283203 -74.00604248046875 40.739898681640625 2 0.71 4.5 0.5 0.5 0 0 0 0.3 5.8 CRE 0 X X yellow -107 69 1 Manhattan 1006900 I MN23 West Village 3810 79 79 1 Manhattan 1007900 I MN23 West Village 3810 +1200008524 2 2015-07-05 2015-07-05 00:33:10 2015-07-05 2015-07-05 00:37:49 0 1 -74.00315856933594 40.73383712768555 -74.00315856933594 40.73383712768555 1 1 5.5 0.5 0.5 0 0 0 0.3 6.8 CRE 0 I I yellow -93 73 1 Manhattan 1007300 I MN23 West Village 3810 163 73 1 Manhattan 1007300 I MN23 West Village 3810 +1200008559 1 2015-07-05 2015-07-05 03:49:48 2015-07-05 2015-07-05 04:00:15 0 1 -73.94409942626953 40.69820785522461 -73.95337677001953 40.672725677490234 1 2.8 10.5 0.5 0.5 0 0 0 0.3 11.8 CRE 0  a yellow 46 257 3 Brooklyn 3025700 E BK75 Bedford 4003 119 219 3 Brooklyn 3021900 E BK61 Crown Heights North 4006 +1200008584 2 2015-07-05 2015-07-05 09:12:15 2015-07-05 2015-07-05 09:21:42 0 1 -73.97987365722656 40.781227111816406 -73.9592514038086 40.76754379272461 2 2.19 9.5 0 0.5 0 0 0 0.3 10.3 CRE 0 9 A yellow -101 163 1 Manhattan 1016300 I MN12 Upper West Side 3806 183 126 1 Manhattan 1012600 I MN31 Lenox Hill-Roosevelt Island 3805 +1200008593 1 2015-07-05 2015-07-05 09:45:29 2015-07-05 2015-07-05 09:53:12 0 1 -73.99146270751953 40.73184585571289 -73.9794692993164 40.74973678588867 1 1.4 7.5 0 0.5 1 0 0 0.3 9.3 CSH 0  p yellow -55 61 1 Manhattan 1006100 I MN23 West Village 3810 93 82 1 Manhattan 1008200 I MN20 Murray Hill-Kips Bay 3808 +1200008598 2 2015-07-05 2015-07-05 09:59:14 2015-07-05 2015-07-05 10:07:15 0 1 -74.00328063964844 40.73167037963867 -73.98741149902344 40.72012710571289 2 1.32 7.5 0 0.5 1.66 0 0 0.3 9.96 CSH 0 I H yellow 36 67 1 Manhattan 1006700 I MN23 West Village 3810 161 30.01 1 Manhattan 1003001 E MN27 Chinatown 3809 +1200008641 2 2015-07-05 2015-07-05 13:59:42 2015-07-05 2015-07-05 14:12:02 0 1 -73.99214935302734 40.74909973144531 -73.96640014648438 40.763118743896484 1 2.4 10.5 0 0.5 2.26 0 0 0.3 13.56 CSH 0 � 7 yellow -125 101 1 Manhattan 1010100 I MN17 Midtown-Midtown South 3807 22 114.02 1 Manhattan 1011402 I MN40 Upper East Side-Carnegie Hill 3805 +1200008643 1 2015-07-05 2015-07-05 14:06:05 2015-07-05 2015-07-05 14:37:04 0 1 -73.99371337890625 40.74991226196289 -73.90554809570312 40.767818450927734 1 6.6 25 0 0.5 0 0 0 0.3 25.8 CRE 0 �  yellow -125 101 1 Manhattan 1010100 I MN17 Midtown-Midtown South 3807 233 141 4 Queens 4014100 E QN70 Astoria 4101 +1200008650 1 2015-07-05 2015-07-05 14:36:37 2015-07-05 2015-07-05 14:53:18 0 1 -73.97502899169922 40.74997329711914 -74.00546264648438 40.725589752197266 1 3.2 13.5 0 0.5 0 0 0 0.3 14.3 CRE 0 p % yellow -123 80 1 Manhattan 1008000 I MN20 Murray Hill-Kips Bay 3808 13 37 1 Manhattan 1003700 I MN24 SoHo-TriBeCa-Civic Center-Little Italy 3810 +1200008658 1 2015-07-05 2015-07-05 15:17:44 2015-07-05 2015-07-05 15:21:11 0 1 -73.97590637207031 40.74449920654297 -73.97109985351562 40.75559997558594 1 1 5 0 0.5 1.3 0 0 0.3 7.1 CSH 0 7 b yellow -10 70 1 Manhattan 1007000 I MN20 Murray Hill-Kips Bay 3808 15 100 1 Manhattan 1010000 I MN19 Turtle Bay-East Midtown 3808 +1200008666 2 2015-07-05 2015-07-05 15:46:21 2015-07-05 2015-07-05 15:58:08 0 1 -74.00679779052734 40.73023986816406 -73.99067687988281 40.736595153808594 2 1.8 9.5 0 0.5 1 0 0 0.3 11.3 CSH 0 I 4 yellow 36 67 1 Manhattan 1006700 I MN23 West Village 3810 245 52 1 Manhattan 1005200 I MN13 Hudson Yards-Chelsea-Flatiron-Union Square 3807 +1200008685 1 2015-07-05 2015-07-05 17:38:46 2015-07-05 2015-07-05 17:57:50 0 1 -73.98519134521484 40.7596549987793 -74.00923919677734 40.726566314697266 1 3.3 15 0 0.5 3.15 0 0 0.3 18.95 CSH 0 0 % yellow -122 125 1 Manhattan 1012500 I MN17 Midtown-Midtown South 3807 13 37 1 Manhattan 1003700 I MN24 SoHo-TriBeCa-Civic Center-Little Italy 3810 +1200008698 1 2015-07-05 2015-07-05 18:25:02 2015-07-05 2015-07-05 19:02:58 0 2 -73.99437713623047 40.76422119140625 -73.7831039428711 40.64385986328125 1 17.9 52 0 0.5 11.65 5.54 0 0.3 69.99 CSH 0 P 2 yellow -116 129 1 Manhattan 1012900 I MN15 Clinton 3807 8 716 4 Queens 4071600 I QN98 Airport 4105 +1200008716 2 2015-07-05 2015-07-05 20:00:51 2015-07-05 2015-07-05 20:11:37 0 1 -73.99126434326172 40.74981689453125 -73.95987701416016 40.7659912109375 2 2.7 10.5 0.5 0.5 2.36 0 0 0.3 14.16 CSH 0 � A yellow -125 101 1 Manhattan 1010100 I MN17 Midtown-Midtown South 3807 182 118 1 Manhattan 1011800 I MN31 Lenox Hill-Roosevelt Island 3805 +1200008765 2 2015-07-05 2015-07-05 23:00:35 2015-07-05 2015-07-05 23:03:36 0 1 -73.96129608154297 40.7700309753418 -73.96123504638672 40.765098571777344 1 0.47 4 0.5 0.5 3 0 0 0.3 8.3 CSH 0 7 A yellow 18 128 1 Manhattan 1012800 I MN40 Upper East Side-Carnegie Hill 3805 182 118 1 Manhattan 1011800 I MN31 Lenox Hill-Roosevelt Island 3805 +1200008773 2 2015-07-05 2015-07-05 23:47:05 2015-07-06 2015-07-06 00:09:50 0 1 -73.80579376220703 40.66035842895508 -73.91576385498047 40.731170654296875 2 15.62 43 0.5 0.5 6 0 0 0.3 50.3 CSH 0 2 ` yellow 8 716 4 Queens 4071600 I QN98 Airport 4105 30 229 4 Queens 4022900 I QN99 park-cemetery-etc-Queens 4109 +1200008794 1 2015-07-08 2015-07-08 16:10:47 2015-07-08 2015-07-08 16:24:47 0 1 -73.95206451416016 40.784446716308594 -73.97003936767578 40.79954528808594 1 1.8 10.5 0 0.5 0 0 0 0.3 11.3 CRE 0 6 Q yellow -103 158.01 1 Manhattan 1015801 I MN40 Upper East Side-Carnegie Hill 3805 112 191 1 Manhattan 1019100 I MN12 Upper West Side 3806 +1200008830 1 2015-07-08 2015-07-08 18:10:35 2015-07-08 2015-07-08 18:36:35 0 1 -73.94889068603516 40.7772216796875 -73.99473571777344 40.76017761230469 1 3.7 18.5 1 0.5 2 0 0 0.3 22.3 CSH 0 b H yellow 0 144.02 1 Manhattan 1014402 I MN32 Yorkville 3805 139 121 1 Manhattan 1012100 E MN15 Clinton 3807 +1200008843 1 2015-07-08 2015-07-08 18:32:53 2015-07-08 2015-07-08 18:39:10 0 1 -73.98208618164062 40.7401237487793 -73.9759521484375 40.74885559082031 1 0.9 6 1 0.5 0 0 0 0.3 7.8 CRE 0 7 p yellow -11 66 1 Manhattan 1006600 I MN20 Murray Hill-Kips Bay 3808 133 80 1 Manhattan 1008000 I MN20 Murray Hill-Kips Bay 3808 +1200008853 1 2015-07-08 2015-07-08 19:00:12 2015-07-08 2015-07-08 19:06:54 0 1 -73.99121856689453 40.7447395324707 -73.9990234375 40.73431396484375 1 1.2 6.5 1 0.5 0 0 0 0.3 8.3 CRE 0 4  yellow -123 58 1 Manhattan 1005800 I MN13 Hudson Yards-Chelsea-Flatiron-Union Square 3807 134 63 1 Manhattan 1006300 I MN23 West Village 3810 +1200008866 2 2015-07-08 2015-07-08 19:23:55 2015-07-08 2015-07-08 19:36:39 0 1 -73.99304962158203 40.734195709228516 -73.98180389404297 40.745933532714844 1 1.31 9.5 1 0.5 0 0 0 0.3 11.3 CRE 0  p yellow -55 61 1 Manhattan 1006100 I MN23 West Village 3810 130 72 1 Manhattan 1007200 I MN20 Murray Hill-Kips Bay 3808 +1200008876 1 2015-07-08 2015-07-08 19:54:09 2015-07-08 2015-07-08 20:13:36 0 1 -73.98316192626953 40.74439239501953 -73.99555206298828 40.75944900512695 1 1.6 12.5 0.5 0.5 1 0 0 0.3 14.8 CSH 0 p H yellow -126 72 1 Manhattan 1007200 I MN20 Murray Hill-Kips Bay 3808 163 115 1 Manhattan 1011500 I MN15 Clinton 3807 +1200008895 1 2015-07-08 2015-07-08 20:41:50 2015-07-08 2015-07-08 21:01:37 0 1 -73.9933090209961 40.74213409423828 -73.97281646728516 40.7930793762207 1 4.1 16.5 0.5 0.5 3.55 0 0 0.3 21.35 CSH 0 4 8 yellow -123 58 1 Manhattan 1005800 I MN13 Hudson Yards-Chelsea-Flatiron-Union Square 3807 167 179 1 Manhattan 1017900 I MN12 Upper West Side 3806 +1200008916 2 2015-07-08 2015-07-08 21:32:17 2015-07-08 2015-07-08 21:34:39 0 1 -73.95814514160156 40.76496887207031 -73.95218658447266 40.77302169799805 1 0.7 4 0.5 0.5 1.06 0 0 0.3 6.36 CSH 0 A c yellow -74 118 1 Manhattan 1011800 I MN31 Lenox Hill-Roosevelt Island 3805 0 138 1 Manhattan 1013800 I MN32 Yorkville 3805 +1200008926 2 2015-07-08 2015-07-08 21:50:20 2015-07-08 2015-07-08 21:57:26 0 1 -73.982421875 40.7740592956543 -73.9754867553711 40.792083740234375 1 1.52 7 0.5 0.5 1.2 0 0 0.3 9.5 CSH 0 B 8 yellow 30 153 1 Manhattan 1015300 I MN14 Lincoln Square 3806 167 179 1 Manhattan 1017900 I MN12 Upper West Side 3806 +1200008933 2 2015-07-08 2015-07-08 22:17:52 2015-07-08 2015-07-08 22:32:30 0 1 -73.94114685058594 40.807926177978516 -73.86376190185547 40.8213005065918 1 5.17 17 0.5 0.5 0 0 0 0.3 18.3 CRE 0 B  yellow 39 206 1 Manhattan 1020600 E MN03 Central Harlem North-Polo Grounds 3803 1 42 2 Bronx 2004200 E BX09 Soundview-Castle Hill-Clason Point-Harding Park 3709 +1200008939 1 2015-07-08 2015-07-08 22:25:52 2015-07-08 2015-07-08 22:27:41 0 1 -73.99847412109375 40.745391845703125 -73.99471282958984 40.75038528442383 2 0.3 3.5 0.5 0.5 0 0 0 0.3 4.8 CRE 0 h � yellow 113 93 1 Manhattan 1009300 I MN13 Hudson Yards-Chelsea-Flatiron-Union Square 3807 131 101 1 Manhattan 1010100 I MN17 Midtown-Midtown South 3807 +1200008956 2 2015-07-08 2015-07-08 23:43:09 2015-07-09 2015-07-09 00:05:05 0 1 -73.98976135253906 40.739471435546875 -73.96318817138672 40.68177032470703 6 5.29 19.5 0.5 0.5 5.2 0 0 0.3 26 CSH 0 4 I yellow -11 52 1 Manhattan 1005200 I MN13 Hudson Yards-Chelsea-Flatiron-Union Square 3807 111 201 3 Brooklyn 3020100 I BK69 Clinton Hill 4004 +1200008973 2 2015-07-09 2015-07-09 00:20:46 2015-07-09 2015-07-09 00:31:05 0 1 -73.95056915283203 40.77125930786133 -73.9914321899414 40.72760009765625 1 4.32 14 0.5 0.5 0 0 0 0.3 15.3 CRE 0 @  yellow -95 132 1 Manhattan 1013200 I MN31 Lenox Hill-Roosevelt Island 3805 192 57 1 Manhattan 1005700 I MN23 West Village 3810 +1200008999 2 2015-07-09 2015-07-09 05:40:01 2015-07-09 2015-07-09 05:46:22 0 1 -73.9262924194336 40.77531814575195 -73.871337890625 40.77418899536133 2 3.42 11.5 0.5 0.5 2.56 0 0 0.3 15.36 CSH 0 y 8 yellow 44 91 4 Queens 4009100 E QN71 Old Astoria 4101 48 331 4 Queens 4033100 I QN98 Airport 4102 +1200009004 2 2015-07-09 2015-07-09 06:39:03 2015-07-09 2015-07-09 07:03:06 0 1 -73.95366668701172 40.82219314575195 -73.86448669433594 40.770111083984375 1 8.62 27.5 0 0.5 8.46 5.54 0 0.3 42.3 CSH 0  8 yellow -117 225 1 Manhattan 1022500 E MN04 Hamilton Heights 3802 48 331 4 Queens 4033100 I QN98 Airport 4102 +1200009010 2 2015-07-09 2015-07-09 07:03:43 2015-07-09 2015-07-09 07:26:19 0 1 -73.97177124023438 40.79451370239258 -73.98777770996094 40.74330520629883 1 4.52 17.5 0 0.5 5.49 0 0 0.3 23.79 CSH 0 8 4 yellow -88 183 1 Manhattan 1018300 I MN12 Upper West Side 3806 166 56 1 Manhattan 1005600 I MN13 Hudson Yards-Chelsea-Flatiron-Union Square 3807 +1200009013 1 2015-07-09 2015-07-09 07:25:52 2015-07-09 2015-07-09 07:55:15 0 1 -73.98300170898438 40.76461410522461 -74.01431274414062 40.70383834838867 2 5.6 23.5 0 0.5 0 0 0 0.3 24.3 CRE 0 c � yellow 4 137 1 Manhattan 1013700 I MN17 Midtown-Midtown South 3807 0 9 1 Manhattan 1000900 I MN25 Battery Park City-Lower Manhattan 3810 +1200009020 2 2015-07-09 2015-07-09 07:40:32 2015-07-09 2015-07-09 07:43:27 0 1 -73.96460723876953 40.79191589355469 -73.9551010131836 40.787628173828125 1 0.66 4.5 0 0.5 1.32 0 0 0.3 6.62 CSH 0 8 u yellow 24 181 1 Manhattan 1018100 I MN12 Upper West Side 3806 72 160.02 1 Manhattan 1016002 I MN33 East Harlem South 3804 +1200009041 1 2015-07-09 2015-07-09 08:39:56 2015-07-09 2015-07-09 08:45:24 0 1 -73.978271484375 40.754722595214844 -73.98178100585938 40.75810623168945 1 0.4 5 0 0.5 1.15 0 0 0.3 6.95 CSH 0 a 0 yellow -31 94 1 Manhattan 1009400 E MN17 Midtown-Midtown South 3807 134 125 1 Manhattan 1012500 I MN17 Midtown-Midtown South 3807 +1200009076 2 2015-07-09 2015-07-09 10:40:32 2015-07-09 2015-07-09 10:56:09 0 1 -73.98145294189453 40.7839241027832 -73.95840454101562 40.781463623046875 1 1.7 11.5 0 0.5 1.7 0 0 0.3 14 CSH 0 9 6 yellow -103 167 1 Manhattan 1016700 I MN12 Upper West Side 3806 74 150.02 1 Manhattan 1015002 I MN40 Upper East Side-Carnegie Hill 3805 +1200009087 2 2015-07-09 2015-07-09 11:29:30 2015-07-09 2015-07-09 11:36:48 0 1 -73.95344543457031 40.77266311645508 -73.95394897460938 40.78462600708008 5 1.22 6.5 0 0.5 0 0 0 0.3 7.3 CRE 0 c 6 yellow 0 138 1 Manhattan 1013800 I MN32 Yorkville 3805 153 158.01 1 Manhattan 1015801 I MN40 Upper East Side-Carnegie Hill 3805 +1200009110 1 2015-07-09 2015-07-09 12:37:45 2015-07-09 2015-07-09 12:55:10 0 1 -73.98773193359375 40.76510238647461 -73.97447967529297 40.75236511230469 1 1.3 11.5 0 0.5 1 0 0 0.3 13.3 CSH 0 H b yellow -119 133 1 Manhattan 1013300 I MN15 Clinton 3807 226 92 1 Manhattan 1009200 I MN19 Turtle Bay-East Midtown 3808 +1200009114 2 2015-07-09 2015-07-09 12:53:54 2015-07-09 2015-07-09 12:59:31 0 1 -73.9622802734375 40.76219177246094 -73.95468139648438 40.7696418762207 1 0.72 5.5 0 0.5 0 0 0 0.3 6.3 CRE 0 A A yellow -35 110 1 Manhattan 1011000 I MN31 Lenox Hill-Roosevelt Island 3805 218 134 1 Manhattan 1013400 I MN31 Lenox Hill-Roosevelt Island 3805 +1200009116 1 2015-07-09 2015-07-09 12:56:03 2015-07-09 2015-07-09 13:06:03 0 1 -74.01443481445312 40.70267868041992 -74.0103530883789 40.71875762939453 1 1.4 8 0 0.5 0 0 0 0.3 8.8 CRE 0  1 yellow 44 319 1 Manhattan 1031900 I MN99 park-cemetery-etc-Manhattan 3810 14 39 1 Manhattan 1003900 I MN24 SoHo-TriBeCa-Civic Center-Little Italy 3810 +1200009144 1 2015-07-09 2015-07-09 14:16:52 2015-07-09 2015-07-09 14:25:23 0 1 -73.9699935913086 40.76326370239258 -73.96038055419922 40.767093658447266 1 0.9 7 0 0.5 1.55 0 0 0.3 9.35 CSH 0 7 A yellow -29 114.01 1 Manhattan 1011401 I MN40 Upper East Side-Carnegie Hill 3805 182 118 1 Manhattan 1011800 I MN31 Lenox Hill-Roosevelt Island 3805 +1200009172 2 2015-07-09 2015-07-09 15:54:34 2015-07-09 2015-07-09 16:16:09 0 1 -74.02074432373047 40.634456634521484 -74.02074432373047 40.634456634521484 1 1.71 14 0 0.5 3.7 0 0 0.3 18.5 CSH 0   yellow 44 126 3 Brooklyn 3012600 E BK31 Bay Ridge 4013 44 126 3 Brooklyn 3012600 E BK31 Bay Ridge 4013 +1200009173 1 2015-07-09 2015-07-09 15:55:42 2015-07-09 2015-07-09 16:13:14 0 1 -74.00880432128906 40.71754837036133 -73.98965454101562 40.7262077331543 1 1.8 12.5 0 0.5 2.65 0 0 0.3 15.95 CSH 0 1 y yellow 15 33 1 Manhattan 1003300 I MN24 SoHo-TriBeCa-Civic Center-Little Italy 3810 190 38 1 Manhattan 1003800 I MN22 East Village 3809 +1200009211 1 2015-07-09 2015-07-09 17:57:38 2015-07-09 2015-07-09 18:29:02 0 1 -73.99899291992188 40.724830627441406 -73.98696899414062 40.761173248291016 1 2.9 20 1 0.5 3 0 0 0.3 24.8 CSH 0  H yellow 35 49 1 Manhattan 1004900 I MN24 SoHo-TriBeCa-Civic Center-Little Italy 3810 138 127 1 Manhattan 1012700 I MN15 Clinton 3807 +1200009237 1 2015-07-09 2015-07-09 18:36:35 2015-07-09 2015-07-09 18:53:17 0 1 -73.87089538574219 40.77369689941406 -73.9179458618164 40.763084411621094 1 3.3 14 1 0.5 4.7 0 0 0.3 20.5 CSH 0 8  yellow 48 331 4 Queens 4033100 I QN98 Airport 4102 105 63 4 Queens 4006300 E QN70 Astoria 4101 +1200009280 1 2015-07-09 2015-07-09 20:09:23 2015-07-09 2015-07-09 20:32:00 0 1 -73.96773529052734 40.67292404174805 -73.94111633300781 40.71257019042969 1 4.9 20 0.5 0.5 3.5 0 0 0.3 24.8 CSH 0 � � yellow -45 177 3 Brooklyn 3017700 I BK99 park-cemetery-etc-Brooklyn 4005 239 495 3 Brooklyn 3049500 I BK90 East Williamsburg 4001 +1200009296 2 2015-07-09 2015-07-09 20:42:22 2015-07-09 2015-07-09 20:47:19 0 1 -73.95503997802734 40.7661018371582 -73.95577239990234 40.771995544433594 1 0.64 5 0.5 0.5 0 0 0 0.3 6.3 CRE 0 @ A yellow -96 124 1 Manhattan 1012400 I MN31 Lenox Hill-Roosevelt Island 3805 218 134 1 Manhattan 1013400 I MN31 Lenox Hill-Roosevelt Island 3805 +1200009303 2 2015-07-09 2015-07-09 20:53:52 2015-07-09 2015-07-09 21:05:51 0 1 -73.99276733398438 40.75675964355469 -74.00424194335938 40.742210388183594 5 1.8 9.5 0.5 0.5 0 0 0 0.3 10.8 CRE 0 H h yellow -93 115 1 Manhattan 1011500 I MN15 Clinton 3807 75 83 1 Manhattan 1008300 E MN13 Hudson Yards-Chelsea-Flatiron-Union Square 3807 +1200009310 1 2015-07-09 2015-07-09 21:13:28 2015-07-09 2015-07-09 21:27:07 0 1 -73.98381042480469 40.742374420166016 -73.98092651367188 40.763614654541016 4 2 10.5 0.5 0.5 0 0 0 0.3 11.8 CRE 0  c yellow 11 68 1 Manhattan 1006800 I MN21 Gramercy 3808 4 137 1 Manhattan 1013700 I MN17 Midtown-Midtown South 3807 +1200009339 2 2015-07-09 2015-07-09 22:15:58 2015-07-09 2015-07-09 22:33:30 0 1 -73.98831939697266 40.763084411621094 -73.98563385009766 40.74973678588867 1 1.51 12 0.5 0.5 0 0 0 0.3 13.3 CRE 0 H d yellow -118 127 1 Manhattan 1012700 I MN15 Clinton 3807 208 76 1 Manhattan 1007600 I MN17 Midtown-Midtown South 3807 +1200009357 1 2015-07-09 2015-07-09 23:07:02 2015-07-09 2015-07-09 23:13:00 0 1 -73.98788452148438 40.75471496582031 -73.98284912109375 40.73917770385742 1 1.5 7 0.5 0.5 1.5 0 0 0.3 9.8 CSH 0   yellow 21 113 1 Manhattan 1011300 I MN17 Midtown-Midtown South 3807 154 64 1 Manhattan 1006400 I MN21 Gramercy 3808 +1200009406 2 2015-07-10 2015-07-10 02:23:13 2015-07-10 2015-07-10 02:33:20 0 1 -73.9908218383789 40.73638916015625 -73.98351287841797 40.75118637084961 1 1.65 9 0.5 0.5 1 0 0 0.3 11.3 CSH 0 4 d yellow -11 52 1 Manhattan 1005200 I MN13 Hudson Yards-Chelsea-Flatiron-Union Square 3807 209 84 1 Manhattan 1008400 I MN17 Midtown-Midtown South 3807 +1200009407 1 2015-07-10 2015-07-10 02:43:36 2015-07-10 2015-07-10 02:50:10 0 1 -73.97883605957031 40.750492095947266 -73.95401000976562 40.775360107421875 1 2.3 9 0.5 0.5 1.5 0 0 0.3 11.8 CSH 0 p c yellow -123 80 1 Manhattan 1008000 I MN20 Murray Hill-Kips Bay 3808 0 138 1 Manhattan 1013800 I MN32 Yorkville 3805 +1200009414 1 2015-07-10 2015-07-10 04:08:08 2015-07-10 2015-07-10 04:22:04 0 1 -73.91725158691406 40.69886016845703 -73.9596939086914 40.68689727783203 1 3.1 13 0.5 0.5 1 0 0 0.3 15.3 CSH 0 6 I yellow -42 433 3 Brooklyn 3043300 E BK77 Bushwick North 4002 128 231 3 Brooklyn 3023100 I BK69 Clinton Hill 4004 +1200009434 1 2015-07-10 2015-07-10 06:54:32 2015-07-10 2015-07-10 06:56:36 0 1 -73.97644805908203 40.75593948364258 -73.97319030761719 40.762840270996094 1 0.5 4 0 0.5 0.96 0 0 0.3 5.76 CSH 0 a c yellow -31 94 1 Manhattan 1009400 E MN17 Midtown-Midtown South 3807 19 112.02 1 Manhattan 1011202 I MN17 Midtown-Midtown South 3807 +1200009441 1 2015-07-10 2015-07-10 06:41:55 2015-07-10 2015-07-10 06:56:57 0 1 -74.0082015991211 40.707786560058594 -73.98695373535156 40.76125717163086 1 6.2 19.5 0 0.5 4.05 0 0 0.3 24.35 CSH 0 � H yellow 12 15.02 1 Manhattan 1001502 I MN25 Battery Park City-Lower Manhattan 3810 138 127 1 Manhattan 1012700 I MN15 Clinton 3807 +1200009454 1 2015-07-10 2015-07-10 07:51:55 2015-07-10 2015-07-10 08:04:10 0 1 -73.9896011352539 40.774513244628906 -73.97552490234375 40.76398468017578 1 1.4 9.5 0 0.5 0 0 0 0.3 10.3 CRE 0 C c yellow -51 151 1 Manhattan 1015100 E MN14 Lincoln Square 3806 18 112.01 1 Manhattan 1011201 I MN17 Midtown-Midtown South 3807 +1200009482 1 2015-07-10 2015-07-10 09:30:08 2015-07-10 2015-07-10 09:45:22 0 1 -73.98802947998047 40.72377014160156 -73.97818756103516 40.7501335144043 1 2.3 12 0 0.5 1 0 0 0.3 13.8 CSH 0 y p yellow -94 30.02 1 Manhattan 1003002 E MN22 East Village 3809 133 80 1 Manhattan 1008000 I MN20 Murray Hill-Kips Bay 3808 +1200009524 1 2015-07-10 2015-07-10 11:55:38 2015-07-10 2015-07-10 12:16:38 0 1 -74.00782012939453 40.70342254638672 -73.97527313232422 40.75495147705078 1 5.2 19 0 0.5 3 0 0 0.3 22.8 CSH 0 � b yellow -1 7 1 Manhattan 1000700 I MN25 Battery Park City-Lower Manhattan 3810 226 92 1 Manhattan 1009200 I MN19 Turtle Bay-East Midtown 3808 +1200009565 1 2015-07-10 2015-07-10 14:18:22 2015-07-10 2015-07-10 14:36:29 0 1 -73.98676300048828 40.740074157714844 -73.9640884399414 40.760650634765625 1 2.2 13 0 0.5 2.5 0 0 0.3 16.3 CSH 0 4 A yellow -90 56 1 Manhattan 1005600 I MN13 Hudson Yards-Chelsea-Flatiron-Union Square 3807 221 110 1 Manhattan 1011000 I MN31 Lenox Hill-Roosevelt Island 3805 +1200009581 1 2015-07-10 2015-07-10 11:09:00 2015-07-10 2015-07-10 11:19:52 0 1 -73.97615814208984 40.78049850463867 -73.9710693359375 40.757972717285156 1 2 9.5 0 0.5 2.55 0 0 0.3 12.85 CSH 0 9 b yellow 63 161 1 Manhattan 1016100 I MN12 Upper West Side 3806 15 100 1 Manhattan 1010000 I MN19 Turtle Bay-East Midtown 3808 +1200009584 1 2015-07-10 2015-07-10 14:55:40 2015-07-10 2015-07-10 15:01:25 0 1 -73.97940826416016 40.752315521240234 -73.98542022705078 40.75969696044922 2 0.8 5.5 0 0.5 0 0 0 0.3 6.3 CRE 0 p 0 yellow 93 82 1 Manhattan 1008200 I MN20 Murray Hill-Kips Bay 3808 134 125 1 Manhattan 1012500 I MN17 Midtown-Midtown South 3807 +1200009591 1 2015-07-10 2015-07-10 15:37:31 2015-07-10 2015-07-10 15:41:40 0 1 -73.95571899414062 40.77912521362305 -73.96280670166016 40.768978118896484 1 0.8 5 0 0.5 1.15 0 0 0.3 6.95 CSH 0 6 7 yellow 28 148.01 1 Manhattan 1014801 I MN40 Upper East Side-Carnegie Hill 3805 18 128 1 Manhattan 1012800 I MN40 Upper East Side-Carnegie Hill 3805 +1200009609 2 2015-07-10 2015-07-10 16:29:18 2015-07-10 2015-07-10 17:04:10 0 1 -73.99871826171875 40.719581604003906 -73.9881820678711 40.759490966796875 3 3.54 22.5 1 0.5 0 0 0 0.3 24.3 CRE 0 D H yellow 124 45 1 Manhattan 1004500 I MN24 SoHo-TriBeCa-Civic Center-Little Italy 3810 139 121 1 Manhattan 1012100 E MN15 Clinton 3807 +1200009618 1 2015-07-10 2015-07-10 17:19:11 2015-07-10 2015-07-10 17:32:13 0 1 -73.99217987060547 40.731964111328125 -73.9817123413086 40.746341705322266 1 1.2 9.5 1 0.5 2.25 0 0 0.3 13.55 CSH 0  p yellow -93 59 1 Manhattan 1005900 I MN23 West Village 3810 130 72 1 Manhattan 1007200 I MN20 Murray Hill-Kips Bay 3808 +1200009634 1 2015-07-12 2015-07-12 23:53:27 2015-07-12 2015-07-12 23:56:30 0 1 -73.97350311279297 40.74784851074219 -73.98282623291016 40.735321044921875 1 0.9 5 0.5 0.5 1 0 0 0.3 7.3 CSH 0 p  yellow -96 78 1 Manhattan 1007800 I MN20 Murray Hill-Kips Bay 3808 52 48 1 Manhattan 1004800 I MN21 Gramercy 3808 +1200009654 2 2015-07-10 2015-07-10 18:18:30 2015-07-10 2015-07-10 18:20:31 0 1 -73.95606994628906 40.77476501464844 -73.95174407958984 40.77371597290039 1 0.29 3.5 1 0.5 1.06 0 0 0.3 6.36 CSH 0 c c yellow 0 138 1 Manhattan 1013800 I MN32 Yorkville 3805 0 138 1 Manhattan 1013800 I MN32 Yorkville 3805 +1200009659 1 2015-07-10 2015-07-10 18:42:17 2015-07-10 2015-07-10 18:44:04 0 1 -73.95977783203125 40.771602630615234 -73.9652099609375 40.77411651611328 1 0.4 3.5 1 0.5 0 0 0 0.3 5.3 CRE 0 7 7 yellow 18 128 1 Manhattan 1012800 I MN40 Upper East Side-Carnegie Hill 3805 23 130 1 Manhattan 1013000 I MN40 Upper East Side-Carnegie Hill 3805 +1200009705 2 2015-07-10 2015-07-10 20:39:00 2015-07-10 2015-07-10 20:44:18 0 1 -73.92694091796875 40.811492919921875 -73.91755676269531 40.80685043334961 2 0.81 5 0.5 0.5 0 0 0 0.3 6.3 CRE 0 h h yellow -114 51 2 Bronx 2005100 E BX39 Mott Haven-Port Morris 3710 168 25 2 Bronx 2002500 E BX39 Mott Haven-Port Morris 3710 +1200009708 1 2015-07-10 2015-07-10 20:47:30 2015-07-10 2015-07-10 20:59:57 0 1 -73.97557830810547 40.76046371459961 -74.0002670288086 40.74140548706055 1 2.3 10 0.5 0.5 2.8 0 0 0.3 14.1 CSH 0 a � yellow 16 102 1 Manhattan 1010200 I MN17 Midtown-Midtown South 3807 141 81 1 Manhattan 1008100 I MN13 Hudson Yards-Chelsea-Flatiron-Union Square 3807 +1200009712 2 2015-07-10 2015-07-10 20:57:15 2015-07-10 2015-07-10 21:10:28 0 1 -73.98423767089844 40.74359893798828 -73.98603057861328 40.76240158081055 1 1.79 10 0.5 0.5 2.26 0 0 0.3 13.56 CSH 0 d H yellow -125 74 1 Manhattan 1007400 I MN17 Midtown-Midtown South 3807 137 133 1 Manhattan 1013300 I MN15 Clinton 3807 +1200009771 2 2015-07-10 2015-07-10 23:51:55 2015-07-11 2015-07-11 00:01:10 0 1 -73.99433898925781 40.71939468383789 -73.98370361328125 40.71601104736328 1 0.86 7.5 0.5 0.5 0 0 0 0.3 8.8 CRE 0 H 2 yellow -61 18 1 Manhattan 1001800 E MN27 Chinatown 3809 100 12 1 Manhattan 1001200 E MN28 Lower East Side 3809 +1200009801 1 2015-07-11 2015-07-11 01:15:30 2015-07-11 2015-07-11 01:21:13 0 1 -73.9893798828125 40.72093963623047 -73.97919464111328 40.72994613647461 1 1 6 0.5 0.5 0 0 0 0.3 7.3 CRE 0 H y yellow -95 30.01 1 Manhattan 1003001 E MN27 Chinatown 3809 98 34 1 Manhattan 1003400 I MN22 East Village 3809 +1200009859 2 2015-07-11 2015-07-11 04:39:46 2015-07-11 2015-07-11 05:09:44 0 1 -74.0039291381836 40.74317169189453 -73.94799041748047 40.705379486083984 5 7.45 27.5 0.5 0.5 0 0 0 0.3 28.8 CRE 0 h � yellow 75 83 1 Manhattan 1008300 E MN13 Hudson Yards-Chelsea-Flatiron-Union Square 3807 169 491 3 Brooklyn 3049100 E BK90 East Williamsburg 4001 +1200009871 2 2015-07-13 2015-07-13 01:24:29 2015-07-13 2015-07-13 01:35:09 0 1 -74.00154876708984 40.73067092895508 -73.9932632446289 40.76277160644531 2 2.85 11 0.5 0.5 0 0 0 0.3 12.3 CRE 0  P yellow 37 65 1 Manhattan 1006500 I MN23 West Village 3810 140 129 1 Manhattan 1012900 I MN15 Clinton 3807 +1200009887 2 2015-07-11 2015-07-11 07:28:34 2015-07-11 2015-07-11 07:42:30 0 1 -74.01284790039062 40.702030181884766 -73.97297668457031 40.75830841064453 6 6.27 19.5 0 0.5 0 0 0 0.3 20.3 CRE 0 � b yellow 0 9 1 Manhattan 1000900 I MN25 Battery Park City-Lower Manhattan 3810 15 100 1 Manhattan 1010000 I MN19 Turtle Bay-East Midtown 3808 +1200009909 2 2015-07-11 2015-07-11 09:23:20 2015-07-11 2015-07-11 09:31:51 0 1 -73.99099731445312 40.75564956665039 -73.9745101928711 40.751041412353516 6 0.98 7.5 0 0.5 0 0 0 0.3 8.3 CRE 0 H b yellow -93 115 1 Manhattan 1011500 I MN15 Clinton 3807 226 92 1 Manhattan 1009200 I MN19 Turtle Bay-East Midtown 3808 +1200009910 2 2015-07-11 2015-07-11 09:29:01 2015-07-11 2015-07-11 09:53:41 0 1 -73.96231842041016 40.779083251953125 -74.00577545166016 40.71702575683594 1 6.88 24 0 0.5 0 0 0 0.3 24.8 CRE 0 C 1 yellow 43 143 1 Manhattan 1014300 I MN99 park-cemetery-etc-Manhattan 3806 15 33 1 Manhattan 1003300 I MN24 SoHo-TriBeCa-Civic Center-Little Italy 3810 +1200009924 2 2015-07-11 2015-07-11 10:23:47 2015-07-11 2015-07-11 10:39:42 0 1 -73.97344970703125 40.74867248535156 -73.99844360351562 40.72359085083008 1 2.45 12.5 0 0.5 1 0 0 0.3 14.3 CSH 0 3  yellow -105 88 1 Manhattan 1008800 I MN19 Turtle Bay-East Midtown 3808 35 49 1 Manhattan 1004900 I MN24 SoHo-TriBeCa-Civic Center-Little Italy 3810 +1200009929 1 2015-07-11 2015-07-11 10:35:55 2015-07-11 2015-07-11 10:48:01 0 1 -73.9866943359375 40.721397399902344 -74.00677490234375 40.70389938354492 1 2.8 12 0 0.5 2.55 0 0 0.3 15.35 CSH 0 H � yellow -95 30.01 1 Manhattan 1003001 E MN27 Chinatown 3809 255 7 1 Manhattan 1000700 I MN25 Battery Park City-Lower Manhattan 3810 +1200009939 2 2015-07-11 2015-07-11 11:10:50 2015-07-11 2015-07-11 11:15:08 0 1 -73.97708129882812 40.755531311035156 -73.9682388305664 40.76243209838867 5 0.86 5 0 0.5 0 0 0 0.3 5.8 CRE 0 a 7 yellow -31 94 1 Manhattan 1009400 E MN17 Midtown-Midtown South 3807 22 114.02 1 Manhattan 1011402 I MN40 Upper East Side-Carnegie Hill 3805 +1200009947 1 2015-07-11 2015-07-11 11:26:01 2015-07-11 2015-07-11 11:31:59 0 1 -73.9447250366211 40.77920913696289 -73.95915985107422 40.773277282714844 3 1.2 6.5 0 0.5 1.46 0 0 0.3 8.76 CSH 0 b 7 yellow 1 152 1 Manhattan 1015200 I MN32 Yorkville 3805 18 128 1 Manhattan 1012800 I MN40 Upper East Side-Carnegie Hill 3805 +1200009964 2 2015-07-11 2015-07-11 12:16:40 2015-07-11 2015-07-11 12:37:42 0 1 -73.95733642578125 40.77019119262695 -73.98934173583984 40.73419189453125 1 4.39 17 0 0.5 3.56 0 0 0.3 21.36 CSH 0 A y yellow -38 134 1 Manhattan 1013400 I MN31 Lenox Hill-Roosevelt Island 3805 200 42 1 Manhattan 1004200 I MN22 East Village 3809 +1200009995 1 2015-07-11 2015-07-11 13:47:55 2015-07-11 2015-07-11 14:12:50 0 1 -73.99453735351562 40.722900390625 -73.98739624023438 40.728824615478516 1 1.8 15.5 0 0.5 3.25 0 0 0.3 19.55 CSH 0 D y yellow -67 43 1 Manhattan 1004300 I MN24 SoHo-TriBeCa-Civic Center-Little Italy 3810 190 38 1 Manhattan 1003800 I MN22 East Village 3809 +1200010002 2 2015-07-13 2015-07-13 05:44:26 2015-07-13 2015-07-13 05:53:37 0 1 -73.97551727294922 40.73318862915039 -73.99541473388672 40.749267578125 1 2.01 9 0.5 0.5 0 0 0 0.3 10.3 CRE 0 $ � yellow 104 60 1 Manhattan 1006000 I MN50 Stuyvesant Town-Cooper Village 3808 131 95 1 Manhattan 1009500 I MN17 Midtown-Midtown South 3807 +1200010031 2 2015-07-11 2015-07-11 15:04:14 2015-07-11 2015-07-11 15:19:38 0 1 -74.00341033935547 40.727691650390625 -74.0088119506836 40.707481384277344 1 3.27 13.5 0 0.5 2.86 0 0 0.3 17.16 CSH 0 % � yellow 13 37 1 Manhattan 1003700 I MN24 SoHo-TriBeCa-Civic Center-Little Italy 3810 255 7 1 Manhattan 1000700 I MN25 Battery Park City-Lower Manhattan 3810 +1200010047 2 2015-07-11 2015-07-11 15:54:46 2015-07-11 2015-07-11 16:04:12 0 1 -73.99411010742188 40.74301528930664 -73.99284362792969 40.76173400878906 1 1.83 8.5 0 0.5 1 0 0 0.3 10.3 CSH 0 � H yellow -124 91 1 Manhattan 1009100 I MN13 Hudson Yards-Chelsea-Flatiron-Union Square 3807 139 121 1 Manhattan 1012100 E MN15 Clinton 3807 +1200010102 1 2015-07-11 2015-07-11 18:12:28 2015-07-11 2015-07-11 18:17:08 0 1 -73.99691009521484 40.7476921081543 -74.00384521484375 40.7508544921875 1 0.8 5.5 0 0.5 1 0 0 0.3 7.3 CSH 0 h F yellow -18 97 1 Manhattan 1009700 E MN13 Hudson Yards-Chelsea-Flatiron-Union Square 3807 50 99 1 Manhattan 1009900 I MN13 Hudson Yards-Chelsea-Flatiron-Union Square 3807 +1200010119 1 2015-07-11 2015-07-11 19:01:26 2015-07-11 2015-07-11 19:06:23 1 1 -73.98989868164062 40.735389709472656 -73.97554779052734 40.73551559448242 3 1 6 0 0.5 1.36 0 0 0.3 8.16 CSH 0 4 $ yellow -11 52 1 Manhattan 1005200 I MN13 Hudson Yards-Chelsea-Flatiron-Union Square 3807 104 60 1 Manhattan 1006000 I MN50 Stuyvesant Town-Cooper Village 3808 +1200010130 1 2015-07-11 2015-07-11 19:19:36 2015-07-11 2015-07-11 19:27:40 0 1 -73.98368072509766 40.75596618652344 -73.97675323486328 40.74350357055664 1 1.4 7.5 0 0.5 0 0 0 0.3 8.3 CRE 0 0 7 yellow -123 119 1 Manhattan 1011900 E MN17 Midtown-Midtown South 3807 246 70 1 Manhattan 1007000 I MN20 Murray Hill-Kips Bay 3808 +1200010166 1 2015-07-11 2015-07-11 21:12:27 2015-07-11 2015-07-11 21:20:38 0 1 -73.9716796875 40.764427185058594 -73.93923950195312 40.799354553222656 1 3.2 11 0.5 0.5 0 0 0 0.3 12.3 CRE 0 7 t yellow -29 114.01 1 Manhattan 1011401 I MN40 Upper East Side-Carnegie Hill 3805 230 182 1 Manhattan 1018200 E MN34 East Harlem North 3804 +1200010170 2 2015-07-11 2015-07-11 21:19:03 2015-07-11 2015-07-11 21:30:30 0 1 -73.98320007324219 40.76249694824219 -73.9908447265625 40.73955154418945 1 1.85 9.5 0.5 0.5 0 0 0 0.3 10.8 CRE 0 0 4 yellow 3 131 1 Manhattan 1013100 I MN17 Midtown-Midtown South 3807 245 52 1 Manhattan 1005200 I MN13 Hudson Yards-Chelsea-Flatiron-Union Square 3807 +1200010188 1 2015-07-11 2015-07-11 22:09:01 2015-07-11 2015-07-11 22:23:33 0 1 -73.98562622070312 40.74138259887695 -73.95020294189453 40.783817291259766 2 3.7 14.5 0.5 0.5 1.5 0 0 0.3 17.3 CSH 0 4 c yellow -90 56 1 Manhattan 1005600 I MN13 Hudson Yards-Chelsea-Flatiron-Union Square 3807 31 154 1 Manhattan 1015400 I MN32 Yorkville 3805 +1200010213 2 2015-07-11 2015-07-11 22:59:45 2015-07-11 2015-07-11 23:15:34 0 1 -73.99884033203125 40.719879150390625 -73.9787826538086 40.74089050292969 1 2.33 12 0.5 0.5 0 0 0 0.3 13.3 CRE 0 D 7 yellow 124 45 1 Manhattan 1004500 I MN24 SoHo-TriBeCa-Civic Center-Little Italy 3810 245 66 1 Manhattan 1006600 I MN20 Murray Hill-Kips Bay 3808 +1200010249 1 2015-07-12 2015-07-12 00:56:34 2015-07-12 2015-07-12 01:06:07 0 1 -73.991943359375 40.725765228271484 -73.99647521972656 40.74115753173828 1 1.5 8.5 0.5 0.5 1.96 0 0 0.3 11.76 CSH 0 y � yellow -62 36.02 1 Manhattan 1003602 I MN22 East Village 3809 143 87 1 Manhattan 1008700 I MN13 Hudson Yards-Chelsea-Flatiron-Union Square 3807 +1200010266 1 2015-07-12 2015-07-12 01:39:56 2015-07-12 2015-07-12 01:48:48 0 1 -73.93499755859375 40.75101852416992 -73.91255187988281 40.75653076171875 1 1.5 8 0.5 0.5 0 0 0 0.3 9.3 CRE 0 F  yellow 122 33 4 Queens 4003300 E QN68 Queensbridge-Ravenswood-Long Island City 4101 73 153 4 Queens 4015300 I QN70 Astoria 4101 +1200010269 1 2015-07-12 2015-07-12 01:49:15 2015-07-12 2015-07-12 01:53:03 0 1 -73.99179077148438 40.7447624206543 -73.9945068359375 40.75499725341797 2 1 5 0.5 0.5 0 0 0 0.3 6.3 CRE 0 � h yellow -124 91 1 Manhattan 1009100 I MN13 Hudson Yards-Chelsea-Flatiron-Union Square 3807 162 111 1 Manhattan 1011100 I MN13 Hudson Yards-Chelsea-Flatiron-Union Square 3807 +1200010270 2 2015-07-12 2015-07-12 01:51:17 2015-07-12 2015-07-12 02:01:36 0 1 -73.98612213134766 40.76729202270508 -73.9791488647461 40.74775314331055 1 2.09 9.5 0.5 0.5 2 0 0 0.3 12.8 CSH 0 H p yellow -54 139 1 Manhattan 1013900 I MN15 Clinton 3807 133 80 1 Manhattan 1008000 I MN20 Murray Hill-Kips Bay 3808 +1200010293 1 2015-07-12 2015-07-12 03:39:22 2015-07-12 2015-07-12 03:47:26 0 1 -73.97799682617188 40.74616241455078 -73.95368194580078 40.78126907348633 1 2.8 10 0.5 0.5 2.8 0 0 0.3 14.1 CSH 0 p 6 yellow -126 72 1 Manhattan 1007200 I MN20 Murray Hill-Kips Bay 3808 29 148.02 1 Manhattan 1014802 I MN40 Upper East Side-Carnegie Hill 3805 +1200010300 2 2015-07-12 2015-07-12 03:59:57 2015-07-12 2015-07-12 04:05:04 0 1 -74.00511169433594 40.719356536865234 -73.98970031738281 40.74354553222656 4 2.09 7.5 0.5 0.5 1 0 0 0.3 9.8 CSH 0 1 4 yellow 15 33 1 Manhattan 1003300 I MN24 SoHo-TriBeCa-Civic Center-Little Italy 3810 133 58 1 Manhattan 1005800 I MN13 Hudson Yards-Chelsea-Flatiron-Union Square 3807 +1200010304 1 2015-07-12 2015-07-12 04:14:54 2015-07-12 2015-07-12 04:20:41 0 1 -73.99327087402344 40.73574447631836 -73.98358154296875 40.750614166259766 1 1.6 7 0.5 0.5 0 0 0 0.3 8.3 CRE 0  d yellow -55 61 1 Manhattan 1006100 I MN23 West Village 3810 209 84 1 Manhattan 1008400 I MN17 Midtown-Midtown South 3807 +1200010307 1 2015-07-12 2015-07-12 06:05:36 2015-07-12 2015-07-12 06:43:14 0 1 -74.00406646728516 40.752891540527344 -73.89688873291016 40.83433532714844 1 11.9 38 0 0.5 0 0 0 0.3 38.8 CRE 0 F ` yellow 50 99 1 Manhattan 1009900 I MN13 Hudson Yards-Chelsea-Flatiron-Union Square 3807 18 153 2 Bronx 2015300 E BX75 Crotona Park East 3705 +1200010349 1 2015-07-12 2015-07-12 10:39:37 2015-07-12 2015-07-12 10:44:37 0 1 -73.97581481933594 40.740726470947266 -73.9757080078125 40.75350570678711 1 1.2 6 0 0.5 1.36 0 0 0.3 8.16 CSH 0 7 b yellow -10 70 1 Manhattan 1007000 I MN20 Murray Hill-Kips Bay 3808 226 92 1 Manhattan 1009200 I MN19 Turtle Bay-East Midtown 3808 +1200010369 1 2015-07-12 2015-07-12 11:57:54 2015-07-12 2015-07-12 12:12:51 0 1 -73.98041534423828 40.73054885864258 -73.97514343261719 40.75936508178711 1 3 13 0 0.5 2.75 0 0 0.3 16.55 CSH 0 $ a yellow 110 44 1 Manhattan 1004400 I MN50 Stuyvesant Town-Cooper Village 3808 16 102 1 Manhattan 1010200 I MN17 Midtown-Midtown South 3807 +1200010374 1 2015-07-12 2015-07-12 12:03:12 2015-07-12 2015-07-12 12:21:23 0 1 -73.98155975341797 40.780792236328125 -73.97818756103516 40.74187469482422 1 3.3 14.5 0 0.5 3.05 0 0 0.3 18.35 CSH 0 9 7 yellow -101 163 1 Manhattan 1016300 I MN12 Upper West Side 3806 246 70 1 Manhattan 1007000 I MN20 Murray Hill-Kips Bay 3808 +1200010384 2 2015-07-12 2015-07-12 12:20:01 2015-07-12 2015-07-12 12:31:30 0 1 -74.01632690429688 40.71137619018555 -73.99507141113281 40.749996185302734 6 3.52 12.5 0 0.5 0 0 0 0.3 13.3 CRE 0  h yellow 54 317.04 1 Manhattan 1031704 I MN25 Battery Park City-Lower Manhattan 3810 103 103 1 Manhattan 1010300 I MN13 Hudson Yards-Chelsea-Flatiron-Union Square 3807 +1200010413 1 2015-07-12 2015-07-12 13:39:23 2015-07-12 2015-07-12 13:45:09 0 1 -74.00782775878906 40.7073860168457 -74.01668548583984 40.70805358886719 2 0.9 6 0 0.5 1.35 0 0 0.3 8.15 CSH 0 �  yellow -1 7 1 Manhattan 1000700 I MN25 Battery Park City-Lower Manhattan 3810 54 317.04 1 Manhattan 1031704 I MN25 Battery Park City-Lower Manhattan 3810 +1200010433 1 2015-07-12 2015-07-12 14:16:25 2015-07-12 2015-07-12 14:28:54 0 1 -73.9900131225586 40.756126403808594 -73.9858627319336 40.73125457763672 1 2.8 11.5 0 0.5 2.46 0 0 0.3 14.76 CSH 0  y yellow 21 113 1 Manhattan 1011300 I MN17 Midtown-Midtown South 3807 93 40 1 Manhattan 1004000 I MN22 East Village 3809 +1200010447 1 2015-07-12 2015-07-12 14:52:28 2015-07-12 2015-07-12 15:13:01 0 1 -74.0007553100586 40.720741271972656 -73.98336791992188 40.770057678222656 1 4.3 16.5 0 0.5 1 0 0 0.3 18.3 CSH 0  B yellow 16 47 1 Manhattan 1004700 I MN24 SoHo-TriBeCa-Civic Center-Little Italy 3810 66 145 1 Manhattan 1014500 I MN14 Lincoln Square 3806 +1200010482 1 2015-07-12 2015-07-12 16:24:32 2015-07-12 2015-07-12 16:30:41 0 1 -73.9622802734375 40.795101165771484 -73.97953796386719 40.790462493896484 2 1.3 7 0 0.5 1.55 0 0 0.3 9.35 CSH 0 Q 8 yellow -45 189 1 Manhattan 1018900 E MN12 Upper West Side 3806 138 175 1 Manhattan 1017500 I MN12 Upper West Side 3806 +1200010484 1 2015-07-12 2015-07-12 16:35:13 2015-07-12 2015-07-12 16:55:58 0 1 -73.98494720458984 40.76422119140625 -74.00991821289062 40.720645904541016 2 4.1 17 0 0.5 0 0 0 0.3 17.8 CRE 0 H 1 yellow -119 133 1 Manhattan 1013300 I MN15 Clinton 3807 14 39 1 Manhattan 1003900 I MN24 SoHo-TriBeCa-Civic Center-Little Italy 3810 +1200010508 2 2015-07-12 2015-07-12 17:49:42 2015-07-12 2015-07-12 18:01:40 0 1 -73.98209381103516 40.778587341308594 -73.96805572509766 40.759822845458984 1 1.97 10 0 0.5 0 0 0 0.3 10.8 CRE 0 C b yellow -102 159 1 Manhattan 1015900 I MN14 Lincoln Square 3806 20 112.03 1 Manhattan 1011203 I MN19 Turtle Bay-East Midtown 3808 +1200010514 1 2015-07-12 2015-07-12 18:01:24 2015-07-12 2015-07-12 18:09:54 0 1 -73.9775161743164 40.755584716796875 -73.97901153564453 40.76528549194336 1 1 7.5 0 0.5 1.65 0 0 0.3 9.95 CSH 0 a c yellow -31 94 1 Manhattan 1009400 E MN17 Midtown-Midtown South 3807 4 137 1 Manhattan 1013700 I MN17 Midtown-Midtown South 3807 +1200010570 2 2015-07-12 2015-07-12 20:25:16 2015-07-12 2015-07-12 21:02:01 0 1 -73.99298858642578 40.75782012939453 -74.03028106689453 40.63511657714844 1 10.91 36 0.5 0.5 5 5.54 0 0.3 47.84 CSH 0 H  yellow -93 115 1 Manhattan 1011500 I MN15 Clinton 3807 95 36 3 Brooklyn 3003600 I BK31 Bay Ridge 4013 +1200010571 1 2015-07-12 2015-07-12 20:31:39 2015-07-12 2015-07-12 20:41:22 0 1 -73.99333953857422 40.722869873046875 -73.9840087890625 40.70236587524414 2 2.2 10 0.5 0.5 2.26 0 0 0.3 13.56 CSH 0 D f yellow -67 43 1 Manhattan 1004300 I MN24 SoHo-TriBeCa-Civic Center-Little Italy 3810 248 21 3 Brooklyn 3002100 I BK38 DUMBO-Vinegar Hill-Downtown Brooklyn-Boerum Hill 4004 +1200010590 1 2015-07-12 2015-07-12 21:39:38 2015-07-12 2015-07-12 22:12:02 0 1 -74.0014877319336 40.707916259765625 -73.92520904541016 40.70517349243164 1 5.7 24.5 0.5 0.5 0 0 0 0.3 25.8 CRE 0 \t 6 yellow -101 15.01 1 Manhattan 1001501 I MN25 Battery Park City-Lower Manhattan 3810 208 427 3 Brooklyn 3042700 E BK77 Bushwick North 4002 +1200010610 2 2015-07-12 2015-07-12 22:19:57 2015-07-12 2015-07-12 22:32:08 0 1 -73.99009704589844 40.732120513916016 -73.99638366699219 40.76253128051758 1 2.92 11.5 0.5 0.5 2 0 0 0.3 14.8 CSH 0 y P yellow -56 42 1 Manhattan 1004200 I MN22 East Village 3809 140 129 1 Manhattan 1012900 I MN15 Clinton 3807 +1200010624 1 2015-07-06 2015-07-06 02:25:07 2015-07-06 2015-07-06 02:33:33 0 1 -73.99698638916016 40.72064208984375 -73.98287200927734 40.75581741333008 1 3.1 10.5 0.5 0.5 0 0 0 0.3 11.8 CRE 0 D a yellow -60 41 1 Manhattan 1004100 E MN24 SoHo-TriBeCa-Civic Center-Little Italy 3810 13 96 1 Manhattan 1009600 E MN17 Midtown-Midtown South 3807 +1200010635 1 2015-07-06 2015-07-06 06:54:12 2015-07-06 2015-07-06 06:57:17 0 1 -74.01034545898438 40.711891174316406 -74.01139068603516 40.70697021484375 1 0.4 4 0 0.5 0 0 0 0.3 4.8 CSH 0 a � yellow 53 13 1 Manhattan 1001300 I MN25 Battery Park City-Lower Manhattan 3810 255 7 1 Manhattan 1000700 I MN25 Battery Park City-Lower Manhattan 3810 +1200010642 2 2015-07-06 2015-07-06 07:44:33 2015-07-06 2015-07-06 08:06:37 0 1 -73.97908020019531 40.747161865234375 -74.01448059082031 40.713890075683594 1 3.44 16.5 0 0.5 2 0 0 0.3 19.3 CSH 0 p  yellow -123 80 1 Manhattan 1008000 I MN20 Murray Hill-Kips Bay 3808 52 317.03 1 Manhattan 1031703 I MN25 Battery Park City-Lower Manhattan 3810 +1200010659 1 2015-07-06 2015-07-06 08:59:17 2015-07-06 2015-07-06 09:05:46 1 1 -73.97396087646484 40.793907165527344 -73.98180389404297 40.77691650390625 2 1.4 7 0 0.5 1.55 0 0 0.3 9.35 CSH 0 8 B yellow -88 183 1 Manhattan 1018300 I MN12 Upper West Side 3806 30 153 1 Manhattan 1015300 I MN14 Lincoln Square 3806 +1200010722 1 2015-07-06 2015-07-06 13:46:56 2015-07-06 2015-07-06 14:01:19 0 1 -73.9880142211914 40.748321533203125 -74.00688171386719 40.70549392700195 1 5 17 0 0.5 0 0 0 0.3 17.8 CRE 0 d � yellow -48 76 1 Manhattan 1007600 I MN17 Midtown-Midtown South 3807 255 7 1 Manhattan 1000700 I MN25 Battery Park City-Lower Manhattan 3810 +1200010750 1 2015-07-06 2015-07-06 15:07:12 2015-07-06 2015-07-06 15:11:11 0 1 -74.00102233886719 40.72698974609375 -73.9934310913086 40.73333740234375 1 0.6 4.5 0 0.5 0 0 0 0.3 5.3 CRE 0   yellow 35 49 1 Manhattan 1004900 I MN24 SoHo-TriBeCa-Civic Center-Little Italy 3810 201 61 1 Manhattan 1006100 I MN23 West Village 3810 +1200010757 2 2015-07-06 2015-07-06 15:31:07 2015-07-06 2015-07-06 15:40:47 0 1 -73.96688842773438 40.79387664794922 -73.95333862304688 40.79854965209961 4 1.48 8.5 0 0.5 0 0 0 0.3 9.3 CRE 0 Q C yellow 25 185 1 Manhattan 1018500 I MN12 Upper West Side 3806 43 143 1 Manhattan 1014300 I MN99 park-cemetery-etc-Manhattan 3806 +1200010758 2 2015-07-06 2015-07-06 15:32:31 2015-07-06 2015-07-06 16:02:30 0 1 -73.98164367675781 40.758460998535156 -73.88945770263672 40.74092102050781 1 6.01 24 0 0.5 0 0 0 0.3 24.8 CRE 0 0 � yellow -122 125 1 Manhattan 1012500 I MN17 Midtown-Midtown South 3807 156 483 4 Queens 4048300 E QN50 Elmhurst-Maspeth 4109 +1200010776 2 2015-07-06 2015-07-06 16:32:00 2015-07-06 2015-07-06 16:47:25 0 1 -73.982421875 40.77490234375 -73.98944091796875 40.743896484375 1 2.46 12 1 0.5 0 0 0 0.3 13.8 CSH 0 B 4 yellow 30 153 1 Manhattan 1015300 I MN14 Lincoln Square 3806 133 58 1 Manhattan 1005800 I MN13 Hudson Yards-Chelsea-Flatiron-Union Square 3807 +1200010797 1 2015-07-06 2015-07-06 17:28:28 2015-07-06 2015-07-06 17:33:04 0 1 -73.98743438720703 40.76055908203125 -73.98878479003906 40.77409362792969 1 1.3 6 1 0.5 0 0 0 0.3 7.8 CRE 0 H C yellow -118 127 1 Manhattan 1012700 I MN15 Clinton 3807 205 151 1 Manhattan 1015100 E MN14 Lincoln Square 3806 +1200010858 2 2015-07-06 2015-07-06 21:15:17 2015-07-06 2015-07-06 21:44:41 0 2 -73.78443145751953 40.64860916137695 -73.9549560546875 40.76730728149414 2 17.41 52 0 0.5 5 0 0 0.3 57.8 CSH 0 2 @ yellow 8 716 4 Queens 4071600 I QN98 Airport 4105 160 124 1 Manhattan 1012400 I MN31 Lenox Hill-Roosevelt Island 3805 +1200010898 2 2015-07-07 2015-07-07 00:04:35 2015-07-07 2015-07-07 00:10:18 0 1 -74.0068588256836 40.719581604003906 -74.00755310058594 40.71061325073242 1 0.86 6 0.5 0.5 2 0 0 0.3 9.3 CSH 0 1 \t yellow 15 33 1 Manhattan 1003300 I MN24 SoHo-TriBeCa-Civic Center-Little Italy 3810 155 15.01 1 Manhattan 1001501 I MN25 Battery Park City-Lower Manhattan 3810 +1200010941 1 2015-07-07 2015-07-07 07:28:21 2015-07-07 2015-07-07 07:39:32 0 1 -73.99567413330078 40.749141693115234 -73.97628784179688 40.756961822509766 1 1.6 9 0 0.5 1 0 0 0.3 10.8 CSH 0 � a yellow -125 95 1 Manhattan 1009500 I MN17 Midtown-Midtown South 3807 225 94 1 Manhattan 1009400 E MN17 Midtown-Midtown South 3807 +1200010955 2 2015-07-07 2015-07-07 08:19:22 2015-07-07 2015-07-07 08:30:36 0 1 -73.97393798828125 40.755043029785156 -73.98787689208984 40.738197326660156 1 1.46 9 0 0.5 1.96 0 0 0.3 11.76 CSH 0 b 4 yellow -30 92 1 Manhattan 1009200 I MN19 Turtle Bay-East Midtown 3808 245 52 1 Manhattan 1005200 I MN13 Hudson Yards-Chelsea-Flatiron-Union Square 3807 +1200010965 1 2015-07-07 2015-07-07 08:34:52 2015-07-07 2015-07-07 08:46:35 0 1 -73.95960998535156 40.77364730834961 -73.98258209228516 40.761837005615234 1 2 10 0 0.5 2.15 0 0 0.3 12.95 CSH 0 6 0 yellow 24 140 1 Manhattan 1014000 I MN40 Upper East Side-Carnegie Hill 3805 3 131 1 Manhattan 1013100 I MN17 Midtown-Midtown South 3807 +1200010991 2 2015-07-07 2015-07-07 10:02:29 2015-07-07 2015-07-07 10:15:12 0 1 -73.9701156616211 40.75287628173828 -73.98331451416016 40.74219512939453 1 1.22 9.5 0 0.5 2.58 0 0 0.3 12.88 CSH 0 3  yellow -104 90 1 Manhattan 1009000 I MN19 Turtle Bay-East Midtown 3808 11 68 1 Manhattan 1006800 I MN21 Gramercy 3808 +1200011011 1 2015-07-07 2015-07-07 11:10:55 2015-07-07 2015-07-07 11:28:56 0 1 -73.98197937011719 40.778839111328125 -73.99563598632812 40.75575256347656 1 1.8 12.5 0 0.5 0 0 0 0.3 13.3 CRE 0 C h yellow -102 159 1 Manhattan 1015900 I MN14 Lincoln Square 3806 162 111 1 Manhattan 1011100 I MN13 Hudson Yards-Chelsea-Flatiron-Union Square 3807 +1200011028 1 2015-07-07 2015-07-07 12:02:27 2015-07-07 2015-07-07 12:11:47 0 1 -73.978759765625 40.75050735473633 -73.969970703125 40.76265335083008 1 1 7.5 0 0.5 0 0 0 0.3 8.3 CRE 0 p b yellow -123 80 1 Manhattan 1008000 I MN20 Murray Hill-Kips Bay 3808 20 112.03 1 Manhattan 1011203 I MN19 Turtle Bay-East Midtown 3808 +1200011036 2 2015-07-07 2015-07-07 12:33:26 2015-07-07 2015-07-07 12:41:45 0 1 -73.96932983398438 40.7983512878418 -73.9781265258789 40.78398132324219 5 0.97 7.5 0 0.5 1 0 0 0.3 9.3 CSH 0 Q 9 yellow -87 187 1 Manhattan 1018700 I MN12 Upper West Side 3806 153 167 1 Manhattan 1016700 I MN12 Upper West Side 3806 +1200011088 1 2015-07-07 2015-07-07 14:50:20 2015-07-07 2015-07-07 14:58:49 0 1 -73.98291778564453 40.76279067993164 -73.99561309814453 40.76234817504883 1 0.9 6.5 0 0.5 1.82 0 0 0.3 9.12 CSH 0 0 P yellow 3 131 1 Manhattan 1013100 I MN17 Midtown-Midtown South 3807 140 129 1 Manhattan 1012900 I MN15 Clinton 3807 +1200011140 2 2015-07-07 2015-07-07 17:21:16 2015-07-07 2015-07-07 17:35:52 0 1 -74.00868225097656 40.704471588134766 -73.97786712646484 40.752620697021484 1 5.6899999999999995 18.5 1 0.5 2 0 0 0.3 22.3 CSH 0 � b yellow -1 7 1 Manhattan 1000700 I MN25 Battery Park City-Lower Manhattan 3810 226 92 1 Manhattan 1009200 I MN19 Turtle Bay-East Midtown 3808 +1200011174 1 2015-07-07 2015-07-07 18:29:01 2015-07-07 2015-07-07 18:35:03 0 1 -73.98243713378906 40.757625579833984 -73.97210693359375 40.753082275390625 2 0.6 5.5 1 0.5 2.15 0 0 0.3 9.45 CSH 0 0 3 yellow -122 125 1 Manhattan 1012500 I MN17 Midtown-Midtown South 3807 152 90 1 Manhattan 1009000 I MN19 Turtle Bay-East Midtown 3808 +1200011187 1 2015-07-07 2015-07-07 18:56:37 2015-07-07 2015-07-07 19:02:25 0 1 -73.99434661865234 40.736270904541016 -74.00436401367188 40.73814010620117 1 0.6 5.5 1 0.5 0 0 0 0.3 7.3 CRE 0  I yellow -122 63 1 Manhattan 1006300 I MN23 West Village 3810 78 77 1 Manhattan 1007700 I MN23 West Village 3810 +1200011189 1 2015-07-07 2015-07-07 18:58:05 2015-07-07 2015-07-07 18:59:10 0 1 -73.99449157714844 40.734535217285156 -73.99678039550781 40.7314453125 2 0.2 3 1 0.5 0 0 0 0.3 4.8 CRE 0   yellow -55 61 1 Manhattan 1006100 I MN23 West Village 3810 163 59 1 Manhattan 1005900 I MN23 West Village 3810 +1200011198 2 2015-07-07 2015-07-07 19:23:21 2015-07-07 2015-07-07 19:36:19 0 1 -73.97767639160156 40.766319274902344 -73.97374725341797 40.75121307373047 1 1.71 10 1 0.5 0 0 0 0.3 11.8 CRE 0 C 3 yellow 43 143 1 Manhattan 1014300 I MN99 park-cemetery-etc-Manhattan 3806 151 88 1 Manhattan 1008800 I MN19 Turtle Bay-East Midtown 3808 +1200011201 1 2015-07-07 2015-07-07 19:30:03 2015-07-07 2015-07-07 19:52:46 0 1 -73.78244018554688 40.644596099853516 -73.75543212890625 40.59383773803711 1 9.2 27.5 1 0.5 7.32 0 0 0.3 36.62 CSH 0 2 � yellow 8 716 4 Queens 4071600 I QN98 Airport 4105 173 998.02 4 Queens 4099802 E QN15 Far Rockaway-Bayswater 4114 +1200011224 1 2015-07-07 2015-07-07 20:45:17 2015-07-07 2015-07-07 20:57:10 0 1 -73.98957824707031 40.71906280517578 -74.00245666503906 40.72650146484375 1 1.4 9.5 0.5 0.5 2.15 0 0 0.3 12.95 CSH 0 H  yellow -61 18 1 Manhattan 1001800 E MN27 Chinatown 3809 35 49 1 Manhattan 1004900 I MN24 SoHo-TriBeCa-Civic Center-Little Italy 3810 +1200011255 1 2015-07-07 2015-07-07 22:24:57 2015-07-07 2015-07-07 22:29:02 0 1 -73.97545623779297 40.7894401550293 -73.97931671142578 40.78337478637695 2 0.6 5 0.5 0.5 1.25 0 0 0.3 7.55 CSH 0 8 9 yellow -118 175 1 Manhattan 1017500 I MN12 Upper West Side 3806 153 167 1 Manhattan 1016700 I MN12 Upper West Side 3806 +1200011265 2 2015-07-07 2015-07-07 22:54:32 2015-07-07 2015-07-07 23:03:30 0 1 -73.98593139648438 40.75716018676758 -73.97379302978516 40.7646598815918 5 1.18 8 0.5 0.5 0 0 0 0.3 9.3 CRE 0 0 C yellow -123 119 1 Manhattan 1011900 E MN17 Midtown-Midtown South 3807 43 143 1 Manhattan 1014300 I MN99 park-cemetery-etc-Manhattan 3806 +1200011295 2 2015-07-08 2015-07-08 01:02:29 2015-07-08 2015-07-08 01:06:52 0 1 -73.9685287475586 40.75880813598633 -73.9530029296875 40.78057861328125 1 1.71 7 0.5 0.5 2.49 0 0 0.3 10.79 CSH 0 ) 6 yellow -101 108 1 Manhattan 1010800 I MN19 Turtle Bay-East Midtown 3808 29 148.02 1 Manhattan 1014802 I MN40 Upper East Side-Carnegie Hill 3805 +1200011314 2 2015-07-08 2015-07-08 06:11:53 2015-07-08 2015-07-08 06:19:54 0 1 -74.00443267822266 40.74783706665039 -74.0138931274414 40.71485137939453 1 2.59 10 0 0.5 0 0 0 0.3 10.8 CSH 0 F  yellow 50 99 1 Manhattan 1009900 I MN13 Hudson Yards-Chelsea-Flatiron-Union Square 3807 52 317.03 1 Manhattan 1031703 I MN25 Battery Park City-Lower Manhattan 3810 +1200011324 2 2015-07-08 2015-07-08 07:18:56 2015-07-08 2015-07-08 07:29:14 0 1 -73.97161865234375 40.78738021850586 -73.99101257324219 40.76083755493164 5 2.25 9.5 0 0.5 1 0 0 0.3 11.3 CSH 0 8 H yellow -119 173 1 Manhattan 1017300 I MN12 Upper West Side 3806 139 121 1 Manhattan 1012100 E MN15 Clinton 3807 +1200011344 2 2015-07-08 2015-07-08 08:16:41 2015-07-08 2015-07-08 08:35:36 0 1 -73.99128723144531 40.75006866455078 -74.00483703613281 40.716529846191406 5 2.6 14 0 0.5 0 0 0 0.3 14.8 CRE 0 � 1 yellow -125 101 1 Manhattan 1010100 I MN17 Midtown-Midtown South 3807 15 33 1 Manhattan 1003300 I MN24 SoHo-TriBeCa-Civic Center-Little Italy 3810 +1200011345 1 2015-07-08 2015-07-08 08:20:49 2015-07-08 2015-07-08 08:27:37 0 1 -73.9626693725586 40.76719665527344 -73.97112274169922 40.75814437866211 1 1 6.5 0 0.5 0 0 0 0.3 7.3 CRE 0 7 b yellow 17 120 1 Manhattan 1012000 I MN40 Upper East Side-Carnegie Hill 3805 15 100 1 Manhattan 1010000 I MN19 Turtle Bay-East Midtown 3808 +1200011354 1 2015-07-08 2015-07-08 08:55:16 2015-07-08 2015-07-08 09:14:59 0 1 -73.97743225097656 40.73270034790039 -73.9748306274414 40.75870895385742 1 2.6 14.5 0 0.5 3.8 0 0 0.3 19.1 CSH 0 $ a yellow 110 44 1 Manhattan 1004400 I MN50 Stuyvesant Town-Cooper Village 3808 16 102 1 Manhattan 1010200 I MN17 Midtown-Midtown South 3807 +1200011425 1 2015-07-08 2015-07-08 12:32:50 2015-07-08 2015-07-08 12:47:56 0 1 -73.99020385742188 40.757415771484375 -73.96739196777344 40.80624771118164 1 4.5 16.5 0 0.5 0 0 0 0.3 17.3 CRE 0 H f yellow -117 121 1 Manhattan 1012100 E MN15 Clinton 3807 47 199 1 Manhattan 1019900 I MN09 Morningside Heights 3802 +1200011446 1 2015-07-08 2015-07-08 13:41:57 2015-07-08 2015-07-08 13:55:50 0 1 -73.98665618896484 40.73976516723633 -74.00254821777344 40.73046112060547 1 1.8 10.5 0 0.5 0 0 0 0.3 11.3 CRE 0  I yellow 11 68 1 Manhattan 1006800 I MN21 Gramercy 3808 36 67 1 Manhattan 1006700 I MN23 West Village 3810 +1200011447 2 2015-07-08 2015-07-08 13:43:40 2015-07-08 2015-07-08 14:08:55 0 1 -73.98616790771484 40.76197052001953 -73.9805679321289 40.74343490600586 2 1.95 15.5 0 0.5 2 0 0 0.3 18.3 CSH 0 0 p yellow -122 125 1 Manhattan 1012500 I MN17 Midtown-Midtown South 3807 130 72 1 Manhattan 1007200 I MN20 Murray Hill-Kips Bay 3808 +1200011476 2 2015-07-08 2015-07-08 15:07:26 2015-07-08 2015-07-08 15:30:27 0 1 -73.97380065917969 40.74776840209961 -73.99076843261719 40.75117874145508 1 1.78 13.5 0 0.5 4.29 0 0 0.3 18.59 CSH 0 p  yellow -96 78 1 Manhattan 1007800 I MN20 Murray Hill-Kips Bay 3808 132 109 1 Manhattan 1010900 I MN17 Midtown-Midtown South 3807 +1200011531 2 2015-07-13 2015-07-13 09:30:58 2015-07-13 2015-07-13 09:36:24 0 1 -73.95953369140625 40.762943267822266 -73.96952819824219 40.7635383605957 5 0.87 5.5 0 0.5 0.82 0 0 0.3 7.12 CSH 0 @ 7 yellow -34 116 1 Manhattan 1011600 I MN31 Lenox Hill-Roosevelt Island 3805 227 114.01 1 Manhattan 1011401 I MN40 Upper East Side-Carnegie Hill 3805 +1200011541 2 2015-07-13 2015-07-13 10:22:55 2015-07-13 2015-07-13 10:32:33 0 1 -73.9717025756836 40.76116180419922 -73.95958709716797 40.774391174316406 1 1.19 7.5 0 0.5 1.66 0 0 0.3 9.96 CSH 0 c 6 yellow 19 112.02 1 Manhattan 1011202 I MN17 Midtown-Midtown South 3807 24 140 1 Manhattan 1014000 I MN40 Upper East Side-Carnegie Hill 3805 +1200011548 1 2015-07-13 2015-07-13 10:53:10 2015-07-13 2015-07-13 11:05:15 0 1 -73.96275329589844 40.76957702636719 -73.97503662109375 40.75102233886719 1 1.5 9 0 0.5 0 0 0 0.3 9.8 CRE 0 7 b yellow 18 128 1 Manhattan 1012800 I MN40 Upper East Side-Carnegie Hill 3805 226 92 1 Manhattan 1009200 I MN19 Turtle Bay-East Midtown 3808 +1200011551 2 2015-07-13 2015-07-13 11:03:00 2015-07-13 2015-07-13 11:15:10 0 1 -73.97801208496094 40.766380310058594 -73.99546813964844 40.7441291809082 1 2.24 10 0 0.5 0 0 0 0.3 10.8 CRE 0 c � yellow 4 137 1 Manhattan 1013700 I MN17 Midtown-Midtown South 3807 132 91 1 Manhattan 1009100 I MN13 Hudson Yards-Chelsea-Flatiron-Union Square 3807 +1200011654 1 2015-07-13 2015-07-13 17:26:54 2015-07-13 2015-07-13 17:51:27 0 1 -73.9947738647461 40.727783203125 -73.97685241699219 40.76350784301758 1 3.3 17 1 0.5 0 0 0 0.3 18.8 CRE 0  c yellow -65 55.02 1 Manhattan 1005502 I MN23 West Village 3810 18 112.01 1 Manhattan 1011201 I MN17 Midtown-Midtown South 3807 +1200011690 2 2015-07-13 2015-07-13 19:45:25 2015-07-13 2015-07-13 19:48:33 0 1 -73.9909896850586 40.766178131103516 -73.99126434326172 40.760318756103516 4 0.52 4 1 0.5 0 0 0 0.3 5.8 CRE 0 P H yellow -121 135 1 Manhattan 1013500 E MN15 Clinton 3807 139 121 1 Manhattan 1012100 E MN15 Clinton 3807 +1200011712 1 2015-07-13 2015-07-13 20:37:53 2015-07-13 2015-07-13 20:47:45 0 1 -73.97563171386719 40.752933502197266 -73.97679901123047 40.71848678588867 1 2.9 11 0.5 0.5 2.45 0 0 0.3 14.75 CSH 0 b 2 yellow -30 92 1 Manhattan 1009200 I MN19 Turtle Bay-East Midtown 3808 89 10.02 1 Manhattan 1001002 E MN28 Lower East Side 3809 +1200011714 2 2015-07-13 2015-07-13 20:45:50 2015-07-13 2015-07-13 20:51:31 0 1 -73.97058868408203 40.76340866088867 -73.97544860839844 40.752498626708984 1 0.89 6 0.5 0.5 1.46 0 0 0.3 8.76 CSH 0 7 b yellow -29 114.01 1 Manhattan 1011401 I MN40 Upper East Side-Carnegie Hill 3805 226 92 1 Manhattan 1009200 I MN19 Turtle Bay-East Midtown 3808 +1200011734 1 2015-07-13 2015-07-13 21:43:41 2015-07-13 2015-07-13 21:56:59 0 1 -73.97134399414062 40.757747650146484 -73.9663314819336 40.79970169067383 1 3.9 14 0.5 0.5 3.05 0 0 0.3 18.35 CSH 0 b Q yellow 15 100 1 Manhattan 1010000 I MN19 Turtle Bay-East Midtown 3808 112 191 1 Manhattan 1019100 I MN12 Upper West Side 3806 +1200011759 1 2015-07-13 2015-07-13 22:40:34 2015-07-13 2015-07-13 22:46:01 0 1 -73.95745086669922 40.76992416381836 -73.96955871582031 40.76313781738281 1 1 6 0.5 0.5 1.45 0 0 0.3 8.75 CSH 0 A 7 yellow -73 126 1 Manhattan 1012600 I MN31 Lenox Hill-Roosevelt Island 3805 22 114.02 1 Manhattan 1011402 I MN40 Upper East Side-Carnegie Hill 3805 +1200011775 2 2015-07-13 2015-07-13 23:31:19 2015-07-13 2015-07-13 23:44:12 0 1 -74.00486755371094 40.74163818359375 -73.98114776611328 40.76483154296875 6 2.49 11.5 0.5 0.5 0 0 0 0.3 12.8 CRE 0 h c yellow 75 83 1 Manhattan 1008300 E MN13 Hudson Yards-Chelsea-Flatiron-Union Square 3807 4 137 1 Manhattan 1013700 I MN17 Midtown-Midtown South 3807 +1200011793 1 2015-07-08 2015-07-08 10:14:46 2015-07-08 2015-07-08 10:41:55 0 1 -73.97244262695312 40.75340270996094 -73.94328308105469 40.83988952636719 1 9.2 29.5 0 0.5 2 0 0 0.3 32.3 CSH 0 3 D yellow -104 90 1 Manhattan 1009000 I MN19 Turtle Bay-East Midtown 3808 124 247 1 Manhattan 1024700 E MN36 Washington Heights South 3801 +1200011796 1 2015-07-14 2015-07-14 00:55:24 2015-07-14 2015-07-14 01:10:56 0 1 -73.9990005493164 40.76112747192383 -73.96028137207031 40.8105583190918 1 5.3 18 0.5 0.5 3.85 0 0 0.3 23.15 CSH 0 P f yellow -116 129 1 Manhattan 1012900 I MN15 Clinton 3807 140 203 1 Manhattan 1020300 E MN09 Morningside Heights 3802 +1200011839 2 2015-07-14 2015-07-14 08:24:17 2015-07-14 2015-07-14 08:55:06 0 1 -73.94213104248047 40.79503631591797 -73.92118072509766 40.74326705932617 1 7.26 27 0 0.5 0 5.54 0 0.3 33.34 CRE 0 t & yellow 36 180 1 Manhattan 1018000 E MN34 East Harlem North 3804 18 183 4 Queens 4018300 I QN31 Hunters Point-Sunnyside-West Maspeth 4109 +1200011840 1 2015-07-14 2015-07-14 08:26:46 2015-07-14 2015-07-14 08:39:59 0 1 -73.98625183105469 40.734718322753906 -73.97169494628906 40.75998306274414 1 2.1 10.5 0 0.5 3.35 0 0 0.3 14.65 CSH 0  b yellow -12 50 1 Manhattan 1005000 I MN21 Gramercy 3808 15 100 1 Manhattan 1010000 I MN19 Turtle Bay-East Midtown 3808 +1200011847 1 2015-07-14 2015-07-14 08:39:53 2015-07-14 2015-07-14 08:51:00 0 1 -73.98278045654297 40.745140075683594 -73.99140167236328 40.73017883300781 1 1.2 9 0 0.5 1.95 0 0 0.3 11.75 CSH 0 d  yellow -125 74 1 Manhattan 1007400 I MN17 Midtown-Midtown South 3807 192 57 1 Manhattan 1005700 I MN23 West Village 3810 +1200011872 1 2015-07-14 2015-07-14 09:56:57 2015-07-14 2015-07-14 10:09:10 0 1 -73.9880142211914 40.73833465576172 -73.9758071899414 40.7557258605957 1 1.7 10 0 0.5 2.16 0 0 0.3 12.96 CSH 0 4 a yellow -11 52 1 Manhattan 1005200 I MN13 Hudson Yards-Chelsea-Flatiron-Union Square 3807 225 94 1 Manhattan 1009400 E MN17 Midtown-Midtown South 3807 +1200011883 2 2015-07-14 2015-07-14 10:28:47 2015-07-14 2015-07-14 10:53:17 0 1 -73.95582580566406 40.764122009277344 -73.98113250732422 40.72266387939453 1 4.15 20 0 0.5 4.16 0 0 0.3 24.96 CSH 0 @  yellow -34 116 1 Manhattan 1011600 I MN31 Lenox Hill-Roosevelt Island 3805 147 26.01 1 Manhattan 1002601 E MN28 Lower East Side 3809 +1200011893 2 2015-07-14 2015-07-14 10:59:23 2015-07-14 2015-07-14 11:13:05 0 1 -73.95489501953125 40.76771545410156 -73.97908020019531 40.77193069458008 1 1.79 10.5 0 0.5 0 0 0 0.3 11.3 CSH 0 @ C yellow -96 124 1 Manhattan 1012400 I MN31 Lenox Hill-Roosevelt Island 3805 43 143 1 Manhattan 1014300 I MN99 park-cemetery-etc-Manhattan 3806 +1200011897 2 2015-07-14 2015-07-14 11:10:19 2015-07-14 2015-07-14 11:23:54 0 1 -73.98660278320312 40.776668548583984 -73.95745086669922 40.781982421875 2 2.38 11 0 0.5 1.77 0 0 0.3 13.57 CSH 0 C 6 yellow -50 155 1 Manhattan 1015500 I MN14 Lincoln Square 3806 74 150.02 1 Manhattan 1015002 I MN40 Upper East Side-Carnegie Hill 3805 +1200011911 2 2015-07-14 2015-07-14 11:55:49 2015-07-14 2015-07-14 12:14:43 0 1 -73.98211669921875 40.76266098022461 -73.98213195800781 40.77801513671875 1 1.34 12 0 0.5 3.2 0 0 0.3 16 CSH 0 0 C yellow 3 131 1 Manhattan 1013100 I MN17 Midtown-Midtown South 3807 154 159 1 Manhattan 1015900 I MN14 Lincoln Square 3806 +1200011920 1 2015-07-14 2015-07-14 12:28:39 2015-07-14 2015-07-14 12:48:24 0 1 -73.97798919677734 40.75251007080078 -74.00808715820312 40.74864196777344 1 2.6 14 0 0.5 0 0 0 0.3 14.8 CRE 0 b F yellow -30 92 1 Manhattan 1009200 I MN19 Turtle Bay-East Midtown 3808 50 99 1 Manhattan 1009900 I MN13 Hudson Yards-Chelsea-Flatiron-Union Square 3807 +1200011943 2 2015-07-14 2015-07-14 13:45:59 2015-07-14 2015-07-14 13:52:42 0 1 -73.98609924316406 40.756961822509766 -73.9915771484375 40.750038146972656 1 0.57 6 0 0.5 2.04 0 0 0.3 8.84 CSH 0 0 � yellow -123 119 1 Manhattan 1011900 E MN17 Midtown-Midtown South 3807 131 101 1 Manhattan 1010100 I MN17 Midtown-Midtown South 3807 +1200011968 1 2015-07-14 2015-07-14 15:13:58 2015-07-14 2015-07-14 15:18:46 0 1 -73.97533416748047 40.76089859008789 -73.98318481445312 40.74984359741211 2 0.8 5.5 0 0.5 3 0 0 0.3 9.3 CSH 0 a p yellow 16 102 1 Manhattan 1010200 I MN17 Midtown-Midtown South 3807 93 82 1 Manhattan 1008200 I MN20 Murray Hill-Kips Bay 3808 +1200011971 1 2015-07-14 2015-07-14 15:27:55 2015-07-14 2015-07-14 15:44:42 0 1 -73.99867248535156 40.72607421875 -73.99226379394531 40.748844146728516 3 2 12.5 0 0.5 0 0 0 0.3 13.3 CRE 0  � yellow 35 49 1 Manhattan 1004900 I MN24 SoHo-TriBeCa-Civic Center-Little Italy 3810 131 101 1 Manhattan 1010100 I MN17 Midtown-Midtown South 3807 +1200012002 1 2015-07-14 2015-07-14 17:54:14 2015-07-14 2015-07-14 18:02:37 0 1 -73.98224639892578 40.74462127685547 -73.96845245361328 40.752403259277344 1 1.2 7.5 1 0.5 1.85 0 0 0.3 11.15 CSH 0 p 3 yellow -126 72 1 Manhattan 1007200 I MN20 Murray Hill-Kips Bay 3808 152 90 1 Manhattan 1009000 I MN19 Turtle Bay-East Midtown 3808 +1200012035 1 2015-07-14 2015-07-14 17:23:47 2015-07-14 2015-07-14 17:34:27 0 1 -73.99134826660156 40.73160934448242 -74.0025634765625 40.74479293823242 2 1.5 9 1 0.5 2.15 0 0 0.3 12.95 CSH 0  h yellow -64 57 1 Manhattan 1005700 I MN23 West Village 3810 12 89 1 Manhattan 1008900 I MN13 Hudson Yards-Chelsea-Flatiron-Union Square 3807 +1200012056 1 2015-07-14 2015-07-14 19:52:29 2015-07-14 2015-07-14 20:23:59 0 2 -73.78318786621094 40.64860534667969 -73.96861267089844 40.75892639160156 2 17.3 52 0 0.5 0 5.54 0 0.3 58.34 CRE 0 2 b yellow 8 716 4 Queens 4071600 I QN98 Airport 4105 15 100 1 Manhattan 1010000 I MN19 Turtle Bay-East Midtown 3808 +1200012108 1 2015-07-14 2015-07-14 22:58:20 2015-07-14 2015-07-14 23:30:25 0 1 -73.98572540283203 40.75239181518555 -73.92191314697266 40.77692794799805 1 5.8 25 0.5 0.5 6.55 0 0 0.3 32.85 CSH 0 d # yellow -47 84 1 Manhattan 1008400 I MN17 Midtown-Midtown South 3807 46 97 4 Queens 4009700 I QN72 Steinway 4101 +1200012113 2 2015-07-14 2015-07-14 23:27:04 2015-07-14 2015-07-14 23:39:13 0 1 -74.00672912597656 40.731178283691406 -73.97639465332031 40.74498748779297 1 2.69 11 0.5 0.5 2.46 0 0 0.3 14.76 CSH 0 X 7 yellow -107 69 1 Manhattan 1006900 I MN23 West Village 3810 246 70 1 Manhattan 1007000 I MN20 Murray Hill-Kips Bay 3808 +1200012120 1 2015-07-14 2015-07-14 23:43:25 2015-07-15 2015-07-15 00:07:30 0 1 -74.0042724609375 40.74221420288086 -73.958251953125 40.71738052368164 1 4.8 19.5 0.5 0.5 4.15 0 0 0.3 24.95 CSH 0 h U yellow 75 83 1 Manhattan 1008300 E MN13 Hudson Yards-Chelsea-Flatiron-Union Square 3807 11 553 3 Brooklyn 3055300 E BK73 North Side-South Side 4001 +1200012136 2 2015-07-15 2015-07-15 00:50:26 2015-07-15 2015-07-15 00:58:03 0 1 -73.9870376586914 40.76422119140625 -73.97216033935547 40.75749969482422 1 0.98 6.5 0.5 0.5 1.56 0 0 0.3 9.36 CSH 0 H b yellow -119 133 1 Manhattan 1013300 I MN15 Clinton 3807 15 100 1 Manhattan 1010000 I MN19 Turtle Bay-East Midtown 3808 +1200012138 1 2015-07-15 2015-07-15 00:58:46 2015-07-15 2015-07-15 01:02:20 0 1 -73.97540283203125 40.75593185424805 -73.97943115234375 40.75025939941406 1 0.8 4.5 0.5 0.5 1.15 0 0 0.3 6.95 CSH 0 a p yellow -31 94 1 Manhattan 1009400 E MN17 Midtown-Midtown South 3807 93 82 1 Manhattan 1008200 I MN20 Murray Hill-Kips Bay 3808 +1200012160 1 2015-07-15 2015-07-15 03:56:53 2015-07-15 2015-07-15 04:13:14 0 1 -73.95416259765625 40.774654388427734 -73.94595336914062 40.714393615722656 1 5.5 18.5 0.5 0.5 0 0 0 0.3 19.8 CSH 0 c � yellow 0 138 1 Manhattan 1013800 I MN32 Yorkville 3805 20 497 3 Brooklyn 3049700 E BK90 East Williamsburg 4001 +1200012174 1 2015-07-15 2015-07-15 06:24:36 2015-07-15 2015-07-15 06:29:53 0 1 -73.9666519165039 40.764278411865234 -73.97090911865234 40.75155258178711 1 1.1 6 0 0.5 0 0 0 0.3 6.8 CSH 0 7 3 yellow 22 114.02 1 Manhattan 1011402 I MN40 Upper East Side-Carnegie Hill 3805 152 90 1 Manhattan 1009000 I MN19 Turtle Bay-East Midtown 3808 +1200012178 1 2015-07-15 2015-07-15 06:30:58 2015-07-15 2015-07-15 06:46:28 0 1 -73.9390869140625 40.80226135253906 -73.97225189208984 40.76533126831055 1 3.3 13.5 0 0.5 2.85 0 0 0.3 17.15 CSH 0 t C yellow -23 196 1 Manhattan 1019600 E MN34 East Harlem North 3804 43 143 1 Manhattan 1014300 I MN99 park-cemetery-etc-Manhattan 3806 +1200012186 1 2015-07-15 2015-07-15 07:01:55 2015-07-15 2015-07-15 07:11:20 0 1 -73.95201110839844 40.78452682495117 -73.97313690185547 40.75466537475586 1 2.3 9.5 0 0.5 2.55 0 0 0.3 12.85 CSH 0 6 b yellow -103 158.01 1 Manhattan 1015801 I MN40 Upper East Side-Carnegie Hill 3805 226 92 1 Manhattan 1009200 I MN19 Turtle Bay-East Midtown 3808 +1200012207 1 2015-07-15 2015-07-15 08:18:34 2015-07-15 2015-07-15 08:32:16 0 1 -73.9820556640625 40.772735595703125 -73.97515869140625 40.75608444213867 1 1.7 10.5 0 0.5 2.25 0 0 0.3 13.55 CSH 0 B a yellow 67 149 1 Manhattan 1014900 I MN14 Lincoln Square 3806 225 94 1 Manhattan 1009400 E MN17 Midtown-Midtown South 3807 +1200012254 1 2015-07-15 2015-07-15 10:16:56 2015-07-15 2015-07-15 10:29:22 0 1 -73.9829330444336 40.74802017211914 -73.9790267944336 40.763221740722656 1 1.3 9 0 0.5 1 0 0 0.3 10.8 CSH 0 d c yellow -125 74 1 Manhattan 1007400 I MN17 Midtown-Midtown South 3807 4 137 1 Manhattan 1013700 I MN17 Midtown-Midtown South 3807 +1200012260 2 2015-07-15 2015-07-15 10:34:11 2015-07-15 2015-07-15 10:38:57 0 1 -73.97757720947266 40.758174896240234 -73.98473358154297 40.75960159301758 5 0.49 5 0 0.5 0 0 0 0.3 5.8 CRE 0 a 0 yellow 17 104 1 Manhattan 1010400 I MN17 Midtown-Midtown South 3807 134 125 1 Manhattan 1012500 I MN17 Midtown-Midtown South 3807 +1200012277 2 2015-07-15 2015-07-15 11:45:23 2015-07-15 2015-07-15 12:01:58 0 1 -73.95174407958984 40.78188705444336 -73.98228454589844 40.74677658081055 1 4.2 15 0 0.5 3.95 0 0 0.3 19.75 CSH 0 c d yellow 31 154 1 Manhattan 1015400 I MN32 Yorkville 3805 131 74 1 Manhattan 1007400 I MN17 Midtown-Midtown South 3807 +1200012286 1 2015-07-15 2015-07-15 12:07:02 2015-07-15 2015-07-15 12:28:08 0 1 -73.87297821044922 40.774078369140625 -73.91344451904297 40.7747802734375 1 4.4 18.5 0 0.5 3 0 0 0.3 22.3 CSH 0 8 # yellow 48 331 4 Queens 4033100 I QN98 Airport 4102 56 115 4 Queens 4011500 E QN72 Steinway 4101 +1200012338 2 2015-07-15 2015-07-15 14:40:01 2015-07-15 2015-07-15 14:46:39 0 1 -73.95442962646484 40.778499603271484 -73.94977569580078 40.79335021972656 1 1.3599999999999999 7 0 0.5 0 0 0 0.3 7.8 CRE 0 6 u yellow 28 148.01 1 Manhattan 1014801 I MN40 Upper East Side-Carnegie Hill 3805 243 168 1 Manhattan 1016800 E MN33 East Harlem South 3804 +1200012340 2 2015-07-15 2015-07-15 14:42:21 2015-07-15 2015-07-15 15:11:24 0 1 -73.94934844970703 40.78104019165039 -73.97138977050781 40.75986099243164 1 2.15 18 0 0.5 3.76 0 0 0.3 22.56 CSH 0 c b yellow 31 154 1 Manhattan 1015400 I MN32 Yorkville 3805 15 100 1 Manhattan 1010000 I MN19 Turtle Bay-East Midtown 3808 +1200012382 2 2015-07-15 2015-07-15 16:29:19 2015-07-15 2015-07-15 16:39:36 0 1 -73.98526763916016 40.74164962768555 -73.98153686523438 40.72467041015625 6 1.6600000000000001 9 1 0.5 2.16 0 0 0.3 12.96 CSH 0  y yellow 11 68 1 Manhattan 1006800 I MN21 Gramercy 3808 10 32 1 Manhattan 1003200 I MN22 East Village 3809 +1200012389 1 2015-07-15 2015-07-15 16:45:55 2015-07-15 2015-07-15 16:57:13 0 1 -73.97564697265625 40.76382827758789 -73.96650695800781 40.7534065246582 1 1.4 9 1 0.5 0 0 0 0.3 10.8 CRE 0 c ) yellow 18 112.01 1 Manhattan 1011201 I MN17 Midtown-Midtown South 3807 14 98 1 Manhattan 1009800 I MN19 Turtle Bay-East Midtown 3808 +1200012403 1 2015-07-15 2015-07-15 17:35:34 2015-07-15 2015-07-15 18:13:26 0 1 -73.96661376953125 40.804378509521484 -73.97960662841797 40.73963165283203 1 6.6 27.5 1 0.5 5 0 0 0.3 34.3 CSH 0 f 7 yellow 47 199 1 Manhattan 1019900 I MN09 Morningside Heights 3802 245 66 1 Manhattan 1006600 I MN20 Murray Hill-Kips Bay 3808 +1200012416 1 2015-07-15 2015-07-15 18:09:33 2015-07-15 2015-07-15 18:15:58 0 1 -73.97614288330078 40.76530838012695 -73.98499298095703 40.76087951660156 1 0.7 6 1 0.5 1.95 0 0 0.3 9.75 CSH 0 c 0 yellow 18 112.01 1 Manhattan 1011201 I MN17 Midtown-Midtown South 3807 134 125 1 Manhattan 1012500 I MN17 Midtown-Midtown South 3807 +1200012428 2 2015-07-15 2015-07-15 18:42:13 2015-07-15 2015-07-15 18:55:55 0 1 -73.98883056640625 40.74242401123047 -73.9779281616211 40.751441955566406 3 1.27 10 1 0.5 2.36 0 0 0.3 14.16 CSH 0 4 p yellow -90 56 1 Manhattan 1005600 I MN13 Hudson Yards-Chelsea-Flatiron-Union Square 3807 133 80 1 Manhattan 1008000 I MN20 Murray Hill-Kips Bay 3808 +1200012456 1 2015-07-15 2015-07-15 19:38:37 2015-07-15 2015-07-15 19:53:16 0 1 -73.98648071289062 40.7404670715332 -73.9927749633789 40.72397994995117 1 1.3 10 1 0.5 0.75 0 0 0.3 12.55 CSH 0 4 D yellow -90 56 1 Manhattan 1005600 I MN13 Hudson Yards-Chelsea-Flatiron-Union Square 3807 189 43 1 Manhattan 1004300 I MN24 SoHo-TriBeCa-Civic Center-Little Italy 3810 +1200012458 2 2015-07-15 2015-07-15 19:40:23 2015-07-15 2015-07-15 19:44:04 0 1 -73.95404815673828 40.766231536865234 -73.9468002319336 40.77236557006836 1 0.69 5 1 0.5 1.36 0 0 0.3 8.16 CSH 0 @ b yellow -96 124 1 Manhattan 1012400 I MN31 Lenox Hill-Roosevelt Island 3805 255 136 1 Manhattan 1013600 I MN32 Yorkville 3805 +1200012475 1 2015-07-15 2015-07-15 20:11:12 2015-07-15 2015-07-15 21:00:22 0 2 -74.00364685058594 40.74028396606445 -73.78292846679688 40.64885711669922 1 18.2 52 0 0.5 0 5.54 0 0.3 58.34 CRE 0 h 2 yellow 75 83 1 Manhattan 1008300 E MN13 Hudson Yards-Chelsea-Flatiron-Union Square 3807 8 716 4 Queens 4071600 I QN98 Airport 4105 +1200012512 2 2015-07-15 2015-07-15 21:54:21 2015-07-15 2015-07-15 22:17:34 0 1 -73.96621704101562 40.76205825805664 -73.86470794677734 40.84546661376953 1 9.24 28 0.5 0.5 0 0 0 0.3 29.3 CRE 0 7 B yellow 22 114.02 1 Manhattan 1011402 I MN40 Upper East Side-Carnegie Hill 3805 90 238 2 Bronx 2023800 E BX37 Van Nest-Morris Park-Westchester Square 3704 +1200012519 1 2015-07-15 2015-07-15 22:06:46 2015-07-15 2015-07-15 22:26:36 0 1 -74.00313568115234 40.71561050415039 -73.98143005371094 40.75873947143555 4 3.4 15 0.5 0.5 0 0 0 0.3 16.3 CRE 0 1 0 yellow -102 31 1 Manhattan 1003100 I MN24 SoHo-TriBeCa-Civic Center-Little Italy 3810 134 125 1 Manhattan 1012500 I MN17 Midtown-Midtown South 3807 +1200012528 2 2015-07-15 2015-07-15 22:24:25 2015-07-15 2015-07-15 22:33:03 0 1 -73.993896484375 40.76700973510742 -74.0041732788086 40.7421989440918 1 2.08 9 0.5 0.5 1 0 0 0.3 11.3 CSH 0 P h yellow -121 135 1 Manhattan 1013500 E MN15 Clinton 3807 75 83 1 Manhattan 1008300 E MN13 Hudson Yards-Chelsea-Flatiron-Union Square 3807 +1200012546 1 2015-07-15 2015-07-15 11:55:04 2015-07-15 2015-07-15 12:56:34 0 1 -73.8745346069336 40.774078369140625 -73.97222900390625 40.759700775146484 1 8.8 42 0 0.5 0 0 0 0.3 42.8 CRE 0 8 a yellow 48 331 4 Queens 4033100 I QN98 Airport 4102 16 102 1 Manhattan 1010200 I MN17 Midtown-Midtown South 3807 +1200012575 1 2015-07-15 2015-07-15 23:17:44 2015-07-15 2015-07-15 23:27:16 0 1 -73.99248504638672 40.73960494995117 -73.98883056640625 40.77670669555664 1 3.3 11.5 0.5 0.5 2.56 0 0 0.3 15.36 CSH 0 4 C yellow -114 54 1 Manhattan 1005400 I MN13 Hudson Yards-Chelsea-Flatiron-Union Square 3807 206 155 1 Manhattan 1015500 I MN14 Lincoln Square 3806 +1200012596 1 2015-07-16 2015-07-16 05:31:50 2015-07-16 2015-07-16 05:44:17 0 1 -73.94467163085938 40.78008270263672 -74.0099868774414 40.721290588378906 1 9 26 0.5 0.5 3 0 0 0.3 30.3 CSH 0 b 1 yellow 1 152 1 Manhattan 1015200 I MN32 Yorkville 3805 14 39 1 Manhattan 1003900 I MN24 SoHo-TriBeCa-Civic Center-Little Italy 3810 +1200012609 2 2015-07-16 2015-07-16 06:54:31 2015-07-16 2015-07-16 06:57:32 0 1 -73.97875213623047 40.75352478027344 -73.9744644165039 40.76150131225586 1 0.66 4.5 0 0.5 1.59 0 0 0.3 6.89 CSH 0 a a yellow -31 94 1 Manhattan 1009400 E MN17 Midtown-Midtown South 3807 16 102 1 Manhattan 1010200 I MN17 Midtown-Midtown South 3807 +1200012613 1 2015-07-16 2015-07-16 07:02:43 2015-07-16 2015-07-16 07:33:55 0 2 -73.98368072509766 40.743106842041016 -73.78292846679688 40.64398956298828 1 16.3 52 0 0.5 13.2 0 0 0.3 66 CSH 0  2 yellow 11 68 1 Manhattan 1006800 I MN21 Gramercy 3808 8 716 4 Queens 4071600 I QN98 Airport 4105 +1200012642 2 2015-07-16 2015-07-16 09:03:00 2015-07-16 2015-07-16 09:30:25 0 1 -74.008544921875 40.71988296508789 -73.95426177978516 40.76587677001953 1 6.06 23.5 0 0.5 4.86 0 0 0.3 29.16 CSH 0 1 @ yellow 15 33 1 Manhattan 1003300 I MN24 SoHo-TriBeCa-Civic Center-Little Italy 3810 160 124 1 Manhattan 1012400 I MN31 Lenox Hill-Roosevelt Island 3805 +1200012673 2 2015-07-16 2015-07-16 10:37:18 2015-07-16 2015-07-16 10:49:02 0 1 -73.964599609375 40.77578353881836 -73.98319244384766 40.76197052001953 2 1.98 9.5 0 0.5 2 0 0 0.3 12.3 CSH 0 6 0 yellow 25 142 1 Manhattan 1014200 I MN40 Upper East Side-Carnegie Hill 3805 3 131 1 Manhattan 1013100 I MN17 Midtown-Midtown South 3807 +1200012676 1 2015-07-16 2015-07-16 10:49:04 2015-07-16 2015-07-16 11:40:10 0 1 -73.96464538574219 40.806610107421875 -73.94750213623047 40.69928741455078 1 13.9 47 0 0.5 9.55 0 0 0.3 57.35 CSH 0 f  yellow 47 199 1 Manhattan 1019900 I MN09 Morningside Heights 3802 46 257 3 Brooklyn 3025700 E BK75 Bedford 4003 +1200012684 1 2015-07-16 2015-07-16 11:13:18 2015-07-16 2015-07-16 11:30:34 0 1 -73.98259735107422 40.75679397583008 -73.97493743896484 40.787803649902344 1 2.4 13 0 0.5 2.7 0 0 0.3 16.5 CSH 0 a 8 yellow 13 96 1 Manhattan 1009600 E MN17 Midtown-Midtown South 3807 138 175 1 Manhattan 1017500 I MN12 Upper West Side 3806 +1200012715 2 2015-07-16 2015-07-16 12:30:12 2015-07-16 2015-07-16 12:49:31 0 1 -74.01607513427734 40.71118927001953 -73.97911834716797 40.7137336730957 1 3.12 14.5 0 0.5 3.06 0 0 0.3 18.36 CSH 0  2 yellow 54 317.04 1 Manhattan 1031704 I MN25 Battery Park City-Lower Manhattan 3810 99 10.01 1 Manhattan 1001001 I MN28 Lower East Side 3809 +1200012718 1 2015-07-16 2015-07-16 12:43:20 2015-07-16 2015-07-16 12:54:53 0 1 -73.99140930175781 40.75014877319336 -73.99287414550781 40.754390716552734 3 0.5 8 0 0.5 1.75 0 0 0.3 10.55 CSH 0 � h yellow -125 101 1 Manhattan 1010100 I MN17 Midtown-Midtown South 3807 162 111 1 Manhattan 1011100 I MN13 Hudson Yards-Chelsea-Flatiron-Union Square 3807 +1200012734 2 2015-07-16 2015-07-16 14:10:19 2015-07-16 2015-07-16 14:24:55 0 1 -73.99054718017578 40.742069244384766 -73.97679138183594 40.74742889404297 1 1.11 10 0 0.5 2.16 0 0 0.3 12.96 CSH 0 4 p yellow -123 58 1 Manhattan 1005800 I MN13 Hudson Yards-Chelsea-Flatiron-Union Square 3807 160 78 1 Manhattan 1007800 I MN20 Murray Hill-Kips Bay 3808 +1200012750 2 2015-07-16 2015-07-16 14:42:58 2015-07-16 2015-07-16 14:46:10 0 1 -73.9912109375 40.760467529296875 -73.99124908447266 40.765533447265625 1 0.54 4.5 0 0.5 0 0 0 0.3 5.3 CRE 0 H P yellow -117 121 1 Manhattan 1012100 E MN15 Clinton 3807 135 135 1 Manhattan 1013500 E MN15 Clinton 3807 +1200012762 1 2015-07-16 2015-07-16 15:34:02 2015-07-16 2015-07-16 15:53:28 0 1 -73.98131561279297 40.758846282958984 -73.95761108398438 40.77998733520508 1 2.3 13.5 0 0.5 0 0 0 0.3 14.3 CSH 0 0 6 yellow -122 125 1 Manhattan 1012500 I MN17 Midtown-Midtown South 3807 75 150.01 1 Manhattan 1015001 I MN40 Upper East Side-Carnegie Hill 3805 +1200012802 1 2015-07-16 2015-07-16 18:15:21 2015-07-16 2015-07-16 19:05:25 0 1 -73.95704650878906 40.77973556518555 -73.95276641845703 40.764102935791016 1 14.1 45.5 1 0.5 0 0 0 0.3 47.3 CRE 0 6 @ yellow 28 148.01 1 Manhattan 1014801 I MN40 Upper East Side-Carnegie Hill 3805 222 116 1 Manhattan 1011600 I MN31 Lenox Hill-Roosevelt Island 3805 +1200012823 2 2015-07-16 2015-07-16 15:03:24 2015-07-16 2015-07-16 15:14:45 0 1 -73.99564361572266 40.75944137573242 -74.0077896118164 40.747310638427734 5 1.69 9.5 0 0.5 2.58 0 0 0.3 12.88 CSH 0 F F yellow 51 117 1 Manhattan 1011700 I MN13 Hudson Yards-Chelsea-Flatiron-Union Square 3807 50 99 1 Manhattan 1009900 I MN13 Hudson Yards-Chelsea-Flatiron-Union Square 3807 +1200012837 2 2015-07-16 2015-07-16 19:40:12 2015-07-16 2015-07-16 19:56:19 0 1 -74.00247955322266 40.75017166137695 -73.97418975830078 40.752132415771484 2 2.08 12 1 0.5 2 0 0 0.3 15.8 CSH 0 F b yellow 50 99 1 Manhattan 1009900 I MN13 Hudson Yards-Chelsea-Flatiron-Union Square 3807 226 92 1 Manhattan 1009200 I MN19 Turtle Bay-East Midtown 3808 +1200012859 2 2015-07-16 2015-07-16 20:41:29 2015-07-16 2015-07-16 20:47:25 0 1 -74.00914001464844 40.70537185668945 -74.00589752197266 40.71965026855469 1 1.18 6 0.5 0.5 1 0 0 0.3 8.3 CSH 0 � 1 yellow -1 7 1 Manhattan 1000700 I MN25 Battery Park City-Lower Manhattan 3810 15 33 1 Manhattan 1003300 I MN24 SoHo-TriBeCa-Civic Center-Little Italy 3810 +1200012872 2 2015-07-16 2015-07-16 21:30:58 2015-07-16 2015-07-16 21:35:39 0 1 -73.99324798583984 40.73508071899414 -74.0041275024414 40.741539001464844 2 0.82 5.5 0.5 0.5 1.36 0 0 0.3 8.16 CSH 0  h yellow -55 61 1 Manhattan 1006100 I MN23 West Village 3810 75 83 1 Manhattan 1008300 E MN13 Hudson Yards-Chelsea-Flatiron-Union Square 3807 +1200012874 2 2015-07-16 2015-07-16 21:32:24 2015-07-16 2015-07-16 21:38:38 0 1 -73.99430847167969 40.75034713745117 -73.9875259399414 40.76047134399414 1 0.84 6 0.5 0.5 1.46 0 0 0.3 8.76 CSH 0 � H yellow -125 101 1 Manhattan 1010100 I MN17 Midtown-Midtown South 3807 138 127 1 Manhattan 1012700 I MN15 Clinton 3807 +1200012895 2 2015-07-16 2015-07-16 22:22:43 2015-07-16 2015-07-16 22:47:00 0 1 -73.9941177368164 40.75121307373047 -73.91242980957031 40.8326416015625 5 11.1 32.5 0.5 0.5 3 0 0 0.3 36.8 CSH 0 � i yellow -125 101 1 Manhattan 1010100 I MN17 Midtown-Midtown South 3807 205 175 2 Bronx 2017500 E BX14 East Concourse-Concourse Village 3708 +1200012901 1 2015-07-16 2015-07-16 22:32:34 2015-07-16 2015-07-16 22:43:00 0 1 -74.01545715332031 40.715309143066406 -74.00019073486328 40.72674560546875 1 1.7 9 0.5 0.5 2.05 0 0 0.3 12.35 CSH 0   yellow 52 317.03 1 Manhattan 1031703 I MN25 Battery Park City-Lower Manhattan 3810 35 49 1 Manhattan 1004900 I MN24 SoHo-TriBeCa-Civic Center-Little Italy 3810 +1200012903 2 2015-07-16 2015-07-16 22:38:09 2015-07-16 2015-07-16 22:55:32 0 1 -73.92902374267578 40.79774475097656 -73.91374969482422 40.842506408691406 1 6.75 21 0.5 0.5 5.58 0 0 0.3 27.88 CSH 0 t G yellow -24 192 1 Manhattan 1019200 E MN34 East Harlem North 3804 68 209 2 Bronx 2020900 E BX63 West Concourse 3708 +1200012962 1 2015-07-17 2015-07-17 01:05:07 2015-07-17 2015-07-17 01:07:27 0 1 -73.95323944091797 40.77570724487305 -73.95514678955078 40.76594161987305 1 0.9 4.5 0.5 0.5 1.15 0 0 0.3 6.95 CSH 0 c @ yellow 0 138 1 Manhattan 1013800 I MN32 Yorkville 3805 160 124 1 Manhattan 1012400 I MN31 Lenox Hill-Roosevelt Island 3805 +1200012984 2 2015-07-17 2015-07-17 04:11:45 2015-07-17 2015-07-17 04:31:13 0 1 -73.95396423339844 40.77031326293945 -73.94390869140625 40.70798873901367 1 7.97 24.5 0.5 0.5 4.5 0 0 0.3 30.3 CSH 0 @ � yellow -95 132 1 Manhattan 1013200 I MN31 Lenox Hill-Roosevelt Island 3805 244 505 3 Brooklyn 3050500 E BK90 East Williamsburg 4001 +1200013003 2 2015-07-17 2015-07-17 06:46:45 2015-07-17 2015-07-17 07:11:14 0 1 -73.97273254394531 40.75630569458008 -73.87294006347656 40.77426528930664 1 9.27 29 0 0.5 10.6 5.54 0 0.3 45.94 CSH 0 b 8 yellow 15 100 1 Manhattan 1010000 I MN19 Turtle Bay-East Midtown 3808 48 331 4 Queens 4033100 I QN98 Airport 4102 +1200013036 1 2015-07-17 2015-07-17 09:01:20 2015-07-17 2015-07-17 09:14:36 1 1 -73.9176254272461 40.746578216552734 -73.97099304199219 40.75062561035156 1 3.7 13.5 0 0.5 0 0 0 0.3 14.3 CRE 0 & 3 yellow 69 169 4 Queens 4016900 I QN31 Hunters Point-Sunnyside-West Maspeth 4109 151 88 1 Manhattan 1008800 I MN19 Turtle Bay-East Midtown 3808 +1200013059 2 2015-07-17 2015-07-17 10:56:13 2015-07-17 2015-07-17 11:12:59 0 1 -73.97270965576172 40.755035400390625 -73.98467254638672 40.74849319458008 1 1.4 11 0 0.5 0 0 0 0.3 11.8 CRE 0 b d yellow -30 92 1 Manhattan 1009200 I MN19 Turtle Bay-East Midtown 3808 208 76 1 Manhattan 1007600 I MN17 Midtown-Midtown South 3807 +1200013065 1 2015-07-17 2015-07-17 11:15:05 2015-07-17 2015-07-17 11:17:50 0 1 -73.96148681640625 40.77415466308594 -73.95537567138672 40.782615661621094 1 0.6 4.5 0 0.5 0 0 0 0.3 5.3 CSH 0 7 6 yellow 18 128 1 Manhattan 1012800 I MN40 Upper East Side-Carnegie Hill 3805 29 148.02 1 Manhattan 1014802 I MN40 Upper East Side-Carnegie Hill 3805 +1200013072 2 2015-07-17 2015-07-17 11:51:15 2015-07-17 2015-07-17 12:00:54 0 1 -73.95521545410156 40.777305603027344 -73.96537017822266 40.770904541015625 1 1.04 8 0 0.5 2.2 0 0 0.3 11 CSH 0 6 7 yellow 24 140 1 Manhattan 1014000 I MN40 Upper East Side-Carnegie Hill 3805 23 130 1 Manhattan 1013000 I MN40 Upper East Side-Carnegie Hill 3805 +1200013093 1 2015-07-17 2015-07-17 13:12:26 2015-07-17 2015-07-17 13:20:51 0 1 -74.0012435913086 40.72073745727539 -74.01473236083984 40.718101501464844 1 1 7 0 0.5 1.56 0 0 0.3 9.36 CSH 0   yellow 16 47 1 Manhattan 1004700 I MN24 SoHo-TriBeCa-Civic Center-Little Italy 3810 52 317.03 1 Manhattan 1031703 I MN25 Battery Park City-Lower Manhattan 3810 +1200013113 2 2015-07-17 2015-07-17 14:27:39 2015-07-17 2015-07-17 15:23:39 0 2 -73.97443389892578 40.75407028198242 -73.790771484375 40.644493103027344 2 15.63 52 0 0.5 14.58 5.54 0 0.3 72.92 CSH 0 b 2 yellow -30 92 1 Manhattan 1009200 I MN19 Turtle Bay-East Midtown 3808 8 716 4 Queens 4071600 I QN98 Airport 4105 +1200013116 1 2015-07-17 2015-07-17 14:34:28 2015-07-17 2015-07-17 14:42:20 0 1 -73.96299743652344 40.80421829223633 -73.97064971923828 40.783660888671875 1 1.8 8 0 0.5 0 0 0 0.3 8.8 CRE 0 f 9 yellow 47 199 1 Manhattan 1019900 I MN09 Morningside Heights 3802 136 169 1 Manhattan 1016900 I MN12 Upper West Side 3806 +1200013122 2 2015-07-17 2015-07-17 14:55:05 2015-07-17 2015-07-17 15:03:38 0 1 -73.99925994873047 40.747127532958984 -74.00193786621094 40.747135162353516 1 0.21 6.5 0 0.5 0 0 0 0.3 7.3 CRE 0 h h yellow 113 93 1 Manhattan 1009300 I MN13 Hudson Yards-Chelsea-Flatiron-Union Square 3807 113 93 1 Manhattan 1009300 I MN13 Hudson Yards-Chelsea-Flatiron-Union Square 3807 +1200013187 2 2015-07-17 2015-07-17 19:14:45 2015-07-17 2015-07-17 19:23:33 0 1 -73.99564361572266 40.73728561401367 -74.00769805908203 40.752010345458984 1 1.5 8 1 0.5 1 0 0 0.3 10.8 CSH 0 4 F yellow -114 54 1 Manhattan 1005400 I MN13 Hudson Yards-Chelsea-Flatiron-Union Square 3807 50 99 1 Manhattan 1009900 I MN13 Hudson Yards-Chelsea-Flatiron-Union Square 3807 +1200013193 2 2015-07-17 2015-07-17 19:35:31 2015-07-17 2015-07-17 19:39:57 0 1 -73.98542022705078 40.7275505065918 -73.99065399169922 40.724613189697266 1 0.97 5.5 1 0.5 1 0 0 0.3 8.3 CSH 0 y y yellow -66 38 1 Manhattan 1003800 I MN22 East Village 3809 194 36.02 1 Manhattan 1003602 I MN22 East Village 3809 +1200013220 1 2015-07-17 2015-07-17 20:58:53 2015-07-17 2015-07-17 21:04:44 0 1 -73.98184967041016 40.77083206176758 -73.97509002685547 40.79058837890625 1 1.6 7 0.5 0.5 0 0 0 0.3 8.3 CRE 0 B 8 yellow 67 149 1 Manhattan 1014900 I MN14 Lincoln Square 3806 138 175 1 Manhattan 1017500 I MN12 Upper West Side 3806 +1200013225 1 2015-07-17 2015-07-17 21:12:03 2015-07-17 2015-07-17 21:40:37 0 1 -73.96880340576172 40.75941467285156 -74.00804901123047 40.738887786865234 1 3.3 19 0.5 0.5 4.05 0 0 0.3 24.35 CSH 0 b X yellow 15 100 1 Manhattan 1010000 I MN19 Turtle Bay-East Midtown 3808 79 79 1 Manhattan 1007900 I MN23 West Village 3810 +1200013256 2 2015-07-17 2015-07-17 22:38:37 2015-07-17 2015-07-17 22:54:45 0 1 -73.96024322509766 40.776458740234375 -73.9864501953125 40.746761322021484 2 2.68 12.5 0.5 0.5 2.76 0 0 0.3 16.56 CSH 0 6 d yellow 25 142 1 Manhattan 1014200 I MN40 Upper East Side-Carnegie Hill 3805 208 76 1 Manhattan 1007600 I MN17 Midtown-Midtown South 3807 +1200013285 1 2015-07-16 2015-07-16 21:39:46 2015-07-16 2015-07-16 22:04:05 0 1 -74.00553131103516 40.7371940612793 -73.96498107910156 40.761749267578125 2 4.2 18 0.5 0.5 0 0 0 0.3 19.3 CRE 0 I A yellow 78 77 1 Manhattan 1007700 I MN23 West Village 3810 221 110 1 Manhattan 1011000 I MN31 Lenox Hill-Roosevelt Island 3805 +1200013297 1 2015-07-18 2015-07-18 00:34:04 2015-07-18 2015-07-18 00:36:56 0 1 -73.94418334960938 40.788124084472656 -73.95409393310547 40.77787780761719 1 0.9 5 0.5 0.5 0.7 0 0 0.3 7 CSH 0 u c yellow 107 164 1 Manhattan 1016400 E MN33 East Harlem South 3804 26 146.01 1 Manhattan 1014601 I MN32 Yorkville 3805 +1200013316 1 2015-07-18 2015-07-18 01:26:09 2015-07-18 2015-07-18 01:39:45 0 1 -73.98978424072266 40.72000503540039 -74.00891876220703 40.740745544433594 1 2.4 12 0.5 0.5 0 0 0 0.3 13.3 CRE 0 H X yellow -61 18 1 Manhattan 1001800 E MN27 Chinatown 3809 79 79 1 Manhattan 1007900 I MN23 West Village 3810 +1200013325 1 2015-07-18 2015-07-18 01:38:35 2015-07-18 2015-07-18 01:56:46 0 1 -74.00861358642578 40.7404899597168 -73.97271728515625 40.7564697265625 1 3.5 15 0.5 0.5 3.25 0 0 0.3 19.55 CSH 0 X b yellow 79 79 1 Manhattan 1007900 I MN23 West Village 3810 15 100 1 Manhattan 1010000 I MN19 Turtle Bay-East Midtown 3808 +1200013337 2 2015-07-18 2015-07-18 02:01:03 2015-07-18 2015-07-18 02:17:39 0 1 -73.99165344238281 40.731842041015625 -73.98002624511719 40.68206024169922 1 3.99 15.5 0.5 0.5 3.36 0 0 0.3 20.16 CSH 0  % yellow -55 61 1 Manhattan 1006100 I MN23 West Village 3810 82 127 3 Brooklyn 3012700 E BK38 DUMBO-Vinegar Hill-Downtown Brooklyn-Boerum Hill 4004 +1200013353 2 2015-07-18 2015-07-18 03:16:45 2015-07-18 2015-07-18 03:21:17 0 1 -73.95804595947266 40.77914810180664 -73.96202087402344 40.76852035522461 2 0.93 5.5 0.5 0.5 0 0 0 0.3 6.8 CSH 0 6 7 yellow 75 150.01 1 Manhattan 1015001 I MN40 Upper East Side-Carnegie Hill 3805 17 120 1 Manhattan 1012000 I MN40 Upper East Side-Carnegie Hill 3805 +1200013366 1 2015-07-18 2015-07-18 06:06:56 2015-07-18 2015-07-18 06:10:17 0 1 -73.98678588867188 40.747657775878906 -73.98717498779297 40.750244140625 4 0.4 4.5 0 0.5 1.05 0 0 0.3 6.35 CSH 0 d d yellow -48 76 1 Manhattan 1007600 I MN17 Midtown-Midtown South 3807 208 76 1 Manhattan 1007600 I MN17 Midtown-Midtown South 3807 +1200013403 2 2015-07-18 2015-07-18 10:02:27 2015-07-18 2015-07-18 10:10:02 0 1 -73.96183013916016 40.7642707824707 -73.97834777832031 40.764801025390625 1 1.15 7 0 0.5 1.56 0 0 0.3 9.36 CSH 0 A c yellow -74 118 1 Manhattan 1011800 I MN31 Lenox Hill-Roosevelt Island 3805 4 137 1 Manhattan 1013700 I MN17 Midtown-Midtown South 3807 +1200013473 2 2015-07-18 2015-07-18 00:16:43 2015-07-18 2015-07-18 00:29:35 0 1 -73.9776611328125 40.743839263916016 -73.96827697753906 40.757999420166016 1 1.3900000000000001 9.5 0.5 0.5 3.24 0 0 0.3 14.04 CSH 0 7 ) yellow -10 70 1 Manhattan 1007000 I MN20 Murray Hill-Kips Bay 3808 14 98 1 Manhattan 1009800 I MN19 Turtle Bay-East Midtown 3808 +1200013492 1 2015-07-18 2015-07-18 15:21:32 2015-07-18 2015-07-18 15:32:31 0 1 -73.96791076660156 40.76285171508789 -73.98600006103516 40.76316833496094 1 1.2 8.5 0 0.5 1 0 0 0.3 10.3 CSH 0 7 H yellow 22 114.02 1 Manhattan 1011402 I MN40 Upper East Side-Carnegie Hill 3805 137 133 1 Manhattan 1013300 I MN15 Clinton 3807 +1200013503 2 2015-07-18 2015-07-18 16:05:00 2015-07-18 2015-07-18 16:24:00 0 1 -73.96934509277344 40.785400390625 -73.9749984741211 40.753273010253906 1 2.89 15 0 0.5 0 0 0 0.3 15.8 CRE 0 8 b yellow -119 173 1 Manhattan 1017300 I MN12 Upper West Side 3806 226 92 1 Manhattan 1009200 I MN19 Turtle Bay-East Midtown 3808 +1200013528 2 2015-07-18 2015-07-18 17:48:41 2015-07-18 2015-07-18 17:57:38 0 1 -73.98809814453125 40.7458381652832 -74.00717163085938 40.74327087402344 5 1.54 8 0 0.5 0 0 0 0.3 8.8 CRE 0 d h yellow -48 76 1 Manhattan 1007600 I MN17 Midtown-Midtown South 3807 75 83 1 Manhattan 1008300 E MN13 Hudson Yards-Chelsea-Flatiron-Union Square 3807 +1200013548 2 2015-07-18 2015-07-18 18:18:18 2015-07-18 2015-07-18 18:26:55 0 1 -73.976806640625 40.758567810058594 -73.9819564819336 40.77202606201172 1 1.41 7.5 0 0.5 0 0 0 0.3 8.3 CRE 0 a B yellow 16 102 1 Manhattan 1010200 I MN17 Midtown-Midtown South 3807 67 149 1 Manhattan 1014900 I MN14 Lincoln Square 3806 +1200013557 2 2015-07-18 2015-07-18 18:35:13 2015-07-18 2015-07-18 18:42:36 0 1 -73.96273040771484 40.80465316772461 -73.97792053222656 40.783084869384766 1 2.01 8.5 0 0.5 1 0 0 0.3 10.3 CSH 0 f 9 yellow 47 199 1 Manhattan 1019900 I MN09 Morningside Heights 3802 22 165 1 Manhattan 1016500 I MN12 Upper West Side 3806 +1200013574 2 2015-07-18 2015-07-18 19:32:08 2015-07-18 2015-07-18 19:51:54 0 1 -73.99077606201172 40.7373161315918 -73.98632049560547 40.750343322753906 1 1.96 13.5 0 0.5 2.86 0 0 0.3 17.16 CSH 0 4 d yellow -11 52 1 Manhattan 1005200 I MN13 Hudson Yards-Chelsea-Flatiron-Union Square 3807 209 84 1 Manhattan 1008400 I MN17 Midtown-Midtown South 3807 +1200013624 1 2015-07-18 2015-07-18 21:33:05 2015-07-18 2015-07-18 21:37:27 0 1 -73.97928619384766 40.77662658691406 -73.97762298583984 40.78668975830078 2 1 5.5 0.5 0.5 0 0 0 0.3 6.8 CRE 0 B 9 yellow 21 157 1 Manhattan 1015700 I MN14 Lincoln Square 3806 166 171 1 Manhattan 1017100 I MN12 Upper West Side 3806 +1200013639 1 2015-07-18 2015-07-18 22:12:51 2015-07-18 2015-07-18 22:14:55 0 1 -73.95818328857422 40.764671325683594 -73.95277404785156 40.772281646728516 2 0.6 4 0.5 0.5 1.05 0 0 0.3 6.35 CSH 0 @ A yellow -34 116 1 Manhattan 1011600 I MN31 Lenox Hill-Roosevelt Island 3805 218 134 1 Manhattan 1013400 I MN31 Lenox Hill-Roosevelt Island 3805 +1200013677 1 2015-07-18 2015-07-18 23:53:48 2015-07-19 2015-07-19 00:08:48 0 1 -73.96012115478516 40.76203918457031 -73.98279571533203 40.763877868652344 1 1.6 10.5 0.5 0.5 2.2 0 0 0.3 14 CSH 0 @ 0 yellow -36 106.02 1 Manhattan 1010602 I MN31 Lenox Hill-Roosevelt Island 3805 3 131 1 Manhattan 1013100 I MN17 Midtown-Midtown South 3807 +1200013680 1 2015-07-19 2015-07-19 00:08:52 2015-07-19 2015-07-19 00:24:18 0 1 -73.99092864990234 40.73918914794922 -73.9936294555664 40.7200927734375 4 1.7 11.5 0.5 0.5 2 0 0 0.3 14.8 CSH 0 4 H yellow -11 52 1 Manhattan 1005200 I MN13 Hudson Yards-Chelsea-Flatiron-Union Square 3807 195 18 1 Manhattan 1001800 E MN27 Chinatown 3809 +1200013682 2 2015-07-19 2015-07-19 00:10:04 2015-07-19 2015-07-19 00:27:44 0 1 -73.99114990234375 40.75525665283203 -73.91960906982422 40.74321365356445 2 4.79 17.5 0.5 0.5 0 0 0 0.3 18.8 CRE 0  & yellow 21 113 1 Manhattan 1011300 I MN17 Midtown-Midtown South 3807 18 183 4 Queens 4018300 I QN31 Hunters Point-Sunnyside-West Maspeth 4109 +1200013708 1 2015-07-19 2015-07-19 01:07:07 2015-07-19 2015-07-19 01:09:05 0 1 -73.96861267089844 40.79607391357422 -73.96228790283203 40.80003356933594 1 0.7 4 0.5 0.5 1.05 0 0 0.3 6.35 CSH 0 Q $ yellow 25 185 1 Manhattan 1018500 I MN12 Upper West Side 3806 110 193 1 Manhattan 1019300 E MN09 Morningside Heights 3802 +1200013715 1 2015-07-19 2015-07-19 01:33:34 2015-07-19 2015-07-19 01:39:39 0 1 -73.9819107055664 40.750732421875 -73.99273681640625 40.74128723144531 4 1.2 6.5 0.5 0.5 0 0 0 0.3 7.8 CRE 0 p 4 yellow 93 82 1 Manhattan 1008200 I MN20 Murray Hill-Kips Bay 3808 133 58 1 Manhattan 1005800 I MN13 Hudson Yards-Chelsea-Flatiron-Union Square 3807 +1200013720 2 2015-07-19 2015-07-19 01:38:09 2015-07-19 2015-07-19 01:47:23 0 1 -74.00508117675781 40.71910095214844 -73.98216247558594 40.75608825683594 5 2.93 10.5 0.5 0.5 3.54 0 0 0.3 15.34 CSH 0 1 a yellow 15 33 1 Manhattan 1003300 I MN24 SoHo-TriBeCa-Civic Center-Little Italy 3810 13 96 1 Manhattan 1009600 E MN17 Midtown-Midtown South 3807 +1200013735 2 2015-07-19 2015-07-19 02:44:48 2015-07-19 2015-07-19 02:51:08 0 1 -74.00657653808594 40.74422836303711 -73.98474884033203 40.75392150878906 1 1.83 7.5 0.5 0.5 1.76 0 0 0.3 10.56 CSH 0 h d yellow 75 83 1 Manhattan 1008300 E MN13 Hudson Yards-Chelsea-Flatiron-Union Square 3807 209 84 1 Manhattan 1008400 I MN17 Midtown-Midtown South 3807 +1200013764 1 2015-07-19 2015-07-19 06:31:43 2015-07-19 2015-07-19 06:40:27 0 1 -73.99662780761719 40.72126388549805 -73.99100494384766 40.75005340576172 1 2.5 10 0 0.5 2.16 0 0 0.3 12.96 CSH 0 D � yellow -60 41 1 Manhattan 1004100 E MN24 SoHo-TriBeCa-Civic Center-Little Italy 3810 131 101 1 Manhattan 1010100 I MN17 Midtown-Midtown South 3807 +1200013797 1 2015-07-19 2015-07-19 10:42:31 2015-07-19 2015-07-19 10:51:01 0 1 -74.00263214111328 40.72315216064453 -73.99066925048828 40.73972702026367 4 1.8 8.5 0 0.5 0 0 0 0.3 9.3 CRE 0  4 yellow 16 47 1 Manhattan 1004700 I MN24 SoHo-TriBeCa-Civic Center-Little Italy 3810 245 52 1 Manhattan 1005200 I MN13 Hudson Yards-Chelsea-Flatiron-Union Square 3807 +1200013828 1 2015-07-19 2015-07-19 12:28:56 2015-07-19 2015-07-19 12:35:59 0 1 -73.97154998779297 40.761924743652344 -73.9829330444336 40.76902770996094 1 0.9 6.5 0 0.5 0 0 0 0.3 7.3 CRE 0 c B yellow 19 112.02 1 Manhattan 1011202 I MN17 Midtown-Midtown South 3807 66 145 1 Manhattan 1014500 I MN14 Lincoln Square 3806 +1200013881 2 2015-07-19 2015-07-19 14:52:57 2015-07-19 2015-07-19 14:57:26 0 1 -73.99624633789062 40.725860595703125 -73.9975814819336 40.72092056274414 1 0.59 4.5 0 0.5 0 0 0 0.3 5.3 CRE 0  D yellow -65 55.02 1 Manhattan 1005502 I MN23 West Village 3810 196 41 1 Manhattan 1004100 E MN24 SoHo-TriBeCa-Civic Center-Little Italy 3810 +1200013902 1 2015-07-19 2015-07-19 16:04:22 2015-07-19 2015-07-19 16:21:21 0 1 -73.98191833496094 40.7724494934082 -74.0041275024414 40.73043441772461 2 4.3 15 0 0.5 3.15 0 0 0.3 18.95 CSH 0 B I yellow 67 149 1 Manhattan 1014900 I MN14 Lincoln Square 3806 36 67 1 Manhattan 1006700 I MN23 West Village 3810 +1200013924 2 2015-07-19 2015-07-19 17:08:38 2015-07-19 2015-07-19 17:14:22 0 1 -74.01626586914062 40.71548843383789 -74.00411987304688 40.71787643432617 1 0.89 5.5 0 0.5 1.26 0 0 0.3 7.56 CSH 0  1 yellow 52 317.03 1 Manhattan 1031703 I MN25 Battery Park City-Lower Manhattan 3810 15 33 1 Manhattan 1003300 I MN24 SoHo-TriBeCa-Civic Center-Little Italy 3810 +1200013925 1 2015-07-19 2015-07-19 17:03:52 2015-07-19 2015-07-19 17:04:55 0 1 -73.95306396484375 40.76787185668945 -73.95862579345703 40.769805908203125 1 0.3 3 0 0.5 0 0 0 0.3 3.8 CRE 0 @ A yellow -96 124 1 Manhattan 1012400 I MN31 Lenox Hill-Roosevelt Island 3805 183 126 1 Manhattan 1012600 I MN31 Lenox Hill-Roosevelt Island 3805 +1200013947 2 2015-07-19 2015-07-19 18:16:48 2015-07-19 2015-07-19 18:21:43 0 1 -74.0042724609375 40.751712799072266 -74.0002670288086 40.737815856933594 1 1.42 6 0 0.5 1.36 0 0 0.3 8.16 CSH 0 F I yellow 50 99 1 Manhattan 1009900 I MN13 Hudson Yards-Chelsea-Flatiron-Union Square 3807 78 77 1 Manhattan 1007700 I MN23 West Village 3810 +1200013948 2 2015-07-19 2015-07-19 18:20:03 2015-07-19 2015-07-19 18:29:34 0 1 -73.97511291503906 40.75524139404297 -73.95480346679688 40.78049850463867 1 2.39 10 0 0.5 0 0 0 0.3 10.8 CRE 0 b 6 yellow -30 92 1 Manhattan 1009200 I MN19 Turtle Bay-East Midtown 3808 29 148.02 1 Manhattan 1014802 I MN40 Upper East Side-Carnegie Hill 3805 +1200014031 2 2015-07-19 2015-07-19 22:14:28 2015-07-19 2015-07-19 22:25:28 0 1 -73.97383880615234 40.76411819458008 -73.98551177978516 40.78007888793945 1 1.99 9.5 0.5 0.5 0 0 0 0.3 10.8 CRE 0 c C yellow 18 112.01 1 Manhattan 1011201 I MN17 Midtown-Midtown South 3807 154 159 1 Manhattan 1015900 I MN14 Lincoln Square 3806 +1200014067 2 2015-07-20 2015-07-20 00:42:00 2015-07-20 2015-07-20 00:49:13 0 1 -73.98690032958984 40.74458312988281 -73.9841079711914 40.75906753540039 1 1.42 7 0.5 0.5 0 0 0 0.3 8.3 CRE 0 d 0 yellow -125 74 1 Manhattan 1007400 I MN17 Midtown-Midtown South 3807 134 125 1 Manhattan 1012500 I MN17 Midtown-Midtown South 3807 +1200014071 1 2015-07-20 2015-07-20 01:26:25 2015-07-20 2015-07-20 01:42:13 0 1 -74.00743865966797 40.74113082885742 -73.94767761230469 40.77507400512695 1 5.6 18 0.5 0.5 4.8 0 0 0.3 24.1 CSH 0 X b yellow 79 79 1 Manhattan 1007900 I MN23 West Village 3810 152 144.01 1 Manhattan 1014401 I MN32 Yorkville 3805 +1200014109 2 2015-07-20 2015-07-20 07:38:00 2015-07-20 2015-07-20 07:54:00 0 1 -73.99556732177734 40.74399185180664 -74.0135498046875 40.713775634765625 5 3.18 13.5 0 0.5 2.86 0 0 0.3 17.16 CSH 0 � 1 yellow -124 91 1 Manhattan 1009100 I MN13 Hudson Yards-Chelsea-Flatiron-Union Square 3807 47 21 1 Manhattan 1002100 I MN24 SoHo-TriBeCa-Civic Center-Little Italy 3810 +1200014119 2 2015-07-20 2015-07-20 08:13:07 2015-07-20 2015-07-20 08:22:12 0 1 -73.993408203125 40.72946548461914 -74.00962829589844 40.72243118286133 1 1.67 8.5 0 0.5 1 0 0 0.3 10.3 CSH 0  1 yellow -64 57 1 Manhattan 1005700 I MN23 West Village 3810 14 39 1 Manhattan 1003900 I MN24 SoHo-TriBeCa-Civic Center-Little Italy 3810 +1200014125 2 2015-07-20 2015-07-20 08:22:09 2015-07-20 2015-07-20 08:40:19 0 1 -73.97222900390625 40.749549865722656 -73.99272155761719 40.73027420043945 1 2.91 14 0 0.5 0 0 0 0.3 14.8 CRE 0 3  yellow -105 88 1 Manhattan 1008800 I MN19 Turtle Bay-East Midtown 3808 192 57 1 Manhattan 1005700 I MN23 West Village 3810 +1200014142 1 2015-07-20 2015-07-20 09:22:09 2015-07-20 2015-07-20 09:27:42 0 1 -73.99312591552734 40.76286315917969 -73.9753189086914 40.77931213378906 1 1.7 7 0 0.5 1.55 0 0 0.3 9.35 CSH 0 P 9 yellow -116 129 1 Manhattan 1012900 I MN15 Clinton 3807 63 161 1 Manhattan 1016100 I MN12 Upper West Side 3806 +1200014160 2 2015-07-20 2015-07-20 10:10:45 2015-07-20 2015-07-20 10:38:40 0 1 -73.99210357666016 40.75017547607422 -73.96321105957031 40.76360321044922 2 2.58 17.5 0 0.5 0 0 0 0.3 18.3 CRE 0 � A yellow -125 101 1 Manhattan 1010100 I MN17 Midtown-Midtown South 3807 221 110 1 Manhattan 1011000 I MN31 Lenox Hill-Roosevelt Island 3805 +1200014168 2 2015-07-20 2015-07-20 10:26:59 2015-07-20 2015-07-20 10:44:34 0 1 -73.98513793945312 40.768951416015625 -73.98016357421875 40.754188537597656 1 1.34 11.5 0 0.5 0 0 0 0.3 12.3 CRE 0 B a yellow 66 145 1 Manhattan 1014500 I MN14 Lincoln Square 3806 225 94 1 Manhattan 1009400 E MN17 Midtown-Midtown South 3807 +1200014240 1 2015-07-20 2015-07-20 14:04:02 2015-07-20 2015-07-20 14:09:50 0 1 -73.96493530273438 40.75581741333008 -73.97367858886719 40.75627899169922 1 0.6 5.5 0 0.5 1.25 0 0 0.3 7.55 CSH 0 ) b yellow 14 98 1 Manhattan 1009800 I MN19 Turtle Bay-East Midtown 3808 15 100 1 Manhattan 1010000 I MN19 Turtle Bay-East Midtown 3808 +1200014247 2 2015-07-20 2015-07-20 14:32:38 2015-07-20 2015-07-20 14:54:26 0 1 -73.96479797363281 40.755332946777344 -73.97591400146484 40.74570846557617 1 1.2 13.5 0 0.5 2 0 0 0.3 16.3 CSH 0 ) p yellow 104 86.03 1 Manhattan 1008603 I MN19 Turtle Bay-East Midtown 3808 160 78 1 Manhattan 1007800 I MN20 Murray Hill-Kips Bay 3808 +1200014268 2 2015-07-20 2015-07-20 15:33:50 2015-07-20 2015-07-20 16:07:08 0 1 -74.01487731933594 40.71420669555664 -73.95170593261719 40.78445053100586 6 9.13 30 0 0.5 3 0 0 0.3 33.8 CSH 0  6 yellow 52 317.03 1 Manhattan 1031703 I MN25 Battery Park City-Lower Manhattan 3810 153 158.01 1 Manhattan 1015801 I MN40 Upper East Side-Carnegie Hill 3805 +1200014275 2 2015-07-20 2015-07-20 13:33:07 2015-07-20 2015-07-20 13:39:19 0 1 -73.97437286376953 40.75111770629883 -73.97437286376953 40.75111770629883 1 0.7 5.5 0 0.5 1.26 0 0 0.3 7.56 CSH 0 b b yellow -30 92 1 Manhattan 1009200 I MN19 Turtle Bay-East Midtown 3808 226 92 1 Manhattan 1009200 I MN19 Turtle Bay-East Midtown 3808 +1200014289 2 2015-07-20 2015-07-20 16:36:56 2015-07-20 2015-07-20 16:53:02 0 1 -73.98209381103516 40.760860443115234 -74.0055160522461 40.750938415527344 6 2.18 12 1 0.5 2.76 0 0 0.3 16.56 CSH 0 0 F yellow 3 131 1 Manhattan 1013100 I MN17 Midtown-Midtown South 3807 50 99 1 Manhattan 1009900 I MN13 Hudson Yards-Chelsea-Flatiron-Union Square 3807 +1200014294 2 2015-07-20 2015-07-20 16:43:52 2015-07-20 2015-07-20 16:49:53 0 1 -73.96342468261719 40.772003173828125 -73.95518493652344 40.78571319580078 2 1.15 6.5 1 0.5 2.08 0 0 0.3 10.38 CSH 0 7 6 yellow 23 130 1 Manhattan 1013000 I MN40 Upper East Side-Carnegie Hill 3805 34 160.01 1 Manhattan 1016001 I MN40 Upper East Side-Carnegie Hill 3805 +1200014307 2 2015-07-20 2015-07-20 17:17:14 2015-07-20 2015-07-20 17:25:16 0 1 -73.98757934570312 40.743492126464844 -73.99929809570312 40.73173141479492 1 1.43 7.5 1 0.5 0 0 0 0.3 9.3 CRE 0 4  yellow -90 56 1 Manhattan 1005600 I MN13 Hudson Yards-Chelsea-Flatiron-Union Square 3807 37 65 1 Manhattan 1006500 I MN23 West Village 3810 +1200014327 1 2015-07-20 2015-07-20 18:17:09 2015-07-20 2015-07-20 18:23:21 0 1 -73.99533081054688 40.72724533081055 -74.00321197509766 40.72646713256836 1 0.5 5.5 1 0.5 1.82 0 0 0.3 9.12 CSH 0   yellow -97 55.01 1 Manhattan 1005501 I MN23 West Village 3810 35 49 1 Manhattan 1004900 I MN24 SoHo-TriBeCa-Civic Center-Little Italy 3810 +1200014347 2 2015-07-20 2015-07-20 19:15:01 2015-07-20 2015-07-20 19:21:51 0 1 -73.98381042480469 40.74620056152344 -73.97657012939453 40.755409240722656 6 0.73 6 1 0.5 0 0 0 0.3 7.8 CRE 0 d a yellow -125 74 1 Manhattan 1007400 I MN17 Midtown-Midtown South 3807 225 94 1 Manhattan 1009400 E MN17 Midtown-Midtown South 3807 +1200014361 1 2015-07-20 2015-07-20 19:36:29 2015-07-20 2015-07-20 19:44:09 0 1 -73.99071502685547 40.742252349853516 -73.98563385009766 40.731327056884766 1 1.1 7 1 0.5 1.75 0 0 0.3 10.55 CSH 0 4 y yellow -123 58 1 Manhattan 1005800 I MN13 Hudson Yards-Chelsea-Flatiron-Union Square 3807 93 40 1 Manhattan 1004000 I MN22 East Village 3809 +1200014428 2 2015-07-20 2015-07-20 22:27:47 2015-07-20 2015-07-20 22:30:50 0 1 -73.97701263427734 40.743568420410156 -73.98594665527344 40.73516082763672 5 0.87 5 0.5 0.5 1.26 0 0 0.3 7.56 CSH 0 7  yellow -10 70 1 Manhattan 1007000 I MN20 Murray Hill-Kips Bay 3808 244 50 1 Manhattan 1005000 I MN21 Gramercy 3808 +1200014445 2 2015-07-20 2015-07-20 23:41:21 2015-07-20 2015-07-20 23:56:19 0 1 -74.00244903564453 40.73434066772461 -73.96761322021484 40.76564025878906 5 3.36 13.5 0.5 0.5 2.96 0 0 0.3 17.76 CSH 0 I 7 yellow -93 73 1 Manhattan 1007300 I MN23 West Village 3810 17 120 1 Manhattan 1012000 I MN40 Upper East Side-Carnegie Hill 3805 +1200014477 2 2015-07-21 2015-07-21 06:12:29 2015-07-21 2015-07-21 06:15:19 0 1 -73.99568176269531 40.73893356323242 -73.98426055908203 40.75546646118164 1 1.32 5.5 0 0.5 1 0 0 0.3 7.3 CSH 0 4 0 yellow -114 54 1 Manhattan 1005400 I MN13 Hudson Yards-Chelsea-Flatiron-Union Square 3807 133 119 1 Manhattan 1011900 E MN17 Midtown-Midtown South 3807 +1200014506 1 2015-07-21 2015-07-21 08:23:53 2015-07-21 2015-07-21 08:26:12 0 1 -73.99508666992188 40.76020812988281 -73.99641418457031 40.76347732543945 1 0.3 3.5 0 0.5 2 0 0 0.3 6.3 CSH 0 P P yellow -116 129 1 Manhattan 1012900 I MN15 Clinton 3807 140 129 1 Manhattan 1012900 I MN15 Clinton 3807 +1200014528 2 2015-07-21 2015-07-21 09:17:00 2015-07-21 2015-07-21 09:31:00 0 1 -73.98853302001953 40.737060546875 -73.97634887695312 40.74824523925781 1 1.21 9.5 0 0.5 2.06 0 0 0.3 12.36 CSH 0  p yellow -12 50 1 Manhattan 1005000 I MN21 Gramercy 3808 133 80 1 Manhattan 1008000 I MN20 Murray Hill-Kips Bay 3808 +1200014536 1 2015-07-21 2015-07-21 09:34:18 2015-07-21 2015-07-21 09:41:47 0 1 -74.00393676757812 40.747962951660156 -73.99693298339844 40.74885177612305 1 0.5 6.5 0 0.5 1.45 0 0 0.3 8.75 CSH 0 h h yellow 113 93 1 Manhattan 1009300 I MN13 Hudson Yards-Chelsea-Flatiron-Union Square 3807 238 97 1 Manhattan 1009700 E MN13 Hudson Yards-Chelsea-Flatiron-Union Square 3807 +1200014556 2 2015-07-21 2015-07-21 10:45:18 2015-07-21 2015-07-21 10:52:21 0 1 -74.0068588256836 40.73366165161133 -73.99783325195312 40.72584915161133 1 0.93 6.5 0 0.5 1 0 0 0.3 8.3 CSH 0 X  yellow -106 75 1 Manhattan 1007500 I MN23 West Village 3810 35 49 1 Manhattan 1004900 I MN24 SoHo-TriBeCa-Civic Center-Little Italy 3810 +1200014561 2 2015-07-21 2015-07-21 11:06:24 2015-07-21 2015-07-21 12:10:19 0 1 -73.8746337890625 40.774009704589844 -73.98082733154297 40.76490783691406 1 8.17 41.5 0 0.5 8.46 0 0 0.3 50.76 CSH 0 8 c yellow 48 331 4 Queens 4033100 I QN98 Airport 4102 4 137 1 Manhattan 1013700 I MN17 Midtown-Midtown South 3807 +1200014580 2 2015-07-21 2015-07-21 12:32:43 2015-07-21 2015-07-21 12:41:00 0 1 -73.96868133544922 40.761863708496094 -73.972900390625 40.75554656982422 1 0.48 6.5 0 0.5 1.82 0 0 0.3 9.12 CSH 0 b b yellow 20 112.03 1 Manhattan 1011203 I MN19 Turtle Bay-East Midtown 3808 226 92 1 Manhattan 1009200 I MN19 Turtle Bay-East Midtown 3808 +1200014582 2 2015-07-21 2015-07-21 12:30:31 2015-07-21 2015-07-21 12:37:57 0 1 -73.94827270507812 40.781436920166016 -73.95230102539062 40.76527404785156 2 1.49 7.5 0 0.5 1.66 0 0 0.3 9.96 CSH 0 c @ yellow 31 154 1 Manhattan 1015400 I MN32 Yorkville 3805 160 124 1 Manhattan 1012400 I MN31 Lenox Hill-Roosevelt Island 3805 +1200014627 2 2015-07-21 2015-07-21 14:30:45 2015-07-21 2015-07-21 14:52:42 0 1 -73.9845199584961 40.743003845214844 -73.96952819824219 40.756553649902344 2 1.53 14 0 0.5 3.7 0 0 0.3 18.5 CSH 0 4 ) yellow -90 56 1 Manhattan 1005600 I MN13 Hudson Yards-Chelsea-Flatiron-Union Square 3807 14 98 1 Manhattan 1009800 I MN19 Turtle Bay-East Midtown 3808 +1200014649 1 2015-07-21 2015-07-21 15:41:57 2015-07-21 2015-07-21 16:15:39 0 1 -73.96784210205078 40.764957427978516 -74.00145721435547 40.72781753540039 2 3.4 21.5 0 0.5 4.45 0 0 0.3 26.75 CSH 0 7  yellow 22 114.02 1 Manhattan 1011402 I MN40 Upper East Side-Carnegie Hill 3805 37 65 1 Manhattan 1006500 I MN23 West Village 3810 +1200014662 2 2015-07-21 2015-07-21 16:25:22 2015-07-21 2015-07-21 17:10:34 0 1 -73.98136138916016 40.773963928222656 -73.98958587646484 40.7261848449707 1 4.31 27.5 1 0.5 0 0 0 0.3 29.3 CRE 0 B y yellow 30 153 1 Manhattan 1015300 I MN14 Lincoln Square 3806 190 38 1 Manhattan 1003800 I MN22 East Village 3809 +1200014713 1 2015-07-21 2015-07-21 19:39:27 2015-07-21 2015-07-21 19:51:49 0 1 -74.00016021728516 40.743656158447266 -73.9849624633789 40.724082946777344 1 2.1 10 1 0.5 1 0 0 0.3 12.8 CSH 0 h y yellow 12 89 1 Manhattan 1008900 I MN13 Hudson Yards-Chelsea-Flatiron-Union Square 3807 10 32 1 Manhattan 1003200 I MN22 East Village 3809 +1200014740 2 2015-07-21 2015-07-21 20:40:59 2015-07-21 2015-07-21 20:57:59 0 1 -73.9542236328125 40.774715423583984 -73.98820495605469 40.7279167175293 1 5.84 20 0.5 0.5 0 0 0 0.3 21.3 CRE 0 c y yellow 0 138 1 Manhattan 1013800 I MN32 Yorkville 3805 190 38 1 Manhattan 1003800 I MN22 East Village 3809 +1200014770 2 2015-07-21 2015-07-21 21:38:41 2015-07-21 2015-07-21 21:52:08 0 1 -73.9744873046875 40.75111389160156 -73.95222473144531 40.77812194824219 3 2.37 10.5 0.5 0.5 0 0 0 0.3 11.8 CRE 0 b c yellow -30 92 1 Manhattan 1009200 I MN19 Turtle Bay-East Midtown 3808 27 146.02 1 Manhattan 1014602 I MN32 Yorkville 3805 +1200014787 1 2015-07-21 2015-07-21 22:26:36 2015-07-21 2015-07-21 22:30:44 0 1 -73.98229217529297 40.76557540893555 -73.9845962524414 40.759151458740234 2 0.5 5 0.5 0.5 0 0 0 0.3 6.3 CRE 0 c 0 yellow 4 137 1 Manhattan 1013700 I MN17 Midtown-Midtown South 3807 134 125 1 Manhattan 1012500 I MN17 Midtown-Midtown South 3807 +1200014797 1 2015-07-21 2015-07-21 23:02:06 2015-07-21 2015-07-21 23:08:04 0 1 -73.99553680419922 40.724937438964844 -73.9870376586914 40.73917007446289 1 1.1 6.5 0.5 0.5 1 0 0 0.3 8.8 CSH 0 D  yellow -67 43 1 Manhattan 1004300 I MN24 SoHo-TriBeCa-Civic Center-Little Italy 3810 11 68 1 Manhattan 1006800 I MN21 Gramercy 3808 +1200014804 2 2015-07-21 2015-07-21 23:14:44 2015-07-21 2015-07-21 23:26:18 0 1 -73.99730682373047 40.736820220947266 -73.96759033203125 40.759979248046875 6 2.67 11 0.5 0.5 0 0 0 0.3 12.3 CRE 0 I ) yellow -94 71 1 Manhattan 1007100 I MN23 West Village 3810 155 108 1 Manhattan 1010800 I MN19 Turtle Bay-East Midtown 3808 +1200014806 2 2015-07-21 2015-07-21 23:14:37 2015-07-21 2015-07-21 23:42:09 0 1 -73.7840347290039 40.648643493652344 -73.82498931884766 40.7957649230957 1 17.53 48 0.5 0.5 0 0 0 0.3 49.3 CRE 0 2 R yellow 8 716 4 Queens 4071600 I QN98 Airport 4105 157 973 4 Queens 4097300 I QN49 Whitestone 4103 +1200014813 1 2015-07-21 2015-07-21 23:28:21 2015-07-21 2015-07-21 23:50:08 0 1 -73.87300872802734 40.77408218383789 -73.96602630615234 40.80634689331055 1 8 25.5 0.5 0.5 6.45 5.54 0 0.3 38.79 CSH 0 8 f yellow 48 331 4 Queens 4033100 I QN98 Airport 4102 47 199 1 Manhattan 1019900 I MN09 Morningside Heights 3802 +1200014826 2 2015-07-21 2015-07-21 23:57:15 2015-07-22 2015-07-22 00:02:22 0 1 -73.96644592285156 40.754398345947266 -73.94761657714844 40.77912902832031 1 2.03 7.5 0.5 0.5 2.2 0 0 0.3 11 CSH 0 ) b yellow 14 98 1 Manhattan 1009800 I MN19 Turtle Bay-East Midtown 3808 1 152 1 Manhattan 1015200 I MN32 Yorkville 3805 +1200014828 1 2015-07-22 2015-07-22 00:05:28 2015-07-22 2015-07-22 00:06:06 0 5 -73.94906616210938 40.78187561035156 -73.94915008544922 40.78192138671875 1 0 13.8 0 0 2.8 0 0 0.3 16.9 CSH 0 c c yellow 31 154 1 Manhattan 1015400 I MN32 Yorkville 3805 31 154 1 Manhattan 1015400 I MN32 Yorkville 3805 +1200014830 2 2015-07-22 2015-07-22 00:11:47 2015-07-22 2015-07-22 00:21:33 0 1 -73.98880004882812 40.76919174194336 -73.95084381103516 40.7716178894043 1 2.9699999999999998 10.5 0.5 0.5 2.36 0 0 0.3 14.16 CSH 0 P b yellow -121 135 1 Manhattan 1013500 E MN15 Clinton 3807 255 136 1 Manhattan 1013600 I MN32 Yorkville 3805 +1200014902 2 2015-07-22 2015-07-22 09:10:29 2015-07-22 2015-07-22 09:27:48 0 1 -73.98059844970703 40.74827575683594 -73.99054718017578 40.756439208984375 1 1.05 11.5 0 0.5 2.46 0 0 0.3 14.76 CSH 0 p H yellow 93 82 1 Manhattan 1008200 I MN20 Murray Hill-Kips Bay 3808 163 115 1 Manhattan 1011500 I MN15 Clinton 3807 +1200014916 1 2015-07-22 2015-07-22 09:33:10 2015-07-22 2015-07-22 09:55:09 0 1 -73.98881530761719 40.7739372253418 -73.97017669677734 40.7595100402832 1 1.9 14.5 0 0.5 0 0 0 0.3 15.3 CRE 0 C b yellow -51 151 1 Manhattan 1015100 E MN14 Lincoln Square 3806 15 100 1 Manhattan 1010000 I MN19 Turtle Bay-East Midtown 3808 +1200015002 2 2015-07-22 2015-07-22 14:23:43 2015-07-22 2015-07-22 14:53:11 0 1 -74.0081787109375 40.714290618896484 -73.98355865478516 40.76438522338867 4 4.12 20 0 0.5 0 0 0 0.3 20.8 CRE 0 1 0 yellow 47 21 1 Manhattan 1002100 I MN24 SoHo-TriBeCa-Civic Center-Little Italy 3810 3 131 1 Manhattan 1013100 I MN17 Midtown-Midtown South 3807 +1200015010 1 2015-07-22 2015-07-22 14:20:35 2015-07-22 2015-07-22 14:39:10 0 1 -73.9788818359375 40.75572967529297 -74.0000228881836 40.74125289916992 2 1.8 12.5 0 0.5 2.65 0 0 0.3 15.95 CSH 0 a � yellow -31 94 1 Manhattan 1009400 E MN17 Midtown-Midtown South 3807 141 81 1 Manhattan 1008100 I MN13 Hudson Yards-Chelsea-Flatiron-Union Square 3807 +1200015012 1 2015-07-22 2015-07-22 15:17:30 2015-07-22 2015-07-22 15:24:28 0 1 -73.98932647705078 40.752586364746094 -73.98943328857422 40.7421875 1 1 6.5 0 0.5 1.45 0 0 0.3 8.75 CSH 0  4 yellow -124 109 1 Manhattan 1010900 I MN17 Midtown-Midtown South 3807 133 58 1 Manhattan 1005800 I MN13 Hudson Yards-Chelsea-Flatiron-Union Square 3807 +1200015030 2 2015-07-22 2015-07-22 16:27:17 2015-07-22 2015-07-22 16:35:40 0 1 -73.95166015625 40.790828704833984 -73.95712280273438 40.78614044189453 1 0.53 6.5 1 0.5 0 0 0 0.3 8.3 CRE 0 u C yellow -13 168 1 Manhattan 1016800 E MN33 East Harlem South 3804 43 143 1 Manhattan 1014300 I MN99 park-cemetery-etc-Manhattan 3806 +1200015038 2 2015-07-22 2015-07-22 16:58:32 2015-07-22 2015-07-22 17:02:47 0 1 -73.95620727539062 40.762969970703125 -73.94851684570312 40.77410888671875 1 0.86 5.5 1 0.5 1.46 0 0 0.3 8.76 CSH 0 @ b yellow -34 116 1 Manhattan 1011600 I MN31 Lenox Hill-Roosevelt Island 3805 255 136 1 Manhattan 1013600 I MN32 Yorkville 3805 +1200015042 2 2015-07-22 2015-07-22 17:11:27 2015-07-22 2015-07-22 17:31:23 0 1 -73.99464416503906 40.72581100463867 -73.97852325439453 40.75082778930664 1 2.02 13 1 0.5 0 0 0 0.3 14.8 CRE 0  p yellow -65 55.02 1 Manhattan 1005502 I MN23 West Village 3810 133 80 1 Manhattan 1008000 I MN20 Murray Hill-Kips Bay 3808 +1200015088 1 2015-07-22 2015-07-22 19:31:36 2015-07-22 2015-07-22 20:10:48 0 1 -73.97285461425781 40.76371765136719 -74.00236511230469 40.752349853515625 1 2.3 23 1 0.5 2.2 0 0 0.3 27 CSH 0 c F yellow 19 112.02 1 Manhattan 1011202 I MN17 Midtown-Midtown South 3807 50 99 1 Manhattan 1009900 I MN13 Hudson Yards-Chelsea-Flatiron-Union Square 3807 +1200015115 2 2015-07-22 2015-07-22 20:53:48 2015-07-22 2015-07-22 20:56:00 0 1 -73.96212005615234 40.7676887512207 -73.9659652709961 40.76521682739258 2 0.36 3.5 0.5 0.5 0 0 0 0.3 4.8 CRE 0 7 7 yellow 17 120 1 Manhattan 1012000 I MN40 Upper East Side-Carnegie Hill 3805 17 120 1 Manhattan 1012000 I MN40 Upper East Side-Carnegie Hill 3805 +1200015121 1 2015-07-22 2015-07-22 21:06:30 2015-07-22 2015-07-22 21:16:34 0 1 -73.97327423095703 40.747398376464844 -73.97720336914062 40.75890350341797 1 1.1 8 0.5 0.5 1.85 0 0 0.3 11.15 CSH 0 p a yellow -96 78 1 Manhattan 1007800 I MN20 Murray Hill-Kips Bay 3808 17 104 1 Manhattan 1010400 I MN17 Midtown-Midtown South 3807 +1200015126 1 2015-07-22 2015-07-22 21:12:40 2015-07-22 2015-07-22 21:18:26 0 1 -73.98530578613281 40.723594665527344 -73.982421875 40.71136474609375 1 1.7 7.5 0.5 0.5 0 0 0 0.3 8.8 CRE 0 y 2 yellow -94 30.02 1 Manhattan 1003002 E MN22 East Village 3809 64 2.02 1 Manhattan 1000202 E MN28 Lower East Side 3809 +1200015129 2 2015-07-22 2015-07-22 21:22:08 2015-07-22 2015-07-22 21:31:12 0 1 -73.97956848144531 40.686649322509766 -73.9706802368164 40.68070983886719 1 1.48 8 0.5 0.5 1.86 0 0 0.3 11.16 CSH 0 % � yellow -70 39 3 Brooklyn 3003900 I BK38 DUMBO-Vinegar Hill-Downtown Brooklyn-Boerum Hill 4004 10 163 3 Brooklyn 3016300 I BK64 Prospect Heights 4006 +1200015142 2 2015-07-22 2015-07-22 21:51:21 2015-07-22 2015-07-22 22:02:45 0 1 -73.9970932006836 40.72135543823242 -73.99539947509766 40.74889373779297 5 2.43 10.5 0.5 0.5 0 0 0 0.3 11.8 CRE 0 D � yellow -60 41 1 Manhattan 1004100 E MN24 SoHo-TriBeCa-Civic Center-Little Italy 3810 131 95 1 Manhattan 1009500 I MN17 Midtown-Midtown South 3807 +1200015148 2 2015-07-22 2015-07-22 22:02:41 2015-07-22 2015-07-22 22:02:48 0 5 -73.94235229492188 40.603050231933594 -73.94235229492188 40.603050231933594 2 0 36.8 0 0.5 9.4 0 0 0.3 47 CSH 0 I I yellow 17 566 3 Brooklyn 3056600 I BK44 Madison 4016 17 566 3 Brooklyn 3056600 I BK44 Madison 4016 +1200015178 1 2015-07-23 2015-07-23 00:00:18 2015-07-23 2015-07-23 00:14:03 0 1 -73.99026489257812 40.71429443359375 -73.9733657836914 40.69038772583008 1 3 12.5 0.5 0.5 3.45 0 0 0.3 17.25 CSH 0 2 � yellow 55 6 1 Manhattan 1000600 E MN28 Lower East Side 3809 201 31 3 Brooklyn 3003100 I BK68 Fort Greene 4004 +1200015180 1 2015-07-23 2015-07-23 00:16:25 2015-07-23 2015-07-23 00:31:49 0 1 -73.98825073242188 40.75661087036133 -73.97314453125 40.793487548828125 4 3.5 13.5 0.5 0.5 0 0 0 0.3 14.8 CRE 0 0 8 yellow -123 119 1 Manhattan 1011900 E MN17 Midtown-Midtown South 3807 167 179 1 Manhattan 1017900 I MN12 Upper West Side 3806 +1200015188 1 2015-07-23 2015-07-23 00:38:48 2015-07-23 2015-07-23 00:45:51 0 1 -73.97435760498047 40.75371170043945 -73.99253845214844 40.75673294067383 1 1.2 7 0.5 0.5 2.05 0 0 0.3 10.35 CSH 0 b H yellow -30 92 1 Manhattan 1009200 I MN19 Turtle Bay-East Midtown 3808 163 115 1 Manhattan 1011500 I MN15 Clinton 3807 +1200015206 1 2015-07-23 2015-07-23 04:40:49 2015-07-23 2015-07-23 04:48:28 0 1 -73.9559097290039 40.74524688720703 -73.9870834350586 40.750553131103516 1 2.6 10 0.5 0.5 3.35 5.54 0 0.3 20.19 CSH 0 E d yellow 33 1 4 Queens 4000100 I QN31 Hunters Point-Sunnyside-West Maspeth 4109 209 84 1 Manhattan 1008400 I MN17 Midtown-Midtown South 3807 +1200015232 2 2015-07-23 2015-07-23 07:09:37 2015-07-23 2015-07-23 07:16:09 0 1 -73.97228240966797 40.74518585205078 -73.97113800048828 40.75833511352539 1 1.3 6.5 0 0.5 1.46 0 0 0.3 8.76 CSH 0 3 b yellow -29 86.01 1 Manhattan 1008601 I MN19 Turtle Bay-East Midtown 3808 15 100 1 Manhattan 1010000 I MN19 Turtle Bay-East Midtown 3808 +1200015243 2 2015-07-23 2015-07-23 07:55:22 2015-07-23 2015-07-23 08:08:07 0 1 -73.95154571533203 40.769737243652344 -73.97239685058594 40.759151458740234 1 1.85 10.5 0 0.5 1 0 0 0.3 12.3 CSH 0 @ b yellow -95 132 1 Manhattan 1013200 I MN31 Lenox Hill-Roosevelt Island 3805 15 100 1 Manhattan 1010000 I MN19 Turtle Bay-East Midtown 3808 +1200015252 2 2015-07-23 2015-07-23 08:20:23 2015-07-23 2015-07-23 08:25:40 0 1 -73.9513168334961 40.770111083984375 -73.96592712402344 40.75288009643555 2 1.8399999999999999 7.5 0 0.5 0 0 0 0.3 8.3 CRE 0 @ ) yellow -95 132 1 Manhattan 1013200 I MN31 Lenox Hill-Roosevelt Island 3805 104 86.03 1 Manhattan 1008603 I MN19 Turtle Bay-East Midtown 3808 +1200015287 2 2015-07-23 2015-07-23 10:25:53 2015-07-23 2015-07-23 10:31:49 0 1 -74.00454711914062 40.72133255004883 -74.00636291503906 40.733177185058594 2 0.93 6 0 0.5 0 0 0 0.3 6.8 CRE 0  I yellow 16 47 1 Manhattan 1004700 I MN24 SoHo-TriBeCa-Civic Center-Little Italy 3810 163 73 1 Manhattan 1007300 I MN23 West Village 3810 +1200015289 1 2015-07-23 2015-07-23 10:29:52 2015-07-23 2015-07-23 10:40:09 0 1 -73.98194122314453 40.768592834472656 -73.97200775146484 40.76222229003906 1 0.8 8 0 0.5 1.76 0 0 0.3 10.56 CSH 0 B c yellow 66 145 1 Manhattan 1014500 I MN14 Lincoln Square 3806 19 112.02 1 Manhattan 1011202 I MN17 Midtown-Midtown South 3807 +1200015292 2 2015-07-23 2015-07-23 10:40:43 2015-07-23 2015-07-23 10:56:27 0 1 -73.9777603149414 40.786861419677734 -73.95427703857422 40.76567840576172 5 2.39 12 0 0.5 2.56 0 0 0.3 15.36 CSH 0 9 @ yellow -90 171 1 Manhattan 1017100 I MN12 Upper West Side 3806 160 124 1 Manhattan 1012400 I MN31 Lenox Hill-Roosevelt Island 3805 +1200015348 2 2015-07-23 2015-07-23 13:58:45 2015-07-23 2015-07-23 14:02:27 0 1 -73.96334838867188 40.761322021484375 -73.966064453125 40.75422668457031 5 0.7 4.5 0 0.5 0 0 0 0.3 5.3 CRE 0 A ) yellow -35 110 1 Manhattan 1011000 I MN31 Lenox Hill-Roosevelt Island 3805 14 98 1 Manhattan 1009800 I MN19 Turtle Bay-East Midtown 3808 +1200015362 1 2015-07-23 2015-07-23 14:48:50 2015-07-23 2015-07-23 15:02:58 0 1 -73.97985076904297 40.76591491699219 -73.967529296875 40.770172119140625 1 1.5 10 0 0.5 2.15 0 0 0.3 12.95 CSH 0 c 7 yellow 4 137 1 Manhattan 1013700 I MN17 Midtown-Midtown South 3807 228 122 1 Manhattan 1012200 I MN40 Upper East Side-Carnegie Hill 3805 +1200015375 1 2015-07-23 2015-07-23 15:51:53 2015-07-23 2015-07-23 16:15:53 0 1 -73.97500610351562 40.75064468383789 -73.97875213623047 40.77259063720703 1 2 15.5 0 0.5 0 0 0 0.3 16.3 CRE 0 p B yellow -123 80 1 Manhattan 1008000 I MN20 Murray Hill-Kips Bay 3808 30 153 1 Manhattan 1015300 I MN14 Lincoln Square 3806 +1200015379 1 2015-07-23 2015-07-23 16:15:28 2015-07-23 2015-07-23 16:30:12 0 1 -73.99656677246094 40.76318359375 -73.99609375 40.74124526977539 1 2.2 11.5 1 0.5 2.65 0 0 0.3 15.95 CSH 0 P � yellow -116 129 1 Manhattan 1012900 I MN15 Clinton 3807 143 87 1 Manhattan 1008700 I MN13 Hudson Yards-Chelsea-Flatiron-Union Square 3807 +1200015398 2 2015-07-23 2015-07-23 17:16:11 2015-07-23 2015-07-23 17:24:47 0 1 -73.99195861816406 40.74420928955078 -73.98320007324219 40.755855560302734 1 0.94 7 1 0.5 0 0 0 0.3 8.8 CRE 0 � a yellow -124 91 1 Manhattan 1009100 I MN13 Hudson Yards-Chelsea-Flatiron-Union Square 3807 13 96 1 Manhattan 1009600 E MN17 Midtown-Midtown South 3807 +1200015427 1 2015-07-23 2015-07-23 18:40:50 2015-07-23 2015-07-23 18:55:56 0 1 -73.9601821899414 40.7734375 -73.96472930908203 40.75642013549805 1 1.4 10.5 1 0.5 1.2 0 0 0.3 13.5 CSH 0 7 ) yellow 18 128 1 Manhattan 1012800 I MN40 Upper East Side-Carnegie Hill 3805 14 98 1 Manhattan 1009800 I MN19 Turtle Bay-East Midtown 3808 +1200015454 2 2015-07-23 2015-07-23 20:03:48 2015-07-23 2015-07-23 20:27:13 0 1 -73.99817657470703 40.726261138916016 -73.9901123046875 40.767120361328125 5 3.61 17 0.5 0.5 4.58 0 0 0.3 22.88 CSH 0  P yellow -97 55.01 1 Manhattan 1005501 I MN23 West Village 3810 135 135 1 Manhattan 1013500 E MN15 Clinton 3807 +1200015466 2 2015-07-23 2015-07-23 20:34:16 2015-07-23 2015-07-23 20:49:23 0 1 -73.95822143554688 40.77537155151367 -73.97441864013672 40.7469596862793 1 2.41 11.5 0.5 0.5 3.2 0 0 0.3 16 CSH 0 6 p yellow 24 140 1 Manhattan 1014000 I MN40 Upper East Side-Carnegie Hill 3805 160 78 1 Manhattan 1007800 I MN20 Murray Hill-Kips Bay 3808 +1200015489 2 2015-07-23 2015-07-23 21:31:16 2015-07-23 2015-07-23 21:37:42 0 1 -73.96756744384766 40.76027297973633 -73.95052337646484 40.7823486328125 1 1.8199999999999998 7.5 0.5 0.5 1.2 0 0 0.3 10 CSH 0 b c yellow 20 112.03 1 Manhattan 1011203 I MN19 Turtle Bay-East Midtown 3808 31 154 1 Manhattan 1015400 I MN32 Yorkville 3805 +1200015500 2 2015-07-23 2015-07-23 21:45:47 2015-07-23 2015-07-23 22:01:29 0 1 -74.00332641601562 40.743656158447266 -73.9793701171875 40.763511657714844 1 2.46 12 0.5 0.5 2 0 0 0.3 15.3 CSH 0 h c yellow 12 89 1 Manhattan 1008900 I MN13 Hudson Yards-Chelsea-Flatiron-Union Square 3807 4 137 1 Manhattan 1013700 I MN17 Midtown-Midtown South 3807 +1200015501 2 2015-07-23 2015-07-23 21:47:06 2015-07-23 2015-07-23 21:58:43 0 1 -74.01197814941406 40.70735549926758 -73.99627685546875 40.73809051513672 1 2.63 10.5 0.5 0.5 1 0 0 0.3 12.8 CSH 0 � 4 yellow -1 7 1 Manhattan 1000700 I MN25 Battery Park City-Lower Manhattan 3810 142 54 1 Manhattan 1005400 I MN13 Hudson Yards-Chelsea-Flatiron-Union Square 3807 +1200015523 1 2015-07-23 2015-07-23 22:42:23 2015-07-23 2015-07-23 22:53:04 0 1 -73.97514343261719 40.76139450073242 -73.97371673583984 40.747474670410156 1 1.3 8.5 0.5 0.5 1.95 0 0 0.3 11.75 CSH 0 a p yellow 17 104 1 Manhattan 1010400 I MN17 Midtown-Midtown South 3807 160 78 1 Manhattan 1007800 I MN20 Murray Hill-Kips Bay 3808 +1200015578 2 2015-07-24 2015-07-24 02:26:42 2015-07-24 2015-07-24 02:40:57 0 1 -73.95210266113281 40.82447814941406 -73.95085906982422 40.77082824707031 6 4.82 15.5 0.5 0.5 0 0 0 0.3 16.8 CRE 0  @ yellow -117 225 1 Manhattan 1022500 E MN04 Hamilton Heights 3802 161 132 1 Manhattan 1013200 I MN31 Lenox Hill-Roosevelt Island 3805 +1200015600 2 2015-07-24 2015-07-24 06:26:40 2015-07-24 2015-07-24 06:47:57 0 1 -73.91108703613281 40.742523193359375 -73.99028778076172 40.75069046020508 2 5.47 19.5 0 0.5 2 0 0 0.3 22.3 CSH 0 ` � yellow 25 251 4 Queens 4025100 E QN63 Woodside 4109 131 101 1 Manhattan 1010100 I MN17 Midtown-Midtown South 3807 +1200015622 2 2015-07-24 2015-07-24 08:42:36 2015-07-24 2015-07-24 08:47:46 0 1 -73.9695053100586 40.76637268066406 -73.95821380615234 40.78315353393555 5 1.3599999999999999 6 0 0.5 1.7 0 0 0.3 8.5 CSH 0 7 6 yellow -28 122 1 Manhattan 1012200 I MN40 Upper East Side-Carnegie Hill 3805 74 150.02 1 Manhattan 1015002 I MN40 Upper East Side-Carnegie Hill 3805 +1200015628 2 2015-07-24 2015-07-24 05:59:29 2015-07-24 2015-07-24 06:02:59 0 1 -73.98065185546875 40.76154708862305 -73.96752166748047 40.760311126708984 2 0.82 5 0 0.5 0 0 0 0.3 5.8 CRE 0 0 b yellow 3 131 1 Manhattan 1013100 I MN17 Midtown-Midtown South 3807 20 112.03 1 Manhattan 1011203 I MN19 Turtle Bay-East Midtown 3808 +1200015629 2 2015-07-24 2015-07-24 06:02:02 2015-07-24 2015-07-24 06:08:04 0 1 -73.98956298828125 40.74169158935547 -74.00562286376953 40.75096130371094 5 1.2 6.5 0 0.5 0 0 0 0.3 7.3 CRE 0 4 F yellow -123 58 1 Manhattan 1005800 I MN13 Hudson Yards-Chelsea-Flatiron-Union Square 3807 50 99 1 Manhattan 1009900 I MN13 Hudson Yards-Chelsea-Flatiron-Union Square 3807 +1200015631 1 2015-07-24 2015-07-24 05:35:34 2015-07-24 2015-07-24 05:43:05 0 1 -73.97815704345703 40.75217819213867 -74.00338745117188 40.753013610839844 1 1.8 8 0.5 0.5 1.25 0 0 0.3 10.55 CSH 0 p F yellow 93 82 1 Manhattan 1008200 I MN20 Murray Hill-Kips Bay 3808 50 99 1 Manhattan 1009900 I MN13 Hudson Yards-Chelsea-Flatiron-Union Square 3807 +1200015640 2 2015-07-24 2015-07-24 09:42:45 2015-07-24 2015-07-24 09:53:51 0 1 -74.01044464111328 40.71120834350586 -73.99744415283203 40.72557830810547 1 1.6 9 0 0.5 0 0 0 0.3 9.8 CRE 0 a  yellow 53 13 1 Manhattan 1001300 I MN25 Battery Park City-Lower Manhattan 3810 35 49 1 Manhattan 1004900 I MN24 SoHo-TriBeCa-Civic Center-Little Italy 3810 +1200015727 1 2015-07-24 2015-07-24 14:33:08 2015-07-24 2015-07-24 14:40:49 0 1 -73.9783935546875 40.76653289794922 -73.9689712524414 40.75779724121094 2 0.9 7 0 0.5 0 0 0 0.3 7.8 CRE 0 c ) yellow 4 137 1 Manhattan 1013700 I MN17 Midtown-Midtown South 3807 14 98 1 Manhattan 1009800 I MN19 Turtle Bay-East Midtown 3808 +1200015751 1 2015-07-24 2015-07-24 16:03:47 2015-07-24 2015-07-24 16:29:02 1 1 -73.9193344116211 40.792747497558594 -73.97142791748047 40.76153564453125 1 6 22 1 0.5 0 0 0 0.3 23.8 CRE 0 � c yellow -85 240 1 Manhattan 1024000 I MN99 park-cemetery-etc-Manhattan 3804 19 112.02 1 Manhattan 1011202 I MN17 Midtown-Midtown South 3807 +1200015764 2 2015-07-24 2015-07-24 16:55:00 2015-07-24 2015-07-24 16:59:53 0 1 -73.99394226074219 40.758155822753906 -74.00045013427734 40.74781799316406 1 0.82 5.5 1 0.5 1.46 0 0 0.3 8.76 CSH 0 H h yellow -93 115 1 Manhattan 1011500 I MN15 Clinton 3807 113 93 1 Manhattan 1009300 I MN13 Hudson Yards-Chelsea-Flatiron-Union Square 3807 +1200015773 2 2015-07-24 2015-07-24 17:29:00 2015-07-24 2015-07-24 17:32:12 0 1 -73.95381164550781 40.781917572021484 -73.96024322509766 40.771366119384766 1 0.88 5 1 0.5 1 0 0 0.3 7.8 CSH 0 6 7 yellow 29 148.02 1 Manhattan 1014802 I MN40 Upper East Side-Carnegie Hill 3805 18 128 1 Manhattan 1012800 I MN40 Upper East Side-Carnegie Hill 3805 +1200015795 1 2015-07-24 2015-07-24 18:21:36 2015-07-24 2015-07-24 18:31:43 0 1 -73.97306060791016 40.758174896240234 -73.98140716552734 40.77340316772461 1 1.6 8.5 1 0.5 2 0 0 0.3 12.3 CSH 0 b B yellow 15 100 1 Manhattan 1010000 I MN19 Turtle Bay-East Midtown 3808 67 149 1 Manhattan 1014900 I MN14 Lincoln Square 3806 +1200015811 1 2015-07-24 2015-07-24 19:15:51 2015-07-24 2015-07-24 19:21:44 0 1 -73.94712829589844 40.78402328491211 -73.95626068115234 40.77150344848633 1 1 6 1 0.5 1 0 0 0.3 8.8 CSH 0 c A yellow 32 156.01 1 Manhattan 1015601 I MN32 Yorkville 3805 218 134 1 Manhattan 1013400 I MN31 Lenox Hill-Roosevelt Island 3805 +1200015834 2 2015-07-24 2015-07-24 20:27:50 2015-07-24 2015-07-24 20:37:42 0 1 -73.98947143554688 40.74108123779297 -73.97789001464844 40.757205963134766 2 1.6099999999999999 8.5 0.5 0.5 1.96 0 0 0.3 11.76 CSH 0 4 a yellow -90 56 1 Manhattan 1005600 I MN13 Hudson Yards-Chelsea-Flatiron-Union Square 3807 225 94 1 Manhattan 1009400 E MN17 Midtown-Midtown South 3807 +1200015856 1 2015-07-24 2015-07-24 21:28:32 2015-07-24 2015-07-24 21:48:39 0 1 -73.86346435546875 40.769779205322266 -73.98567962646484 40.74401092529297 1 9.3 27 0.5 0.5 6 5.54 0 0.3 39.84 CSH 0 8 d yellow 48 331 4 Queens 4033100 I QN98 Airport 4102 131 74 1 Manhattan 1007400 I MN17 Midtown-Midtown South 3807 +1200015892 2 2015-07-24 2015-07-24 23:07:52 2015-07-24 2015-07-24 23:18:25 0 1 -73.96995544433594 40.78562545776367 -73.9679946899414 40.762813568115234 1 2.13 10 0.5 0.5 2.26 0 0 0.3 13.56 CSH 0 8 7 yellow -119 173 1 Manhattan 1017300 I MN12 Upper West Side 3806 22 114.02 1 Manhattan 1011402 I MN40 Upper East Side-Carnegie Hill 3805 +1200015916 2 2015-07-25 2015-07-25 00:21:23 2015-07-25 2015-07-25 00:21:38 0 1 -73.97742462158203 40.760292053222656 -73.97742462158203 40.760292053222656 4 0 2.5 0.5 0.5 0 0 0 0.3 3.8 CRE 0 a a yellow 17 104 1 Manhattan 1010400 I MN17 Midtown-Midtown South 3807 17 104 1 Manhattan 1010400 I MN17 Midtown-Midtown South 3807 +1200015919 2 2015-07-25 2015-07-25 00:25:45 2015-07-25 2015-07-25 00:31:34 0 1 -73.98829650878906 40.76438522338867 -73.99408721923828 40.75640869140625 3 0.63 5.5 0.5 0.5 1.7 0 0 0.3 8.5 CSH 0 H H yellow -119 133 1 Manhattan 1013300 I MN15 Clinton 3807 163 115 1 Manhattan 1011500 I MN15 Clinton 3807 +1200015951 2 2015-07-25 2015-07-25 02:08:45 2015-07-25 2015-07-25 02:35:25 0 1 -73.99746704101562 40.725921630859375 -73.87348937988281 40.75294876098633 6 8.62 29.5 0.5 0.5 6.16 0 0 0.3 36.96 CSH 0  ) yellow -97 55.01 1 Manhattan 1005501 I MN23 West Village 3810 103 273 4 Queens 4027300 E QN28 Jackson Heights 4102 +1200015958 2 2015-07-25 2015-07-25 02:31:32 2015-07-25 2015-07-25 02:35:24 0 1 -73.9715576171875 40.75749969482422 -73.98457336425781 40.73971176147461 1 1.41 6 0.5 0.5 1.46 0 0 0.3 8.76 CSH 0 b  yellow 15 100 1 Manhattan 1010000 I MN19 Turtle Bay-East Midtown 3808 11 68 1 Manhattan 1006800 I MN21 Gramercy 3808 +1200015973 1 2015-07-25 2015-07-25 03:25:43 2015-07-25 2015-07-25 03:32:48 0 1 -73.98551940917969 40.75846481323242 -74.00355529785156 40.733558654785156 1 2 8 0.5 0.5 0 0 0 0.3 9.3 CRE 0 0 I yellow -123 119 1 Manhattan 1011900 E MN17 Midtown-Midtown South 3807 163 73 1 Manhattan 1007300 I MN23 West Village 3810 +1200015983 1 2015-07-25 2015-07-25 04:06:44 2015-07-25 2015-07-25 04:19:56 0 1 -73.9813232421875 40.72481155395508 -73.95309448242188 40.72499465942383 1 4.2 15 0.5 0.5 3.25 0 0 0.3 19.55 CSH 0   yellow -108 26.02 1 Manhattan 1002602 E MN28 Lower East Side 3809 245 561 3 Brooklyn 3056100 I BK76 Greenpoint 4001 +1200015991 2 2015-07-25 2015-07-25 05:25:20 2015-07-25 2015-07-25 05:39:52 0 1 -73.978271484375 40.74556350708008 -73.92499542236328 40.77336883544922 1 4.74 16 0.5 0.5 0 0 0 0.3 17.3 CRE 0 p y yellow -126 72 1 Manhattan 1007200 I MN20 Murray Hill-Kips Bay 3808 234 69 4 Queens 4006900 E QN71 Old Astoria 4101 +1200016008 2 2015-07-25 2015-07-25 08:33:22 2015-07-25 2015-07-25 08:47:28 0 1 -73.99121856689453 40.72771453857422 -73.99227905273438 40.74922561645508 1 2.04 11 0 0.5 1.5 0 0 0.3 13.3 CSH 0 y � yellow -66 38 1 Manhattan 1003800 I MN22 East Village 3809 131 101 1 Manhattan 1010100 I MN17 Midtown-Midtown South 3807 +1200016041 2 2015-07-25 2015-07-25 10:40:25 2015-07-25 2015-07-25 10:46:39 0 1 -73.95236206054688 40.772212982177734 -73.97090148925781 40.755226135253906 1 2.17 8.5 0 0.5 0.5 0 0 0.3 9.8 CSH 0 b ) yellow -1 136 1 Manhattan 1013600 I MN32 Yorkville 3805 14 98 1 Manhattan 1009800 I MN19 Turtle Bay-East Midtown 3808 +1200016062 2 2015-07-25 2015-07-25 11:47:52 2015-07-25 2015-07-25 12:13:09 0 1 -74.0142593383789 40.71160888671875 -73.99208068847656 40.75989532470703 1 3.74 18.5 0 0.5 3.86 0 0 0.3 23.16 CSH 0  H yellow 52 317.03 1 Manhattan 1031703 I MN25 Battery Park City-Lower Manhattan 3810 139 121 1 Manhattan 1012100 E MN15 Clinton 3807 +1200016063 2 2015-07-25 2015-07-25 12:00:21 2015-07-25 2015-07-25 12:15:24 0 1 -73.97897338867188 40.75621795654297 -73.98292541503906 40.722774505615234 6 3.05 13 0 0.5 0 0 0 0.3 13.8 CRE 0 a y yellow 13 96 1 Manhattan 1009600 E MN17 Midtown-Midtown South 3807 10 32 1 Manhattan 1003200 I MN22 East Village 3809 +1200016069 2 2015-07-25 2015-07-25 12:09:08 2015-07-25 2015-07-25 12:21:02 0 1 -73.9623794555664 40.770328521728516 -73.98925018310547 40.73973083496094 6 2.7199999999999998 11 0 0.5 2.36 0 0 0.3 14.16 CSH 0 7 4 yellow 18 128 1 Manhattan 1012800 I MN40 Upper East Side-Carnegie Hill 3805 166 56 1 Manhattan 1005600 I MN13 Hudson Yards-Chelsea-Flatiron-Union Square 3807 +1200016098 1 2015-07-25 2015-07-25 14:27:44 2015-07-25 2015-07-25 14:44:45 0 1 -73.99183654785156 40.726104736328125 -73.98487854003906 40.75748825073242 4 2.9 13.5 0 0.5 0 0 0 0.3 14.3 CRE 0 y 0 yellow -62 36.02 1 Manhattan 1003602 I MN22 East Village 3809 133 119 1 Manhattan 1011900 E MN17 Midtown-Midtown South 3807 +1200016117 2 2015-07-25 2015-07-25 11:53:15 2015-07-25 2015-07-25 11:57:59 0 1 -73.99217224121094 40.76399612426758 -73.98059844970703 40.77809524536133 2 1.21 6 0 0.5 1.36 0 0 0.3 8.16 CSH 0 H B yellow -118 127 1 Manhattan 1012700 I MN15 Clinton 3807 21 157 1 Manhattan 1015700 I MN14 Lincoln Square 3806 +1200016138 2 2015-07-25 2015-07-25 16:33:51 2015-07-25 2015-07-25 16:47:56 0 1 -74.0067367553711 40.744056701660156 -73.98663330078125 40.75947189331055 1 1.88 11 0 0.5 0 0 0 0.3 11.8 CRE 0 h 0 yellow 75 83 1 Manhattan 1008300 E MN13 Hudson Yards-Chelsea-Flatiron-Union Square 3807 134 125 1 Manhattan 1012500 I MN17 Midtown-Midtown South 3807 +1200016140 2 2015-07-25 2015-07-25 16:50:59 2015-07-25 2015-07-25 17:16:51 0 1 -73.98135375976562 40.75352096557617 -73.96038818359375 40.714088439941406 1 5.2 21 0 0.5 4.36 0 0 0.3 26.16 CSH 0 d V yellow -47 84 1 Manhattan 1008400 I MN17 Midtown-Midtown South 3807 117 551 3 Brooklyn 3055100 E BK73 North Side-South Side 4001 +1200016166 2 2015-07-25 2015-07-25 17:59:59 2015-07-25 2015-07-25 18:51:35 0 2 -73.77870178222656 40.64668655395508 -73.97850799560547 40.77276611328125 1 21.51 52 0 0.5 6 5.54 0 0.3 64.34 CSH 0 2 B yellow 8 716 4 Queens 4071600 I QN98 Airport 4105 30 153 1 Manhattan 1015300 I MN14 Lincoln Square 3806 +1200016178 2 2015-07-25 2015-07-25 18:34:52 2015-07-25 2015-07-25 19:00:03 0 1 -73.9773178100586 40.75130844116211 -74.00382232666016 40.72031021118164 1 3.42 17.5 0 0.5 0 0 0 0.3 18.3 CRE 0 p 1 yellow -123 80 1 Manhattan 1008000 I MN20 Murray Hill-Kips Bay 3808 15 33 1 Manhattan 1003300 I MN24 SoHo-TriBeCa-Civic Center-Little Italy 3810 +1200016193 2 2015-07-25 2015-07-25 19:11:44 2015-07-25 2015-07-25 19:13:29 0 1 -73.96006774902344 40.77796173095703 -73.95751190185547 40.77525329589844 1 0.34 3.5 0 0.5 1.29 0 0 0.3 5.59 CSH 0 6 6 yellow 25 142 1 Manhattan 1014200 I MN40 Upper East Side-Carnegie Hill 3805 24 140 1 Manhattan 1014000 I MN40 Upper East Side-Carnegie Hill 3805 +1200016216 2 2015-07-25 2015-07-25 20:16:56 2015-07-25 2015-07-25 20:22:31 0 1 -73.97776794433594 40.755008697509766 -73.96794128417969 40.76240158081055 2 0.93 6 0.5 0.5 1.46 0 0 0.3 8.76 CSH 0 a 7 yellow -31 94 1 Manhattan 1009400 E MN17 Midtown-Midtown South 3807 22 114.02 1 Manhattan 1011402 I MN40 Upper East Side-Carnegie Hill 3805 +1200016279 2 2015-07-25 2015-07-25 23:23:29 2015-07-25 2015-07-25 23:36:00 0 1 -73.98928833007812 40.73817825317383 -74.00515747070312 40.72270965576172 1 1.62 9.5 0.5 0.5 0 0 0 0.3 10.8 CRE 0 4  yellow -11 52 1 Manhattan 1005200 I MN13 Hudson Yards-Chelsea-Flatiron-Union Square 3807 16 47 1 Manhattan 1004700 I MN24 SoHo-TriBeCa-Civic Center-Little Italy 3810 +1200016289 1 2015-07-25 2015-07-25 23:47:24 2015-07-25 2015-07-25 23:55:19 0 1 -73.97900390625 40.740848541259766 -73.9926528930664 40.74775314331055 1 0.9 6.5 0.5 0.5 1.55 0 0 0.3 9.35 CSH 0 7 � yellow -11 66 1 Manhattan 1006600 I MN20 Murray Hill-Kips Bay 3808 131 95 1 Manhattan 1009500 I MN17 Midtown-Midtown South 3807 +1200016299 1 2015-07-26 2015-07-26 00:17:08 2015-07-26 2015-07-26 00:21:08 0 1 -74.00114440917969 40.75205993652344 -73.99022674560547 40.760799407958984 2 1 5 0.5 0.5 1.25 0 0 0.3 7.55 CSH 0 F H yellow 50 99 1 Manhattan 1009900 I MN13 Hudson Yards-Chelsea-Flatiron-Union Square 3807 138 127 1 Manhattan 1012700 I MN15 Clinton 3807 +1200016336 1 2015-07-26 2015-07-26 01:55:34 2015-07-26 2015-07-26 02:07:58 0 1 -73.9831771850586 40.73051071166992 -73.9950942993164 40.75963592529297 2 2.9 12 0.5 0.5 3.3 0 0 0.3 16.6 CSH 0 y H yellow 93 40 1 Manhattan 1004000 I MN22 East Village 3809 139 121 1 Manhattan 1012100 E MN15 Clinton 3807 +1200016353 1 2015-07-26 2015-07-26 03:39:31 2015-07-26 2015-07-26 03:45:13 0 1 -73.99057006835938 40.724666595458984 -73.97665405273438 40.73228073120117 1 1.5 7 0.5 0.5 1.65 0 0 0.3 9.95 CSH 0 y $ yellow -62 36.02 1 Manhattan 1003602 I MN22 East Village 3809 110 44 1 Manhattan 1004400 I MN50 Stuyvesant Town-Cooper Village 3808 +1200016357 1 2015-07-26 2015-07-26 03:59:30 2015-07-26 2015-07-26 04:13:12 0 1 -74.00686645507812 40.74398422241211 -73.96722412109375 40.755916595458984 2 3.2 13 0.5 0.5 2.85 0 0 0.3 17.15 CSH 0 h ) yellow 75 83 1 Manhattan 1008300 E MN13 Hudson Yards-Chelsea-Flatiron-Union Square 3807 14 98 1 Manhattan 1009800 I MN19 Turtle Bay-East Midtown 3808 +1200016372 1 2015-07-26 2015-07-26 06:44:27 2015-07-26 2015-07-26 06:55:31 0 1 -73.99138641357422 40.749935150146484 -73.95246887207031 40.780784606933594 1 3.8 13 0 0.5 1 0 0 0.3 14.8 CSH 0 � c yellow -125 101 1 Manhattan 1010100 I MN17 Midtown-Midtown South 3807 31 154 1 Manhattan 1015400 I MN32 Yorkville 3805 +1200016382 2 2015-07-26 2015-07-26 08:19:52 2015-07-26 2015-07-26 08:24:24 0 1 -73.96878814697266 40.758609771728516 -73.9697036743164 40.76858139038086 2 0.97 5.5 0 0.5 1 0 0 0.3 7.3 CSH 0 b 7 yellow 15 100 1 Manhattan 1010000 I MN19 Turtle Bay-East Midtown 3808 228 122 1 Manhattan 1012200 I MN40 Upper East Side-Carnegie Hill 3805 +1200016395 1 2015-07-26 2015-07-26 09:27:30 2015-07-26 2015-07-26 09:33:19 0 1 -73.98845672607422 40.7588996887207 -73.97749328613281 40.77402877807617 3 1.2 6.5 0 0.5 0 0 0 0.3 7.3 CRE 0 0 C yellow -123 119 1 Manhattan 1011900 E MN17 Midtown-Midtown South 3807 43 143 1 Manhattan 1014300 I MN99 park-cemetery-etc-Manhattan 3806 +1200016401 1 2015-07-26 2015-07-26 10:11:55 2015-07-26 2015-07-26 10:21:08 0 1 -73.99456024169922 40.760780334472656 -74.00082397460938 40.732444763183594 1 2.5 9.5 0 0.5 2 0 0 0.3 12.3 CSH 0 H I yellow -117 121 1 Manhattan 1012100 E MN15 Clinton 3807 162 71 1 Manhattan 1007100 I MN23 West Village 3810 +1200016428 2 2015-07-26 2015-07-26 11:42:31 2015-07-26 2015-07-26 11:52:38 0 1 -73.96227264404297 40.779136657714844 -73.97877502441406 40.754371643066406 1 1.98 9 0 0.5 2.94 0 0 0.3 12.74 CSH 0 C a yellow 43 143 1 Manhattan 1014300 I MN99 park-cemetery-etc-Manhattan 3806 225 94 1 Manhattan 1009400 E MN17 Midtown-Midtown South 3807 +1200016436 2 2015-07-26 2015-07-26 12:14:56 2015-07-26 2015-07-26 12:18:51 0 1 -73.98883056640625 40.73671340942383 -73.9813232421875 40.74121856689453 1 0.67 4.5 0 0.5 1.06 0 0 0.3 6.36 CSH 0  7 yellow -12 50 1 Manhattan 1005000 I MN21 Gramercy 3808 245 66 1 Manhattan 1006600 I MN20 Murray Hill-Kips Bay 3808 +1200016451 2 2015-07-26 2015-07-26 13:33:49 2015-07-26 2015-07-26 13:42:12 0 1 -74.00940704345703 40.715248107910156 -74.01544952392578 40.7098388671875 3 0.79 7 0 0.5 1.56 0 0 0.3 9.36 CSH 0 1  yellow 47 21 1 Manhattan 1002100 I MN24 SoHo-TriBeCa-Civic Center-Little Italy 3810 54 317.04 1 Manhattan 1031704 I MN25 Battery Park City-Lower Manhattan 3810 +1200016456 2 2015-07-26 2015-07-26 13:52:47 2015-07-26 2015-07-26 14:15:24 0 1 -73.97685241699219 40.76585388183594 -73.9831314086914 40.74184799194336 1 2.57 15 0 0.5 0 0 0 0.3 15.8 CRE 0 c  yellow 4 137 1 Manhattan 1013700 I MN17 Midtown-Midtown South 3807 11 68 1 Manhattan 1006800 I MN21 Gramercy 3808 +1200016497 1 2015-07-26 2015-07-26 16:17:06 2015-07-26 2015-07-26 16:28:24 0 1 -73.96219635009766 40.77907943725586 -73.98352813720703 40.76445388793945 1 1.8 10 0 0.5 0 0 0 0.3 10.8 CRE 0 6 0 yellow 25 142 1 Manhattan 1014200 I MN40 Upper East Side-Carnegie Hill 3805 3 131 1 Manhattan 1013100 I MN17 Midtown-Midtown South 3807 +1200016530 2 2015-07-26 2015-07-26 17:57:38 2015-07-26 2015-07-26 18:04:03 0 1 -73.9578628540039 40.761260986328125 -73.96173095703125 40.77670669555664 1 1.5699999999999998 7 0 0.5 0 0 0 0.3 7.8 CRE 0 @ 6 yellow -36 106.02 1 Manhattan 1010602 I MN31 Lenox Hill-Roosevelt Island 3805 25 142 1 Manhattan 1014200 I MN40 Upper East Side-Carnegie Hill 3805 +1200016536 2 2015-07-26 2015-07-26 18:09:49 2015-07-26 2015-07-26 18:19:18 0 1 -74.00824737548828 40.740440368652344 -74.00629425048828 40.740333557128906 5 1.06 7.5 0 0.5 0 0 0 0.3 8.3 CRE 0 X X yellow 79 79 1 Manhattan 1007900 I MN23 West Village 3810 79 79 1 Manhattan 1007900 I MN23 West Village 3810 +1200016543 1 2015-07-26 2015-07-26 18:51:10 2015-07-26 2015-07-26 19:00:18 0 1 -73.98307037353516 40.73475646972656 -74.00562286376953 40.72577667236328 1 1.9 9 0 0.5 1.95 0 0 0.3 11.75 CSH 0  % yellow 52 48 1 Manhattan 1004800 I MN21 Gramercy 3808 13 37 1 Manhattan 1003700 I MN24 SoHo-TriBeCa-Civic Center-Little Italy 3810 +1200016569 2 2015-07-26 2015-07-26 20:34:47 2015-07-26 2015-07-26 20:51:13 0 1 -73.9788589477539 40.76211166381836 -73.98835754394531 40.738731384277344 1 2.63 12.5 0.5 0.5 2.76 0 0 0.3 16.56 CSH 0 0 4 yellow 3 131 1 Manhattan 1013100 I MN17 Midtown-Midtown South 3807 245 52 1 Manhattan 1005200 I MN13 Hudson Yards-Chelsea-Flatiron-Union Square 3807 +1200016578 2 2015-07-26 2015-07-26 20:56:54 2015-07-26 2015-07-26 21:09:22 0 1 -73.98929595947266 40.76034164428711 -73.96894836425781 40.78840255737305 5 2.86 11.5 0.5 0.5 1.2 0 0 0.3 14 CSH 0 H 8 yellow -117 121 1 Manhattan 1012100 E MN15 Clinton 3807 137 173 1 Manhattan 1017300 I MN12 Upper West Side 3806 +1200016579 2 2015-07-26 2015-07-26 21:02:55 2015-07-26 2015-07-26 21:25:31 0 1 -73.99510192871094 40.74997329711914 -73.98836517333984 40.72013473510742 1 3.33 16 0.5 0.5 1.5 0 0 0.3 18.8 CSH 0 h H yellow 103 103 1 Manhattan 1010300 I MN13 Hudson Yards-Chelsea-Flatiron-Union Square 3807 161 30.01 1 Manhattan 1003001 E MN27 Chinatown 3809 +1200016587 2 2015-07-26 2015-07-26 21:38:23 2015-07-26 2015-07-26 22:03:47 0 1 -73.86274719238281 40.7690315246582 -73.98358154296875 40.7218017578125 1 9.85 30.5 0.5 0.5 6.36 0 0 0.3 38.16 CSH 0 8  yellow 48 331 4 Queens 4033100 I QN98 Airport 4102 9 22.02 1 Manhattan 1002202 I MN28 Lower East Side 3809 +1200016637 2 2015-07-27 2015-07-27 02:09:28 2015-07-27 2015-07-27 02:15:31 0 1 -74.0130615234375 40.71571350097656 -74.004638671875 40.737693786621094 1 1.75 7.5 0.5 0.5 1.76 0 0 0.3 10.56 CSH 0 1 I yellow 47 21 1 Manhattan 1002100 I MN24 SoHo-TriBeCa-Civic Center-Little Italy 3810 78 77 1 Manhattan 1007700 I MN23 West Village 3810 +1200016649 2 2015-07-27 2015-07-27 05:27:22 2015-07-27 2015-07-27 05:52:52 0 2 -73.98400115966797 40.743408203125 -73.79016876220703 40.6468620300293 1 16.34 52 0 0.5 11.67 5.54 0 0.3 70.01 CSH 0 p 2 yellow -126 72 1 Manhattan 1007200 I MN20 Murray Hill-Kips Bay 3808 8 716 4 Queens 4071600 I QN98 Airport 4105 +1200016669 1 2015-07-27 2015-07-27 06:23:13 2015-07-27 2015-07-27 06:29:05 0 1 -73.98023223876953 40.739322662353516 -73.98213958740234 40.75185012817383 1 1.2 6.5 0 0.5 1 0 0 0.3 8.3 CSH 0 7 d yellow -11 66 1 Manhattan 1006600 I MN20 Murray Hill-Kips Bay 3808 209 84 1 Manhattan 1008400 I MN17 Midtown-Midtown South 3807 +1200016797 1 2015-07-27 2015-07-27 15:06:50 2015-07-27 2015-07-27 15:08:50 0 1 -73.96161651611328 40.77128601074219 -73.96227264404297 40.76762008666992 1 0.3 3.5 1 0.5 0 0 0 0.3 5.3 CRE 0 7 7 yellow 18 128 1 Manhattan 1012800 I MN40 Upper East Side-Carnegie Hill 3805 17 120 1 Manhattan 1012000 I MN40 Upper East Side-Carnegie Hill 3805 +1200016861 2 2015-07-27 2015-07-27 18:45:14 2015-07-27 2015-07-27 18:52:42 0 1 -73.9780044555664 40.762428283691406 -73.96321868896484 40.75747299194336 1 0.95 6.5 1 0.5 0.75 0 0 0.3 9.05 CSH 0 a ) yellow 17 104 1 Manhattan 1010400 I MN17 Midtown-Midtown South 3807 114 106.01 1 Manhattan 1010601 I MN19 Turtle Bay-East Midtown 3808 +1200016884 1 2015-07-27 2015-07-27 20:02:10 2015-07-27 2015-07-27 20:11:19 0 1 -73.98209381103516 40.765716552734375 -73.97746276855469 40.78622817993164 1 2 9 1 0.5 0 0 0 0.3 10.8 CRE 0 c 9 yellow 4 137 1 Manhattan 1013700 I MN17 Midtown-Midtown South 3807 166 171 1 Manhattan 1017100 I MN12 Upper West Side 3806 +1200016901 1 2015-07-27 2015-07-27 20:42:36 2015-07-27 2015-07-27 21:10:01 0 1 -73.7767105102539 40.64537048339844 -73.91156768798828 40.771568298339844 1 13.8 38.5 0.5 0.5 0 0 0 0.3 39.8 CRE 0 2 # yellow 8 716 4 Queens 4071600 I QN98 Airport 4105 58 117 4 Queens 4011700 I QN72 Steinway 4101 +1200016916 1 2015-07-27 2015-07-27 21:15:25 2015-07-27 2015-07-27 21:19:03 0 1 -73.9781265258789 40.765018463134766 -73.97034454345703 40.76416015625 1 0.7 4.5 0.5 0.5 0 0 0 0.3 5.8 CRE 0 c 7 yellow 4 137 1 Manhattan 1013700 I MN17 Midtown-Midtown South 3807 227 114.01 1 Manhattan 1011401 I MN40 Upper East Side-Carnegie Hill 3805 +1200016938 2 2015-07-27 2015-07-27 22:24:46 2015-07-27 2015-07-27 22:38:19 0 1 -73.99626159667969 40.76656723022461 -74.00994110107422 40.734954833984375 4 3.39 13 0.5 0.5 2.86 0 0 0.3 17.16 CSH 0 P X yellow -116 129 1 Manhattan 1012900 I MN15 Clinton 3807 150 75 1 Manhattan 1007500 I MN23 West Village 3810 +1200016974 2 2015-07-28 2015-07-28 00:47:55 2015-07-28 2015-07-28 00:59:36 0 1 -73.99604797363281 40.74850082397461 -73.97833251953125 40.729469299316406 1 2.34 10.5 0.5 0.5 2.36 0 0 0.3 14.16 CSH 0 � y yellow -125 95 1 Manhattan 1009500 I MN17 Midtown-Midtown South 3807 98 34 1 Manhattan 1003400 I MN22 East Village 3809 +1200016979 1 2015-07-28 2015-07-28 01:06:17 2015-07-28 2015-07-28 01:12:14 0 1 -73.9763412475586 40.751319885253906 -73.99356842041016 40.75648498535156 1 1.2 6.5 0.5 0.5 0 0 0 0.3 7.8 CRE 0 p H yellow -123 80 1 Manhattan 1008000 I MN20 Murray Hill-Kips Bay 3808 163 115 1 Manhattan 1011500 I MN15 Clinton 3807 +1200016984 1 2015-07-28 2015-07-28 01:39:12 2015-07-28 2015-07-28 01:54:23 0 1 -73.97196960449219 40.75859451293945 -73.89158630371094 40.74673843383789 1 4.8 16.5 0.5 0.5 0 0 0 0.3 17.8 CRE 0 b � yellow 15 100 1 Manhattan 1010000 I MN19 Turtle Bay-East Midtown 3808 45 267 4 Queens 4026700 E QN29 Elmhurst 4107 +1200016985 2 2015-07-28 2015-07-28 01:51:23 2015-07-28 2015-07-28 02:01:59 0 1 -73.98677825927734 40.76425552368164 -73.96615600585938 40.753971099853516 1 2.11 10 0.5 0.5 2.26 0 0 0.3 13.56 CSH 0 H ) yellow -119 133 1 Manhattan 1013300 I MN15 Clinton 3807 14 98 1 Manhattan 1009800 I MN19 Turtle Bay-East Midtown 3808 +1200017001 2 2015-07-28 2015-07-28 06:24:25 2015-07-28 2015-07-28 06:28:04 0 1 -73.97698211669922 40.74436950683594 -73.97562408447266 40.753536224365234 1 0.97 5 0 0.5 1.74 0 0 0.3 7.54 CSH 0 7 b yellow -10 70 1 Manhattan 1007000 I MN20 Murray Hill-Kips Bay 3808 226 92 1 Manhattan 1009200 I MN19 Turtle Bay-East Midtown 3808 +1200017070 2 2015-07-28 2015-07-28 10:43:58 2015-07-28 2015-07-28 11:05:36 0 1 -73.98497009277344 40.742183685302734 -74.01399993896484 40.71398162841797 2 3.86 18 0 0.5 0 0 0 0.3 18.8 CSH 0 4  yellow -90 56 1 Manhattan 1005600 I MN13 Hudson Yards-Chelsea-Flatiron-Union Square 3807 52 317.03 1 Manhattan 1031703 I MN25 Battery Park City-Lower Manhattan 3810 +1200017099 2 2015-07-28 2015-07-28 12:01:17 2015-07-28 2015-07-28 12:33:50 0 1 -73.96441650390625 40.79218673706055 -73.98910522460938 40.763126373291016 1 4.42 22 0 0.5 2.5 0 0 0.3 25.3 CSH 0 8 H yellow 24 181 1 Manhattan 1018100 I MN12 Upper West Side 3806 138 127 1 Manhattan 1012700 I MN15 Clinton 3807 +1200017109 1 2015-07-28 2015-07-28 12:36:01 2015-07-28 2015-07-28 13:10:38 0 1 -74.00177764892578 40.74055099487305 -73.87265014648438 40.77439880371094 1 10.6 36 0 0.5 8.45 5.54 0 0.3 50.79 CSH 0 � 8 yellow -115 81 1 Manhattan 1008100 I MN13 Hudson Yards-Chelsea-Flatiron-Union Square 3807 48 331 4 Queens 4033100 I QN98 Airport 4102 +1200017116 1 2015-07-28 2015-07-28 12:43:17 2015-07-28 2015-07-28 13:01:19 0 1 -73.99146270751953 40.74981689453125 -73.9718017578125 40.76035690307617 1 1.9 12.5 0 0.5 0 0 0 0.3 13.3 CRE 0 � a yellow -125 101 1 Manhattan 1010100 I MN17 Midtown-Midtown South 3807 16 102 1 Manhattan 1010200 I MN17 Midtown-Midtown South 3807 +1200017199 2 2015-07-28 2015-07-28 17:27:52 2015-07-28 2015-07-28 17:52:50 0 1 -73.9822006225586 40.772674560546875 -74.0029525756836 40.72337341308594 1 5.04 21 1 0.5 2 0 0 0.3 24.8 CSH 0 B  yellow 67 149 1 Manhattan 1014900 I MN14 Lincoln Square 3806 16 47 1 Manhattan 1004700 I MN24 SoHo-TriBeCa-Civic Center-Little Italy 3810 +1200017226 1 2015-07-28 2015-07-28 18:54:48 2015-07-28 2015-07-28 19:17:16 0 1 -74.00302124023438 40.728065490722656 -73.96629333496094 40.76388168334961 1 3.7 17 1 0.5 0 0 0 0.3 18.8 CRE 0 % 7 yellow 13 37 1 Manhattan 1003700 I MN24 SoHo-TriBeCa-Civic Center-Little Italy 3810 22 114.02 1 Manhattan 1011402 I MN40 Upper East Side-Carnegie Hill 3805 +1200017229 1 2015-07-28 2015-07-28 19:01:46 2015-07-28 2015-07-28 19:05:40 0 1 -73.970703125 40.76081085205078 -73.96141815185547 40.76532745361328 1 0.7 5 1 0.5 1.35 0 0 0.3 8.15 CSH 0 b A yellow 20 112.03 1 Manhattan 1011203 I MN19 Turtle Bay-East Midtown 3808 182 118 1 Manhattan 1011800 I MN31 Lenox Hill-Roosevelt Island 3805 +1200017244 1 2015-07-28 2015-07-28 19:37:12 2015-07-28 2015-07-28 19:53:51 0 1 -74.00997161865234 40.73249816894531 -73.97714233398438 40.79117202758789 2 4.7 16.5 0.5 0.5 1 0 0 0.3 18.8 CSH 0 X 8 yellow -106 75 1 Manhattan 1007500 I MN23 West Village 3810 138 175 1 Manhattan 1017500 I MN12 Upper West Side 3806 +1200017248 1 2015-07-28 2015-07-28 19:41:38 2015-07-28 2015-07-28 19:47:01 0 1 -73.99111938476562 40.74939727783203 -74.00306701660156 40.73832321166992 1 1.1 6 1 0.5 1 0 0 0.3 8.8 CSH 0 � I yellow -125 101 1 Manhattan 1010100 I MN17 Midtown-Midtown South 3807 78 77 1 Manhattan 1007700 I MN23 West Village 3810 +1200017306 1 2015-07-28 2015-07-28 21:54:22 2015-07-28 2015-07-28 22:16:51 0 1 -73.96709442138672 40.80368423461914 -73.98595428466797 40.72225570678711 1 7.9 25.5 0.5 0.5 5.35 0 0 0.3 32.15 CSH 0 $ y yellow 111 195 1 Manhattan 1019500 I MN09 Morningside Heights 3802 162 30.02 1 Manhattan 1003002 E MN22 East Village 3809 +1200017319 2 2015-07-28 2015-07-28 22:42:07 2015-07-28 2015-07-28 23:29:05 0 1 -73.98033905029297 40.77035140991211 -73.98873901367188 40.685298919677734 1 7.95 33.5 0.5 0.5 6.96 0 0 0.3 41.76 CSH 0 B % yellow 67 149 1 Manhattan 1014900 I MN14 Lincoln Square 3806 65 69 3 Brooklyn 3006900 I BK38 DUMBO-Vinegar Hill-Downtown Brooklyn-Boerum Hill 4004 +1200017348 2 2015-07-29 2015-07-29 00:34:34 2015-07-29 2015-07-29 01:00:30 0 1 -73.78941345214844 40.64738845825195 -73.9529800415039 40.67289352416992 1 11.42 33.5 0.5 0.5 6.96 0 0 0.3 41.76 CSH 0 2 a yellow 8 716 4 Queens 4071600 I QN98 Airport 4105 177 317.01 3 Brooklyn 3031701 E BK61 Crown Heights North 4006 +1200017400 1 2015-07-29 2015-07-29 08:49:35 2015-07-29 2015-07-29 08:52:46 0 1 -73.96286010742188 40.76658248901367 -73.96470642089844 40.7729377746582 1 0.6 4.5 0 0.5 1.5 0 0 0.3 6.8 CSH 0 A 7 yellow -74 118 1 Manhattan 1011800 I MN31 Lenox Hill-Roosevelt Island 3805 23 130 1 Manhattan 1013000 I MN40 Upper East Side-Carnegie Hill 3805 +1200017403 2 2015-07-29 2015-07-29 08:55:07 2015-07-29 2015-07-29 09:05:48 0 1 -73.97953033447266 40.77120590209961 -73.95823669433594 40.76464080810547 5 1.37 8.5 0 0.5 1 0 0 0.3 10.3 CSH 0 C @ yellow 43 143 1 Manhattan 1014300 I MN99 park-cemetery-etc-Manhattan 3806 222 116 1 Manhattan 1011600 I MN31 Lenox Hill-Roosevelt Island 3805 +1200017407 2 2015-07-29 2015-07-29 09:02:34 2015-07-29 2015-07-29 09:06:56 0 1 -73.98629760742188 40.772239685058594 -73.98182678222656 40.765380859375 1 0.85 5 0 0.5 1.74 0 0 0.3 7.54 CSH 0 C c yellow -51 151 1 Manhattan 1015100 E MN14 Lincoln Square 3806 4 137 1 Manhattan 1013700 I MN17 Midtown-Midtown South 3807 +1200017408 2 2015-07-29 2015-07-29 09:03:48 2015-07-29 2015-07-29 09:10:20 0 1 -73.98148345947266 40.74112319946289 -73.97891235351562 40.75092315673828 4 0.87 6 0 0.5 0 0 0 0.3 6.8 CRE 0  p yellow 11 68 1 Manhattan 1006800 I MN21 Gramercy 3808 93 82 1 Manhattan 1008200 I MN20 Murray Hill-Kips Bay 3808 +1200017420 2 2015-07-29 2015-07-29 09:49:22 2015-07-29 2015-07-29 10:06:14 0 1 -73.981689453125 40.76258087158203 -73.99871826171875 40.76438522338867 1 1.51 11.5 0 0.5 0 0 0 0.3 12.3 CRE 0 0 P yellow 3 131 1 Manhattan 1013100 I MN17 Midtown-Midtown South 3807 140 129 1 Manhattan 1012900 I MN15 Clinton 3807 +1200017461 2 2015-07-29 2015-07-29 12:35:12 2015-07-29 2015-07-29 12:53:12 0 1 -73.98368072509766 40.74655532836914 -73.95536804199219 40.76747131347656 2 2.61 13 0 0.5 2 0 0 0.3 15.8 CSH 0 d @ yellow -125 74 1 Manhattan 1007400 I MN17 Midtown-Midtown South 3807 160 124 1 Manhattan 1012400 I MN31 Lenox Hill-Roosevelt Island 3805 +1200017464 1 2015-07-29 2015-07-29 07:46:01 2015-07-29 2015-07-29 07:54:21 0 1 -73.98493957519531 40.732452392578125 -74.00563049316406 40.726470947265625 1 1.8 8.5 0 0.5 1.85 0 0 0.3 11.15 CSH 0  % yellow 52 48 1 Manhattan 1004800 I MN21 Gramercy 3808 13 37 1 Manhattan 1003700 I MN24 SoHo-TriBeCa-Civic Center-Little Italy 3810 +1200017529 1 2015-07-29 2015-07-29 15:45:36 2015-07-29 2015-07-29 16:02:15 0 1 -73.98775482177734 40.7227783203125 -74.00405883789062 40.707454681396484 1 3.1 13.5 0 0.5 0 0 0 0.3 14.3 CRE 0 y \t yellow -94 30.02 1 Manhattan 1003002 E MN22 East Village 3809 155 15.01 1 Manhattan 1001501 I MN25 Battery Park City-Lower Manhattan 3810 +1200017532 1 2015-07-29 2015-07-29 15:51:15 2015-07-29 2015-07-29 16:06:06 0 1 -73.99359893798828 40.74224853515625 -73.99362182617188 40.76387023925781 1 2.2 11.5 1 0.5 2.65 0 0 0.3 15.95 CSH 0 � P yellow -113 87 1 Manhattan 1008700 I MN13 Hudson Yards-Chelsea-Flatiron-Union Square 3807 140 129 1 Manhattan 1012900 I MN15 Clinton 3807 +1200017575 2 2015-07-29 2015-07-29 17:58:29 2015-07-29 2015-07-29 18:35:32 0 1 -73.96412658691406 40.77067184448242 -74.00127410888672 40.73201370239258 1 3.92 23 1 0.5 4.96 0 0 0.3 29.76 CSH 0 7 I yellow 18 128 1 Manhattan 1012800 I MN40 Upper East Side-Carnegie Hill 3805 162 71 1 Manhattan 1007100 I MN23 West Village 3810 +1200017584 2 2015-07-29 2015-07-29 18:16:40 2015-07-29 2015-07-29 18:25:12 0 1 -73.9506607055664 40.782291412353516 -73.96286010742188 40.76681900024414 1 1.5 8 1 0.5 0 0 0 0.3 9.8 CRE 0 c 7 yellow 31 154 1 Manhattan 1015400 I MN32 Yorkville 3805 17 120 1 Manhattan 1012000 I MN40 Upper East Side-Carnegie Hill 3805 +1200017597 1 2015-07-29 2015-07-29 18:57:58 2015-07-29 2015-07-29 19:26:35 0 1 -73.98580169677734 40.76377868652344 -73.9972152709961 40.72012710571289 1 3.8 19 1 0.5 5.2 0 0 0.3 26 CSH 0 H D yellow -119 133 1 Manhattan 1013300 I MN15 Clinton 3807 196 41 1 Manhattan 1004100 E MN24 SoHo-TriBeCa-Civic Center-Little Italy 3810 +1200017638 2 2015-07-29 2015-07-29 20:34:25 2015-07-29 2015-07-29 20:56:35 0 1 -73.96286010742188 40.719120025634766 -73.98399353027344 40.66391372680664 1 6.91 22.5 0.5 0.5 3.2 0 0 0.3 27 CSH 0 U � yellow -65 555 3 Brooklyn 3055500 I BK73 North Side-South Side 4001 221 149 3 Brooklyn 3014900 I BK37 Park Slope-Gowanus 4005 +1200017724 2 2015-07-30 2015-07-30 00:32:10 2015-07-30 2015-07-30 00:53:42 0 1 -74.0023193359375 40.740211486816406 -73.8655014038086 40.77067184448242 1 11.15 31 0.5 0.5 0 5.54 0 0.3 37.84 CRE 0 h 8 yellow 75 83 1 Manhattan 1008300 E MN13 Hudson Yards-Chelsea-Flatiron-Union Square 3807 48 331 4 Queens 4033100 I QN98 Airport 4102 +1200017736 1 2015-07-30 2015-07-30 02:31:47 2015-07-30 2015-07-30 02:37:29 1 1 -73.97508239746094 40.77745819091797 -73.96678924560547 40.79352951049805 1 1.3 7 0.5 0.5 0 0 0 0.3 8.3 CRE 0 B Q yellow 21 157 1 Manhattan 1015700 I MN14 Lincoln Square 3806 25 185 1 Manhattan 1018500 I MN12 Upper West Side 3806 +1200017740 1 2015-07-30 2015-07-30 04:29:25 2015-07-30 2015-07-30 04:35:31 0 1 -73.9973373413086 40.72175216674805 -73.99876403808594 40.734031677246094 1 1.2 6.5 0.5 0.5 1.56 0 0 0.3 9.36 CSH 0 D  yellow 124 45 1 Manhattan 1004500 I MN24 SoHo-TriBeCa-Civic Center-Little Italy 3810 134 63 1 Manhattan 1006300 I MN23 West Village 3810 +1200017744 2 2015-07-30 2015-07-30 05:25:37 2015-07-30 2015-07-30 05:36:37 0 1 -74.01275634765625 40.716861724853516 -73.9859619140625 40.7606201171875 1 3.91 13.5 0.5 0.5 0 0 0 0.3 14.8 CRE 0 1 0 yellow 47 21 1 Manhattan 1002100 I MN24 SoHo-TriBeCa-Civic Center-Little Italy 3810 134 125 1 Manhattan 1012500 I MN17 Midtown-Midtown South 3807 +1200017763 2 2015-07-30 2015-07-30 06:54:31 2015-07-30 2015-07-30 07:00:46 0 1 -73.95655059814453 40.766929626464844 -73.97112274169922 40.757911682128906 1 1.3900000000000001 6.5 0 0.5 1 0 0 0.3 8.3 CSH 0 @ b yellow -96 124 1 Manhattan 1012400 I MN31 Lenox Hill-Roosevelt Island 3805 15 100 1 Manhattan 1010000 I MN19 Turtle Bay-East Midtown 3808 +1200017777 2 2015-07-30 2015-07-30 07:51:14 2015-07-30 2015-07-30 08:10:26 0 1 -73.97950744628906 40.746700286865234 -74.01618194580078 40.70905303955078 3 5.82 21 0 0.5 4.36 0 0 0.3 26.16 CSH 0 p  yellow -126 72 1 Manhattan 1007200 I MN20 Murray Hill-Kips Bay 3808 54 317.04 1 Manhattan 1031704 I MN25 Battery Park City-Lower Manhattan 3810 +1200017781 1 2015-07-30 2015-07-30 08:07:44 2015-07-30 2015-07-30 08:11:35 0 1 -73.97814178466797 40.74580001831055 -73.98471069335938 40.747276306152344 1 0.5 4.5 0 0.5 1.05 0 0 0.3 6.35 CSH 0 p d yellow -126 72 1 Manhattan 1007200 I MN20 Murray Hill-Kips Bay 3808 131 74 1 Manhattan 1007400 I MN17 Midtown-Midtown South 3807 +1200017800 2 2015-07-30 2015-07-30 09:06:06 2015-07-30 2015-07-30 09:12:36 0 1 -73.939697265625 40.80534362792969 -73.95298767089844 40.81489181518555 1 1.22 6.5 0 0.5 0 0 0 0.3 7.3 CRE 0 t R yellow 56 198 1 Manhattan 1019800 I MN34 East Harlem North 3804 182 213.03 1 Manhattan 1021303 E MN06 Manhattanville 3802 +1200017835 2 2015-07-30 2015-07-30 11:23:11 2015-07-30 2015-07-30 11:44:56 0 1 -73.99160766601562 40.750179290771484 -74.0114974975586 40.70362091064453 1 5.39 21 0 0.5 5.45 0 0 0.3 27.25 CSH 0 � � yellow -125 101 1 Manhattan 1010100 I MN17 Midtown-Midtown South 3807 0 9 1 Manhattan 1000900 I MN25 Battery Park City-Lower Manhattan 3810 +1200017839 1 2015-07-30 2015-07-30 11:44:58 2015-07-30 2015-07-30 11:54:26 0 1 -73.98810577392578 40.74974060058594 -73.98690795898438 40.74279022216797 1 1 7.5 0 0.5 1.65 0 0 0.3 9.95 CSH 0 � 4 yellow -125 101 1 Manhattan 1010100 I MN17 Midtown-Midtown South 3807 166 56 1 Manhattan 1005600 I MN13 Hudson Yards-Chelsea-Flatiron-Union Square 3807 +1200017853 2 2015-07-30 2015-07-30 12:17:38 2015-07-30 2015-07-30 12:37:39 0 1 -73.9915771484375 40.750038146972656 -73.97966766357422 40.739540100097656 1 1.18 12.5 0 0.5 0 0 0 0.3 13.3 CRE 0 � 7 yellow -125 101 1 Manhattan 1010100 I MN17 Midtown-Midtown South 3807 245 66 1 Manhattan 1006600 I MN20 Murray Hill-Kips Bay 3808 +1200017979 2 2015-07-30 2015-07-30 19:11:06 2015-07-30 2015-07-30 19:29:11 0 1 -73.9789047241211 40.740970611572266 -73.89105224609375 40.719459533691406 1 6.12 19.5 1 0.5 4 5.54 0 0.3 30.84 CSH 0 7 ` yellow -11 66 1 Manhattan 1006600 I MN20 Murray Hill-Kips Bay 3808 172 665.01 4 Queens 4066501 I QN21 Middle Village 4110 +1200017997 2 2015-07-30 2015-07-30 19:52:14 2015-07-30 2015-07-30 20:01:27 0 1 -73.9710464477539 40.76177978515625 -73.9533920288086 40.77286911010742 6 1.6 8.5 1 0.5 0 0 0 0.3 10.3 CRE 0 c c yellow 19 112.02 1 Manhattan 1011202 I MN17 Midtown-Midtown South 3807 0 138 1 Manhattan 1013800 I MN32 Yorkville 3805 +1200018004 2 2015-07-30 2015-07-30 20:27:37 2015-07-30 2015-07-30 20:42:31 0 1 -74.00823211669922 40.73757553100586 -74.01374053955078 40.702232360839844 1 2.68 12.5 0.5 0.5 1 0 0 0.3 14.8 CSH 0 X � yellow 79 79 1 Manhattan 1007900 I MN23 West Village 3810 0 9 1 Manhattan 1000900 I MN25 Battery Park City-Lower Manhattan 3810 +1200018009 2 2015-07-30 2015-07-30 20:35:29 2015-07-30 2015-07-30 21:08:55 0 1 -74.00675964355469 40.709075927734375 -74.00646209716797 40.744503021240234 2 2.83 21 0.5 0.5 5.58 0 0 0.3 27.88 CSH 0 � h yellow 12 15.02 1 Manhattan 1001502 I MN25 Battery Park City-Lower Manhattan 3810 75 83 1 Manhattan 1008300 E MN13 Hudson Yards-Chelsea-Flatiron-Union Square 3807 +1200018022 1 2015-07-30 2015-07-30 21:17:17 2015-07-30 2015-07-30 21:21:30 0 1 -73.8730239868164 40.77409362792969 -73.86754608154297 40.767921447753906 1 0.9 5 0.5 0.5 0 0 0 0.3 6.3 CRE 0 8 p yellow 48 331 4 Queens 4033100 I QN98 Airport 4102 73 357 4 Queens 4035700 I QN27 East Elmhurst 4102 +1200018052 1 2015-07-30 2015-07-30 22:36:21 2015-07-30 2015-07-30 22:41:08 0 1 -73.99054718017578 40.71885299682617 -73.98653411865234 40.72903060913086 2 0.8 5 0.5 0.5 0 0 0 0.3 6.3 CSH 0 H y yellow -61 18 1 Manhattan 1001800 E MN27 Chinatown 3809 93 40 1 Manhattan 1004000 I MN22 East Village 3809 +1200018066 1 2015-07-30 2015-07-30 23:26:59 2015-07-30 2015-07-30 23:44:07 0 1 -73.96642303466797 40.764610290527344 -73.99488067626953 40.760154724121094 1 2.1 12.5 0.5 0.5 0 0 0 0.3 13.8 CRE 0 7 H yellow 22 114.02 1 Manhattan 1011402 I MN40 Upper East Side-Carnegie Hill 3805 139 121 1 Manhattan 1012100 E MN15 Clinton 3807 +1200018092 1 2015-07-31 2015-07-31 00:12:58 2015-07-31 2015-07-31 00:31:04 0 1 -73.78865051269531 40.64781951904297 -73.94070434570312 40.61478042602539 1 13.4 36.5 0.5 0.5 0 0 0 0.3 37.8 CRE 0 2 U yellow 8 716 4 Queens 4071600 I QN98 Airport 4105 125 644 3 Brooklyn 3064400 I BK45 Georgetown-Marine Park-Bergen Beach-Mill Basin 4009 +1200018109 2 2015-07-31 2015-07-31 02:04:35 2015-07-31 2015-07-31 02:19:16 0 1 -74.00676727294922 40.73946762084961 -73.9912338256836 40.717403411865234 1 2.26 11.5 0.5 0.5 2.56 0 0 0.3 15.36 CSH 0 X H yellow 79 79 1 Manhattan 1007900 I MN23 West Village 3810 195 18 1 Manhattan 1001800 E MN27 Chinatown 3809 +1200018128 2 2015-07-31 2015-07-31 05:12:28 2015-07-31 2015-07-31 05:32:40 0 1 -73.91934204101562 40.743446350097656 -74.00828552246094 40.74658203125 1 6.49 22 0.5 0.5 0 0 0 0.3 23.3 CRE 0 & F yellow 18 183 4 Queens 4018300 I QN31 Hunters Point-Sunnyside-West Maspeth 4109 50 99 1 Manhattan 1009900 I MN13 Hudson Yards-Chelsea-Flatiron-Union Square 3807 +1200018140 1 2015-07-31 2015-07-31 06:43:47 2015-07-31 2015-07-31 06:55:02 0 1 -73.98245239257812 40.777103424072266 -73.94294738769531 40.840572357177734 1 5.6 17.5 0 0.5 2 0 0 0.3 20.3 CSH 0 B D yellow 30 153 1 Manhattan 1015300 I MN14 Lincoln Square 3806 187 255 1 Manhattan 1025500 E MN36 Washington Heights South 3801 +1200018149 2 2015-07-31 2015-07-31 08:07:03 2015-07-31 2015-07-31 08:19:40 0 1 -73.9942626953125 40.72845458984375 -73.97552490234375 40.750423431396484 4 2.03 10.5 0 0.5 2.26 0 0 0.3 13.56 CSH 0  p yellow -64 57 1 Manhattan 1005700 I MN23 West Village 3810 133 80 1 Manhattan 1008000 I MN20 Murray Hill-Kips Bay 3808 +1200018220 1 2015-07-31 2015-07-31 12:53:54 2015-07-31 2015-07-31 13:05:57 0 1 -73.96943664550781 40.76631164550781 -73.94035339355469 40.799835205078125 2 2.9 11.5 0 0.5 0 0 0 0.3 12.3 CRE 0 7 t yellow -28 122 1 Manhattan 1012200 I MN40 Upper East Side-Carnegie Hill 3805 230 182 1 Manhattan 1018200 E MN34 East Harlem North 3804 +1200018245 2 2015-07-31 2015-07-31 14:11:29 2015-07-31 2015-07-31 14:12:37 0 1 -73.9869384765625 40.76028823852539 -73.98323059082031 40.76589584350586 1 0.44 3.5 0 0.5 0 0 0 0.3 4.3 CRE 0 0 c yellow -122 125 1 Manhattan 1012500 I MN17 Midtown-Midtown South 3807 4 137 1 Manhattan 1013700 I MN17 Midtown-Midtown South 3807 +1200018282 1 2015-07-31 2015-07-31 17:02:42 2015-07-31 2015-07-31 17:37:25 0 1 -73.87059783935547 40.773563385009766 -73.95394134521484 40.743316650390625 1 6.3 26 1 0.5 5.55 0 0 0.3 33.35 CSH 0 8 E yellow 48 331 4 Queens 4033100 I QN98 Airport 4102 34 7 4 Queens 4000700 I QN31 Hunters Point-Sunnyside-West Maspeth 4109 +1200018288 2 2015-07-31 2015-07-31 17:19:53 2015-07-31 2015-07-31 17:22:51 0 1 -73.95670318603516 40.77098846435547 -73.95491790771484 40.7776985168457 1 0.59 4 1 0.5 1.45 0 0 0.3 7.25 CSH 0 A 6 yellow -38 134 1 Manhattan 1013400 I MN31 Lenox Hill-Roosevelt Island 3805 28 148.01 1 Manhattan 1014801 I MN40 Upper East Side-Carnegie Hill 3805 +1200018297 2 2015-07-31 2015-07-31 18:06:39 2015-07-31 2015-07-31 18:26:27 0 1 -73.98350524902344 40.76032638549805 -74.00687408447266 40.733070373535156 1 2.34 14 1 0.5 0 0 0 0.3 15.8 CRE 0 0 X yellow -122 125 1 Manhattan 1012500 I MN17 Midtown-Midtown South 3807 150 75 1 Manhattan 1007500 I MN23 West Village 3810 +1200018301 2 2015-07-31 2015-07-31 18:12:40 2015-07-31 2015-07-31 18:30:33 0 1 -73.98137664794922 40.75775146484375 -73.96910858154297 40.78609848022461 1 2.33 13 1 0.5 2.96 0 0 0.3 17.76 CSH 0 a 8 yellow 13 96 1 Manhattan 1009600 E MN17 Midtown-Midtown South 3807 137 173 1 Manhattan 1017300 I MN12 Upper West Side 3806 +1200018312 2 2015-07-31 2015-07-31 18:57:05 2015-07-31 2015-07-31 19:20:56 0 1 -73.99134063720703 40.75016784667969 -73.99193572998047 40.719810485839844 3 2.8 16 1 0.5 0 0 0 0.3 17.8 CRE 0 � H yellow -125 101 1 Manhattan 1010100 I MN17 Midtown-Midtown South 3807 195 18 1 Manhattan 1001800 E MN27 Chinatown 3809 +1200018313 1 2015-07-31 2015-07-31 18:58:14 2015-07-31 2015-07-31 19:16:33 0 1 -73.86396026611328 40.76956558227539 -73.93118286132812 40.756126403808594 2 4.5 17 1 0.5 3.75 0 0 0.3 22.55 CSH 0 8 F yellow 48 331 4 Queens 4033100 I QN98 Airport 4102 38 51 4 Queens 4005100 E QN68 Queensbridge-Ravenswood-Long Island City 4101 +1200018332 2 2015-07-31 2015-07-31 19:36:34 2015-07-31 2015-07-31 20:03:40 0 1 -73.98391723632812 40.69621276855469 -73.99517822265625 40.75007629394531 1 4.58 20.5 1 0.5 5.58 0 0 0.3 27.88 CSH 0 e h yellow -56 15 3 Brooklyn 3001500 E BK38 DUMBO-Vinegar Hill-Downtown Brooklyn-Boerum Hill 4004 103 103 1 Manhattan 1010300 I MN13 Hudson Yards-Chelsea-Flatiron-Union Square 3807 +1200018380 2 2015-07-31 2015-07-31 21:29:17 2015-07-31 2015-07-31 22:06:57 0 1 -73.9892578125 40.75263595581055 -73.86386108398438 40.85472869873047 2 12.69 40 0.5 0.5 0 0 0 0.3 41.3 CSH 0  � yellow -124 109 1 Manhattan 1010900 I MN17 Midtown-Midtown South 3807 36 228 2 Bronx 2022800 E BX49 Pelham Parkway 3704 +1200018453 1 2015-07-06 2015-07-06 17:34:37 2015-07-06 2015-07-06 18:06:24 0 1 0 0 0 0 1 3.6 20.5 1 0.5 4.45 0 0 0.3 26.75 CSH 0 yellow 0 0 0 0000000 0000 0 0 0 0 0000000 0000 0 +1200018498 1 2015-07-01 2015-07-01 11:40:37 2015-07-01 2015-07-01 12:17:18 0 1 -74.2427749633789 40.75882339477539 -74.24462890625 40.763023376464844 1 2.6 22 0 0.5 4.56 0 0 0.3 27.36 CSH 0 yellow 0 0 0 0000000 0000 0 0 0 0 0000000 0000 0 +1200018507 1 2015-07-06 2015-07-06 14:21:01 2015-07-06 2015-07-06 14:34:14 0 1 0 0 0 0 1 6.6 19.5 0 0.5 0 0 0 0.3 20.3 CRE 0 yellow 0 0 0 0000000 0000 0 0 0 0 0000000 0000 0 +1200018576 1 2015-07-07 2015-07-07 22:04:01 2015-07-07 2015-07-07 22:04:01 0 1 -73.99372863769531 40.732826232910156 0 0 1 0 2.5 0.5 0.5 0 0 0 0.3 3.8 CRE 0  yellow -55 61 1 Manhattan 1006100 I MN23 West Village 3810 0 0 0 0000000 0000 0 +1200018592 2 2015-07-05 2015-07-05 00:17:27 2015-07-05 2015-07-05 00:27:36 0 1 0 0 0 0 1 1.12 8 0.5 0.5 1.86 0 0 0.3 11.16 CSH 0 yellow 0 0 0 0000000 0000 0 0 0 0 0000000 0000 0 +1200018597 1 2015-07-09 2015-07-09 23:57:35 2015-07-10 2015-07-10 00:08:17 0 1 0 0 0 0 1 4.5 14.5 0.5 0.5 0 0 0 0.3 15.8 CRE 0 yellow 0 0 0 0000000 0000 0 0 0 0 0000000 0000 0 +1200018599 1 2015-07-17 2015-07-17 20:12:21 2015-07-17 2015-07-17 20:15:38 0 1 0 0 0 0 1 0.6 4.5 0.5 0.5 1.15 0 0 0.3 6.95 CSH 0 yellow 0 0 0 0000000 0000 0 0 0 0 0000000 0000 0 +1200018606 1 2015-07-10 2015-07-10 00:21:18 2015-07-10 2015-07-10 00:36:28 0 1 0 0 0 0 2 4.7 16 0.5 0.5 2.7 0 0 0.3 20 CSH 0 yellow 0 0 0 0000000 0000 0 0 0 0 0000000 0000 0 +1200018648 2 2015-07-01 2015-07-01 00:00:16 2015-07-01 2015-07-01 00:02:57 0 1 -73.78358459472656 40.648677825927734 -73.80242919921875 40.64767837524414 1 1.45 6 0.5 0.5 0 0 0 0.3 7.3 CRE 0 2 2 yellow 8 716 4 Queens 4071600 I QN98 Airport 4105 8 716 4 Queens 4071600 I QN98 Airport 4105 +1200018703 2 2015-07-01 2015-07-01 07:32:19 2015-07-01 2015-07-01 07:52:34 0 1 -73.99331665039062 40.74565124511719 -73.98870849609375 40.6933708190918 1 4.49 17 0 0.5 3.56 0 0 0.3 21.36 CSH 0 � e yellow -125 95 1 Manhattan 1009500 I MN17 Midtown-Midtown South 3807 149 11 3 Brooklyn 3001100 I BK38 DUMBO-Vinegar Hill-Downtown Brooklyn-Boerum Hill 4004 +1200018721 1 2015-07-01 2015-07-01 08:43:00 2015-07-01 2015-07-01 09:01:44 0 1 -73.98637390136719 40.71818923950195 -73.9807357788086 40.75232696533203 1 3 14 0 0.5 2.95 0 0 0.3 17.75 CSH 0 2 p yellow -30 14.02 1 Manhattan 1001402 E MN28 Lower East Side 3809 93 82 1 Manhattan 1008200 I MN20 Murray Hill-Kips Bay 3808 +1200018743 2 2015-07-01 2015-07-01 10:09:02 2015-07-01 2015-07-01 10:15:57 0 1 -73.96907806396484 40.785640716552734 -73.98242950439453 40.774131774902344 5 1.2 7 0 0.5 0 0 0 0.3 7.8 CRE 0 C B yellow 43 143 1 Manhattan 1014300 I MN99 park-cemetery-etc-Manhattan 3806 30 153 1 Manhattan 1015300 I MN14 Lincoln Square 3806 +1200018746 1 2015-07-01 2015-07-01 10:16:11 2015-07-01 2015-07-01 10:35:43 0 1 -73.9482650756836 40.78974151611328 -73.97135162353516 40.7584114074707 3 2.8 15 0 0.5 3.15 0 0 0.3 18.95 CSH 0 u b yellow 120 166 1 Manhattan 1016600 E MN33 East Harlem South 3804 15 100 1 Manhattan 1010000 I MN19 Turtle Bay-East Midtown 3808 +1200018766 2 2015-07-01 2015-07-01 11:34:48 2015-07-01 2015-07-01 11:44:54 0 1 -73.99299621582031 40.7684440612793 -73.95198059082031 40.813865661621094 2 4.64 15 0 0.5 3.16 0 0 0.3 18.96 CSH 0 P R yellow -121 135 1 Manhattan 1013500 E MN15 Clinton 3807 182 213.03 1 Manhattan 1021303 E MN06 Manhattanville 3802 +1200018780 2 2015-07-01 2015-07-01 12:18:24 2015-07-01 2015-07-01 12:44:39 0 1 -73.99076843261719 40.74204635620117 -73.9783706665039 40.76279067993164 1 1.87 15.5 0 0.5 0 0 0 0.3 16.3 CRE 0 4 c yellow -123 58 1 Manhattan 1005800 I MN13 Hudson Yards-Chelsea-Flatiron-Union Square 3807 4 137 1 Manhattan 1013700 I MN17 Midtown-Midtown South 3807 +1200018804 1 2015-07-01 2015-07-01 13:39:53 2015-07-01 2015-07-01 14:06:16 0 1 -73.98249816894531 40.75565719604492 -73.9769287109375 40.775760650634766 1 3.9 18 0 0.5 0.5 0 0 0.3 19.3 CSH 0 a B yellow 13 96 1 Manhattan 1009600 E MN17 Midtown-Midtown South 3807 21 157 1 Manhattan 1015700 I MN14 Lincoln Square 3806 +1200018815 1 2015-07-01 2015-07-01 14:17:19 2015-07-01 2015-07-01 14:51:11 0 1 -73.97748565673828 40.779319763183594 -74.01673126220703 40.715538024902344 3 5.5 24.5 0 0.5 5.06 0 0 0.3 30.36 CSH 0 9  yellow 63 161 1 Manhattan 1016100 I MN12 Upper West Side 3806 52 317.03 1 Manhattan 1031703 I MN25 Battery Park City-Lower Manhattan 3810 +1200018865 2 2015-07-01 2015-07-01 15:16:52 2015-07-01 2015-07-01 15:33:10 0 1 -73.9778823852539 40.7540283203125 -73.98359680175781 40.75899887084961 5 0.62 10.5 0 0.5 0 0 0 0.3 11.3 CRE 0 a 0 yellow -31 94 1 Manhattan 1009400 E MN17 Midtown-Midtown South 3807 134 125 1 Manhattan 1012500 I MN17 Midtown-Midtown South 3807 +1200018873 2 2015-07-01 2015-07-01 17:47:47 2015-07-01 2015-07-01 17:57:21 0 1 -73.98241424560547 40.75059509277344 -73.9901123046875 40.733028411865234 1 1.53 8 1 0.5 0 0 0 0.3 9.8 CRE 0 p  yellow 93 82 1 Manhattan 1008200 I MN20 Murray Hill-Kips Bay 3808 201 61 1 Manhattan 1006100 I MN23 West Village 3810 +1200018893 1 2015-07-01 2015-07-01 18:32:00 2015-07-01 2015-07-01 18:41:38 0 1 -73.99095916748047 40.72394943237305 -73.98959350585938 40.70162582397461 1 2.4 9.5 1 0.5 2.8 0 0 0.3 14.1 CSH 0 y f yellow -62 36.02 1 Manhattan 1003602 I MN22 East Village 3809 248 21 3 Brooklyn 3002100 I BK38 DUMBO-Vinegar Hill-Downtown Brooklyn-Boerum Hill 4004 +1200019028 2 2015-07-02 2015-07-02 04:43:13 2015-07-02 2015-07-02 04:49:25 0 1 -73.9906005859375 40.74587631225586 -73.9846420288086 40.76765823364258 1 2.27 8.5 0.5 0.5 2.45 0 0 0.3 12.25 CSH 0 4 H yellow -123 58 1 Manhattan 1005800 I MN13 Hudson Yards-Chelsea-Flatiron-Union Square 3807 202 139 1 Manhattan 1013900 I MN15 Clinton 3807 +1200019036 2 2015-07-02 2015-07-02 06:01:01 2015-07-02 2015-07-02 06:12:45 0 1 -73.97764587402344 40.75246810913086 -74.01012420654297 40.72008514404297 1 3.48 12.5 0 0.5 2 0 0 0.3 15.3 CSH 0 b 1 yellow -30 92 1 Manhattan 1009200 I MN19 Turtle Bay-East Midtown 3808 14 39 1 Manhattan 1003900 I MN24 SoHo-TriBeCa-Civic Center-Little Italy 3810 +1200019039 2 2015-07-02 2015-07-02 06:13:51 2015-07-02 2015-07-02 06:20:50 0 1 -74.00222778320312 40.7331428527832 -73.98384857177734 40.760189056396484 2 2.39 9 0 0.5 1.5 0 0 0.3 11.3 CSH 0 I 0 yellow -94 71 1 Manhattan 1007100 I MN23 West Village 3810 134 125 1 Manhattan 1012500 I MN17 Midtown-Midtown South 3807 +1200019049 2 2015-07-02 2015-07-02 07:01:43 2015-07-02 2015-07-02 07:12:22 0 1 -73.95487213134766 40.765750885009766 -73.98316192626953 40.760231018066406 3 2.14 9.5 0 0.5 2.06 0 0 0.3 12.36 CSH 0 @ 0 yellow -96 124 1 Manhattan 1012400 I MN31 Lenox Hill-Roosevelt Island 3805 134 125 1 Manhattan 1012500 I MN17 Midtown-Midtown South 3807 +1200019070 1 2015-07-02 2015-07-02 08:05:20 2015-07-02 2015-07-02 08:14:21 1 1 -73.99263000488281 40.73563003540039 -73.98604583740234 40.747379302978516 1 1.3 7.5 0 0.5 1.65 0 0 0.3 9.95 CSH 0 4 d yellow -11 52 1 Manhattan 1005200 I MN13 Hudson Yards-Chelsea-Flatiron-Union Square 3807 208 76 1 Manhattan 1007600 I MN17 Midtown-Midtown South 3807 +1200019086 1 2015-07-02 2015-07-02 08:35:02 2015-07-02 2015-07-02 08:41:20 0 1 -73.96299743652344 40.75933074951172 -73.95545196533203 40.76837158203125 1 0.8 6 0 0.5 1 0 0 0.3 7.8 CSH 0 ) @ yellow -101 108 1 Manhattan 1010800 I MN19 Turtle Bay-East Midtown 3808 160 124 1 Manhattan 1012400 I MN31 Lenox Hill-Roosevelt Island 3805 +1200019112 1 2015-07-02 2015-07-02 10:12:32 2015-07-02 2015-07-02 10:25:32 0 1 -73.96263885498047 40.79478454589844 -73.98505401611328 40.768646240234375 1 2.3 11 0 0.5 2.36 0 0 0.3 14.16 CSH 0 Q B yellow -45 189 1 Manhattan 1018900 E MN12 Upper West Side 3806 66 145 1 Manhattan 1014500 I MN14 Lincoln Square 3806 +1200019113 2 2015-07-02 2015-07-02 10:21:42 2015-07-02 2015-07-02 10:25:49 0 1 -73.99498748779297 40.76026153564453 -73.9892578125 40.768070220947266 1 0.69 5 0 0.5 0 0 0 0.3 5.8 CRE 0 H H yellow -117 121 1 Manhattan 1012100 E MN15 Clinton 3807 202 139 1 Manhattan 1013900 I MN15 Clinton 3807 +1200019126 1 2015-07-02 2015-07-02 11:21:27 2015-07-02 2015-07-02 11:39:03 0 1 -73.8729476928711 40.774051666259766 -73.95443725585938 40.779998779296875 1 8.2 24 0 0.5 5 5.54 0 0.3 35.34 CSH 0 8 6 yellow 48 331 4 Queens 4033100 I QN98 Airport 4102 29 148.02 1 Manhattan 1014802 I MN40 Upper East Side-Carnegie Hill 3805 +1200019142 1 2015-07-02 2015-07-02 12:20:07 2015-07-02 2015-07-02 12:25:02 0 1 -73.96717071533203 40.78779983520508 -73.96018981933594 40.779991149902344 1 0.9 5.5 0 0.5 0 0 0 0.3 6.3 CRE 0 C 6 yellow 43 143 1 Manhattan 1014300 I MN99 park-cemetery-etc-Manhattan 3806 75 150.01 1 Manhattan 1015001 I MN40 Upper East Side-Carnegie Hill 3805 +1200019151 2 2015-07-02 2015-07-02 12:49:26 2015-07-02 2015-07-02 13:08:02 0 1 -73.97540283203125 40.75239944458008 -73.98258972167969 40.76826095581055 1 1.71 12.5 0 0.5 0 0 0 0.3 13.3 CRE 0 b B yellow -30 92 1 Manhattan 1009200 I MN19 Turtle Bay-East Midtown 3808 66 145 1 Manhattan 1014500 I MN14 Lincoln Square 3806 +1200019220 1 2015-07-02 2015-07-02 16:13:21 2015-07-02 2015-07-02 16:16:57 0 1 -73.95344543457031 40.76762390136719 -73.95146179199219 40.773948669433594 2 0.5 4.5 1 0.5 0 0 0 0.3 6.3 CRE 0 @ b yellow -96 124 1 Manhattan 1012400 I MN31 Lenox Hill-Roosevelt Island 3805 255 136 1 Manhattan 1013600 I MN32 Yorkville 3805 +1200019256 1 2015-07-02 2015-07-02 18:06:36 2015-07-02 2015-07-02 18:11:57 0 1 -73.97371673583984 40.76405334472656 -73.96581268310547 40.76272201538086 1 0.6 5.5 1 0.5 1.45 0 0 0.3 8.75 CSH 0 c 7 yellow 18 112.01 1 Manhattan 1011201 I MN17 Midtown-Midtown South 3807 22 114.02 1 Manhattan 1011402 I MN40 Upper East Side-Carnegie Hill 3805 +1200019368 2 2015-07-02 2015-07-02 22:49:54 2015-07-02 2015-07-02 23:04:36 0 1 -73.97615051269531 40.748870849609375 -73.95228576660156 40.7763786315918 1 2.8 12 0.5 0.5 0 0 0 0.3 13.3 CRE 0 p c yellow -123 80 1 Manhattan 1008000 I MN20 Murray Hill-Kips Bay 3808 0 138 1 Manhattan 1013800 I MN32 Yorkville 3805 +1200019404 1 2015-07-03 2015-07-03 00:56:36 2015-07-03 2015-07-03 01:02:57 0 1 -73.999267578125 40.72796630859375 -73.98654174804688 40.7221565246582 1 0.9 6 0.5 0.5 1.45 0 0 0.3 8.75 CSH 0  H yellow 37 65 1 Manhattan 1006500 I MN23 West Village 3810 161 30.01 1 Manhattan 1003001 E MN27 Chinatown 3809 +1200019447 2 2015-07-03 2015-07-03 05:01:05 2015-07-03 2015-07-03 05:20:31 0 1 -73.78177642822266 40.644744873046875 -73.94544219970703 40.63022994995117 5 14.39 39 0.5 0.5 0 0 0 0.3 40.3 CRE 0 2 � yellow 8 716 4 Queens 4071600 I QN98 Airport 4105 79 774 3 Brooklyn 3077400 I BK42 Flatbush 4015 +1200019463 2 2015-07-03 2015-07-03 08:12:21 2015-07-03 2015-07-03 08:28:07 0 1 -73.95025634765625 40.77980041503906 -73.9864273071289 40.754478454589844 1 4.01 14.5 0 0.5 3.06 0 0 0.3 18.36 CSH 0 c  yellow 31 154 1 Manhattan 1015400 I MN32 Yorkville 3805 21 113 1 Manhattan 1011300 I MN17 Midtown-Midtown South 3807 +1200019465 1 2015-07-03 2015-07-03 08:25:27 2015-07-03 2015-07-03 08:35:35 0 1 -73.98072052001953 40.73814010620117 -73.99241638183594 40.71406173706055 4 3.2 12 0 0.5 0 0 0 0.3 12.8 CSH 0  2 yellow -102 64 1 Manhattan 1006400 I MN21 Gramercy 3808 55 6 1 Manhattan 1000600 E MN28 Lower East Side 3809 +1200019475 1 2015-07-03 2015-07-03 09:48:49 2015-07-03 2015-07-03 09:53:19 0 1 -73.99024200439453 40.73503494262695 -74.0003662109375 40.73399353027344 1 0.8 5 0 0.5 0 0 0 0.3 5.8 CRE 0 4 I yellow -11 52 1 Manhattan 1005200 I MN13 Hudson Yards-Chelsea-Flatiron-Union Square 3807 162 71 1 Manhattan 1007100 I MN23 West Village 3810 +1200019528 1 2015-07-03 2015-07-03 14:12:38 2015-07-03 2015-07-03 14:24:38 0 1 -74.00205993652344 40.719505310058594 -73.9920425415039 40.71792984008789 3 1.4 9 0 0.5 0 0 0 0.3 9.8 CRE 0 1 H yellow 15 33 1 Manhattan 1003300 I MN24 SoHo-TriBeCa-Civic Center-Little Italy 3810 195 18 1 Manhattan 1001800 E MN27 Chinatown 3809 +1200019531 2 2015-07-03 2015-07-03 14:31:20 2015-07-03 2015-07-03 14:37:26 0 1 -73.87139129638672 40.774070739746094 -73.89405059814453 40.77037811279297 1 1.9 7.5 0 0.5 0 0 0 0.3 8.3 CRE 0 8 # yellow 48 331 4 Queens 4033100 I QN98 Airport 4102 60 317 4 Queens 4031700 I QN72 Steinway 4101 +1200019610 1 2015-07-03 2015-07-03 20:52:14 2015-07-03 2015-07-03 21:22:24 0 1 -74.01606750488281 40.71506881713867 -73.96094512939453 40.718936920166016 1 6.1 24 0.5 0.5 0 0 0 0.3 25.3 CRE 0  U yellow 52 317.03 1 Manhattan 1031703 I MN25 Battery Park City-Lower Manhattan 3810 191 555 3 Brooklyn 3055500 I BK73 North Side-South Side 4001 +1200019625 2 2015-07-03 2015-07-03 21:56:26 2015-07-03 2015-07-03 22:10:22 0 1 -73.99723815917969 40.72071075439453 -73.97364044189453 40.75564956665039 2 2.98 12.5 0.5 0.5 1 0 0 0.3 14.8 CSH 0 D b yellow -60 41 1 Manhattan 1004100 E MN24 SoHo-TriBeCa-Civic Center-Little Italy 3810 226 92 1 Manhattan 1009200 I MN19 Turtle Bay-East Midtown 3808 +1200019646 1 2015-07-03 2015-07-03 22:54:55 2015-07-03 2015-07-03 22:59:12 0 1 -74.00178527832031 40.74125671386719 -73.99092102050781 40.73495864868164 2 0.8 5 0.5 0.5 1.25 0 0 0.3 7.55 CSH 0 h 4 yellow 75 83 1 Manhattan 1008300 E MN13 Hudson Yards-Chelsea-Flatiron-Union Square 3807 245 52 1 Manhattan 1005200 I MN13 Hudson Yards-Chelsea-Flatiron-Union Square 3807 +1200019674 1 2015-07-04 2015-07-04 00:58:59 2015-07-04 2015-07-04 01:03:09 0 1 -73.9609146118164 40.80141830444336 -73.95250701904297 40.801727294921875 2 0.6 4.5 0.5 0.5 0 0 0 0.3 5.8 CRE 0 $ A yellow 110 193 1 Manhattan 1019300 E MN09 Morningside Heights 3802 40 218 1 Manhattan 1021800 E MN11 Central Harlem South 3803 +1200019702 1 2015-07-04 2015-07-04 03:04:48 2015-07-04 2015-07-04 03:08:02 0 1 -73.98236846923828 40.7713737487793 -73.97945404052734 40.76155090332031 1 0.9 5 0.5 0.5 0 0 0 0.3 6.3 CRE 0 B 0 yellow 67 149 1 Manhattan 1014900 I MN14 Lincoln Square 3806 3 131 1 Manhattan 1013100 I MN17 Midtown-Midtown South 3807 +1200019718 2 2015-07-04 2015-07-04 07:24:07 2015-07-04 2015-07-04 07:38:18 0 1 -73.98503112792969 40.746070861816406 -73.99854278564453 40.7134895324707 1 2.64 12.5 0 0.5 0 0 0 0.3 13.3 CRE 0 d E yellow -125 74 1 Manhattan 1007400 I MN17 Midtown-Midtown South 3807 156 29 1 Manhattan 1002900 E MN27 Chinatown 3809 +1200019733 2 2015-07-04 2015-07-04 09:44:38 2015-07-04 2015-07-04 09:59:18 0 1 -73.96162414550781 40.777015686035156 -73.87223052978516 40.77442169189453 3 8.63 24.5 0 0.5 6.17 5.54 0 0.3 37.01 CSH 0 6 8 yellow 25 142 1 Manhattan 1014200 I MN40 Upper East Side-Carnegie Hill 3805 48 331 4 Queens 4033100 I QN98 Airport 4102 +1200019742 1 2015-07-04 2015-07-04 10:18:26 2015-07-04 2015-07-04 10:30:52 0 1 -73.7767105102539 40.64530563354492 -73.85044860839844 40.681556701660156 1 6.5 19.5 0 0.5 0 0 0 0.3 20.3 CRE 0 2 � yellow 8 716 4 Queens 4071600 I QN98 Airport 4105 40 54 4 Queens 4005400 I QN56 Ozone Park 4113 diff --git a/tests/it/insert_formatted.rs b/tests/it/insert_formatted.rs new file mode 100644 index 0000000..0bbaf2a --- /dev/null +++ b/tests/it/insert_formatted.rs @@ -0,0 +1,168 @@ +use crate::{SimpleRow, create_simple_table, fetch_rows}; +use bytes::Bytes; +use clickhouse::{Client, Compression}; +use clickhouse_macros::Row; +use serde::Deserialize; +use std::cmp; +use tokio::io::AsyncWriteExt; + +/// First 1000 records of the [NYC taxi dataset] in `TabSeparated` format. +/// +/// [NYC taxi dataset]: https://clickhouse.com/docs/getting-started/example-datasets/nyc-taxi#create-the-table-trips +const TAXI_DATA_TSV: &[u8] = include_bytes!("fixtures/nyc-taxi_trips_0_head_1000.tsv"); + +#[tokio::test] +async fn empty_insert() { + let table_name = "insert_empty"; + let query_id = uuid::Uuid::new_v4().to_string(); + + let client = prepare_database!(); + create_simple_table(&client, table_name).await; + + let insert = client + .insert_formatted_with(format!("INSERT INTO {table_name} FORMAT TabSeparated")) + .with_option("query_id", query_id); + insert.end().await.unwrap(); + + let rows = fetch_rows::(&client, table_name).await; + assert!(rows.is_empty()) +} + +#[tokio::test] +async fn insert() { + let client = prepare_database!() + // Separate test for compression + .with_compression(Compression::None); + + create_table(&client).await; + + let bytes = Bytes::copy_from_slice(TAXI_DATA_TSV); + + let mut insert = + client.insert_formatted_with("INSERT INTO nyc_taxi_trips_small FORMAT TabSeparated"); + + insert.send(bytes).await.unwrap(); + + insert.end().await.unwrap(); + + verify_insert(&client).await; +} + +#[tokio::test] +#[cfg(feature = "lz4")] +async fn insert_compressed() { + use clickhouse::insert_formatted::CompressedData; + + let client = prepare_database!() + // `test-util` turns compression off + .with_compression(Compression::Lz4); + + create_table(&client).await; + + let data = CompressedData::from_slice(TAXI_DATA_TSV); + + let mut insert = + client.insert_formatted_with("INSERT INTO nyc_taxi_trips_small FORMAT TabSeparated"); + + insert.send_compressed(data).await.unwrap(); + + insert.end().await.unwrap(); + + verify_insert(&client).await; +} + +#[tokio::test] +async fn insert_buffered() { + let client = prepare_database!(); + + create_table(&client).await; + + let mut data = TAXI_DATA_TSV; + let capacity = 8192; + + let mut insert = client + .insert_formatted_with("INSERT INTO nyc_taxi_trips_small FORMAT TabSeparated") + .buffered_with_capacity(capacity); + + // Cycle different read sizes in an attempt to break the buffer + let read_sizes = [1, 10, 100, 1000, 1024, capacity]; + + while !data.is_empty() { + for size in read_sizes { + if data.is_empty() { + break; + } + + let written = insert + .write(&data[..cmp::min(size, data.len())]) + .await + .unwrap(); + assert_ne!(written, 0); + data = &data[written..]; + } + } + + insert.end().await.unwrap(); + + verify_insert(&client).await; +} + +async fn create_table(client: &Client) { + client + .query( + r#" + CREATE TABLE IF NOT EXISTS nyc_taxi_trips_small ( + trip_id UInt32, + pickup_datetime DateTime, + dropoff_datetime DateTime, + pickup_longitude Nullable(Float64), + pickup_latitude Nullable(Float64), + dropoff_longitude Nullable(Float64), + dropoff_latitude Nullable(Float64), + passenger_count UInt8, + trip_distance Float32, + fare_amount Float32, + extra Float32, + tip_amount Float32, + tolls_amount Float32, + total_amount Float32, + payment_type Enum('CSH' = 1, 'CRE' = 2, 'NOC' = 3, 'DIS' = 4, 'UNK' = 5), + pickup_ntaname LowCardinality(String), + dropoff_ntaname LowCardinality(String) + ) + ENGINE = MergeTree + PRIMARY KEY (pickup_datetime, dropoff_datetime) + "#, + ) + .execute() + .await + .unwrap(); +} + +async fn verify_insert(client: &Client) { + #[derive(Row, Deserialize)] + struct Results { + min_trip_id: u32, + max_trip_id: u32, + avg_trip_distance: f64, + count: u64, + } + + let results = client + .query( + "SELECT \ + min(trip_id) min_trip_id, \ + max(trip_id) max_trip_id, \ + avg(trip_distance) avg_trip_distance,\ + count(*) count \ + FROM nyc_taxi_trips_small", + ) + .fetch_one::() + .await + .unwrap(); + + assert_eq!(results.min_trip_id, 1199999902); + assert_eq!(results.max_trip_id, 1200019742); + assert_eq!(results.avg_trip_distance, 2.983289997249842); + assert_eq!(results.count, 1000); +} diff --git a/tests/it/main.rs b/tests/it/main.rs index 81a8236..af648e5 100644 --- a/tests/it/main.rs +++ b/tests/it/main.rs @@ -254,6 +254,7 @@ mod cursor_stats; mod fetch_bytes; mod https_errors; mod insert; +mod insert_formatted; mod inserter; mod int128; mod int256;