ScaDaMaLe Course site and book

Introduction to Spark SQL

  • This notebook explains the motivation behind Spark SQL, one of Spark's main libraries built on top of Spark Core.
  • It introduces interactive SparkSQL queries and visualizations
  • This notebook uses content from Databricks SparkSQL notebook and SparkSQL programming guide
displayHTML(frameIt("https://en.wikipedia.org/wiki/Apache_Hive#HiveQL",175))

This is an elaboration of the Apache Spark latest sql-progamming-guide.

Overview

Spark SQL is a Spark module for structured data processing. Unlike the basic Spark RDD API, the interfaces provided by Spark SQL provide Spark with more information about the structure of both the data and the computation being performed. Internally, Spark SQL uses this extra information to perform extra optimizations. There are several ways to interact with Spark SQL including SQL and the Dataset API. When computing a result the same execution engine is used, independent of which API/language you are using to express the computation. This unification means that developers can easily switch back and forth between different APIs based on which provides the most natural way to express a given transformation.

All of the examples on this page use sample data included in the Spark distribution and can be run in the spark-shell, pyspark shell, or sparkR shell.

Datasets and DataFrames

A Dataset is a distributed collection of data. Dataset is a new interface added in Spark 1.6 that provides the benefits of RDDs (strong typing, ability to use powerful lambda functions) with the benefits of Spark SQL’s optimized execution engine. A Dataset can be constructed from JVM objects and then manipulated using functional transformations (map, flatMap, filter, etc.). The Dataset API is available in Scala and Java. Python does not have the support for the Dataset API. But due to Python’s dynamic nature, many of the benefits of the Dataset API are already available (i.e. you can access the field of a row by name naturally row.columnName). The case for R is similar.

A DataFrame is a Dataset organized into named columns. It is conceptually equivalent to a table in a relational database or a data frame in R/Python, but with richer optimizations under the hood. DataFrames can be constructed from a wide array of sources such as: structured data files, tables in Hive, external databases, or existing RDDs. The DataFrame API is available in Scala, Java, Python, and R. In Scala and Java, a DataFrame is represented by a Dataset of Rows. In the Scala API, DataFrame is simply a type alias of Dataset[Row]. While, in Java API, users need to use Dataset<Row> to represent a DataFrame.

Throughout this document, we will often refer to Scala/Java Datasets of Rows as DataFrames.

Getting Started in Spark

Starting Point: SparkSession

The entry point into all functionality in Spark is the SparkSession. To create a basic SparkSession in your scala Spark code, just use SparkSession.builder():

import org.apache.spark.sql.SparkSession

val spark = SparkSession
  .builder()
  .appName("Spark SQL basic example")
  .config("spark.some.config.option", "some-value")
  .getOrCreate()

// For implicit conversions like converting RDDs to DataFrames
import spark.implicits._

Conveniently, in Databricks notebook (similar to spark-shell) SparkSession is already created for you and is available as spark.

spark // ready-made Spark-Session
res2: org.apache.spark.sql.SparkSession = org.apache.spark.sql.SparkSession@cc46118

Creating DataFrames

With a SparkSession or SQLContext, applications can create DataFrame

  • from an existing RDD,
  • from a Hive table, or
  • from various other data sources.

Just to recap:

  • A DataFrame is a distributed collection of data organized into named columns (it is not strogly typed).
  • You can think of it as being organized into table RDD of case class Row (which is not exactly true).
  • DataFrames, in comparison to RDDs, are backed by rich optimizations, including:
    • tracking their own schema,
    • adaptive query execution,
    • code generation including whole stage codegen,
    • extensible Catalyst optimizer, and
    • project Tungsten for optimized storage.

Note that performance for DataFrames is the same across languages Scala, Java, Python, and R. This is due to the fact that the only planning phase is language-specific (logical + physical SQL plan), not the actual execution of the SQL plan.

1. Making an empty DataFrame

Spark has some of the pre-built methods to create simple DataFrames

  • let us make an Empty DataFrame
val emptyDF = spark.emptyDataFrame // Ctrl+Enter to make an empty DataFrame
emptyDF: org.apache.spark.sql.DataFrame = []

Not really interesting, or is it?

You Try!

Uncomment the following cell, put your cursor after emptyDF. below and hit Tab to see what can be done with emptyDF.

//emptyDF.

2. Making a DataFrame from a range

Let us make a DataFrame next

  • from a range of numbers, as follows:
val rangeDF = spark.range(0, 3).toDF() // Ctrl+Enter to make DataFrame with 0,1,2
// sc.parallelize(1 to 3).toDF() 
rangeDF: org.apache.spark.sql.DataFrame = [id: bigint]

Note that Spark automatically names column as id and casts integers to type bigint for big integer or Long.

In order to get a preview of data in DataFrame use show() as follows:

rangeDF.show() // Ctrl+Enter
+---+
| id|
+---+
|  0|
|  1|
|  2|
+---+

3. Making a DataFrame from an RDD

  • Make an RDD
  • Conver the RDD into a DataFrame using the defualt .toDF() method
  • Conver the RDD into a DataFrame using the non-default .toDF(...) method
  • Do it all in one line

Let's first make an RDD using the sc.parallelize method, transform it by a map and perform the collect action to display it, as follows:

val rdd1 = sc.parallelize(1 to 5).map(i => (i, i*2))
rdd1.collect() // Ctrl+Enter
rdd1: org.apache.spark.rdd.RDD[(Int, Int)] = MapPartitionsRDD[171] at map at command-2971213210277924:1
res7: Array[(Int, Int)] = Array((1,2), (2,4), (3,6), (4,8), (5,10))

Next, let us convert the RDD into DataFrame using the .toDF() method, as follows:

val df1 = rdd1.toDF() // Ctrl+Enter 
df1: org.apache.spark.sql.DataFrame = [_1: int, _2: int]

As it is clear, the DataFrame has columns named _1 and _2, each of type int. Let us see its content using the .show() method next.

df1.show() // Ctrl+Enter
+---+---+
| _1| _2|
+---+---+
|  1|  2|
|  2|  4|
|  3|  6|
|  4|  8|
|  5| 10|
+---+---+

Note that by default, i.e. without specifying any options as in toDF(), the column names are given by _1 and _2.

We can easily specify column names as follows:

val df1 = rdd1.toDF("once", "twice") // Ctrl+Enter
df1.show()
+----+-----+
|once|twice|
+----+-----+
|   1|    2|
|   2|    4|
|   3|    6|
|   4|    8|
|   5|   10|
+----+-----+

df1: org.apache.spark.sql.DataFrame = [once: int, twice: int]

Of course, we can do all of the above steps to make the DataFrame df1 in one line and then show it, as follows:

val df1 = sc.parallelize(1 to 5)
            .map(i => (i, i*2))
            .toDF("once", "twice") //Ctrl+enter
df1.show()
+----+-----+
|once|twice|
+----+-----+
|   1|    2|
|   2|    4|
|   3|    6|
|   4|    8|
|   5|   10|
+----+-----+

df1: org.apache.spark.sql.DataFrame = [once: int, twice: int]

4. DataFrame Operations (aka Untyped Dataset Operations)

DataFrames provide a domain-specific language for structured data manipulation in Scala, Java, Python and R.

As mentioned above, in Spark 2.0, DataFrames are just Dataset of Rows in Scala and Java API. These operations are also referred as “untyped transformations” in contrast to “typed transformations” come with strongly typed Scala/Java Datasets.

Here we include some basic examples of structured data processing using Datasets:

// This import is needed to use the $-notation
import spark.implicits._
// Print the schema in a tree format
df1.printSchema()
root
 |-- once: integer (nullable = false)
 |-- twice: integer (nullable = false)

import spark.implicits._
// Select only the "name" column
df1.select("once").show()
+----+
|once|
+----+
|   1|
|   2|
|   3|
|   4|
|   5|
+----+
// Select both columns, but increment the double column by 1
df1.select($"once", $"once" + 1).show()
+----+----------+
|once|(once + 1)|
+----+----------+
|   1|         2|
|   2|         3|
|   3|         4|
|   4|         5|
|   5|         6|
+----+----------+
// Select both columns, but increment the double column by 1 and rename it as "oncemore"
df1.select($"once", ($"once" * 1).as("oncemore")).show()
+----+--------+
|once|oncemore|
+----+--------+
|   1|       1|
|   2|       2|
|   3|       3|
|   4|       4|
|   5|       5|
+----+--------+
df1.filter($"once" > 2).show()
+----+-----+
|once|twice|
+----+-----+
|   3|    6|
|   4|    8|
|   5|   10|
+----+-----+
// Count the number of distinct singles -  a bit boring
df1.groupBy("once").count().show()
+----+-----+
|once|count|
+----+-----+
|   1|    1|
|   2|    1|
|   3|    1|
|   5|    1|
|   4|    1|
+----+-----+

Let's make a more interesting DataFrame for groupBy with repeated elements so that the count will be more than 1.

df1.show()
+----+-----+
|once|twice|
+----+-----+
|   1|    2|
|   2|    4|
|   3|    6|
|   4|    8|
|   5|   10|
+----+-----+
val df11 = sc.parallelize(3 to 5).map(i => (i, i*2)).toDF("once", "twice") // just make a small one
df11.show()
+----+-----+
|once|twice|
+----+-----+
|   3|    6|
|   4|    8|
|   5|   10|
+----+-----+

df11: org.apache.spark.sql.DataFrame = [once: int, twice: int]
val df111 = df1.union(df11) // let's take the unionAll of df1 and df11 into df111
df111.show() // df111 is obtained by simply appending the rows of df11 to df1
+----+-----+
|once|twice|
+----+-----+
|   1|    2|
|   2|    4|
|   3|    6|
|   4|    8|
|   5|   10|
|   3|    6|
|   4|    8|
|   5|   10|
+----+-----+

df111: org.apache.spark.sql.Dataset[org.apache.spark.sql.Row] = [once: int, twice: int]
// Count the number of distinct singles -  a bit less boring
df111.groupBy("once").count().show()
+----+-----+
|once|count|
+----+-----+
|   1|    1|
|   2|    1|
|   3|    2|
|   5|    2|
|   4|    2|
+----+-----+

For a complete list of the types of operations that can be performed on a Dataset refer to the API Documentation.

In addition to simple column references and expressions, Datasets also have a rich library of functions including string manipulation, date arithmetic, common math operations and more. The complete list is available in the DataFrame Function Reference.

You Try!

Uncomment the two lines in the next cell, and then fill in the ??? below to get a DataFrame df2 whose first two columns are the same as df1 and whose third column named triple has values that are three times the values in the first column.

//val df2 = sc.parallelize(1 to 5).map(i => (i, i*2, i????)).toDF("single", "double", "triple") // Ctrl+enter after editing ???
//df2.show()

5. Running SQL Queries Programmatically

The sql function on a SparkSession enables applications to run SQL queries programmatically and returns the result as a DataFrame.

df1
res27: org.apache.spark.sql.DataFrame = [once: int, twice: int]
// Register the DataFrame as a SQL temporary view
df1.createOrReplaceTempView("sdtable")

val sqlDF = spark.sql("SELECT * FROM sdtable")
sqlDF.show()
+----+-----+
|once|twice|
+----+-----+
|   1|    2|
|   2|    4|
|   3|    6|
|   4|    8|
|   5|   10|
+----+-----+

sqlDF: org.apache.spark.sql.DataFrame = [once: int, twice: int]
spark.sql("SELECT * FROM SDTable WHERE once>2").show()
+----+-----+
|once|twice|
+----+-----+
|   3|    6|
|   4|    8|
|   5|   10|
+----+-----+
SELECT * FROM SDTable WHERE once>2
once twice
3.0 6.0
4.0 8.0
5.0 10.0

5. Using SQL for interactively querying a table is very powerful!

Note -- comments are how you add comments in SQL cells beginning with %sql.

  • You can run SQL select * statement to see all columns of the table, as follows:
    • This is equivalent to the above `display(diamondsDF)' with the DataFrame
-- Ctrl+Enter to select all columns of the table
select * from SDTable
once twice
1.0 2.0
2.0 4.0
3.0 6.0
4.0 8.0
5.0 10.0
-- Ctrl+Enter to select all columns of the table
-- note table names of registered tables are case-insensitive
select * from sdtable
once twice
1.0 2.0
2.0 4.0
3.0 6.0
4.0 8.0
5.0 10.0

Global Temporary View

Temporary views in Spark SQL are session-scoped and will disappear if the session that creates it terminates. If you want to have a temporary view that is shared among all sessions and keep alive until the Spark application terminates, you can create a global temporary view. Global temporary view is tied to a system preserved database global_temp, and we must use the qualified name to refer it, e.g. SELECT * FROM global_temp.view1. See http://spark.apache.org/docs/latest/sql-programming-guide.html#global-temporary-view for details.

  1. Creating Datasets

Datasets are similar to RDDs, however, instead of using Java serialization or Kryo they use a specialized Encoder to serialize the objects for processing or transmitting over the network. While both encoders and standard serialization are responsible for turning an object into bytes, encoders are code generated dynamically and use a format that allows Spark to perform many operations like filtering, sorting and hashing without deserializing the bytes back into an object.

val rangeDS = spark.range(0, 3) // Ctrl+Enter to make DataSet with 0,1,2; Note we added '.toDF()' to this to create a DataFrame
rangeDS: org.apache.spark.sql.Dataset[Long] = [id: bigint]
rangeDS.show() // the column name 'id' is made by default here
+---+
| id|
+---+
|  0|
|  1|
|  2|
+---+

We can have more complicated objects in a DataSet too.

// Note: Case classes in Scala 2.10 can support only up to 22 fields. To work around this limit,
// you can use custom classes that implement the Product interface
case class Person(name: String, age: Long)

// Encoders are created for case classes
val caseClassDS = Seq(Person("Andy", 32), Person("Erik",44), Person("Anna", 15)).toDS()
caseClassDS.show()
+----+---+
|name|age|
+----+---+
|Andy| 32|
|Erik| 44|
|Anna| 15|
+----+---+

defined class Person
caseClassDS: org.apache.spark.sql.Dataset[Person] = [name: string, age: bigint]
// Encoders for most common types are automatically provided by importing spark.implicits._
val primitiveDS = Seq(1, 2, 3).toDS()
primitiveDS.map(_ + 1).collect() // Returns: Array(2, 3, 4)
primitiveDS: org.apache.spark.sql.Dataset[Int] = [value: int]
res36: Array[Int] = Array(2, 3, 4)
df1
res38: org.apache.spark.sql.DataFrame = [once: int, twice: int]
df1.show
+----+-----+
|once|twice|
+----+-----+
|   1|    2|
|   2|    4|
|   3|    6|
|   4|    8|
|   5|   10|
+----+-----+
// let's make a case class for our DF so we can convert it to Dataset
case class singleAndDoubleIntegers(once: Integer, twice: Integer)
defined class singleAndDoubleIntegers
val ds1 = df1.as[singleAndDoubleIntegers]
ds1: org.apache.spark.sql.Dataset[singleAndDoubleIntegers] = [once: int, twice: int]
ds1.show()
+----+-----+
|once|twice|
+----+-----+
|   1|    2|
|   2|    4|
|   3|    6|
|   4|    8|
|   5|   10|
+----+-----+

This week's recommended homework is a deep dive into the SparkSQL programming guide.

ScaDaMaLe Course site and book

This and the next sequence of notebooks are an elaboration of the http://spark.apache.org/docs/latest/sql-programming-guide.html by Ivan Sadikov and Raazesh Sainudiin.

Spark Sql Programming Guide

  • Overview
    • SQL
    • DataFrames
    • Datasets
  • Getting Started
    • Starting Point: SQLContext
    • Creating DataFrames
    • DataFrame Operations
    • Running SQL Queries Programmatically
    • Creating Datasets
    • Interoperating with RDDs
      • Inferring the Schema Using Reflection
      • Programmatically Specifying the Schema
  • Data Sources
    • Generic Load/Save Functions
      • Manually Specifying Options
      • Run SQL on files directly
      • Save Modes
      • Saving to Persistent Tables
    • Parquet Files
      • Loading Data Programmatically
      • Partition Discovery
      • Schema Merging
      • Hive metastore Parquet table conversion
        • Hive/Parquet Schema Reconciliation
        • Metadata Refreshing
      • Configuration
    • JSON Datasets
    • Hive Tables
      • Interacting with Different Versions of Hive Metastore
    • JDBC To Other Databases
    • Troubleshooting
  • Performance Tuning
    • Caching Data In Memory
    • Other Configuration Options
  • Distributed SQL Engine
    • Running the Thrift JDBC/ODBC server
    • Running the Spark SQL CLI
  • SQL Reference

What could one do with these notebooks?

One could read the Spark SQL Programming Guide that is embedded below and also linked above while going through the cells and doing the YouTrys in the following notebooks.

Why might one do it?

This homework/self-study will help you solve the assigned lab and theory exercises in the sequel, much faster by introducing you to some basic knowledge you need about Spark SQL.

NOTE on intra-iframe html navigation within a notebook:

  • When navigating in the html-page embedded as an iframe, as in the cell below, you can:
    • click on a link in the displayed html page to see the content of the clicked link and
    • then right-click on the page and click on the arrow keys <- and -> to go back or forward.

Let's go through the programming guide in databricks now

This is an elaboration of the http://spark.apache.org/docs/latest/sql-programming-guide.html by Ivan Sadikov and Raazesh Sainudiin.

Spark SQL, DataFrames and Datasets Guide

Spark SQL is a Spark module for structured data processing. Unlike the basic Spark RDD API, the interfaces provided by Spark SQL provide Spark with more information about the structure of both the data and the computation being performed. Internally, Spark SQL uses this extra information to perform extra optimizations. There are several ways to interact with Spark SQL including SQL and the Dataset API. When computing a result, the same execution engine is used, independent of which API/language you are using to express the computation. This unification means that developers can easily switch back and forth between different APIs based on which provides the most natural way to express a given transformation.

All of the examples on this page use sample data included in the Spark distribution and can be run in the spark-shell, pyspark shell, or sparkR shell.

SQL

One use of Spark SQL is to execute SQL queries. Spark SQL can also be used to read data from an existing Hive installation. For more on how to configure this feature, please refer to the Hive Tables section. When running SQL from within another programming language the results will be returned as a Dataset/DataFrame. You can also interact with the SQL interface using the command-line or over JDBC/ODBC.

Datasets and DataFrames

A Dataset is a distributed collection of data. Dataset is a new interface added in Spark 1.6 that provides the benefits of RDDs (strong typing, ability to use powerful lambda functions) with the benefits of Spark SQL’s optimized execution engine. A Dataset can be constructed from JVM objects and then manipulated using functional transformations (map, flatMap, filter, etc.). The Dataset API is available in Scala and Java. Python does not have the support for the Dataset API. But due to Python’s dynamic nature, many of the benefits of the Dataset API are already available (i.e. you can access the field of a row by name naturally row.columnName). The case for R is similar.

A DataFrame is a Dataset organized into named columns. It is conceptually equivalent to a table in a relational database or a data frame in R/Python, but with richer optimizations under the hood. DataFrames can be constructed from a wide array of sources such as: structured data files, tables in Hive, external databases, or existing RDDs. The DataFrame API is available in Scala, Java, Python, and R. In Scala and Java, a DataFrame is represented by a Dataset of Rows. In the Scala API, DataFrame is simply a type alias of Dataset[Row]. While, in Java API, users need to use Dataset<Row> to represent a DataFrame.

Throughout this document, we will often refer to Scala/Java Datasets of Rows as DataFrames.

Background and Preparation

  • If you are unfamiliar with SQL please brush-up from the basic links below.
  • SQL allows one to systematically explore any structured data (i.e., tables) using queries. This is necessary part of the data science process.

One can use the SQL Reference at https://spark.apache.org/docs/latest/sql-ref.html to learn SQL quickly.

displayHTML(frameIt("https://en.wikipedia.org/wiki/SQL",500))
displayHTML(frameIt("https://en.wikipedia.org/wiki/Apache_Hive#HiveQL",175))
displayHTML(frameIt("https://spark.apache.org/docs/latest/sql-ref.html",700))

ScaDaMaLe Course site and book

This is an elaboration of the http://spark.apache.org/docs/latest/sql-programming-guide.html by Ivan Sadikov and Raazesh Sainudiin.

Getting Started

Spark Sql Programming Guide

  • Starting Point: SparkSession
  • Creating DataFrames
  • Untyped Dataset Operations (aka DataFrame Operations)
  • Running SQL Queries Programmatically
  • Global Temporary View
  • Creating Datasets
  • Interoperating with RDDs
    • Inferring the Schema Using Reflection
    • Programmatically Specifying the Schema
  • Scalar Functions
  • Aggregate Functions

Getting Started

Starting Point: SparkSession

The entry point into all functionality in Spark is the SparkSession class and/or SQLContext/HiveContext. SparkSession is created for you as spark when you start spark-shell on command-line REPL or through a notebook server (databricks, zeppelin, jupyter, etc.). You will need to create SparkSession usually when building an application for submission to a Spark cluster. To create a basic SparkSession, just use SparkSession.builder():

import org.apache.spark.sql.SparkSession

val spark = SparkSession
  .builder()
  .appName("Spark SQL basic example")
  .config("spark.some.config.option", "some-value")
  .getOrCreate()

// For implicit conversions like converting RDDs to DataFrames
import spark.implicits._

Find full example code in the Spark repo at:

SparkSession in Spark 2.0 provides builtin support for Hive features including the ability to write queries using HiveQL, access to Hive UDFs, and the ability to read data from Hive tables. To use these features, you do not need to have an existing Hive setup.

// You could get SparkContext and SQLContext from SparkSession
val sc = spark.sparkContext
val sqlContext = spark.sqlContext

But in Databricks notebook (similar to spark-shell) SparkSession is already created for you and is available as spark (similarly, sc and sqlContext are also available).

// Evaluation of the cell by Ctrl+Enter will print spark session available in notebook
spark
res0: org.apache.spark.sql.SparkSession = org.apache.spark.sql.SparkSession@7f57ef36

After evaluation you should see something like this, i.e., a reference to the SparkSession you just created:

res0: org.apache.spark.sql.SparkSession = org.apache.spark.sql.SparkSession@5a289bf5

Creating DataFrames

With a SparkSessions, applications can create Dataset or DataFrame from an existing RDD, from a Hive table, or from various datasources.

Just to recap, a DataFrame is a distributed collection of data organized into named columns. You can think of it as an organized into table RDD of case class Row (which is not exactly true). DataFrames, in comparison to RDDs, are backed by rich optimizations, including tracking their own schema, adaptive query execution, code generation including whole stage codegen, extensible Catalyst optimizer, and project Tungsten.

Dataset provides type-safety when working with SQL, since Row is mapped to a case class, so that each column can be referenced by property of that class.

Note that performance for Dataset/DataFrames is the same across languages Scala, Java, Python, and R. This is due to the fact that the planning phase is just language-specific, only logical plan is constructed in Python, and all the physical execution is compiled and executed as JVM bytecode.

As an example, the following creates a DataFrame based on the content of a JSON file:

val df = spark.read.json("examples/src/main/resources/people.json")

// Displays the content of the DataFrame to stdout
df.show()
// +----+-------+
// | age|   name|
// +----+-------+
// |null|Michael|
// |  30|   Andy|
// |  19| Justin|
// +----+-------+

Find full example code at - https://raw.githubusercontent.com/apache/spark/master/examples/src/main/scala/org/apache/spark/examples/sql/SparkSQLExample.scala in the Spark repo.

To be able to try this example in databricks we need to load the people.json file into dbfs. Let us do this programmatically next.

// you should not have to uncomment the block below as we already loaded data from 002_02_dbcCEdataLoader notebook
/*
// the following lines merely fetch the file from the URL and load it into the dbfs for us to try in databricks
// getLines from the file at the URL
val peopleJsonLinesFromURL = scala.io.Source.fromURL("https://raw.githubusercontent.com/apache/spark/master/examples/src/main/resources/people.json").getLines
// remove any pre-existing file at the dbfs location
dbutils.fs.rm("/datasets/sds/spark-examples/people.json",recurse=true)
// convert the lines fetched from the URL to a Seq, then make it a RDD of String and finally save it as textfile to dbfs
sc.parallelize(peopleJsonLinesFromURL.toSeq).saveAsTextFile("/datasets/sds/spark-examples/people.json")
*/
// read the text file we just saved or already loaded and see what it has
sc.textFile("/datasets/sds/spark-examples/people.json").collect.mkString("\n")
res1: String =
{"name":"Michael"}
{"name":"Andy", "age":30}
{"name":"Justin", "age":19}
val df = spark.read.json("/datasets/sds/spark-examples/people.json")
df: org.apache.spark.sql.DataFrame = [age: bigint, name: string]
// you can also read into df like this - makes the same dataframe as the above call
val df = spark.read.format("json").load("/datasets/sds/spark-examples/people.json")
df: org.apache.spark.sql.DataFrame = [age: bigint, name: string]
df.show()
+----+-------+
| age|   name|
+----+-------+
|  30|   Andy|
|  19| Justin|
|null|Michael|
+----+-------+

Untyped Dataset Operations (aka DataFrame Operations)

DataFrames provide a domain-specific language for structured data manipulation in Scala, Java, Python and R.

As mentioned above, in Spark 2.0 or higher, DataFrames are just Dataset of Rows in Scala and Java API. These operations are also referred as “untyped transformations” in contrast to “typed transformations” come with strongly typed Scala/Java Datasets.

Here we include some basic examples of structured data processing using Datasets:

// This import is needed to use the $-notation
import spark.implicits._
// Print the schema in a tree format
df.printSchema()
root
 |-- age: long (nullable = true)
 |-- name: string (nullable = true)

import spark.implicits._
// Select only the "name" column
df.select("name").show()
+-------+
|   name|
+-------+
|   Andy|
| Justin|
|Michael|
+-------+
// Select everybody, but increment the age by 1
df.select($"name", $"age" + 1).show()
+-------+---------+
|   name|(age + 1)|
+-------+---------+
|   Andy|       31|
| Justin|       20|
|Michael|     null|
+-------+---------+
// Select people older than 21
df.filter($"age" > 21).show()
+---+----+
|age|name|
+---+----+
| 30|Andy|
+---+----+
// Count people by age
df.groupBy("age").count().show()
+----+-----+
| age|count|
+----+-----+
|  19|    1|
|  30|    1|
|null|    1|
+----+-----+

Find full example code at - https://raw.githubusercontent.com/apache/spark/master/examples/src/main/scala/org/apache/spark/examples/sql/SparkSQLExample.scala in the Spark repo.

For a complete list of the types of operations that can be performed on a Dataset, refer to the API Documentation.

In addition to simple column references and expressions, Datasets also have a rich library of functions including string manipulation, date arithmetic, common math operations and more. The complete list is available in the DataFrame Function Reference.

Running SQL Queries Programmatically

The sql function on a SparkSession enables applications to run SQL queries programmatically and returns the result as a DataFrame.

// Register the DataFrame as a SQL temporary view
df.createOrReplaceTempView("people")
val sqlDF = spark.sql("SELECT * FROM people")
sqlDF.show()
+----+-------+
| age|   name|
+----+-------+
|  30|   Andy|
|  19| Justin|
|null|Michael|
+----+-------+

sqlDF: org.apache.spark.sql.DataFrame = [age: bigint, name: string]

Global Temporary View

Temporary views in Spark SQL are session-scoped and will disappear if the session that creates it terminates. If you want to have a temporary view that is shared among all sessions and keep alive until the Spark application terminates, you can create a global temporary view. Global temporary view is tied to a system preserved database global_temp, and we must use the qualified name to refer it, e.g. SELECT * FROM global_temp.view1.

// Register the DataFrame as a global temporary view
df.createGlobalTempView("people")
// Global temporary view is tied to a system preserved database `global_temp`
spark.sql("SELECT * FROM global_temp.people").show()
+----+-------+
| age|   name|
+----+-------+
|  30|   Andy|
|  19| Justin|
|null|Michael|
+----+-------+
// Global temporary view is cross-session
spark.newSession().sql("SELECT * FROM global_temp.people").show()
+----+-------+
| age|   name|
+----+-------+
|  30|   Andy|
|  19| Justin|
|null|Michael|
+----+-------+

Creating Datasets

See https://spark.apache.org/docs/latest/sql-getting-started.html#creating-datasets

Datasets are similar to RDDs, however, instead of using Java serialization or Kryo they use a specialized Encoder to serialize the objects for processing or transmitting over the network. While both encoders and standard serialization are responsible for turning an object into bytes, encoders are code generated dynamically and use a format that allows Spark to perform many operations like filtering, sorting and hashing without deserializing the bytes back into an object.

case class Person(name: String, age: Long)

// Encoders are created for case classes
val caseClassDS = Seq(Person("Andy", 32)).toDS()
caseClassDS.show()
+----+---+
|name|age|
+----+---+
|Andy| 32|
+----+---+

defined class Person
caseClassDS: org.apache.spark.sql.Dataset[Person] = [name: string, age: bigint]
// Encoders for most common types are automatically provided by importing spark.implicits._
val primitiveDS = Seq(1, 2, 3).toDS()
primitiveDS.map(_ + 1).collect() // Returns: Array(2, 3, 4)
primitiveDS: org.apache.spark.sql.Dataset[Int] = [value: int]
res18: Array[Int] = Array(2, 3, 4)
// DataFrames can be converted to a Dataset by providing a class. Mapping will be done by name
val path = "/datasets/sds/spark-examples/people.json"
val peopleDS = spark.read.json(path).as[Person]
peopleDS.show()
+----+-------+
| age|   name|
+----+-------+
|  30|   Andy|
|  19| Justin|
|null|Michael|
+----+-------+

path: String = /datasets/sds/spark-examples/people.json
peopleDS: org.apache.spark.sql.Dataset[Person] = [age: bigint, name: string]

Dataset is not available directly in PySpark or SparkR.

Interoperating with RDDs

Spark SQL supports two different methods for converting existing RDDs into Datasets. The first method uses reflection to infer the schema of an RDD that contains specific types of objects. This reflection-based approach leads to more concise code and works well when you already know the schema while writing your Spark application.

The second method for creating Datasets is through a programmatic interface that allows you to construct a schema and then apply it to an existing RDD. While this method is more verbose, it allows you to construct Datasets when the columns and their types are not known until runtime.

Inferring the Schema Using Reflection

The Scala interface for Spark SQL supports automatically converting an RDD containing case classes to a DataFrame. The case class defines the schema of the table. The names of the arguments to the case class are read using reflection and become the names of the columns. Case classes can also be nested or contain complex types such as Seqs or Arrays. This RDD can be implicitly converted to a DataFrame and then be registered as a table. Tables can be used in subsequent SQL statements.

sc.textFile("/datasets/sds/spark-examples/people.txt").collect.mkString("\n")
res22: String =
Michael, 29
Andy, 30
Justin, 19
// For implicit conversions from RDDs to DataFrames
import spark.implicits._

// make a case class
case class Person(name: String, age: Long)

// Create an RDD of Person objects from a text file, convert it to a Dataframe
val peopleDF = spark.sparkContext
  .textFile("/datasets/sds/spark-examples/people.txt")
  .map(_.split(","))
  .map(attributes => Person(attributes(0), attributes(1).trim.toLong))
  .toDF()
import spark.implicits._
defined class Person
peopleDF: org.apache.spark.sql.DataFrame = [name: string, age: bigint]
peopleDF.show
+-------+---+
|   name|age|
+-------+---+
|Michael| 29|
|   Andy| 30|
| Justin| 19|
+-------+---+
// Register the DataFrame as a temporary view
peopleDF.createOrReplaceTempView("people")
// SQL statements can be run by using the sql methods provided by Spark
val teenagersDF = spark.sql("SELECT name, age FROM people WHERE age BETWEEN 13 AND 19")
teenagersDF: org.apache.spark.sql.DataFrame = [name: string, age: bigint]
teenagersDF.show()
+------+---+
|  name|age|
+------+---+
|Justin| 19|
+------+---+
// The columns of a row in the result can be accessed by field index
teenagersDF.map(teenager => "Name: " + teenager(0)).show()
+------------+
|       value|
+------------+
|Name: Justin|
+------------+
// or by field name
teenagersDF.map(teenager => "Name: " + teenager.getAs[String]("name")).show()
+------------+
|       value|
+------------+
|Name: Justin|
+------------+
// advanced ...
// No pre-defined encoders for Dataset[Map[K,V]], define explicitly
//implicit val mapEncoder = org.apache.spark.sql.Encoders.kryo[Map[String, Any]]
// Primitive types and case classes can be also be defined as follows
// import more classes here...
//implicit val stringIntMapEncoder: Encoder[Map[String, Any]] = ExpressionEncoder()
// row.getValuesMap[T] retrieves multiple columns at once into a Map[String, T]
//teenagersDF.map(teenager => teenager.getValuesMap[Any](List("name", "age"))).collect()

Programmatically Specifying the Schema

When case classes cannot be defined ahead of time (for example, the structure of records is encoded in a string, or a text dataset will be parsed and fields will be projected differently for different users), a DataFrame can be created programmatically with three steps.

  1. Create an RDD of Rows from the original RDD;
  2. Create the schema represented by a StructType matching the structure of Rows in the RDD created in Step 1.
  3. Apply the schema to the RDD of Rows via createDataFrame method provided by SparkSession.

For example:

import org.apache.spark.sql.Row

import org.apache.spark.sql.types._

// Create an RDD
val peopleRDD = spark.sparkContext.textFile("/datasets/sds/spark-examples/people.txt")
import org.apache.spark.sql.Row
import org.apache.spark.sql.types._
peopleRDD: org.apache.spark.rdd.RDD[String] = /datasets/sds/spark-examples/people.txt MapPartitionsRDD[236] at textFile at command-2971213210278122:6
// The schema is encoded in a string
val schemaString = "name age"
schemaString: String = name age
// Generate the schema based on the string of schema
val fields = schemaString.split(" ")
  .map(fieldName => StructField(fieldName, StringType, nullable = true))
fields: Array[org.apache.spark.sql.types.StructField] = Array(StructField(name,StringType,true), StructField(age,StringType,true))
val schema = StructType(fields)
schema: org.apache.spark.sql.types.StructType = StructType(StructField(name,StringType,true),StructField(age,StringType,true))
// Convert records of the RDD (people) to Rows
val rowRDD = peopleRDD
  .map(_.split(","))
  .map(attributes => Row(attributes(0), attributes(1).trim))
rowRDD: org.apache.spark.rdd.RDD[org.apache.spark.sql.Row] = MapPartitionsRDD[238] at map at command-2971213210278126:4
// Apply the schema to the RDD
val peopleDF = spark.createDataFrame(rowRDD, schema)
peopleDF: org.apache.spark.sql.DataFrame = [name: string, age: string]
peopleDF.show
+-------+---+
|   name|age|
+-------+---+
|Michael| 29|
|   Andy| 30|
| Justin| 19|
+-------+---+
// Creates a temporary view using the DataFrame
peopleDF.createOrReplaceTempView("people")
// SQL can be run over a temporary view created using DataFrames
val results = spark.sql("SELECT name FROM people")
results: org.apache.spark.sql.DataFrame = [name: string]
results.show
+-------+
|   name|
+-------+
|Michael|
|   Andy|
| Justin|
+-------+
// The results of SQL queries are DataFrames and support all the normal RDD operations
// The columns of a row in the result can be accessed by field index or by field name
results.map(attributes => "Name: " + attributes(0)).show()
+-------------+
|        value|
+-------------+
|Name: Michael|
|   Name: Andy|
| Name: Justin|
+-------------+

Find full example code at - https://raw.githubusercontent.com/apache/spark/master/examples/src/main/scala/org/apache/spark/examples/sql/SparkSQLExample.scala in the Spark repo.

Scalar Functions

Scalar functions are functions that return a single value per row, as opposed to aggregation functions, which return a value for a group of rows. Spark SQL supports a variety of Built-in Scalar Functions. It also supports User Defined Scalar Functions.

Aggregate Functions

Aggregate functions are functions that return a single value on a group of rows. The Built-in Aggregation Functions provide common aggregations such as count(), countDistinct(), avg(), max(), min(), etc. Users are not limited to the predefined aggregate functions and can create their own. For more details about user defined aggregate functions, please refer to the documentation of User Defined Aggregate Functions.

ScaDaMaLe Course site and book

This is an elaboration of the http://spark.apache.org/docs/latest/sql-programming-guide.html by Ivan Sadikov and Raazesh Sainudiin.

Getting Started - Exercise

After having gone through the simple example dataset in the programming guide, let's try a slightly larger dataset next.

Let us first create a table of social media usage from NYC

See the Load Data section to create this social_media_usage table from raw data.

First let's make sure this table is available for us. If you don't see social_media_usage as a named table in the output of the next cell then we first need to ingest this dataset. Let's do it using the databricks' GUI for creating Data as done next.

// Let's find out what tables are already available for loading
spark.catalog.listTables.show(20,false) // only showing first 20 tables, if any
+----------------------------+--------+-----------+---------+-----------+
|name                        |database|description|tableType|isTemporary|
+----------------------------+--------+-----------+---------+-----------+
|all_prices                  |default |null       |MANAGED  |false      |
|bitcoin_normed_window       |default |null       |MANAGED  |false      |
|bitcoin_reversals_window    |default |null       |MANAGED  |false      |
|countrycodes                |default |null       |EXTERNAL |false      |
|gold_normed_window          |default |null       |MANAGED  |false      |
|gold_reversals_window       |default |null       |MANAGED  |false      |
|ltcar_locations_2_csv       |default |null       |MANAGED  |false      |
|magellan                    |default |null       |MANAGED  |false      |
|mobile_sample               |default |null       |EXTERNAL |false      |
|oil_normed_window           |default |null       |MANAGED  |false      |
|oil_reversals_window        |default |null       |MANAGED  |false      |
|oil_reversals_window2       |default |null       |MANAGED  |false      |
|over300all_2_txt            |default |null       |MANAGED  |false      |
|person                      |default |null       |MANAGED  |false      |
|personer                    |default |null       |MANAGED  |false      |
|persons                     |default |null       |MANAGED  |false      |
|simple_range                |default |null       |MANAGED  |false      |
|social_media_usage          |default |null       |MANAGED  |false      |
|social_media_usage_csv_gui  |default |null       |MANAGED  |false      |
|voronoi20191213uppsla1st_txt|default |null       |MANAGED  |false      |
+----------------------------+--------+-----------+---------+-----------+
only showing top 20 rows

NYC Social Media Usage Data

This dataset is from https://datahub.io/JohnSnowLabs/nyc-social-media-usage#readme

The Demographic Reports are produced by the Economic, Demographic and Statistical Research unit within the Countywide Service Integration and Planning Management (CSIPM) Division of the Fairfax County Department of Neighborhood and Community Services. Information produced by the Economic, Demographic and Statistical Research unit is used by every county department, board, authority and the Fairfax County Public Schools. In addition to the small area estimates and forecasts, state and federal data on Fairfax County are collected and summarized, and special studies and Quantitative research are conducted by the unit.

We could fetch this data, with slightly simplified column names, from the following URL:

  • http://lamastex.org/datasets/public/NYCUSA/social-media-usage.csv

Overview

Below we will show you how to create and query a table or DataFrame that you uploaded to DBFS. DBFS is a Databricks File System (their distributed file system) that allows you to store data for querying inside of Databricks.

In other setups, you can have the data in s3 (say in AWS) or in hdfs in your hadoop cluster, etc.

Alternatively, you can use curl or wget to download it to the local file system in /databricks/driver and then load it into dbfs, after this you can use read it via spark session into a dataframe and register it as a hive table.

You can also get the data directly from here (but in this case you need to change the column names in the databricks Data upload GUI or programmatically to follow this notebook):

  • http://datahub.io/JohnSnowLabs/nyc-social-media-usage

Load Data

How to load csv file from URL and make a table in databricks

Okay, so how did we actually make table social_media_usage? Databricks allows us to upload/link external data and make it available as registerd SQL table. It involves several steps:

  • load this social-media-usage.csv file from the following URL to databricks directly:
    • http://lamastex.org/datasets/public/NYCUSA/social-media-usage.csv
  • Then load it into dbfs using the GUI and/or manipulate as follows programmatically.
// File location from URL is loaded
val socialMediaUsageFromURL = scala.io.Source.fromURL("http://lamastex.org/datasets/public/NYCUSA/social-media-usage.csv").getLines
// First line is the header - let's save it
val header = socialMediaUsageFromURL.next.split(",")
// remove any pre-existing file at the dbfs location
dbutils.fs.rm("/datasets/sds/social_media_usage.csv",recurse=true)
// convert the lines fetched from the URL to a Seq, then make it a RDD of String and finally save it as textfile to dbfs
sc.parallelize(socialMediaUsageFromURL.toSeq).saveAsTextFile("/datasets/sds/social_media_usage.csv")
// read the text file we just saved or already loaded and see what it has
sc.textFile("/datasets/sds/social_media_usage.csv").take(10).mkString("\n")
socialMediaUsageFromURL: Iterator[String] = <iterator>
header: Array[String] = Array(agency, platform, url, date, visits)
res1: String =
OEM,SMS,,2012-02-17,61652
OEM,SMS,,2012-11-09,44547
EDC,Flickr,http://www.flickr.com/nycedc,2012-05-09,
NYCHA,Newsletter,,2012-05-09,
DHS,Twitter,www.twitter.com/nycdhs,2012-06-13,389
DHS,Twitter,www.twitter.com/nycdhs,2012-08-02,431
DOH,Android,Condom Finder,2011-08-08,5026
DOT,Android,You The Man,2011-08-08,
MOME,Android,MiNY Venor app,2011-08-08,313
DOT,Broadcastr,,2011-08-08,
header
res2: Array[String] = Array(agency, platform, url, date, visits)
val socialMediaDF = spark
                      .read
                      .options(Map("infer_schema" -> "true"))
                      .csv("/datasets/sds/social_media_usage.csv")
                      .toDF(header:_*)
socialMediaDF: org.apache.spark.sql.DataFrame = [agency: string, platform: string ... 3 more fields]
socialMediaDF.count
res4: Long = 5898
// Let's create a view or table

val temp_table_name = "social_media_usage_temp"

socialMediaDF.createOrReplaceTempView(temp_table_name)
temp_table_name: String = social_media_usage_temp
// Let's find out what tables are already available for loading
spark.catalog.listTables.where($"name" startsWith "soc" ).show(5,false)
+--------------------------+--------+-----------+---------+-----------+
|name                      |database|description|tableType|isTemporary|
+--------------------------+--------+-----------+---------+-----------+
|social_media_usage        |default |null       |MANAGED  |false      |
|social_media_usage_csv_gui|default |null       |MANAGED  |false      |
|social_media_usage_temp   |null    |null       |TEMPORARY|true       |
+--------------------------+--------+-----------+---------+-----------+

With this registered as a temporary view, social_media_usage_temp will only be available to this particular notebook.

If you'd like other users to be able to query this table (in the databricks professional shard - not the free community edition; or in a managed on-premise cluster), you can also create a table from the DataFrame.

Once saved, this table will persist across cluster restarts as well as allow various users across different notebooks to query this data. To do so, choose your table name and use saveAsTable as done in the next cell.

val permanent_table_name = "social_media_usage"
socialMediaDF.write.mode("overwrite").format("parquet").saveAsTable(permanent_table_name)
permanent_table_name: String = social_media_usage
// Let's find out what tables starting with "soc" in their name are already available for loading
spark.catalog.listTables.where($"name" startsWith "soc" ).show(5,false)
+--------------------------+--------+-----------+---------+-----------+
|name                      |database|description|tableType|isTemporary|
+--------------------------+--------+-----------+---------+-----------+
|social_media_usage        |default |null       |MANAGED  |false      |
|social_media_usage_csv_gui|default |null       |MANAGED  |false      |
|social_media_usage_temp   |null    |null       |TEMPORARY|true       |
+--------------------------+--------+-----------+---------+-----------+

It looks like the table social_media_usage is available as a permanent table (isTemporary set as false), if you have not uncommented the last line in the previous cell (otherwise it will be available from a parquet file as a permanent table - we will see more about parquet in the sequel).

Next let us do the following:

  • load this table as a DataFrame (yes, the dataframe already exists as socialMediaDF, but we want to make a new DataFrame directly from the table)
  • print its schema and
  • show the first 20 rows.
-- Ctrl+Enter to achieve the same result using standard SQL syntax!
SELECT * FROM social_media_usage
agency platform url date visits
MOAE YouTube http://www.youtube.com/YouCanTooNYC 2012-03-14 5
NYC Gov YouTube http://www.youtube.com/nycgov 2012-03-14 7
NYC Water Youtube http://www.youtube.com/nycwater 2012-03-14 null
Veteran's Affairs Youtube http://www.youtube.com/channel/UCi6IvOszIb3hHPMUsaNKyXA 2012-03-14 null
DRIS YouTube http://www.youtube.com/nycdeptofrecords 2012-03-14 null
TLC Facebook http://www.facebook.com/pages/NYC-Taxi-and-Limousine-Commission/101679939900978?v=wall 2012-03-14 585
nycgov Google+ https://plus.google.com/u/0/b/104030911277642419611/104030911277642419611/posts/p/pub 2012-03-14 null
TOTAL TOTAL TOTAL 2012-03-14 1688764
DOH Android Condom Finder 2012-04-09 null
DOT Android You The Man 2012-04-09 null
MOME Android MiNY Venor app 2012-04-09 343
DOT Broadcastr null 2012-04-09 null
DPR Broadcastr http://beta.broadcastr.com/Echo.html?audioId=670026-4001 2012-04-09 null
ENDHT Facebook http://www.facebook.com/pages/NYC-Lets-End-Human-Trafficking/125730490795659?sk=wall 2012-04-09 9
VAC Facebook https://www.facebook.com/pages/NYC-Voter-Assistance-Commission/110226709012110 2012-04-09 55
PlaNYC Facebook http://www.facebook.com/pages/New-York-NY/PlaNYC/160454173971169?ref=ts 2012-04-09 92
DFTA Facebook http://www.facebook.com/pages/NYC-Department-for-the-Aging/109028655823590 2012-04-09 178
DOT Broadcastr null 2012-10-24 null
energyNYC Facebook http://www.facebook.com/EnergyNYC?sk=wall 2012-04-09 181
MOIA Facebook http://www.facebook.com/ihwnyc 2012-04-09 171
City Store Facebook http://www.facebook.com/citystorenyc 2012-04-09 214
OCDV Facebook http://www.facebook.com/pages/NYC-Healthy-Relationship-Training-Academy/134637829901065 2012-04-09 313
HIA Facebook http://www.facebook.com/pages/New-York-City-Health-Insurance-Link/145920551598 2012-04-09 228
MOPD Facebook http://www.facebook.com/pages/New-York-City-Mayors-Office-for-People-with-Disabilities/145237285504681?sk=wall 2012-04-09 309
DOB: UrbanCanvas Facebook http://www.facebook.com/NYCurbancanvas 2012-04-09 247
DOT Facebook http://www.facebook.com/JanetteSadikKhan 2012-04-09 354
HRA Facebook http://www.facebook.com/#!/pages/New-York-NY/NYC-DADS/111504588886342 2012-04-09 276
MOPD Facebook http://www.facebook.com/profile.php?id=1570569347 2012-04-09 267
DFTA Facebook http://www.facebook.com/timebanksnyc 2012-04-09 291
DOB: Cool Roofs Facebook http://www.facebook.com/coolroofs?sk=wall 2012-04-09 323
NYC & Co Facebook http://www.facebook.com/nycgo.nl 2012-04-09 353
MOIA Facebook http://www.facebook.com/pages/NYC-Mayors-Office-of-Immigrant-Affairs/118622031512497 2012-04-09 362
CAU Facebook http://www.facebook.com/NYCMayorsCAU 2012-04-09 350
DOITT Facebook http://www.facebook.com/pages/New-York-NY/NYC-INFORMATION-TECHNOLOGY-TELECOMMUNICATIONS/104786059565184 2012-04-09 347
City Charter Facebook http://www.facebook.com/pages/New-York-NY/NYC-Charter-Revision-Commission/110528715643388 2012-04-09 287
Vets Facebook http://www.facebook.com/pages/NYC-Mayors-Office-of-Veterans-Affairs/128003537214726 2012-04-09 363
DHS Facebook http://www.facebook.com/pages/New-York-NY/HOPE-2011/157690657606772 2012-04-09 302
NYC & Co Facebook http://www.facebook.com/nycgo.de 2012-04-09 483
NYC & Co Facebook http://www.facebook.com/nycgo.fr 2012-04-09 602
SICB1 Facebook https://www.facebook.com/CB1SI 2012-04-09 356
ACS Facebook http://www.facebook.com/FamilyConnectionsNYC 2012-04-09 425
DCA Facebook http://www.facebook.com/NYCDCA 2012-04-09 458
NYC & Co Facebook http://www.facebook.com/nycgo.au 2012-04-09 471
NYC & Co Facebook http://www.facebook.com/nycgo.ca 2012-04-09 470
NYCHA Facebook http://www.facebook.com/NYCHA 2012-04-09 880
NYC & Co Facebook http://www.facebook.com/nycgo.uk 2012-04-09 977
NYC & Co Facebook http://www.facebook.com/nycgo.it 2012-04-09 1328
Culture Facebook https://www.facebook.com/piypnyc 2012-04-09 816
SBS Facebook http://www.facebook.com/NYCBusiness 2012-04-09 1051
FUND Facebook http://www.facebook.com/mayorsfundtoadvancenyc 2012-04-09 887
DOT Facebook http://www.facebook.com/YouTheManNYC 2012-04-09 1031
NYC & Co Facebook http://www.facebook.com/nycgo.es 2012-04-09 1955
HHC Facebook http://www.facebook.com/nychhc 2012-04-09 1115
MOME Facebook http://www.facebook.com/nycmedia.jobhunt 2012-04-09 1164
GreeNYC Facebook https://www.facebook.com/birdienyc 2012-04-09 1401
311 Facebook http://www.facebook.com/pages/New-York-City-311/84372567650 2012-04-09 1451
DOH Facebook http://www.facebook.com/NYCKnows 2012-04-09 1712
DOE Facebook http://www.facebook.com/nycgrads 2012-04-09 1830
DEP Facebook http://www.facebook.com/nycwater 2012-04-09 2423
MOME Facebook https://www.facebook.com/NYCMedia 2012-04-09 2862
EDC Facebook http://www.facebook.com/NYCEDC 2012-04-09 3180
SBS - Workforce1 Facebook http://www.facebook.com/nycworkforce1 2012-04-09 4570
DOT Facebook http://www.facebook.com/NYCDOT 2012-04-09 3696
EDC Facebook http://www.facebook.com/AppSciNYC 2012-04-09 3769
DYCD Facebook http://www.facebook.com/nycyouth 2012-04-09 4618
DOH Facebook http://www.facebook.com/EatingHealthyNYC 2012-04-09 24618
NYC & Co Facebook http://www.facebook.com/nycgo.br 2012-04-09 4985
DOE Facebook http://www.facebook.com/NYCTeachingFellows 2012-04-09 5277
NYCService Facebook http://www.facebook.com/nycservice 2012-04-09 5470
NYC Mayors Cup Facebook https://www.facebook.com/nycmayorscup 2012-04-09 9598
DOH Facebook http://www.facebook.com/nycquits 2012-04-09 8753
DOE Facebook http://www.facebook.com/pages/I-TEACH-NYC/11409913191 2012-04-09 7662
DOE Facebook http://www.facebook.com/fundforpublicschools 2012-04-09 8316
DPR Facebook http://www.facebook.com/nycparks 2012-04-09 13339
OEM Facebook http://www.facebook.com/nycemergencymanagement 2012-04-09 13966
DOE Facebook http://www.facebook.com/NYCschools 2012-04-09 20101
DOH Facebook http://www.facebook.com/NYCcondom 2012-04-09 18517
NYC & Co Facebook http://www.facebook.com/nycgo 2012-04-09 41163
FDNY Facebook http://www.facebook.com/FDNYhome 2012-04-09 78265
DSNY Flickr http://www.flickr.com/photos/86722064@N03/ 2012-04-09 null
CCRB Facebook https://www.facebook.com/home.php#!/pages/NYC-Civilian-Complaint-Review-Board/152765208087880 2012-04-09 14
Commission on Human Rights Facebook http://www.facebook.com/NYCCommissionOnHumanRights 2012-04-09 47
DOB Facebook http://www.facebook.com/NYCBuildings 2012-04-09 1038
DSNY Facebook http://www.facebook.com/pages/NYC-Recycle-More-Waste-Less/152173854860863 2012-04-09 99
HDP Facebook http://www.facebook.com/pages/NYC-HPD-POE/128962093860639 2012-04-09 187
HPD/Commission on Human Rights Facebook http://www.facebook.com/FairHousingNyc 2012-04-09 null
LPC Facebook http://www.facebook.com/pages/NYC-Landmarks-Preservation-Commission/133261836703216 2012-04-09 95
Materials for the Arts Facebook https://www.facebook.com/mftanyc 2012-04-09 2745
MOAE Facebook http://www.facebook.com/pages/You-Can-Too/203525729692056 2012-04-09 58
MOIA Facebook http://www.facebook.com/pages/WE-ARE-NEW-YORK/174438697072 2012-04-09 1881
MOME Facebook http://www.facebook.com/NYCMINY 2012-04-09 492
NYC Gov Facebook http://www.facebook.com/nycgov 2012-04-09 7391
NYCCFB Facebook http://www.facebook.com/nycvotes 2012-04-09 83
NYPD Facebook https://www.facebook.com/NYPD 2012-04-09 null
DRIS Facebook http://www.facebook.com/NycDeptOfRecords 2012-04-09 null
DRIS Facebook http://www.facebook.com/MayorEdKochNYCRecords 2012-04-09 null
DRIS Facebook http://www.facebook.com/MayorJohnLindsayNYCRecords 2012-04-09 null
DRIS Facebook http://www.facebook.com/MayorFiorelloLaGuardiaNYCRecords 2012-04-09 null
MOME Android MiNY Venor app 2012-11-02 343
LMEC Flickr http://www.facebook.com/pages/New-York-NY/Latin-Media-and-Entertainment-Week/122259604487271 2012-04-09 261
DEP Flickr http://www.flickr.com/photos/nycep 2012-04-09 null
DOB Flickr http://www.flickr.com/photos/nyc_buildings/ 2012-04-09 null
DOE Flickr http://www.flickr.com/photos/nycschools 2012-04-09 null
DOITT Flickr http://www.flickr.com/photos/nyc_doitt 2012-04-09 null
DOT Flickr http://www.flickr.com/photos/nycstreets 2012-04-09 null
DPR Flickr http://www.flickr.com/photos/nycparks/ 2012-04-09 null
DSNY Flickr http://flickr.com/nycrecyclemore 2012-04-09 null
EDC Flickr http://www.flickr.com/nycedc 2012-04-09 null
FDNY Flickr http://www.flickr.com/groups/fdny-ems 2012-04-09 null
HHC Flickr http://www.flickr.com/hhcnyc 2012-04-09 null
LMEC Flickr http://www.flickr.com/photos/nyclatinmedia/ 2012-04-09 null
LPC Flickr http://www.flickr.com/photos/nyclandmarks 2012-04-09 null
Materials for the Arts Flickr http://www.flickr.com/photos/materialsforthearts 2012-04-09 null
Mayor's Office Flickr http://www.flickr.com/photos/nycmayorsoffice/ 2012-04-09 null
NYC Digital Flickr http://www.flickr.com/photos/nycdigital/ 2012-04-09 null
NYCHA Flickr http://www.flickr.com/photos/nychapics 2012-04-09 null
PlaNYC Flickr http://www.flickr.com/photos/planyc/ 2012-04-09 null
Prob Flickr http://www.flickr.com/photos/nycprobation/ 2012-04-09 null
SnowUpdate Flickr http://www.flickr.com/groups/1604085@N23/ 2012-04-09 null
HRA Flickr http://www.flickr.com/people/nychra/ 2012-04-09 null
GreeNYC Foursquare http://foursquare.com/birdie_nyc 2012-04-09 79
DOH Foursquare https://foursquare.com/nychealthy 2012-04-09 84
DOT Foursquare http://foursquare.com/user/7474166 2012-04-09 4
DPR Foursquare https://foursquare.com/nycparks 2012-04-09 10619
EDC Foursquare https://foursquare.com/user/3045331 2012-04-09 19
FDNY Foursquare https://foursquare.com/fdny 2012-04-09 null
Materials for the Arts Foursquare https://foursquare.com/mftanyc 2012-04-09 11
NYC Gov Foursquare http://foursquare.com/nycgov 2012-04-09 16975
NYC Gov Foursquare (Badge Unlock) https://foursquare.com/nycgov 2012-04-09 10076
NYCHA Foursquare https://foursquare.com/nycha 2012-04-09 32
Mayor's Office Instagram http://web.stagram.com/n/nycmayorsoffice 2012-04-09 null
DOT Instagram http://web.stagram.com/n/nyc_dot 2012-04-09 null
DOH iPhone http://itunes.apple.com/us/app/abceats/id502867547?mt=8 2012-04-09 null
311 iPhone App http://itunes.apple.com/us/app/nyc-311/id324897619?mt=8 2012-04-09 16879
DOH iPhone app http://itunes.apple.com/app/nyc-condom-finder-by-nyc-health/id418902795 2012-04-09 8041
DSNY iPhone App http://itunes.apple.com/us/app/nycrecycles/id445457149?ls=1&mt=8 2012-04-09 171
DSNY iPhone App http://itunes.apple.com/us/app/stuff-ex/id445438603?ls=1&mt=8 2012-04-09 142
DOT iPhone app You The Man 2012-04-09 2635
Mayor's Office iPhone App http://itunes.apple.com/us/app/nyc-city-hall/id375398827?mt=8 2012-04-09 3186
DOITT Tumblr http://nycdoitt.tumblr.com/ 2012-04-09 null
MOME iPhone App http://itunes.apple.com/us/app/nyc-media-app/id433177943?mt=8 2012-04-09 944
MOME iPhone App http://itunes.apple.com/us/app/miny-discount-vendors/id372448233?mt=8 2012-04-09 350
DOE Linked-In http://www.linkedin.com/groups?gid=1545057&home= 2012-04-09 287
DOE Linked-In http://www.linkedin.com/company/nyc-teaching-fellows 2012-04-09 1913
DOE Linked-In http://www.linkedin.com/company/nyc-department-of-education 2012-04-09 18745
ACS Linked-In http://www.linkedin.com/companies/260392/City+of+New+York%2C+Administration+for+Children%27s+Services?trk=ncsrch_hits&goback=%2Efcs_GLHD_city+of+new+york_false_*2_*2_*2_*2_*2_*2_*2_*2_*2_*2_*2_*2 2012-04-09 null
All Linked-In http://www.linkedin.com/company/2904?trk=tyah 2012-04-09 null
Dept of Consumer Affairs Linked-In http://www.linkedin.com/company/831694?trk=tyah 2012-04-09 null
DOF Linked-In http://www.linkedin.com/company/298308?trk=tyah 2012-04-09 null
DOHMH Linked-In http://www.linkedin.com/company/245926?trk=tyah 2012-04-09 null
SBS Linked-In http://www.linkedin.com/company/workforce1 2012-04-09 322
SBS Linked-In http://www.linkedin.com/company/small-business-services 2012-04-09 137
DOE Newsletter null 2012-04-09 187180
DOT Newsletter null 2012-04-09 72614
NYC Digital: external newsletter Newsletter null 2012-04-09 null
NYC Gov Newsletter 0 2012-04-09 543533
NYCHA Newsletter null 2012-04-09 null
OEM Newsletter null 2012-04-09 null
null nyc.gov 2800000 2012-04-09 null
nycgov Pinterest http://pinterest.com/nycgov 2012-04-09 null
DOH Pinterest http://pinterest.com/nychealth 2012-04-09 null
FDNY Pinterest http://pinterest.com/fdnyhome/ 2012-04-09 null
DRIS Pinterest http://pinterest.com/recordsnyc 2012-04-09 null
DOE SMS 877877 2012-04-09 382
NYCHA SMS null 2012-04-09 null
OEM SMS null 2012-04-09 61652
NYC Digital Tumblr http://nycdigital.tumblr.com 2012-04-09 2279
Change By Us Tumblr http://nycchangebyus.tumblr.com/ 2012-04-09 null
DOB Tumblr http://nycbuildings.tumblr.com 2012-04-09 null
DOH Tumblr http://nychealth.tumblr.com/ 2012-04-09 null
DOH Tumblr http://mygooddognyc.tumblr.com/ 2012-04-09 null
DOT Tumblr http://thedailypothole.tumblr.com 2012-04-09 406
EDC Tumblr http://nycedc.tumblr.com/ 2012-04-09 819
HRA Tumblr http://nycdads.tumblr.com 2012-04-09 null
NYC & Co Tumblr nycgo.tumblr.com 2012-04-09 null
NYC Digital Tumblr madeinny.tumblr.com 2012-04-09 null
NYC Digital Tumblr http://nycopendata.tumblr.com 2012-04-09 323
NYC Gov Tumblr nycgov.tumblr.com 2012-04-09 2610
OEM Tumblr http://nyccert.tumblr.com/ 2012-04-09 null
OEM Tumblr http://nyccitizencorpscouncil.tumblr.com/ 2012-04-09 null
OMB Tumblr http://nycarra.tumblr.com 2012-04-09 null
TLC Tumblr http://nyctlc.tumblr.com/ 2012-04-09 null
OEM Tumblr http://oempublicprivate.tumblr.com/ 2012-04-09 null
Young Mens Initiative Tumblr http://nycyoungmen.tumblr.com 2012-04-09 null
FDNY Tumblr http://fdny.tumblr.com/ 2012-04-09 null
SBS Tumblr http://nycheighborhoods.tumblr.com 2012-04-09 null
NYCDCAS Twitter http://twitter.com/NYCDCAS 2012-04-09 17
SBS Twitter https://twitter.com/NYCWorkforce1 2012-04-09 356
City Store Twitter http://twitter.com/citystorenyc 2012-04-09 193
LPC Twitter https://twitter.com/NYCLPC 2012-04-09 448
DOE Twitter http://twitter.com/alumny 2012-04-09 153
OEM Twitter http://twitter.com/NYCOEM 2012-04-09 141
NYC & Co Twitter http://twitter.com/nycgoshop 2012-04-09 337
LMEC Twitter http://twitter.com/nyclmew 2012-04-09 338
DPR Twitter http://www.twitter.com/Pearl_Squirrel 2012-04-09 241
HRA Twitter http://twitter.com/nychra 2012-04-09 555
NYC Digital Twitter http://twitter.com/nycdigital 2012-04-09 1470
energyNYC Twitter http://twitter.com/energy_nyc 2012-04-09 509
DFTA Twitter http://twitter.com/NYCaging 2012-04-09 587
DOE Twitter http://twitter.com/TheFundforPS 2012-04-09 367
MOPD Twitter http://twitter.com/nyc_mopd 2012-04-09 609
NYCCFB Twitter http://twitter.com/NYCCFB 2012-04-09 616
EDC Twitter http://twitter.com/AppSciNYC 2012-04-09 1153
NYCHA Newsletter null 2012-07-05 44983
DOB: Cool Roofs Twitter http://twitter.com/nycCoolRoofs 2012-04-09 746
DOE Twitter http://twitter.com/nyctf 2012-04-09 694
DOT Twitter http://www.Twitter.com/YouTheManNYC 2012-04-09 956
City Charter Twitter http://twitter.com/CityCharterNYC 2012-04-09 715
DFTA Twitter http://www.twitter.com/timebanksnyc 2012-04-09 805
SBS Twitter http://www.twitter.com/NYCBusSolutions 2012-04-09 1292
MOME Twitter http://www.twitter.com/madeinny 2012-04-09 1731
GreeNYC Twitter http://www.twitter.com/Birdie_NYC 2012-04-09 1511
DOI Twitter http://twitter.com/DOINews 2012-04-09 1689
DCA Twitter http://twitter.com/nycdca 2012-04-09 1965
DOE Twitter http://twitter.com/iteachnyc 2012-04-09 1479
CAU Twitter www.twitter.com/mayorscau 2012-04-09 1821
HHC Twitter http://twitter.com/HHCnyc 2012-04-09 2447
MOIA Twitter https://twitter.com/NYCimmigrants 2012-04-09 1984
SBS Twitter http://www.twitter.com/NYCBusinessExp 2012-04-09 2656
DEP Twitter http://twitter.com/nycwater 2012-04-09 3046
DOITT Twitter http://twitter.com/nycdoitt 2012-04-09 3011
PlaNYC Twitter http://twitter.com/PlaNYC 2012-04-09 3190
DYCD Twitter http://twitter.com/nycyouth 2012-04-09 2784
NYCService Twitter http://twitter.com/nycservice 2012-04-09 3113
TLC Twitter http://twitter.com/NYCTaxi 2012-04-09 3168
DOB Twitter http://twitter.com/nyc_buildings 2012-04-09 3703
NYCHA Twitter http://twitter.com/NYCHA 2012-04-09 3880
MOME Twitter http://www.twitter.com/nyc_media 2012-04-09 7035
EDC Twitter http://twitter.com/nycedc 2012-04-09 7525
DOH Twitter http://twitter.com/nycHealthy 2012-04-09 8313
NYC Digital Twitter http://twitter.com/nycgov 2012-04-09 18652
311 Twitter http://www.twitter.com/NYCASP 2012-04-09 10391
DOT Twitter http://twitter.com/NYC_DOT 2012-04-09 14789
FDNY Twitter http://www.twitter.com/FDNY 2012-04-09 26002
DOE Twitter http://twitter.com/NYCSchools 2012-04-09 22240
DPR Twitter http://twitter.com/NYCParks 2012-04-09 23027
NYPD Twitter http://twitter.com/NYPDnews 2012-04-09 32239
311 Twitter http://www.twitter.com/311NYC 2012-04-09 26934
OEM Twitter http://twitter.com/NotifyNYC 2012-04-09 38391
NYC & Co Twitter http://twitter.com/nycgo 2012-04-09 53462
Mayor's Office Twitter http://www.twitter.com/nycmayorsoffice 2012-04-09 64299
Change by Us Twitter www.twitter.com/ChangebyUs_NYC 2012-04-09 1058
DHS Twitter www.twitter.com/nycdhs 2012-04-09 294
DOF Twitter https://twitter.com/nycfinance 2012-04-09 null
DOHMH Twitter https://twitter.com/DrFarleyDOHMH 2012-04-09 null
DSNY Twitter www.twitter.com/nycrecycles 2012-04-09 35
FDNY Twitter https://twitter.com/joinFDNY 2012-04-09 1413
Materials for the Arts Twitter https://twitter.com/mftanyc 2012-04-09 1878
MOAE Twitter http://twitter.com/youcantoonyc 2012-04-09 27
NYC Digital Twitter https://twitter.com/nycgob 2012-04-09 null
NYC Waterfront Twitter http://twitter.com/nycwaterfront 2012-04-09 99
NYCCFB Twitter http://twitter.com/NYCVotes 2012-04-09 160
NYCGLOBAL Twitter www.twitter.com/nycglobal 2012-04-09 147
Prob Twitter www.twitter.com/nycprobation 2012-04-09 121
Vets Twitter http://twitter.com/NYCVeterans 2012-04-09 257
DRIS Twitter https://twitter.com/NYCRecords 2012-04-09 null
YMI Twitter http://www.twitter.com/nycyoungmen 2012-04-09 null
DOE Vimeo http://vimeo.com/nycschools 2012-04-09 null
NYCSevereWeather WordPress http://nycsevereweather.wordpress.com/ 2012-04-09 null
311 WordPress http://311nyc.wordpress.com/ 2012-04-09 null
DOITT WordPress http://nycitymap.wordpress.com/ 2012-04-09 null
DOITT WordPress http://nycitt.wordpress.com/ 2012-04-09 null
HHS WordPress http://hsdatanyc.wordpress.com/ 2012-04-09 null
HHS WordPress http://hsdata-nyc.org 2012-04-09 null
LMEC WordPress http://lmew.wordpress.com/ 2012-04-09 null
Materials for the Arts WordPress http://mfta.wordpress.com/ 2012-04-09 null
MOIA WordPress http://ihwnyc.wordpress.com 2012-04-09 null
SBS WordPress http://nycworkforce1partners.wordpress.com/ 2012-04-09 null
SBS WordPress http://workforce1.org 2012-04-09 null
SimpliCity WordPress http://nycsimplicity.wordpress.com/ 2012-04-09 null
DOT iPhone app You The Man 2012-04-30 2598
DOE YouTube http://www.youtube.com/thefundforps 2012-04-09 1
DOH YouTube http://www.youtube.com/user/NYCcondoms 2012-04-09 null
LMEC YouTube http://www.youtube.com/user/NYCLMEW 2012-04-09 6
Probation YouTube http://www.youtube.com/NYCProbation 2012-04-09 8
DOITT YouTube http://www.youtube.com/doittnews 2012-04-09 29
GreeNYC YouTube http://www.youtube.com/BirdieNYCity 2012-04-09 15
HRA YouTube http://www.youtube.com/user/HRANYC 2012-04-09 25
NYCHA YouTube http://www.youtube.com/NYCHAHousing 2012-04-09 30
HHC YouTube http://www.youtube.com/HHCnyc 2012-04-09 36
DOE YouTube http://www.youtube.com/nycschools 2012-04-09 44
DYCD YouTube http://www.youtube.com/dycdnyc 2012-04-09 53
OEM YouTube http://www.youtube.com/nycoem 2012-04-09 67
DOB YouTube http://www.youtube.com/NYCBUILDINGS 2012-04-09 84
EDC YouTube http://www.youtube.com/NYCEDC 2012-04-09 117
MOME YouTube http://www.youtube.com/nycmedia25 2012-04-09 153
DPR YouTube http://www.youtube.com/user/NYCParksDepartment 2012-04-09 237
DOT YouTube http://www.youtube.com/NYCDOT 2012-04-09 328
Mayor's Office YouTube http://www.youtube.com/mayorbloomberg 2012-04-09 663
NYPD YouTube http://www.youtube.com/nypd 2012-04-09 3066
DOH YouTube http://www.youtube.com/NYCHealth 2012-04-09 125
FDNY YouTube http://www.youtube.com/user/yourFDNY 2012-04-09 1082
Child Services Youtube http://www.youtube.com/user/childservices 2012-04-09 null
DCA YouTube http://www.youtube.com/nycdca 2012-04-09 17
DOC YouTube http://www.youtube.com/user/OFFICIALNYCDOC 2012-04-09 null
DOH Youtube http://www.youtube.com/user/drinkingsugar 2012-04-09 176
DSNY YouTube http://www.youtube.com/nycrecyclemore 2012-04-09 4
Materials for the Arts YouTube http://www.youtube.com/user/MaterialsForTheArts 2012-04-09 1
Mayor's Fund Youtube http://www.youtube.com/mayorsfundnyc 2012-04-09 null
MOAE YouTube http://www.youtube.com/YouCanTooNYC 2012-04-09 6
NYC Gov YouTube http://www.youtube.com/nycgov 2012-04-09 6
NYC Water Youtube http://www.youtube.com/nycwater 2012-04-09 null
Veteran's Affairs Youtube http://www.youtube.com/channel/UCi6IvOszIb3hHPMUsaNKyXA 2012-04-09 null
DRIS YouTube http://www.youtube.com/nycdeptofrecords 2012-04-09 null
TLC Facebook http://www.facebook.com/pages/NYC-Taxi-and-Limousine-Commission/101679939900978?v=wall 2012-04-09 622
nycgov Google+ https://plus.google.com/u/0/b/104030911277642419611/104030911277642419611/posts/p/pub 2012-04-09 null
TOTAL TOTAL TOTAL 2012-04-09 1752711
DOH Android Condom Finder 2012-04-30 null
DOT Android You The Man 2012-04-30 38
MOME Android MiNY Venor app 2012-04-30 343
DOT Broadcastr null 2012-04-30 null
DPR Broadcastr http://beta.broadcastr.com/Echo.html?audioId=670026-4001 2012-04-30 null
ENDHT Facebook http://www.facebook.com/pages/NYC-Lets-End-Human-Trafficking/125730490795659?sk=wall 2012-04-30 11
VAC Facebook https://www.facebook.com/pages/NYC-Voter-Assistance-Commission/110226709012110 2012-04-30 59
PlaNYC Facebook http://www.facebook.com/pages/New-York-NY/PlaNYC/160454173971169?ref=ts 2012-04-30 107
DFTA Facebook http://www.facebook.com/pages/NYC-Department-for-the-Aging/109028655823590 2012-04-30 186
energyNYC Facebook http://www.facebook.com/EnergyNYC?sk=wall 2012-04-30 192
MOIA Facebook http://www.facebook.com/ihwnyc 2012-04-30 220
City Store Facebook http://www.facebook.com/citystorenyc 2012-04-30 222
OCDV Facebook http://www.facebook.com/pages/NYC-Healthy-Relationship-Training-Academy/134637829901065 2012-04-30 318
HIA Facebook http://www.facebook.com/pages/New-York-City-Health-Insurance-Link/145920551598 2012-04-30 241
MOPD Facebook http://www.facebook.com/pages/New-York-City-Mayors-Office-for-People-with-Disabilities/145237285504681?sk=wall 2012-04-30 319
DOB: UrbanCanvas Facebook http://www.facebook.com/NYCurbancanvas 2012-04-30 255
DOT Facebook http://www.facebook.com/JanetteSadikKhan 2012-04-30 360
HRA Facebook http://www.facebook.com/#!/pages/New-York-NY/NYC-DADS/111504588886342 2012-04-30 296
MOPD Facebook http://www.facebook.com/profile.php?id=1570569347 2012-04-30 268
DFTA Facebook http://www.facebook.com/timebanksnyc 2012-04-30 305
DOB: Cool Roofs Facebook http://www.facebook.com/coolroofs?sk=wall 2012-04-30 343
NYC & Co Facebook http://www.facebook.com/nycgo.nl 2012-04-30 356
MOIA Facebook http://www.facebook.com/pages/NYC-Mayors-Office-of-Immigrant-Affairs/118622031512497 2012-04-30 387
CAU Facebook http://www.facebook.com/NYCMayorsCAU 2012-04-30 363
DOITT Facebook http://www.facebook.com/pages/New-York-NY/NYC-INFORMATION-TECHNOLOGY-TELECOMMUNICATIONS/104786059565184 2012-04-30 363
City Charter Facebook http://www.facebook.com/pages/New-York-NY/NYC-Charter-Revision-Commission/110528715643388 2012-04-30 289
Vets Facebook http://www.facebook.com/pages/NYC-Mayors-Office-of-Veterans-Affairs/128003537214726 2012-04-30 372
DHS Facebook http://www.facebook.com/pages/New-York-NY/HOPE-2011/157690657606772 2012-04-30 303
NYC & Co Facebook http://www.facebook.com/nycgo.de 2012-04-30 505
NYC & Co Facebook http://www.facebook.com/nycgo.fr 2012-04-30 657
SICB1 Facebook https://www.facebook.com/CB1SI 2012-04-30 359
ACS Facebook http://www.facebook.com/FamilyConnectionsNYC 2012-04-30 434
DCA Facebook http://www.facebook.com/NYCDCA 2012-04-30 482
NYC & Co Facebook http://www.facebook.com/nycgo.au 2012-04-30 490
NYC & Co Facebook http://www.facebook.com/nycgo.ca 2012-04-30 478
NYCHA Facebook http://www.facebook.com/NYCHA 2012-04-30 922
NYC & Co Facebook http://www.facebook.com/nycgo.uk 2012-04-30 1018
NYC & Co Facebook http://www.facebook.com/nycgo.it 2012-04-30 1440
Culture Facebook https://www.facebook.com/piypnyc 2012-04-30 null
SBS Facebook http://www.facebook.com/NYCBusiness 2012-04-30 1084
FUND Facebook http://www.facebook.com/mayorsfundtoadvancenyc 2012-04-30 909
DOT Facebook http://www.facebook.com/YouTheManNYC 2012-04-30 1037
NYC & Co Facebook http://www.facebook.com/nycgo.es 2012-04-30 2073
HHC Facebook http://www.facebook.com/nychhc 2012-04-30 1133
MOME Facebook http://www.facebook.com/nycmedia.jobhunt 2012-04-30 1171
GreeNYC Facebook https://www.facebook.com/birdienyc 2012-04-30 1434
311 Facebook http://www.facebook.com/pages/New-York-City-311/84372567650 2012-04-30 1484
DOH Facebook http://www.facebook.com/NYCKnows 2012-04-30 1731
DOE Facebook http://www.facebook.com/nycgrads 2012-04-30 1831
DEP Facebook http://www.facebook.com/nycwater 2012-04-30 2481
MOME Facebook https://www.facebook.com/NYCMedia 2012-04-30 2925
EDC Facebook http://www.facebook.com/NYCEDC 2012-04-30 3251
SBS - Workforce1 Facebook http://www.facebook.com/nycworkforce1 2012-04-30 4738
DOT Facebook http://www.facebook.com/NYCDOT 2012-04-30 3758
EDC Facebook http://www.facebook.com/AppSciNYC 2012-04-30 3815
DYCD Facebook http://www.facebook.com/nycyouth 2012-04-30 4735
DOH Facebook http://www.facebook.com/EatingHealthyNYC 2012-04-30 33861
NYC & Co Facebook http://www.facebook.com/nycgo.br 2012-04-30 5109
DOE Facebook http://www.facebook.com/NYCTeachingFellows 2012-04-30 5354
NYCService Facebook http://www.facebook.com/nycservice 2012-04-30 5524
NYC Mayors Cup Facebook https://www.facebook.com/nycmayorscup 2012-04-30 10383
DOH Facebook http://www.facebook.com/nycquits 2012-04-30 8798
DOE Facebook http://www.facebook.com/pages/I-TEACH-NYC/11409913191 2012-04-30 7708
DOE Facebook http://www.facebook.com/fundforpublicschools 2012-04-30 8326
DPR Facebook http://www.facebook.com/nycparks 2012-04-30 13803
OEM Facebook http://www.facebook.com/nycemergencymanagement 2012-04-30 14073
DOE Facebook http://www.facebook.com/NYCschools 2012-04-30 20447
DOH Facebook http://www.facebook.com/NYCcondom 2012-04-30 18808
NYC & Co Facebook http://www.facebook.com/nycgo 2012-04-30 42590
FDNY Facebook http://www.facebook.com/FDNYhome 2012-04-30 81588
CCRB Facebook https://www.facebook.com/home.php#!/pages/NYC-Civilian-Complaint-Review-Board/152765208087880 2012-04-30 14
Commission on Human Rights Facebook http://www.facebook.com/NYCCommissionOnHumanRights 2012-04-30 58
DOB Facebook http://www.facebook.com/NYCBuildings 2012-04-30 1070
DSNY Facebook http://www.facebook.com/pages/NYC-Recycle-More-Waste-Less/152173854860863 2012-04-30 107
HDP Facebook http://www.facebook.com/pages/NYC-HPD-POE/128962093860639 2012-04-30 225
HPD/Commission on Human Rights Facebook http://www.facebook.com/FairHousingNyc 2012-04-30 20
LPC Facebook http://www.facebook.com/pages/NYC-Landmarks-Preservation-Commission/133261836703216 2012-04-30 111
Materials for the Arts Facebook https://www.facebook.com/mftanyc 2012-04-30 2829
MOAE Facebook http://www.facebook.com/pages/You-Can-Too/203525729692056 2012-04-30 68
MOIA Facebook http://www.facebook.com/pages/WE-ARE-NEW-YORK/174438697072 2012-04-30 1914
MOME Facebook http://www.facebook.com/NYCMINY 2012-04-30 539
NYC Gov Facebook http://www.facebook.com/nycgov 2012-04-30 11085
NYCCFB Facebook http://www.facebook.com/nycvotes 2012-04-30 94
NYPD Facebook https://www.facebook.com/NYPD 2012-04-30 null
DRIS Facebook http://www.facebook.com/NycDeptOfRecords 2012-04-30 null
DRIS Facebook http://www.facebook.com/MayorEdKochNYCRecords 2012-04-30 null
DRIS Facebook http://www.facebook.com/MayorJohnLindsayNYCRecords 2012-04-30 null
DRIS Facebook http://www.facebook.com/MayorFiorelloLaGuardiaNYCRecords 2012-04-30 null
LMEC Flickr http://www.facebook.com/pages/New-York-NY/Latin-Media-and-Entertainment-Week/122259604487271 2012-04-30 284
DEP Flickr http://www.flickr.com/photos/nycep 2012-04-30 null
DOB Flickr http://www.flickr.com/photos/nyc_buildings/ 2012-04-30 null
DOE Flickr http://www.flickr.com/photos/nycschools 2012-04-30 null
DOITT Flickr http://www.flickr.com/photos/nyc_doitt 2012-04-30 null
DOT Flickr http://www.flickr.com/photos/nycstreets 2012-04-30 null
DPR Flickr http://www.flickr.com/photos/nycparks/ 2012-04-30 null
DSNY Flickr http://flickr.com/nycrecyclemore 2012-04-30 null
EDC Flickr http://www.flickr.com/nycedc 2012-04-30 null
FDNY Flickr http://www.flickr.com/groups/fdny-ems 2012-04-30 null
HHC Flickr http://www.flickr.com/hhcnyc 2012-04-30 null
LMEC Flickr http://www.flickr.com/photos/nyclatinmedia/ 2012-04-30 null
LPC Flickr http://www.flickr.com/photos/nyclandmarks 2012-04-30 null
Materials for the Arts Flickr http://www.flickr.com/photos/materialsforthearts 2012-04-30 null
Mayor's Office Flickr http://www.flickr.com/photos/nycmayorsoffice/ 2012-04-30 null
NYC Digital Flickr http://www.flickr.com/photos/nycdigital/ 2012-04-30 null
NYCHA Flickr http://www.flickr.com/photos/nychapics 2012-04-30 null
PlaNYC Flickr http://www.flickr.com/photos/planyc/ 2012-04-30 null
Prob Flickr http://www.flickr.com/photos/nycprobation/ 2012-04-30 null
SnowUpdate Flickr http://www.flickr.com/groups/1604085@N23/ 2012-04-30 null
HRA Flickr http://www.flickr.com/people/nychra/ 2012-04-30 null
DSNY Flickr http://www.flickr.com/photos/86722064@N03/ 2012-04-30 null
GreeNYC Foursquare http://foursquare.com/birdie_nyc 2012-04-30 88
DOH Foursquare https://foursquare.com/nychealthy 2012-04-30 83
DOT Foursquare http://foursquare.com/user/7474166 2012-04-30 4
DPR Foursquare https://foursquare.com/nycparks 2012-04-30 10943
EDC Foursquare https://foursquare.com/user/3045331 2012-04-30 19
FDNY Foursquare https://foursquare.com/fdny 2012-04-30 115
Materials for the Arts Foursquare https://foursquare.com/mftanyc 2012-04-30 11
NYC Gov Foursquare http://foursquare.com/nycgov 2012-04-30 18211
NYC Gov Foursquare (Badge Unlock) https://foursquare.com/nycgov 2012-04-30 10076
NYCHA Foursquare https://foursquare.com/nycha 2012-04-30 40
Mayor's Office Instagram http://web.stagram.com/n/nycmayorsoffice 2012-04-30 3404
DOT Instagram http://web.stagram.com/n/nyc_dot 2012-04-30 null
DOH iPhone http://itunes.apple.com/us/app/abceats/id502867547?mt=8 2012-04-30 8203
311 iPhone App http://itunes.apple.com/us/app/nyc-311/id324897619?mt=8 2012-04-30 24806
DOH iPhone app http://itunes.apple.com/app/nyc-condom-finder-by-nyc-health/id418902795 2012-04-30 28000
DSNY iPhone App http://itunes.apple.com/us/app/nycrecycles/id445457149?ls=1&mt=8 2012-04-30 772
DSNY iPhone App http://itunes.apple.com/us/app/stuff-ex/id445438603?ls=1&mt=8 2012-04-30 709
Mayor's Office iPhone App http://itunes.apple.com/us/app/nyc-city-hall/id375398827?mt=8 2012-04-30 5383
MOME iPhone App http://itunes.apple.com/us/app/nyc-media-app/id433177943?mt=8 2012-04-30 2268
MOME iPhone App http://itunes.apple.com/us/app/miny-discount-vendors/id372448233?mt=8 2012-04-30 350
DOE Linked-In http://www.linkedin.com/groups?gid=1545057&home= 2012-04-30 284
DOE Linked-In http://www.linkedin.com/company/nyc-teaching-fellows 2012-04-30 1975
DOE Linked-In http://www.linkedin.com/company/nyc-department-of-education 2012-04-30 19418
ACS Linked-In http://www.linkedin.com/companies/260392/City+of+New+York%2C+Administration+for+Children%27s+Services?trk=ncsrch_hits&goback=%2Efcs_GLHD_city+of+new+york_false_*2_*2_*2_*2_*2_*2_*2_*2_*2_*2_*2_*2 2012-04-30 null
All Linked-In http://www.linkedin.com/company/2904?trk=tyah 2012-04-30 null
Dept of Consumer Affairs Linked-In http://www.linkedin.com/company/831694?trk=tyah 2012-04-30 null
DOF Linked-In http://www.linkedin.com/company/298308?trk=tyah 2012-04-30 null
DOHMH Linked-In http://www.linkedin.com/company/245926?trk=tyah 2012-04-30 null
SBS Linked-In http://www.linkedin.com/company/workforce1 2012-04-30 364
SBS Linked-In http://www.linkedin.com/company/small-business-services 2012-04-30 137
DOE Newsletter null 2012-04-30 187180
DOT Newsletter null 2012-04-30 72614
NYC Digital: external newsletter Newsletter null 2012-04-30 null
NYC Gov Newsletter 0 2012-04-30 543533
NYCHA Newsletter null 2012-04-30 null
OEM Newsletter null 2012-04-30 null
null nyc.gov 2800000 2012-04-30 null
nycgov Pinterest http://pinterest.com/nycgov 2012-04-30 null
DOH Pinterest http://pinterest.com/nychealth 2012-04-30 null
FDNY Pinterest http://pinterest.com/fdnyhome/ 2012-04-30 null
DRIS Pinterest http://pinterest.com/recordsnyc 2012-04-30 null
DOE SMS 877877 2012-04-30 382
NYCHA SMS null 2012-04-30 null
OEM SMS null 2012-04-30 61652
NYC Digital Tumblr http://nycdigital.tumblr.com 2012-04-30 3700
Change By Us Tumblr http://nycchangebyus.tumblr.com/ 2012-04-30 null
DOB Tumblr http://nycbuildings.tumblr.com 2012-04-30 null
DOH Tumblr http://nychealth.tumblr.com/ 2012-04-30 null
DOH Tumblr http://mygooddognyc.tumblr.com/ 2012-04-30 null
DOT Tumblr http://thedailypothole.tumblr.com 2012-04-30 406
EDC Tumblr http://nycedc.tumblr.com/ 2012-04-30 14607
HRA Tumblr http://nycdads.tumblr.com 2012-04-30 null
NYC & Co Tumblr nycgo.tumblr.com 2012-04-30 134
NYC Digital Tumblr madeinny.tumblr.com 2012-04-30 null
NYC Digital Tumblr http://nycopendata.tumblr.com 2012-04-30 324
NYC Gov Tumblr nycgov.tumblr.com 2012-04-30 4313
OEM Tumblr http://nyccert.tumblr.com/ 2012-04-30 null
OEM Tumblr http://nyccitizencorpscouncil.tumblr.com/ 2012-04-30 null
OMB Tumblr http://nycarra.tumblr.com 2012-04-30 1326
TLC Tumblr http://nyctlc.tumblr.com/ 2012-04-30 null
OEM Tumblr http://oempublicprivate.tumblr.com/ 2012-04-30 null
DOITT Tumblr http://nycdoitt.tumblr.com/ 2012-04-30 null
Young Mens Initiative Tumblr http://nycyoungmen.tumblr.com 2012-04-30 null
FDNY Tumblr http://fdny.tumblr.com/ 2012-04-30 null
SBS Tumblr http://nycheighborhoods.tumblr.com 2012-04-30 null
NYCDCAS Twitter http://twitter.com/NYCDCAS 2012-04-30 22
SBS Twitter https://twitter.com/NYCWorkforce1 2012-04-30 393
City Store Twitter http://twitter.com/citystorenyc 2012-04-30 222
LPC Twitter https://twitter.com/NYCLPC 2012-04-30 495
DOE Twitter http://twitter.com/alumny 2012-04-30 157
OEM Twitter http://twitter.com/NYCOEM 2012-04-30 142
NYC & Co Twitter http://twitter.com/nycgoshop 2012-04-30 346
LMEC Twitter http://twitter.com/nyclmew 2012-04-30 354
DPR Twitter http://www.twitter.com/Pearl_Squirrel 2012-04-30 253
HRA Twitter http://twitter.com/nychra 2012-04-30 585
NYC Digital Twitter http://twitter.com/nycdigital 2012-04-30 1618
energyNYC Twitter http://twitter.com/energy_nyc 2012-04-30 567
DFTA Twitter http://twitter.com/NYCaging 2012-04-30 628
DOE Twitter http://twitter.com/TheFundforPS 2012-04-30 375
MOPD Twitter http://twitter.com/nyc_mopd 2012-04-30 624
NYCCFB Twitter http://twitter.com/NYCCFB 2012-04-30 663
EDC Twitter http://twitter.com/AppSciNYC 2012-04-30 1225
DOB: Cool Roofs Twitter http://twitter.com/nycCoolRoofs 2012-04-30 803
DOE Twitter http://twitter.com/nyctf 2012-04-30 727
DOT Twitter http://www.Twitter.com/YouTheManNYC 2012-04-30 948
City Charter Twitter http://twitter.com/CityCharterNYC 2012-04-30 739
DFTA Twitter http://www.twitter.com/timebanksnyc 2012-04-30 820
SBS Twitter http://www.twitter.com/NYCBusSolutions 2012-04-30 1373
MOME Twitter http://www.twitter.com/madeinny 2012-04-30 1903
GreeNYC Twitter http://www.twitter.com/Birdie_NYC 2012-04-30 1566
DOI Twitter http://twitter.com/DOINews 2012-04-30 1759
DCA Twitter http://twitter.com/nycdca 2012-04-30 2051
DOE Twitter http://twitter.com/iteachnyc 2012-04-30 1495
CAU Twitter www.twitter.com/mayorscau 2012-04-30 1872
HHC Twitter http://twitter.com/HHCnyc 2012-04-30 2579
MOIA Twitter https://twitter.com/NYCimmigrants 2012-04-30 2060
SBS Twitter http://www.twitter.com/NYCBusinessExp 2012-04-30 2740
DEP Twitter http://twitter.com/nycwater 2012-04-30 3178
DOITT Twitter http://twitter.com/nycdoitt 2012-04-30 3170
PlaNYC Twitter http://twitter.com/PlaNYC 2012-04-30 3327
DYCD Twitter http://twitter.com/nycyouth 2012-04-30 2896
NYCService Twitter http://twitter.com/nycservice 2012-04-30 3243
TLC Twitter http://twitter.com/NYCTaxi 2012-04-30 3323
DOB Twitter http://twitter.com/nyc_buildings 2012-04-30 3864
NYCHA Twitter http://twitter.com/NYCHA 2012-04-30 3994
MOME Twitter http://www.twitter.com/nyc_media 2012-04-30 7299
EDC Twitter http://twitter.com/nycedc 2012-04-30 7767
DOH Twitter http://twitter.com/nycHealthy 2012-04-30 8661
NYC Digital Twitter http://twitter.com/nycgov 2012-04-30 21346
311 Twitter http://www.twitter.com/NYCASP 2012-04-30 10562
DOT Twitter http://twitter.com/NYC_DOT 2012-04-30 15170
FDNY Twitter http://www.twitter.com/FDNY 2012-04-30 27399
DOE Twitter http://twitter.com/NYCSchools 2012-04-30 22751
DPR Twitter http://twitter.com/NYCParks 2012-04-30 23721
NYPD Twitter http://twitter.com/NYPDnews 2012-04-30 33469
311 Twitter http://www.twitter.com/311NYC 2012-04-30 27616
OEM Twitter http://twitter.com/NotifyNYC 2012-04-30 38883
NYC & Co Twitter http://twitter.com/nycgo 2012-04-30 54671
Mayor's Office Twitter http://www.twitter.com/nycmayorsoffice 2012-04-30 65021
Change by Us Twitter www.twitter.com/ChangebyUs_NYC 2012-04-30 1128
DHS Twitter www.twitter.com/nycdhs 2012-04-30 315
DOF Twitter https://twitter.com/nycfinance 2012-04-30 null
DOHMH Twitter https://twitter.com/DrFarleyDOHMH 2012-04-30 147
DSNY Twitter www.twitter.com/nycrecycles 2012-04-30 36
FDNY Twitter https://twitter.com/joinFDNY 2012-04-30 1497
Materials for the Arts Twitter https://twitter.com/mftanyc 2012-04-30 1948
MOAE Twitter http://twitter.com/youcantoonyc 2012-04-30 39
NYC Digital Twitter https://twitter.com/nycgob 2012-04-30 null
NYC Waterfront Twitter http://twitter.com/nycwaterfront 2012-04-30 146
NYCCFB Twitter http://twitter.com/NYCVotes 2012-04-30 199
NYCGLOBAL Twitter www.twitter.com/nycglobal 2012-04-30 150
Prob Twitter www.twitter.com/nycprobation 2012-04-30 139
Vets Twitter http://twitter.com/NYCVeterans 2012-04-30 275
DRIS Twitter https://twitter.com/NYCRecords 2012-04-30 null
YMI Twitter http://www.twitter.com/nycyoungmen 2012-04-30 null
DOE Vimeo http://vimeo.com/nycschools 2012-04-30 null
NYCSevereWeather WordPress http://nycsevereweather.wordpress.com/ 2012-04-30 null
311 WordPress http://311nyc.wordpress.com/ 2012-04-30 null
DOITT WordPress http://nycitymap.wordpress.com/ 2012-04-30 null
DOITT WordPress http://nycitt.wordpress.com/ 2012-04-30 null
HHS WordPress http://hsdatanyc.wordpress.com/ 2012-04-30 null
HHS WordPress http://hsdata-nyc.org 2012-04-30 null
LMEC WordPress http://lmew.wordpress.com/ 2012-04-30 null
Materials for the Arts WordPress http://mfta.wordpress.com/ 2012-04-30 null
MOIA WordPress http://ihwnyc.wordpress.com 2012-04-30 null
SBS WordPress http://nycworkforce1partners.wordpress.com/ 2012-04-30 null
DOT Android You The Man 2012-07-05 102
SBS WordPress http://workforce1.org 2012-04-30 null
SimpliCity WordPress http://nycsimplicity.wordpress.com/ 2012-04-30 null
DOE YouTube http://www.youtube.com/thefundforps 2012-04-30 1
DOH YouTube http://www.youtube.com/user/NYCcondoms 2012-04-30 null
LMEC YouTube http://www.youtube.com/user/NYCLMEW 2012-04-30 6
Probation YouTube http://www.youtube.com/NYCProbation 2012-04-30 8
DOITT YouTube http://www.youtube.com/doittnews 2012-04-30 29
GreeNYC YouTube http://www.youtube.com/BirdieNYCity 2012-04-30 15
HRA YouTube http://www.youtube.com/user/HRANYC 2012-04-30 27
NYCHA YouTube http://www.youtube.com/NYCHAHousing 2012-04-30 30
HHC YouTube http://www.youtube.com/HHCnyc 2012-04-30 37
DOE YouTube http://www.youtube.com/nycschools 2012-04-30 44
DYCD YouTube http://www.youtube.com/dycdnyc 2012-04-30 52
OEM YouTube http://www.youtube.com/nycoem 2012-04-30 67
DOB YouTube http://www.youtube.com/NYCBUILDINGS 2012-04-30 85
EDC YouTube http://www.youtube.com/NYCEDC 2012-04-30 123
MOME YouTube http://www.youtube.com/nycmedia25 2012-04-30 159
DPR YouTube http://www.youtube.com/user/NYCParksDepartment 2012-04-30 240
DOT YouTube http://www.youtube.com/NYCDOT 2012-04-30 330
Mayor's Office YouTube http://www.youtube.com/mayorbloomberg 2012-04-30 670
NYPD YouTube http://www.youtube.com/nypd 2012-04-30 3090
DOH YouTube http://www.youtube.com/NYCHealth 2012-04-30 129
FDNY YouTube http://www.youtube.com/user/yourFDNY 2012-04-30 1159
Child Services Youtube http://www.youtube.com/user/childservices 2012-04-30 null
DCA YouTube http://www.youtube.com/nycdca 2012-04-30 17
DOC YouTube http://www.youtube.com/user/OFFICIALNYCDOC 2012-04-30 null
DOH Youtube http://www.youtube.com/user/drinkingsugar 2012-04-30 191
DSNY YouTube http://www.youtube.com/nycrecyclemore 2012-04-30 4
Materials for the Arts YouTube http://www.youtube.com/user/MaterialsForTheArts 2012-04-30 1
Mayor's Fund Youtube http://www.youtube.com/mayorsfundnyc 2012-04-30 null
MOAE YouTube http://www.youtube.com/YouCanTooNYC 2012-04-30 8
NYC Gov YouTube http://www.youtube.com/nycgov 2012-04-30 8
NYC Water Youtube http://www.youtube.com/nycwater 2012-04-30 null
Veteran's Affairs Youtube http://www.youtube.com/channel/UCi6IvOszIb3hHPMUsaNKyXA 2012-04-30 null
DRIS YouTube http://www.youtube.com/nycdeptofrecords 2012-04-30 null
TLC Facebook http://www.facebook.com/pages/NYC-Taxi-and-Limousine-Commission/101679939900978?v=wall 2012-04-30 652
nycgov Google+ https://plus.google.com/u/0/b/104030911277642419611/104030911277642419611/posts/p/pub 2012-04-30 null
TOTAL TOTAL TOTAL 2012-04-30 1853118
DOH Android Condom Finder 2012-05-09 null
DOT Android You The Man 2012-05-09 38
MOME Android MiNY Venor app 2012-05-09 343
DOT Broadcastr null 2012-05-09 null
MOME Android MiNY Venor app 2012-07-05 343
DPR Broadcastr http://beta.broadcastr.com/Echo.html?audioId=670026-4001 2012-05-09 null
NYC & Co Facebook http://www.facebook.com/nycgo.es 2012-05-09 2107
ENDHT Facebook http://www.facebook.com/pages/NYC-Lets-End-Human-Trafficking/125730490795659?sk=wall 2012-05-09 11
VAC Facebook https://www.facebook.com/pages/NYC-Voter-Assistance-Commission/110226709012110 2012-05-09 60
PlaNYC Facebook http://www.facebook.com/pages/New-York-NY/PlaNYC/160454173971169?ref=ts 2012-05-09 112
DFTA Facebook http://www.facebook.com/pages/NYC-Department-for-the-Aging/109028655823590 2012-05-09 187
energyNYC Facebook http://www.facebook.com/EnergyNYC?sk=wall 2012-05-09 197
MOIA Facebook http://www.facebook.com/ihwnyc 2012-05-09 225
OEM Newsletter null 2012-07-05 47473
City Store Facebook http://www.facebook.com/citystorenyc 2012-05-09 224
OCDV Facebook http://www.facebook.com/pages/NYC-Healthy-Relationship-Training-Academy/134637829901065 2012-05-09 320
HIA Facebook http://www.facebook.com/pages/New-York-City-Health-Insurance-Link/145920551598 2012-05-09 240
MOPD Facebook http://www.facebook.com/pages/New-York-City-Mayors-Office-for-People-with-Disabilities/145237285504681?sk=wall 2012-05-09 320
DOB: UrbanCanvas Facebook http://www.facebook.com/NYCurbancanvas 2012-05-09 257
DOT Facebook http://www.facebook.com/JanetteSadikKhan 2012-05-09 361
HRA Facebook http://www.facebook.com/#!/pages/New-York-NY/NYC-DADS/111504588886342 2012-05-09 304
MOPD Facebook http://www.facebook.com/profile.php?id=1570569347 2012-05-09 272
DFTA Facebook http://www.facebook.com/timebanksnyc 2012-05-09 305
DOB: Cool Roofs Facebook http://www.facebook.com/coolroofs?sk=wall 2012-05-09 350
NYC & Co Facebook http://www.facebook.com/nycgo.nl 2012-05-09 360
MOIA Facebook http://www.facebook.com/pages/NYC-Mayors-Office-of-Immigrant-Affairs/118622031512497 2012-05-09 396
CAU Facebook http://www.facebook.com/NYCMayorsCAU 2012-05-09 371
DOITT Facebook http://www.facebook.com/pages/New-York-NY/NYC-INFORMATION-TECHNOLOGY-TELECOMMUNICATIONS/104786059565184 2012-05-09 371
City Charter Facebook http://www.facebook.com/pages/New-York-NY/NYC-Charter-Revision-Commission/110528715643388 2012-05-09 289
Vets Facebook http://www.facebook.com/pages/NYC-Mayors-Office-of-Veterans-Affairs/128003537214726 2012-05-09 381
DHS Facebook http://www.facebook.com/pages/New-York-NY/HOPE-2011/157690657606772 2012-05-09 303
NYC & Co Facebook http://www.facebook.com/nycgo.de 2012-05-09 510
NYC & Co Facebook http://www.facebook.com/nycgo.fr 2012-05-09 675
SICB1 Facebook https://www.facebook.com/CB1SI 2012-05-09 360
ACS Facebook http://www.facebook.com/FamilyConnectionsNYC 2012-05-09 442
DCA Facebook http://www.facebook.com/NYCDCA 2012-05-09 491
NYC & Co Facebook http://www.facebook.com/nycgo.au 2012-05-09 496
NYC & Co Facebook http://www.facebook.com/nycgo.ca 2012-05-09 481
NYCHA Facebook http://www.facebook.com/NYCHA 2012-05-09 942
NYC & Co Facebook http://www.facebook.com/nycgo.uk 2012-05-09 1029
NYC & Co Facebook http://www.facebook.com/nycgo.it 2012-05-09 1475
Culture Facebook https://www.facebook.com/piypnyc 2012-05-09 964
SBS Facebook http://www.facebook.com/NYCBusiness 2012-05-09 1101
FUND Facebook http://www.facebook.com/mayorsfundtoadvancenyc 2012-05-09 917
DOT Facebook http://www.facebook.com/YouTheManNYC 2012-05-09 1051
HHC Facebook http://www.facebook.com/nychhc 2012-05-09 1148
MOME Facebook http://www.facebook.com/nycmedia.jobhunt 2012-05-09 1173
GreeNYC Facebook https://www.facebook.com/birdienyc 2012-05-09 1451
311 Facebook http://www.facebook.com/pages/New-York-City-311/84372567650 2012-05-09 1495
DOH Facebook http://www.facebook.com/NYCKnows 2012-05-09 1728
DOE Facebook http://www.facebook.com/nycgrads 2012-05-09 1828
null nyc.gov 2800000 2012-07-05 null
DEP Facebook http://www.facebook.com/nycwater 2012-05-09 2505
MOME Facebook https://www.facebook.com/NYCMedia 2012-05-09 2953
EDC Facebook http://www.facebook.com/NYCEDC 2012-05-09 3293
SBS - Workforce1 Facebook http://www.facebook.com/nycworkforce1 2012-05-09 4815
DOT Facebook http://www.facebook.com/NYCDOT 2012-05-09 3880
EDC Facebook http://www.facebook.com/AppSciNYC 2012-05-09 3829
DYCD Facebook http://www.facebook.com/nycyouth 2012-05-09 4758
DOH Facebook http://www.facebook.com/EatingHealthyNYC 2012-05-09 34087
NYC & Co Facebook http://www.facebook.com/nycgo.br 2012-05-09 5148
DOE Facebook http://www.facebook.com/NYCTeachingFellows 2012-05-09 5382
NYCService Facebook http://www.facebook.com/nycservice 2012-05-09 5541
NYC Mayors Cup Facebook https://www.facebook.com/nycmayorscup 2012-05-09 10750
DOH Facebook http://www.facebook.com/nycquits 2012-05-09 8793
DOE Facebook http://www.facebook.com/pages/I-TEACH-NYC/11409913191 2012-05-09 7726
DOE Facebook http://www.facebook.com/fundforpublicschools 2012-05-09 8327
DPR Facebook http://www.facebook.com/nycparks 2012-05-09 13992
OEM Facebook http://www.facebook.com/nycemergencymanagement 2012-05-09 14066
DOE Facebook http://www.facebook.com/NYCschools 2012-05-09 20583
DOH Facebook http://www.facebook.com/NYCcondom 2012-05-09 18804
NYC & Co Facebook http://www.facebook.com/nycgo 2012-05-09 42898
FDNY Facebook http://www.facebook.com/FDNYhome 2012-05-09 82259
CCRB Facebook https://www.facebook.com/home.php#!/pages/NYC-Civilian-Complaint-Review-Board/152765208087880 2012-05-09 15
Commission on Human Rights Facebook http://www.facebook.com/NYCCommissionOnHumanRights 2012-05-09 62
DOB Facebook http://www.facebook.com/NYCBuildings 2012-05-09 1068
DSNY Facebook http://www.facebook.com/pages/NYC-Recycle-More-Waste-Less/152173854860863 2012-05-09 null
HDP Facebook http://www.facebook.com/pages/NYC-HPD-POE/128962093860639 2012-05-09 241
HPD/Commission on Human Rights Facebook http://www.facebook.com/FairHousingNyc 2012-05-09 23
LPC Facebook http://www.facebook.com/pages/NYC-Landmarks-Preservation-Commission/133261836703216 2012-05-09 114
Materials for the Arts Facebook https://www.facebook.com/mftanyc 2012-05-09 2842
MOAE Facebook http://www.facebook.com/pages/You-Can-Too/203525729692056 2012-05-09 70
MOIA Facebook http://www.facebook.com/pages/WE-ARE-NEW-YORK/174438697072 2012-05-09 1924
MOME Facebook http://www.facebook.com/NYCMINY 2012-05-09 563
NYC Gov Facebook http://www.facebook.com/nycgov 2012-05-09 11452
NYCCFB Facebook http://www.facebook.com/nycvotes 2012-05-09 96
NYPD Facebook https://www.facebook.com/NYPD 2012-05-09 null
DRIS Facebook http://www.facebook.com/NycDeptOfRecords 2012-05-09 null
DRIS Facebook http://www.facebook.com/MayorEdKochNYCRecords 2012-05-09 null
DRIS Facebook http://www.facebook.com/MayorJohnLindsayNYCRecords 2012-05-09 null
DRIS Facebook http://www.facebook.com/MayorFiorelloLaGuardiaNYCRecords 2012-05-09 null
LMEC Flickr http://www.facebook.com/pages/New-York-NY/Latin-Media-and-Entertainment-Week/122259604487271 2012-05-09 287
DEP Flickr http://www.flickr.com/photos/nycep 2012-05-09 null
DOB Flickr http://www.flickr.com/photos/nyc_buildings/ 2012-05-09 null
DOE Flickr http://www.flickr.com/photos/nycschools 2012-05-09 null
DOITT Flickr http://www.flickr.com/photos/nyc_doitt 2012-05-09 null
DOT Flickr http://www.flickr.com/photos/nycstreets 2012-05-09 null
DPR Flickr http://www.flickr.com/photos/nycparks/ 2012-05-09 null
DSNY Flickr http://flickr.com/nycrecyclemore 2012-05-09 null
FDNY Flickr http://www.flickr.com/groups/fdny-ems 2012-05-09 null
HHC Flickr http://www.flickr.com/hhcnyc 2012-05-09 null
LMEC Flickr http://www.flickr.com/photos/nyclatinmedia/ 2012-05-09 null
LPC Flickr http://www.flickr.com/photos/nyclandmarks 2012-05-09 null
Materials for the Arts Flickr http://www.flickr.com/photos/materialsforthearts 2012-05-09 null
Mayor's Office Flickr http://www.flickr.com/photos/nycmayorsoffice/ 2012-05-09 null
NYC Digital Flickr http://www.flickr.com/photos/nycdigital/ 2012-05-09 null
NYCHA Flickr http://www.flickr.com/photos/nychapics 2012-05-09 null
PlaNYC Flickr http://www.flickr.com/photos/planyc/ 2012-05-09 null
Prob Flickr http://www.flickr.com/photos/nycprobation/ 2012-05-09 null
SnowUpdate Flickr http://www.flickr.com/groups/1604085@N23/ 2012-05-09 null
HRA Flickr http://www.flickr.com/people/nychra/ 2012-05-09 null
DSNY Flickr http://www.flickr.com/photos/86722064@N03/ 2012-05-09 null
GreeNYC Foursquare http://foursquare.com/birdie_nyc 2012-05-09 88
DOH Foursquare https://foursquare.com/nychealthy 2012-05-09 83
DOT Foursquare http://foursquare.com/user/7474166 2012-05-09 4
DPR Foursquare https://foursquare.com/nycparks 2012-05-09 11054
EDC Foursquare https://foursquare.com/user/3045331 2012-05-09 19
FDNY Foursquare https://foursquare.com/fdny 2012-05-09 136
Materials for the Arts Foursquare https://foursquare.com/mftanyc 2012-05-09 11
NYC Gov Foursquare http://foursquare.com/nycgov 2012-05-09 18556
NYC Gov Foursquare (Badge Unlock) https://foursquare.com/nycgov 2012-05-09 null
NYCHA Foursquare https://foursquare.com/nycha 2012-05-09 40
Mayor's Office Instagram http://web.stagram.com/n/nycmayorsoffice 2012-05-09 null
DOT Instagram http://web.stagram.com/n/nyc_dot 2012-05-09 null
DOH iPhone http://itunes.apple.com/us/app/abceats/id502867547?mt=8 2012-05-09 8203
311 iPhone App http://itunes.apple.com/us/app/nyc-311/id324897619?mt=8 2012-05-09 24806
DOH iPhone app http://itunes.apple.com/app/nyc-condom-finder-by-nyc-health/id418902795 2012-05-09 28000
DSNY iPhone App http://itunes.apple.com/us/app/nycrecycles/id445457149?ls=1&mt=8 2012-05-09 772
DSNY iPhone App http://itunes.apple.com/us/app/stuff-ex/id445438603?ls=1&mt=8 2012-05-09 709
DOT iPhone app You The Man 2012-05-09 2598
Mayor's Office iPhone App http://itunes.apple.com/us/app/nyc-city-hall/id375398827?mt=8 2012-05-09 5383
MOME iPhone App http://itunes.apple.com/us/app/nyc-media-app/id433177943?mt=8 2012-05-09 2268
MOME iPhone App http://itunes.apple.com/us/app/miny-discount-vendors/id372448233?mt=8 2012-05-09 350
DOE Linked-In http://www.linkedin.com/groups?gid=1545057&home= 2012-05-09 283
DOE Linked-In http://www.linkedin.com/company/nyc-teaching-fellows 2012-05-09 1999
DOE Linked-In http://www.linkedin.com/company/nyc-department-of-education 2012-05-09 19658
ACS Linked-In http://www.linkedin.com/companies/260392/City+of+New+York%2C+Administration+for+Children%27s+Services?trk=ncsrch_hits&goback=%2Efcs_GLHD_city+of+new+york_false_*2_*2_*2_*2_*2_*2_*2_*2_*2_*2_*2_*2 2012-05-09 917
All Linked-In http://www.linkedin.com/company/2904?trk=tyah 2012-05-09 4266
Dept of Consumer Affairs Linked-In http://www.linkedin.com/company/831694?trk=tyah 2012-05-09 87
DOF Linked-In http://www.linkedin.com/company/298308?trk=tyah 2012-05-09 144
DOHMH Linked-In http://www.linkedin.com/company/245926?trk=tyah 2012-05-09 1927
SBS Linked-In http://www.linkedin.com/company/workforce1 2012-05-09 390
SBS Linked-In http://www.linkedin.com/company/small-business-services 2012-05-09 137
DOE Newsletter null 2012-05-09 187180
DOT Newsletter null 2012-05-09 72614
NYC Digital: external newsletter Newsletter null 2012-05-09 174
NYC Gov Newsletter 0 2012-05-09 587845
OEM Newsletter null 2012-05-09 null
null nyc.gov 2800000 2012-05-09 null
nycgov Pinterest http://pinterest.com/nycgov 2012-05-09 null
DOH Pinterest http://pinterest.com/nychealth 2012-05-09 null
FDNY Pinterest http://pinterest.com/fdnyhome/ 2012-05-09 null
DRIS Pinterest http://pinterest.com/recordsnyc 2012-05-09 null
DOE SMS 877877 2012-05-09 null
NYCHA SMS null 2012-05-09 null
OEM SMS null 2012-05-09 null
NYC Digital Tumblr http://nycdigital.tumblr.com 2012-05-09 4237
Change By Us Tumblr http://nycchangebyus.tumblr.com/ 2012-05-09 null
DOB Tumblr http://nycbuildings.tumblr.com 2012-05-09 null
DOH Tumblr http://nychealth.tumblr.com/ 2012-05-09 null
DOH Tumblr http://mygooddognyc.tumblr.com/ 2012-05-09 null
DOT Tumblr http://thedailypothole.tumblr.com 2012-05-09 406
EDC Tumblr http://nycedc.tumblr.com/ 2012-05-09 14607
HRA Tumblr http://nycdads.tumblr.com 2012-05-09 null
NYC & Co Tumblr nycgo.tumblr.com 2012-05-09 134
NYC Digital Tumblr madeinny.tumblr.com 2012-05-09 null
NYC Digital Tumblr http://nycopendata.tumblr.com 2012-05-09 326
NYC Gov Tumblr nycgov.tumblr.com 2012-05-09 5507
OEM Tumblr http://nyccert.tumblr.com/ 2012-05-09 null
OEM Tumblr http://nyccitizencorpscouncil.tumblr.com/ 2012-05-09 null
OMB Tumblr http://nycarra.tumblr.com 2012-05-09 1715
TLC Tumblr http://nyctlc.tumblr.com/ 2012-05-09 null
OEM Tumblr http://oempublicprivate.tumblr.com/ 2012-05-09 null
DOITT Tumblr http://nycdoitt.tumblr.com/ 2012-05-09 null
Young Mens Initiative Tumblr http://nycyoungmen.tumblr.com 2012-05-09 null
FDNY Tumblr http://fdny.tumblr.com/ 2012-05-09 null
SBS Tumblr http://nycheighborhoods.tumblr.com 2012-05-09 null
NYCDCAS Twitter http://twitter.com/NYCDCAS 2012-05-09 4032
SBS Twitter https://twitter.com/NYCWorkforce1 2012-05-09 3364
City Store Twitter http://twitter.com/citystorenyc 2012-05-09 236
LPC Twitter https://twitter.com/NYCLPC 2012-05-09 65351
DOE Twitter http://twitter.com/alumny 2012-05-09 160
OEM Twitter http://twitter.com/NYCOEM 2012-05-09 39028
NYC & Co Twitter http://twitter.com/nycgoshop 2012-05-09 23
LMEC Twitter http://twitter.com/nyclmew 2012-05-09 512
DPR Twitter http://www.twitter.com/Pearl_Squirrel 2012-05-09 36
HRA Twitter http://twitter.com/nychra 2012-05-09 359
NYC Digital Twitter http://twitter.com/nycdigital 2012-05-09 39
energyNYC Twitter http://twitter.com/energy_nyc 2012-05-09 27974
DFTA Twitter http://twitter.com/NYCaging 2012-05-09 3928
DOE Twitter http://twitter.com/TheFundforPS 2012-05-09 379
MOPD Twitter http://twitter.com/nyc_mopd 2012-05-09 153
NYCCFB Twitter http://twitter.com/NYCCFB 2012-05-09 673
EDC Twitter http://twitter.com/AppSciNYC 2012-05-09 580
DOB: Cool Roofs Twitter http://twitter.com/nycCoolRoofs 2012-05-09 22909
DOE Twitter http://twitter.com/nyctf 2012-05-09 737
DOT Twitter http://www.Twitter.com/YouTheManNYC 2012-05-09 23969
City Charter Twitter http://twitter.com/CityCharterNYC 2012-05-09 751
DFTA Twitter http://www.twitter.com/timebanksnyc 2012-05-09 831
SBS Twitter http://www.twitter.com/NYCBusSolutions 2012-05-09 411
MOME Twitter http://www.twitter.com/madeinny 2012-05-09 639
GreeNYC Twitter http://www.twitter.com/Birdie_NYC 2012-05-09 2629
DOI Twitter http://twitter.com/DOINews 2012-05-09 3214
DCA Twitter http://twitter.com/nycdca 2012-05-09 2086
DOE Twitter http://twitter.com/iteachnyc 2012-05-09 1500
CAU Twitter www.twitter.com/mayorscau 2012-05-09 1889
HHC Twitter http://twitter.com/HHCnyc 2012-05-09 603
MOIA Twitter https://twitter.com/NYCimmigrants 2012-05-09 7404
SBS Twitter http://www.twitter.com/NYCBusinessExp 2012-05-09 1400
DEP Twitter http://twitter.com/nycwater 2012-05-09 3203
DOITT Twitter http://twitter.com/nycdoitt 2012-05-09 15375
PlaNYC Twitter http://twitter.com/PlaNYC 2012-05-09 2769
DYCD Twitter http://twitter.com/nycyouth 2012-05-09 7860
NYCService Twitter http://twitter.com/nycservice 2012-05-09 141
TLC Twitter http://twitter.com/NYCTaxi 2012-05-09 287
DOB Twitter http://twitter.com/nyc_buildings 2012-05-09 830
NYCHA Twitter http://twitter.com/NYCHA 2012-05-09 34071
MOME Twitter http://www.twitter.com/nyc_media 2012-05-09 1973
EDC Twitter http://twitter.com/nycedc 2012-05-09 1249
DOH Twitter http://twitter.com/nycHealthy 2012-05-09 1785
NYC Digital Twitter http://twitter.com/nycgov 2012-05-09 1677
311 Twitter http://www.twitter.com/NYCASP 2012-05-09 10622
DOT Twitter http://twitter.com/NYC_DOT 2012-05-09 944
FDNY Twitter http://www.twitter.com/FDNY 2012-05-09 1507
DOE Twitter http://twitter.com/NYCSchools 2012-05-09 8770
DPR Twitter http://twitter.com/NYCParks 2012-05-09 257
NYPD Twitter http://twitter.com/NYPDnews 2012-05-09 3279
311 Twitter http://www.twitter.com/311NYC 2012-05-09 27847
OEM Twitter http://twitter.com/NotifyNYC 2012-05-09 147
NYC & Co Twitter http://twitter.com/nycgo 2012-05-09 345
Mayor's Office Twitter http://www.twitter.com/nycmayorsoffice 2012-05-09 2079
Change by Us Twitter www.twitter.com/ChangebyUs_NYC 2012-05-09 1145
DHS Twitter www.twitter.com/nycdhs 2012-05-09 646
DOF Twitter https://twitter.com/nycfinance 2012-05-09 776
DOHMH Twitter https://twitter.com/DrFarleyDOHMH 2012-05-09 null
DSNY Twitter www.twitter.com/nycrecycles 2012-05-09 2935
FDNY Twitter https://twitter.com/joinFDNY 2012-05-09 1590
Materials for the Arts Twitter https://twitter.com/mftanyc 2012-05-09 154
MOAE Twitter http://twitter.com/youcantoonyc 2012-05-09 1966
NYC Digital Twitter https://twitter.com/nycgob 2012-05-09 null
NYC Waterfront Twitter http://twitter.com/nycwaterfront 2012-05-09 180
NYCCFB Twitter http://twitter.com/NYCVotes 2012-05-09 212
NYCGLOBAL Twitter www.twitter.com/nycglobal 2012-05-09 55138
Prob Twitter www.twitter.com/nycprobation 2012-05-09 3397
Vets Twitter http://twitter.com/NYCVeterans 2012-05-09 22193
DRIS Twitter https://twitter.com/NYCRecords 2012-05-09 null
YMI Twitter http://www.twitter.com/nycyoungmen 2012-05-09 null
DOE Vimeo http://vimeo.com/nycschools 2012-05-09 null
NYCSevereWeather WordPress http://nycsevereweather.wordpress.com/ 2012-05-09 null
311 WordPress http://311nyc.wordpress.com/ 2012-05-09 null
DOITT WordPress http://nycitymap.wordpress.com/ 2012-05-09 null
DOITT WordPress http://nycitt.wordpress.com/ 2012-05-09 null
HHS WordPress http://hsdatanyc.wordpress.com/ 2012-05-09 null
HHS WordPress http://hsdata-nyc.org 2012-05-09 null
LMEC WordPress http://lmew.wordpress.com/ 2012-05-09 null
Materials for the Arts WordPress http://mfta.wordpress.com/ 2012-05-09 null
MOIA WordPress http://ihwnyc.wordpress.com 2012-05-09 null
SBS WordPress http://nycworkforce1partners.wordpress.com/ 2012-05-09 null
SBS WordPress http://workforce1.org 2012-05-09 null
SimpliCity WordPress http://nycsimplicity.wordpress.com/ 2012-05-09 null
DOE YouTube http://www.youtube.com/thefundforps 2012-05-09 null
DOH YouTube http://www.youtube.com/user/NYCcondoms 2012-05-09 null
LMEC YouTube http://www.youtube.com/user/NYCLMEW 2012-05-09 null
Probation YouTube http://www.youtube.com/NYCProbation 2012-05-09 null
DOITT YouTube http://www.youtube.com/doittnews 2012-05-09 null
GreeNYC YouTube http://www.youtube.com/BirdieNYCity 2012-05-09 null
HRA YouTube http://www.youtube.com/user/HRANYC 2012-05-09 null
NYCHA YouTube http://www.youtube.com/NYCHAHousing 2012-05-09 null
HHC YouTube http://www.youtube.com/HHCnyc 2012-05-09 null
DOE YouTube http://www.youtube.com/nycschools 2012-05-09 null
DYCD YouTube http://www.youtube.com/dycdnyc 2012-05-09 null
OEM YouTube http://www.youtube.com/nycoem 2012-05-09 null
DOB YouTube http://www.youtube.com/NYCBUILDINGS 2012-05-09 null
EDC YouTube http://www.youtube.com/NYCEDC 2012-05-09 null
MOME YouTube http://www.youtube.com/nycmedia25 2012-05-09 null
DPR YouTube http://www.youtube.com/user/NYCParksDepartment 2012-05-09 null
DOT YouTube http://www.youtube.com/NYCDOT 2012-05-09 null
Mayor's Office YouTube http://www.youtube.com/mayorbloomberg 2012-05-09 null
NYPD YouTube http://www.youtube.com/nypd 2012-05-09 null
DOH YouTube http://www.youtube.com/NYCHealth 2012-05-09 null
FDNY YouTube http://www.youtube.com/user/yourFDNY 2012-05-09 null
Child Services Youtube http://www.youtube.com/user/childservices 2012-05-09 null
DCA YouTube http://www.youtube.com/nycdca 2012-05-09 null
DOC YouTube http://www.youtube.com/user/OFFICIALNYCDOC 2012-05-09 null
DOH Youtube http://www.youtube.com/user/drinkingsugar 2012-05-09 null
DSNY YouTube http://www.youtube.com/nycrecyclemore 2012-05-09 null
Materials for the Arts YouTube http://www.youtube.com/user/MaterialsForTheArts 2012-05-09 null
Mayor's Fund Youtube http://www.youtube.com/mayorsfundnyc 2012-05-09 null
MOAE YouTube http://www.youtube.com/YouCanTooNYC 2012-05-09 null
NYC Gov YouTube http://www.youtube.com/nycgov 2012-05-09 null
NYC Water Youtube http://www.youtube.com/nycwater 2012-05-09 null
Veteran's Affairs Youtube http://www.youtube.com/channel/UCi6IvOszIb3hHPMUsaNKyXA 2012-05-09 null
DRIS YouTube http://www.youtube.com/nycdeptofrecords 2012-05-09 null
TLC Facebook http://www.facebook.com/pages/NYC-Taxi-and-Limousine-Commission/101679939900978?v=wall 2012-05-09 662
nycgov Google+ https://plus.google.com/u/0/b/104030911277642419611/104030911277642419611/posts/p/pub 2012-05-09 null
TOTAL TOTAL TOTAL 2012-05-09 1835426
DOH Android Condom Finder 2012-06-13 null
DOT Android You The Man 2012-06-13 102
MOME Android MiNY Venor app 2012-06-13 null
DOT Broadcastr null 2012-06-13 null
DPR Broadcastr http://beta.broadcastr.com/Echo.html?audioId=670026-4001 2012-06-13 null
ENDHT Facebook http://www.facebook.com/pages/NYC-Lets-End-Human-Trafficking/125730490795659?sk=wall 2012-06-13 11
VAC Facebook https://www.facebook.com/pages/NYC-Voter-Assistance-Commission/110226709012110 2012-06-13 86
PlaNYC Facebook http://www.facebook.com/pages/New-York-NY/PlaNYC/160454173971169?ref=ts 2012-06-13 157
DFTA Facebook http://www.facebook.com/pages/NYC-Department-for-the-Aging/109028655823590 2012-06-13 200
energyNYC Facebook http://www.facebook.com/EnergyNYC?sk=wall 2012-06-13 221
MOIA Facebook http://www.facebook.com/ihwnyc 2012-06-13 287
City Store Facebook http://www.facebook.com/citystorenyc 2012-06-13 254
OCDV Facebook http://www.facebook.com/pages/NYC-Healthy-Relationship-Training-Academy/134637829901065 2012-06-13 329
HIA Facebook http://www.facebook.com/pages/New-York-City-Health-Insurance-Link/145920551598 2012-06-13 251
MOPD Facebook http://www.facebook.com/pages/New-York-City-Mayors-Office-for-People-with-Disabilities/145237285504681?sk=wall 2012-06-13 356
DOB: UrbanCanvas Facebook http://www.facebook.com/NYCurbancanvas 2012-06-13 272
DOT Facebook http://www.facebook.com/JanetteSadikKhan 2012-06-13 371
HRA Facebook http://www.facebook.com/#!/pages/New-York-NY/NYC-DADS/111504588886342 2012-06-13 338
MOPD Facebook http://www.facebook.com/profile.php?id=1570569347 2012-06-13 285
DFTA Facebook http://www.facebook.com/timebanksnyc 2012-06-13 318
DOB: Cool Roofs Facebook http://www.facebook.com/coolroofs?sk=wall 2012-06-13 418
NYC & Co Facebook http://www.facebook.com/nycgo.nl 2012-06-13 373
MOIA Facebook http://www.facebook.com/pages/NYC-Mayors-Office-of-Immigrant-Affairs/118622031512497 2012-06-13 420
CAU Facebook http://www.facebook.com/NYCMayorsCAU 2012-06-13 416
DOITT Facebook http://www.facebook.com/pages/New-York-NY/NYC-INFORMATION-TECHNOLOGY-TELECOMMUNICATIONS/104786059565184 2012-06-13 402
City Charter Facebook http://www.facebook.com/pages/New-York-NY/NYC-Charter-Revision-Commission/110528715643388 2012-06-13 289
Vets Facebook http://www.facebook.com/pages/NYC-Mayors-Office-of-Veterans-Affairs/128003537214726 2012-06-13 419
DHS Facebook http://www.facebook.com/pages/New-York-NY/HOPE-2011/157690657606772 2012-06-13 304
NYC & Co Facebook http://www.facebook.com/nycgo.de 2012-06-13 565
NYC & Co Facebook http://www.facebook.com/nycgo.fr 2012-06-13 770
SICB1 Facebook https://www.facebook.com/CB1SI 2012-06-13 365
ACS Facebook http://www.facebook.com/FamilyConnectionsNYC 2012-06-13 462
DCA Facebook http://www.facebook.com/NYCDCA 2012-06-13 566
NYC & Co Facebook http://www.facebook.com/nycgo.au 2012-06-13 531
NYC & Co Facebook http://www.facebook.com/nycgo.ca 2012-06-13 504
NYCHA Facebook http://www.facebook.com/NYCHA 2012-06-13 1039
NYC & Co Facebook http://www.facebook.com/nycgo.uk 2012-06-13 1067
NYC & Co Facebook http://www.facebook.com/nycgo.it 2012-06-13 1584
Culture Facebook https://www.facebook.com/piypnyc 2012-06-13 994
SBS Facebook http://www.facebook.com/NYCBusiness 2012-06-13 1176
FUND Facebook http://www.facebook.com/mayorsfundtoadvancenyc 2012-06-13 965
DOT Facebook http://www.facebook.com/YouTheManNYC 2012-06-13 1071
NYC & Co Facebook http://www.facebook.com/nycgo.es 2012-06-13 2300
HHC Facebook http://www.facebook.com/nychhc 2012-06-13 1192
MOME Facebook http://www.facebook.com/nycmedia.jobhunt 2012-06-13 1205
GreeNYC Facebook https://www.facebook.com/birdienyc 2012-06-13 1508
311 Facebook http://www.facebook.com/pages/New-York-City-311/84372567650 2012-06-13 1540
DOH Facebook http://www.facebook.com/NYCKnows 2012-06-13 1760
DOE Facebook http://www.facebook.com/nycgrads 2012-06-13 1822
DEP Facebook http://www.facebook.com/nycwater 2012-06-13 2618
MOME Facebook https://www.facebook.com/NYCMedia 2012-06-13 3091
EDC Facebook http://www.facebook.com/NYCEDC 2012-06-13 3519
SBS - Workforce1 Facebook http://www.facebook.com/nycworkforce1 2012-06-13 5048
DOT Facebook http://www.facebook.com/NYCDOT 2012-06-13 4059
EDC Facebook http://www.facebook.com/AppSciNYC 2012-06-13 3894
DYCD Facebook http://www.facebook.com/nycyouth 2012-06-13 4867
DOH Facebook http://www.facebook.com/EatingHealthyNYC 2012-06-13 34771
NYC & Co Facebook http://www.facebook.com/nycgo.br 2012-06-13 5404
DOE Facebook http://www.facebook.com/NYCTeachingFellows 2012-06-13 5491
NYCService Facebook http://www.facebook.com/nycservice 2012-06-13 5669
NYC Mayors Cup Facebook https://www.facebook.com/nycmayorscup 2012-06-13 11915
DOH Facebook http://www.facebook.com/nycquits 2012-06-13 9687
DOE Facebook http://www.facebook.com/pages/I-TEACH-NYC/11409913191 2012-06-13 7789
DOE Facebook http://www.facebook.com/fundforpublicschools 2012-06-13 8337
DPR Facebook http://www.facebook.com/nycparks 2012-06-13 15375
OEM Facebook http://www.facebook.com/nycemergencymanagement 2012-06-13 14286
DOE Facebook http://www.facebook.com/NYCschools 2012-06-13 21228
DOH Facebook http://www.facebook.com/NYCcondom 2012-06-13 18814
NYC & Co Facebook http://www.facebook.com/nycgo 2012-06-13 44625
FDNY Facebook http://www.facebook.com/FDNYhome 2012-06-13 85205
CCRB Facebook https://www.facebook.com/home.php#!/pages/NYC-Civilian-Complaint-Review-Board/152765208087880 2012-06-13 18
Commission on Human Rights Facebook http://www.facebook.com/NYCCommissionOnHumanRights 2012-06-13 76
DOB Facebook http://www.facebook.com/NYCBuildings 2012-06-13 1133
DSNY Facebook http://www.facebook.com/pages/NYC-Recycle-More-Waste-Less/152173854860863 2012-06-13 110
HDP Facebook http://www.facebook.com/pages/NYC-HPD-POE/128962093860639 2012-06-13 664
HPD/Commission on Human Rights Facebook http://www.facebook.com/FairHousingNyc 2012-06-13 50
LPC Facebook http://www.facebook.com/pages/NYC-Landmarks-Preservation-Commission/133261836703216 2012-06-13 164
Materials for the Arts Facebook https://www.facebook.com/mftanyc 2012-06-13 2934
MOAE Facebook http://www.facebook.com/pages/You-Can-Too/203525729692056 2012-06-13 91
MOIA Facebook http://www.facebook.com/pages/WE-ARE-NEW-YORK/174438697072 2012-06-13 1961
MOME Facebook http://www.facebook.com/NYCMINY 2012-06-13 42
NYC Gov Facebook http://www.facebook.com/nycgov 2012-06-13 18855
NYCCFB Facebook http://www.facebook.com/nycvotes 2012-06-13 112
NYPD Facebook https://www.facebook.com/NYPD 2012-06-13 26933
DRIS Facebook http://www.facebook.com/NycDeptOfRecords 2012-06-13 null
DRIS Facebook http://www.facebook.com/MayorEdKochNYCRecords 2012-06-13 null
DRIS Facebook http://www.facebook.com/MayorJohnLindsayNYCRecords 2012-06-13 null
DRIS Facebook http://www.facebook.com/MayorFiorelloLaGuardiaNYCRecords 2012-06-13 null
LMEC Flickr http://www.facebook.com/pages/New-York-NY/Latin-Media-and-Entertainment-Week/122259604487271 2012-06-13 332
DEP Flickr http://www.flickr.com/photos/nycep 2012-06-13 null
DOB Flickr http://www.flickr.com/photos/nyc_buildings/ 2012-06-13 null
DOE Flickr http://www.flickr.com/photos/nycschools 2012-06-13 null
DOITT Flickr http://www.flickr.com/photos/nyc_doitt 2012-06-13 null
DOT Flickr http://www.flickr.com/photos/nycstreets 2012-06-13 null
DPR Flickr http://www.flickr.com/photos/nycparks/ 2012-06-13 null
DSNY Flickr http://flickr.com/nycrecyclemore 2012-06-13 null
EDC Flickr http://www.flickr.com/nycedc 2012-06-13 null
FDNY Flickr http://www.flickr.com/groups/fdny-ems 2012-06-13 null
HHC Flickr http://www.flickr.com/hhcnyc 2012-06-13 null
LMEC Flickr http://www.flickr.com/photos/nyclatinmedia/ 2012-06-13 null
LPC Flickr http://www.flickr.com/photos/nyclandmarks 2012-06-13 null
Materials for the Arts Flickr http://www.flickr.com/photos/materialsforthearts 2012-06-13 null
val df = spark.table("social_media_usage") // Ctrl+Enter
df: org.apache.spark.sql.DataFrame = [agency: string, platform: string ... 3 more fields]

As you can see the immutable value df is a DataFrame and more specifically it is:

org.apache.spark.sql.DataFrame = [agency: string, platform: string, url: string, date: timestamp, visits: integer].

Now let us print schema of the DataFrame df and have a look at the actual data:

// Ctrl+Enter
df.printSchema() // prints schema of the DataFrame
df.show() // shows first n (default is 20) rows
root
 |-- agency: string (nullable = true)
 |-- platform: string (nullable = true)
 |-- url: string (nullable = true)
 |-- date: string (nullable = true)
 |-- visits: string (nullable = true)

+-----------------+----------+--------------------+----------+-------+
|           agency|  platform|                 url|      date| visits|
+-----------------+----------+--------------------+----------+-------+
|             MOAE|   YouTube|http://www.youtub...|2012-03-14|      5|
|          NYC Gov|   YouTube|http://www.youtub...|2012-03-14|      7|
|        NYC Water|   Youtube|http://www.youtub...|2012-03-14|   null|
|Veteran's Affairs|   Youtube|http://www.youtub...|2012-03-14|   null|
|             DRIS|   YouTube|http://www.youtub...|2012-03-14|   null|
|              TLC|  Facebook|http://www.facebo...|2012-03-14|    585|
|           nycgov|   Google+|https://plus.goog...|2012-03-14|   null|
|            TOTAL|     TOTAL|               TOTAL|2012-03-14|1688764|
|              DOH|   Android|       Condom Finder|2012-04-09|   null|
|              DOT|   Android|         You The Man|2012-04-09|   null|
|             MOME|   Android|      MiNY Venor app|2012-04-09|    343|
|              DOT|Broadcastr|                null|2012-04-09|   null|
|              DPR|Broadcastr|http://beta.broad...|2012-04-09|   null|
|            ENDHT|  Facebook|http://www.facebo...|2012-04-09|      9|
|              VAC|  Facebook|https://www.faceb...|2012-04-09|     55|
|           PlaNYC|  Facebook|http://www.facebo...|2012-04-09|     92|
|             DFTA|  Facebook|http://www.facebo...|2012-04-09|    178|
|              DOT|Broadcastr|                null|2012-10-24|   null|
|        energyNYC|  Facebook|http://www.facebo...|2012-04-09|    181|
|             MOIA|  Facebook|http://www.facebo...|2012-04-09|    171|
+-----------------+----------+--------------------+----------+-------+
only showing top 20 rows

Note that (nullable = true) simply means if the value is allowed to be null.

Let us count the number of rows in df.

df.count() // Ctrl+Enter to get 5898
res11: Long = 5898

So there are 5899 records or rows in the DataFrame df. Pretty good! You can also select individual columns using so-called DataFrame API, as follows:

val platforms = df.select("platform") // Shift+Enter
platforms: org.apache.spark.sql.DataFrame = [platform: string]
platforms.count() // Shift+Enter to count the number of rows
res13: Long = 5898
platforms.show(5) // Ctrl+Enter to show top 5 rows
+--------+
|platform|
+--------+
| YouTube|
| YouTube|
| Youtube|
| Youtube|
| YouTube|
+--------+
only showing top 5 rows
platforms.rdd.getNumPartitions
res15: Int = 2

You can also apply .distinct() to extract only unique entries as follows:

val uniquePlatforms = df.select("platform").distinct() // Shift+Enter
uniquePlatforms: org.apache.spark.sql.Dataset[org.apache.spark.sql.Row] = [platform: string]
uniquePlatforms.count() // Ctrl+Enter to count the number of distinct platforms
res17: Long = 23

Let's see all the rows of the DataFrame uniquePlatforms.

Note that display(uniquePlatforms) unlike uniquePlatforms.show() displays all rows of the DataFrame + gives you ability to select different view, e.g. charts.

uniquePlatforms.show(25,false)
+-------------------------+
|platform                 |
+-------------------------+
|nyc.gov                  |
|Flickr                   |
|Vimeo                    |
|iPhone                   |
|YouTube                  |
|WordPress                |
|SMS                      |
|iPhone App               |
|Youtube                  |
|Instagram                |
|iPhone app               |
|Linked-In                |
|Twitter                  |
|TOTAL                    |
|Tumblr                   |
|Newsletter               |
|Pinterest                |
|Broadcastr               |
|Android                  |
|Foursquare               |
|Google+                  |
|Foursquare (Badge Unlock)|
|Facebook                 |
+-------------------------+
//display(uniquePlatforms) // Ctrl+Enter to show all rows; use the scroll-bar on the right of the display to see all platforms

Spark SQL and DataFrame API

Spark SQL provides DataFrame API that can perform relational operations on both external data sources and internal collections, which is similar to widely used data frame concept in R, but evaluates operations support lazily (remember RDDs?), so that it can perform relational optimizations. This API is also available in Java, Python and R, but some functionality may not be available, although with every release of Spark people minimize this gap.

So we give some examples how to query data in Python and R, but continue with Scala. You can do all DataFrame operations in this notebook using Python or R.

# Ctrl+Enter to evaluate this python cell, recall '#' is the pre-comment character in python
# Using Python to query our "social_media_usage" table
pythonDF = spark.table("social_media_usage").select("platform").distinct()
pythonDF.show(3)
-- Ctrl+Enter to achieve the same result using standard SQL syntax!
select distinct platform from social_media_usage
platform
nyc.gov
Flickr
Vimeo
iPhone
YouTube
WordPress
SMS
iPhone App
Youtube
Instagram
iPhone app
Linked-In
Twitter
TOTAL
Tumblr
Newsletter
Pinterest
Broadcastr
Android
Foursquare
Google+
Foursquare (Badge Unlock)
Facebook

Now it is time for some tips around how you use select and what the difference is between $"a", col("a"), df("a").

As you probably have noticed by now, you can specify individual columns to select by providing String values in select statement. But sometimes you need to: - distinguish between columns with the same name - use it to filter (actually you can still filter using full String expression) - do some "magic" with joins and user-defined functions (this will be shown later)

So Spark gives you ability to actually specify columns when you select. Now the difference between all those three notations is ... none, those things are just aliases for a Column in Spark SQL, which means following expressions yield the same result:

// Using string expressions
df.select("agency", "visits")

// Using "$" alias for column
df.select($"agency", $"visits")

import org.apache.spark.sql.functions.col
// Using "col" alias for column
df.select(col("agency"), col("visits"))

// Using DataFrame name for column
df.select(df("agency"), df("visits"))
import org.apache.spark.sql.functions.col
res20: org.apache.spark.sql.DataFrame = [agency: string, visits: string]

This "same-difference" applies to filtering, i.e. you can either use full expression to filter, or column as shown in the following example:

// Using column to filter
df.select("visits").filter($"visits" > 100)

// Or you can use full expression as string
df.select("visits").filter("visits > 100")
res22: org.apache.spark.sql.Dataset[org.apache.spark.sql.Row] = [visits: string]
df.select("visits").filter('visits > 100).count // same as df.select("visits").filter($"visits" > 100)
res24: Long = 3279
df.select("visits").filter(col("visits") > 100).count// df.select("visits").filter($"visits" > 100)
res25: Long = 3279

Note that $"visits" > 100 expression looks amazing, but under the hood it is just another column, and it equals to df("visits").>(100), where, thanks to Scala paradigm > is just another function that you can define.

val sms = df.select($"agency", $"platform", $"visits").filter($"platform" === "SMS")
sms.show() // Ctrl+Enter
+------+--------+------+
|agency|platform|visits|
+------+--------+------+
|   DOE|     SMS|   382|
| NYCHA|     SMS|  null|
|   OEM|     SMS| 61652|
|   DOE|     SMS|   382|
| NYCHA|     SMS|  null|
|   OEM|     SMS| 61652|
|   DOE|     SMS|  null|
| NYCHA|     SMS|  null|
|   OEM|     SMS|  null|
|   DOE|     SMS|  1253|
| NYCHA|     SMS|  8300|
|   OEM|     SMS| 44547|
|   DOE|     SMS|  1253|
| NYCHA|     SMS|  8300|
|   OEM|     SMS| 44547|
|   DOE|     SMS|  1253|
| NYCHA|     SMS|  8300|
|   OEM|     SMS| 44547|
|   DOE|     SMS|  1253|
| NYCHA|     SMS|  8300|
+------+--------+------+
only showing top 20 rows

sms: org.apache.spark.sql.Dataset[org.apache.spark.sql.Row] = [agency: string, platform: string ... 1 more field]

Again you could have written the query above using any column aliases or String names or even writing the query directly.

For example, we can do it using String names, as follows:

// Ctrl+Enter Note that we are using "platform = 'SMS'" since it will be evaluated as actual SQL
val sms = df.select(df("agency"), df("platform"), df("visits")).filter("platform = 'SMS'")
sms.show(5)
+------+--------+------+
|agency|platform|visits|
+------+--------+------+
|   DOE|     SMS|   382|
| NYCHA|     SMS|  null|
|   OEM|     SMS| 61652|
|   DOE|     SMS|   382|
| NYCHA|     SMS|  null|
+------+--------+------+
only showing top 5 rows

sms: org.apache.spark.sql.Dataset[org.apache.spark.sql.Row] = [agency: string, platform: string ... 1 more field]

Refer to the DataFrame API for more detailed API. In addition to simple column references and expressions, DataFrames also have a rich library of functions including string manipulation, date arithmetic, common math operations and more. The complete list is available in the DataFrame Function Reference.

Let's next explore some of the functionality that is available by transforming this DataFrame df into a new DataFrame called fixedDF.

  • First, note that some columns are not exactly what we want them to be.
    • visits should not contain null values, but 0s instead.
  • Let us fix it using some code that is briefly explained here (don't worry if you don't get it completely now, you will get the hang of it by playing more)
    • The coalesce function is similar to if-else statement, i.e., if first column in expression is null, then the value of the second column is used and so on.
    • lit just means column of constant value (literally speaking!).
    • we also remove TOTAL value from platform column.
// Ctrl+Enter to make fixedDF

// import the needed sql functions
import org.apache.spark.sql.functions.{coalesce, lit}

// make the fixedDF DataFrame
val fixedDF = df.
   select(
     $"agency", 
     $"platform", 
     $"url", 
     $"date", 
     coalesce($"visits", lit(0)).as("visits"))
    .filter($"platform" =!= "TOTAL")

fixedDF.printSchema() // print its schema 
// and show the top 20 records fully
fixedDF.show(false) // the false argument does not truncate the rows, so you will not see something like this "anot..."
root
 |-- agency: string (nullable = true)
 |-- platform: string (nullable = true)
 |-- url: string (nullable = true)
 |-- date: string (nullable = true)
 |-- visits: string (nullable = false)

+-----------------+----------+--------------------------------------------------------------------------------------+----------+------+
|agency           |platform  |url                                                                                   |date      |visits|
+-----------------+----------+--------------------------------------------------------------------------------------+----------+------+
|MOAE             |YouTube   |http://www.youtube.com/YouCanTooNYC                                                   |2012-03-14|5     |
|NYC Gov          |YouTube   |http://www.youtube.com/nycgov                                                         |2012-03-14|7     |
|NYC Water        |Youtube   |http://www.youtube.com/nycwater                                                       |2012-03-14|0     |
|Veteran's Affairs|Youtube   |http://www.youtube.com/channel/UCi6IvOszIb3hHPMUsaNKyXA                               |2012-03-14|0     |
|DRIS             |YouTube   |http://www.youtube.com/nycdeptofrecords                                               |2012-03-14|0     |
|TLC              |Facebook  |http://www.facebook.com/pages/NYC-Taxi-and-Limousine-Commission/101679939900978?v=wall|2012-03-14|585   |
|nycgov           |Google+   |https://plus.google.com/u/0/b/104030911277642419611/104030911277642419611/posts/p/pub |2012-03-14|0     |
|DOH              |Android   |Condom Finder                                                                         |2012-04-09|0     |
|DOT              |Android   |You The Man                                                                           |2012-04-09|0     |
|MOME             |Android   |MiNY Venor app                                                                        |2012-04-09|343   |
|DOT              |Broadcastr|null                                                                                  |2012-04-09|0     |
|DPR              |Broadcastr|http://beta.broadcastr.com/Echo.html?audioId=670026-4001                              |2012-04-09|0     |
|ENDHT            |Facebook  |http://www.facebook.com/pages/NYC-Lets-End-Human-Trafficking/125730490795659?sk=wall  |2012-04-09|9     |
|VAC              |Facebook  |https://www.facebook.com/pages/NYC-Voter-Assistance-Commission/110226709012110        |2012-04-09|55    |
|PlaNYC           |Facebook  |http://www.facebook.com/pages/New-York-NY/PlaNYC/160454173971169?ref=ts               |2012-04-09|92    |
|DFTA             |Facebook  |http://www.facebook.com/pages/NYC-Department-for-the-Aging/109028655823590            |2012-04-09|178   |
|DOT              |Broadcastr|null                                                                                  |2012-10-24|0     |
|energyNYC        |Facebook  |http://www.facebook.com/EnergyNYC?sk=wall                                             |2012-04-09|181   |
|MOIA             |Facebook  |http://www.facebook.com/ihwnyc                                                        |2012-04-09|171   |
|City Store       |Facebook  |http://www.facebook.com/citystorenyc                                                  |2012-04-09|214   |
+-----------------+----------+--------------------------------------------------------------------------------------+----------+------+
only showing top 20 rows

import org.apache.spark.sql.functions.{coalesce, lit}
fixedDF: org.apache.spark.sql.Dataset[org.apache.spark.sql.Row] = [agency: string, platform: string ... 3 more fields]

Okay, this is better, but urls are still inconsistent.

Let's fix this by writing our own UDF (user-defined function) to deal with special cases.

Note that if you CAN USE Spark functions library, do it. But for the sake of the example, custom UDF is shown below.

We take value of a column as String type and return the same String type, but ignore values that do not start with http.

// Ctrl+Enter to evaluate this UDF which takes a input String called "value"
// and converts it into lower-case if it begins with http and otherwise leaves it as null, so we sort of remove non valid web-urls
val cleanUrl = udf((value: String) => if (value != null && value.startsWith("http")) value.toLowerCase() else null)
cleanUrl: org.apache.spark.sql.expressions.UserDefinedFunction = SparkUserDefinedFunction($Lambda$9812/558964277@3b5bdfcc,StringType,List(Some(class[value[0]: string])),Some(class[value[0]: string]),None,true,true)

Let us apply our UDF on fixedDF to create a new DataFrame called cleanedDF as follows:

// Ctrl+Enter
val cleanedDF = fixedDF.select($"agency", $"platform", cleanUrl($"url").as("url"), $"date", $"visits")
cleanedDF: org.apache.spark.sql.DataFrame = [agency: string, platform: string ... 3 more fields]

Now, let's check that it actually worked by seeing the first 5 rows of the cleanedDF whose url isNull and isNotNull, as follows:

// Shift+Enter
cleanedDF.filter($"url".isNull).show(5)
+------+----------+----+----------+------+
|agency|  platform| url|      date|visits|
+------+----------+----+----------+------+
|   DOH|   Android|null|2012-04-09|     0|
|   DOT|   Android|null|2012-04-09|     0|
|  MOME|   Android|null|2012-04-09|   343|
|   DOT|Broadcastr|null|2012-04-09|     0|
|   DOT|Broadcastr|null|2012-10-24|     0|
+------+----------+----+----------+------+
only showing top 5 rows
// Ctrl+Enter
cleanedDF.filter($"url".isNotNull).show(5, false) // false in .show(5, false) shows rows untruncated
+-----------------+--------+-------------------------------------------------------+----------+------+
|agency           |platform|url                                                    |date      |visits|
+-----------------+--------+-------------------------------------------------------+----------+------+
|MOAE             |YouTube |http://www.youtube.com/youcantoonyc                    |2012-03-14|5     |
|NYC Gov          |YouTube |http://www.youtube.com/nycgov                          |2012-03-14|7     |
|NYC Water        |Youtube |http://www.youtube.com/nycwater                        |2012-03-14|0     |
|Veteran's Affairs|Youtube |http://www.youtube.com/channel/uci6ivoszib3hhpmusankyxa|2012-03-14|0     |
|DRIS             |YouTube |http://www.youtube.com/nycdeptofrecords                |2012-03-14|0     |
+-----------------+--------+-------------------------------------------------------+----------+------+
only showing top 5 rows

Now there is a suggestion from you manager's manager's manager that due to some perceived privacy concerns we want to replace agency with some unique identifier.

So we need to do the following:

  • create unique list of agencies with ids and
  • join them with main DataFrame.

Sounds easy, right? Let's do it.

// Crtl+Enter
// Import Spark SQL function that will give us unique id across all the records in this DataFrame
import org.apache.spark.sql.functions.monotonically_increasing_id

// We append column as SQL function that creates unique ids across all records in DataFrames 
val agencies = cleanedDF.select(cleanedDF("agency"))
                        .distinct()
                        .withColumn("id", monotonically_increasing_id())
agencies.show(5)
+--------------------+---+
|              agency| id|
+--------------------+---+
|              PlaNYC|  0|
|                 HIA|  1|
|NYC Digital: exte...|  2|
|           NYCGLOBAL|  3|
|              nycgov|  4|
+--------------------+---+
only showing top 5 rows

import org.apache.spark.sql.functions.monotonically_increasing_id
agencies: org.apache.spark.sql.DataFrame = [agency: string, id: bigint]
// Ctrl+Enter
// And join with the rest of the data, note how join condition is specified 
val anonym = cleanedDF.join(agencies, cleanedDF("agency") === agencies("agency"), "inner").select("id", "platform", "url", "date", "visits")

// We also cache DataFrame since it can be quite expensive to recompute join
anonym.cache()

// Display result
anonym.show(5)
+-------------+--------+--------------------+----------+------+
|           id|platform|                 url|      date|visits|
+-------------+--------+--------------------+----------+------+
| 841813590016| YouTube|http://www.youtub...|2012-03-14|     5|
| 901943132160| YouTube|http://www.youtub...|2012-03-14|     7|
| 575525617664| Youtube|http://www.youtub...|2012-03-14|     0|
|1657857376256| Youtube|http://www.youtub...|2012-03-14|     0|
| 171798691840| YouTube|http://www.youtub...|2012-03-14|     0|
+-------------+--------+--------------------+----------+------+
only showing top 5 rows

anonym: org.apache.spark.sql.DataFrame = [id: bigint, platform: string ... 3 more fields]
spark.catalog.listTables().show() // look at the available tables
+--------------------+--------+-----------+---------+-----------+
|                name|database|description|tableType|isTemporary|
+--------------------+--------+-----------+---------+-----------+
|          all_prices| default|       null|  MANAGED|      false|
|bitcoin_normed_wi...| default|       null|  MANAGED|      false|
|bitcoin_reversals...| default|       null|  MANAGED|      false|
|        countrycodes| default|       null| EXTERNAL|      false|
|  gold_normed_window| default|       null|  MANAGED|      false|
|gold_reversals_wi...| default|       null|  MANAGED|      false|
|ltcar_locations_2...| default|       null|  MANAGED|      false|
|            magellan| default|       null|  MANAGED|      false|
|       mobile_sample| default|       null| EXTERNAL|      false|
|   oil_normed_window| default|       null|  MANAGED|      false|
|oil_reversals_window| default|       null|  MANAGED|      false|
|oil_reversals_win...| default|       null|  MANAGED|      false|
|    over300all_2_txt| default|       null|  MANAGED|      false|
|              person| default|       null|  MANAGED|      false|
|            personer| default|       null|  MANAGED|      false|
|             persons| default|       null|  MANAGED|      false|
|        simple_range| default|       null|  MANAGED|      false|
|  social_media_usage| default|       null|  MANAGED|      false|
|social_media_usag...| default|       null|  MANAGED|      false|
|voronoi20191213up...| default|       null|  MANAGED|      false|
+--------------------+--------+-----------+---------+-----------+
only showing top 20 rows
-- to remove a TempTable if it exists already
drop table if exists anonym
// Register table for Spark SQL, we also import "month" function 
import org.apache.spark.sql.functions.month

anonym.createOrReplaceTempView("anonym")
import org.apache.spark.sql.functions.month
-- Interesting. Now let's do some aggregation. Display platform, month, visits
-- Date column allows us to extract month directly

select platform, month(date) as month, sum(visits) as visits from anonym group by platform, month(date)
platform month visits
Foursquare (Badge Unlock) 6.0 0.0
Instagram 9.0 27891.0
Linked-In 10.0 60156.0
iPhone 8.0 10336.0
Twitter 9.0 819290.0
Vimeo 11.0 0.0
iPhone app 9.0 33348.0
SMS 10.0 54100.0
YouTube 3.0 6066.0
Instagram 11.0 58968.0
Android 4.0 724.0
Newsletter 11.0 3079091.0
Android 11.0 11259.0
Youtube 10.0 429.0
iPhone App 4.0 55960.0
Facebook 3.0 291971.0
Google+ 5.0 0.0
Newsletter 5.0 847813.0
Instagram 10.0 28145.0
Linked-In 7.0 31758.0
Tumblr 5.0 26932.0
Linked-In 6.0 30867.0
YouTube 11.0 22820.0
SMS 11.0 170234.0
Foursquare (Badge Unlock) 11.0 22512.0
Youtube 11.0 770.0
Foursquare (Badge Unlock) 4.0 20152.0
Facebook 5.0 351601.0
Foursquare 8.0 42230.0
Facebook 6.0 399330.0
Linked-In 8.0 45346.0
Google+ 10.0 8.0
YouTube 4.0 12542.0
Newsletter 6.0 1137677.0
YouTube 7.0 6748.0
Google+ 4.0 0.0
Youtube 6.0 229.0
Youtube 5.0 0.0
Linked-In 11.0 75747.0
Vimeo 10.0 0.0
Android 5.0 381.0
Flickr 8.0 493.0
iPhone app 8.0 41389.0
Facebook 10.0 1010914.0
Foursquare (Badge Unlock) 10.0 11256.0
Instagram 8.0 5995.0
Android 6.0 102.0
Tumblr 7.0 47121.0
Android 10.0 5473.0
iPhone app 10.0 30713.0
Youtube 3.0 163.0
WordPress 6.0 4641.0
iPhone 11.0 25848.0
Youtube 7.0 240.0
Facebook 4.0 674303.0
iPhone App 11.0 96342.0
iPhone 5.0 8203.0
iPhone app 5.0 30598.0
Tumblr 8.0 54224.0
Vimeo 6.0 0.0
Twitter 10.0 859354.0
Vimeo 8.0 0.0
iPhone App 6.0 34966.0
iPhone 6.0 9643.0
SMS 5.0 0.0
Vimeo 7.0 0.0
iPhone app 6.0 2713.0
Newsletter 10.0 2359712.0
Pinterest 11.0 312.0
Broadcastr 6.0 0.0
Pinterest 6.0 0.0
Android 8.0 5784.0
Newsletter 4.0 1606654.0
Newsletter 9.0 1941202.0
WordPress 10.0 66299.0
SMS 8.0 116134.0
Twitter 7.0 470477.0
iPhone 4.0 8203.0
iPhone app 11.0 72102.0
Instagram 4.0 3404.0
Flickr 10.0 153231.0
iPhone app 4.0 41274.0
WordPress 8.0 5017.0
YouTube 9.0 12107.0
Linked-In 4.0 43582.0
YouTube 8.0 10974.0
Broadcastr 9.0 0.0
Flickr 5.0 287.0
Tumblr 10.0 97401.0
Broadcastr 8.0 0.0
iPhone App 9.0 42128.0
Twitter 4.0 844718.0
Broadcastr 10.0 0.0
Tumblr 6.0 41248.0
WordPress 4.0 0.0
Twitter 11.0 1660542.0
Linked-In 9.0 49299.0
Android 9.0 445.0
iPhone 9.0 12924.0
WordPress 9.0 4897.0
Foursquare 10.0 69598.0
Pinterest 10.0 141.0
Facebook 9.0 754875.0
YouTube 10.0 9100.0
Broadcastr 11.0 0.0
Pinterest 8.0 38.0
Twitter 6.0 461261.0
Google+ 3.0 0.0
Linked-In 5.0 29808.0
Foursquare 5.0 29991.0
Foursquare 7.0 38590.0
iPhone app 7.0 30713.0
Broadcastr 4.0 0.0
Instagram 5.0 0.0
Twitter 8.0 704438.0
Android 7.0 445.0
Flickr 9.0 549.0
WordPress 11.0 9294.0
Pinterest 4.0 0.0
Flickr 11.0 1007.0
iPhone 10.0 12924.0
Flickr 4.0 545.0
Facebook 7.0 451076.0
Google+ 8.0 0.0
Tumblr 9.0 62742.0
Foursquare 4.0 57337.0
Tumblr 4.0 31247.0
Broadcastr 5.0 0.0
YouTube 5.0 0.0
Pinterest 9.0 74.0
iPhone App 5.0 34288.0
Facebook 8.0 657312.0
Vimeo 5.0 0.0
SMS 9.0 116134.0
Google+ 7.0 0.0
SMS 6.0 54100.0
Vimeo 9.0 0.0
Pinterest 7.0 0.0
Google+ 11.0 26.0
Instagram 7.0 5450.0
Foursquare (Badge Unlock) 9.0 11256.0
Instagram 6.0 4764.0
iPhone App 10.0 64589.0
Foursquare (Badge Unlock) 7.0 0.0
Flickr 6.0 332.0
SMS 7.0 54100.0
Twitter 5.0 435148.0
Youtube 9.0 281.0
iPhone App 8.0 57513.0
Google+ 9.0 0.0
Pinterest 5.0 0.0
Foursquare (Badge Unlock) 8.0 11256.0
Facebook 11.0 1408965.0
iPhone App 7.0 35841.0
SMS 4.0 124068.0
WordPress 7.0 4647.0
Youtube 4.0 367.0
Foursquare (Badge Unlock) 5.0 0.0
Foursquare 6.0 34193.0
Google+ 6.0 0.0
Youtube 8.0 258.0
Newsletter 7.0 1137868.0
Flickr 7.0 342.0
Foursquare 9.0 50489.0
Foursquare 11.0 118323.0
Newsletter 8.0 1941197.0
Tumblr 11.0 195881.0
Broadcastr 7.0 0.0
WordPress 5.0 0.0
YouTube 6.0 6509.0
Vimeo 4.0 0.0
iPhone 7.0 10336.0
Instagram 1.0 0.0
Linked-In 1.0 19007.0
YouTube 2.0 4937.0
iPhone 2.0 0.0
Google+ 2.0 0.0
Instagram 2.0 0.0
Android 3.0 343.0
Newsletter 12.0 1606654.0
iPhone app 12.0 21352.0
SMS 2.0 62034.0
Foursquare 3.0 25786.0
iPhone App 2.0 21672.0
YouTube 12.0 9505.0
Tumblr 3.0 5098.0
Foursquare 1.0 10126.0
Facebook 1.0 259797.0
iPhone App 1.0 21672.0
Vimeo 12.0 0.0
Flickr 1.0 217.0
WordPress 2.0 0.0
Youtube 2.0 155.0
Pinterest 1.0 0.0
Linked-In 3.0 20761.0
Broadcastr 3.0 0.0
SMS 12.0 124068.0
Youtube 12.0 291.0
Twitter 1.0 364376.0
Newsletter 3.0 803327.0
iPhone App 12.0 43344.0
WordPress 1.0 0.0
iPhone app 3.0 10676.0
Flickr 12.0 432.0
Android 12.0 686.0
Android 1.0 343.0
WordPress 12.0 0.0
Google+ 1.0 0.0
iPhone 1.0 0.0
Foursquare (Badge Unlock) 12.0 0.0
Linked-In 12.0 35231.0
Foursquare (Badge Unlock) 1.0 0.0
Flickr 3.0 227.0
YouTube 1.0 4904.0
Broadcastr 12.0 0.0
iPhone 12.0 0.0
Newsletter 1.0 803327.0
Broadcastr 2.0 0.0
Linked-In 2.0 19920.0
Pinterest 2.0 0.0
Foursquare 12.0 19110.0
Vimeo 2.0 0.0
Twitter 12.0 690189.0
Tumblr 1.0 2645.0
Foursquare 2.0 21181.0
Broadcastr 1.0 0.0
Twitter 3.0 400250.0
Foursquare (Badge Unlock) 2.0 8878.0
Google+ 12.0 0.0
Vimeo 1.0 0.0
Newsletter 2.0 803327.0
Facebook 2.0 107993.0
Pinterest 12.0 0.0
Android 2.0 343.0
SMS 1.0 62034.0
WordPress 3.0 0.0
Instagram 3.0 0.0
iPhone App 3.0 21672.0
Youtube 1.0 150.0
iPhone 3.0 0.0
SMS 3.0 62034.0
Instagram 12.0 0.0
Flickr 2.0 219.0
Tumblr 12.0 5005.0
Twitter 2.0 385091.0
iPhone app 1.0 10676.0
Vimeo 3.0 0.0
Pinterest 3.0 0.0
iPhone app 2.0 10676.0
Foursquare (Badge Unlock) 3.0 0.0
Facebook 12.0 502687.0
Tumblr 2.0 4406.0

Note, that we could have done aggregation using DataFrame API instead of Spark SQL.

Alright, now let's see some cool operations with window functions.

Our next task is to compute (daily visits / monthly average) for all platforms.

import org.apache.spark.sql.functions.{dayofmonth, month, row_number, sum}
import org.apache.spark.sql.expressions.Window

val coolDF = anonym.select($"id", $"platform", dayofmonth($"date").as("day"), month($"date").as("month"), $"visits").
  groupBy($"id", $"platform", $"day", $"month").agg(sum("visits").as("visits"))

// Run window aggregation on visits per month and platform
val window = coolDF.select($"id", $"day", $"visits", sum($"visits").over(Window.partitionBy("platform", "month")).as("monthly_visits"))

// Create and register percent table
val percent = window.select($"id", $"day", ($"visits" / $"monthly_visits").as("percent"))

percent.createOrReplaceTempView("percent")
import org.apache.spark.sql.functions.{dayofmonth, month, row_number, sum}
import org.apache.spark.sql.expressions.Window
coolDF: org.apache.spark.sql.DataFrame = [id: bigint, platform: string ... 3 more fields]
window: org.apache.spark.sql.DataFrame = [id: bigint, day: int ... 2 more fields]
percent: org.apache.spark.sql.DataFrame = [id: bigint, day: int ... 1 more field]
-- A little bit of visualization as result of our efforts
select id, day, `percent` from percent where `percent` > 0.3 and day = 2
id day percent
6.52835028992e11 2.0 0.446576072475353
1.408749273089e12 2.0 0.6937119675456389
1.408749273089e12 2.0 0.394240317775571
9.0194313216e11 2.0 0.6180914042150131
2.147483648e11 2.0 0.3663035756571158
9.0194313216e11 2.0 1.0
9.0194313216e11 2.0 0.5
6.8719476736e10 2.0 0.38461538461538464
5.06806140929e11 2.0 1.0
5.06806140929e11 2.0 0.4993894993894994
1.322849927168e12 2.0 0.5265514047545539
1.322849927168e12 2.0 0.3109034021149352
9.0194313216e11 2.0 0.3060168545490231
1.666447310848e12 2.0 0.9473684210526315
1.580547964928e12 2.0 0.383582757848692
4.12316860416e11 2.0 0.3408084980820301
2.06158430208e11 2.0 0.9262507474586407
2.06158430208e11 2.0 0.5
6.52835028992e11 2.0 0.8449612403100775
6.52835028992e11 2.0 0.3181818181818182
6.52835028992e11 2.0 1.0
6.52835028992e11 2.0 0.5
1.640677507072e12 2.0 0.44748143897901344
6.52835028992e11 2.0 0.6765082509845611
6.52835028992e11 2.0 0.38833874233724447
-- You also could just use plain SQL to write query above, note that you might need to group by id and day as well.
with aggr as (
  select id, dayofmonth(date) as day, visits / sum(visits) over (partition by (platform, month(date))) as percent
  from anonym
)
select * from aggr where day = 2 and percent > 0.3
id day percent
6.52835028992e11 2.0 0.446576072475353
1.408749273089e12 2.0 0.6937119675456389
1.408749273089e12 2.0 0.394240317775571
2.147483648e11 2.0 0.3663035756571158
9.0194313216e11 2.0 0.6180914042150131
9.0194313216e11 2.0 1.0
9.0194313216e11 2.0 0.5
6.8719476736e10 2.0 0.38461538461538464
5.06806140929e11 2.0 1.0
5.06806140929e11 2.0 0.4993894993894994
1.322849927168e12 2.0 0.4718608035989944
9.0194313216e11 2.0 0.3060168545490231
1.666447310848e12 2.0 0.9473684210526315
1.580547964928e12 2.0 0.383582757848692
4.12316860416e11 2.0 0.3408084980820301
2.06158430208e11 2.0 0.9262507474586407
2.06158430208e11 2.0 0.5
6.52835028992e11 2.0 0.8449612403100775
6.52835028992e11 2.0 0.3181818181818182
6.52835028992e11 2.0 1.0
6.52835028992e11 2.0 0.5
1.640677507072e12 2.0 0.44748143897901344
6.52835028992e11 2.0 0.6765082509845611
6.52835028992e11 2.0 0.38833874233724447

Interoperating with RDDs

Spark SQL supports two different methods for converting existing RDDs into DataFrames. The first method uses reflection to infer the schema of an RDD that contains specific types of objects. This reflection based approach leads to more concise code and works well when you already know the schema.

The second method for creating DataFrames is through a programmatic interface that allows you to construct a schema and then apply it to an existing RDD. While this method is more verbose, it allows you to construct DataFrames when the columns and their types are not known until runtime.

Inferring the Schema Using Reflection

The Scala interface for Spark SQL supports automatically converting an RDD containing case classes to a DataFrame. The case class defines the schema of the table. The names of the arguments to the case class are read using reflection and become the names of the columns. Case classes can also be nested or contain complex types such as Sequences or Arrays. This RDD can be implicitly converted to a DataFrame and then be registered as a table.

// Define case class that will be our schema for DataFrame
case class Hubot(name: String, year: Int, manufacturer: String, version: Array[Int], details: Map[String, String])

// You can process a text file, for example, to convert every row to our Hubot, but we will create RDD manually
val rdd = sc.parallelize(
  Array(
    Hubot("Jerry", 2015, "LCorp", Array(1, 2, 3), Map("eat" -> "yes", "sleep" -> "yes", "drink" -> "yes")),
    Hubot("Mozart", 2010, "LCorp", Array(1, 2), Map("eat" -> "no", "sleep" -> "no", "drink" -> "no")),
    Hubot("Einstein", 2012, "LCorp", Array(1, 2, 3), Map("eat" -> "yes", "sleep" -> "yes", "drink" -> "no"))
  )
)
defined class Hubot
rdd: org.apache.spark.rdd.RDD[Hubot] = ParallelCollectionRDD[376] at parallelize at command-2971213210278329:5
// In order to convert RDD into DataFrame you need to do this:
val hubots = rdd.toDF()

// Display DataFrame, note how array and map fields are displayed
hubots.printSchema()
hubots.show()
root
 |-- name: string (nullable = true)
 |-- year: integer (nullable = false)
 |-- manufacturer: string (nullable = true)
 |-- version: array (nullable = true)
 |    |-- element: integer (containsNull = false)
 |-- details: map (nullable = true)
 |    |-- key: string
 |    |-- value: string (valueContainsNull = true)

+--------+----+------------+---------+--------------------+
|    name|year|manufacturer|  version|             details|
+--------+----+------------+---------+--------------------+
|   Jerry|2015|       LCorp|[1, 2, 3]|{eat -> yes, slee...|
|  Mozart|2010|       LCorp|   [1, 2]|{eat -> no, sleep...|
|Einstein|2012|       LCorp|[1, 2, 3]|{eat -> yes, slee...|
+--------+----+------------+---------+--------------------+

hubots: org.apache.spark.sql.DataFrame = [name: string, year: int ... 3 more fields]
// You can query complex type the same as you query any other column
// By the way you can use `sql` function to invoke Spark SQL to create DataFrame
hubots.createOrReplaceTempView("hubots")

val onesThatEat = sqlContext.sql("select name, details.eat from hubots where details.eat = 'yes'")

onesThatEat.show()
+--------+---+
|    name|eat|
+--------+---+
|   Jerry|yes|
|Einstein|yes|
+--------+---+

onesThatEat: org.apache.spark.sql.DataFrame = [name: string, eat: string]

Programmatically Specifying the Schema

When case classes cannot be defined ahead of time (for example, the structure of records is encoded in a string, or a text dataset will be parsed and fields will be projected differently for different users), a DataFrame can be created programmatically with three steps.

  1. Create an RDD of Rows from the original RDD
  2. Create the schema represented by a StructType and StructField classes matching the structure of Rows in the RDD created in Step 1.
  3. Apply the schema to the RDD of Rows via createDataFrame method provided by SQLContext.
import org.apache.spark.sql.types._
import org.apache.spark.sql.Row

// Let's say we have an RDD of String and we need to convert it into a DataFrame with schema "name", "year", and "manufacturer"
// As you can see every record is space-separated.
val rdd = sc.parallelize(
  Array(
    "Jerry 2015 LCorp",
    "Mozart 2010 LCorp",
    "Einstein 2012 LCorp"
  )
)

// Create schema as StructType //
val schema = StructType(
  StructField("name", StringType, false) :: 
  StructField("year", IntegerType, false) :: 
  StructField("manufacturer", StringType, false) :: 
  Nil
)

// Prepare RDD[Row]
val rows = rdd.map { entry => 
  val arr = entry.split("\\s+")
  val name = arr(0)
  val year = arr(1).toInt
  val manufacturer = arr(2)
  
  Row(name, year, manufacturer)
}

// Create DataFrame
val bots = sqlContext.createDataFrame(rows, schema)
bots.printSchema()
bots.show()
root
 |-- name: string (nullable = false)
 |-- year: integer (nullable = false)
 |-- manufacturer: string (nullable = false)

+--------+----+------------+
|    name|year|manufacturer|
+--------+----+------------+
|   Jerry|2015|       LCorp|
|  Mozart|2010|       LCorp|
|Einstein|2012|       LCorp|
+--------+----+------------+

import org.apache.spark.sql.types._
import org.apache.spark.sql.Row
rdd: org.apache.spark.rdd.RDD[String] = ParallelCollectionRDD[381] at parallelize at command-2971213210278333:6
schema: org.apache.spark.sql.types.StructType = StructType(StructField(name,StringType,false),StructField(year,IntegerType,false),StructField(manufacturer,StringType,false))
rows: org.apache.spark.rdd.RDD[org.apache.spark.sql.Row] = MapPartitionsRDD[382] at map at command-2971213210278333:23
bots: org.apache.spark.sql.DataFrame = [name: string, year: int ... 1 more field]

Creating Datasets

A Dataset is a strongly-typed, immutable collection of objects that are mapped to a relational schema. At the core of the Dataset API is a new concept called an encoder, which is responsible for converting between JVM objects and tabular representation. The tabular representation is stored using Spark’s internal Tungsten binary format, allowing for operations on serialized data and improved memory utilization. Spark 2.2 comes with support for automatically generating encoders for a wide variety of types, including primitive types (e.g. String, Integer, Long), and Scala case classes.

Simply put, you will get all the benefits of DataFrames with fair amount of flexibility of RDD API.

// We can start working with Datasets by using our "hubots" table

// To create Dataset from DataFrame do this (assuming that case class Hubot exists):
val ds = hubots.as[Hubot]
ds.show(false)
+--------+----+------------+---------+----------------------------------------+
|name    |year|manufacturer|version  |details                                 |
+--------+----+------------+---------+----------------------------------------+
|Jerry   |2015|LCorp       |[1, 2, 3]|{eat -> yes, sleep -> yes, drink -> yes}|
|Mozart  |2010|LCorp       |[1, 2]   |{eat -> no, sleep -> no, drink -> no}   |
|Einstein|2012|LCorp       |[1, 2, 3]|{eat -> yes, sleep -> yes, drink -> no} |
+--------+----+------------+---------+----------------------------------------+

ds: org.apache.spark.sql.Dataset[Hubot] = [name: string, year: int ... 3 more fields]

Side-note: Dataset API is first-class citizen in Spark, and DataFrame is an alias for Dataset[Row]. Note that Python and R use DataFrames (since they are dynamically typed), but it is essentially a Dataset.

Finally

DataFrames and Datasets can simplify and improve most of the applications pipelines by bringing concise syntax and performance optimizations. We would highly recommend you to check out the official API documentation, specifically around

Unfortunately, this is just a getting started quickly course, and we skip features like custom aggregations, types, pivoting, etc., but if you are keen to know then start from the links above and this notebook and others in this directory. You may need them in a real-world project.

ScaDaMaLe Course site and book

This is an elaboration of the http://spark.apache.org/docs/latest/sql-programming-guide.html by Ivan Sadikov and Raazesh Sainudiin.

Data Sources

Spark Sql Programming Guide

  • Data Sources
    • Generic Load/Save Functions
      • Manually Specifying Options
      • Run SQL on files directly
      • Save Modes
      • Saving to Persistent Tables
    • Parquet Files
      • Loading Data Programmatically
      • Partition Discovery
      • Schema Merging
      • Hive metastore Parquet table conversion
        • Hive/Parquet Schema Reconciliation
        • Metadata Refreshing
      • Configuration
    • JSON Datasets
    • Hive Tables
      • Interacting with Different Versions of Hive Metastore
    • JDBC To Other Databases
    • Troubleshooting

Data Sources

Spark SQL supports operating on a variety of data sources through the DataFrame or DataFrame interfaces. A Dataset can be operated on as normal RDDs and can also be registered as a temporary table. Registering a Dataset as a table allows you to run SQL queries over its data. But from time to time you would need to either load or save Dataset. Spark SQL provides built-in data sources as well as Data Source API to define your own data source and use it read / write data into Spark.

Overview

Spark provides some built-in datasources that you can use straight out of the box, such as Parquet, JSON, JDBC, ORC (available with enabled Hive Support, but this is changing, and ORC will not require Hive support and will work with default Spark session starting from next release), and Text (since Spark 1.6) and CSV (since Spark 2.0, before that it is accessible as a package).

Third-party datasource packages

Community also have built quite a few datasource packages to provide easy access to the data from other formats. You can find list of those packages on http://spark-packages.org/, e.g. Avro, CSV, Amazon Redshit (for Spark < 2.0), XML, NetFlow and many others.

Generic Load/Save functions

In order to load or save DataFrame you have to call either read or write. This will return DataFrameReader or DataFrameWriter depending on what you are trying to achieve. Essentially these classes are entry points to the reading / writing actions. They allow you to specify writing mode or provide additional options to read data source.

// This will return DataFrameReader to read data source
println(spark.read)

val df = spark.range(0, 10)

// This will return DataFrameWriter to save DataFrame
println(df.write)
org.apache.spark.sql.DataFrameReader@52275b1e
org.apache.spark.sql.DataFrameWriter@3bdaf3ae
df: org.apache.spark.sql.Dataset[Long] = [id: bigint]
// Saving Parquet table in Scala
// DataFrames and tables can be saved as Parquet files, maintaining the schema information
val df_save = spark.table("social_media_usage").select("platform", "visits") // assuming you made the social_media_usage table permanent in previous notebook
df_save.write.mode("overwrite").parquet("/datasets/sds/tmp/platforms.parquet")

// Read in the parquet file created above
// Parquet files are self-describing so the schema is preserved
// The result of loading a Parquet file is also a DataFrame
val df = spark.read.parquet("/datasets/sds/tmp//platforms.parquet")
df.show(5)
ls /datasets/sds/tmp/platforms.parquet

Note /datasets/sds/tmp/platforms.parquet/ is a directory with many files in it... and files beginning with part have content in possibly many partitions.

We will take a brief look at Parquet very soon below.

# Loading Parquet table in Python
dfPy = spark.read.parquet("/datasets/sds/tmp/platforms.parquet")
dfPy.show(5)
// Saving JSON dataset in Scala
val df_save = spark.table("social_media_usage").select("platform", "visits")
df_save.write.mode("overwrite").json("/datasets/sds/tmp/platforms.json")

// Loading JSON dataset in Scala
val df = spark.read.json("/datasets/sds/tmp/platforms.json")
df.show(5)
# Loading JSON dataset in Python
dfPy = spark.read.json("/datasets/sds/tmp/platforms.json")
dfPy.show(5)

Manually Specifying Options

You can also manually specify the data source that will be used along with any extra options that you would like to pass to the data source. Data sources are specified by their fully qualified name (i.e., org.apache.spark.sql.parquet), but for built-in sources you can also use their short names (json, parquet, jdbc). DataFrames of any type can be converted into other types using this syntax.

val json = sqlContext.read.format("json").load("/datasets/sds/tmp/platforms.json")
json.select("platform").show(10)

val parquet = sqlContext.read.format("parquet").load("/datasets/sds/tmp/platforms.parquet")
parquet.select("platform").show(10)

Run SQL on files directly

Instead of using read API to load a file into DataFrame and query it, you can also query that file directly with SQL.

val df = sqlContext.sql("SELECT * FROM parquet.`/datasets/sds/tmp/platforms.parquet`")
df.printSchema()

Save Modes

Save operations can optionally take a SaveMode, that specifies how to handle existing data if present. It is important to realize that these save modes do not utilize any locking and are not atomic. Additionally, when performing a Overwrite, the data will be deleted before writing out the new data.

Scala/JavaAny languageMeaning
SaveMode.ErrorIfExists (default)"error" (default)When saving a DataFrame to a data source, if data already exists, an exception is expected to be thrown.
SaveMode.Append"append"When saving a DataFrame to a data source, if data/table already exists, contents of the DataFrame are expected to be appended to existing data.
SaveMode.Overwrite"overwrite"Overwrite mode means that when saving a DataFrame to a data source, if data/table already exists, existing data is expected to be overwritten by the contents of the DataFrame.
SaveMode.Ignore"ignore"Ignore mode means that when saving a DataFrame to a data source, if data already exists, the save operation is expected to not save the contents of the DataFrame and to not change the existing data. This is similar to a CREATE TABLE IF NOT EXISTS in SQL.

Saving to Persistent Tables

DataFrame and Dataset can also be saved as persistent tables using the saveAsTable command. Unlike the createOrReplaceTempView command, saveAsTable will materialize the contents of the dataframe and create a pointer to the data in the metastore. Persistent tables will still exist even after your Spark program has restarted, as long as you maintain your connection to the same metastore. A DataFrame for a persistent table can be created by calling the table method on a SparkSession with the name of the table.

By default saveAsTable will create a “managed table”, meaning that the location of the data will be controlled by the metastore. Managed tables will also have their data deleted automatically when a table is dropped.

// First of all list tables to see that table we are about to create does not exist
spark.catalog.listTables.where($"name" startsWith "social").show(false)
drop table if exists simple_range
val df = spark.range(0, 100)
df.write.mode(SaveMode.Overwrite).saveAsTable("simple_range")

// Verify that table is saved and it is marked as persistent ("isTemporary" value should be "false")
spark.catalog.listTables.where($"name" startsWith "s").show(false)

Parquet Files

Parquet is a columnar format that is supported by many other data processing systems. Spark SQL provides support for both reading and writing Parquet files that automatically preserves the schema of the original data. When writing Parquet files, all columns are automatically converted to be nullable for compatibility reasons.

More on Parquet

Apache Parquet is a columnar storage format available to any project in the Hadoop ecosystem, regardless of the choice of data processing framework, data model or programming language. It is a more efficient way to store data frames.

// Read in the parquet file created above. Parquet files are self-describing so the schema is preserved.
// The result of loading a Parquet file is also a DataFrame.
val parquetFile = sqlContext.read.parquet("/datasets/sds/tmp/platforms.parquet")

// Parquet files can also be registered as tables and then used in SQL statements.
parquetFile.createOrReplaceTempView("parquetFile")
val platforms = sqlContext.sql("SELECT platform FROM parquetFile WHERE visits > 0")
platforms.distinct.map(t => "Name: " + t(0)).collect().foreach(println)

Bucketing, Sorting and Partitioning

For file-based data source, it is also possible to bucket and sort or partition the output. Bucketing and sorting are applicable only to persistent tables:

val social_media_usage_DF = spark.table("social_media_usage") // DF from table

Find full example code at - https://raw.githubusercontent.com/apache/spark/master/examples/src/main/scala/org/apache/spark/examples/sql/SQLDataSourceExample.scala in the Spark repo.

Note that partitioning can be used with both save and saveAsTable when using the Dataset APIs.

partitionBy creates a directory structure as described in the Partition Discovery section. Thus, it has limited applicability to columns with high cardinality. In contrast bucketBy distributes data across a fixed number of buckets and can be used when the number of unique values is unbounded. One can use partitionBy by itself or along with `bucketBy.

social_media_usage_DF.write.mode("overwrite").parquet("/datasets/sds/tmp/social_media_usage.parquet") // write to parquet
ls /datasets/sds/tmp/social_media_usage.parquet
val social_media_usage_readFromParquet_DF = spark.read.parquet("/datasets/sds/tmp/social_media_usage.parquet") // read it back as DF
social_media_usage_readFromParquet_DF.count
social_media_usage_readFromParquet_DF.rdd.getNumPartitions
social_media_usage_readFromParquet_DF.printSchema
social_media_usage_readFromParquet_DF.select("platform").distinct.count
social_media_usage_readFromParquet_DF
  .write
  .partitionBy("platform") // now we are partitioning by "platform"
  .mode("overwrite").parquet("/datasets/sds/tmp/social_media_usage_partitionedByPlatform.parquet") 

Understand the directory structure of the parquet files we wrote.

There are many /platform=*/ folders inside the parquet folder.

In zeppelin use hdfs or local fs to view the same.

ls /datasets/sds/tmp/social_media_usage_partitionedByPlatform.parquet

There are part-0000*- files with contents inside each platform=* folder in the parquet folder.

ls /datasets/sds/tmp/social_media_usage_partitionedByPlatform.parquet/platform=Android
spark.read.parquet("/datasets/sds/tmp/social_media_usage_partitionedByPlatform.parquet").rdd.getNumPartitions
  • Why does partitioning by a column name matter?
  • This is a standard way to distribute the dataset into partitions according to the ideal column
    • want to make sure that all partitions are roughly of the same size, otherwise we have to wait for the largest partition to be processed before moving to the next stage (this is called partition skew)

Advanced Topics

We can also use a fixed number of buckets and sort by a column within each partition. Such finer control of the dataframe written as a parquet file can help with optimizing downstream operations on the dataframe.

  • https://jaceklaskowski.gitbooks.io/mastering-spark-sql/content/spark-sql-bucketing.html
  • https://spark.apache.org/docs/latest/sql-data-sources-load-save-functions.html#bucketing-sorting-and-partitioning

Partition Discovery

Table partitioning is a common optimization approach used in systems like Hive. In a partitioned table, data are usually stored in different directories, with partitioning column values encoded in the path of each partition directory. The Parquet data source is now able to discover and infer partitioning information automatically. For example, we can store all our previously used population data (from the programming guide example!) into a partitioned table using the following directory structure, with two extra columns, gender and country as partitioning columns:

    path
    └── to
        └── table
            ├── gender=male
            │   ├── ...
            │   │
            │   ├── country=US
            │   │   └── data.parquet
            │   ├── country=CN
            │   │   └── data.parquet
            │   └── ...
            └── gender=female
                ├── ...
                │
                ├── country=US
                │   └── data.parquet
                ├── country=CN
                │   └── data.parquet
                └── ...

By passing path/to/table to either SparkSession.read.parquet or SparkSession.read.load, Spark SQL will automatically extract the partitioning information from the paths. Now the schema of the returned DataFrame becomes:

    root
    |-- name: string (nullable = true)
    |-- age: long (nullable = true)
    |-- gender: string (nullable = true)
    |-- country: string (nullable = true)

Notice that the data types of the partitioning columns are automatically inferred. Currently, numeric data types and string type are supported. Sometimes users may not want to automatically infer the data types of the partitioning columns. For these use cases, the automatic type inference can be configured by spark.sql.sources.partitionColumnTypeInference.enabled, which is default to true. When type inference is disabled, string type will be used for the partitioning columns.

Starting from Spark 1.6.0, partition discovery only finds partitions under the given paths by default. For the above example, if users pass path/to/table/gender=male to either SparkSession.read.parquet or SparkSession.read.load, gender will not be considered as a partitioning column. If users need to specify the base path that partition discovery should start with, they can set basePath in the data source options. For example, when path/to/table/gender=male is the path of the data and users set basePath to path/to/table/, gender will be a partitioning column.

Schema Merging

Like ProtocolBuffer, Avro, and Thrift, Parquet also supports schema evolution. Users can start with a simple schema, and gradually add more columns to the schema as needed. In this way, users may end up with multiple Parquet files with different but mutually compatible schemas. The Parquet data source is now able to automatically detect this case and merge schemas of all these files.

Since schema merging is a relatively expensive operation, and is not a necessity in most cases, we turned it off by default starting from 1.5.0. You may enable it by:

  1. setting data source option mergeSchema to true when reading Parquet files (as shown in the examples below), or
  2. setting the global SQL option spark.sql.parquet.mergeSchema to true.
// Create a simple DataFrame, stored into a partition directory
val df1 = sc.parallelize(1 to 5).map(i => (i, i * 2)).toDF("single", "double")
df1.write.mode("overwrite").parquet("/datasets/sds/tmp/data/test_table/key=1")

// Create another DataFrame in a new partition directory, adding a new column and dropping an existing column
val df2 = sc.parallelize(6 to 10).map(i => (i, i * 3)).toDF("single", "triple")
df2.write.mode("overwrite").parquet("/datasets/sds/tmp/data/test_table/key=2")

// Read the partitioned table
val df3 = spark.read.option("mergeSchema", "true").parquet("/datasets/sds/tmp/data/test_table")
df3.printSchema()

// The final schema consists of all 3 columns in the Parquet files together
// with the partitioning column appeared in the partition directory paths.
// root
//  |-- single: integer (nullable = true)
//  |-- double: integer (nullable = true)
//  |-- triple: integer (nullable = true)
//  |-- key: integer (nullable = true))
df3.show

Hive metastore Parquet table conversion

When reading from and writing to Hive metastore Parquet tables, Spark SQL will try to use its own Parquet support instead of Hive SerDe for better performance. This behavior is controlled by the spark.sql.hive.convertMetastoreParquet configuration, and is turned on by default.

Hive/Parquet Schema Reconciliation

There are two key differences between Hive and Parquet from the perspective of table schema processing.

  1. Hive is case insensitive, while Parquet is not
  2. Hive considers all columns nullable, while nullability in Parquet is significant

Due to this reason, we must reconcile Hive metastore schema with Parquet schema when converting a Hive metastore Parquet table to a Spark SQL Parquet table. The reconciliation rules are:

  1. Fields that have the same name in both schema must have the same data type regardless of nullability. The reconciled field should have the data type of the Parquet side, so that nullability is respected.
  2. The reconciled schema contains exactly those fields defined in Hive metastore schema.
  • Any fields that only appear in the Parquet schema are dropped in the reconciled schema.
  • Any fileds that only appear in the Hive metastore schema are added as nullable field in the reconciled schema.

Metadata Refreshing

Spark SQL caches Parquet metadata for better performance. When Hive metastore Parquet table conversion is enabled, metadata of those converted tables are also cached. If these tables are updated by Hive or other external tools, you need to refresh them manually to ensure consistent metadata.

// should refresh table metadata
spark.catalog.refreshTable("simple_range")
-- Or you can use SQL to refresh table
REFRESH TABLE simple_range;

Configuration

Configuration of Parquet can be done using the setConf method on SQLContext or by running SET key=value commands using SQL.

Property NameDefaultMeaning
spark.sql.parquet.binaryAsStringfalseSome other Parquet-producing systems, in particular Impala, Hive, and older versions of Spark SQL, do not differentiate between binary data and strings when writing out the Parquet schema. This flag tells Spark SQL to interpret binary data as a string to provide compatibility with these systems.
spark.sql.parquet.int96AsTimestamptrueSome Parquet-producing systems, in particular Impala and Hive, store Timestamp into INT96. This flag tells Spark SQL to interpret INT96 data as a timestamp to provide compatibility with these systems.
spark.sql.parquet.cacheMetadatatrueTurns on caching of Parquet schema metadata. Can speed up querying of static data.
spark.sql.parquet.compression.codecgzipSets the compression codec use when writing Parquet files. Acceptable values include: uncompressed, snappy, gzip, lzo.
spark.sql.parquet.filterPushdowntrueEnables Parquet filter push-down optimization when set to true.
spark.sql.hive.convertMetastoreParquettrueWhen set to false, Spark SQL will use the Hive SerDe for parquet tables instead of the built in support.
spark.sql.parquet.output.committer.classorg.apache.parquet.hadoop.ParquetOutputCommitterThe output committer class used by Parquet. The specified class needs to be a subclass of org.apache.hadoop.mapreduce.OutputCommitter. Typically, it's also a subclass of org.apache.parquet.hadoop.ParquetOutputCommitter. Spark SQL comes with a builtin org.apache.spark.sql.parquet.DirectParquetOutputCommitter, which can be more efficient then the default Parquet output committer when writing data to S3.
spark.sql.parquet.mergeSchemafalseWhen true, the Parquet data source merges schemas collected from all data files, otherwise the schema is picked from the summary file or a random data file if no summary file is available.

JSON Datasets

Spark SQL can automatically infer the schema of a JSON dataset and load it as a DataFrame. This conversion can be done using SparkSession.read.json() on either an RDD of String, or a JSON file.

Note that the file that is offered as a json file is not a typical JSON file. Each line must contain a separate, self-contained valid JSON object. As a consequence, a regular multi-line JSON file will most often fail.

// A JSON dataset is pointed to by path.
// The path can be either a single text file or a directory storing text files.
val path = "/datasets/sds/tmp/platforms.json"
val platforms = spark.read.json(path)

// The inferred schema can be visualized using the printSchema() method.
platforms.printSchema()
// root
//  |-- platform: string (nullable = true)
//  |-- visits: long (nullable = true)

// Register this DataFrame as a table.
platforms.createOrReplaceTempView("platforms")

// SQL statements can be run by using the sql methods provided by sqlContext.
val facebook = spark.sql("SELECT platform, visits FROM platforms WHERE platform like 'Face%k'")
facebook.show()

// Alternatively, a DataFrame can be created for a JSON dataset represented by
// an RDD[String] storing one JSON object per string.
val rdd = sc.parallelize("""{"name":"IWyn","address":{"city":"Columbus","state":"Ohio"}}""" :: Nil)
val anotherPlatforms = spark.read.json(rdd)
anotherPlatforms.show()

Hive Tables

Spark SQL also supports reading and writing data stored in Apache Hive. However, since Hive has a large number of dependencies, it is not included in the default Spark assembly. Hive support is enabled by adding the -Phive and -Phive-thriftserver flags to Spark’s build. This command builds a new assembly jar that includes Hive. Note that this Hive assembly jar must also be present on all of the worker nodes, as they will need access to the Hive serialization and deserialization libraries (SerDes) in order to access data stored in Hive.

Configuration of Hive is done by placing your hive-site.xml, core-site.xml (for security configuration), hdfs-site.xml (for HDFS configuration) file in conf/. Please note when running the query on a YARN cluster (cluster mode), the datanucleus jars under the lib_managed/jars directory and hive-site.xml under conf/ directory need to be available on the driver and all executors launched by the YARN cluster. The convenient way to do this is adding them through the --jars option and --file option of the spark-submit command.

When working with Hive one must construct a HiveContext, which inherits from SQLContext, and adds support for finding tables in the MetaStore and writing queries using HiveQL. Users who do not have an existing Hive deployment can still create a HiveContext. When not configured by the hive-site.xml, the context automatically creates metastore_db in the current directory and creates warehouse directory indicated by HiveConf, which defaults to /user/hive/warehouse. Note that you may need to grant write privilege on /user/hive/warehouse to the user who starts the spark application.

val spark = SparkSession.builder.enableHiveSupport().getOrCreate()

spark.sql("CREATE TABLE IF NOT EXISTS src (key INT, value STRING)")
spark.sql("LOAD DATA LOCAL INPATH 'examples/src/main/resources/kv1.txt' INTO TABLE src")

// Queries are expressed in HiveQL
spark.sql("FROM src SELECT key, value").collect().foreach(println)

Interacting with Different Versions of Hive Metastore

One of the most important pieces of Spark SQL’s Hive support is interaction with Hive metastore, which enables Spark SQL to access metadata of Hive tables. Starting from Spark 1.4.0, a single binary build of Spark SQL can be used to query different versions of Hive metastores, using the configuration described below. Note that independent of the version of Hive that is being used to talk to the metastore, internally Spark SQL will compile against Hive 1.2.1 and use those classes for internal execution (serdes, UDFs, UDAFs, etc).

The following options can be used to configure the version of Hive that is used to retrieve metadata:

Property NameDefaultMeaning
spark.sql.hive.metastore.version1.2.1Version of the Hive metastore. Available options are 0.12.0 through 1.2.1.
spark.sql.hive.metastore.jarsbuiltinLocation of the jars that should be used to instantiate the HiveMetastoreClient. This property can be one of three options: builtin, maven, a classpath in the standard format for the JVM. This classpath must include all of Hive and its dependencies, including the correct version of Hadoop. These jars only need to be present on the driver, but if you are running in yarn cluster mode then you must ensure they are packaged with you application.
spark.sql.hive.metastore.sharedPrefixescom.mysql.jdbc,org.postgresql,com.microsoft.sqlserver,oracle.jdbcA comma separated list of class prefixes that should be loaded using the classloader that is shared between Spark SQL and a specific version of Hive. An example of classes that should be shared is JDBC drivers that are needed to talk to the metastore. Other classes that need to be shared are those that interact with classes that are already shared. For example, custom appenders that are used by log4j.
spark.sql.hive.metastore.barrierPrefixes(empty)A comma separated list of class prefixes that should explicitly be reloaded for each version of Hive that Spark SQL is communicating with. For example, Hive UDFs that are declared in a prefix that typically would be shared (i.e. org.apache.spark.*).

JDBC To Other Databases

Spark SQL also includes a data source that can read data from other databases using JDBC. This functionality should be preferred over using JdbcRDD. This is because the results are returned as a DataFrame and they can easily be processed in Spark SQL or joined with other data sources. The JDBC data source is also easier to use from Java or Python as it does not require the user to provide a ClassTag. (Note that this is different than the Spark SQL JDBC server, which allows other applications to run queries using Spark SQL).

To get started you will need to include the JDBC driver for you particular database on the spark classpath. For example, to connect to postgres from the Spark Shell you would run the following command:

SPARK_CLASSPATH=postgresql-9.3-1102-jdbc41.jar bin/spark-shell

Tables from the remote database can be loaded as a DataFrame or Spark SQL Temporary table using the Data Sources API. The following options are supported:

Property NameMeaning
urlThe JDBC URL to connect to.
dbtableThe JDBC table that should be read. Note that anything that is valid in a FROM clause of a SQL query can be used. For example, instead of a full table you could also use a subquery in parentheses.
driverThe class name of the JDBC driver needed to connect to this URL. This class will be loaded on the master and workers before running an JDBC commands to allow the driver to register itself with the JDBC subsystem.
partitionColumn, lowerBound, upperBound, numPartitionsThese options must all be specified if any of them is specified. They describe how to partition the table when reading in parallel from multiple workers. partitionColumn must be a numeric column from the table in question. Notice that lowerBound and upperBound are just used to decide the partition stride, not for filtering the rows in table. So all rows in the table will be partitioned and returned.
fetchSizeThe JDBC fetch size, which determines how many rows to fetch per round trip. This can help performance on JDBC drivers which default to low fetch size (eg. Oracle with 10 rows).
// Example of using JDBC datasource
val jdbcDF = spark.read.format("jdbc").options(Map("url" -> "jdbc:postgresql:dbserver", "dbtable" -> "schema.tablename")).load()

-- Or using JDBC datasource in SQL
CREATE TEMPORARY TABLE jdbcTable
USING org.apache.spark.sql.jdbc
OPTIONS (
  url "jdbc:postgresql:dbserver",
  dbtable "schema.tablename"
)

Troubleshooting

  • The JDBC driver class must be visible to the primordial class loader on the client session and on all executors. This is because Java’s DriverManager class does a security check that results in it ignoring all drivers not visible to the primordial class loader when one goes to open a connection. One convenient way to do this is to modify compute_classpath.sh on all worker nodes to include your driver JARs.
  • Some databases, such as H2, convert all names to upper case. You’ll need to use upper case to refer to those names in Spark SQL.

ScaDaMaLe Course site and book

Performance Tuning

Spark Sql Programming Guide

If you have read the spark-SQL paper and have some idea of how distributed sorting and joining work then you will need to know the following part of the programming guide to tune the performance of Spark SQL queries:

Normally, you only need to get into the internals of optimizing Spark when Spark's automatic optimizations fail, which can happen from time to time.

ScaDaMaLe Course site and book

This is an elaboration of the http://spark.apache.org/docs/latest/sql-programming-guide.html by Ivan Sadikov and Raazesh Sainudiin.

Distributed SQL Engine

Spark Sql Programming Guide

  • Distributed SQL Engine
    • Running the Thrift JDBC/ODBC server
    • Running the Spark SQL CLI

Distributed SQL Engine

Spark SQL can also act as a distributed query engine using its JDBC/ODBC or command-line interface. In this mode, end-users or applications can interact with Spark SQL directly to run SQL queries, without the need to write any code.

Running the Thrift JDBC/ODBC server

The Thrift JDBC/ODBC server implemented here corresponds to the HiveServer2 in Hive 1.2.1 You can test the JDBC server with the beeline script that comes with either Spark or Hive 1.2.1.

To start the JDBC/ODBC server, run the following in the Spark directory:

./sbin/start-thriftserver.sh

This script accepts all bin/spark-submit command line options, plus a --hiveconf option to specify Hive properties. You may run ./sbin/start-thriftserver.sh --help for a complete list of all available options. By default, the server listens on localhost:10000. You may override this behaviour via either environment variables, i.e.:

export HIVE_SERVER2_THRIFT_PORT=<listening-port>
export HIVE_SERVER2_THRIFT_BIND_HOST=<listening-host>
./sbin/start-thriftserver.sh \
  --master <master-uri> \
  ...

or system properties:

./sbin/start-thriftserver.sh \
  --hiveconf hive.server2.thrift.port=<listening-port> \
  --hiveconf hive.server2.thrift.bind.host=<listening-host> \
  --master <master-uri>
  ...

Now you can use beeline to test the Thrift JDBC/ODBC server:

./bin/beeline

Connect to the JDBC/ODBC server in beeline with:

beeline> !connect jdbc:hive2://localhost:10000

Beeline will ask you for a username and password. In non-secure mode, simply enter the username on your machine and a blank password. For secure mode, please follow the instructions given in the beeline documentation.

Configuration of Hive is done by placing your hive-site.xml, core-site.xml and hdfs-site.xml files in conf/.

You may also use the beeline script that comes with Hive.

Thrift JDBC server also supports sending thrift RPC messages over HTTP transport. Use the following setting to enable HTTP mode as system property or in hive-site.xml file in conf/:

hive.server2.transport.mode - Set this to value: http
hive.server2.thrift.http.port - HTTP port number fo listen on; default is 10001
hive.server2.http.endpoint - HTTP endpoint; default is cliservice

To test, use beeline to connect to the JDBC/ODBC server in http mode with:

beeline> !connect jdbc:hive2://<host>:<port>/<database>?hive.server2.transport.mode=http;hive.server2.thrift.http.path=<http_endpoint>

Running the Spark SQL CLI

The Spark SQL CLI is a convenient tool to run the Hive metastore service in local mode and execute queries input from the command line. Note that the Spark SQL CLI cannot talk to the Thrift JDBC server.

To start the Spark SQL CLI, run the following in the Spark directory:

./bin/spark-sql

Configuration of Hive is done by placing your hive-site.xml, core-site.xml and hdfs-site.xml files in conf/. You may run ./bin/spark-sql --help for a complete list of all available options.

ScaDaMaLe Course site and book

SQL Pivoting since Spark 2.4

SQL Pivot: Converting Rows to Columns

This is from the following blogpost: - https://databricks.com/blog/2018/11/01/sql-pivot-converting-rows-to-columns.html

This is a useful trick to know when having to do ETL before exploring datasets that need row to column conversions.

Load Data

Create tables and load temperature data

CREATE OR REPLACE TEMPORARY VIEW high_temps
  USING csv
  OPTIONS (path "/datasets/sds/weather/high_temps", header "true", mode "FAILFAST")
SELECT * FROM high_temps
date temp
2015-01-01 42
2015-01-02 42
2015-01-03 41
2015-01-04 51
2015-01-05 54
2015-01-06 54
2015-01-07 46
2015-01-08 46
2015-01-09 50
2015-01-10 46
2015-01-11 49
2015-01-12 52
2015-01-13 49
2015-01-14 43
2015-01-15 46
2015-01-16 53
2015-01-17 56
2015-01-18 57
2015-01-19 50
2015-01-20 50
2015-01-21 45
2015-01-22 49
2015-01-23 54
2015-01-24 58
2015-01-25 63
2015-01-26 61
2015-01-27 52
2015-01-28 54
2015-01-29 54
2015-01-30 47
2015-01-31 45
2015-02-01 49
2015-02-02 52
2015-02-03 50
2015-02-04 51
2015-02-05 56
2015-02-06 58
2015-02-07 54
2015-02-08 59
2015-02-09 56
2015-02-10 55
2015-02-11 55
2015-02-12 62
2015-02-13 60
2015-02-14 58
2015-02-15 54
2015-02-16 59
2015-02-17 61
2015-02-18 54
2015-02-19 51
2015-02-20 52
2015-02-21 54
2015-02-22 53
2015-02-23 55
2015-02-24 52
2015-02-25 50
2015-02-26 53
2015-02-27 50
2015-02-28 54
2015-03-01 52
2015-03-02 52
2015-03-03 51
2015-03-04 55
2015-03-05 56
2015-03-06 59
2015-03-07 62
2015-03-08 63
2015-03-09 58
2015-03-10 56
2015-03-11 58
2015-03-12 64
2015-03-13 63
2015-03-14 57
2015-03-15 51
2015-03-16 57
2015-03-17 56
2015-03-18 60
2015-03-19 60
2015-03-20 57
2015-03-21 56
2015-03-22 53
2015-03-23 52
2015-03-24 55
2015-03-25 58
2015-03-26 69
2015-03-27 65
2015-03-28 60
2015-03-29 60
2015-03-30 64
2015-03-31 55
2015-04-01 55
2015-04-02 56
2015-04-03 52
2015-04-04 55
2015-04-05 62
2015-04-06 57
2015-04-07 58
2015-04-08 63
2015-04-09 63
2015-04-10 57
2015-04-11 53
2015-04-12 56
2015-04-13 53
2015-04-14 53
2015-04-15 57
2015-04-16 64
2015-04-17 66
2015-04-18 66
2015-04-19 70
2015-04-20 73
2015-04-21 63
2015-04-22 60
2015-04-23 54
2015-04-24 54
2015-04-25 56
2015-04-26 60
2015-04-27 77
2015-04-28 60
2015-04-29 61
2015-04-30 63
2015-05-01 65
2015-05-02 65
2015-05-03 69
2015-05-04 63
2015-05-05 58
2015-05-06 62
2015-05-07 69
2015-05-08 75
2015-05-09 80
2015-05-10 67
2015-05-11 57
2015-05-12 60
2015-05-13 54
2015-05-14 64
2015-05-15 68
2015-05-16 60
2015-05-17 67
2015-05-18 78
2015-05-19 71
2015-05-20 74
2015-05-21 78
2015-05-22 62
2015-05-23 61
2015-05-24 64
2015-05-25 60
2015-05-26 71
2015-05-27 76
2015-05-28 82
2015-05-29 79
2015-05-30 73
2015-05-31 77
2015-06-01 61
2015-06-02 64
2015-06-03 68
2015-06-04 73
2015-06-05 80
2015-06-06 85
2015-06-07 88
2015-06-08 87
2015-06-09 84
2015-06-10 78
2015-06-11 76
2015-06-12 68
2015-06-13 75
2015-06-14 82
2015-06-15 86
2015-06-16 73
2015-06-17 77
2015-06-18 76
2015-06-19 75
2015-06-20 77
2015-06-21 78
2015-06-22 77
2015-06-23 79
2015-06-24 78
2015-06-25 87
2015-06-26 89
2015-06-27 92
2015-06-28 83
2015-06-29 84
2015-06-30 87
2015-07-01 90
2015-07-02 93
2015-07-03 92
2015-07-04 92
2015-07-05 91
2015-07-06 85
2015-07-07 81
2015-07-08 86
2015-07-09 84
2015-07-10 70
2015-07-11 72
2015-07-12 79
2015-07-13 78
2015-07-14 82
2015-07-15 79
2015-07-16 79
2015-07-17 82
2015-07-18 92
2015-07-19 95
2015-07-20 80
2015-07-21 75
2015-07-22 75
2015-07-23 79
2015-07-24 73
2015-07-25 70
2015-07-26 72
2015-07-27 74
2015-07-28 82
2015-07-29 90
2015-07-30 94
2015-07-31 94
2015-08-01 92
2015-08-02 87
2015-08-03 83
2015-08-04 79
2015-08-05 74
2015-08-06 77
2015-08-07 83
2015-08-08 77
2015-08-09 83
2015-08-10 84
2015-08-11 86
2015-08-12 83
2015-08-13 83
2015-08-14 65
2015-08-15 71
2015-08-16 77
2015-08-17 81
2015-08-18 86
2015-08-19 89
2015-08-20 73
2015-08-21 72
2015-08-22 80
2015-08-23 82
2015-08-24 75
2015-08-25 78
2015-08-26 83
2015-08-27 85
2015-08-28 74
2015-08-29 72
2015-08-30 68
2015-08-31 66
2015-09-01 67
2015-09-02 67
2015-09-03 65
2015-09-04 65
2015-09-05 69
2015-09-06 61
2015-09-07 70
2015-09-08 73
2015-09-09 76
2015-09-10 77
2015-09-11 81
2015-09-12 80
2015-09-13 69
2015-09-14 62
2015-09-15 64
2015-09-16 68
2015-09-17 65
2015-09-18 67
2015-09-19 70
2015-09-20 73
2015-09-21 65
2015-09-22 66
2015-09-23 69
2015-09-24 72
2015-09-25 60
2015-09-26 65
2015-09-27 64
2015-09-28 70
2015-09-29 71
2015-09-30 65
2015-10-01 70
2015-10-02 60
2015-10-03 67
2015-10-04 73
2015-10-05 74
2015-10-06 65
2015-10-07 61
2015-10-08 66
2015-10-09 67
2015-10-10 70
2015-10-11 64
2015-10-12 65
2015-10-13 62
2015-10-14 59
2015-10-15 70
2015-10-16 68
2015-10-17 67
2015-10-18 59
2015-10-19 63
2015-10-20 64
2015-10-21 61
2015-10-22 61
2015-10-23 55
2015-10-24 59
2015-10-25 67
2015-10-26 54
2015-10-27 61
2015-10-28 57
2015-10-29 59
2015-10-30 63
2015-10-31 60
2015-11-01 54
2015-11-02 52
2015-11-03 51
2015-11-04 50
2015-11-05 53
2015-11-06 60
2015-11-07 54
2015-11-08 52
2015-11-09 50
2015-11-10 52
2015-11-11 52
2015-11-12 52
2015-11-13 56
2015-11-14 49
2015-11-15 48
2015-11-16 48
2015-11-17 56
2015-11-18 48
2015-11-19 48
2015-11-20 47
2015-11-21 48
2015-11-22 50
2015-11-23 44
2015-11-24 44
2015-11-25 45
2015-11-26 49
2015-11-27 49
2015-11-28 45
2015-11-29 35
2015-11-30 42
2015-12-01 50
2015-12-02 51
2015-12-03 60
2015-12-04 51
2015-12-05 50
2015-12-06 55
2015-12-07 52
2015-12-08 60
2015-12-09 54
2015-12-10 53
2015-12-11 49
2015-12-12 48
2015-12-13 46
2015-12-14 46
2015-12-15 44
2015-12-16 43
2015-12-17 44
2015-12-18 48
2015-12-19 47
2015-12-20 46
2015-12-21 42
2015-12-22 46
2015-12-23 41
2015-12-24 42
2015-12-25 41
2015-12-26 40
2015-12-27 40
2015-12-28 41
2015-12-29 45
2015-12-30 42
2015-12-31 42
2016-01-01 46
2016-01-02 42
2016-01-03 40
2016-01-04 38
2016-01-05 46
2016-01-06 53
2016-01-07 44
2016-01-08 48
2016-01-09 49
2016-01-10 52
2016-01-11 47
2016-01-12 48
2016-01-13 54
2016-01-14 48
2016-01-15 47
2016-01-16 52
2016-01-17 49
2016-01-18 52
2016-01-19 47
2016-01-20 50
2016-01-21 53
2016-01-22 55
2016-01-23 46
2016-01-24 50
2016-01-25 55
2016-01-26 56
2016-01-27 58
2016-01-28 56
2016-01-29 49
2016-01-30 46
2016-01-31 45
2016-02-01 45
2016-02-02 50
2016-02-03 47
2016-02-04 49
2016-02-05 52
2016-02-06 48
2016-02-07 52
2016-02-08 59
2016-02-09 63
2016-02-10 56
2016-02-11 55
2016-02-12 57
2016-02-13 50
2016-02-14 53
2016-02-15 54
2016-02-16 54
2016-02-17 63
2016-02-18 51
2016-02-19 50
2016-02-20 51
2016-02-21 50
2016-02-22 51
2016-02-23 58
2016-02-24 57
2016-02-25 62
2016-02-26 57
2016-02-27 56
2016-02-28 50
2016-02-29 53
2016-03-01 57
2016-03-02 54
2016-03-03 58
2016-03-04 55
2016-03-05 64
2016-03-06 59
2016-03-07 51
2016-03-08 53
2016-03-09 55
2016-03-10 55
2016-03-11 60
2016-03-12 50
2016-03-13 54
2016-03-14 47
2016-03-15 50
2016-03-16 53
2016-03-17 57
2016-03-18 63
2016-03-19 61
2016-03-20 54
2016-03-21 55
2016-03-22 55
2016-03-23 51
2016-03-24 52
2016-03-25 56
2016-03-26 58
2016-03-27 55
2016-03-28 55
2016-03-29 63
2016-03-30 68
2016-03-31 71
2016-04-01 71
2016-04-02 62
2016-04-03 68
2016-04-04 58
2016-04-05 56
2016-04-06 69
2016-04-07 78
2016-04-08 76
2016-04-09 64
2016-04-10 58
2016-04-11 58
2016-04-12 57
2016-04-13 58
2016-04-14 59
2016-04-15 59
2016-04-16 67
2016-04-17 80
2016-04-18 89
2016-04-19 84
2016-04-20 81
2016-04-21 72
2016-04-22 62
2016-04-23 64
2016-04-24 55
2016-04-25 59
2016-04-26 59
2016-04-27 63
2016-04-28 63
2016-04-29 61
2016-04-30 67
2016-05-01 79
2016-05-02 87
2016-05-03 72
2016-05-04 60
2016-05-05 68
2016-05-06 77
2016-05-07 82
2016-05-08 66
2016-05-09 67
2016-05-10 75
2016-05-11 80
2016-05-12 76
2016-05-13 84
2016-05-14 58
2016-05-15 56
2016-05-16 59
2016-05-17 69
2016-05-18 65
2016-05-19 63
2016-05-20 66
2016-05-21 58
2016-05-22 66
2016-05-23 66
2016-05-24 66
2016-05-25 65
2016-05-26 62
2016-05-27 64
2016-05-28 62
2016-05-29 64
2016-05-30 72
2016-05-31 80
2016-06-01 74
2016-06-02 70
2016-06-03 82
2016-06-04 85
2016-06-05 93
2016-06-06 88
2016-06-07 84
2016-06-08 67
2016-06-09 64
2016-06-10 65
2016-06-11 67
2016-06-12 70
2016-06-13 65
2016-06-14 60
2016-06-15 66
2016-06-16 70
2016-06-17 69
2016-06-18 66
2016-06-19 72
2016-06-20 75
2016-06-21 71
2016-06-22 75
2016-06-23 68
2016-06-24 69
2016-06-25 72
2016-06-26 80
2016-06-27 85
2016-06-28 76
2016-06-29 72
2016-06-30 73
2016-07-01 75
2016-07-02 75
2016-07-03 71
2016-07-04 69
2016-07-05 67
2016-07-06 76
2016-07-07 68
2016-07-08 72
2016-07-09 71
2016-07-10 73
2016-07-11 73
2016-07-12 76
2016-07-13 74
2016-07-14 76
2016-07-15 76
2016-07-16 73
2016-07-17 79
2016-07-18 71
2016-07-19 77
2016-07-20 80
2016-07-21 84
2016-07-22 73
2016-07-23 73
2016-07-24 81
2016-07-25 86
2016-07-26 78
2016-07-27 84
2016-07-28 88
2016-07-29 89
2016-07-30 75
2016-07-31 72
2016-08-01 77
2016-08-02 71
2016-08-03 76
2016-08-04 82
2016-08-05 79
2016-08-06 71
2016-08-07 69
2016-08-08 73
2016-08-09 71
2016-08-10 76
2016-08-11 81
2016-08-12 90
2016-08-13 91
2016-08-14 85
2016-08-15 83
2016-08-16 83
2016-08-17 80
2016-08-18 87
2016-08-19 95
2016-08-20 91
2016-08-21 73
2016-08-22 72
2016-08-23 80
2016-08-24 85
2016-08-25 91
2016-08-26 92
2016-08-27 73
2016-08-28 70
2016-08-29 79
2016-08-30 71
2016-08-31 69
2016-09-01 66
2016-09-02 69
2016-09-03 69
2016-09-04 70
2016-09-05 64
2016-09-06 65
2016-09-07 66
2016-09-08 70
2016-09-09 73
2016-09-10 77
2016-09-11 70
2016-09-12 75
2016-09-13 77
2016-09-14 78
2016-09-15 72
2016-09-16 74
2016-09-17 67
2016-09-18 68
2016-09-19 68
2016-09-20 67
2016-09-21 68
2016-09-22 65
2016-09-23 63
2016-09-24 66
2016-09-25 76
2016-09-26 77
2016-09-27 69
2016-09-28 67
2016-09-29 66
2016-09-30 65
2016-10-01 63
2016-10-02 63
2016-10-03 60
2016-10-04 60
2016-10-05 65
2016-10-06 62
2016-10-07 63
2016-10-08 65
2016-10-09 56
2016-10-10 60
2016-10-11 62
2016-10-12 66
2016-10-13 59
2016-10-14 57
2016-10-15 62
2016-10-16 60
2016-10-17 59
2016-10-18 58
2016-10-19 58
2016-10-20 60
2016-10-21 57
2016-10-22 62
2016-10-23 62
2016-10-24 63
2016-10-25 63
2016-10-26 58
2016-10-27 58
2016-10-28 67
2016-10-29 60
2016-10-30 52
2016-10-31 57
2016-11-01 56
2016-11-02 56
2016-11-03 65
2016-11-04 63
2016-11-05 60
2016-11-06 60
2016-11-07 66
2016-11-08 70
2016-11-09 64
2016-11-10 62
2016-11-11 61
2016-11-12 58
2016-11-13 54
2016-11-14 57
2016-11-15 54
2016-11-16 50
2016-11-17 51
2016-11-18 54
2016-11-19 57
2016-11-20 57
2016-11-21 53
2016-11-22 53
2016-11-23 48
2016-11-24 51
2016-11-25 52
2016-11-26 51
2016-11-27 47
2016-11-28 50
2016-11-29 51
2016-11-30 50
2016-12-01 46
2016-12-02 49
2016-12-03 48
2016-12-04 45
2016-12-05 39
2016-12-06 40
2016-12-07 38
2016-12-08 38
2016-12-09 36
2016-12-10 44
2016-12-11 42
2016-12-12 42
2016-12-13 38
2016-12-14 39
2016-12-15 38
2016-12-16 34
2016-12-17 33
2016-12-18 39
2016-12-19 45
2016-12-20 50
2016-12-21 47
2016-12-22 44
2016-12-23 39
2016-12-24 40
2016-12-25 42
2016-12-26 42
2016-12-27 44
2016-12-28 47
2016-12-29 48
2016-12-30 45
2016-12-31 38
2017-01-01 37
2017-01-02 34
2017-01-03 33
2017-01-04 36
2017-01-05 35
2017-01-06 40
2017-01-07 37
2017-01-08 45
2017-01-09 42
2017-01-10 40
2017-01-11 34
2017-01-12 40
2017-01-13 37
2017-01-14 44
2017-01-15 43
2017-01-16 45
2017-01-17 49
2017-01-18 53
2017-01-19 50
2017-01-20 52
2017-01-21 54
2017-01-22 49
2017-01-23 48
2017-01-24 43
2017-01-25 44
2017-01-26 47
2017-01-27 54
2017-01-28 52
2017-01-29 48
2017-01-30 45
2017-01-31 44
2017-02-01 43
2017-02-02 44
2017-02-03 40
2017-02-04 46
2017-02-05 41
2017-02-06 37
2017-02-07 38
2017-02-08 40
2017-02-09 56
2017-02-10 48
2017-02-11 49
2017-02-12 51
2017-02-13 57
2017-02-14 58
2017-02-15 54
2017-02-16 53
2017-02-17 56
2017-02-18 45
2017-02-19 47
2017-02-20 45
2017-02-21 48
2017-02-22 46
2017-02-23 44
2017-02-24 47
2017-02-25 44
2017-02-26 41
2017-02-27 41
2017-02-28 47
2017-03-01 49
2017-03-02 47
2017-03-03 51
2017-03-04 46
2017-03-05 45
2017-03-06 43
2017-03-07 42
2017-03-08 43
2017-03-09 49
2017-03-10 52
2017-03-11 53
2017-03-12 53
2017-03-13 53
2017-03-14 57
2017-03-15 50
2017-03-16 52
2017-03-17 53
2017-03-18 57
2017-03-19 53
2017-03-20 54
2017-03-21 56
2017-03-22 56
2017-03-23 58
2017-03-24 54
2017-03-25 52
2017-03-26 49
2017-03-27 53
2017-03-28 53
2017-03-29 55
2017-03-30 54
2017-03-31 56
2017-04-01 56
2017-04-02 55
2017-04-03 53
2017-04-04 61
2017-04-05 54
2017-04-06 60
2017-04-07 59
2017-04-08 52
2017-04-09 56
2017-04-10 53
2017-04-11 57
2017-04-12 59
2017-04-13 54
2017-04-14 52
2017-04-15 57
2017-04-16 66
2017-04-17 60
2017-04-18 60
2017-04-19 56
2017-04-20 58
2017-04-21 66
2017-04-22 63
2017-04-23 53
2017-04-24 57
2017-04-25 56
2017-04-26 58
2017-04-27 56
2017-04-28 59
2017-04-29 57
2017-04-30 56
2017-05-01 50
2017-05-02 61
2017-05-03 74
2017-05-04 76
2017-05-05 59
2017-05-06 60
2017-05-07 62
2017-05-08 64
2017-05-09 70
2017-05-10 71
2017-05-11 60
2017-05-12 54
2017-05-13 57
2017-05-14 59
2017-05-15 52
2017-05-16 54
2017-05-17 60
2017-05-18 64
2017-05-19 71
2017-05-20 72
2017-05-21 76
2017-05-22 83
2017-05-23 78
2017-05-24 63
2017-05-25 70
2017-05-26 81
2017-05-27 86
2017-05-28 86
2017-05-29 74
2017-05-30 58
2017-05-31 72
2017-06-01 67
2017-06-02 73
2017-06-03 67
2017-06-04 67
2017-06-05 74
2017-06-06 84
2017-06-07 79
2017-06-08 63
2017-06-09 63
2017-06-10 67
2017-06-11 70
2017-06-12 64
2017-06-13 65
2017-06-14 65
2017-06-15 61
2017-06-16 68
2017-06-17 63
2017-06-18 71
2017-06-19 74
2017-06-20 73
2017-06-21 69
2017-06-22 75
2017-06-23 81
2017-06-24 89
2017-06-25 96
2017-06-26 72
2017-06-27 71
2017-06-28 68
2017-06-29 78
2017-06-30 86
2017-07-01 72
2017-07-02 77
2017-07-03 70
2017-07-04 78
2017-07-05 85
2017-07-06 85
2017-07-07 72
2017-07-08 81
2017-07-09 80
2017-07-10 73
2017-07-11 75
2017-07-12 78
2017-07-13 75
2017-07-14 79
2017-07-15 77
2017-07-16 72
2017-07-17 78
2017-07-18 82
2017-07-19 78
2017-07-20 74
2017-07-21 78
2017-07-22 85
2017-07-23 76
2017-07-24 82
2017-07-25 87
2017-07-26 84
2017-07-27 70
2017-07-28 79
2017-07-29 81
2017-07-30 80
2017-07-31 85
2017-08-01 87
2017-08-02 91
2017-08-03 94
2017-08-04 91
2017-08-05 82
2017-08-06 82
2017-08-07 84
2017-08-08 89
2017-08-09 91
2017-08-10 90
2017-08-11 77
2017-08-12 76
2017-08-13 74
2017-08-14 73
2017-08-15 78
2017-08-16 79
2017-08-17 75
2017-08-18 73
2017-08-19 76
2017-08-20 77
2017-08-21 83
2017-08-22 86
2017-08-23 75
2017-08-24 72
2017-08-25 76
2017-08-26 84
2017-08-27 87
2017-08-28 88
2017-08-29 85
2017-08-30 77
2017-08-31 75
2017-09-01 85
2017-09-02 90
2017-09-03 90
2017-09-04 88
2017-09-05 85
2017-09-06 81
2017-09-07 75
2017-09-08 71
2017-09-09 68
2017-09-10 73
2017-09-11 80
2017-09-12 77
2017-09-13 70
2017-09-14 74
2017-09-15 76
2017-09-16 76
2017-09-17 64
2017-09-18 62
2017-09-19 64
2017-09-20 57
2017-09-21 64
2017-09-22 68
2017-09-23 68
2017-09-24 72
2017-09-25 65
2017-09-26 74
CREATE OR REPLACE TEMPORARY VIEW low_temps
  USING csv
  OPTIONS (path "/datasets/sds/weather/low_temps", header "true", mode "FAILFAST")
SELECT * FROM low_temps
date temp
2015-01-01 26
2015-01-02 32
2015-01-03 35
2015-01-04 38
2015-01-05 49
2015-01-06 43
2015-01-07 42
2015-01-08 35
2015-01-09 38
2015-01-10 43
2015-01-11 45
2015-01-12 40
2015-01-13 37
2015-01-14 33
2015-01-15 34
2015-01-16 42
2015-01-17 38
2015-01-18 45
2015-01-19 43
2015-01-20 38
2015-01-21 31
2015-01-22 43
2015-01-23 47
2015-01-24 52
2015-01-25 45
2015-01-26 43
2015-01-27 47
2015-01-28 41
2015-01-29 38
2015-01-30 34
2015-01-31 38
2015-02-01 40
2015-02-02 41
2015-02-03 42
2015-02-04 40
2015-02-05 47
2015-02-06 50
2015-02-07 49
2015-02-08 47
2015-02-09 47
2015-02-10 47
2015-02-11 42
2015-02-12 49
2015-02-13 44
2015-02-14 44
2015-02-15 39
2015-02-16 42
2015-02-17 40
2015-02-18 40
2015-02-19 47
2015-02-20 45
2015-02-21 42
2015-02-22 38
2015-02-23 33
2015-02-24 36
2015-02-25 44
2015-02-26 46
2015-02-27 44
2015-02-28 38
2015-03-01 34
2015-03-02 40
2015-03-03 32
2015-03-04 31
2015-03-05 37
2015-03-06 38
2015-03-07 39
2015-03-08 39
2015-03-09 40
2015-03-10 41
2015-03-11 48
2015-03-12 49
2015-03-13 46
2015-03-14 49
2015-03-15 43
2015-03-16 43
2015-03-17 40
2015-03-18 45
2015-03-19 47
2015-03-20 48
2015-03-21 47
2015-03-22 43
2015-03-23 42
2015-03-24 43
2015-03-25 45
2015-03-26 50
2015-03-27 48
2015-03-28 49
2015-03-29 48
2015-03-30 51
2015-03-31 43
2015-04-01 42
2015-04-02 42
2015-04-03 41
2015-04-04 39
2015-04-05 37
2015-04-06 44
2015-04-07 44
2015-04-08 43
2015-04-09 43
2015-04-10 46
2015-04-11 42
2015-04-12 42
2015-04-13 39
2015-04-14 37
2015-04-15 38
2015-04-16 39
2015-04-17 43
2015-04-18 47
2015-04-19 47
2015-04-20 46
2015-04-21 44
2015-04-22 41
2015-04-23 44
2015-04-24 43
2015-04-25 42
2015-04-26 40
2015-04-27 51
2015-04-28 48
2015-04-29 45
2015-04-30 46
2015-05-01 48
2015-05-02 46
2015-05-03 46
2015-05-04 45
2015-05-05 45
2015-05-06 45
2015-05-07 43
2015-05-08 47
2015-05-09 49
2015-05-10 52
2015-05-11 50
2015-05-12 51
2015-05-13 50
2015-05-14 49
2015-05-15 49
2015-05-16 52
2015-05-17 51
2015-05-18 54
2015-05-19 53
2015-05-20 51
2015-05-21 53
2015-05-22 53
2015-05-23 53
2015-05-24 52
2015-05-25 52
2015-05-26 53
2015-05-27 53
2015-05-28 54
2015-05-29 55
2015-05-30 50
2015-05-31 53
2015-06-01 53
2015-06-02 55
2015-06-03 53
2015-06-04 53
2015-06-05 55
2015-06-06 56
2015-06-07 60
2015-06-08 58
2015-06-09 58
2015-06-10 52
2015-06-11 52
2015-06-12 53
2015-06-13 49
2015-06-14 53
2015-06-15 61
2015-06-16 52
2015-06-17 52
2015-06-18 57
2015-06-19 56
2015-06-20 55
2015-06-21 57
2015-06-22 55
2015-06-23 53
2015-06-24 61
2015-06-25 60
2015-06-26 64
2015-06-27 63
2015-06-28 65
2015-06-29 63
2015-06-30 59
2015-07-01 63
2015-07-02 64
2015-07-03 64
2015-07-04 59
2015-07-05 62
2015-07-06 60
2015-07-07 57
2015-07-08 58
2015-07-09 58
2015-07-10 62
2015-07-11 62
2015-07-12 62
2015-07-13 61
2015-07-14 61
2015-07-15 58
2015-07-16 59
2015-07-17 57
2015-07-18 64
2015-07-19 63
2015-07-20 62
2015-07-21 59
2015-07-22 57
2015-07-23 58
2015-07-24 56
2015-07-25 58
2015-07-26 57
2015-07-27 54
2015-07-28 57
2015-07-29 58
2015-07-30 63
2015-07-31 64
2015-08-01 60
2015-08-02 61
2015-08-03 63
2015-08-04 58
2015-08-05 54
2015-08-06 59
2015-08-07 60
2015-08-08 60
2015-08-09 59
2015-08-10 61
2015-08-11 62
2015-08-12 62
2015-08-13 60
2015-08-14 59
2015-08-15 57
2015-08-16 58
2015-08-17 57
2015-08-18 59
2015-08-19 61
2015-08-20 58
2015-08-21 58
2015-08-22 54
2015-08-23 57
2015-08-24 54
2015-08-25 54
2015-08-26 57
2015-08-27 58
2015-08-28 60
2015-08-29 56
2015-08-30 55
2015-08-31 61
2015-09-01 57
2015-09-02 52
2015-09-03 51
2015-09-04 50
2015-09-05 48
2015-09-06 53
2015-09-07 56
2015-09-08 56
2015-09-09 57
2015-09-10 58
2015-09-11 59
2015-09-12 58
2015-09-13 55
2015-09-14 51
2015-09-15 50
2015-09-16 50
2015-09-17 55
2015-09-18 55
2015-09-19 58
2015-09-20 54
2015-09-21 49
2015-09-22 46
2015-09-23 47
2015-09-24 52
2015-09-25 55
2015-09-26 50
2015-09-27 45
2015-09-28 49
2015-09-29 48
2015-09-30 50
2015-10-01 49
2015-10-02 50
2015-10-03 52
2015-10-04 50
2015-10-05 49
2015-10-06 50
2015-10-07 57
2015-10-08 56
2015-10-09 54
2015-10-10 56
2015-10-11 51
2015-10-12 51
2015-10-13 49
2015-10-14 50
2015-10-15 49
2015-10-16 48
2015-10-17 53
2015-10-18 55
2015-10-19 54
2015-10-20 51
2015-10-21 47
2015-10-22 48
2015-10-23 45
2015-10-24 48
2015-10-25 48
2015-10-26 50
2015-10-27 46
2015-10-28 52
2015-10-29 54
2015-10-30 53
2015-10-31 53
2015-11-01 48
2015-11-02 45
2015-11-03 41
2015-11-04 38
2015-11-05 46
2015-11-06 47
2015-11-07 49
2015-11-08 46
2015-11-09 41
2015-11-10 39
2015-11-11 43
2015-11-12 41
2015-11-13 49
2015-11-14 43
2015-11-15 36
2015-11-16 35
2015-11-17 44
2015-11-18 38
2015-11-19 37
2015-11-20 33
2015-11-21 33
2015-11-22 35
2015-11-23 32
2015-11-24 37
2015-11-25 32
2015-11-26 30
2015-11-27 29
2015-11-28 27
2015-11-29 28
2015-11-30 25
2015-12-01 39
2015-12-02 40
2015-12-03 46
2015-12-04 43
2015-12-05 43
2015-12-06 45
2015-12-07 47
2015-12-08 50
2015-12-09 46
2015-12-10 43
2015-12-11 40
2015-12-12 42
2015-12-13 43
2015-12-14 35
2015-12-15 34
2015-12-16 37
2015-12-17 39
2015-12-18 40
2015-12-19 37
2015-12-20 40
2015-12-21 37
2015-12-22 37
2015-12-23 37
2015-12-24 36
2015-12-25 36
2015-12-26 32
2015-12-27 35
2015-12-28 35
2015-12-29 33
2015-12-30 30
2015-12-31 28
2016-01-01 28
2016-01-02 25
2016-01-03 31
2016-01-04 35
2016-01-05 36
2016-01-06 37
2016-01-07 34
2016-01-08 36
2016-01-09 30
2016-01-10 38
2016-01-11 35
2016-01-12 40
2016-01-13 41
2016-01-14 36
2016-01-15 33
2016-01-16 42
2016-01-17 43
2016-01-18 43
2016-01-19 42
2016-01-20 40
2016-01-21 45
2016-01-22 44
2016-01-23 43
2016-01-24 37
2016-01-25 38
2016-01-26 46
2016-01-27 49
2016-01-28 46
2016-01-29 41
2016-01-30 38
2016-01-31 37
2016-02-01 38
2016-02-02 35
2016-02-03 39
2016-02-04 42
2016-02-05 39
2016-02-06 39
2016-02-07 34
2016-02-08 39
2016-02-09 38
2016-02-10 46
2016-02-11 45
2016-02-12 47
2016-02-13 43
2016-02-14 47
2016-02-15 49
2016-02-16 48
2016-02-17 47
2016-02-18 44
2016-02-19 39
2016-02-20 38
2016-02-21 35
2016-02-22 36
2016-02-23 34
2016-02-24 43
2016-02-25 39
2016-02-26 44
2016-02-27 47
2016-02-28 40
2016-02-29 42
2016-03-01 43
2016-03-02 45
2016-03-03 48
2016-03-04 42
2016-03-05 47
2016-03-06 43
2016-03-07 40
2016-03-08 39
2016-03-09 42
2016-03-10 42
2016-03-11 36
2016-03-12 40
2016-03-13 39
2016-03-14 39
2016-03-15 38
2016-03-16 39
2016-03-17 34
2016-03-18 47
2016-03-19 50
2016-03-20 48
2016-03-21 46
2016-03-22 44
2016-03-23 45
2016-03-24 43
2016-03-25 43
2016-03-26 42
2016-03-27 40
2016-03-28 38
2016-03-29 40
2016-03-30 42
2016-03-31 47
2016-04-01 48
2016-04-02 46
2016-04-03 45
2016-04-04 45
2016-04-05 44
2016-04-06 44
2016-04-07 50
2016-04-08 50
2016-04-09 47
2016-04-10 46
2016-04-11 48
2016-04-12 44
2016-04-13 44
2016-04-14 44
2016-04-15 47
2016-04-16 44
2016-04-17 48
2016-04-18 54
2016-04-19 56
2016-04-20 53
2016-04-21 51
2016-04-22 52
2016-04-23 51
2016-04-24 46
2016-04-25 44
2016-04-26 43
2016-04-27 48
2016-04-28 50
2016-04-29 48
2016-04-30 45
2016-05-01 50
2016-05-02 54
2016-05-03 56
2016-05-04 52
2016-05-05 48
2016-05-06 50
2016-05-07 52
2016-05-08 48
2016-05-09 46
2016-05-10 47
2016-05-11 52
2016-05-12 49
2016-05-13 50
2016-05-14 53
2016-05-15 52
2016-05-16 50
2016-05-17 51
2016-05-18 51
2016-05-19 48
2016-05-20 49
2016-05-21 50
2016-05-22 52
2016-05-23 53
2016-05-24 55
2016-05-25 53
2016-05-26 51
2016-05-27 49
2016-05-28 49
2016-05-29 52
2016-05-30 48
2016-05-31 52
2016-06-01 55
2016-06-02 58
2016-06-03 56
2016-06-04 62
2016-06-05 64
2016-06-06 62
2016-06-07 59
2016-06-08 55
2016-06-09 51
2016-06-10 49
2016-06-11 50
2016-06-12 53
2016-06-13 49
2016-06-14 47
2016-06-15 44
2016-06-16 51
2016-06-17 50
2016-06-18 52
2016-06-19 48
2016-06-20 55
2016-06-21 54
2016-06-22 55
2016-06-23 53
2016-06-24 52
2016-06-25 56
2016-06-26 56
2016-06-27 58
2016-06-28 54
2016-06-29 55
2016-06-30 55
2016-07-01 55
2016-07-02 57
2016-07-03 56
2016-07-04 56
2016-07-05 55
2016-07-06 56
2016-07-07 58
2016-07-08 58
2016-07-09 56
2016-07-10 56
2016-07-11 57
2016-07-12 59
2016-07-13 57
2016-07-14 57
2016-07-15 56
2016-07-16 57
2016-07-17 58
2016-07-18 60
2016-07-19 60
2016-07-20 60
2016-07-21 58
2016-07-22 59
2016-07-23 58
2016-07-24 58
2016-07-25 59
2016-07-26 59
2016-07-27 56
2016-07-28 61
2016-07-29 62
2016-07-30 57
2016-07-31 56
2016-08-01 53
2016-08-02 57
2016-08-03 58
2016-08-04 56
2016-08-05 57
2016-08-06 54
2016-08-07 57
2016-08-08 56
2016-08-09 58
2016-08-10 58
2016-08-11 58
2016-08-12 63
2016-08-13 64
2016-08-14 57
2016-08-15 57
2016-08-16 56
2016-08-17 56
2016-08-18 60
2016-08-19 69
2016-08-20 62
2016-08-21 57
2016-08-22 55
2016-08-23 54
2016-08-24 58
2016-08-25 61
2016-08-26 64
2016-08-27 58
2016-08-28 55
2016-08-29 53
2016-08-30 56
2016-08-31 58
2016-09-01 56
2016-09-02 55
2016-09-03 54
2016-09-04 51
2016-09-05 54
2016-09-06 55
2016-09-07 57
2016-09-08 55
2016-09-09 50
2016-09-10 54
2016-09-11 55
2016-09-12 54
2016-09-13 51
2016-09-14 51
2016-09-15 52
2016-09-16 52
2016-09-17 57
2016-09-18 54
2016-09-19 53
2016-09-20 49
2016-09-21 52
2016-09-22 49
2016-09-23 53
2016-09-24 53
2016-09-25 58
2016-09-26 53
2016-09-27 54
2016-09-28 52
2016-09-29 47
2016-09-30 46
2016-10-01 50
2016-10-02 50
2016-10-03 49
2016-10-04 52
2016-10-05 52
2016-10-06 53
2016-10-07 53
2016-10-08 52
2016-10-09 48
2016-10-10 45
2016-10-11 43
2016-10-12 43
2016-10-13 51
2016-10-14 50
2016-10-15 50
2016-10-16 50
2016-10-17 50
2016-10-18 49
2016-10-19 45
2016-10-20 50
2016-10-21 48
2016-10-22 44
2016-10-23 49
2016-10-24 51
2016-10-25 50
2016-10-26 52
2016-10-27 52
2016-10-28 50
2016-10-29 48
2016-10-30 44
2016-10-31 50
2016-11-01 50
2016-11-02 51
2016-11-03 48
2016-11-04 42
2016-11-05 51
2016-11-06 48
2016-11-07 50
2016-11-08 49
2016-11-09 51
2016-11-10 45
2016-11-11 52
2016-11-12 48
2016-11-13 47
2016-11-14 50
2016-11-15 46
2016-11-16 43
2016-11-17 41
2016-11-18 39
2016-11-19 48
2016-11-20 47
2016-11-21 44
2016-11-22 43
2016-11-23 42
2016-11-24 46
2016-11-25 41
2016-11-26 43
2016-11-27 42
2016-11-28 40
2016-11-29 41
2016-11-30 43
2016-12-01 40
2016-12-02 44
2016-12-03 44
2016-12-04 35
2016-12-05 33
2016-12-06 31
2016-12-07 28
2016-12-08 30
2016-12-09 31
2016-12-10 35
2016-12-11 39
2016-12-12 37
2016-12-13 31
2016-12-14 30
2016-12-15 32
2016-12-16 25
2016-12-17 23
2016-12-18 27
2016-12-19 35
2016-12-20 40
2016-12-21 34
2016-12-22 33
2016-12-23 35
2016-12-24 35
2016-12-25 33
2016-12-26 33
2016-12-27 40
2016-12-28 38
2016-12-29 40
2016-12-30 34
2016-12-31 29
2017-01-01 28
2017-01-02 26
2017-01-03 21
2017-01-04 22
2017-01-05 21
2017-01-06 20
2017-01-07 29
2017-01-08 35
2017-01-09 34
2017-01-10 32
2017-01-11 28
2017-01-12 25
2017-01-13 25
2017-01-14 26
2017-01-15 27
2017-01-16 30
2017-01-17 39
2017-01-18 44
2017-01-19 41
2017-01-20 37
2017-01-21 40
2017-01-22 37
2017-01-23 34
2017-01-24 31
2017-01-25 35
2017-01-26 41
2017-01-27 37
2017-01-28 37
2017-01-29 37
2017-01-30 40
2017-01-31 34
2017-02-01 29
2017-02-02 32
2017-02-03 34
2017-02-04 37
2017-02-05 33
2017-02-06 30
2017-02-07 30
2017-02-08 34
2017-02-09 39
2017-02-10 39
2017-02-11 38
2017-02-12 34
2017-02-13 35
2017-02-14 36
2017-02-15 45
2017-02-16 46
2017-02-17 42
2017-02-18 42
2017-02-19 40
2017-02-20 40
2017-02-21 39
2017-02-22 35
2017-02-23 34
2017-02-24 32
2017-02-25 32
2017-02-26 35
2017-02-27 33
2017-02-28 33
2017-03-01 41
2017-03-02 41
2017-03-03 39
2017-03-04 34
2017-03-05 34
2017-03-06 35
2017-03-07 35
2017-03-08 37
2017-03-09 39
2017-03-10 44
2017-03-11 40
2017-03-12 45
2017-03-13 47
2017-03-14 48
2017-03-15 44
2017-03-16 39
2017-03-17 37
2017-03-18 35
2017-03-19 34
2017-03-20 38
2017-03-21 46
2017-03-22 43
2017-03-23 39
2017-03-24 44
2017-03-25 43
2017-03-26 43
2017-03-27 44
2017-03-28 44
2017-03-29 45
2017-03-30 43
2017-03-31 39
2017-04-01 45
2017-04-02 41
2017-04-03 40
2017-04-04 43
2017-04-05 48
2017-04-06 47
2017-04-07 46
2017-04-08 41
2017-04-09 39
2017-04-10 41
2017-04-11 38
2017-04-12 45
2017-04-13 43
2017-04-14 42
2017-04-15 41
2017-04-16 42
2017-04-17 50
2017-04-18 48
2017-04-19 46
2017-04-20 44
2017-04-21 43
2017-04-22 48
2017-04-23 45
2017-04-24 45
2017-04-25 47
2017-04-26 45
2017-04-27 44
2017-04-28 43
2017-04-29 42
2017-04-30 47
2017-05-01 45
2017-05-02 43
2017-05-03 55
2017-05-04 53
2017-05-05 46
2017-05-06 43
2017-05-07 45
2017-05-08 46
2017-05-09 47
2017-05-10 51
2017-05-11 46
2017-05-12 43
2017-05-13 45
2017-05-14 47
2017-05-15 46
2017-05-16 45
2017-05-17 45
2017-05-18 48
2017-05-19 50
2017-05-20 53
2017-05-21 52
2017-05-22 56
2017-05-23 53
2017-05-24 46
2017-05-25 47
2017-05-26 51
2017-05-27 55
2017-05-28 56
2017-05-29 52
2017-05-30 52
2017-05-31 52
2017-06-01 56
2017-06-02 53
2017-06-03 53
2017-06-04 52
2017-06-05 50
2017-06-06 53
2017-06-07 56
2017-06-08 54
2017-06-09 50
2017-06-10 48
2017-06-11 52
2017-06-12 52
2017-06-13 52
2017-06-14 51
2017-06-15 53
2017-06-16 53
2017-06-17 51
2017-06-18 55
2017-06-19 56
2017-06-20 56
2017-06-21 53
2017-06-22 50
2017-06-23 55
2017-06-24 61
2017-06-25 64
2017-06-26 55
2017-06-27 53
2017-06-28 54
2017-06-29 53
2017-06-30 57
2017-07-01 55
2017-07-02 54
2017-07-03 56
2017-07-04 53
2017-07-05 57
2017-07-06 55
2017-07-07 55
2017-07-08 58
2017-07-09 56
2017-07-10 58
2017-07-11 54
2017-07-12 54
2017-07-13 57
2017-07-14 57
2017-07-15 54
2017-07-16 55
2017-07-17 55
2017-07-18 56
2017-07-19 53
2017-07-20 59
2017-07-21 55
2017-07-22 63
2017-07-23 62
2017-07-24 56
2017-07-25 58
2017-07-26 56
2017-07-27 57
2017-07-28 56
2017-07-29 54
2017-07-30 58
2017-07-31 60
2017-08-01 60
2017-08-02 69
2017-08-03 68
2017-08-04 62
2017-08-05 58
2017-08-06 58
2017-08-07 57
2017-08-08 59
2017-08-09 61
2017-08-10 60
2017-08-11 57
2017-08-12 58
2017-08-13 60
2017-08-14 56
2017-08-15 55
2017-08-16 57
2017-08-17 59
2017-08-18 55
2017-08-19 58
2017-08-20 56
2017-08-21 57
2017-08-22 60
2017-08-23 59
2017-08-24 58
2017-08-25 54
2017-08-26 54
2017-08-27 61
2017-08-28 62
2017-08-29 60
2017-08-30 60
2017-08-31 61
2017-09-01 57
2017-09-02 62
2017-09-03 65
2017-09-04 65
2017-09-05 66
2017-09-06 63
2017-09-07 61
2017-09-08 60
2017-09-09 59
2017-09-10 55
2017-09-11 52
2017-09-12 57
2017-09-13 53
2017-09-14 54
2017-09-15 50
2017-09-16 53
2017-09-17 54
2017-09-18 50
2017-09-19 50
2017-09-20 50
2017-09-21 50
2017-09-22 51
2017-09-23 54
2017-09-24 50
2017-09-25 56
2017-09-26 55

Pivoting in SQL

Getting the monthly average high temperatures with month as columns and year as rows.

SELECT * FROM (
  SELECT year(date) year, month(date) month, temp
  FROM high_temps
  WHERE date BETWEEN DATE '2015-01-01' AND DATE '2018-08-31'
)
year month temp
2015.0 1.0 42
2015.0 1.0 42
2015.0 1.0 41
2015.0 1.0 51
2015.0 1.0 54
2015.0 1.0 54
2015.0 1.0 46
2015.0 1.0 46
2015.0 1.0 50
2015.0 1.0 46
2015.0 1.0 49
2015.0 1.0 52
2015.0 1.0 49
2015.0 1.0 43
2015.0 1.0 46
2015.0 1.0 53
2015.0 1.0 56
2015.0 1.0 57
2015.0 1.0 50
2015.0 1.0 50
2015.0 1.0 45
2015.0 1.0 49
2015.0 1.0 54
2015.0 1.0 58
2015.0 1.0 63
2015.0 1.0 61
2015.0 1.0 52
2015.0 1.0 54
2015.0 1.0 54
2015.0 1.0 47
2015.0 1.0 45
2015.0 2.0 49
2015.0 2.0 52
2015.0 2.0 50
2015.0 2.0 51
2015.0 2.0 56
2015.0 2.0 58
2015.0 2.0 54
2015.0 2.0 59
2015.0 2.0 56
2015.0 2.0 55
2015.0 2.0 55
2015.0 2.0 62
2015.0 2.0 60
2015.0 2.0 58
2015.0 2.0 54
2015.0 2.0 59
2015.0 2.0 61
2015.0 2.0 54
2015.0 2.0 51
2015.0 2.0 52
2015.0 2.0 54
2015.0 2.0 53
2015.0 2.0 55
2015.0 2.0 52
2015.0 2.0 50
2015.0 2.0 53
2015.0 2.0 50
2015.0 2.0 54
2015.0 3.0 52
2015.0 3.0 52
2015.0 3.0 51
2015.0 3.0 55
2015.0 3.0 56
2015.0 3.0 59
2015.0 3.0 62
2015.0 3.0 63
2015.0 3.0 58
2015.0 3.0 56
2015.0 3.0 58
2015.0 3.0 64
2015.0 3.0 63
2015.0 3.0 57
2015.0 3.0 51
2015.0 3.0 57
2015.0 3.0 56
2015.0 3.0 60
2015.0 3.0 60
2015.0 3.0 57
2015.0 3.0 56
2015.0 3.0 53
2015.0 3.0 52
2015.0 3.0 55
2015.0 3.0 58
2015.0 3.0 69
2015.0 3.0 65
2015.0 3.0 60
2015.0 3.0 60
2015.0 3.0 64
2015.0 3.0 55
2015.0 4.0 55
2015.0 4.0 56
2015.0 4.0 52
2015.0 4.0 55
2015.0 4.0 62
2015.0 4.0 57
2015.0 4.0 58
2015.0 4.0 63
2015.0 4.0 63
2015.0 4.0 57
2015.0 4.0 53
2015.0 4.0 56
2015.0 4.0 53
2015.0 4.0 53
2015.0 4.0 57
2015.0 4.0 64
2015.0 4.0 66
2015.0 4.0 66
2015.0 4.0 70
2015.0 4.0 73
2015.0 4.0 63
2015.0 4.0 60
2015.0 4.0 54
2015.0 4.0 54
2015.0 4.0 56
2015.0 4.0 60
2015.0 4.0 77
2015.0 4.0 60
2015.0 4.0 61
2015.0 4.0 63
2015.0 5.0 65
2015.0 5.0 65
2015.0 5.0 69
2015.0 5.0 63
2015.0 5.0 58
2015.0 5.0 62
2015.0 5.0 69
2015.0 5.0 75
2015.0 5.0 80
2015.0 5.0 67
2015.0 5.0 57
2015.0 5.0 60
2015.0 5.0 54
2015.0 5.0 64
2015.0 5.0 68
2015.0 5.0 60
2015.0 5.0 67
2015.0 5.0 78
2015.0 5.0 71
2015.0 5.0 74
2015.0 5.0 78
2015.0 5.0 62
2015.0 5.0 61
2015.0 5.0 64
2015.0 5.0 60
2015.0 5.0 71
2015.0 5.0 76
2015.0 5.0 82
2015.0 5.0 79
2015.0 5.0 73
2015.0 5.0 77
2015.0 6.0 61
2015.0 6.0 64
2015.0 6.0 68
2015.0 6.0 73
2015.0 6.0 80
2015.0 6.0 85
2015.0 6.0 88
2015.0 6.0 87
2015.0 6.0 84
2015.0 6.0 78
2015.0 6.0 76
2015.0 6.0 68
2015.0 6.0 75
2015.0 6.0 82
2015.0 6.0 86
2015.0 6.0 73
2015.0 6.0 77
2015.0 6.0 76
2015.0 6.0 75
2015.0 6.0 77
2015.0 6.0 78
2015.0 6.0 77
2015.0 6.0 79
2015.0 6.0 78
2015.0 6.0 87
2015.0 6.0 89
2015.0 6.0 92
2015.0 6.0 83
2015.0 6.0 84
2015.0 6.0 87
2015.0 7.0 90
2015.0 7.0 93
2015.0 7.0 92
2015.0 7.0 92
2015.0 7.0 91
2015.0 7.0 85
2015.0 7.0 81
2015.0 7.0 86
2015.0 7.0 84
2015.0 7.0 70
2015.0 7.0 72
2015.0 7.0 79
2015.0 7.0 78
2015.0 7.0 82
2015.0 7.0 79
2015.0 7.0 79
2015.0 7.0 82
2015.0 7.0 92
2015.0 7.0 95
2015.0 7.0 80
2015.0 7.0 75
2015.0 7.0 75
2015.0 7.0 79
2015.0 7.0 73
2015.0 7.0 70
2015.0 7.0 72
2015.0 7.0 74
2015.0 7.0 82
2015.0 7.0 90
2015.0 7.0 94
2015.0 7.0 94
2015.0 8.0 92
2015.0 8.0 87
2015.0 8.0 83
2015.0 8.0 79
2015.0 8.0 74
2015.0 8.0 77
2015.0 8.0 83
2015.0 8.0 77
2015.0 8.0 83
2015.0 8.0 84
2015.0 8.0 86
2015.0 8.0 83
2015.0 8.0 83
2015.0 8.0 65
2015.0 8.0 71
2015.0 8.0 77
2015.0 8.0 81
2015.0 8.0 86
2015.0 8.0 89
2015.0 8.0 73
2015.0 8.0 72
2015.0 8.0 80
2015.0 8.0 82
2015.0 8.0 75
2015.0 8.0 78
2015.0 8.0 83
2015.0 8.0 85
2015.0 8.0 74
2015.0 8.0 72
2015.0 8.0 68
2015.0 8.0 66
2015.0 9.0 67
2015.0 9.0 67
2015.0 9.0 65
2015.0 9.0 65
2015.0 9.0 69
2015.0 9.0 61
2015.0 9.0 70
2015.0 9.0 73
2015.0 9.0 76
2015.0 9.0 77
2015.0 9.0 81
2015.0 9.0 80
2015.0 9.0 69
2015.0 9.0 62
2015.0 9.0 64
2015.0 9.0 68
2015.0 9.0 65
2015.0 9.0 67
2015.0 9.0 70
2015.0 9.0 73
2015.0 9.0 65
2015.0 9.0 66
2015.0 9.0 69
2015.0 9.0 72
2015.0 9.0 60
2015.0 9.0 65
2015.0 9.0 64
2015.0 9.0 70
2015.0 9.0 71
2015.0 9.0 65
2015.0 10.0 70
2015.0 10.0 60
2015.0 10.0 67
2015.0 10.0 73
2015.0 10.0 74
2015.0 10.0 65
2015.0 10.0 61
2015.0 10.0 66
2015.0 10.0 67
2015.0 10.0 70
2015.0 10.0 64
2015.0 10.0 65
2015.0 10.0 62
2015.0 10.0 59
2015.0 10.0 70
2015.0 10.0 68
2015.0 10.0 67
2015.0 10.0 59
2015.0 10.0 63
2015.0 10.0 64
2015.0 10.0 61
2015.0 10.0 61
2015.0 10.0 55
2015.0 10.0 59
2015.0 10.0 67
2015.0 10.0 54
2015.0 10.0 61
2015.0 10.0 57
2015.0 10.0 59
2015.0 10.0 63
2015.0 10.0 60
2015.0 11.0 54
2015.0 11.0 52
2015.0 11.0 51
2015.0 11.0 50
2015.0 11.0 53
2015.0 11.0 60
2015.0 11.0 54
2015.0 11.0 52
2015.0 11.0 50
2015.0 11.0 52
2015.0 11.0 52
2015.0 11.0 52
2015.0 11.0 56
2015.0 11.0 49
2015.0 11.0 48
2015.0 11.0 48
2015.0 11.0 56
2015.0 11.0 48
2015.0 11.0 48
2015.0 11.0 47
2015.0 11.0 48
2015.0 11.0 50
2015.0 11.0 44
2015.0 11.0 44
2015.0 11.0 45
2015.0 11.0 49
2015.0 11.0 49
2015.0 11.0 45
2015.0 11.0 35
2015.0 11.0 42
2015.0 12.0 50
2015.0 12.0 51
2015.0 12.0 60
2015.0 12.0 51
2015.0 12.0 50
2015.0 12.0 55
2015.0 12.0 52
2015.0 12.0 60
2015.0 12.0 54
2015.0 12.0 53
2015.0 12.0 49
2015.0 12.0 48
2015.0 12.0 46
2015.0 12.0 46
2015.0 12.0 44
2015.0 12.0 43
2015.0 12.0 44
2015.0 12.0 48
2015.0 12.0 47
2015.0 12.0 46
2015.0 12.0 42
2015.0 12.0 46
2015.0 12.0 41
2015.0 12.0 42
2015.0 12.0 41
2015.0 12.0 40
2015.0 12.0 40
2015.0 12.0 41
2015.0 12.0 45
2015.0 12.0 42
2015.0 12.0 42
2016.0 1.0 46
2016.0 1.0 42
2016.0 1.0 40
2016.0 1.0 38
2016.0 1.0 46
2016.0 1.0 53
2016.0 1.0 44
2016.0 1.0 48
2016.0 1.0 49
2016.0 1.0 52
2016.0 1.0 47
2016.0 1.0 48
2016.0 1.0 54
2016.0 1.0 48
2016.0 1.0 47
2016.0 1.0 52
2016.0 1.0 49
2016.0 1.0 52
2016.0 1.0 47
2016.0 1.0 50
2016.0 1.0 53
2016.0 1.0 55
2016.0 1.0 46
2016.0 1.0 50
2016.0 1.0 55
2016.0 1.0 56
2016.0 1.0 58
2016.0 1.0 56
2016.0 1.0 49
2016.0 1.0 46
2016.0 1.0 45
2016.0 2.0 45
2016.0 2.0 50
2016.0 2.0 47
2016.0 2.0 49
2016.0 2.0 52
2016.0 2.0 48
2016.0 2.0 52
2016.0 2.0 59
2016.0 2.0 63
2016.0 2.0 56
2016.0 2.0 55
2016.0 2.0 57
2016.0 2.0 50
2016.0 2.0 53
2016.0 2.0 54
2016.0 2.0 54
2016.0 2.0 63
2016.0 2.0 51
2016.0 2.0 50
2016.0 2.0 51
2016.0 2.0 50
2016.0 2.0 51
2016.0 2.0 58
2016.0 2.0 57
2016.0 2.0 62
2016.0 2.0 57
2016.0 2.0 56
2016.0 2.0 50
2016.0 2.0 53
2016.0 3.0 57
2016.0 3.0 54
2016.0 3.0 58
2016.0 3.0 55
2016.0 3.0 64
2016.0 3.0 59
2016.0 3.0 51
2016.0 3.0 53
2016.0 3.0 55
2016.0 3.0 55
2016.0 3.0 60
2016.0 3.0 50
2016.0 3.0 54
2016.0 3.0 47
2016.0 3.0 50
2016.0 3.0 53
2016.0 3.0 57
2016.0 3.0 63
2016.0 3.0 61
2016.0 3.0 54
2016.0 3.0 55
2016.0 3.0 55
2016.0 3.0 51
2016.0 3.0 52
2016.0 3.0 56
2016.0 3.0 58
2016.0 3.0 55
2016.0 3.0 55
2016.0 3.0 63
2016.0 3.0 68
2016.0 3.0 71
2016.0 4.0 71
2016.0 4.0 62
2016.0 4.0 68
2016.0 4.0 58
2016.0 4.0 56
2016.0 4.0 69
2016.0 4.0 78
2016.0 4.0 76
2016.0 4.0 64
2016.0 4.0 58
2016.0 4.0 58
2016.0 4.0 57
2016.0 4.0 58
2016.0 4.0 59
2016.0 4.0 59
2016.0 4.0 67
2016.0 4.0 80
2016.0 4.0 89
2016.0 4.0 84
2016.0 4.0 81
2016.0 4.0 72
2016.0 4.0 62
2016.0 4.0 64
2016.0 4.0 55
2016.0 4.0 59
2016.0 4.0 59
2016.0 4.0 63
2016.0 4.0 63
2016.0 4.0 61
2016.0 4.0 67
2016.0 5.0 79
2016.0 5.0 87
2016.0 5.0 72
2016.0 5.0 60
2016.0 5.0 68
2016.0 5.0 77
2016.0 5.0 82
2016.0 5.0 66
2016.0 5.0 67
2016.0 5.0 75
2016.0 5.0 80
2016.0 5.0 76
2016.0 5.0 84
2016.0 5.0 58
2016.0 5.0 56
2016.0 5.0 59
2016.0 5.0 69
2016.0 5.0 65
2016.0 5.0 63
2016.0 5.0 66
2016.0 5.0 58
2016.0 5.0 66
2016.0 5.0 66
2016.0 5.0 66
2016.0 5.0 65
2016.0 5.0 62
2016.0 5.0 64
2016.0 5.0 62
2016.0 5.0 64
2016.0 5.0 72
2016.0 5.0 80
2016.0 6.0 74
2016.0 6.0 70
2016.0 6.0 82
2016.0 6.0 85
2016.0 6.0 93
2016.0 6.0 88
2016.0 6.0 84
2016.0 6.0 67
2016.0 6.0 64
2016.0 6.0 65
2016.0 6.0 67
2016.0 6.0 70
2016.0 6.0 65
2016.0 6.0 60
2016.0 6.0 66
2016.0 6.0 70
2016.0 6.0 69
2016.0 6.0 66
2016.0 6.0 72
2016.0 6.0 75
2016.0 6.0 71
2016.0 6.0 75
2016.0 6.0 68
2016.0 6.0 69
2016.0 6.0 72
2016.0 6.0 80
2016.0 6.0 85
2016.0 6.0 76
2016.0 6.0 72
2016.0 6.0 73
2016.0 7.0 75
2016.0 7.0 75
2016.0 7.0 71
2016.0 7.0 69
2016.0 7.0 67
2016.0 7.0 76
2016.0 7.0 68
2016.0 7.0 72
2016.0 7.0 71
2016.0 7.0 73
2016.0 7.0 73
2016.0 7.0 76
2016.0 7.0 74
2016.0 7.0 76
2016.0 7.0 76
2016.0 7.0 73
2016.0 7.0 79
2016.0 7.0 71
2016.0 7.0 77
2016.0 7.0 80
2016.0 7.0 84
2016.0 7.0 73
2016.0 7.0 73
2016.0 7.0 81
2016.0 7.0 86
2016.0 7.0 78
2016.0 7.0 84
2016.0 7.0 88
2016.0 7.0 89
2016.0 7.0 75
2016.0 7.0 72
2016.0 8.0 77
2016.0 8.0 71
2016.0 8.0 76
2016.0 8.0 82
2016.0 8.0 79
2016.0 8.0 71
2016.0 8.0 69
2016.0 8.0 73
2016.0 8.0 71
2016.0 8.0 76
2016.0 8.0 81
2016.0 8.0 90
2016.0 8.0 91
2016.0 8.0 85
2016.0 8.0 83
2016.0 8.0 83
2016.0 8.0 80
2016.0 8.0 87
2016.0 8.0 95
2016.0 8.0 91
2016.0 8.0 73
2016.0 8.0 72
2016.0 8.0 80
2016.0 8.0 85
2016.0 8.0 91
2016.0 8.0 92
2016.0 8.0 73
2016.0 8.0 70
2016.0 8.0 79
2016.0 8.0 71
2016.0 8.0 69
2016.0 9.0 66
2016.0 9.0 69
2016.0 9.0 69
2016.0 9.0 70
2016.0 9.0 64
2016.0 9.0 65
2016.0 9.0 66
2016.0 9.0 70
2016.0 9.0 73
2016.0 9.0 77
2016.0 9.0 70
2016.0 9.0 75
2016.0 9.0 77
2016.0 9.0 78
2016.0 9.0 72
2016.0 9.0 74
2016.0 9.0 67
2016.0 9.0 68
2016.0 9.0 68
2016.0 9.0 67
2016.0 9.0 68
2016.0 9.0 65
2016.0 9.0 63
2016.0 9.0 66
2016.0 9.0 76
2016.0 9.0 77
2016.0 9.0 69
2016.0 9.0 67
2016.0 9.0 66
2016.0 9.0 65
2016.0 10.0 63
2016.0 10.0 63
2016.0 10.0 60
2016.0 10.0 60
2016.0 10.0 65
2016.0 10.0 62
2016.0 10.0 63
2016.0 10.0 65
2016.0 10.0 56
2016.0 10.0 60
2016.0 10.0 62
2016.0 10.0 66
2016.0 10.0 59
2016.0 10.0 57
2016.0 10.0 62
2016.0 10.0 60
2016.0 10.0 59
2016.0 10.0 58
2016.0 10.0 58
2016.0 10.0 60
2016.0 10.0 57
2016.0 10.0 62
2016.0 10.0 62
2016.0 10.0 63
2016.0 10.0 63
2016.0 10.0 58
2016.0 10.0 58
2016.0 10.0 67
2016.0 10.0 60
2016.0 10.0 52
2016.0 10.0 57
2016.0 11.0 56
2016.0 11.0 56
2016.0 11.0 65
2016.0 11.0 63
2016.0 11.0 60
2016.0 11.0 60
2016.0 11.0 66
2016.0 11.0 70
2016.0 11.0 64
2016.0 11.0 62
2016.0 11.0 61
2016.0 11.0 58
2016.0 11.0 54
2016.0 11.0 57
2016.0 11.0 54
2016.0 11.0 50
2016.0 11.0 51
2016.0 11.0 54
2016.0 11.0 57
2016.0 11.0 57
2016.0 11.0 53
2016.0 11.0 53
2016.0 11.0 48
2016.0 11.0 51
2016.0 11.0 52
2016.0 11.0 51
2016.0 11.0 47
2016.0 11.0 50
2016.0 11.0 51
2016.0 11.0 50
2016.0 12.0 46
2016.0 12.0 49
2016.0 12.0 48
2016.0 12.0 45
2016.0 12.0 39
2016.0 12.0 40
2016.0 12.0 38
2016.0 12.0 38
2016.0 12.0 36
2016.0 12.0 44
2016.0 12.0 42
2016.0 12.0 42
2016.0 12.0 38
2016.0 12.0 39
2016.0 12.0 38
2016.0 12.0 34
2016.0 12.0 33
2016.0 12.0 39
2016.0 12.0 45
2016.0 12.0 50
2016.0 12.0 47
2016.0 12.0 44
2016.0 12.0 39
2016.0 12.0 40
2016.0 12.0 42
2016.0 12.0 42
2016.0 12.0 44
2016.0 12.0 47
2016.0 12.0 48
2016.0 12.0 45
2016.0 12.0 38
2017.0 1.0 37
2017.0 1.0 34
2017.0 1.0 33
2017.0 1.0 36
2017.0 1.0 35
2017.0 1.0 40
2017.0 1.0 37
2017.0 1.0 45
2017.0 1.0 42
2017.0 1.0 40
2017.0 1.0 34
2017.0 1.0 40
2017.0 1.0 37
2017.0 1.0 44
2017.0 1.0 43
2017.0 1.0 45
2017.0 1.0 49
2017.0 1.0 53
2017.0 1.0 50
2017.0 1.0 52
2017.0 1.0 54
2017.0 1.0 49
2017.0 1.0 48
2017.0 1.0 43
2017.0 1.0 44
2017.0 1.0 47
2017.0 1.0 54
2017.0 1.0 52
2017.0 1.0 48
2017.0 1.0 45
2017.0 1.0 44
2017.0 2.0 43
2017.0 2.0 44
2017.0 2.0 40
2017.0 2.0 46
2017.0 2.0 41
2017.0 2.0 37
2017.0 2.0 38
2017.0 2.0 40
2017.0 2.0 56
2017.0 2.0 48
2017.0 2.0 49
2017.0 2.0 51
2017.0 2.0 57
2017.0 2.0 58
2017.0 2.0 54
2017.0 2.0 53
2017.0 2.0 56
2017.0 2.0 45
2017.0 2.0 47
2017.0 2.0 45
2017.0 2.0 48
2017.0 2.0 46
2017.0 2.0 44
2017.0 2.0 47
2017.0 2.0 44
2017.0 2.0 41
2017.0 2.0 41
2017.0 2.0 47
2017.0 3.0 49
2017.0 3.0 47
2017.0 3.0 51
2017.0 3.0 46
2017.0 3.0 45
2017.0 3.0 43
2017.0 3.0 42
2017.0 3.0 43
2017.0 3.0 49
2017.0 3.0 52
2017.0 3.0 53
2017.0 3.0 53
2017.0 3.0 53
2017.0 3.0 57
2017.0 3.0 50
2017.0 3.0 52
2017.0 3.0 53
2017.0 3.0 57
2017.0 3.0 53
2017.0 3.0 54
2017.0 3.0 56
2017.0 3.0 56
2017.0 3.0 58
2017.0 3.0 54
2017.0 3.0 52
2017.0 3.0 49
2017.0 3.0 53
2017.0 3.0 53
2017.0 3.0 55
2017.0 3.0 54
2017.0 3.0 56
2017.0 4.0 56
2017.0 4.0 55
2017.0 4.0 53
2017.0 4.0 61
2017.0 4.0 54
2017.0 4.0 60
2017.0 4.0 59
2017.0 4.0 52
2017.0 4.0 56
2017.0 4.0 53
2017.0 4.0 57
2017.0 4.0 59
2017.0 4.0 54
2017.0 4.0 52
2017.0 4.0 57
2017.0 4.0 66
2017.0 4.0 60
2017.0 4.0 60
2017.0 4.0 56
2017.0 4.0 58
2017.0 4.0 66
2017.0 4.0 63
2017.0 4.0 53
2017.0 4.0 57
2017.0 4.0 56
2017.0 4.0 58
2017.0 4.0 56
2017.0 4.0 59
2017.0 4.0 57
2017.0 4.0 56
2017.0 5.0 50
2017.0 5.0 61
2017.0 5.0 74
2017.0 5.0 76
2017.0 5.0 59
2017.0 5.0 60
2017.0 5.0 62
2017.0 5.0 64
2017.0 5.0 70
2017.0 5.0 71
2017.0 5.0 60
2017.0 5.0 54
2017.0 5.0 57
2017.0 5.0 59
2017.0 5.0 52
2017.0 5.0 54
2017.0 5.0 60
2017.0 5.0 64
2017.0 5.0 71
2017.0 5.0 72
2017.0 5.0 76
2017.0 5.0 83
2017.0 5.0 78
2017.0 5.0 63
2017.0 5.0 70
2017.0 5.0 81
2017.0 5.0 86
2017.0 5.0 86
2017.0 5.0 74
2017.0 5.0 58
2017.0 5.0 72
2017.0 6.0 67
2017.0 6.0 73
2017.0 6.0 67
2017.0 6.0 67
2017.0 6.0 74
2017.0 6.0 84
2017.0 6.0 79
2017.0 6.0 63
2017.0 6.0 63
2017.0 6.0 67
2017.0 6.0 70
2017.0 6.0 64
2017.0 6.0 65
2017.0 6.0 65
2017.0 6.0 61
2017.0 6.0 68
2017.0 6.0 63
2017.0 6.0 71
2017.0 6.0 74
2017.0 6.0 73
2017.0 6.0 69
2017.0 6.0 75
2017.0 6.0 81
2017.0 6.0 89
2017.0 6.0 96
2017.0 6.0 72
2017.0 6.0 71
2017.0 6.0 68
2017.0 6.0 78
2017.0 6.0 86
2017.0 7.0 72
2017.0 7.0 77
2017.0 7.0 70
2017.0 7.0 78
2017.0 7.0 85
2017.0 7.0 85
2017.0 7.0 72
2017.0 7.0 81
2017.0 7.0 80
2017.0 7.0 73
2017.0 7.0 75
2017.0 7.0 78
2017.0 7.0 75
2017.0 7.0 79
2017.0 7.0 77
2017.0 7.0 72
2017.0 7.0 78
2017.0 7.0 82
2017.0 7.0 78
2017.0 7.0 74
2017.0 7.0 78
2017.0 7.0 85
2017.0 7.0 76
2017.0 7.0 82
2017.0 7.0 87
2017.0 7.0 84
2017.0 7.0 70
2017.0 7.0 79
2017.0 7.0 81
2017.0 7.0 80
2017.0 7.0 85
2017.0 8.0 87
2017.0 8.0 91
2017.0 8.0 94
2017.0 8.0 91
2017.0 8.0 82
2017.0 8.0 82
2017.0 8.0 84
2017.0 8.0 89
2017.0 8.0 91
2017.0 8.0 90
2017.0 8.0 77
2017.0 8.0 76
2017.0 8.0 74
2017.0 8.0 73
2017.0 8.0 78
2017.0 8.0 79
2017.0 8.0 75
2017.0 8.0 73
2017.0 8.0 76
2017.0 8.0 77
2017.0 8.0 83
2017.0 8.0 86
2017.0 8.0 75
2017.0 8.0 72
2017.0 8.0 76
2017.0 8.0 84
2017.0 8.0 87
2017.0 8.0 88
2017.0 8.0 85
2017.0 8.0 77
2017.0 8.0 75
2017.0 9.0 85
2017.0 9.0 90
2017.0 9.0 90
2017.0 9.0 88
2017.0 9.0 85
2017.0 9.0 81
2017.0 9.0 75
2017.0 9.0 71
2017.0 9.0 68
2017.0 9.0 73
2017.0 9.0 80
2017.0 9.0 77
2017.0 9.0 70
2017.0 9.0 74
2017.0 9.0 76
2017.0 9.0 76
2017.0 9.0 64
2017.0 9.0 62
2017.0 9.0 64
2017.0 9.0 57
2017.0 9.0 64
2017.0 9.0 68
2017.0 9.0 68
2017.0 9.0 72
2017.0 9.0 65
2017.0 9.0 74
SELECT * FROM (
  SELECT year(date) year, month(date) month, temp
  FROM high_temps
  WHERE date BETWEEN DATE '2015-01-01' AND DATE '2018-08-31'
)
PIVOT (
  CAST(avg(temp) AS DECIMAL(4, 1))
  FOR month in (
    1 JAN, 2 FEB, 3 MAR, 4 APR, 5 MAY, 6 JUN,
    7 JUL, 8 AUG, 9 SEP, 10 OCT, 11 NOV, 12 DEC
  )
)
ORDER BY year DESC
year JAN FEB MAR APR MAY JUN JUL AUG SEP OCT NOV DEC
2018.0 49.7 45.8 54.0 58.6 70.8 71.9 82.8 79.1 null null null null
2017.0 43.7 46.6 51.5 57.3 67.0 72.1 78.3 81.5 73.8 61.1 51.3 45.5
2016.0 49.1 53.6 56.4 65.9 68.8 73.1 76.0 79.5 69.6 60.5 56.0 41.9
2015.0 50.3 54.5 57.9 59.9 68.0 78.9 82.6 79.0 68.5 63.6 49.4 47.1

Pivoting with Multiple Aggregate Expressions

Getting monthly average and maximum high temperatures with month as columns and year as rows.

SELECT * FROM (
  SELECT year(date) year, month(date) month, temp
  FROM high_temps
  WHERE date BETWEEN DATE '2015-01-01' AND DATE '2018-08-31'
)
PIVOT (
  CAST(avg(temp) AS DECIMAL(4, 1)) avg, max(temp) max
  FOR month in (6 JUN, 7 JUL, 8 AUG, 9 SEP)
)
ORDER BY year DESC
year JUN_avg JUN_max JUL_avg JUL_max AUG_avg AUG_max SEP_avg SEP_max
2018.0 71.9 88 82.8 94 79.1 94 null null
2017.0 72.1 96 78.3 87 81.5 94 73.8 90
2016.0 73.1 93 76.0 89 79.5 95 69.6 78
2015.0 78.9 92 82.6 95 79.0 92 68.5 81

Pivoting with Multiple Grouping Columns

Getting monthly average high and average low temperatures with month as columns and (year, hi/lo) as rows.

SELECT * FROM (
  SELECT year(date) year, month(date) month, temp, flag `H/L`
  FROM (
    SELECT date, temp, 'H' as flag
    FROM high_temps
    UNION ALL
    SELECT date, temp, 'L' as flag
    FROM low_temps
  )
  WHERE date BETWEEN DATE '2015-01-01' AND DATE '2018-08-31'
)
PIVOT (
  CAST(avg(temp) AS DECIMAL(4, 1))
  FOR month in (6 JUN, 7 JUL, 8 AUG, 9 SEP)
)
ORDER BY year DESC, `H/L` ASC
year H/L JUN JUL AUG SEP
2018.0 H 71.9 82.8 79.1 null
2018.0 L 53.4 58.5 58.5 null
2017.0 H 72.1 78.3 81.5 73.8
2017.0 L 53.7 56.3 59.0 55.6
2016.0 H 73.1 76.0 79.5 69.6
2016.0 L 53.9 57.6 57.9 52.9
2015.0 H 78.9 82.6 79.0 68.5
2015.0 L 56.4 59.9 58.5 52.5

Pivoting with Multiple Pivot Columns

Getting monthly average high and average low temperatures with (month, hi/lo) as columns and year as rows.

SELECT * FROM (
  SELECT year(date) year, month(date) month, temp, flag
  FROM (
    SELECT date, temp, 'H' as flag
    FROM high_temps
    UNION ALL
    SELECT date, temp, 'L' as flag
    FROM low_temps
  )
  WHERE date BETWEEN DATE '2015-01-01' AND DATE '2018-08-31'
)
PIVOT (
  CAST(avg(temp) AS DECIMAL(4, 1))
  FOR (month, flag) in (
    (6, 'H') JUN_hi, (6, 'L') JUN_lo,
    (7, 'H') JUL_hi, (7, 'L') JUL_lo,
    (8, 'H') AUG_hi, (8, 'L') AUG_lo,
    (9, 'H') SEP_hi, (9, 'L') SEP_lo
  )
)
ORDER BY year DESC
year JUN_hi JUN_lo JUL_hi JUL_lo AUG_hi AUG_lo SEP_hi SEP_lo
2018.0 71.9 53.4 82.8 58.5 79.1 58.5 null null
2017.0 72.1 53.7 78.3 56.3 81.5 59.0 73.8 55.6
2016.0 73.1 53.9 76.0 57.6 79.5 57.9 69.6 52.9
2015.0 78.9 56.4 82.6 59.9 79.0 58.5 68.5 52.5

ScaDaMaLe Course site and book

Diamonds ML Pipeline Workflow - DataFrame ETL and EDA Part

This is the Spark SQL parts that are focussed on extract-transform-Load (ETL) and exploratory-data-analysis (EDA) parts of an end-to-end example of a Machine Learning (ML) workflow.

Why are we using DataFrames? This is because of the Announcement in the Spark MLlib Main Guide for Spark 2.2 https://spark.apache.org/docs/latest/ml-guide.html that "DataFrame-based API is primary API".

This notebook is a scalarific break-down of the pythonic 'Diamonds ML Pipeline Workflow' from the Databricks Guide.

We will see this example again in the sequel

For this example, we analyze the Diamonds dataset from the R Datasets hosted on DBC.

Later on, we will use the DecisionTree algorithm to predict the price of a diamond from its characteristics.

Here is an outline of our pipeline:

  • Step 1. Load data: Load data as DataFrame
  • Step 2. Understand the data: Compute statistics and create visualizations to get a better understanding of the data.
  • Step 3. Hold out data: Split the data randomly into training and test sets. We will not look at the test data until after learning.
  • Step 4. On the training dataset:
    • Extract features: We will index categorical (String-valued) features so that DecisionTree can handle them.
    • Learn a model: Run DecisionTree to learn how to predict a diamond's price from a description of the diamond.
    • Tune the model: Tune the tree depth (complexity) using the training data. (This process is also called model selection.)
  • Step 5. Evaluate the model: Now look at the test dataset. Compare the initial model with the tuned model to see the benefit of tuning parameters.
  • Step 6. Understand the model: We will examine the learned model and results to gain further insight.

In this notebook, we will only cover Step 1 and Step 2. above. The other Steps will be revisited in the sequel.

Step 1. Load data as DataFrame

This section loads a dataset as a DataFrame and examines a few rows of it to understand the schema.

For more info, see the DB guide on importing data.

// We'll use the Diamonds dataset from the R datasets hosted on DBC.
val diamondsFilePath = "/datasets/sds/Rdatasets/data-001/csv/ggplot2/diamonds.csv"
diamondsFilePath: String = /datasets/sds/Rdatasets/data-001/csv/ggplot2/diamonds.csv
sc.textFile(diamondsFilePath).take(2) // looks like a csv file as it should
res0: Array[String] = Array("","carat","cut","color","clarity","depth","table","price","x","y","z", "1",0.23,"Ideal","E","SI2",61.5,55,326,3.95,3.98,2.43)
val diamondsRawDF = sqlContext.read    // we can use sqlContext instead of SparkSession for backwards compatibility to 1.x
    .format("csv") // use csv reader
    .option("header", "true") // Use first line of all files as header
    .option("inferSchema", "true") // Automatically infer data types
    //.option("delimiter", ",") // Specify the delimiter as comma or ',' DEFAULT
    .load(diamondsFilePath)
diamondsRawDF: org.apache.spark.sql.DataFrame = [_c0: int, carat: double ... 9 more fields]
//There are 10 columns.  We will try to predict the price of diamonds, treating the other 9 columns as features.
diamondsRawDF.printSchema()
root
 |-- _c0: integer (nullable = true)
 |-- carat: double (nullable = true)
 |-- cut: string (nullable = true)
 |-- color: string (nullable = true)
 |-- clarity: string (nullable = true)
 |-- depth: double (nullable = true)
 |-- table: double (nullable = true)
 |-- price: integer (nullable = true)
 |-- x: double (nullable = true)
 |-- y: double (nullable = true)
 |-- z: double (nullable = true)

Note: (nullable = true) simply means if the value is allowed to be null.

Let us count the number of rows in diamondsDF.

diamondsRawDF.count() // Ctrl+Enter
res3: Long = 53940

So there are 53940 records or rows in the DataFrame.

Use the show(n) method to see the first n (default is 20) rows of the DataFrame, as folows:

diamondsRawDF.show(10)
+---+-----+---------+-----+-------+-----+-----+-----+----+----+----+
|_c0|carat|      cut|color|clarity|depth|table|price|   x|   y|   z|
+---+-----+---------+-----+-------+-----+-----+-----+----+----+----+
|  1| 0.23|    Ideal|    E|    SI2| 61.5| 55.0|  326|3.95|3.98|2.43|
|  2| 0.21|  Premium|    E|    SI1| 59.8| 61.0|  326|3.89|3.84|2.31|
|  3| 0.23|     Good|    E|    VS1| 56.9| 65.0|  327|4.05|4.07|2.31|
|  4| 0.29|  Premium|    I|    VS2| 62.4| 58.0|  334| 4.2|4.23|2.63|
|  5| 0.31|     Good|    J|    SI2| 63.3| 58.0|  335|4.34|4.35|2.75|
|  6| 0.24|Very Good|    J|   VVS2| 62.8| 57.0|  336|3.94|3.96|2.48|
|  7| 0.24|Very Good|    I|   VVS1| 62.3| 57.0|  336|3.95|3.98|2.47|
|  8| 0.26|Very Good|    H|    SI1| 61.9| 55.0|  337|4.07|4.11|2.53|
|  9| 0.22|     Fair|    E|    VS2| 65.1| 61.0|  337|3.87|3.78|2.49|
| 10| 0.23|Very Good|    H|    VS1| 59.4| 61.0|  338| 4.0|4.05|2.39|
+---+-----+---------+-----+-------+-----+-----+-----+----+----+----+
only showing top 10 rows

If you notice the schema of diamondsRawDF you will see that the automatic schema inference of SqlContext.read method has cast the values in the column price as integer.

To cleanup:

  • let's recast the column price as double for downstream ML tasks later and
  • let's also get rid of the first column of row indices.
import org.apache.spark.sql.types.DoubleType
//we will convert price column from int to double for being able to model, fit and predict in downstream ML task
val diamondsDF = diamondsRawDF
                  .select($"carat", $"cut", $"color", $"clarity", $"depth", $"table",$"price".cast(DoubleType).as("price"), $"x", $"y", $"z")
diamondsDF.cache() // let's cache it for reuse
diamondsDF.printSchema // print schema
root
 |-- carat: double (nullable = true)
 |-- cut: string (nullable = true)
 |-- color: string (nullable = true)
 |-- clarity: string (nullable = true)
 |-- depth: double (nullable = true)
 |-- table: double (nullable = true)
 |-- price: double (nullable = true)
 |-- x: double (nullable = true)
 |-- y: double (nullable = true)
 |-- z: double (nullable = true)

import org.apache.spark.sql.types.DoubleType
diamondsDF: org.apache.spark.sql.DataFrame = [carat: double, cut: string ... 8 more fields]
diamondsDF.show(10,false) // notice that price column has Double values that end in '.0' now
+-----+---------+-----+-------+-----+-----+-----+----+----+----+
|carat|cut      |color|clarity|depth|table|price|x   |y   |z   |
+-----+---------+-----+-------+-----+-----+-----+----+----+----+
|0.23 |Ideal    |E    |SI2    |61.5 |55.0 |326.0|3.95|3.98|2.43|
|0.21 |Premium  |E    |SI1    |59.8 |61.0 |326.0|3.89|3.84|2.31|
|0.23 |Good     |E    |VS1    |56.9 |65.0 |327.0|4.05|4.07|2.31|
|0.29 |Premium  |I    |VS2    |62.4 |58.0 |334.0|4.2 |4.23|2.63|
|0.31 |Good     |J    |SI2    |63.3 |58.0 |335.0|4.34|4.35|2.75|
|0.24 |Very Good|J    |VVS2   |62.8 |57.0 |336.0|3.94|3.96|2.48|
|0.24 |Very Good|I    |VVS1   |62.3 |57.0 |336.0|3.95|3.98|2.47|
|0.26 |Very Good|H    |SI1    |61.9 |55.0 |337.0|4.07|4.11|2.53|
|0.22 |Fair     |E    |VS2    |65.1 |61.0 |337.0|3.87|3.78|2.49|
|0.23 |Very Good|H    |VS1    |59.4 |61.0 |338.0|4.0 |4.05|2.39|
+-----+---------+-----+-------+-----+-----+-----+----+----+----+
only showing top 10 rows
//View DataFrame in databricks
// note this 'display' is a databricks notebook specific command that is quite powerful for visual interaction with the data
// other notebooks like zeppelin have similar commands for interactive visualisation
display(diamondsDF) 
carat cut color clarity depth table price x y z
0.23 Ideal E SI2 61.5 55.0 326.0 3.95 3.98 2.43
0.21 Premium E SI1 59.8 61.0 326.0 3.89 3.84 2.31
0.23 Good E VS1 56.9 65.0 327.0 4.05 4.07 2.31
0.29 Premium I VS2 62.4 58.0 334.0 4.2 4.23 2.63
0.31 Good J SI2 63.3 58.0 335.0 4.34 4.35 2.75
0.24 Very Good J VVS2 62.8 57.0 336.0 3.94 3.96 2.48
0.24 Very Good I VVS1 62.3 57.0 336.0 3.95 3.98 2.47
0.26 Very Good H SI1 61.9 55.0 337.0 4.07 4.11 2.53
0.22 Fair E VS2 65.1 61.0 337.0 3.87 3.78 2.49
0.23 Very Good H VS1 59.4 61.0 338.0 4.0 4.05 2.39
0.3 Good J SI1 64.0 55.0 339.0 4.25 4.28 2.73
0.23 Ideal J VS1 62.8 56.0 340.0 3.93 3.9 2.46
0.22 Premium F SI1 60.4 61.0 342.0 3.88 3.84 2.33
0.31 Ideal J SI2 62.2 54.0 344.0 4.35 4.37 2.71
0.2 Premium E SI2 60.2 62.0 345.0 3.79 3.75 2.27
0.32 Premium E I1 60.9 58.0 345.0 4.38 4.42 2.68
0.3 Ideal I SI2 62.0 54.0 348.0 4.31 4.34 2.68
0.3 Good J SI1 63.4 54.0 351.0 4.23 4.29 2.7
0.3 Good J SI1 63.8 56.0 351.0 4.23 4.26 2.71
0.3 Very Good J SI1 62.7 59.0 351.0 4.21 4.27 2.66
0.3 Good I SI2 63.3 56.0 351.0 4.26 4.3 2.71
0.23 Very Good E VS2 63.8 55.0 352.0 3.85 3.92 2.48
0.23 Very Good H VS1 61.0 57.0 353.0 3.94 3.96 2.41
0.31 Very Good J SI1 59.4 62.0 353.0 4.39 4.43 2.62
0.31 Very Good J SI1 58.1 62.0 353.0 4.44 4.47 2.59
0.23 Very Good G VVS2 60.4 58.0 354.0 3.97 4.01 2.41
0.24 Premium I VS1 62.5 57.0 355.0 3.97 3.94 2.47
0.3 Very Good J VS2 62.2 57.0 357.0 4.28 4.3 2.67
0.23 Very Good D VS2 60.5 61.0 357.0 3.96 3.97 2.4
0.23 Very Good F VS1 60.9 57.0 357.0 3.96 3.99 2.42
0.23 Very Good F VS1 60.0 57.0 402.0 4.0 4.03 2.41
0.23 Very Good F VS1 59.8 57.0 402.0 4.04 4.06 2.42
0.23 Very Good E VS1 60.7 59.0 402.0 3.97 4.01 2.42
0.23 Very Good E VS1 59.5 58.0 402.0 4.01 4.06 2.4
0.23 Very Good D VS1 61.9 58.0 402.0 3.92 3.96 2.44
0.23 Good F VS1 58.2 59.0 402.0 4.06 4.08 2.37
0.23 Good E VS1 64.1 59.0 402.0 3.83 3.85 2.46
0.31 Good H SI1 64.0 54.0 402.0 4.29 4.31 2.75
0.26 Very Good D VS2 60.8 59.0 403.0 4.13 4.16 2.52
0.33 Ideal I SI2 61.8 55.0 403.0 4.49 4.51 2.78
0.33 Ideal I SI2 61.2 56.0 403.0 4.49 4.5 2.75
0.33 Ideal J SI1 61.1 56.0 403.0 4.49 4.55 2.76
0.26 Good D VS2 65.2 56.0 403.0 3.99 4.02 2.61
0.26 Good D VS1 58.4 63.0 403.0 4.19 4.24 2.46
0.32 Good H SI2 63.1 56.0 403.0 4.34 4.37 2.75
0.29 Premium F SI1 62.4 58.0 403.0 4.24 4.26 2.65
0.32 Very Good H SI2 61.8 55.0 403.0 4.35 4.42 2.71
0.32 Good H SI2 63.8 56.0 403.0 4.36 4.38 2.79
0.25 Very Good E VS2 63.3 60.0 404.0 4.0 4.03 2.54
0.29 Very Good H SI2 60.7 60.0 404.0 4.33 4.37 2.64
0.24 Very Good F SI1 60.9 61.0 404.0 4.02 4.03 2.45
0.23 Ideal G VS1 61.9 54.0 404.0 3.93 3.95 2.44
0.32 Ideal I SI1 60.9 55.0 404.0 4.45 4.48 2.72
0.22 Premium E VS2 61.6 58.0 404.0 3.93 3.89 2.41
0.22 Premium D VS2 59.3 62.0 404.0 3.91 3.88 2.31
0.3 Ideal I SI2 61.0 59.0 405.0 4.3 4.33 2.63
0.3 Premium J SI2 59.3 61.0 405.0 4.43 4.38 2.61
0.3 Very Good I SI1 62.6 57.0 405.0 4.25 4.28 2.67
0.3 Very Good I SI1 63.0 57.0 405.0 4.28 4.32 2.71
0.3 Good I SI1 63.2 55.0 405.0 4.25 4.29 2.7
0.35 Ideal I VS1 60.9 57.0 552.0 4.54 4.59 2.78
0.3 Premium D SI1 62.6 59.0 552.0 4.23 4.27 2.66
0.3 Ideal D SI1 62.5 57.0 552.0 4.29 4.32 2.69
0.3 Ideal D SI1 62.1 56.0 552.0 4.3 4.33 2.68
0.42 Premium I SI2 61.5 59.0 552.0 4.78 4.84 2.96
0.28 Ideal G VVS2 61.4 56.0 553.0 4.19 4.22 2.58
0.32 Ideal I VVS1 62.0 55.3 553.0 4.39 4.42 2.73
0.31 Very Good G SI1 63.3 57.0 553.0 4.33 4.3 2.73
0.31 Premium G SI1 61.8 58.0 553.0 4.35 4.32 2.68
0.24 Premium E VVS1 60.7 58.0 553.0 4.01 4.03 2.44
0.24 Very Good D VVS1 61.5 60.0 553.0 3.97 4.0 2.45
0.3 Very Good H SI1 63.1 56.0 554.0 4.29 4.27 2.7
0.3 Premium H SI1 62.9 59.0 554.0 4.28 4.24 2.68
0.3 Premium H SI1 62.5 57.0 554.0 4.29 4.25 2.67
0.3 Good H SI1 63.7 57.0 554.0 4.28 4.26 2.72
0.26 Very Good F VVS2 59.2 60.0 554.0 4.19 4.22 2.49
0.26 Very Good E VVS2 59.9 58.0 554.0 4.15 4.23 2.51
0.26 Very Good D VVS2 62.4 54.0 554.0 4.08 4.13 2.56
0.26 Very Good D VVS2 62.8 60.0 554.0 4.01 4.05 2.53
0.26 Very Good E VVS1 62.6 59.0 554.0 4.06 4.09 2.55
0.26 Very Good E VVS1 63.4 59.0 554.0 4.0 4.04 2.55
0.26 Very Good D VVS1 62.1 60.0 554.0 4.03 4.12 2.53
0.26 Ideal E VVS2 62.9 58.0 554.0 4.02 4.06 2.54
0.38 Ideal I SI2 61.6 56.0 554.0 4.65 4.67 2.87
0.26 Good E VVS1 57.9 60.0 554.0 4.22 4.25 2.45
0.24 Premium G VVS1 62.3 59.0 554.0 3.95 3.92 2.45
0.24 Premium H VVS1 61.2 58.0 554.0 4.01 3.96 2.44
0.24 Premium H VVS1 60.8 59.0 554.0 4.02 4.0 2.44
0.24 Premium H VVS2 60.7 58.0 554.0 4.07 4.04 2.46
0.32 Premium I SI1 62.9 58.0 554.0 4.35 4.33 2.73
0.7 Ideal E SI1 62.5 57.0 2757.0 5.7 5.72 3.57
0.86 Fair E SI2 55.1 69.0 2757.0 6.45 6.33 3.52
0.7 Ideal G VS2 61.6 56.0 2757.0 5.7 5.67 3.5
0.71 Very Good E VS2 62.4 57.0 2759.0 5.68 5.73 3.56
0.78 Very Good G SI2 63.8 56.0 2759.0 5.81 5.85 3.72
0.7 Good E VS2 57.5 58.0 2759.0 5.85 5.9 3.38
0.7 Good F VS1 59.4 62.0 2759.0 5.71 5.76 3.4
0.96 Fair F SI2 66.3 62.0 2759.0 6.27 5.95 4.07
0.73 Very Good E SI1 61.6 59.0 2760.0 5.77 5.78 3.56
0.8 Premium H SI1 61.5 58.0 2760.0 5.97 5.93 3.66
0.75 Very Good D SI1 63.2 56.0 2760.0 5.8 5.75 3.65
0.75 Premium E SI1 59.9 54.0 2760.0 6.0 5.96 3.58
0.74 Ideal G SI1 61.6 55.0 2760.0 5.8 5.85 3.59
0.75 Premium G VS2 61.7 58.0 2760.0 5.85 5.79 3.59
0.8 Ideal I VS1 62.9 56.0 2760.0 5.94 5.87 3.72
0.75 Ideal G SI1 62.2 55.0 2760.0 5.87 5.8 3.63
0.8 Premium G SI1 63.0 59.0 2760.0 5.9 5.81 3.69
0.74 Ideal I VVS2 62.3 55.0 2761.0 5.77 5.81 3.61
0.81 Ideal F SI2 58.8 57.0 2761.0 6.14 6.11 3.6
0.59 Ideal E VVS2 62.0 55.0 2761.0 5.38 5.43 3.35
0.8 Ideal F SI2 61.4 57.0 2761.0 5.96 6.0 3.67
0.74 Ideal E SI2 62.2 56.0 2761.0 5.8 5.84 3.62
0.9 Premium I VS2 63.0 58.0 2761.0 6.16 6.12 3.87
0.74 Very Good G SI1 62.2 59.0 2762.0 5.73 5.82 3.59
0.73 Ideal F VS2 62.6 56.0 2762.0 5.77 5.74 3.6
0.73 Ideal F VS2 62.7 53.0 2762.0 5.8 5.75 3.62
0.8 Premium F SI2 61.7 58.0 2762.0 5.98 5.94 3.68
0.71 Ideal G VS2 62.4 54.0 2762.0 5.72 5.76 3.58
0.7 Ideal E VS2 60.7 58.0 2762.0 5.73 5.76 3.49
0.8 Ideal F SI2 59.9 59.0 2762.0 6.01 6.07 3.62
0.71 Ideal D SI2 62.3 56.0 2762.0 5.73 5.69 3.56
0.74 Ideal E SI1 62.3 54.0 2762.0 5.8 5.83 3.62
0.7 Very Good F VS2 61.7 63.0 2762.0 5.64 5.61 3.47
0.7 Fair F VS2 64.5 57.0 2762.0 5.57 5.53 3.58
0.7 Fair F VS2 65.3 55.0 2762.0 5.63 5.58 3.66
0.7 Premium F VS2 61.6 60.0 2762.0 5.65 5.59 3.46
0.91 Premium H SI1 61.4 56.0 2763.0 6.09 5.97 3.7
0.61 Very Good D VVS2 59.6 57.0 2763.0 5.56 5.58 3.32
0.91 Fair H SI2 64.4 57.0 2763.0 6.11 6.09 3.93
0.91 Fair H SI2 65.7 60.0 2763.0 6.03 5.99 3.95
0.77 Ideal H VS2 62.0 56.0 2763.0 5.89 5.86 3.64
0.71 Very Good D SI1 63.6 58.0 2764.0 5.64 5.68 3.6
0.71 Ideal D SI1 61.9 59.0 2764.0 5.69 5.72 3.53
0.7 Very Good E VS2 62.6 60.0 2765.0 5.62 5.65 3.53
0.77 Very Good H VS1 61.3 60.0 2765.0 5.88 5.9 3.61
0.63 Premium E VVS1 60.9 60.0 2765.0 5.52 5.55 3.37
0.71 Very Good F VS1 60.1 62.0 2765.0 5.74 5.77 3.46
0.71 Premium F VS1 61.8 59.0 2765.0 5.69 5.73 3.53
0.76 Ideal H SI1 61.2 57.0 2765.0 5.88 5.91 3.61
0.64 Ideal G VVS1 61.9 56.0 2766.0 5.53 5.56 3.43
0.71 Premium G VS2 60.9 57.0 2766.0 5.78 5.75 3.51
0.71 Premium G VS2 59.8 56.0 2766.0 5.89 5.81 3.5
0.7 Very Good D VS2 61.8 55.0 2767.0 5.68 5.72 3.52
0.7 Very Good F VS1 60.0 57.0 2767.0 5.8 5.87 3.5
0.71 Ideal D SI2 61.6 55.0 2767.0 5.74 5.76 3.54
0.7 Good H VVS2 62.1 64.0 2767.0 5.62 5.65 3.5
0.71 Very Good G VS1 63.3 59.0 2768.0 5.52 5.61 3.52
0.73 Very Good D SI1 60.2 56.0 2768.0 5.83 5.87 3.52
0.7 Very Good D SI1 61.1 58.0 2768.0 5.66 5.73 3.48
0.7 Ideal E SI1 60.9 57.0 2768.0 5.73 5.76 3.5
0.71 Premium D SI2 61.7 59.0 2768.0 5.71 5.67 3.51
0.74 Ideal I SI1 61.3 56.0 2769.0 5.82 5.86 3.57
0.71 Premium D VS2 62.5 60.0 2770.0 5.65 5.61 3.52
0.73 Premium G VS2 61.4 59.0 2770.0 5.83 5.76 3.56
0.76 Very Good F SI1 62.9 57.0 2770.0 5.79 5.81 3.65
0.76 Ideal D SI2 62.4 57.0 2770.0 5.78 5.83 3.62
0.71 Ideal F SI1 60.7 56.0 2770.0 5.77 5.8 3.51
0.73 Premium G VS2 60.7 58.0 2770.0 5.87 5.82 3.55
0.73 Premium G VS1 61.5 58.0 2770.0 5.79 5.75 3.55
0.73 Ideal D SI2 59.9 57.0 2770.0 5.92 5.89 3.54
0.73 Premium G VS2 59.2 59.0 2770.0 5.92 5.87 3.49
0.72 Very Good H VVS2 60.3 56.0 2771.0 5.81 5.83 3.51
0.73 Very Good F SI1 61.7 60.0 2771.0 5.79 5.82 3.58
0.71 Ideal G VS2 61.9 57.0 2771.0 5.73 5.77 3.56
0.79 Ideal F SI2 61.9 55.0 2771.0 5.97 5.92 3.68
0.73 Very Good H VVS1 60.4 59.0 2772.0 5.83 5.89 3.54
0.8 Very Good F SI2 61.0 57.0 2772.0 6.01 6.03 3.67
0.58 Ideal G VVS1 61.5 55.0 2772.0 5.39 5.44 3.33
0.58 Ideal F VVS1 61.7 56.0 2772.0 5.33 5.37 3.3
0.71 Good E VS2 59.2 61.0 2772.0 5.8 5.88 3.46
0.75 Ideal D SI2 61.3 56.0 2773.0 5.85 5.89 3.6
0.7 Premium D VS2 58.0 62.0 2773.0 5.87 5.78 3.38
1.17 Very Good J I1 60.2 61.0 2774.0 6.83 6.9 4.13
0.6 Ideal E VS1 61.7 55.0 2774.0 5.41 5.44 3.35
0.7 Ideal E SI1 62.7 55.0 2774.0 5.68 5.74 3.58
0.83 Good I VS2 64.6 54.0 2774.0 5.85 5.88 3.79
0.74 Very Good F VS2 61.3 61.0 2775.0 5.8 5.84 3.57
0.72 Very Good G VS2 63.7 56.4 2776.0 5.62 5.69 3.61
0.71 Premium E VS2 62.7 58.0 2776.0 5.74 5.68 3.58
0.71 Ideal E VS2 62.2 57.0 2776.0 5.79 5.62 3.55
0.54 Ideal E VVS2 61.6 56.0 2776.0 5.25 5.27 3.24
0.54 Ideal E VVS2 61.5 57.0 2776.0 5.24 5.26 3.23
0.72 Ideal G SI1 61.8 56.0 2776.0 5.72 5.75 3.55
0.72 Ideal G SI1 60.7 56.0 2776.0 5.79 5.82 3.53
0.72 Good G VS2 59.7 60.5 2776.0 5.8 5.84 3.47
0.71 Ideal G SI1 60.5 56.0 2776.0 5.8 5.76 3.5
0.7 Very Good D VS1 62.7 58.0 2777.0 5.66 5.73 3.57
0.71 Premium F VS2 62.1 58.0 2777.0 5.67 5.7 3.53
0.71 Very Good F VS2 62.8 57.0 2777.0 5.64 5.69 3.56
0.71 Good F VS2 63.8 58.0 2777.0 5.61 5.64 3.59
0.71 Good F VS2 57.8 60.0 2777.0 5.87 5.9 3.4
0.7 Ideal E VS2 62.1 55.0 2777.0 5.7 5.67 3.53
0.7 Premium E VS2 61.1 60.0 2777.0 5.71 5.64 3.47
0.7 Premium E SI1 60.0 59.0 2777.0 5.79 5.75 3.46
0.7 Premium E SI1 61.2 57.0 2777.0 5.73 5.68 3.49
0.7 Premium E SI1 62.7 59.0 2777.0 5.67 5.63 3.54
0.7 Premium E SI1 61.0 57.0 2777.0 5.73 5.68 3.48
0.7 Premium E SI1 61.0 58.0 2777.0 5.78 5.72 3.51
0.7 Ideal E SI1 61.4 57.0 2777.0 5.76 5.7 3.52
0.72 Premium F SI1 61.8 61.0 2777.0 5.82 5.71 3.56
0.7 Very Good E SI1 59.9 63.0 2777.0 5.76 5.7 3.43
0.7 Premium E SI1 61.3 58.0 2777.0 5.71 5.68 3.49
0.7 Premium E SI1 60.5 58.0 2777.0 5.77 5.74 3.48
0.7 Good E VS2 64.1 59.0 2777.0 5.64 5.59 3.6
0.98 Fair H SI2 67.9 60.0 2777.0 6.05 5.97 4.08
0.78 Premium F SI1 62.4 58.0 2777.0 5.83 5.8 3.63
0.7 Very Good E SI1 63.2 60.0 2777.0 5.6 5.51 3.51
0.52 Ideal F VVS1 61.3 55.0 2778.0 5.19 5.22 3.19
0.73 Very Good H VS2 60.8 56.0 2779.0 5.82 5.84 3.55
0.74 Ideal E SI1 61.7 56.0 2779.0 5.84 5.8 3.59
0.7 Very Good F VS2 63.6 57.0 2780.0 5.61 5.65 3.58
0.77 Premium G VS2 61.2 58.0 2780.0 5.9 5.93 3.62
0.71 Ideal F VS2 62.1 54.0 2780.0 5.68 5.72 3.54
0.74 Ideal G VS1 61.5 55.0 2780.0 5.81 5.86 3.59
0.7 Ideal G VS1 61.4 59.0 2780.0 5.64 5.73 3.49
1.01 Premium F I1 61.8 60.0 2781.0 6.39 6.36 3.94
0.77 Ideal H SI1 62.2 56.0 2781.0 5.83 5.88 3.64
0.78 Ideal H SI1 61.2 56.0 2781.0 5.92 5.99 3.64
0.72 Very Good H VS1 60.6 63.0 2782.0 5.83 5.76 3.51
0.53 Very Good D VVS2 57.5 64.0 2782.0 5.34 5.37 3.08
0.76 Ideal G VS2 61.3 56.0 2782.0 5.9 5.94 3.63
0.7 Good E VS1 57.2 62.0 2782.0 5.81 5.77 3.31
0.7 Premium E VS1 62.9 60.0 2782.0 5.62 5.54 3.51
0.75 Very Good D SI2 63.1 58.0 2782.0 5.78 5.73 3.63
0.72 Ideal D SI1 60.8 57.0 2782.0 5.76 5.75 3.5
0.72 Premium D SI1 62.7 59.0 2782.0 5.73 5.69 3.58
0.7 Premium D SI1 62.8 60.0 2782.0 5.68 5.66 3.56
0.84 Fair G SI1 55.1 67.0 2782.0 6.39 6.2 3.47
0.75 Premium F SI1 61.4 59.0 2782.0 5.88 5.85 3.6
0.52 Ideal F IF 62.2 55.0 2783.0 5.14 5.18 3.21
0.72 Very Good F VS2 63.0 54.0 2784.0 5.69 5.73 3.6
0.79 Very Good H VS1 63.7 56.0 2784.0 5.85 5.92 3.75
0.72 Very Good F VS2 63.6 58.0 2787.0 5.66 5.69 3.61
0.51 Ideal F VVS1 62.0 57.0 2787.0 5.11 5.15 3.18
0.64 Ideal D VS1 61.5 56.0 2787.0 5.54 5.55 3.41
0.7 Very Good H VVS1 60.5 60.0 2788.0 5.74 5.77 3.48
0.83 Very Good I VS1 61.1 60.0 2788.0 6.07 6.1 3.72
0.76 Ideal I VVS2 61.8 56.0 2788.0 5.85 5.87 3.62
0.71 Good D VS2 63.3 56.0 2788.0 5.64 5.68 3.58
0.77 Good G VS1 59.4 64.0 2788.0 5.97 5.92 3.53
0.71 Ideal F SI1 62.5 55.0 2788.0 5.71 5.65 3.55
1.01 Fair E I1 64.5 58.0 2788.0 6.29 6.21 4.03
1.01 Premium H SI2 62.7 59.0 2788.0 6.31 6.22 3.93
0.77 Good F SI1 64.2 52.0 2789.0 5.81 5.77 3.72
0.76 Good E SI1 63.7 54.0 2789.0 5.76 5.85 3.7
0.76 Premium E SI1 60.4 58.0 2789.0 5.92 5.94 3.58
0.76 Premium E SI1 61.8 58.0 2789.0 5.82 5.86 3.61
1.05 Very Good J SI2 63.2 56.0 2789.0 6.49 6.45 4.09
0.81 Ideal G SI2 61.6 56.0 2789.0 5.97 6.01 3.69
0.7 Ideal E SI1 61.6 56.0 2789.0 5.72 5.75 3.53
0.55 Ideal G IF 60.9 57.0 2789.0 5.28 5.3 3.22
0.81 Good G SI2 61.0 61.0 2789.0 5.94 5.99 3.64
0.63 Premium E VVS2 62.1 57.0 2789.0 5.48 5.41 3.38
0.63 Premium E VVS1 60.9 60.0 2789.0 5.55 5.52 3.37
0.77 Premium H VS1 61.3 60.0 2789.0 5.9 5.88 3.61
1.05 Fair J SI2 65.8 59.0 2789.0 6.41 6.27 4.18
0.64 Ideal G IF 61.3 56.0 2790.0 5.54 5.58 3.41
0.76 Premium I VVS1 58.8 59.0 2790.0 6.0 5.94 3.51
0.83 Ideal F SI2 62.3 55.0 2790.0 6.02 6.05 3.76
0.71 Premium F VS1 60.1 62.0 2790.0 5.77 5.74 3.46
0.71 Premium F VS1 61.8 59.0 2790.0 5.73 5.69 3.53
0.87 Very Good I SI1 63.6 55.8 2791.0 6.07 6.1 3.87
0.73 Ideal E SI1 62.2 56.0 2791.0 5.74 5.78 3.58
0.71 Premium E SI1 59.2 59.0 2792.0 5.83 5.86 3.46
0.71 Premium E SI1 61.8 59.0 2792.0 5.7 5.75 3.54
0.71 Ideal E SI1 61.3 55.0 2792.0 5.72 5.77 3.52
0.7 Premium F VS1 62.1 60.0 2792.0 5.71 5.65 3.53
0.7 Premium F VS1 60.7 60.0 2792.0 5.78 5.75 3.5
0.76 Premium H VVS2 59.6 57.0 2792.0 5.91 5.86 3.51
0.7 Ideal F VS1 62.2 56.0 2792.0 5.73 5.68 3.55
0.79 Very Good G SI1 60.6 57.0 2793.0 5.98 6.06 3.65
0.7 Very Good E VS2 62.9 57.0 2793.0 5.66 5.69 3.57
0.7 Good E VS2 64.1 55.0 2793.0 5.6 5.66 3.61
0.76 Ideal I VS2 61.3 56.0 2793.0 5.87 5.91 3.61
0.73 Ideal H VS2 62.7 55.0 2793.0 5.72 5.76 3.6
0.79 Very Good E SI1 63.2 56.0 2794.0 5.91 5.86 3.72
0.71 Very Good E VS2 60.7 56.0 2795.0 5.81 5.82 3.53
0.81 Premium I VVS2 61.9 60.0 2795.0 5.91 5.86 3.64
0.81 Ideal F SI2 62.6 55.0 2795.0 5.92 5.96 3.72
0.72 Good F VS1 60.7 60.0 2795.0 5.74 5.72 3.48
0.72 Premium D SI2 62.0 60.0 2795.0 5.73 5.69 3.54
0.72 Premium I IF 63.0 57.0 2795.0 5.72 5.7 3.6
0.81 Premium H VS2 58.0 59.0 2795.0 6.17 6.13 3.57
0.72 Premium G VS2 62.9 57.0 2795.0 5.73 5.65 3.58
1.0 Premium I SI2 58.2 60.0 2795.0 6.61 6.55 3.83
0.73 Good E SI1 63.2 58.0 2796.0 5.7 5.76 3.62
0.81 Very Good H SI2 61.3 59.0 2797.0 5.94 6.01 3.66
0.81 Very Good E SI1 60.3 60.0 2797.0 6.07 6.1 3.67
0.71 Premium D SI1 62.7 60.0 2797.0 5.67 5.71 3.57
0.71 Premium D SI1 61.3 58.0 2797.0 5.73 5.75 3.52
0.71 Premium D SI1 61.6 60.0 2797.0 5.74 5.69 3.52
0.57 Ideal F VVS2 61.9 55.0 2797.0 5.34 5.35 3.31
0.51 Ideal D VVS1 61.7 56.0 2797.0 5.12 5.16 3.17
0.72 Ideal G VS2 61.9 58.0 2797.0 5.72 5.75 3.55
0.74 Ideal H VS1 61.8 58.0 2797.0 5.77 5.81 3.58
0.74 Ideal H VS1 61.6 56.0 2797.0 5.81 5.82 3.58
0.7 Fair G VVS1 58.8 66.0 2797.0 5.81 5.9 3.44
0.8 Premium F SI2 61.0 57.0 2797.0 6.03 6.01 3.67
1.01 Fair E SI2 67.4 60.0 2797.0 6.19 6.05 4.13
0.8 Very Good H VS2 63.4 60.0 2797.0 5.92 5.82 3.72
0.77 Ideal I VS1 61.5 59.0 2798.0 5.87 5.91 3.62
0.83 Very Good E SI2 58.0 62.0 2799.0 6.19 6.25 3.61
0.82 Ideal F SI2 62.4 54.0 2799.0 5.97 6.02 3.74
0.78 Ideal D SI1 61.9 57.0 2799.0 5.91 5.86 3.64
0.6 Very Good G IF 61.6 56.0 2800.0 5.43 5.46 3.35
0.9 Good I SI2 62.2 59.0 2800.0 6.07 6.11 3.79
0.7 Premium E VS1 62.2 58.0 2800.0 5.6 5.66 3.5
0.9 Very Good I SI2 61.3 56.0 2800.0 6.17 6.23 3.8
0.83 Ideal G SI1 62.3 57.0 2800.0 5.99 6.08 3.76
0.83 Ideal G SI1 61.8 57.0 2800.0 6.03 6.07 3.74
0.83 Very Good H SI1 62.5 59.0 2800.0 5.95 6.02 3.74
0.74 Premium G VS1 62.9 60.0 2800.0 5.74 5.68 3.59
0.79 Ideal I VS1 61.8 59.0 2800.0 5.92 5.95 3.67
0.61 Ideal G IF 62.3 56.0 2800.0 5.43 5.45 3.39
0.76 Fair G VS1 59.0 70.0 2800.0 5.89 5.8 3.46
0.96 Ideal F I1 60.7 55.0 2801.0 6.37 6.41 3.88
0.73 Ideal F VS2 62.5 55.0 2801.0 5.8 5.76 3.61
0.73 Premium F VS2 62.7 58.0 2801.0 5.76 5.7 3.59
0.75 Ideal H SI1 60.4 57.0 2801.0 5.93 5.96 3.59
0.71 Premium F VS2 62.1 58.0 2801.0 5.7 5.67 3.53
0.71 Good F VS2 57.8 60.0 2801.0 5.9 5.87 3.4
0.71 Good F VS2 63.8 58.0 2801.0 5.64 5.61 3.59
0.71 Premium F VS2 62.8 57.0 2801.0 5.69 5.64 3.56
1.04 Premium G I1 62.2 58.0 2801.0 6.46 6.41 4.0
1.0 Premium J SI2 62.3 58.0 2801.0 6.45 6.34 3.98
0.87 Very Good G SI2 59.9 58.0 2802.0 6.19 6.23 3.72
0.53 Ideal F IF 61.9 54.0 2802.0 5.22 5.25 3.24
0.72 Premium E VS2 63.0 55.0 2802.0 5.79 5.61 3.59
0.72 Premium F VS1 62.4 58.0 2802.0 5.83 5.7 3.6
0.7 Very Good F VS2 62.9 58.0 2803.0 5.63 5.65 3.55
0.74 Very Good E SI1 63.5 56.0 2803.0 5.74 5.79 3.66
0.71 Ideal G VS2 61.3 56.0 2803.0 5.75 5.71 3.51
0.73 Ideal E SI1 60.6 54.0 2803.0 5.84 5.89 3.55
0.7 Good G VS1 65.1 58.0 2803.0 5.56 5.59 3.63
0.71 Premium F VS2 62.6 58.0 2803.0 5.7 5.67 3.56
0.71 Premium F VS2 58.0 62.0 2803.0 5.85 5.81 3.38
0.71 Premium G VS1 62.4 61.0 2803.0 5.7 5.65 3.54
0.77 Premium G VS2 61.3 57.0 2803.0 5.93 5.88 3.62
0.71 Premium G VS2 59.9 60.0 2803.0 5.81 5.77 3.47
0.78 Premium G VS2 60.8 58.0 2803.0 6.03 5.95 3.64
0.71 Very Good G VS1 63.5 55.0 2803.0 5.66 5.64 3.59
0.91 Ideal D SI2 62.2 57.0 2803.0 6.21 6.15 3.85
0.71 Very Good E VS2 63.8 58.0 2804.0 5.62 5.66 3.6
0.71 Very Good E VS2 64.0 57.0 2804.0 5.66 5.68 3.63
0.8 Very Good E SI2 62.5 56.0 2804.0 5.88 5.96 3.7
0.7 Very Good D SI1 62.3 58.0 2804.0 5.69 5.73 3.56
0.72 Ideal F VS1 61.7 57.0 2804.0 5.74 5.77 3.55
0.72 Very Good F VS1 62.2 58.0 2804.0 5.75 5.7 3.56
0.82 Ideal H VS2 61.5 56.0 2804.0 6.01 6.08 3.72
0.7 Ideal D SI1 61.0 59.0 2804.0 5.68 5.7 3.47
0.72 Ideal D SI1 62.2 56.0 2804.0 5.74 5.77 3.58
0.72 Ideal D SI1 61.5 54.0 2804.0 5.77 5.8 3.56
0.9 Fair I SI1 67.3 59.0 2804.0 5.93 5.84 3.96
0.74 Premium F VS2 61.7 58.0 2805.0 5.85 5.78 3.59
0.74 Premium F VS2 61.9 56.0 2805.0 5.8 5.77 3.58
0.73 Ideal E SI2 61.8 58.0 2805.0 5.77 5.81 3.58
0.57 Fair E VVS1 58.7 66.0 2805.0 5.34 5.43 3.16
0.73 Premium F VS2 62.5 57.0 2805.0 5.75 5.7 3.58
0.72 Ideal G VS2 62.8 56.0 2805.0 5.74 5.7 3.59
0.74 Fair F VS2 61.1 68.0 2805.0 5.82 5.75 3.53
0.82 Good G VS2 64.0 57.0 2805.0 5.92 5.89 3.78
0.81 Very Good G SI1 62.5 60.0 2806.0 5.89 5.94 3.69
0.75 Very Good H VVS1 60.6 58.0 2806.0 5.85 5.9 3.56
0.7 Ideal F SI1 61.6 55.0 2806.0 5.72 5.74 3.53
0.71 Very Good F VS1 62.2 58.0 2807.0 5.66 5.72 3.54
0.71 Very Good F VS1 60.0 57.0 2807.0 5.84 5.9 3.52
0.93 Premium J SI2 61.9 57.0 2807.0 6.21 6.19 3.84
0.8 Very Good H VS2 62.8 57.0 2808.0 5.87 5.91 3.7
0.7 Very Good F VS1 62.0 57.0 2808.0 5.64 5.71 3.52
1.0 Fair G I1 66.4 59.0 2808.0 6.16 6.09 4.07
0.75 Very Good G VS2 63.4 56.0 2808.0 5.78 5.74 3.65
0.58 Ideal E VVS2 60.9 56.0 2808.0 5.41 5.43 3.3
0.73 Very Good D SI1 63.1 57.0 2808.0 5.74 5.7 3.61
0.81 Very Good F SI1 63.1 59.0 2809.0 5.85 5.79 3.67
0.81 Premium D SI2 59.2 57.0 2809.0 6.15 6.05 3.61
0.71 Premium F SI1 60.7 54.0 2809.0 5.84 5.8 3.53
1.2 Fair F I1 64.6 56.0 2809.0 6.73 6.66 4.33
0.7 Very Good F VS1 61.8 56.0 2810.0 5.63 5.7 3.5
0.7 Very Good F VS1 59.9 60.0 2810.0 5.77 5.84 3.48
0.74 Ideal D SI2 61.7 55.0 2810.0 5.81 5.85 3.6
0.7 Good F VS1 62.8 61.0 2810.0 5.57 5.61 3.51
0.8 Good G SI1 62.7 57.0 2810.0 5.84 5.93 3.69
0.75 Very Good F SI1 63.4 58.0 2811.0 5.72 5.76 3.64
0.83 Very Good D SI1 63.5 54.0 2811.0 5.98 5.95 3.79
1.0 Fair J VS2 65.7 59.0 2811.0 6.14 6.07 4.01
0.99 Fair I SI2 68.1 56.0 2811.0 6.21 6.06 4.18
0.7 Very Good G VS1 63.0 60.0 2812.0 5.57 5.64 3.53
0.7 Very Good F VS2 59.5 58.0 2812.0 5.75 5.85 3.45
0.7 Good E SI1 63.5 59.0 2812.0 5.49 5.53 3.5
0.7 Very Good F VS2 61.7 58.0 2812.0 5.63 5.69 3.49
0.32 Premium I SI1 62.7 58.0 554.0 4.37 4.34 2.73
0.32 Premium I SI1 62.8 58.0 554.0 4.39 4.34 2.74
0.32 Ideal I SI1 62.4 57.0 554.0 4.37 4.35 2.72
0.32 Premium I SI1 61.0 59.0 554.0 4.39 4.36 2.67
0.32 Very Good I SI1 63.1 56.0 554.0 4.39 4.36 2.76
0.32 Ideal I SI1 60.7 57.0 554.0 4.47 4.42 2.7
0.3 Premium H SI1 60.9 59.0 554.0 4.31 4.29 2.62
0.3 Premium H SI1 60.1 55.0 554.0 4.41 4.38 2.64
0.3 Premium H SI1 62.9 58.0 554.0 4.28 4.24 2.68
0.3 Very Good H SI1 63.3 56.0 554.0 4.29 4.27 2.71
0.3 Good H SI1 63.8 55.0 554.0 4.26 4.2 2.7
0.3 Ideal H SI1 62.9 57.0 554.0 4.27 4.22 2.67
0.3 Very Good H SI1 63.4 60.0 554.0 4.25 4.23 2.69
0.32 Good I SI1 63.9 55.0 554.0 4.36 4.34 2.78
0.33 Ideal H SI2 61.4 56.0 554.0 4.85 4.79 2.95
0.29 Very Good E VS1 61.9 55.0 555.0 4.28 4.33 2.66
0.29 Very Good E VS1 62.4 55.0 555.0 4.2 4.25 2.63
0.31 Very Good F SI1 61.8 58.0 555.0 4.32 4.35 2.68
0.34 Ideal H VS2 61.5 56.0 555.0 4.47 4.5 2.76
0.34 Ideal H VS2 60.4 57.0 555.0 4.54 4.57 2.75
0.34 Ideal I VS1 61.8 55.0 555.0 4.48 4.52 2.78
0.34 Ideal I VS1 62.0 56.0 555.0 4.5 4.53 2.8
0.3 Ideal G VS1 62.3 56.0 555.0 4.29 4.31 2.68
0.29 Ideal F VS1 61.6 56.0 555.0 4.26 4.31 2.64
0.35 Ideal G SI1 60.6 56.0 555.0 4.56 4.58 2.77
0.43 Very Good E I1 58.4 62.0 555.0 4.94 5.0 2.9
0.32 Very Good F VS2 61.4 58.0 556.0 4.37 4.42 2.7
0.36 Ideal I VS2 61.9 56.0 556.0 4.54 4.57 2.82
0.3 Ideal G VS2 62.0 56.0 556.0 4.28 4.3 2.66
0.26 Ideal E VS1 61.5 57.0 556.0 4.09 4.12 2.52
0.7 Very Good F VS2 62.3 58.0 2812.0 5.64 5.72 3.54
0.7 Very Good F VS2 60.9 61.0 2812.0 5.66 5.71 3.46
0.71 Ideal D SI1 62.4 57.0 2812.0 5.69 5.72 3.56
0.99 Fair J SI1 55.0 61.0 2812.0 6.72 6.67 3.68
0.73 Premium E VS2 58.6 60.0 2812.0 5.92 5.89 3.46
0.51 Ideal F VVS1 62.0 57.0 2812.0 5.15 5.11 3.18
0.91 Premium G SI2 59.8 58.0 2813.0 6.3 6.29 3.77
0.84 Very Good E SI1 63.4 55.0 2813.0 6.0 5.95 3.79
0.91 Good I VS2 64.3 58.0 2813.0 6.09 6.05 3.9
0.76 Premium E SI1 62.2 59.0 2814.0 5.86 5.81 3.63
0.76 Ideal E SI1 61.7 57.0 2814.0 5.88 5.85 3.62
0.75 Premium E SI1 61.1 59.0 2814.0 5.86 5.83 3.57
0.55 Very Good D VVS1 61.5 56.0 2815.0 5.23 5.27 3.23
0.76 Very Good F SI2 58.5 62.0 2815.0 5.93 6.01 3.49
0.74 Premium G VS1 61.7 58.0 2815.0 5.79 5.81 3.58
0.7 Ideal H SI1 60.4 56.0 2815.0 5.75 5.81 3.49
0.7 Ideal H SI1 61.4 56.0 2815.0 5.7 5.76 3.52
0.7 Ideal H SI1 61.5 55.0 2815.0 5.73 5.79 3.54
0.7 Ideal H SI1 61.4 56.0 2815.0 5.72 5.77 3.53
0.9 Fair J VS2 65.0 56.0 2815.0 6.08 6.04 3.94
0.95 Fair F SI2 56.0 60.0 2815.0 6.62 6.53 3.68
0.89 Premium H SI2 60.2 59.0 2815.0 6.26 6.23 3.76
0.72 Premium E VS2 58.3 58.0 2815.0 5.99 5.92 3.47
0.96 Fair E SI2 53.1 63.0 2815.0 6.73 6.65 3.55
1.02 Premium G I1 60.3 58.0 2815.0 6.55 6.5 3.94
0.78 Very Good I VVS2 61.4 56.0 2816.0 5.91 5.95 3.64
0.61 Ideal G VVS2 60.1 57.0 2816.0 5.52 5.54 3.32
0.71 Good D VS1 63.4 55.0 2816.0 5.61 5.69 3.58
0.78 Premium F SI1 61.5 59.0 2816.0 5.96 5.88 3.64
0.87 Ideal H SI2 62.7 56.0 2816.0 6.16 6.13 3.85
0.83 Ideal H SI1 62.5 55.0 2816.0 6.04 6.0 3.76
0.71 Premium E SI1 61.3 56.0 2817.0 5.78 5.73 3.53
0.71 Ideal I VVS2 60.2 56.0 2817.0 5.84 5.89 3.53
0.71 Ideal E VS2 62.7 57.0 2817.0 5.66 5.64 3.54
0.71 Premium E VS2 62.3 58.0 2817.0 5.69 5.65 3.53
0.63 Ideal F VVS2 61.5 56.0 2817.0 5.48 5.52 3.38
0.71 Premium E SI1 59.2 59.0 2817.0 5.86 5.83 3.46
0.71 Premium E SI1 61.8 59.0 2817.0 5.75 5.7 3.54
0.71 Ideal E SI1 61.3 55.0 2817.0 5.77 5.72 3.52
0.71 Premium E SI1 61.4 58.0 2817.0 5.77 5.73 3.53
0.9 Ideal J VS2 62.8 55.0 2817.0 6.2 6.16 3.88
0.71 Good E SI1 62.8 64.0 2817.0 5.6 5.54 3.5
0.7 Premium E VS2 62.4 61.0 2818.0 5.66 5.63 3.52
0.7 Premium E VS2 59.3 60.0 2818.0 5.78 5.73 3.41
0.7 Premium E VS2 63.0 60.0 2818.0 5.64 5.6 3.54
1.0 Premium H I1 61.3 60.0 2818.0 6.43 6.39 3.93
0.86 Premium F SI2 59.3 62.0 2818.0 6.36 6.22 3.73
0.8 Ideal H SI1 61.0 57.0 2818.0 6.07 6.0 3.68
0.7 Ideal E VS1 62.9 57.0 2818.0 5.66 5.61 3.54
0.7 Premium E VS1 59.6 57.0 2818.0 5.91 5.83 3.5
0.7 Premium F VS2 61.8 60.0 2818.0 5.69 5.64 3.5
0.7 Premium E VS1 62.7 57.0 2818.0 5.68 5.64 3.55
1.0 Fair H SI2 65.3 62.0 2818.0 6.34 6.12 4.08
0.72 Very Good G VS1 63.8 58.0 2819.0 5.64 5.68 3.61
0.72 Ideal H VS1 62.3 56.0 2819.0 5.73 5.77 3.58
0.7 Good F VS1 59.7 63.0 2819.0 5.76 5.79 3.45
0.86 Good F SI2 64.3 60.0 2819.0 5.97 5.95 3.83
0.71 Ideal G VS1 62.9 58.0 2820.0 5.66 5.69 3.57
0.75 Ideal E SI1 62.0 57.0 2821.0 5.8 5.78 3.59
0.73 Premium E VS2 61.6 59.0 2821.0 5.77 5.73 3.54
0.53 Ideal E VVS1 61.9 55.0 2821.0 5.2 5.21 3.22
0.73 Premium E SI1 61.3 58.0 2821.0 5.83 5.76 3.55
0.73 Good E SI1 63.6 57.0 2821.0 5.72 5.7 3.63
0.73 Premium E SI1 59.6 61.0 2821.0 5.92 5.85 3.51
0.73 Premium E SI1 62.2 59.0 2821.0 5.77 5.68 3.56
0.73 Premium D SI1 61.7 55.0 2821.0 5.84 5.82 3.6
0.73 Very Good E SI1 63.2 58.0 2821.0 5.76 5.7 3.62
0.7 Premium E VS1 60.8 60.0 2822.0 5.74 5.71 3.48
0.72 Premium E VS2 60.3 59.0 2822.0 5.84 5.8 3.51
0.72 Premium E VS2 60.9 60.0 2822.0 5.8 5.76 3.52
0.72 Premium E VS2 62.4 59.0 2822.0 5.77 5.7 3.58
0.7 Premium E VS2 60.2 60.0 2822.0 5.73 5.7 3.44
0.6 Ideal F VVS2 62.0 55.0 2822.0 5.37 5.4 3.34
0.74 Ideal I VVS1 60.8 57.0 2822.0 5.85 5.89 3.57
0.73 Ideal F SI1 62.1 55.0 2822.0 5.75 5.78 3.58
0.71 Premium D SI1 62.7 60.0 2822.0 5.71 5.67 3.57
0.71 Premium D SI1 61.3 58.0 2822.0 5.75 5.73 3.52
0.7 Premium D SI1 60.2 60.0 2822.0 5.82 5.75 3.48
0.7 Ideal D SI1 60.7 56.0 2822.0 5.75 5.72 3.48
0.9 Good J VS2 64.0 61.0 2822.0 6.04 6.03 3.86
0.71 Ideal D SI1 60.2 56.0 2822.0 5.86 5.83 3.52
0.7 Premium E VS2 61.5 59.0 2822.0 5.73 5.68 3.51
0.7 Premium E VS2 62.6 56.0 2822.0 5.71 5.66 3.56
0.7 Ideal D SI1 59.7 58.0 2822.0 5.82 5.77 3.46
0.7 Good E SI1 61.4 64.0 2822.0 5.71 5.66 3.49
0.7 Ideal D SI1 62.5 57.0 2822.0 5.62 5.59 3.51
0.7 Ideal D SI1 61.8 56.0 2822.0 5.73 5.63 3.51
0.7 Premium E VS2 60.7 62.0 2822.0 5.72 5.68 3.46
0.7 Premium F VS2 60.6 58.0 2822.0 5.8 5.72 3.49
0.7 Ideal D SI1 61.4 54.0 2822.0 5.75 5.71 3.52
0.79 Very Good D SI2 62.8 59.0 2823.0 5.86 5.9 3.69
0.9 Good I SI1 63.8 57.0 2823.0 6.06 6.13 3.89
0.71 Premium E VS2 62.3 58.0 2823.0 5.71 5.66 3.54
0.61 Ideal E VVS2 61.3 54.0 2823.0 5.51 5.59 3.4
0.9 Fair H SI2 65.8 54.0 2823.0 6.05 5.98 3.96
0.71 Ideal E SI1 60.5 56.0 2823.0 5.77 5.73 3.47
0.71 Premium D VS2 61.2 59.0 2824.0 5.74 5.69 3.5
0.77 Ideal I VVS2 62.1 57.0 2824.0 5.84 5.86 3.63
0.74 Good E VS1 63.1 58.0 2824.0 5.73 5.75 3.62
0.82 Ideal F SI2 62.4 54.0 2824.0 6.02 5.97 3.74
0.82 Premium E SI2 60.8 60.0 2824.0 6.05 6.03 3.67
0.71 Premium G VS1 62.2 59.0 2825.0 5.73 5.66 3.54
0.83 Premium H SI1 60.0 59.0 2825.0 6.08 6.05 3.64
0.73 Very Good G VS1 62.0 57.0 2825.0 5.75 5.8 3.58
0.83 Premium H SI1 62.5 59.0 2825.0 6.02 5.95 3.74
1.17 Premium J I1 60.2 61.0 2825.0 6.9 6.83 4.13
0.91 Fair H SI2 61.3 67.0 2825.0 6.24 6.19 3.81
0.73 Premium E VS1 62.6 60.0 2826.0 5.75 5.68 3.58
0.7 Good E VS1 57.2 59.0 2826.0 5.94 5.88 3.38
0.9 Premium I SI2 62.2 59.0 2826.0 6.11 6.07 3.79
0.7 Premium E VS1 62.2 58.0 2826.0 5.66 5.6 3.5
0.7 Very Good D VS2 63.3 56.0 2826.0 5.6 5.58 3.54
0.7 Premium E VS1 59.4 61.0 2826.0 5.78 5.74 3.42
0.9 Very Good I SI2 63.5 56.0 2826.0 6.17 6.07 3.88
0.78 Premium F SI1 60.8 60.0 2826.0 5.97 5.94 3.62
0.96 Ideal F I1 60.7 55.0 2826.0 6.41 6.37 3.88
0.7 Very Good D SI1 62.3 59.0 2827.0 5.67 5.7 3.54
0.72 Good D VS2 64.0 54.0 2827.0 5.68 5.7 3.64
0.79 Premium H VVS2 62.6 58.0 2827.0 5.96 5.9 3.71
0.7 Ideal H VVS1 61.6 57.0 2827.0 5.69 5.74 3.52
0.7 Ideal H VVS1 62.3 55.0 2827.0 5.66 5.7 3.54
0.7 Ideal D SI2 60.6 57.0 2828.0 5.74 5.77 3.49
1.01 Premium H SI2 61.6 61.0 2828.0 6.39 6.31 3.91
0.72 Premium F VS1 62.2 58.0 2829.0 5.75 5.7 3.56
0.8 Good E SI2 63.7 54.0 2829.0 5.91 5.87 3.75
0.59 Ideal E VVS1 62.0 56.0 2829.0 5.36 5.38 3.33
0.72 Ideal F VS1 61.7 57.0 2829.0 5.77 5.74 3.55
0.75 Premium E SI2 61.9 57.0 2829.0 5.88 5.82 3.62
0.8 Premium E SI2 60.2 57.0 2829.0 6.05 6.01 3.63
0.71 Very Good E VS2 62.7 59.0 2830.0 5.65 5.7 3.56
0.77 Very Good H SI1 61.7 56.0 2830.0 5.84 5.89 3.62
0.97 Ideal F I1 60.7 56.0 2830.0 6.41 6.43 3.9
0.53 Ideal F VVS1 60.9 57.0 2830.0 5.23 5.29 3.19
0.53 Ideal F VVS1 61.8 57.0 2830.0 5.16 5.19 3.2
0.8 Ideal I VS2 62.1 54.4 2830.0 5.94 5.99 3.7
0.9 Premium G SI1 60.6 62.0 2830.0 6.21 6.13 3.74
0.76 Very Good E SI2 60.8 60.0 2831.0 5.89 5.98 3.61
0.72 Ideal E SI1 62.3 57.0 2831.0 5.7 5.76 3.57
0.75 Ideal E SI1 61.4 57.0 2831.0 5.82 5.87 3.59
0.72 Premium E SI1 62.1 58.0 2831.0 5.73 5.76 3.57
0.79 Ideal G SI1 61.8 56.0 2831.0 5.93 5.91 3.66
0.72 Very Good F VS2 62.5 58.0 2832.0 5.71 5.75 3.58
0.91 Very Good I SI2 62.8 61.0 2832.0 6.15 6.18 3.87
0.71 Premium G VVS2 62.1 57.0 2832.0 5.75 5.65 3.54
0.81 Premium G SI1 63.0 60.0 2832.0 5.87 5.81 3.68
0.82 Ideal H SI1 62.5 57.0 2832.0 5.91 5.97 3.71
0.71 Premium F VS1 62.2 58.0 2832.0 5.72 5.66 3.54
0.9 Good J SI1 64.3 63.0 2832.0 6.05 6.01 3.88
0.8 Very Good I VS2 62.0 58.0 2833.0 5.86 5.95 3.66
0.56 Very Good E IF 61.0 59.0 2833.0 5.28 5.34 3.24
0.7 Very Good D VS2 59.6 61.0 2833.0 5.77 5.8 3.45
0.7 Ideal D VS2 61.0 57.0 2833.0 5.74 5.76 3.51
0.61 Ideal F VVS2 61.7 55.0 2833.0 5.45 5.48 3.37
0.85 Ideal H SI2 62.5 57.0 2833.0 6.02 6.07 3.78
0.7 Ideal F SI1 60.7 57.0 2833.0 5.73 5.75 3.49
0.8 Ideal G VS2 62.2 56.0 2834.0 5.94 5.87 3.67
0.8 Ideal H VS2 62.8 57.0 2834.0 5.91 5.87 3.7
0.51 Very Good D VVS1 59.9 58.0 2834.0 5.16 5.19 3.1
0.53 Ideal F VVS1 61.4 57.0 2834.0 5.2 5.23 3.2
0.78 Ideal I VS2 61.8 55.0 2834.0 5.92 5.95 3.67
0.9 Very Good J SI1 63.4 54.0 2834.0 6.17 6.14 3.9
0.9 Fair G SI2 65.3 59.0 2834.0 6.07 6.0 3.94
0.77 Ideal E SI2 60.7 55.0 2834.0 6.01 5.95 3.63
0.73 Ideal F VS1 61.2 56.0 2835.0 5.89 5.81 3.58
0.63 Ideal F VVS2 61.9 57.0 2835.0 5.47 5.51 3.4
0.7 Ideal E VS2 61.5 54.0 2835.0 5.7 5.75 3.52
0.72 Ideal E VS2 62.8 57.0 2835.0 5.71 5.73 3.59
0.72 Ideal E SI1 61.0 57.0 2835.0 5.78 5.8 3.53
0.75 Premium F VS2 59.6 59.0 2835.0 6.04 5.94 3.57
0.82 Very Good H SI1 60.7 56.0 2836.0 6.04 6.06 3.67
0.71 Good E VS2 62.8 60.0 2836.0 5.6 5.65 3.53
0.7 Premium E VS1 62.6 59.0 2837.0 5.69 5.66 3.55
0.7 Ideal E VS1 61.8 56.0 2837.0 5.74 5.69 3.53
0.71 Ideal F SI1 59.8 53.0 2838.0 5.86 5.82 3.49
0.76 Very Good H SI1 60.9 55.0 2838.0 5.92 5.94 3.61
0.82 Fair F SI1 64.9 58.0 2838.0 5.83 5.79 3.77
0.72 Premium F VS1 58.8 60.0 2838.0 5.91 5.89 3.47
0.7 Premium F VS2 62.3 58.0 2838.0 5.72 5.64 3.54
0.7 Premium F VS2 61.7 58.0 2838.0 5.69 5.63 3.49
0.7 Premium G VS1 62.6 55.0 2838.0 5.73 5.64 3.56
0.7 Premium F VS2 59.4 61.0 2838.0 5.83 5.79 3.45
0.7 Very Good E SI1 63.5 59.0 2838.0 5.53 5.49 3.5
0.7 Premium F VS2 60.9 61.0 2838.0 5.71 5.66 3.46
0.7 Premium F VS2 59.5 58.0 2838.0 5.85 5.75 3.45
0.7 Premium G VS1 63.0 60.0 2838.0 5.64 5.57 3.53
0.74 Very Good E SI1 60.0 57.0 2839.0 5.85 5.89 3.52
0.71 Ideal F VS1 61.5 57.0 2839.0 5.74 5.71 3.52
0.7 Ideal F VS1 61.6 54.0 2839.0 5.75 5.72 3.53
0.71 Ideal F VS1 62.1 55.0 2839.0 5.82 5.68 3.57
0.71 Premium F VS1 59.1 61.0 2839.0 5.84 5.81 3.44
0.71 Premium F VS1 59.0 60.0 2839.0 5.82 5.8 3.43
0.71 Premium F VS1 60.5 58.0 2839.0 5.75 5.72 3.47
0.7 Ideal F VS1 62.4 53.0 2839.0 5.73 5.71 3.57
0.73 Ideal G VS2 61.8 54.0 2839.0 5.8 5.82 3.59
0.7 Ideal E VS2 62.1 54.0 2839.0 5.69 5.72 3.54
0.7 Ideal G VS1 61.3 57.0 2839.0 5.71 5.74 3.51
0.71 Premium G VVS2 60.3 58.0 2839.0 5.82 5.78 3.5
0.71 Premium F VS1 59.2 58.0 2839.0 5.87 5.82 3.46
0.79 Premium G VS2 59.3 62.0 2839.0 6.09 6.01 3.59
0.71 Premium F VS1 62.7 59.0 2839.0 5.7 5.62 3.55
0.77 Very Good H VS1 61.0 60.0 2840.0 5.9 5.87 3.59
0.75 Very Good F SI2 59.8 56.0 2840.0 5.85 5.92 3.52
0.7 Ideal F SI1 61.0 56.0 2840.0 5.75 5.8 3.52
0.71 Premium F VS2 59.3 56.0 2840.0 5.88 5.82 3.47
0.92 Ideal D SI2 61.9 56.0 2840.0 6.27 6.2 3.86
0.83 Premium F SI2 61.4 59.0 2840.0 6.08 6.04 3.72
0.7 Premium H VVS1 59.2 60.0 2840.0 5.87 5.78 3.45
0.73 Premium F VS2 60.3 59.0 2841.0 5.9 5.87 3.55
0.71 Very Good D VS1 63.4 55.0 2841.0 5.69 5.61 3.58
0.73 Very Good D SI1 63.9 57.0 2841.0 5.66 5.71 3.63
0.82 Ideal F SI2 61.7 53.0 2841.0 6.0 6.12 3.74
0.82 Ideal F SI2 62.3 56.0 2841.0 5.96 6.02 3.73
0.82 Very Good F SI2 59.7 57.0 2841.0 6.12 6.14 3.66
0.52 Ideal F VVS1 61.2 56.0 2841.0 5.19 5.21 3.18
1.0 Premium F I1 58.9 60.0 2841.0 6.6 6.55 3.87
0.95 Fair G SI1 66.7 56.0 2841.0 6.16 6.03 4.06
0.73 Ideal D SI1 61.4 57.0 2841.0 5.76 5.8 3.55
0.73 Premium F VS2 59.9 59.0 2841.0 5.87 5.77 3.5
0.73 Premium G VS1 61.4 58.0 2841.0 5.82 5.77 3.56
0.8 Ideal I VS1 62.6 54.0 2842.0 5.92 5.96 3.72
0.7 Premium F VS2 58.7 61.0 2842.0 5.8 5.72 3.38
0.7 Very Good E VS2 60.2 62.0 2843.0 5.71 5.75 3.45
0.7 Very Good E VS2 62.7 58.0 2843.0 5.65 5.67 3.55
0.71 Very Good E VS2 59.4 58.0 2843.0 5.76 5.82 3.44
0.81 Very Good F SI2 63.2 58.0 2843.0 5.91 5.92 3.74
0.71 Very Good D SI1 61.5 58.0 2843.0 5.73 5.79 3.54
0.73 Ideal G VVS2 61.3 57.0 2843.0 5.81 5.84 3.57
0.73 Very Good F VS1 61.8 59.0 2843.0 5.73 5.79 3.56
0.72 Ideal E VS2 62.0 57.0 2843.0 5.71 5.74 3.55
0.81 Ideal F SI2 62.1 57.0 2843.0 5.91 5.95 3.68
0.71 Ideal G VVS2 60.7 57.0 2843.0 5.81 5.78 3.52
0.73 Very Good E SI1 57.7 61.0 2844.0 5.92 5.96 3.43
0.7 Very Good E VS1 62.0 59.0 2844.0 5.65 5.68 3.51
1.01 Ideal I I1 61.5 57.0 2844.0 6.45 6.46 3.97
1.01 Good I I1 63.1 57.0 2844.0 6.35 6.39 4.02
0.79 Ideal H VS2 62.5 57.0 2844.0 5.91 5.93 3.7
0.7 Very Good E VS2 61.8 59.0 2845.0 5.65 5.68 3.5
0.7 Very Good E VS2 58.9 60.0 2845.0 5.83 5.85 3.44
0.8 Good H VS2 63.4 60.0 2845.0 5.92 5.82 3.72
1.27 Premium H SI2 59.3 61.0 2845.0 7.12 7.05 4.2
0.79 Ideal D SI1 61.5 56.0 2846.0 5.96 5.91 3.65
0.72 Very Good F VS1 60.2 59.0 2846.0 5.79 5.84 3.5
0.73 Ideal H VVS2 61.6 56.0 2846.0 5.79 5.84 3.58
1.01 Fair H SI2 65.4 59.0 2846.0 6.3 6.26 4.11
1.01 Good H I1 64.2 61.0 2846.0 6.25 6.18 3.99
0.73 Ideal E SI1 59.1 59.0 2846.0 5.92 5.95 3.51
0.7 Ideal E SI1 61.6 57.0 2846.0 5.71 5.76 3.53
0.7 Good F VS2 59.1 61.0 2846.0 5.76 5.84 3.43
0.77 Premium E SI1 62.9 59.0 2846.0 5.84 5.79 3.66
0.77 Premium G VS2 61.3 60.0 2846.0 5.91 5.81 3.59
0.77 Premium G VS1 61.4 58.0 2846.0 5.94 5.89 3.63
0.84 Very Good H SI1 61.2 57.0 2847.0 6.1 6.12 3.74
0.72 Ideal E SI1 60.3 57.0 2847.0 5.83 5.85 3.52
0.76 Premium D SI1 61.1 59.0 2847.0 5.93 5.88 3.61
0.7 Very Good G VVS2 62.9 59.0 2848.0 5.61 5.68 3.55
0.54 Ideal D VVS2 61.5 55.0 2848.0 5.25 5.29 3.24
0.75 Fair D SI2 64.6 57.0 2848.0 5.74 5.72 3.7
0.79 Good E SI1 64.1 54.0 2849.0 5.86 5.84 3.75
0.74 Very Good E VS1 63.1 58.0 2849.0 5.75 5.73 3.62
0.7 Very Good E VS2 61.0 60.0 2850.0 5.74 5.77 3.51
0.7 Ideal F VS2 60.8 59.0 2850.0 5.69 5.79 3.49
0.75 Ideal J SI1 61.5 56.0 2850.0 5.83 5.87 3.6
1.2 Very Good H I1 63.1 60.0 2850.0 6.75 6.67 4.23
0.8 Very Good F SI1 63.4 57.0 2851.0 5.89 5.82 3.71
0.66 Ideal D VS1 62.1 56.0 2851.0 5.54 5.57 3.45
0.87 Very Good F SI2 61.0 63.0 2851.0 6.22 6.07 3.75
0.86 Premium H SI1 62.7 59.0 2851.0 6.04 5.98 3.77
0.74 Ideal F SI1 61.0 57.0 2851.0 5.85 5.81 3.56
0.58 Very Good E IF 60.6 59.0 2852.0 5.37 5.43 3.27
0.78 Ideal I VS1 61.5 57.0 2852.0 5.88 5.92 3.63
0.74 Ideal G SI1 61.3 55.0 2852.0 5.85 5.86 3.59
0.73 Ideal E SI1 62.7 55.0 2852.0 5.7 5.79 3.6
0.91 Very Good I SI1 63.5 57.0 2852.0 6.12 6.07 3.87
0.71 Premium F VS2 62.6 58.0 2853.0 5.67 5.7 3.56
0.71 Good G VS1 63.5 55.0 2853.0 5.64 5.66 3.59
0.79 Ideal D SI2 62.8 57.0 2853.0 5.9 5.85 3.69
0.79 Premium D SI2 60.0 60.0 2853.0 6.07 6.03 3.63
0.71 Premium E SI1 62.7 58.0 2853.0 5.73 5.66 3.57
0.82 Premium I VS1 61.9 58.0 2853.0 5.99 5.97 3.7
0.78 Very Good H VS1 61.9 57.1 2854.0 5.87 5.95 3.66
0.7 Very Good E VS1 62.4 56.0 2854.0 5.64 5.7 3.54
1.12 Premium H I1 59.1 61.0 2854.0 6.78 6.75 4.0
0.73 Premium E VS2 62.0 57.0 2854.0 5.86 5.76 3.6
0.91 Fair J VS2 64.4 62.0 2854.0 6.06 6.03 3.89
0.91 Fair J VS2 65.4 60.0 2854.0 6.04 6.0 3.94
0.91 Good J VS2 64.2 58.0 2854.0 6.12 6.09 3.92
0.91 Fair H SI1 65.8 58.0 2854.0 6.04 6.01 3.96
0.7 Premium E VS1 58.4 59.0 2854.0 5.91 5.83 3.43
0.68 Premium F VVS2 61.7 57.0 2854.0 5.67 5.64 3.49
0.73 Very Good F VS2 62.5 57.0 2855.0 5.7 5.75 3.58
1.03 Good J SI1 63.6 57.0 2855.0 6.38 6.29 4.03
0.74 Premium D VS2 62.4 57.0 2855.0 5.8 5.74 3.6
0.98 Fair E SI2 53.3 67.0 2855.0 6.82 6.74 3.61
1.02 Fair I SI1 53.0 63.0 2856.0 6.84 6.77 3.66
1.0 Fair G SI2 67.8 61.0 2856.0 5.96 5.9 4.02
1.02 Ideal H SI2 61.6 55.0 2856.0 6.49 6.43 3.98
0.6 Ideal F VVS2 60.8 57.0 2856.0 5.44 5.49 3.32
0.8 Ideal G SI2 61.6 56.0 2856.0 5.97 6.01 3.69
0.97 Ideal F I1 60.7 56.0 2856.0 6.43 6.41 3.9
1.0 Fair I SI1 67.9 62.0 2856.0 6.19 6.03 4.15
0.26 Ideal E VS1 62.3 57.0 556.0 4.05 4.08 2.53
0.26 Ideal E VS1 62.1 56.0 556.0 4.09 4.12 2.55
0.36 Ideal H SI1 61.9 55.0 556.0 4.57 4.59 2.83
0.34 Good G VS2 57.5 61.0 556.0 4.6 4.66 2.66
0.34 Good E SI1 63.3 57.0 556.0 4.44 4.47 2.82
0.34 Good E SI1 63.5 55.0 556.0 4.44 4.47 2.83
0.34 Good E SI1 63.4 55.0 556.0 4.44 4.46 2.82
0.34 Very Good G VS2 59.6 62.0 556.0 4.54 4.56 2.71
0.34 Ideal E SI1 62.2 54.0 556.0 4.47 4.5 2.79
0.32 Good E VS2 64.1 54.0 556.0 4.34 4.37 2.79
0.31 Ideal I VVS1 61.6 55.0 557.0 4.36 4.41 2.7
0.31 Ideal I VVS1 61.3 56.0 557.0 4.36 4.38 2.68
0.31 Ideal I VVS1 62.3 54.0 557.0 4.37 4.4 2.73
0.31 Ideal I VVS1 62.0 54.0 557.0 4.37 4.4 2.72
0.31 Ideal I VVS1 62.7 53.0 557.0 4.33 4.35 2.72
0.31 Ideal I VVS1 62.2 53.0 557.0 4.36 4.38 2.72
0.31 Ideal G VS2 62.2 53.6 557.0 4.32 4.35 2.7
0.31 Ideal H VS1 61.6 54.8 557.0 4.35 4.37 2.69
0.31 Ideal H VS1 61.8 54.2 557.0 4.33 4.37 2.69
0.33 Premium G SI2 59.4 59.0 557.0 4.52 4.5 2.68
0.33 Premium F SI2 62.3 58.0 557.0 4.43 4.4 2.75
0.33 Premium G SI2 62.6 58.0 557.0 4.42 4.4 2.76
0.33 Ideal G SI2 61.9 56.0 557.0 4.45 4.41 2.74
0.33 Premium F SI2 63.0 58.0 557.0 4.42 4.4 2.78
0.33 Premium J VS1 62.8 58.0 557.0 4.41 4.38 2.76
0.33 Premium J VS1 61.5 61.0 557.0 4.46 4.39 2.72
0.33 Ideal J VS1 62.1 55.0 557.0 4.44 4.41 2.75
0.33 Ideal I SI1 63.0 57.0 557.0 4.39 4.37 2.76
0.33 Good I SI1 63.6 53.0 557.0 4.43 4.4 2.81
0.33 Premium I SI1 60.4 59.0 557.0 4.54 4.5 2.73
1.0 Fair H SI2 66.1 56.0 2856.0 6.21 5.97 4.04
0.77 Premium F SI1 60.8 59.0 2856.0 5.92 5.86 3.58
0.77 Premium F SI1 61.0 58.0 2856.0 5.94 5.9 3.61
0.7 Good E VVS2 60.1 63.0 2857.0 5.68 5.71 3.42
0.9 Very Good G SI2 63.1 58.0 2857.0 6.08 6.02 3.82
0.72 Ideal E SI1 62.3 57.0 2857.0 5.76 5.7 3.57
0.9 Premium I VS2 61.9 59.0 2857.0 6.2 6.14 3.82
0.72 Premium E SI1 62.1 58.0 2857.0 5.76 5.73 3.57
0.7 Ideal G VVS2 62.1 56.0 2858.0 5.63 5.71 3.52
0.81 Very Good F SI1 61.3 57.0 2858.0 6.02 6.05 3.7
0.81 Very Good F SI1 61.7 57.0 2858.0 6.0 6.05 3.72
0.71 Premium E VS2 61.0 60.0 2858.0 5.76 5.69 3.49
0.7 Premium E VS2 61.4 59.0 2858.0 5.73 5.7 3.51
0.71 Premium E VS2 61.5 60.0 2858.0 5.76 5.68 3.52
0.71 Very Good E VS2 63.5 59.0 2858.0 5.68 5.59 3.58
0.92 Premium J SI1 62.9 58.0 2858.0 6.22 6.18 3.9
0.76 Ideal E SI1 62.7 54.0 2858.0 5.88 5.83 3.67
0.73 Ideal D SI1 61.5 56.0 2858.0 5.84 5.8 3.58
0.71 Premium D VS2 60.4 62.0 2858.0 5.74 5.72 3.46
0.7 Good E VVS2 63.6 62.0 2858.0 5.61 5.58 3.56
0.9 Fair G SI2 64.5 56.0 2858.0 6.06 6.0 3.89
0.71 Fair D VS2 56.9 65.0 2858.0 5.89 5.84 3.34
0.7 Ideal D VS2 61.0 57.0 2859.0 5.76 5.74 3.51
0.7 Premium D VS2 62.4 56.0 2859.0 5.72 5.66 3.55
0.77 Premium F VS1 60.9 60.0 2859.0 5.91 5.88 3.59
0.71 Ideal G VS1 61.5 56.0 2859.0 5.74 5.78 3.54
0.7 Premium D VS2 59.6 61.0 2859.0 5.8 5.77 3.45
0.75 Fair F VS1 55.8 70.0 2859.0 6.09 5.98 3.37
0.83 Premium E SI2 59.2 60.0 2859.0 6.17 6.12 3.64
0.71 Very Good F VS2 61.3 61.0 2860.0 5.68 5.73 3.5
0.9 Very Good J SI2 63.6 58.0 2860.0 6.07 6.1 3.87
0.6 Ideal E VVS2 61.9 54.9 2860.0 5.41 5.44 3.35
0.71 Premium D VS1 62.9 57.0 2860.0 5.66 5.6 3.54
0.53 Ideal F VVS1 61.4 57.0 2860.0 5.23 5.2 3.2
0.71 Premium D SI1 60.7 58.0 2861.0 5.95 5.78 3.56
0.62 Ideal G VVS2 61.6 56.0 2861.0 5.45 5.48 3.37
0.62 Ideal G VVS2 61.6 56.0 2861.0 5.48 5.51 3.38
0.9 Premium I SI1 63.0 58.0 2861.0 6.09 6.01 3.81
0.62 Fair F IF 60.1 61.0 2861.0 5.53 5.56 3.33
0.82 Premium E SI2 61.7 59.0 2861.0 6.01 5.98 3.7
0.66 Premium D VS1 61.0 58.0 2861.0 5.67 5.57 3.43
0.7 Very Good D SI1 62.5 55.0 2862.0 5.67 5.72 3.56
0.8 Very Good F SI1 62.6 58.0 2862.0 5.9 5.92 3.7
0.8 Very Good D SI2 62.5 59.0 2862.0 5.88 5.92 3.69
0.79 Premium F SI1 62.3 54.0 2862.0 5.97 5.91 3.7
0.71 Very Good F VVS1 63.2 60.0 2862.0 5.65 5.61 3.56
0.7 Ideal H VS2 61.1 57.0 2862.0 5.71 5.74 3.5
0.7 Very Good E VS2 58.7 63.0 2862.0 5.73 5.69 3.35
0.79 Premium H VS1 60.0 60.0 2862.0 6.07 5.99 3.64
0.7 Premium E VS2 59.5 59.0 2862.0 5.82 5.77 3.45
1.22 Premium E I1 60.9 57.0 2862.0 6.93 6.88 4.21
1.01 Fair E SI2 67.6 57.0 2862.0 6.21 6.11 4.18
0.73 Premium E VS2 62.5 61.0 2862.0 5.78 5.64 3.59
0.91 Good I VS2 64.3 58.0 2863.0 6.05 6.09 3.9
0.71 Ideal D SI1 60.8 56.0 2863.0 5.8 5.77 3.52
0.83 Premium G SI1 62.3 58.0 2863.0 6.01 5.97 3.73
0.84 Premium F SI2 62.3 59.0 2863.0 6.06 6.01 3.76
0.71 Premium D SI1 61.0 61.0 2863.0 5.82 5.75 3.53
0.71 Premium D SI1 59.7 59.0 2863.0 5.82 5.8 3.47
0.71 Premium D SI1 61.7 56.0 2863.0 5.8 5.68 3.54
0.71 Ideal D SI1 61.7 57.0 2863.0 5.75 5.7 3.53
0.71 Premium D SI1 61.4 58.0 2863.0 5.79 5.75 3.54
0.71 Premium D SI1 60.6 58.0 2863.0 5.79 5.77 3.5
0.91 Premium J SI1 59.5 62.0 2863.0 6.4 6.18 3.74
0.9 Premium J VS2 59.8 62.0 2863.0 6.24 6.21 3.72
0.71 Premium H VVS2 61.5 62.0 2863.0 5.74 5.68 3.51
0.71 Premium E SI1 59.1 61.0 2863.0 5.84 5.8 3.44
0.72 Ideal F VS2 59.5 57.0 2863.0 5.91 5.86 3.5
0.72 Premium E SI1 60.9 60.0 2863.0 5.78 5.74 3.51
0.71 Ideal E VS2 61.0 55.0 2863.0 5.79 5.75 3.52
0.81 Ideal E SI2 60.3 57.0 2864.0 6.07 6.04 3.65
0.83 Very Good I VS2 61.6 58.0 2865.0 6.05 6.07 3.73
0.73 Premium D SI1 60.8 55.0 2865.0 5.87 5.81 3.55
0.56 Very Good D VVS1 62.0 56.0 2866.0 5.25 5.3 3.27
0.56 Very Good D VVS1 61.8 55.0 2866.0 5.27 5.31 3.27
0.71 Ideal E VS1 62.2 55.0 2866.0 5.74 5.7 3.56
0.7 Ideal H VVS1 62.3 58.0 2866.0 5.66 5.7 3.54
0.96 Premium I SI1 61.3 58.0 2866.0 6.39 6.3 3.89
0.71 Very Good H VVS1 62.9 57.0 2867.0 5.67 5.69 3.57
0.7 Ideal D VS2 62.4 57.0 2867.0 5.68 5.61 3.52
0.71 Ideal H VVS1 60.4 57.0 2867.0 5.78 5.81 3.5
0.8 Premium H VS2 61.2 53.0 2867.0 6.05 5.98 3.68
0.95 Premium F SI2 58.4 57.0 2867.0 6.49 6.41 3.77
0.82 Ideal F SI2 62.3 56.0 2867.0 5.99 5.95 3.72
0.52 Ideal F VVS1 61.2 56.0 2867.0 5.21 5.19 3.18
0.82 Ideal F SI2 61.7 53.0 2867.0 6.12 6.0 3.74
0.82 Ideal F SI2 62.3 56.0 2867.0 6.02 5.96 3.73
0.82 Premium F SI2 59.7 57.0 2867.0 6.14 6.12 3.66
0.8 Ideal G SI1 61.3 57.0 2867.0 5.96 5.91 3.64
0.96 Fair F SI2 68.2 61.0 2867.0 6.07 5.88 4.1
0.72 Ideal I VS1 62.4 55.0 2868.0 5.72 5.75 3.58
0.62 Ideal G IF 60.5 57.0 2868.0 5.52 5.56 3.35
0.79 Premium E SI2 61.0 58.0 2868.0 5.96 5.9 3.62
0.75 Very Good E SI1 63.1 56.0 2868.0 5.78 5.7 3.62
1.08 Premium D I1 61.9 60.0 2869.0 6.55 6.48 4.03
0.72 Ideal E SI1 60.8 55.0 2869.0 5.77 5.84 3.53
0.62 Ideal G IF 61.8 56.0 2869.0 5.43 5.47 3.37
0.73 Ideal G VVS2 61.3 57.0 2869.0 5.84 5.81 3.57
0.72 Ideal H VVS2 60.9 57.0 2869.0 5.79 5.77 3.52
0.52 Premium F VVS2 61.8 60.0 2870.0 5.16 5.13 3.18
0.83 Ideal E SI2 62.2 57.0 2870.0 6.0 6.05 3.75
0.64 Premium E VVS2 62.1 58.0 2870.0 5.56 5.51 3.44
0.8 Ideal G SI1 62.5 57.0 2870.0 5.94 5.9 3.7
0.74 Ideal H SI1 62.1 56.0 2870.0 5.77 5.83 3.6
0.72 Ideal F SI1 61.5 56.0 2870.0 5.72 5.79 3.54
0.82 Ideal H VS2 59.5 57.0 2870.0 6.12 6.09 3.63
0.73 Premium E VS1 61.3 59.0 2870.0 5.81 5.78 3.55
1.04 Premium I I1 61.6 61.0 2870.0 6.47 6.45 3.98
0.73 Very Good E SI1 61.3 58.0 2871.0 5.76 5.83 3.55
0.73 Good E SI1 63.6 57.0 2871.0 5.7 5.72 3.63
0.9 Premium J SI1 62.8 59.0 2871.0 6.13 6.03 3.82
0.75 Ideal I SI1 61.8 55.0 2871.0 5.83 5.85 3.61
0.79 Ideal G SI1 62.6 55.0 2871.0 5.91 5.95 3.71
0.7 Good D SI1 62.5 56.7 2872.0 5.59 5.62 3.51
0.75 Very Good D SI1 60.7 55.0 2872.0 5.87 5.92 3.58
1.02 Ideal I I1 61.7 56.0 2872.0 6.44 6.49 3.99
0.7 Very Good G SI2 59.0 62.0 2872.0 5.79 5.81 3.42
0.7 Ideal D SI1 61.8 56.0 2872.0 5.63 5.73 3.51
0.7 Good E SI1 61.4 64.0 2872.0 5.66 5.71 3.49
0.7 Ideal D SI1 61.4 54.0 2872.0 5.71 5.75 3.52
0.7 Ideal D SI1 60.7 56.0 2872.0 5.72 5.75 3.48
0.7 Very Good D SI1 60.2 60.0 2872.0 5.75 5.82 3.48
0.72 Very Good E VS2 58.3 57.0 2872.0 5.89 5.94 3.45
0.74 Ideal E SI1 62.3 58.0 2872.0 5.74 5.78 3.59
0.84 Good G SI1 65.1 55.0 2872.0 5.88 5.97 3.86
0.76 Very Good F VS2 62.0 58.0 2873.0 5.8 5.86 3.62
0.77 Very Good E SI1 63.2 58.0 2873.0 5.8 5.84 3.68
0.76 Ideal E SI2 62.8 56.0 2873.0 5.78 5.82 3.64
1.0 Ideal I SI2 61.7 56.0 2873.0 6.45 6.41 3.97
1.0 Fair H SI1 65.5 62.0 2873.0 6.14 6.07 4.0
0.9 Fair I SI1 65.7 58.0 2873.0 6.03 6.0 3.95
0.9 Premium J SI1 61.8 58.0 2873.0 6.16 6.13 3.8
0.9 Good J SI1 64.0 61.0 2873.0 6.0 5.96 3.83
0.9 Fair I SI1 65.3 61.0 2873.0 5.98 5.94 3.89
0.9 Fair I SI1 65.8 56.0 2873.0 6.01 5.96 3.94
0.9 Premium J SI1 60.9 61.0 2873.0 6.26 6.22 3.8
0.78 Premium F VS2 62.6 58.0 2874.0 5.91 5.82 3.67
0.71 Premium D VS2 61.2 59.0 2874.0 5.69 5.74 3.5
0.7 Premium F VS1 59.0 59.0 2874.0 5.79 5.77 3.41
0.7 Premium F VS1 60.8 62.0 2874.0 5.71 5.67 3.46
0.7 Premium G VVS2 61.8 58.0 2874.0 5.67 5.63 3.49
0.7 Ideal F VS1 61.0 55.0 2874.0 5.77 5.73 3.51
0.7 Ideal F VS1 61.6 55.0 2874.0 5.75 5.71 3.53
0.7 Ideal F VS1 62.4 56.0 2874.0 5.69 5.65 3.54
0.7 Premium G VVS2 62.9 59.0 2874.0 5.68 5.61 3.55
1.0 Fair H SI2 67.7 60.0 2875.0 6.11 5.98 4.09
0.77 Ideal H SI1 62.4 56.0 2875.0 5.84 5.9 3.66
1.0 Fair J VS1 65.5 55.0 2875.0 6.3 6.25 4.11
1.0 Fair I SI1 66.3 61.0 2875.0 6.08 6.03 4.01
1.0 Fair H SI2 69.5 55.0 2875.0 6.17 6.1 4.26
0.73 Premium E VS1 62.6 60.0 2876.0 5.68 5.75 3.58
0.79 Premium E VS2 60.6 53.0 2876.0 6.04 5.98 3.64
0.72 Very Good H VS1 62.2 54.0 2877.0 5.74 5.76 3.57
0.71 Ideal E VS1 62.4 56.0 2877.0 5.75 5.7 3.57
0.74 Ideal G VS2 62.3 55.0 2877.0 5.8 5.83 3.62
0.7 Good H VVS1 62.7 56.0 2877.0 5.6 5.66 3.53
0.7 Good F VS1 59.1 62.0 2877.0 5.82 5.86 3.44
0.79 Very Good F SI1 62.8 59.0 2878.0 5.86 5.89 3.69
0.79 Very Good F SI1 62.7 60.0 2878.0 5.82 5.89 3.67
0.79 Very Good D SI2 59.7 58.0 2878.0 6.0 6.07 3.6
0.71 Ideal I VS2 61.5 55.0 2878.0 5.76 5.78 3.55
0.79 Ideal F SI1 62.8 56.0 2878.0 5.88 5.9 3.7
0.73 Very Good F SI1 61.4 56.0 2879.0 5.81 5.86 3.58
0.63 Premium E IF 60.3 62.0 2879.0 5.55 5.53 3.34
0.7 Premium F VS1 60.4 60.0 2879.0 5.73 5.7 3.45
0.71 Premium F VS1 62.7 58.0 2879.0 5.71 5.67 3.57
0.84 Ideal G SI2 61.0 56.0 2879.0 6.13 6.1 3.73
0.84 Ideal G SI2 62.3 55.0 2879.0 6.08 6.03 3.77
1.02 Ideal J SI2 60.3 54.0 2879.0 6.53 6.5 3.93
0.72 Fair F VS1 56.9 69.0 2879.0 5.93 5.77 3.33
0.72 Ideal F VS1 62.0 56.0 2879.0 5.76 5.73 3.56
0.92 Very Good J SI2 58.7 61.0 2880.0 6.34 6.43 3.75
0.74 Very Good D SI1 63.9 57.0 2880.0 5.72 5.74 3.66
0.7 Ideal H VVS1 62.0 55.0 2881.0 5.74 5.71 3.55
0.71 Very Good E VS2 60.0 59.0 2881.0 5.84 5.83 3.5
1.05 Premium H I1 62.0 59.0 2881.0 6.5 6.47 4.02
0.7 Very Good H IF 62.8 56.0 2882.0 5.62 5.65 3.54
0.54 Ideal F VVS1 61.8 56.0 2882.0 5.23 5.26 3.24
0.73 Premium F VS2 59.9 58.0 2882.0 5.87 5.84 3.51
0.88 Fair F SI1 56.6 65.0 2882.0 6.39 6.32 3.6
0.73 Premium F VS2 58.7 57.0 2882.0 5.97 5.92 3.49
0.72 Ideal D SI1 61.8 56.0 2883.0 5.75 5.81 3.57
0.9 Good H SI2 62.7 64.0 2883.0 6.09 6.0 3.79
0.9 Fair H SI2 65.0 61.0 2883.0 6.01 5.96 3.89
1.03 Fair I SI2 65.3 55.0 2884.0 6.32 6.27 4.11
0.84 Very Good F SI1 63.8 57.0 2885.0 5.95 6.0 3.81
1.01 Premium I SI1 62.7 60.0 2885.0 6.36 6.27 3.96
0.77 Ideal D SI2 61.5 55.0 2885.0 5.9 5.93 3.64
0.8 Fair E SI1 56.3 63.0 2885.0 6.22 6.14 3.48
0.9 Fair D SI2 66.9 57.0 2885.0 6.02 5.9 3.99
0.73 Ideal E SI1 61.4 56.0 2886.0 5.79 5.81 3.56
0.72 Ideal E SI1 62.7 55.0 2886.0 5.64 5.69 3.55
0.71 Very Good D SI1 62.4 54.0 2887.0 5.71 5.79 3.59
0.7 Premium E VS1 62.6 59.0 2887.0 5.66 5.69 3.55
0.79 Ideal I VS1 61.7 59.0 2888.0 5.93 5.96 3.67
0.72 Very Good G VVS2 62.5 58.0 2889.0 5.68 5.72 3.56
0.7 Very Good E VS2 63.5 54.0 2889.0 5.62 5.66 3.58
0.7 Very Good F VS1 62.2 58.0 2889.0 5.64 5.75 3.54
0.9 Good H SI2 63.5 58.0 2889.0 6.09 6.14 3.88
0.71 Very Good F VS1 62.8 56.0 2889.0 5.69 5.72 3.58
0.5 Ideal E VVS2 62.2 54.0 2889.0 5.08 5.12 3.17
0.5 Ideal E VVS2 62.2 54.0 2889.0 5.09 5.11 3.17
0.74 Ideal F SI1 61.2 56.0 2889.0 5.83 5.87 3.58
0.77 Premium F VS2 61.8 56.0 2889.0 5.94 5.9 3.66
0.77 Premium E SI1 59.8 61.0 2889.0 5.99 5.95 3.57
0.8 Ideal F SI1 61.5 54.0 2890.0 6.07 6.0 3.71
0.8 Ideal F SI1 62.4 57.0 2890.0 5.9 5.87 3.67
0.8 Premium F SI1 61.5 60.0 2890.0 5.97 5.94 3.66
0.8 Good F SI1 63.8 59.0 2890.0 5.87 5.83 3.73
0.66 Ideal G VVS1 61.5 56.0 2890.0 5.61 5.58 3.44
0.71 Very Good E VS2 61.2 58.0 2891.0 5.71 5.79 3.52
0.71 Ideal F VS2 61.2 56.0 2891.0 5.73 5.77 3.52
0.71 Ideal E VS2 61.6 56.0 2891.0 5.74 5.76 3.54
0.71 Ideal E VS2 62.7 56.0 2891.0 5.71 5.75 3.59
0.72 Ideal D SI1 61.1 56.0 2891.0 5.78 5.81 3.54
0.71 Good D VS2 62.3 61.0 2891.0 5.7 5.73 3.56
0.86 Ideal H SI2 61.8 55.0 2892.0 6.12 6.14 3.79
1.19 Fair H I1 65.1 59.0 2892.0 6.62 6.55 4.29
0.71 Very Good F VS1 62.6 55.0 2893.0 5.66 5.71 3.56
0.82 Very Good G SI2 62.5 56.0 2893.0 5.99 6.04 3.76
0.71 Ideal G VVS2 61.5 57.0 2893.0 5.73 5.75 3.53
0.75 Ideal F VS2 62.5 57.0 2893.0 5.78 5.83 3.63
0.7 Very Good H VVS1 59.2 60.0 2893.0 5.87 5.78 3.45
0.8 Ideal G SI2 62.5 55.0 2893.0 5.89 5.92 3.69
0.82 Good G SI2 59.9 62.0 2893.0 6.02 6.04 3.61
0.82 Very Good G SI1 63.4 55.0 2893.0 6.0 5.93 3.78
0.82 Premium G SI1 59.9 59.0 2893.0 6.09 6.06 3.64
0.81 Very Good E SI2 62.4 57.0 2894.0 5.91 5.99 3.71
0.81 Ideal G SI2 62.2 57.0 2894.0 5.96 6.0 3.72
0.76 Ideal F SI1 61.4 56.0 2894.0 5.88 5.92 3.62
0.71 Very Good G VS2 60.9 56.0 2895.0 5.75 5.78 3.51
0.7 Very Good F VS1 61.8 59.0 2895.0 5.66 5.76 3.53
0.7 Ideal G VVS2 62.1 53.0 2895.0 5.71 5.75 3.56
0.74 Very Good G VS1 59.8 58.0 2896.0 5.85 5.89 3.51
0.77 Very Good G VS2 61.3 60.0 2896.0 5.81 5.91 3.59
0.77 Very Good G VS2 58.3 63.0 2896.0 6.0 6.05 3.51
0.53 Ideal F VVS1 61.6 56.0 2896.0 5.18 5.24 3.21
0.79 Ideal D SI1 61.5 56.0 2896.0 5.91 5.96 3.65
0.73 Ideal E SI2 61.5 55.0 2896.0 5.82 5.86 3.59
0.77 Ideal D SI2 62.1 56.0 2896.0 5.83 5.89 3.64
0.77 Premium E SI1 60.9 58.0 2896.0 5.94 5.88 3.6
1.01 Very Good I I1 63.1 57.0 2896.0 6.39 6.35 4.02
1.01 Ideal I I1 61.5 57.0 2896.0 6.46 6.45 3.97
0.6 Very Good D VVS2 60.6 57.0 2897.0 5.48 5.51 3.33
0.76 Premium E SI1 61.1 58.0 2897.0 5.91 5.85 3.59
0.54 Ideal D VVS2 61.4 52.0 2897.0 5.3 5.34 3.26
0.72 Ideal E SI1 62.5 55.0 2897.0 5.69 5.74 3.57
0.72 Good F VS1 59.4 61.0 2897.0 5.82 5.89 3.48
0.74 Premium D VS2 61.8 58.0 2897.0 5.81 5.77 3.58
1.12 Premium J SI2 60.6 59.0 2898.0 6.68 6.61 4.03

Step 2. Understand the data

Let's examine the data to get a better understanding of what is there. We only examine a couple of features (columns), but it gives an idea of the type of exploration you might do to understand a new dataset.

For more examples of using Databricks's visualization (even across languages) see https://docs.databricks.com/user-guide/visualizations/index.html NOW.

We can see that we have a mix of

  • categorical features (cut, color, clarity) and
  • continuous features (depth, x, y, z).
Let's first look at the categorical features.

You can also select one or more individual columns using so-called DataFrame API.

Let us select the column cut from diamondsDF and create a new DataFrame called cutsDF and then display it as follows:

val cutsDF = diamondsDF.select("cut") // Shift+Enter
cutsDF: org.apache.spark.sql.DataFrame = [cut: string]
cutsDF.show(10) // Ctrl+Enter
+---------+
|      cut|
+---------+
|    Ideal|
|  Premium|
|     Good|
|  Premium|
|     Good|
|Very Good|
|Very Good|
|Very Good|
|     Fair|
|Very Good|
+---------+
only showing top 10 rows

Let us use distinct to find the distinct types of cut's in the dataset.

// View distinct diamond cuts in dataset
val cutsDistinctDF = diamondsDF.select("cut").distinct()
cutsDistinctDF: org.apache.spark.sql.Dataset[org.apache.spark.sql.Row] = [cut: string]
cutsDistinctDF.show()
+---------+
|      cut|
+---------+
|  Premium|
|    Ideal|
|     Good|
|     Fair|
|Very Good|
+---------+

Clearly, there are just 5 kinds of cuts.

// View distinct diamond colors in dataset
val colorsDistinctDF = diamondsDF.select("color").distinct() //.collect()
colorsDistinctDF.show()
+-----+
|color|
+-----+
|    F|
|    E|
|    D|
|    J|
|    G|
|    I|
|    H|
+-----+

colorsDistinctDF: org.apache.spark.sql.Dataset[org.apache.spark.sql.Row] = [color: string]
// View distinct diamond clarities in dataset
val claritiesDistinctDF = diamondsDF.select("clarity").distinct() // .collect()
claritiesDistinctDF.show()
+-------+
|clarity|
+-------+
|   VVS2|
|    SI1|
|     IF|
|     I1|
|   VVS1|
|    VS2|
|    SI2|
|    VS1|
+-------+

claritiesDistinctDF: org.apache.spark.sql.Dataset[org.apache.spark.sql.Row] = [clarity: string]

We can examine the distribution of a particular feature by using display(),

You Try!

  1. Click on the chart icon and Plot Options, and setting:
  • Value=<id>
  • Series groupings='cut'
  • and Aggregation=COUNT.
  1. You can also try this using columns "color" and "clarity"
display(diamondsDF.select("cut"))
cut
Ideal
Premium
Good
Premium
Good
Very Good
Very Good
Very Good
Fair
Very Good
Good
Ideal
Premium
Ideal
Premium
Premium
Ideal
Good
Good
Very Good
Good
Very Good
Very Good
Very Good
Very Good
Very Good
Premium
Very Good
Very Good
Very Good
Very Good
Very Good
Very Good
Very Good
Very Good
Good
Good
Good
Very Good
Ideal
Ideal
Ideal
Good
Good
Good
Premium
Very Good
Good
Very Good
Very Good
Very Good
Ideal
Ideal
Premium
Premium
Ideal
Premium
Very Good
Very Good
Good
Ideal
Premium
Ideal
Ideal
Premium
Ideal
Ideal
Very Good
Premium
Premium
Very Good
Very Good
Premium
Premium
Good
Very Good
Very Good
Very Good
Very Good
Very Good
Very Good
Very Good
Ideal
Ideal
Good
Premium
Premium
Premium
Premium
Premium
Ideal
Fair
Ideal
Very Good
Very Good
Good
Good
Fair
Very Good
Premium
Very Good
Premium
Ideal
Premium
Ideal
Ideal
Premium
Ideal
Ideal
Ideal
Ideal
Ideal
Premium
Very Good
Ideal
Ideal
Premium
Ideal
Ideal
Ideal
Ideal
Ideal
Very Good
Fair
Fair
Premium
Premium
Very Good
Fair
Fair
Ideal
Very Good
Ideal
Very Good
Very Good
Premium
Very Good
Premium
Ideal
Ideal
Premium
Premium
Very Good
Very Good
Ideal
Good
Very Good
Very Good
Very Good
Ideal
Premium
Ideal
Premium
Premium
Very Good
Ideal
Ideal
Premium
Premium
Ideal
Premium
Very Good
Very Good
Ideal
Ideal
Very Good
Very Good
Ideal
Ideal
Good
Ideal
Premium
Very Good
Ideal
Ideal
Good
Very Good
Very Good
Premium
Ideal
Ideal
Ideal
Ideal
Ideal
Good
Ideal
Very Good
Premium
Very Good
Good
Good
Ideal
Premium
Premium
Premium
Premium
Premium
Premium
Ideal
Premium
Very Good
Premium
Premium
Good
Fair
Premium
Very Good
Ideal
Very Good
Ideal
Very Good
Premium
Ideal
Ideal
Ideal
Premium
Ideal
Ideal
Very Good
Very Good
Ideal
Good
Premium
Very Good
Ideal
Premium
Premium
Fair
Premium
Ideal
Very Good
Very Good
Very Good
Ideal
Ideal
Very Good
Very Good
Ideal
Good
Good
Ideal
Fair
Premium
Good
Good
Premium
Premium
Very Good
Ideal
Ideal
Ideal
Good
Premium
Premium
Premium
Fair
Ideal
Premium
Ideal
Premium
Premium
Very Good
Ideal
Premium
Premium
Ideal
Premium
Premium
Premium
Ideal
Very Good
Very Good
Good
Ideal
Ideal
Very Good
Very Good
Premium
Ideal
Good
Premium
Premium
Premium
Premium
Premium
Good
Very Good
Very Good
Premium
Premium
Premium
Ideal
Ideal
Ideal
Ideal
Ideal
Fair
Premium
Fair
Very Good
Ideal
Very Good
Ideal
Ideal
Very Good
Good
Premium
Very Good
Ideal
Ideal
Very Good
Premium
Ideal
Ideal
Fair
Ideal
Ideal
Premium
Ideal
Premium
Good
Good
Premium
Premium
Premium
Very Good
Ideal
Premium
Premium
Very Good
Very Good
Ideal
Ideal
Good
Premium
Premium
Premium
Premium
Premium
Premium
Very Good
Ideal
Very Good
Very Good
Very Good
Very Good
Ideal
Very Good
Ideal
Ideal
Ideal
Ideal
Fair
Premium
Premium
Ideal
Fair
Premium
Ideal
Fair
Good
Very Good
Very Good
Ideal
Very Good
Very Good
Premium
Very Good
Very Good
Fair
Very Good
Ideal
Very Good
Very Good
Premium
Premium
Fair
Very Good
Very Good
Ideal
Good
Good
Very Good
Very Good
Fair
Fair
Very Good
Very Good
Good
Very Good
Premium
Premium
Ideal
Premium
Very Good
Ideal
Premium
Premium
Premium
Very Good
Good
Ideal
Very Good
Good
Ideal
Very Good
Very Good
Very Good
Ideal
Ideal
Ideal
Ideal
Ideal
Ideal
Ideal
Very Good
Very Good
Ideal
Ideal
Ideal
Very Good
Very Good
Ideal
Fair
Premium
Ideal
Premium
Very Good
Good
Premium
Ideal
Premium
Very Good
Very Good
Premium
Ideal
Ideal
Ideal
Ideal
Fair
Fair
Premium
Premium
Fair
Premium
Very Good
Ideal
Good
Premium
Ideal
Ideal
Premium
Ideal
Ideal
Premium
Ideal
Premium
Premium
Ideal
Premium
Ideal
Good
Premium
Premium
Premium
Premium
Premium
Ideal
Ideal
Premium
Premium
Premium
Fair
Very Good
Ideal
Good
Good
Ideal
Ideal
Premium
Ideal
Premium
Good
Premium
Premium
Premium
Very Good
Premium
Premium
Premium
Premium
Premium
Ideal
Ideal
Ideal
Premium
Premium
Premium
Ideal
Good
Ideal
Premium
Premium
Ideal
Good
Ideal
Ideal
Premium
Premium
Ideal
Very Good
Good
Premium
Ideal
Fair
Ideal
Premium
Ideal
Good
Ideal
Premium
Premium
Premium
Very Good
Premium
Premium
Fair
Premium
Good
Premium
Premium
Very Good
Premium
Very Good
Premium
Ideal
Very Good
Good
Premium
Ideal
Ideal
Ideal
Premium
Premium
Good
Ideal
Ideal
Premium
Premium
Very Good
Very Good
Ideal
Ideal
Ideal
Ideal
Premium
Very Good
Ideal
Ideal
Premium
Ideal
Very Good
Very Good
Premium
Premium
Ideal
Premium
Good
Very Good
Very Good
Very Good
Ideal
Ideal
Ideal
Ideal
Ideal
Ideal
Very Good
Ideal
Ideal
Very Good
Fair
Ideal
Ideal
Ideal
Ideal
Ideal
Ideal
Premium
Very Good
Good
Premium
Ideal
Ideal
Very Good
Fair
Premium
Premium
Premium
Premium
Premium
Very Good
Premium
Premium
Premium
Very Good
Ideal
Ideal
Ideal
Premium
Premium
Premium
Ideal
Ideal
Ideal
Ideal
Premium
Premium
Premium
Premium
Very Good
Very Good
Ideal
Premium
Ideal
Premium
Premium
Premium
Very Good
Very Good
Ideal
Ideal
Very Good
Ideal
Premium
Fair
Ideal
Premium
Premium
Ideal
Premium
Very Good
Very Good
Very Good
Very Good
Very Good
Ideal
Very Good
Ideal
Ideal
Ideal
Very Good
Very Good
Ideal
Good
Ideal
Very Good
Very Good
Good
Premium
Ideal
Very Good
Ideal
Fair
Good
Ideal
Ideal
Good
Premium
Premium
Premium
Very Good
Ideal
Premium
Very Good
Ideal
Fair
Good
Very Good
Very Good
Ideal
Ideal
Very Good
Very Good
Ideal
Very Good
Premium
Ideal
Very Good
Ideal
Ideal
Ideal
Very Good
Premium
Good
Ideal
Premium
Premium
Premium
Very Good
Very Good
Premium
Premium
Fair
Fair
Good
Fair
Premium
Premium
Very Good
Good
Premium
Fair
Fair
Fair
Ideal
Ideal
Ideal
Ideal
Fair
Ideal
Ideal
Ideal
Good
Good
Good
Good
Very Good
Ideal
Good
Ideal
Ideal
Ideal
Ideal
Ideal
Ideal
Ideal
Ideal
Ideal
Premium
Premium
Premium
Ideal
Premium
Premium
Premium
Ideal
Ideal
Good
Premium
Fair
Premium
Premium
Good
Very Good
Ideal
Premium
Premium
Ideal
Very Good
Very Good
Premium
Premium
Premium
Very Good
Premium
Ideal
Ideal
Premium
Good
Fair
Fair
Ideal
Premium
Premium
Ideal
Premium
Fair
Premium
Very Good
Very Good
Ideal
Premium
Ideal
Premium
Ideal
Ideal
Premium
Fair
Premium
Premium
Very Good
Very Good
Very Good
Premium
Very Good
Ideal
Very Good
Premium
Premium
Premium
Fair
Premium
Good
Ideal
Premium
Premium
Premium
Premium
Premium
Ideal
Premium
Premium
Premium
Premium
Premium
Premium
Ideal
Premium
Ideal
Ideal
Very Good
Premium
Very Good
Very Good
Ideal
Ideal
Premium
Very Good
Ideal
Ideal
Premium
Premium
Ideal
Ideal
Ideal
Ideal
Premium
Ideal
Fair
Ideal
Ideal
Premium
Very Good
Premium
Ideal
Ideal
Ideal
Ideal
Premium
Ideal
Premium
Ideal
Ideal
Ideal
Ideal
Premium
Premium
Very Good
Good
Premium
Ideal
Ideal
Good
Very Good
Ideal
Very Good
Ideal
Good
Ideal
Ideal
Very Good
Very Good
Ideal
Good
Very Good
Very Good
Ideal
Ideal
Fair
Fair
Premium
Good
Fair
Fair
Premium
Premium
Premium
Premium
Premium
Premium
Ideal
Ideal
Ideal
Premium
Fair
Ideal
Fair
Fair
Fair
Premium
Premium
Very Good
Ideal
Ideal
Good
Good
Very Good
Very Good
Very Good
Ideal
Ideal
Very Good
Premium
Premium
Premium
Ideal
Ideal
Ideal
Fair
Ideal
Very Good
Very Good
Ideal
Very Good
Premium
Very Good
Ideal
Premium
Fair
Premium
Ideal
Good
Fair
Fair
Very Good
Premium
Ideal
Fair
Fair
Ideal
Ideal
Very Good
Premium
Ideal
Very Good
Very Good
Very Good
Good
Very Good
Ideal
Ideal
Ideal
Premium
Premium
Ideal
Ideal
Premium
Good
Ideal
Very Good
Ideal
Ideal
Ideal
Ideal
Good
Ideal
Fair
Very Good
Very Good
Ideal
Ideal
Very Good
Ideal
Good
Very Good
Premium
Very Good
Ideal
Ideal
Very Good
Very Good
Ideal
Very Good
Very Good
Very Good
Ideal
Ideal
Ideal
Ideal
Premium
Very Good
Ideal
Very Good
Premium
Ideal
Ideal
Good
Premium
Premium
// come on do the same for color NOW!
// and clarity too...

You Try!

Now play around with display of the entire DF and choosing what you want in the GUI as opposed to a .select(...) statement earlier.

For instance, the following display(diamondsDF) shows the counts of the colors by choosing in the Plot Options a bar-chart that is grouped with Series Grouping as color, values as <id> and Aggregation as COUNT. You can click on Plot Options to see these settings and can change them as you wish by dragging and dropping.

 display(diamondsDF)
carat cut color clarity depth table price x y z
0.23 Ideal E SI2 61.5 55.0 326.0 3.95 3.98 2.43
0.21 Premium E SI1 59.8 61.0 326.0 3.89 3.84 2.31
0.23 Good E VS1 56.9 65.0 327.0 4.05 4.07 2.31
0.29 Premium I VS2 62.4 58.0 334.0 4.2 4.23 2.63
0.31 Good J SI2 63.3 58.0 335.0 4.34 4.35 2.75
0.24 Very Good J VVS2 62.8 57.0 336.0 3.94 3.96 2.48
0.24 Very Good I VVS1 62.3 57.0 336.0 3.95 3.98 2.47
0.26 Very Good H SI1 61.9 55.0 337.0 4.07 4.11 2.53
0.22 Fair E VS2 65.1 61.0 337.0 3.87 3.78 2.49
0.23 Very Good H VS1 59.4 61.0 338.0 4.0 4.05 2.39
0.3 Good J SI1 64.0 55.0 339.0 4.25 4.28 2.73
0.23 Ideal J VS1 62.8 56.0 340.0 3.93 3.9 2.46
0.22 Premium F SI1 60.4 61.0 342.0 3.88 3.84 2.33
0.31 Ideal J SI2 62.2 54.0 344.0 4.35 4.37 2.71
0.2 Premium E SI2 60.2 62.0 345.0 3.79 3.75 2.27
0.32 Premium E I1 60.9 58.0 345.0 4.38 4.42 2.68
0.3 Ideal I SI2 62.0 54.0 348.0 4.31 4.34 2.68
0.3 Good J SI1 63.4 54.0 351.0 4.23 4.29 2.7
0.3 Good J SI1 63.8 56.0 351.0 4.23 4.26 2.71
0.3 Very Good J SI1 62.7 59.0 351.0 4.21 4.27 2.66
0.3 Good I SI2 63.3 56.0 351.0 4.26 4.3 2.71
0.23 Very Good E VS2 63.8 55.0 352.0 3.85 3.92 2.48
0.23 Very Good H VS1 61.0 57.0 353.0 3.94 3.96 2.41
0.31 Very Good J SI1 59.4 62.0 353.0 4.39 4.43 2.62
0.31 Very Good J SI1 58.1 62.0 353.0 4.44 4.47 2.59
0.23 Very Good G VVS2 60.4 58.0 354.0 3.97 4.01 2.41
0.24 Premium I VS1 62.5 57.0 355.0 3.97 3.94 2.47
0.3 Very Good J VS2 62.2 57.0 357.0 4.28 4.3 2.67
0.23 Very Good D VS2 60.5 61.0 357.0 3.96 3.97 2.4
0.23 Very Good F VS1 60.9 57.0 357.0 3.96 3.99 2.42
0.23 Very Good F VS1 60.0 57.0 402.0 4.0 4.03 2.41
0.23 Very Good F VS1 59.8 57.0 402.0 4.04 4.06 2.42
0.23 Very Good E VS1 60.7 59.0 402.0 3.97 4.01 2.42
0.23 Very Good E VS1 59.5 58.0 402.0 4.01 4.06 2.4
0.23 Very Good D VS1 61.9 58.0 402.0 3.92 3.96 2.44
0.23 Good F VS1 58.2 59.0 402.0 4.06 4.08 2.37
0.23 Good E VS1 64.1 59.0 402.0 3.83 3.85 2.46
0.31 Good H SI1 64.0 54.0 402.0 4.29 4.31 2.75
0.26 Very Good D VS2 60.8 59.0 403.0 4.13 4.16 2.52
0.33 Ideal I SI2 61.8 55.0 403.0 4.49 4.51 2.78
0.33 Ideal I SI2 61.2 56.0 403.0 4.49 4.5 2.75
0.33 Ideal J SI1 61.1 56.0 403.0 4.49 4.55 2.76
0.26 Good D VS2 65.2 56.0 403.0 3.99 4.02 2.61
0.26 Good D VS1 58.4 63.0 403.0 4.19 4.24 2.46
0.32 Good H SI2 63.1 56.0 403.0 4.34 4.37 2.75
0.29 Premium F SI1 62.4 58.0 403.0 4.24 4.26 2.65
0.32 Very Good H SI2 61.8 55.0 403.0 4.35 4.42 2.71
0.32 Good H SI2 63.8 56.0 403.0 4.36 4.38 2.79
0.25 Very Good E VS2 63.3 60.0 404.0 4.0 4.03 2.54
0.29 Very Good H SI2 60.7 60.0 404.0 4.33 4.37 2.64
0.24 Very Good F SI1 60.9 61.0 404.0 4.02 4.03 2.45
0.23 Ideal G VS1 61.9 54.0 404.0 3.93 3.95 2.44
0.32 Ideal I SI1 60.9 55.0 404.0 4.45 4.48 2.72
0.22 Premium E VS2 61.6 58.0 404.0 3.93 3.89 2.41
0.22 Premium D VS2 59.3 62.0 404.0 3.91 3.88 2.31
0.3 Ideal I SI2 61.0 59.0 405.0 4.3 4.33 2.63
0.3 Premium J SI2 59.3 61.0 405.0 4.43 4.38 2.61
0.3 Very Good I SI1 62.6 57.0 405.0 4.25 4.28 2.67
0.3 Very Good I SI1 63.0 57.0 405.0 4.28 4.32 2.71
0.3 Good I SI1 63.2 55.0 405.0 4.25 4.29 2.7
0.35 Ideal I VS1 60.9 57.0 552.0 4.54 4.59 2.78
0.3 Premium D SI1 62.6 59.0 552.0 4.23 4.27 2.66
0.3 Ideal D SI1 62.5 57.0 552.0 4.29 4.32 2.69
0.3 Ideal D SI1 62.1 56.0 552.0 4.3 4.33 2.68
0.42 Premium I SI2 61.5 59.0 552.0 4.78 4.84 2.96
0.28 Ideal G VVS2 61.4 56.0 553.0 4.19 4.22 2.58
0.32 Ideal I VVS1 62.0 55.3 553.0 4.39 4.42 2.73
0.31 Very Good G SI1 63.3 57.0 553.0 4.33 4.3 2.73
0.31 Premium G SI1 61.8 58.0 553.0 4.35 4.32 2.68
0.24 Premium E VVS1 60.7 58.0 553.0 4.01 4.03 2.44
0.24 Very Good D VVS1 61.5 60.0 553.0 3.97 4.0 2.45
0.3 Very Good H SI1 63.1 56.0 554.0 4.29 4.27 2.7
0.3 Premium H SI1 62.9 59.0 554.0 4.28 4.24 2.68
0.3 Premium H SI1 62.5 57.0 554.0 4.29 4.25 2.67
0.3 Good H SI1 63.7 57.0 554.0 4.28 4.26 2.72
0.26 Very Good F VVS2 59.2 60.0 554.0 4.19 4.22 2.49
0.26 Very Good E VVS2 59.9 58.0 554.0 4.15 4.23 2.51
0.26 Very Good D VVS2 62.4 54.0 554.0 4.08 4.13 2.56
0.26 Very Good D VVS2 62.8 60.0 554.0 4.01 4.05 2.53
0.26 Very Good E VVS1 62.6 59.0 554.0 4.06 4.09 2.55
0.26 Very Good E VVS1 63.4 59.0 554.0 4.0 4.04 2.55
0.26 Very Good D VVS1 62.1 60.0 554.0 4.03 4.12 2.53
0.26 Ideal E VVS2 62.9 58.0 554.0 4.02 4.06 2.54
0.38 Ideal I SI2 61.6 56.0 554.0 4.65 4.67 2.87
0.26 Good E VVS1 57.9 60.0 554.0 4.22 4.25 2.45
0.24 Premium G VVS1 62.3 59.0 554.0 3.95 3.92 2.45
0.24 Premium H VVS1 61.2 58.0 554.0 4.01 3.96 2.44
0.24 Premium H VVS1 60.8 59.0 554.0 4.02 4.0 2.44
0.24 Premium H VVS2 60.7 58.0 554.0 4.07 4.04 2.46
0.32 Premium I SI1 62.9 58.0 554.0 4.35 4.33 2.73
0.7 Ideal E SI1 62.5 57.0 2757.0 5.7 5.72 3.57
0.86 Fair E SI2 55.1 69.0 2757.0 6.45 6.33 3.52
0.7 Ideal G VS2 61.6 56.0 2757.0 5.7 5.67 3.5
0.71 Very Good E VS2 62.4 57.0 2759.0 5.68 5.73 3.56
0.78 Very Good G SI2 63.8 56.0 2759.0 5.81 5.85 3.72
0.7 Good E VS2 57.5 58.0 2759.0 5.85 5.9 3.38
0.7 Good F VS1 59.4 62.0 2759.0 5.71 5.76 3.4
0.96 Fair F SI2 66.3 62.0 2759.0 6.27 5.95 4.07
0.73 Very Good E SI1 61.6 59.0 2760.0 5.77 5.78 3.56
0.8 Premium H SI1 61.5 58.0 2760.0 5.97 5.93 3.66
0.75 Very Good D SI1 63.2 56.0 2760.0 5.8 5.75 3.65
0.75 Premium E SI1 59.9 54.0 2760.0 6.0 5.96 3.58
0.74 Ideal G SI1 61.6 55.0 2760.0 5.8 5.85 3.59
0.75 Premium G VS2 61.7 58.0 2760.0 5.85 5.79 3.59
0.8 Ideal I VS1 62.9 56.0 2760.0 5.94 5.87 3.72
0.75 Ideal G SI1 62.2 55.0 2760.0 5.87 5.8 3.63
0.8 Premium G SI1 63.0 59.0 2760.0 5.9 5.81 3.69
0.74 Ideal I VVS2 62.3 55.0 2761.0 5.77 5.81 3.61
0.81 Ideal F SI2 58.8 57.0 2761.0 6.14 6.11 3.6
0.59 Ideal E VVS2 62.0 55.0 2761.0 5.38 5.43 3.35
0.8 Ideal F SI2 61.4 57.0 2761.0 5.96 6.0 3.67
0.74 Ideal E SI2 62.2 56.0 2761.0 5.8 5.84 3.62
0.9 Premium I VS2 63.0 58.0 2761.0 6.16 6.12 3.87
0.74 Very Good G SI1 62.2 59.0 2762.0 5.73 5.82 3.59
0.73 Ideal F VS2 62.6 56.0 2762.0 5.77 5.74 3.6
0.73 Ideal F VS2 62.7 53.0 2762.0 5.8 5.75 3.62
0.8 Premium F SI2 61.7 58.0 2762.0 5.98 5.94 3.68
0.71 Ideal G VS2 62.4 54.0 2762.0 5.72 5.76 3.58
0.7 Ideal E VS2 60.7 58.0 2762.0 5.73 5.76 3.49
0.8 Ideal F SI2 59.9 59.0 2762.0 6.01 6.07 3.62
0.71 Ideal D SI2 62.3 56.0 2762.0 5.73 5.69 3.56
0.74 Ideal E SI1 62.3 54.0 2762.0 5.8 5.83 3.62
0.7 Very Good F VS2 61.7 63.0 2762.0 5.64 5.61 3.47
0.7 Fair F VS2 64.5 57.0 2762.0 5.57 5.53 3.58
0.7 Fair F VS2 65.3 55.0 2762.0 5.63 5.58 3.66
0.7 Premium F VS2 61.6 60.0 2762.0 5.65 5.59 3.46
0.91 Premium H SI1 61.4 56.0 2763.0 6.09 5.97 3.7
0.61 Very Good D VVS2 59.6 57.0 2763.0 5.56 5.58 3.32
0.91 Fair H SI2 64.4 57.0 2763.0 6.11 6.09 3.93
0.91 Fair H SI2 65.7 60.0 2763.0 6.03 5.99 3.95
0.77 Ideal H VS2 62.0 56.0 2763.0 5.89 5.86 3.64
0.71 Very Good D SI1 63.6 58.0 2764.0 5.64 5.68 3.6
0.71 Ideal D SI1 61.9 59.0 2764.0 5.69 5.72 3.53
0.7 Very Good E VS2 62.6 60.0 2765.0 5.62 5.65 3.53
0.77 Very Good H VS1 61.3 60.0 2765.0 5.88 5.9 3.61
0.63 Premium E VVS1 60.9 60.0 2765.0 5.52 5.55 3.37
0.71 Very Good F VS1 60.1 62.0 2765.0 5.74 5.77 3.46
0.71 Premium F VS1 61.8 59.0 2765.0 5.69 5.73 3.53
0.76 Ideal H SI1 61.2 57.0 2765.0 5.88 5.91 3.61
0.64 Ideal G VVS1 61.9 56.0 2766.0 5.53 5.56 3.43
0.71 Premium G VS2 60.9 57.0 2766.0 5.78 5.75 3.51
0.71 Premium G VS2 59.8 56.0 2766.0 5.89 5.81 3.5
0.7 Very Good D VS2 61.8 55.0 2767.0 5.68 5.72 3.52
0.7 Very Good F VS1 60.0 57.0 2767.0 5.8 5.87 3.5
0.71 Ideal D SI2 61.6 55.0 2767.0 5.74 5.76 3.54
0.7 Good H VVS2 62.1 64.0 2767.0 5.62 5.65 3.5
0.71 Very Good G VS1 63.3 59.0 2768.0 5.52 5.61 3.52
0.73 Very Good D SI1 60.2 56.0 2768.0 5.83 5.87 3.52
0.7 Very Good D SI1 61.1 58.0 2768.0 5.66 5.73 3.48
0.7 Ideal E SI1 60.9 57.0 2768.0 5.73 5.76 3.5
0.71 Premium D SI2 61.7 59.0 2768.0 5.71 5.67 3.51
0.74 Ideal I SI1 61.3 56.0 2769.0 5.82 5.86 3.57
0.71 Premium D VS2 62.5 60.0 2770.0 5.65 5.61 3.52
0.73 Premium G VS2 61.4 59.0 2770.0 5.83 5.76 3.56
0.76 Very Good F SI1 62.9 57.0 2770.0 5.79 5.81 3.65
0.76 Ideal D SI2 62.4 57.0 2770.0 5.78 5.83 3.62
0.71 Ideal F SI1 60.7 56.0 2770.0 5.77 5.8 3.51
0.73 Premium G VS2 60.7 58.0 2770.0 5.87 5.82 3.55
0.73 Premium G VS1 61.5 58.0 2770.0 5.79 5.75 3.55
0.73 Ideal D SI2 59.9 57.0 2770.0 5.92 5.89 3.54
0.73 Premium G VS2 59.2 59.0 2770.0 5.92 5.87 3.49
0.72 Very Good H VVS2 60.3 56.0 2771.0 5.81 5.83 3.51
0.73 Very Good F SI1 61.7 60.0 2771.0 5.79 5.82 3.58
0.71 Ideal G VS2 61.9 57.0 2771.0 5.73 5.77 3.56
0.79 Ideal F SI2 61.9 55.0 2771.0 5.97 5.92 3.68
0.73 Very Good H VVS1 60.4 59.0 2772.0 5.83 5.89 3.54
0.8 Very Good F SI2 61.0 57.0 2772.0 6.01 6.03 3.67
0.58 Ideal G VVS1 61.5 55.0 2772.0 5.39 5.44 3.33
0.58 Ideal F VVS1 61.7 56.0 2772.0 5.33 5.37 3.3
0.71 Good E VS2 59.2 61.0 2772.0 5.8 5.88 3.46
0.75 Ideal D SI2 61.3 56.0 2773.0 5.85 5.89 3.6
0.7 Premium D VS2 58.0 62.0 2773.0 5.87 5.78 3.38
1.17 Very Good J I1 60.2 61.0 2774.0 6.83 6.9 4.13
0.6 Ideal E VS1 61.7 55.0 2774.0 5.41 5.44 3.35
0.7 Ideal E SI1 62.7 55.0 2774.0 5.68 5.74 3.58
0.83 Good I VS2 64.6 54.0 2774.0 5.85 5.88 3.79
0.74 Very Good F VS2 61.3 61.0 2775.0 5.8 5.84 3.57
0.72 Very Good G VS2 63.7 56.4 2776.0 5.62 5.69 3.61
0.71 Premium E VS2 62.7 58.0 2776.0 5.74 5.68 3.58
0.71 Ideal E VS2 62.2 57.0 2776.0 5.79 5.62 3.55
0.54 Ideal E VVS2 61.6 56.0 2776.0 5.25 5.27 3.24
0.54 Ideal E VVS2 61.5 57.0 2776.0 5.24 5.26 3.23
0.72 Ideal G SI1 61.8 56.0 2776.0 5.72 5.75 3.55
0.72 Ideal G SI1 60.7 56.0 2776.0 5.79 5.82 3.53
0.72 Good G VS2 59.7 60.5 2776.0 5.8 5.84 3.47
0.71 Ideal G SI1 60.5 56.0 2776.0 5.8 5.76 3.5
0.7 Very Good D VS1 62.7 58.0 2777.0 5.66 5.73 3.57
0.71 Premium F VS2 62.1 58.0 2777.0 5.67 5.7 3.53
0.71 Very Good F VS2 62.8 57.0 2777.0 5.64 5.69 3.56
0.71 Good F VS2 63.8 58.0 2777.0 5.61 5.64 3.59
0.71 Good F VS2 57.8 60.0 2777.0 5.87 5.9 3.4
0.7 Ideal E VS2 62.1 55.0 2777.0 5.7 5.67 3.53
0.7 Premium E VS2 61.1 60.0 2777.0 5.71 5.64 3.47
0.7 Premium E SI1 60.0 59.0 2777.0 5.79 5.75 3.46
0.7 Premium E SI1 61.2 57.0 2777.0 5.73 5.68 3.49
0.7 Premium E SI1 62.7 59.0 2777.0 5.67 5.63 3.54
0.7 Premium E SI1 61.0 57.0 2777.0 5.73 5.68 3.48
0.7 Premium E SI1 61.0 58.0 2777.0 5.78 5.72 3.51
0.7 Ideal E SI1 61.4 57.0 2777.0 5.76 5.7 3.52
0.72 Premium F SI1 61.8 61.0 2777.0 5.82 5.71 3.56
0.7 Very Good E SI1 59.9 63.0 2777.0 5.76 5.7 3.43
0.7 Premium E SI1 61.3 58.0 2777.0 5.71 5.68 3.49
0.7 Premium E SI1 60.5 58.0 2777.0 5.77 5.74 3.48
0.7 Good E VS2 64.1 59.0 2777.0 5.64 5.59 3.6
0.98 Fair H SI2 67.9 60.0 2777.0 6.05 5.97 4.08
0.78 Premium F SI1 62.4 58.0 2777.0 5.83 5.8 3.63
0.7 Very Good E SI1 63.2 60.0 2777.0 5.6 5.51 3.51
0.52 Ideal F VVS1 61.3 55.0 2778.0 5.19 5.22 3.19
0.73 Very Good H VS2 60.8 56.0 2779.0 5.82 5.84 3.55
0.74 Ideal E SI1 61.7 56.0 2779.0 5.84 5.8 3.59
0.7 Very Good F VS2 63.6 57.0 2780.0 5.61 5.65 3.58
0.77 Premium G VS2 61.2 58.0 2780.0 5.9 5.93 3.62
0.71 Ideal F VS2 62.1 54.0 2780.0 5.68 5.72 3.54
0.74 Ideal G VS1 61.5 55.0 2780.0 5.81 5.86 3.59
0.7 Ideal G VS1 61.4 59.0 2780.0 5.64 5.73 3.49
1.01 Premium F I1 61.8 60.0 2781.0 6.39 6.36 3.94
0.77 Ideal H SI1 62.2 56.0 2781.0 5.83 5.88 3.64
0.78 Ideal H SI1 61.2 56.0 2781.0 5.92 5.99 3.64
0.72 Very Good H VS1 60.6 63.0 2782.0 5.83 5.76 3.51
0.53 Very Good D VVS2 57.5 64.0 2782.0 5.34 5.37 3.08
0.76 Ideal G VS2 61.3 56.0 2782.0 5.9 5.94 3.63
0.7 Good E VS1 57.2 62.0 2782.0 5.81 5.77 3.31
0.7 Premium E VS1 62.9 60.0 2782.0 5.62 5.54 3.51
0.75 Very Good D SI2 63.1 58.0 2782.0 5.78 5.73 3.63
0.72 Ideal D SI1 60.8 57.0 2782.0 5.76 5.75 3.5
0.72 Premium D SI1 62.7 59.0 2782.0 5.73 5.69 3.58
0.7 Premium D SI1 62.8 60.0 2782.0 5.68 5.66 3.56
0.84 Fair G SI1 55.1 67.0 2782.0 6.39 6.2 3.47
0.75 Premium F SI1 61.4 59.0 2782.0 5.88 5.85 3.6
0.52 Ideal F IF 62.2 55.0 2783.0 5.14 5.18 3.21
0.72 Very Good F VS2 63.0 54.0 2784.0 5.69 5.73 3.6
0.79 Very Good H VS1 63.7 56.0 2784.0 5.85 5.92 3.75
0.72 Very Good F VS2 63.6 58.0 2787.0 5.66 5.69 3.61
0.51 Ideal F VVS1 62.0 57.0 2787.0 5.11 5.15 3.18
0.64 Ideal D VS1 61.5 56.0 2787.0 5.54 5.55 3.41
0.7 Very Good H VVS1 60.5 60.0 2788.0 5.74 5.77 3.48
0.83 Very Good I VS1 61.1 60.0 2788.0 6.07 6.1 3.72
0.76 Ideal I VVS2 61.8 56.0 2788.0 5.85 5.87 3.62
0.71 Good D VS2 63.3 56.0 2788.0 5.64 5.68 3.58
0.77 Good G VS1 59.4 64.0 2788.0 5.97 5.92 3.53
0.71 Ideal F SI1 62.5 55.0 2788.0 5.71 5.65 3.55
1.01 Fair E I1 64.5 58.0 2788.0 6.29 6.21 4.03
1.01 Premium H SI2 62.7 59.0 2788.0 6.31 6.22 3.93
0.77 Good F SI1 64.2 52.0 2789.0 5.81 5.77 3.72
0.76 Good E SI1 63.7 54.0 2789.0 5.76 5.85 3.7
0.76 Premium E SI1 60.4 58.0 2789.0 5.92 5.94 3.58
0.76 Premium E SI1 61.8 58.0 2789.0 5.82 5.86 3.61
1.05 Very Good J SI2 63.2 56.0 2789.0 6.49 6.45 4.09
0.81 Ideal G SI2 61.6 56.0 2789.0 5.97 6.01 3.69
0.7 Ideal E SI1 61.6 56.0 2789.0 5.72 5.75 3.53
0.55 Ideal G IF 60.9 57.0 2789.0 5.28 5.3 3.22
0.81 Good G SI2 61.0 61.0 2789.0 5.94 5.99 3.64
0.63 Premium E VVS2 62.1 57.0 2789.0 5.48 5.41 3.38
0.63 Premium E VVS1 60.9 60.0 2789.0 5.55 5.52 3.37
0.77 Premium H VS1 61.3 60.0 2789.0 5.9 5.88 3.61
1.05 Fair J SI2 65.8 59.0 2789.0 6.41 6.27 4.18
0.64 Ideal G IF 61.3 56.0 2790.0 5.54 5.58 3.41
0.76 Premium I VVS1 58.8 59.0 2790.0 6.0 5.94 3.51
0.83 Ideal F SI2 62.3 55.0 2790.0 6.02 6.05 3.76
0.71 Premium F VS1 60.1 62.0 2790.0 5.77 5.74 3.46
0.71 Premium F VS1 61.8 59.0 2790.0 5.73 5.69 3.53
0.87 Very Good I SI1 63.6 55.8 2791.0 6.07 6.1 3.87
0.73 Ideal E SI1 62.2 56.0 2791.0 5.74 5.78 3.58
0.71 Premium E SI1 59.2 59.0 2792.0 5.83 5.86 3.46
0.71 Premium E SI1 61.8 59.0 2792.0 5.7 5.75 3.54
0.71 Ideal E SI1 61.3 55.0 2792.0 5.72 5.77 3.52
0.7 Premium F VS1 62.1 60.0 2792.0 5.71 5.65 3.53
0.7 Premium F VS1 60.7 60.0 2792.0 5.78 5.75 3.5
0.76 Premium H VVS2 59.6 57.0 2792.0 5.91 5.86 3.51
0.7 Ideal F VS1 62.2 56.0 2792.0 5.73 5.68 3.55
0.79 Very Good G SI1 60.6 57.0 2793.0 5.98 6.06 3.65
0.7 Very Good E VS2 62.9 57.0 2793.0 5.66 5.69 3.57
0.7 Good E VS2 64.1 55.0 2793.0 5.6 5.66 3.61
0.76 Ideal I VS2 61.3 56.0 2793.0 5.87 5.91 3.61
0.73 Ideal H VS2 62.7 55.0 2793.0 5.72 5.76 3.6
0.79 Very Good E SI1 63.2 56.0 2794.0 5.91 5.86 3.72
0.71 Very Good E VS2 60.7 56.0 2795.0 5.81 5.82 3.53
0.81 Premium I VVS2 61.9 60.0 2795.0 5.91 5.86 3.64
0.81 Ideal F SI2 62.6 55.0 2795.0 5.92 5.96 3.72
0.72 Good F VS1 60.7 60.0 2795.0 5.74 5.72 3.48
0.72 Premium D SI2 62.0 60.0 2795.0 5.73 5.69 3.54
0.72 Premium I IF 63.0 57.0 2795.0 5.72 5.7 3.6
0.81 Premium H VS2 58.0 59.0 2795.0 6.17 6.13 3.57
0.72 Premium G VS2 62.9 57.0 2795.0 5.73 5.65 3.58
1.0 Premium I SI2 58.2 60.0 2795.0 6.61 6.55 3.83
0.73 Good E SI1 63.2 58.0 2796.0 5.7 5.76 3.62
0.81 Very Good H SI2 61.3 59.0 2797.0 5.94 6.01 3.66
0.81 Very Good E SI1 60.3 60.0 2797.0 6.07 6.1 3.67
0.71 Premium D SI1 62.7 60.0 2797.0 5.67 5.71 3.57
0.71 Premium D SI1 61.3 58.0 2797.0 5.73 5.75 3.52
0.71 Premium D SI1 61.6 60.0 2797.0 5.74 5.69 3.52
0.57 Ideal F VVS2 61.9 55.0 2797.0 5.34 5.35 3.31
0.51 Ideal D VVS1 61.7 56.0 2797.0 5.12 5.16 3.17
0.72 Ideal G VS2 61.9 58.0 2797.0 5.72 5.75 3.55
0.74 Ideal H VS1 61.8 58.0 2797.0 5.77 5.81 3.58
0.74 Ideal H VS1 61.6 56.0 2797.0 5.81 5.82 3.58
0.7 Fair G VVS1 58.8 66.0 2797.0 5.81 5.9 3.44
0.8 Premium F SI2 61.0 57.0 2797.0 6.03 6.01 3.67
1.01 Fair E SI2 67.4 60.0 2797.0 6.19 6.05 4.13
0.8 Very Good H VS2 63.4 60.0 2797.0 5.92 5.82 3.72
0.77 Ideal I VS1 61.5 59.0 2798.0 5.87 5.91 3.62
0.83 Very Good E SI2 58.0 62.0 2799.0 6.19 6.25 3.61
0.82 Ideal F SI2 62.4 54.0 2799.0 5.97 6.02 3.74
0.78 Ideal D SI1 61.9 57.0 2799.0 5.91 5.86 3.64
0.6 Very Good G IF 61.6 56.0 2800.0 5.43 5.46 3.35
0.9 Good I SI2 62.2 59.0 2800.0 6.07 6.11 3.79
0.7 Premium E VS1 62.2 58.0 2800.0 5.6 5.66 3.5
0.9 Very Good I SI2 61.3 56.0 2800.0 6.17 6.23 3.8
0.83 Ideal G SI1 62.3 57.0 2800.0 5.99 6.08 3.76
0.83 Ideal G SI1 61.8 57.0 2800.0 6.03 6.07 3.74
0.83 Very Good H SI1 62.5 59.0 2800.0 5.95 6.02 3.74
0.74 Premium G VS1 62.9 60.0 2800.0 5.74 5.68 3.59
0.79 Ideal I VS1 61.8 59.0 2800.0 5.92 5.95 3.67
0.61 Ideal G IF 62.3 56.0 2800.0 5.43 5.45 3.39
0.76 Fair G VS1 59.0 70.0 2800.0 5.89 5.8 3.46
0.96 Ideal F I1 60.7 55.0 2801.0 6.37 6.41 3.88
0.73 Ideal F VS2 62.5 55.0 2801.0 5.8 5.76 3.61
0.73 Premium F VS2 62.7 58.0 2801.0 5.76 5.7 3.59
0.75 Ideal H SI1 60.4 57.0 2801.0 5.93 5.96 3.59
0.71 Premium F VS2 62.1 58.0 2801.0 5.7 5.67 3.53
0.71 Good F VS2 57.8 60.0 2801.0 5.9 5.87 3.4
0.71 Good F VS2 63.8 58.0 2801.0 5.64 5.61 3.59
0.71 Premium F VS2 62.8 57.0 2801.0 5.69 5.64 3.56
1.04 Premium G I1 62.2 58.0 2801.0 6.46 6.41 4.0
1.0 Premium J SI2 62.3 58.0 2801.0 6.45 6.34 3.98
0.87 Very Good G SI2 59.9 58.0 2802.0 6.19 6.23 3.72
0.53 Ideal F IF 61.9 54.0 2802.0 5.22 5.25 3.24
0.72 Premium E VS2 63.0 55.0 2802.0 5.79 5.61 3.59
0.72 Premium F VS1 62.4 58.0 2802.0 5.83 5.7 3.6
0.7 Very Good F VS2 62.9 58.0 2803.0 5.63 5.65 3.55
0.74 Very Good E SI1 63.5 56.0 2803.0 5.74 5.79 3.66
0.71 Ideal G VS2 61.3 56.0 2803.0 5.75 5.71 3.51
0.73 Ideal E SI1 60.6 54.0 2803.0 5.84 5.89 3.55
0.7 Good G VS1 65.1 58.0 2803.0 5.56 5.59 3.63
0.71 Premium F VS2 62.6 58.0 2803.0 5.7 5.67 3.56
0.71 Premium F VS2 58.0 62.0 2803.0 5.85 5.81 3.38
0.71 Premium G VS1 62.4 61.0 2803.0 5.7 5.65 3.54
0.77 Premium G VS2 61.3 57.0 2803.0 5.93 5.88 3.62
0.71 Premium G VS2 59.9 60.0 2803.0 5.81 5.77 3.47
0.78 Premium G VS2 60.8 58.0 2803.0 6.03 5.95 3.64
0.71 Very Good G VS1 63.5 55.0 2803.0 5.66 5.64 3.59
0.91 Ideal D SI2 62.2 57.0 2803.0 6.21 6.15 3.85
0.71 Very Good E VS2 63.8 58.0 2804.0 5.62 5.66 3.6
0.71 Very Good E VS2 64.0 57.0 2804.0 5.66 5.68 3.63
0.8 Very Good E SI2 62.5 56.0 2804.0 5.88 5.96 3.7
0.7 Very Good D SI1 62.3 58.0 2804.0 5.69 5.73 3.56
0.72 Ideal F VS1 61.7 57.0 2804.0 5.74 5.77 3.55
0.72 Very Good F VS1 62.2 58.0 2804.0 5.75 5.7 3.56
0.82 Ideal H VS2 61.5 56.0 2804.0 6.01 6.08 3.72
0.7 Ideal D SI1 61.0 59.0 2804.0 5.68 5.7 3.47
0.72 Ideal D SI1 62.2 56.0 2804.0 5.74 5.77 3.58
0.72 Ideal D SI1 61.5 54.0 2804.0 5.77 5.8 3.56
0.9 Fair I SI1 67.3 59.0 2804.0 5.93 5.84 3.96
0.74 Premium F VS2 61.7 58.0 2805.0 5.85 5.78 3.59
0.74 Premium F VS2 61.9 56.0 2805.0 5.8 5.77 3.58
0.73 Ideal E SI2 61.8 58.0 2805.0 5.77 5.81 3.58
0.57 Fair E VVS1 58.7 66.0 2805.0 5.34 5.43 3.16
0.73 Premium F VS2 62.5 57.0 2805.0 5.75 5.7 3.58
0.72 Ideal G VS2 62.8 56.0 2805.0 5.74 5.7 3.59
0.74 Fair F VS2 61.1 68.0 2805.0 5.82 5.75 3.53
0.82 Good G VS2 64.0 57.0 2805.0 5.92 5.89 3.78
0.81 Very Good G SI1 62.5 60.0 2806.0 5.89 5.94 3.69
0.75 Very Good H VVS1 60.6 58.0 2806.0 5.85 5.9 3.56
0.7 Ideal F SI1 61.6 55.0 2806.0 5.72 5.74 3.53
0.71 Very Good F VS1 62.2 58.0 2807.0 5.66 5.72 3.54
0.71 Very Good F VS1 60.0 57.0 2807.0 5.84 5.9 3.52
0.93 Premium J SI2 61.9 57.0 2807.0 6.21 6.19 3.84
0.8 Very Good H VS2 62.8 57.0 2808.0 5.87 5.91 3.7
0.7 Very Good F VS1 62.0 57.0 2808.0 5.64 5.71 3.52
1.0 Fair G I1 66.4 59.0 2808.0 6.16 6.09 4.07
0.75 Very Good G VS2 63.4 56.0 2808.0 5.78 5.74 3.65
0.58 Ideal E VVS2 60.9 56.0 2808.0 5.41 5.43 3.3
0.73 Very Good D SI1 63.1 57.0 2808.0 5.74 5.7 3.61
0.81 Very Good F SI1 63.1 59.0 2809.0 5.85 5.79 3.67
0.81 Premium D SI2 59.2 57.0 2809.0 6.15 6.05 3.61
0.71 Premium F SI1 60.7 54.0 2809.0 5.84 5.8 3.53
1.2 Fair F I1 64.6 56.0 2809.0 6.73 6.66 4.33
0.7 Very Good F VS1 61.8 56.0 2810.0 5.63 5.7 3.5
0.7 Very Good F VS1 59.9 60.0 2810.0 5.77 5.84 3.48
0.74 Ideal D SI2 61.7 55.0 2810.0 5.81 5.85 3.6
0.7 Good F VS1 62.8 61.0 2810.0 5.57 5.61 3.51
0.8 Good G SI1 62.7 57.0 2810.0 5.84 5.93 3.69
0.75 Very Good F SI1 63.4 58.0 2811.0 5.72 5.76 3.64
0.83 Very Good D SI1 63.5 54.0 2811.0 5.98 5.95 3.79
1.0 Fair J VS2 65.7 59.0 2811.0 6.14 6.07 4.01
0.99 Fair I SI2 68.1 56.0 2811.0 6.21 6.06 4.18
0.7 Very Good G VS1 63.0 60.0 2812.0 5.57 5.64 3.53
0.7 Very Good F VS2 59.5 58.0 2812.0 5.75 5.85 3.45
0.7 Good E SI1 63.5 59.0 2812.0 5.49 5.53 3.5
0.7 Very Good F VS2 61.7 58.0 2812.0 5.63 5.69 3.49
0.32 Premium I SI1 62.7 58.0 554.0 4.37 4.34 2.73
0.32 Premium I SI1 62.8 58.0 554.0 4.39 4.34 2.74
0.32 Ideal I SI1 62.4 57.0 554.0 4.37 4.35 2.72
0.32 Premium I SI1 61.0 59.0 554.0 4.39 4.36 2.67
0.32 Very Good I SI1 63.1 56.0 554.0 4.39 4.36 2.76
0.32 Ideal I SI1 60.7 57.0 554.0 4.47 4.42 2.7
0.3 Premium H SI1 60.9 59.0 554.0 4.31 4.29 2.62
0.3 Premium H SI1 60.1 55.0 554.0 4.41 4.38 2.64
0.3 Premium H SI1 62.9 58.0 554.0 4.28 4.24 2.68
0.3 Very Good H SI1 63.3 56.0 554.0 4.29 4.27 2.71
0.3 Good H SI1 63.8 55.0 554.0 4.26 4.2 2.7
0.3 Ideal H SI1 62.9 57.0 554.0 4.27 4.22 2.67
0.3 Very Good H SI1 63.4 60.0 554.0 4.25 4.23 2.69
0.32 Good I SI1 63.9 55.0 554.0 4.36 4.34 2.78
0.33 Ideal H SI2 61.4 56.0 554.0 4.85 4.79 2.95
0.29 Very Good E VS1 61.9 55.0 555.0 4.28 4.33 2.66
0.29 Very Good E VS1 62.4 55.0 555.0 4.2 4.25 2.63
0.31 Very Good F SI1 61.8 58.0 555.0 4.32 4.35 2.68
0.34 Ideal H VS2 61.5 56.0 555.0 4.47 4.5 2.76
0.34 Ideal H VS2 60.4 57.0 555.0 4.54 4.57 2.75
0.34 Ideal I VS1 61.8 55.0 555.0 4.48 4.52 2.78
0.34 Ideal I VS1 62.0 56.0 555.0 4.5 4.53 2.8
0.3 Ideal G VS1 62.3 56.0 555.0 4.29 4.31 2.68
0.29 Ideal F VS1 61.6 56.0 555.0 4.26 4.31 2.64
0.35 Ideal G SI1 60.6 56.0 555.0 4.56 4.58 2.77
0.43 Very Good E I1 58.4 62.0 555.0 4.94 5.0 2.9
0.32 Very Good F VS2 61.4 58.0 556.0 4.37 4.42 2.7
0.36 Ideal I VS2 61.9 56.0 556.0 4.54 4.57 2.82
0.3 Ideal G VS2 62.0 56.0 556.0 4.28 4.3 2.66
0.26 Ideal E VS1 61.5 57.0 556.0 4.09 4.12 2.52
0.7 Very Good F VS2 62.3 58.0 2812.0 5.64 5.72 3.54
0.7 Very Good F VS2 60.9 61.0 2812.0 5.66 5.71 3.46
0.71 Ideal D SI1 62.4 57.0 2812.0 5.69 5.72 3.56
0.99 Fair J SI1 55.0 61.0 2812.0 6.72 6.67 3.68
0.73 Premium E VS2 58.6 60.0 2812.0 5.92 5.89 3.46
0.51 Ideal F VVS1 62.0 57.0 2812.0 5.15 5.11 3.18
0.91 Premium G SI2 59.8 58.0 2813.0 6.3 6.29 3.77
0.84 Very Good E SI1 63.4 55.0 2813.0 6.0 5.95 3.79
0.91 Good I VS2 64.3 58.0 2813.0 6.09 6.05 3.9
0.76 Premium E SI1 62.2 59.0 2814.0 5.86 5.81 3.63
0.76 Ideal E SI1 61.7 57.0 2814.0 5.88 5.85 3.62
0.75 Premium E SI1 61.1 59.0 2814.0 5.86 5.83 3.57
0.55 Very Good D VVS1 61.5 56.0 2815.0 5.23 5.27 3.23
0.76 Very Good F SI2 58.5 62.0 2815.0 5.93 6.01 3.49
0.74 Premium G VS1 61.7 58.0 2815.0 5.79 5.81 3.58
0.7 Ideal H SI1 60.4 56.0 2815.0 5.75 5.81 3.49
0.7 Ideal H SI1 61.4 56.0 2815.0 5.7 5.76 3.52
0.7 Ideal H SI1 61.5 55.0 2815.0 5.73 5.79 3.54
0.7 Ideal H SI1 61.4 56.0 2815.0 5.72 5.77 3.53
0.9 Fair J VS2 65.0 56.0 2815.0 6.08 6.04 3.94
0.95 Fair F SI2 56.0 60.0 2815.0 6.62 6.53 3.68
0.89 Premium H SI2 60.2 59.0 2815.0 6.26 6.23 3.76
0.72 Premium E VS2 58.3 58.0 2815.0 5.99 5.92 3.47
0.96 Fair E SI2 53.1 63.0 2815.0 6.73 6.65 3.55
1.02 Premium G I1 60.3 58.0 2815.0 6.55 6.5 3.94
0.78 Very Good I VVS2 61.4 56.0 2816.0 5.91 5.95 3.64
0.61 Ideal G VVS2 60.1 57.0 2816.0 5.52 5.54 3.32
0.71 Good D VS1 63.4 55.0 2816.0 5.61 5.69 3.58
0.78 Premium F SI1 61.5 59.0 2816.0 5.96 5.88 3.64
0.87 Ideal H SI2 62.7 56.0 2816.0 6.16 6.13 3.85
0.83 Ideal H SI1 62.5 55.0 2816.0 6.04 6.0 3.76
0.71 Premium E SI1 61.3 56.0 2817.0 5.78 5.73 3.53
0.71 Ideal I VVS2 60.2 56.0 2817.0 5.84 5.89 3.53
0.71 Ideal E VS2 62.7 57.0 2817.0 5.66 5.64 3.54
0.71 Premium E VS2 62.3 58.0 2817.0 5.69 5.65 3.53
0.63 Ideal F VVS2 61.5 56.0 2817.0 5.48 5.52 3.38
0.71 Premium E SI1 59.2 59.0 2817.0 5.86 5.83 3.46
0.71 Premium E SI1 61.8 59.0 2817.0 5.75 5.7 3.54
0.71 Ideal E SI1 61.3 55.0 2817.0 5.77 5.72 3.52
0.71 Premium E SI1 61.4 58.0 2817.0 5.77 5.73 3.53
0.9 Ideal J VS2 62.8 55.0 2817.0 6.2 6.16 3.88
0.71 Good E SI1 62.8 64.0 2817.0 5.6 5.54 3.5
0.7 Premium E VS2 62.4 61.0 2818.0 5.66 5.63 3.52
0.7 Premium E VS2 59.3 60.0 2818.0 5.78 5.73 3.41
0.7 Premium E VS2 63.0 60.0 2818.0 5.64 5.6 3.54
1.0 Premium H I1 61.3 60.0 2818.0 6.43 6.39 3.93
0.86 Premium F SI2 59.3 62.0 2818.0 6.36 6.22 3.73
0.8 Ideal H SI1 61.0 57.0 2818.0 6.07 6.0 3.68
0.7 Ideal E VS1 62.9 57.0 2818.0 5.66 5.61 3.54
0.7 Premium E VS1 59.6 57.0 2818.0 5.91 5.83 3.5
0.7 Premium F VS2 61.8 60.0 2818.0 5.69 5.64 3.5
0.7 Premium E VS1 62.7 57.0 2818.0 5.68 5.64 3.55
1.0 Fair H SI2 65.3 62.0 2818.0 6.34 6.12 4.08
0.72 Very Good G VS1 63.8 58.0 2819.0 5.64 5.68 3.61
0.72 Ideal H VS1 62.3 56.0 2819.0 5.73 5.77 3.58
0.7 Good F VS1 59.7 63.0 2819.0 5.76 5.79 3.45
0.86 Good F SI2 64.3 60.0 2819.0 5.97 5.95 3.83
0.71 Ideal G VS1 62.9 58.0 2820.0 5.66 5.69 3.57
0.75 Ideal E SI1 62.0 57.0 2821.0 5.8 5.78 3.59
0.73 Premium E VS2 61.6 59.0 2821.0 5.77 5.73 3.54
0.53 Ideal E VVS1 61.9 55.0 2821.0 5.2 5.21 3.22
0.73 Premium E SI1 61.3 58.0 2821.0 5.83 5.76 3.55
0.73 Good E SI1 63.6 57.0 2821.0 5.72 5.7 3.63
0.73 Premium E SI1 59.6 61.0 2821.0 5.92 5.85 3.51
0.73 Premium E SI1 62.2 59.0 2821.0 5.77 5.68 3.56
0.73 Premium D SI1 61.7 55.0 2821.0 5.84 5.82 3.6
0.73 Very Good E SI1 63.2 58.0 2821.0 5.76 5.7 3.62
0.7 Premium E VS1 60.8 60.0 2822.0 5.74 5.71 3.48
0.72 Premium E VS2 60.3 59.0 2822.0 5.84 5.8 3.51
0.72 Premium E VS2 60.9 60.0 2822.0 5.8 5.76 3.52
0.72 Premium E VS2 62.4 59.0 2822.0 5.77 5.7 3.58
0.7 Premium E VS2 60.2 60.0 2822.0 5.73 5.7 3.44
0.6 Ideal F VVS2 62.0 55.0 2822.0 5.37 5.4 3.34
0.74 Ideal I VVS1 60.8 57.0 2822.0 5.85 5.89 3.57
0.73 Ideal F SI1 62.1 55.0 2822.0 5.75 5.78 3.58
0.71 Premium D SI1 62.7 60.0 2822.0 5.71 5.67 3.57
0.71 Premium D SI1 61.3 58.0 2822.0 5.75 5.73 3.52
0.7 Premium D SI1 60.2 60.0 2822.0 5.82 5.75 3.48
0.7 Ideal D SI1 60.7 56.0 2822.0 5.75 5.72 3.48
0.9 Good J VS2 64.0 61.0 2822.0 6.04 6.03 3.86
0.71 Ideal D SI1 60.2 56.0 2822.0 5.86 5.83 3.52
0.7 Premium E VS2 61.5 59.0 2822.0 5.73 5.68 3.51
0.7 Premium E VS2 62.6 56.0 2822.0 5.71 5.66 3.56
0.7 Ideal D SI1 59.7 58.0 2822.0 5.82 5.77 3.46
0.7 Good E SI1 61.4 64.0 2822.0 5.71 5.66 3.49
0.7 Ideal D SI1 62.5 57.0 2822.0 5.62 5.59 3.51
0.7 Ideal D SI1 61.8 56.0 2822.0 5.73 5.63 3.51
0.7 Premium E VS2 60.7 62.0 2822.0 5.72 5.68 3.46
0.7 Premium F VS2 60.6 58.0 2822.0 5.8 5.72 3.49
0.7 Ideal D SI1 61.4 54.0 2822.0 5.75 5.71 3.52
0.79 Very Good D SI2 62.8 59.0 2823.0 5.86 5.9 3.69
0.9 Good I SI1 63.8 57.0 2823.0 6.06 6.13 3.89
0.71 Premium E VS2 62.3 58.0 2823.0 5.71 5.66 3.54
0.61 Ideal E VVS2 61.3 54.0 2823.0 5.51 5.59 3.4
0.9 Fair H SI2 65.8 54.0 2823.0 6.05 5.98 3.96
0.71 Ideal E SI1 60.5 56.0 2823.0 5.77 5.73 3.47
0.71 Premium D VS2 61.2 59.0 2824.0 5.74 5.69 3.5
0.77 Ideal I VVS2 62.1 57.0 2824.0 5.84 5.86 3.63
0.74 Good E VS1 63.1 58.0 2824.0 5.73 5.75 3.62
0.82 Ideal F SI2 62.4 54.0 2824.0 6.02 5.97 3.74
0.82 Premium E SI2 60.8 60.0 2824.0 6.05 6.03 3.67
0.71 Premium G VS1 62.2 59.0 2825.0 5.73 5.66 3.54
0.83 Premium H SI1 60.0 59.0 2825.0 6.08 6.05 3.64
0.73 Very Good G VS1 62.0 57.0 2825.0 5.75 5.8 3.58
0.83 Premium H SI1 62.5 59.0 2825.0 6.02 5.95 3.74
1.17 Premium J I1 60.2 61.0 2825.0 6.9 6.83 4.13
0.91 Fair H SI2 61.3 67.0 2825.0 6.24 6.19 3.81
0.73 Premium E VS1 62.6 60.0 2826.0 5.75 5.68 3.58
0.7 Good E VS1 57.2 59.0 2826.0 5.94 5.88 3.38
0.9 Premium I SI2 62.2 59.0 2826.0 6.11 6.07 3.79
0.7 Premium E VS1 62.2 58.0 2826.0 5.66 5.6 3.5
0.7 Very Good D VS2 63.3 56.0 2826.0 5.6 5.58 3.54
0.7 Premium E VS1 59.4 61.0 2826.0 5.78 5.74 3.42
0.9 Very Good I SI2 63.5 56.0 2826.0 6.17 6.07 3.88
0.78 Premium F SI1 60.8 60.0 2826.0 5.97 5.94 3.62
0.96 Ideal F I1 60.7 55.0 2826.0 6.41 6.37 3.88
0.7 Very Good D SI1 62.3 59.0 2827.0 5.67 5.7 3.54
0.72 Good D VS2 64.0 54.0 2827.0 5.68 5.7 3.64
0.79 Premium H VVS2 62.6 58.0 2827.0 5.96 5.9 3.71
0.7 Ideal H VVS1 61.6 57.0 2827.0 5.69 5.74 3.52
0.7 Ideal H VVS1 62.3 55.0 2827.0 5.66 5.7 3.54
0.7 Ideal D SI2 60.6 57.0 2828.0 5.74 5.77 3.49
1.01 Premium H SI2 61.6 61.0 2828.0 6.39 6.31 3.91
0.72 Premium F VS1 62.2 58.0 2829.0 5.75 5.7 3.56
0.8 Good E SI2 63.7 54.0 2829.0 5.91 5.87 3.75
0.59 Ideal E VVS1 62.0 56.0 2829.0 5.36 5.38 3.33
0.72 Ideal F VS1 61.7 57.0 2829.0 5.77 5.74 3.55
0.75 Premium E SI2 61.9 57.0 2829.0 5.88 5.82 3.62
0.8 Premium E SI2 60.2 57.0 2829.0 6.05 6.01 3.63
0.71 Very Good E VS2 62.7 59.0 2830.0 5.65 5.7 3.56
0.77 Very Good H SI1 61.7 56.0 2830.0 5.84 5.89 3.62
0.97 Ideal F I1 60.7 56.0 2830.0 6.41 6.43 3.9
0.53 Ideal F VVS1 60.9 57.0 2830.0 5.23 5.29 3.19
0.53 Ideal F VVS1 61.8 57.0 2830.0 5.16 5.19 3.2
0.8 Ideal I VS2 62.1 54.4 2830.0 5.94 5.99 3.7
0.9 Premium G SI1 60.6 62.0 2830.0 6.21 6.13 3.74
0.76 Very Good E SI2 60.8 60.0 2831.0 5.89 5.98 3.61
0.72 Ideal E SI1 62.3 57.0 2831.0 5.7 5.76 3.57
0.75 Ideal E SI1 61.4 57.0 2831.0 5.82 5.87 3.59
0.72 Premium E SI1 62.1 58.0 2831.0 5.73 5.76 3.57
0.79 Ideal G SI1 61.8 56.0 2831.0 5.93 5.91 3.66
0.72 Very Good F VS2 62.5 58.0 2832.0 5.71 5.75 3.58
0.91 Very Good I SI2 62.8 61.0 2832.0 6.15 6.18 3.87
0.71 Premium G VVS2 62.1 57.0 2832.0 5.75 5.65 3.54
0.81 Premium G SI1 63.0 60.0 2832.0 5.87 5.81 3.68
0.82 Ideal H SI1 62.5 57.0 2832.0 5.91 5.97 3.71
0.71 Premium F VS1 62.2 58.0 2832.0 5.72 5.66 3.54
0.9 Good J SI1 64.3 63.0 2832.0 6.05 6.01 3.88
0.8 Very Good I VS2 62.0 58.0 2833.0 5.86 5.95 3.66
0.56 Very Good E IF 61.0 59.0 2833.0 5.28 5.34 3.24
0.7 Very Good D VS2 59.6 61.0 2833.0 5.77 5.8 3.45
0.7 Ideal D VS2 61.0 57.0 2833.0 5.74 5.76 3.51
0.61 Ideal F VVS2 61.7 55.0 2833.0 5.45 5.48 3.37
0.85 Ideal H SI2 62.5 57.0 2833.0 6.02 6.07 3.78
0.7 Ideal F SI1 60.7 57.0 2833.0 5.73 5.75 3.49
0.8 Ideal G VS2 62.2 56.0 2834.0 5.94 5.87 3.67
0.8 Ideal H VS2 62.8 57.0 2834.0 5.91 5.87 3.7
0.51 Very Good D VVS1 59.9 58.0 2834.0 5.16 5.19 3.1
0.53 Ideal F VVS1 61.4 57.0 2834.0 5.2 5.23 3.2
0.78 Ideal I VS2 61.8 55.0 2834.0 5.92 5.95 3.67
0.9 Very Good J SI1 63.4 54.0 2834.0 6.17 6.14 3.9
0.9 Fair G SI2 65.3 59.0 2834.0 6.07 6.0 3.94
0.77 Ideal E SI2 60.7 55.0 2834.0 6.01 5.95 3.63
0.73 Ideal F VS1 61.2 56.0 2835.0 5.89 5.81 3.58
0.63 Ideal F VVS2 61.9 57.0 2835.0 5.47 5.51 3.4
0.7 Ideal E VS2 61.5 54.0 2835.0 5.7 5.75 3.52
0.72 Ideal E VS2 62.8 57.0 2835.0 5.71 5.73 3.59
0.72 Ideal E SI1 61.0 57.0 2835.0 5.78 5.8 3.53
0.75 Premium F VS2 59.6 59.0 2835.0 6.04 5.94 3.57
0.82 Very Good H SI1 60.7 56.0 2836.0 6.04 6.06 3.67
0.71 Good E VS2 62.8 60.0 2836.0 5.6 5.65 3.53
0.7 Premium E VS1 62.6 59.0 2837.0 5.69 5.66 3.55
0.7 Ideal E VS1 61.8 56.0 2837.0 5.74 5.69 3.53
0.71 Ideal F SI1 59.8 53.0 2838.0 5.86 5.82 3.49
0.76 Very Good H SI1 60.9 55.0 2838.0 5.92 5.94 3.61
0.82 Fair F SI1 64.9 58.0 2838.0 5.83 5.79 3.77
0.72 Premium F VS1 58.8 60.0 2838.0 5.91 5.89 3.47
0.7 Premium F VS2 62.3 58.0 2838.0 5.72 5.64 3.54
0.7 Premium F VS2 61.7 58.0 2838.0 5.69 5.63 3.49
0.7 Premium G VS1 62.6 55.0 2838.0 5.73 5.64 3.56
0.7 Premium F VS2 59.4 61.0 2838.0 5.83 5.79 3.45
0.7 Very Good E SI1 63.5 59.0 2838.0 5.53 5.49 3.5
0.7 Premium F VS2 60.9 61.0 2838.0 5.71 5.66 3.46
0.7 Premium F VS2 59.5 58.0 2838.0 5.85 5.75 3.45
0.7 Premium G VS1 63.0 60.0 2838.0 5.64 5.57 3.53
0.74 Very Good E SI1 60.0 57.0 2839.0 5.85 5.89 3.52
0.71 Ideal F VS1 61.5 57.0 2839.0 5.74 5.71 3.52
0.7 Ideal F VS1 61.6 54.0 2839.0 5.75 5.72 3.53
0.71 Ideal F VS1 62.1 55.0 2839.0 5.82 5.68 3.57
0.71 Premium F VS1 59.1 61.0 2839.0 5.84 5.81 3.44
0.71 Premium F VS1 59.0 60.0 2839.0 5.82 5.8 3.43
0.71 Premium F VS1 60.5 58.0 2839.0 5.75 5.72 3.47
0.7 Ideal F VS1 62.4 53.0 2839.0 5.73 5.71 3.57
0.73 Ideal G VS2 61.8 54.0 2839.0 5.8 5.82 3.59
0.7 Ideal E VS2 62.1 54.0 2839.0 5.69 5.72 3.54
0.7 Ideal G VS1 61.3 57.0 2839.0 5.71 5.74 3.51
0.71 Premium G VVS2 60.3 58.0 2839.0 5.82 5.78 3.5
0.71 Premium F VS1 59.2 58.0 2839.0 5.87 5.82 3.46
0.79 Premium G VS2 59.3 62.0 2839.0 6.09 6.01 3.59
0.71 Premium F VS1 62.7 59.0 2839.0 5.7 5.62 3.55
0.77 Very Good H VS1 61.0 60.0 2840.0 5.9 5.87 3.59
0.75 Very Good F SI2 59.8 56.0 2840.0 5.85 5.92 3.52
0.7 Ideal F SI1 61.0 56.0 2840.0 5.75 5.8 3.52
0.71 Premium F VS2 59.3 56.0 2840.0 5.88 5.82 3.47
0.92 Ideal D SI2 61.9 56.0 2840.0 6.27 6.2 3.86
0.83 Premium F SI2 61.4 59.0 2840.0 6.08 6.04 3.72
0.7 Premium H VVS1 59.2 60.0 2840.0 5.87 5.78 3.45
0.73 Premium F VS2 60.3 59.0 2841.0 5.9 5.87 3.55
0.71 Very Good D VS1 63.4 55.0 2841.0 5.69 5.61 3.58
0.73 Very Good D SI1 63.9 57.0 2841.0 5.66 5.71 3.63
0.82 Ideal F SI2 61.7 53.0 2841.0 6.0 6.12 3.74
0.82 Ideal F SI2 62.3 56.0 2841.0 5.96 6.02 3.73
0.82 Very Good F SI2 59.7 57.0 2841.0 6.12 6.14 3.66
0.52 Ideal F VVS1 61.2 56.0 2841.0 5.19 5.21 3.18
1.0 Premium F I1 58.9 60.0 2841.0 6.6 6.55 3.87
0.95 Fair G SI1 66.7 56.0 2841.0 6.16 6.03 4.06
0.73 Ideal D SI1 61.4 57.0 2841.0 5.76 5.8 3.55
0.73 Premium F VS2 59.9 59.0 2841.0 5.87 5.77 3.5
0.73 Premium G VS1 61.4 58.0 2841.0 5.82 5.77 3.56
0.8 Ideal I VS1 62.6 54.0 2842.0 5.92 5.96 3.72
0.7 Premium F VS2 58.7 61.0 2842.0 5.8 5.72 3.38
0.7 Very Good E VS2 60.2 62.0 2843.0 5.71 5.75 3.45
0.7 Very Good E VS2 62.7 58.0 2843.0 5.65 5.67 3.55
0.71 Very Good E VS2 59.4 58.0 2843.0 5.76 5.82 3.44
0.81 Very Good F SI2 63.2 58.0 2843.0 5.91 5.92 3.74
0.71 Very Good D SI1 61.5 58.0 2843.0 5.73 5.79 3.54
0.73 Ideal G VVS2 61.3 57.0 2843.0 5.81 5.84 3.57
0.73 Very Good F VS1 61.8 59.0 2843.0 5.73 5.79 3.56
0.72 Ideal E VS2 62.0 57.0 2843.0 5.71 5.74 3.55
0.81 Ideal F SI2 62.1 57.0 2843.0 5.91 5.95 3.68
0.71 Ideal G VVS2 60.7 57.0 2843.0 5.81 5.78 3.52
0.73 Very Good E SI1 57.7 61.0 2844.0 5.92 5.96 3.43
0.7 Very Good E VS1 62.0 59.0 2844.0 5.65 5.68 3.51
1.01 Ideal I I1 61.5 57.0 2844.0 6.45 6.46 3.97
1.01 Good I I1 63.1 57.0 2844.0 6.35 6.39 4.02
0.79 Ideal H VS2 62.5 57.0 2844.0 5.91 5.93 3.7
0.7 Very Good E VS2 61.8 59.0 2845.0 5.65 5.68 3.5
0.7 Very Good E VS2 58.9 60.0 2845.0 5.83 5.85 3.44
0.8 Good H VS2 63.4 60.0 2845.0 5.92 5.82 3.72
1.27 Premium H SI2 59.3 61.0 2845.0 7.12 7.05 4.2
0.79 Ideal D SI1 61.5 56.0 2846.0 5.96 5.91 3.65
0.72 Very Good F VS1 60.2 59.0 2846.0 5.79 5.84 3.5
0.73 Ideal H VVS2 61.6 56.0 2846.0 5.79 5.84 3.58
1.01 Fair H SI2 65.4 59.0 2846.0 6.3 6.26 4.11
1.01 Good H I1 64.2 61.0 2846.0 6.25 6.18 3.99
0.73 Ideal E SI1 59.1 59.0 2846.0 5.92 5.95 3.51
0.7 Ideal E SI1 61.6 57.0 2846.0 5.71 5.76 3.53
0.7 Good F VS2 59.1 61.0 2846.0 5.76 5.84 3.43
0.77 Premium E SI1 62.9 59.0 2846.0 5.84 5.79 3.66
0.77 Premium G VS2 61.3 60.0 2846.0 5.91 5.81 3.59
0.77 Premium G VS1 61.4 58.0 2846.0 5.94 5.89 3.63
0.84 Very Good H SI1 61.2 57.0 2847.0 6.1 6.12 3.74
0.72 Ideal E SI1 60.3 57.0 2847.0 5.83 5.85 3.52
0.76 Premium D SI1 61.1 59.0 2847.0 5.93 5.88 3.61
0.7 Very Good G VVS2 62.9 59.0 2848.0 5.61 5.68 3.55
0.54 Ideal D VVS2 61.5 55.0 2848.0 5.25 5.29 3.24
0.75 Fair D SI2 64.6 57.0 2848.0 5.74 5.72 3.7
0.79 Good E SI1 64.1 54.0 2849.0 5.86 5.84 3.75
0.74 Very Good E VS1 63.1 58.0 2849.0 5.75 5.73 3.62
0.7 Very Good E VS2 61.0 60.0 2850.0 5.74 5.77 3.51
0.7 Ideal F VS2 60.8 59.0 2850.0 5.69 5.79 3.49
0.75 Ideal J SI1 61.5 56.0 2850.0 5.83 5.87 3.6
1.2 Very Good H I1 63.1 60.0 2850.0 6.75 6.67 4.23
0.8 Very Good F SI1 63.4 57.0 2851.0 5.89 5.82 3.71
0.66 Ideal D VS1 62.1 56.0 2851.0 5.54 5.57 3.45
0.87 Very Good F SI2 61.0 63.0 2851.0 6.22 6.07 3.75
0.86 Premium H SI1 62.7 59.0 2851.0 6.04 5.98 3.77
0.74 Ideal F SI1 61.0 57.0 2851.0 5.85 5.81 3.56
0.58 Very Good E IF 60.6 59.0 2852.0 5.37 5.43 3.27
0.78 Ideal I VS1 61.5 57.0 2852.0 5.88 5.92 3.63
0.74 Ideal G SI1 61.3 55.0 2852.0 5.85 5.86 3.59
0.73 Ideal E SI1 62.7 55.0 2852.0 5.7 5.79 3.6
0.91 Very Good I SI1 63.5 57.0 2852.0 6.12 6.07 3.87
0.71 Premium F VS2 62.6 58.0 2853.0 5.67 5.7 3.56
0.71 Good G VS1 63.5 55.0 2853.0 5.64 5.66 3.59
0.79 Ideal D SI2 62.8 57.0 2853.0 5.9 5.85 3.69
0.79 Premium D SI2 60.0 60.0 2853.0 6.07 6.03 3.63
0.71 Premium E SI1 62.7 58.0 2853.0 5.73 5.66 3.57
0.82 Premium I VS1 61.9 58.0 2853.0 5.99 5.97 3.7
0.78 Very Good H VS1 61.9 57.1 2854.0 5.87 5.95 3.66
0.7 Very Good E VS1 62.4 56.0 2854.0 5.64 5.7 3.54
1.12 Premium H I1 59.1 61.0 2854.0 6.78 6.75 4.0
0.73 Premium E VS2 62.0 57.0 2854.0 5.86 5.76 3.6
0.91 Fair J VS2 64.4 62.0 2854.0 6.06 6.03 3.89
0.91 Fair J VS2 65.4 60.0 2854.0 6.04 6.0 3.94
0.91 Good J VS2 64.2 58.0 2854.0 6.12 6.09 3.92
0.91 Fair H SI1 65.8 58.0 2854.0 6.04 6.01 3.96
0.7 Premium E VS1 58.4 59.0 2854.0 5.91 5.83 3.43
0.68 Premium F VVS2 61.7 57.0 2854.0 5.67 5.64 3.49
0.73 Very Good F VS2 62.5 57.0 2855.0 5.7 5.75 3.58
1.03 Good J SI1 63.6 57.0 2855.0 6.38 6.29 4.03
0.74 Premium D VS2 62.4 57.0 2855.0 5.8 5.74 3.6
0.98 Fair E SI2 53.3 67.0 2855.0 6.82 6.74 3.61
1.02 Fair I SI1 53.0 63.0 2856.0 6.84 6.77 3.66
1.0 Fair G SI2 67.8 61.0 2856.0 5.96 5.9 4.02
1.02 Ideal H SI2 61.6 55.0 2856.0 6.49 6.43 3.98
0.6 Ideal F VVS2 60.8 57.0 2856.0 5.44 5.49 3.32
0.8 Ideal G SI2 61.6 56.0 2856.0 5.97 6.01 3.69
0.97 Ideal F I1 60.7 56.0 2856.0 6.43 6.41 3.9
1.0 Fair I SI1 67.9 62.0 2856.0 6.19 6.03 4.15
0.26 Ideal E VS1 62.3 57.0 556.0 4.05 4.08 2.53
0.26 Ideal E VS1 62.1 56.0 556.0 4.09 4.12 2.55
0.36 Ideal H SI1 61.9 55.0 556.0 4.57 4.59 2.83
0.34 Good G VS2 57.5 61.0 556.0 4.6 4.66 2.66
0.34 Good E SI1 63.3 57.0 556.0 4.44 4.47 2.82
0.34 Good E SI1 63.5 55.0 556.0 4.44 4.47 2.83
0.34 Good E SI1 63.4 55.0 556.0 4.44 4.46 2.82
0.34 Very Good G VS2 59.6 62.0 556.0 4.54 4.56 2.71
0.34 Ideal E SI1 62.2 54.0 556.0 4.47 4.5 2.79
0.32 Good E VS2 64.1 54.0 556.0 4.34 4.37 2.79
0.31 Ideal I VVS1 61.6 55.0 557.0 4.36 4.41 2.7
0.31 Ideal I VVS1 61.3 56.0 557.0 4.36 4.38 2.68
0.31 Ideal I VVS1 62.3 54.0 557.0 4.37 4.4 2.73
0.31 Ideal I VVS1 62.0 54.0 557.0 4.37 4.4 2.72
0.31 Ideal I VVS1 62.7 53.0 557.0 4.33 4.35 2.72
0.31 Ideal I VVS1 62.2 53.0 557.0 4.36 4.38 2.72
0.31 Ideal G VS2 62.2 53.6 557.0 4.32 4.35 2.7
0.31 Ideal H VS1 61.6 54.8 557.0 4.35 4.37 2.69
0.31 Ideal H VS1 61.8 54.2 557.0 4.33 4.37 2.69
0.33 Premium G SI2 59.4 59.0 557.0 4.52 4.5 2.68
0.33 Premium F SI2 62.3 58.0 557.0 4.43 4.4 2.75
0.33 Premium G SI2 62.6 58.0 557.0 4.42 4.4 2.76
0.33 Ideal G SI2 61.9 56.0 557.0 4.45 4.41 2.74
0.33 Premium F SI2 63.0 58.0 557.0 4.42 4.4 2.78
0.33 Premium J VS1 62.8 58.0 557.0 4.41 4.38 2.76
0.33 Premium J VS1 61.5 61.0 557.0 4.46 4.39 2.72
0.33 Ideal J VS1 62.1 55.0 557.0 4.44 4.41 2.75
0.33 Ideal I SI1 63.0 57.0 557.0 4.39 4.37 2.76
0.33 Good I SI1 63.6 53.0 557.0 4.43 4.4 2.81
0.33 Premium I SI1 60.4 59.0 557.0 4.54 4.5 2.73
1.0 Fair H SI2 66.1 56.0 2856.0 6.21 5.97 4.04
0.77 Premium F SI1 60.8 59.0 2856.0 5.92 5.86 3.58
0.77 Premium F SI1 61.0 58.0 2856.0 5.94 5.9 3.61
0.7 Good E VVS2 60.1 63.0 2857.0 5.68 5.71 3.42
0.9 Very Good G SI2 63.1 58.0 2857.0 6.08 6.02 3.82
0.72 Ideal E SI1 62.3 57.0 2857.0 5.76 5.7 3.57
0.9 Premium I VS2 61.9 59.0 2857.0 6.2 6.14 3.82
0.72 Premium E SI1 62.1 58.0 2857.0 5.76 5.73 3.57
0.7 Ideal G VVS2 62.1 56.0 2858.0 5.63 5.71 3.52
0.81 Very Good F SI1 61.3 57.0 2858.0 6.02 6.05 3.7
0.81 Very Good F SI1 61.7 57.0 2858.0 6.0 6.05 3.72
0.71 Premium E VS2 61.0 60.0 2858.0 5.76 5.69 3.49
0.7 Premium E VS2 61.4 59.0 2858.0 5.73 5.7 3.51
0.71 Premium E VS2 61.5 60.0 2858.0 5.76 5.68 3.52
0.71 Very Good E VS2 63.5 59.0 2858.0 5.68 5.59 3.58
0.92 Premium J SI1 62.9 58.0 2858.0 6.22 6.18 3.9
0.76 Ideal E SI1 62.7 54.0 2858.0 5.88 5.83 3.67
0.73 Ideal D SI1 61.5 56.0 2858.0 5.84 5.8 3.58
0.71 Premium D VS2 60.4 62.0 2858.0 5.74 5.72 3.46
0.7 Good E VVS2 63.6 62.0 2858.0 5.61 5.58 3.56
0.9 Fair G SI2 64.5 56.0 2858.0 6.06 6.0 3.89
0.71 Fair D VS2 56.9 65.0 2858.0 5.89 5.84 3.34
0.7 Ideal D VS2 61.0 57.0 2859.0 5.76 5.74 3.51
0.7 Premium D VS2 62.4 56.0 2859.0 5.72 5.66 3.55
0.77 Premium F VS1 60.9 60.0 2859.0 5.91 5.88 3.59
0.71 Ideal G VS1 61.5 56.0 2859.0 5.74 5.78 3.54
0.7 Premium D VS2 59.6 61.0 2859.0 5.8 5.77 3.45
0.75 Fair F VS1 55.8 70.0 2859.0 6.09 5.98 3.37
0.83 Premium E SI2 59.2 60.0 2859.0 6.17 6.12 3.64
0.71 Very Good F VS2 61.3 61.0 2860.0 5.68 5.73 3.5
0.9 Very Good J SI2 63.6 58.0 2860.0 6.07 6.1 3.87
0.6 Ideal E VVS2 61.9 54.9 2860.0 5.41 5.44 3.35
0.71 Premium D VS1 62.9 57.0 2860.0 5.66 5.6 3.54
0.53 Ideal F VVS1 61.4 57.0 2860.0 5.23 5.2 3.2
0.71 Premium D SI1 60.7 58.0 2861.0 5.95 5.78 3.56
0.62 Ideal G VVS2 61.6 56.0 2861.0 5.45 5.48 3.37
0.62 Ideal G VVS2 61.6 56.0 2861.0 5.48 5.51 3.38
0.9 Premium I SI1 63.0 58.0 2861.0 6.09 6.01 3.81
0.62 Fair F IF 60.1 61.0 2861.0 5.53 5.56 3.33
0.82 Premium E SI2 61.7 59.0 2861.0 6.01 5.98 3.7
0.66 Premium D VS1 61.0 58.0 2861.0 5.67 5.57 3.43
0.7 Very Good D SI1 62.5 55.0 2862.0 5.67 5.72 3.56
0.8 Very Good F SI1 62.6 58.0 2862.0 5.9 5.92 3.7
0.8 Very Good D SI2 62.5 59.0 2862.0 5.88 5.92 3.69
0.79 Premium F SI1 62.3 54.0 2862.0 5.97 5.91 3.7
0.71 Very Good F VVS1 63.2 60.0 2862.0 5.65 5.61 3.56
0.7 Ideal H VS2 61.1 57.0 2862.0 5.71 5.74 3.5
0.7 Very Good E VS2 58.7 63.0 2862.0 5.73 5.69 3.35
0.79 Premium H VS1 60.0 60.0 2862.0 6.07 5.99 3.64
0.7 Premium E VS2 59.5 59.0 2862.0 5.82 5.77 3.45
1.22 Premium E I1 60.9 57.0 2862.0 6.93 6.88 4.21
1.01 Fair E SI2 67.6 57.0 2862.0 6.21 6.11 4.18
0.73 Premium E VS2 62.5 61.0 2862.0 5.78 5.64 3.59
0.91 Good I VS2 64.3 58.0 2863.0 6.05 6.09 3.9
0.71 Ideal D SI1 60.8 56.0 2863.0 5.8 5.77 3.52
0.83 Premium G SI1 62.3 58.0 2863.0 6.01 5.97 3.73
0.84 Premium F SI2 62.3 59.0 2863.0 6.06 6.01 3.76
0.71 Premium D SI1 61.0 61.0 2863.0 5.82 5.75 3.53
0.71 Premium D SI1 59.7 59.0 2863.0 5.82 5.8 3.47
0.71 Premium D SI1 61.7 56.0 2863.0 5.8 5.68 3.54
0.71 Ideal D SI1 61.7 57.0 2863.0 5.75 5.7 3.53
0.71 Premium D SI1 61.4 58.0 2863.0 5.79 5.75 3.54
0.71 Premium D SI1 60.6 58.0 2863.0 5.79 5.77 3.5
0.91 Premium J SI1 59.5 62.0 2863.0 6.4 6.18 3.74
0.9 Premium J VS2 59.8 62.0 2863.0 6.24 6.21 3.72
0.71 Premium H VVS2 61.5 62.0 2863.0 5.74 5.68 3.51
0.71 Premium E SI1 59.1 61.0 2863.0 5.84 5.8 3.44
0.72 Ideal F VS2 59.5 57.0 2863.0 5.91 5.86 3.5
0.72 Premium E SI1 60.9 60.0 2863.0 5.78 5.74 3.51
0.71 Ideal E VS2 61.0 55.0 2863.0 5.79 5.75 3.52
0.81 Ideal E SI2 60.3 57.0 2864.0 6.07 6.04 3.65
0.83 Very Good I VS2 61.6 58.0 2865.0 6.05 6.07 3.73
0.73 Premium D SI1 60.8 55.0 2865.0 5.87 5.81 3.55
0.56 Very Good D VVS1 62.0 56.0 2866.0 5.25 5.3 3.27
0.56 Very Good D VVS1 61.8 55.0 2866.0 5.27 5.31 3.27
0.71 Ideal E VS1 62.2 55.0 2866.0 5.74 5.7 3.56
0.7 Ideal H VVS1 62.3 58.0 2866.0 5.66 5.7 3.54
0.96 Premium I SI1 61.3 58.0 2866.0 6.39 6.3 3.89
0.71 Very Good H VVS1 62.9 57.0 2867.0 5.67 5.69 3.57
0.7 Ideal D VS2 62.4 57.0 2867.0 5.68 5.61 3.52
0.71 Ideal H VVS1 60.4 57.0 2867.0 5.78 5.81 3.5
0.8 Premium H VS2 61.2 53.0 2867.0 6.05 5.98 3.68
0.95 Premium F SI2 58.4 57.0 2867.0 6.49 6.41 3.77
0.82 Ideal F SI2 62.3 56.0 2867.0 5.99 5.95 3.72
0.52 Ideal F VVS1 61.2 56.0 2867.0 5.21 5.19 3.18
0.82 Ideal F SI2 61.7 53.0 2867.0 6.12 6.0 3.74
0.82 Ideal F SI2 62.3 56.0 2867.0 6.02 5.96 3.73
0.82 Premium F SI2 59.7 57.0 2867.0 6.14 6.12 3.66
0.8 Ideal G SI1 61.3 57.0 2867.0 5.96 5.91 3.64
0.96 Fair F SI2 68.2 61.0 2867.0 6.07 5.88 4.1
0.72 Ideal I VS1 62.4 55.0 2868.0 5.72 5.75 3.58
0.62 Ideal G IF 60.5 57.0 2868.0 5.52 5.56 3.35
0.79 Premium E SI2 61.0 58.0 2868.0 5.96 5.9 3.62
0.75 Very Good E SI1 63.1 56.0 2868.0 5.78 5.7 3.62
1.08 Premium D I1 61.9 60.0 2869.0 6.55 6.48 4.03
0.72 Ideal E SI1 60.8 55.0 2869.0 5.77 5.84 3.53
0.62 Ideal G IF 61.8 56.0 2869.0 5.43 5.47 3.37
0.73 Ideal G VVS2 61.3 57.0 2869.0 5.84 5.81 3.57
0.72 Ideal H VVS2 60.9 57.0 2869.0 5.79 5.77 3.52
0.52 Premium F VVS2 61.8 60.0 2870.0 5.16 5.13 3.18
0.83 Ideal E SI2 62.2 57.0 2870.0 6.0 6.05 3.75
0.64 Premium E VVS2 62.1 58.0 2870.0 5.56 5.51 3.44
0.8 Ideal G SI1 62.5 57.0 2870.0 5.94 5.9 3.7
0.74 Ideal H SI1 62.1 56.0 2870.0 5.77 5.83 3.6
0.72 Ideal F SI1 61.5 56.0 2870.0 5.72 5.79 3.54
0.82 Ideal H VS2 59.5 57.0 2870.0 6.12 6.09 3.63
0.73 Premium E VS1 61.3 59.0 2870.0 5.81 5.78 3.55
1.04 Premium I I1 61.6 61.0 2870.0 6.47 6.45 3.98
0.73 Very Good E SI1 61.3 58.0 2871.0 5.76 5.83 3.55
0.73 Good E SI1 63.6 57.0 2871.0 5.7 5.72 3.63
0.9 Premium J SI1 62.8 59.0 2871.0 6.13 6.03 3.82
0.75 Ideal I SI1 61.8 55.0 2871.0 5.83 5.85 3.61
0.79 Ideal G SI1 62.6 55.0 2871.0 5.91 5.95 3.71
0.7 Good D SI1 62.5 56.7 2872.0 5.59 5.62 3.51
0.75 Very Good D SI1 60.7 55.0 2872.0 5.87 5.92 3.58
1.02 Ideal I I1 61.7 56.0 2872.0 6.44 6.49 3.99
0.7 Very Good G SI2 59.0 62.0 2872.0 5.79 5.81 3.42
0.7 Ideal D SI1 61.8 56.0 2872.0 5.63 5.73 3.51
0.7 Good E SI1 61.4 64.0 2872.0 5.66 5.71 3.49
0.7 Ideal D SI1 61.4 54.0 2872.0 5.71 5.75 3.52
0.7 Ideal D SI1 60.7 56.0 2872.0 5.72 5.75 3.48
0.7 Very Good D SI1 60.2 60.0 2872.0 5.75 5.82 3.48
0.72 Very Good E VS2 58.3 57.0 2872.0 5.89 5.94 3.45
0.74 Ideal E SI1 62.3 58.0 2872.0 5.74 5.78 3.59
0.84 Good G SI1 65.1 55.0 2872.0 5.88 5.97 3.86
0.76 Very Good F VS2 62.0 58.0 2873.0 5.8 5.86 3.62
0.77 Very Good E SI1 63.2 58.0 2873.0 5.8 5.84 3.68
0.76 Ideal E SI2 62.8 56.0 2873.0 5.78 5.82 3.64
1.0 Ideal I SI2 61.7 56.0 2873.0 6.45 6.41 3.97
1.0 Fair H SI1 65.5 62.0 2873.0 6.14 6.07 4.0
0.9 Fair I SI1 65.7 58.0 2873.0 6.03 6.0 3.95
0.9 Premium J SI1 61.8 58.0 2873.0 6.16 6.13 3.8
0.9 Good J SI1 64.0 61.0 2873.0 6.0 5.96 3.83
0.9 Fair I SI1 65.3 61.0 2873.0 5.98 5.94 3.89
0.9 Fair I SI1 65.8 56.0 2873.0 6.01 5.96 3.94
0.9 Premium J SI1 60.9 61.0 2873.0 6.26 6.22 3.8
0.78 Premium F VS2 62.6 58.0 2874.0 5.91 5.82 3.67
0.71 Premium D VS2 61.2 59.0 2874.0 5.69 5.74 3.5
0.7 Premium F VS1 59.0 59.0 2874.0 5.79 5.77 3.41
0.7 Premium F VS1 60.8 62.0 2874.0 5.71 5.67 3.46
0.7 Premium G VVS2 61.8 58.0 2874.0 5.67 5.63 3.49
0.7 Ideal F VS1 61.0 55.0 2874.0 5.77 5.73 3.51
0.7 Ideal F VS1 61.6 55.0 2874.0 5.75 5.71 3.53
0.7 Ideal F VS1 62.4 56.0 2874.0 5.69 5.65 3.54
0.7 Premium G VVS2 62.9 59.0 2874.0 5.68 5.61 3.55
1.0 Fair H SI2 67.7 60.0 2875.0 6.11 5.98 4.09
0.77 Ideal H SI1 62.4 56.0 2875.0 5.84 5.9 3.66
1.0 Fair J VS1 65.5 55.0 2875.0 6.3 6.25 4.11
1.0 Fair I SI1 66.3 61.0 2875.0 6.08 6.03 4.01
1.0 Fair H SI2 69.5 55.0 2875.0 6.17 6.1 4.26
0.73 Premium E VS1 62.6 60.0 2876.0 5.68 5.75 3.58
0.79 Premium E VS2 60.6 53.0 2876.0 6.04 5.98 3.64
0.72 Very Good H VS1 62.2 54.0 2877.0 5.74 5.76 3.57
0.71 Ideal E VS1 62.4 56.0 2877.0 5.75 5.7 3.57
0.74 Ideal G VS2 62.3 55.0 2877.0 5.8 5.83 3.62
0.7 Good H VVS1 62.7 56.0 2877.0 5.6 5.66 3.53
0.7 Good F VS1 59.1 62.0 2877.0 5.82 5.86 3.44
0.79 Very Good F SI1 62.8 59.0 2878.0 5.86 5.89 3.69
0.79 Very Good F SI1 62.7 60.0 2878.0 5.82 5.89 3.67
0.79 Very Good D SI2 59.7 58.0 2878.0 6.0 6.07 3.6
0.71 Ideal I VS2 61.5 55.0 2878.0 5.76 5.78 3.55
0.79 Ideal F SI1 62.8 56.0 2878.0 5.88 5.9 3.7
0.73 Very Good F SI1 61.4 56.0 2879.0 5.81 5.86 3.58
0.63 Premium E IF 60.3 62.0 2879.0 5.55 5.53 3.34
0.7 Premium F VS1 60.4 60.0 2879.0 5.73 5.7 3.45
0.71 Premium F VS1 62.7 58.0 2879.0 5.71 5.67 3.57
0.84 Ideal G SI2 61.0 56.0 2879.0 6.13 6.1 3.73
0.84 Ideal G SI2 62.3 55.0 2879.0 6.08 6.03 3.77
1.02 Ideal J SI2 60.3 54.0 2879.0 6.53 6.5 3.93
0.72 Fair F VS1 56.9 69.0 2879.0 5.93 5.77 3.33
0.72 Ideal F VS1 62.0 56.0 2879.0 5.76 5.73 3.56
0.92 Very Good J SI2 58.7 61.0 2880.0 6.34 6.43 3.75
0.74 Very Good D SI1 63.9 57.0 2880.0 5.72 5.74 3.66
0.7 Ideal H VVS1 62.0 55.0 2881.0 5.74 5.71 3.55
0.71 Very Good E VS2 60.0 59.0 2881.0 5.84 5.83 3.5
1.05 Premium H I1 62.0 59.0 2881.0 6.5 6.47 4.02
0.7 Very Good H IF 62.8 56.0 2882.0 5.62 5.65 3.54
0.54 Ideal F VVS1 61.8 56.0 2882.0 5.23 5.26 3.24
0.73 Premium F VS2 59.9 58.0 2882.0 5.87 5.84 3.51
0.88 Fair F SI1 56.6 65.0 2882.0 6.39 6.32 3.6
0.73 Premium F VS2 58.7 57.0 2882.0 5.97 5.92 3.49
0.72 Ideal D SI1 61.8 56.0 2883.0 5.75 5.81 3.57
0.9 Good H SI2 62.7 64.0 2883.0 6.09 6.0 3.79
0.9 Fair H SI2 65.0 61.0 2883.0 6.01 5.96 3.89
1.03 Fair I SI2 65.3 55.0 2884.0 6.32 6.27 4.11
0.84 Very Good F SI1 63.8 57.0 2885.0 5.95 6.0 3.81
1.01 Premium I SI1 62.7 60.0 2885.0 6.36 6.27 3.96
0.77 Ideal D SI2 61.5 55.0 2885.0 5.9 5.93 3.64
0.8 Fair E SI1 56.3 63.0 2885.0 6.22 6.14 3.48
0.9 Fair D SI2 66.9 57.0 2885.0 6.02 5.9 3.99
0.73 Ideal E SI1 61.4 56.0 2886.0 5.79 5.81 3.56
0.72 Ideal E SI1 62.7 55.0 2886.0 5.64 5.69 3.55
0.71 Very Good D SI1 62.4 54.0 2887.0 5.71 5.79 3.59
0.7 Premium E VS1 62.6 59.0 2887.0 5.66 5.69 3.55
0.79 Ideal I VS1 61.7 59.0 2888.0 5.93 5.96 3.67
0.72 Very Good G VVS2 62.5 58.0 2889.0 5.68 5.72 3.56
0.7 Very Good E VS2 63.5 54.0 2889.0 5.62 5.66 3.58
0.7 Very Good F VS1 62.2 58.0 2889.0 5.64 5.75 3.54
0.9 Good H SI2 63.5 58.0 2889.0 6.09 6.14 3.88
0.71 Very Good F VS1 62.8 56.0 2889.0 5.69 5.72 3.58
0.5 Ideal E VVS2 62.2 54.0 2889.0 5.08 5.12 3.17
0.5 Ideal E VVS2 62.2 54.0 2889.0 5.09 5.11 3.17
0.74 Ideal F SI1 61.2 56.0 2889.0 5.83 5.87 3.58
0.77 Premium F VS2 61.8 56.0 2889.0 5.94 5.9 3.66
0.77 Premium E SI1 59.8 61.0 2889.0 5.99 5.95 3.57
0.8 Ideal F SI1 61.5 54.0 2890.0 6.07 6.0 3.71
0.8 Ideal F SI1 62.4 57.0 2890.0 5.9 5.87 3.67
0.8 Premium F SI1 61.5 60.0 2890.0 5.97 5.94 3.66
0.8 Good F SI1 63.8 59.0 2890.0 5.87 5.83 3.73
0.66 Ideal G VVS1 61.5 56.0 2890.0 5.61 5.58 3.44
0.71 Very Good E VS2 61.2 58.0 2891.0 5.71 5.79 3.52
0.71 Ideal F VS2 61.2 56.0 2891.0 5.73 5.77 3.52
0.71 Ideal E VS2 61.6 56.0 2891.0 5.74 5.76 3.54
0.71 Ideal E VS2 62.7 56.0 2891.0 5.71 5.75 3.59
0.72 Ideal D SI1 61.1 56.0 2891.0 5.78 5.81 3.54
0.71 Good D VS2 62.3 61.0 2891.0 5.7 5.73 3.56
0.86 Ideal H SI2 61.8 55.0 2892.0 6.12 6.14 3.79
1.19 Fair H I1 65.1 59.0 2892.0 6.62 6.55 4.29
0.71 Very Good F VS1 62.6 55.0 2893.0 5.66 5.71 3.56
0.82 Very Good G SI2 62.5 56.0 2893.0 5.99 6.04 3.76
0.71 Ideal G VVS2 61.5 57.0 2893.0 5.73 5.75 3.53
0.75 Ideal F VS2 62.5 57.0 2893.0 5.78 5.83 3.63
0.7 Very Good H VVS1 59.2 60.0 2893.0 5.87 5.78 3.45
0.8 Ideal G SI2 62.5 55.0 2893.0 5.89 5.92 3.69
0.82 Good G SI2 59.9 62.0 2893.0 6.02 6.04 3.61
0.82 Very Good G SI1 63.4 55.0 2893.0 6.0 5.93 3.78
0.82 Premium G SI1 59.9 59.0 2893.0 6.09 6.06 3.64
0.81 Very Good E SI2 62.4 57.0 2894.0 5.91 5.99 3.71
0.81 Ideal G SI2 62.2 57.0 2894.0 5.96 6.0 3.72
0.76 Ideal F SI1 61.4 56.0 2894.0 5.88 5.92 3.62
0.71 Very Good G VS2 60.9 56.0 2895.0 5.75 5.78 3.51
0.7 Very Good F VS1 61.8 59.0 2895.0 5.66 5.76 3.53
0.7 Ideal G VVS2 62.1 53.0 2895.0 5.71 5.75 3.56
0.74 Very Good G VS1 59.8 58.0 2896.0 5.85 5.89 3.51
0.77 Very Good G VS2 61.3 60.0 2896.0 5.81 5.91 3.59
0.77 Very Good G VS2 58.3 63.0 2896.0 6.0 6.05 3.51
0.53 Ideal F VVS1 61.6 56.0 2896.0 5.18 5.24 3.21
0.79 Ideal D SI1 61.5 56.0 2896.0 5.91 5.96 3.65
0.73 Ideal E SI2 61.5 55.0 2896.0 5.82 5.86 3.59
0.77 Ideal D SI2 62.1 56.0 2896.0 5.83 5.89 3.64
0.77 Premium E SI1 60.9 58.0 2896.0 5.94 5.88 3.6
1.01 Very Good I I1 63.1 57.0 2896.0 6.39 6.35 4.02
1.01 Ideal I I1 61.5 57.0 2896.0 6.46 6.45 3.97
0.6 Very Good D VVS2 60.6 57.0 2897.0 5.48 5.51 3.33
0.76 Premium E SI1 61.1 58.0 2897.0 5.91 5.85 3.59
0.54 Ideal D VVS2 61.4 52.0 2897.0 5.3 5.34 3.26
0.72 Ideal E SI1 62.5 55.0 2897.0 5.69 5.74 3.57
0.72 Good F VS1 59.4 61.0 2897.0 5.82 5.89 3.48
0.74 Premium D VS2 61.8 58.0 2897.0 5.81 5.77 3.58
1.12 Premium J SI2 60.6 59.0 2898.0 6.68 6.61 4.03

Now let's examine one of the continuous features as an example.

//Select: "Plot Options..." --> "Display type" --> "histogram plot" and choose to "Plot over all results" OTHERWISE you get the image from first 1000 rows only
display(diamondsDF.select("carat"))
carat
0.23
0.21
0.23
0.29
0.31
0.24
0.24
0.26
0.22
0.23
0.3
0.23
0.22
0.31
0.2
0.32
0.3
0.3
0.3
0.3
0.3
0.23
0.23
0.31
0.31
0.23
0.24
0.3
0.23
0.23
0.23
0.23
0.23
0.23
0.23
0.23
0.23
0.31
0.26
0.33
0.33
0.33
0.26
0.26
0.32
0.29
0.32
0.32
0.25
0.29
0.24
0.23
0.32
0.22
0.22
0.3
0.3
0.3
0.3
0.3
0.35
0.3
0.3
0.3
0.42
0.28
0.32
0.31
0.31
0.24
0.24
0.3
0.3
0.3
0.3
0.26
0.26
0.26
0.26
0.26
0.26
0.26
0.26
0.38
0.26
0.24
0.24
0.24
0.24
0.32
0.7
0.86
0.7
0.71
0.78
0.7
0.7
0.96
0.73
0.8
0.75
0.75
0.74
0.75
0.8
0.75
0.8
0.74
0.81
0.59
0.8
0.74
0.9
0.74
0.73
0.73
0.8
0.71
0.7
0.8
0.71
0.74
0.7
0.7
0.7
0.7
0.91
0.61
0.91
0.91
0.77
0.71
0.71
0.7
0.77
0.63
0.71
0.71
0.76
0.64
0.71
0.71
0.7
0.7
0.71
0.7
0.71
0.73
0.7
0.7
0.71
0.74
0.71
0.73
0.76
0.76
0.71
0.73
0.73
0.73
0.73
0.72
0.73
0.71
0.79
0.73
0.8
0.58
0.58
0.71
0.75
0.7
1.17
0.6
0.7
0.83
0.74
0.72
0.71
0.71
0.54
0.54
0.72
0.72
0.72
0.71
0.7
0.71
0.71
0.71
0.71
0.7
0.7
0.7
0.7
0.7
0.7
0.7
0.7
0.72
0.7
0.7
0.7
0.7
0.98
0.78
0.7
0.52
0.73
0.74
0.7
0.77
0.71
0.74
0.7
1.01
0.77
0.78
0.72
0.53
0.76
0.7
0.7
0.75
0.72
0.72
0.7
0.84
0.75
0.52
0.72
0.79
0.72
0.51
0.64
0.7
0.83
0.76
0.71
0.77
0.71
1.01
1.01
0.77
0.76
0.76
0.76
1.05
0.81
0.7
0.55
0.81
0.63
0.63
0.77
1.05
0.64
0.76
0.83
0.71
0.71
0.87
0.73
0.71
0.71
0.71
0.7
0.7
0.76
0.7
0.79
0.7
0.7
0.76
0.73
0.79
0.71
0.81
0.81
0.72
0.72
0.72
0.81
0.72
1.0
0.73
0.81
0.81
0.71
0.71
0.71
0.57
0.51
0.72
0.74
0.74
0.7
0.8
1.01
0.8
0.77
0.83
0.82
0.78
0.6
0.9
0.7
0.9
0.83
0.83
0.83
0.74
0.79
0.61
0.76
0.96
0.73
0.73
0.75
0.71
0.71
0.71
0.71
1.04
1.0
0.87
0.53
0.72
0.72
0.7
0.74
0.71
0.73
0.7
0.71
0.71
0.71
0.77
0.71
0.78
0.71
0.91
0.71
0.71
0.8
0.7
0.72
0.72
0.82
0.7
0.72
0.72
0.9
0.74
0.74
0.73
0.57
0.73
0.72
0.74
0.82
0.81
0.75
0.7
0.71
0.71
0.93
0.8
0.7
1.0
0.75
0.58
0.73
0.81
0.81
0.71
1.2
0.7
0.7
0.74
0.7
0.8
0.75
0.83
1.0
0.99
0.7
0.7
0.7
0.7
0.32
0.32
0.32
0.32
0.32
0.32
0.3
0.3
0.3
0.3
0.3
0.3
0.3
0.32
0.33
0.29
0.29
0.31
0.34
0.34
0.34
0.34
0.3
0.29
0.35
0.43
0.32
0.36
0.3
0.26
0.7
0.7
0.71
0.99
0.73
0.51
0.91
0.84
0.91
0.76
0.76
0.75
0.55
0.76
0.74
0.7
0.7
0.7
0.7
0.9
0.95
0.89
0.72
0.96
1.02
0.78
0.61
0.71
0.78
0.87
0.83
0.71
0.71
0.71
0.71
0.63
0.71
0.71
0.71
0.71
0.9
0.71
0.7
0.7
0.7
1.0
0.86
0.8
0.7
0.7
0.7
0.7
1.0
0.72
0.72
0.7
0.86
0.71
0.75
0.73
0.53
0.73
0.73
0.73
0.73
0.73
0.73
0.7
0.72
0.72
0.72
0.7
0.6
0.74
0.73
0.71
0.71
0.7
0.7
0.9
0.71
0.7
0.7
0.7
0.7
0.7
0.7
0.7
0.7
0.7
0.79
0.9
0.71
0.61
0.9
0.71
0.71
0.77
0.74
0.82
0.82
0.71
0.83
0.73
0.83
1.17
0.91
0.73
0.7
0.9
0.7
0.7
0.7
0.9
0.78
0.96
0.7
0.72
0.79
0.7
0.7
0.7
1.01
0.72
0.8
0.59
0.72
0.75
0.8
0.71
0.77
0.97
0.53
0.53
0.8
0.9
0.76
0.72
0.75
0.72
0.79
0.72
0.91
0.71
0.81
0.82
0.71
0.9
0.8
0.56
0.7
0.7
0.61
0.85
0.7
0.8
0.8
0.51
0.53
0.78
0.9
0.9
0.77
0.73
0.63
0.7
0.72
0.72
0.75
0.82
0.71
0.7
0.7
0.71
0.76
0.82
0.72
0.7
0.7
0.7
0.7
0.7
0.7
0.7
0.7
0.74
0.71
0.7
0.71
0.71
0.71
0.71
0.7
0.73
0.7
0.7
0.71
0.71
0.79
0.71
0.77
0.75
0.7
0.71
0.92
0.83
0.7
0.73
0.71
0.73
0.82
0.82
0.82
0.52
1.0
0.95
0.73
0.73
0.73
0.8
0.7
0.7
0.7
0.71
0.81
0.71
0.73
0.73
0.72
0.81
0.71
0.73
0.7
1.01
1.01
0.79
0.7
0.7
0.8
1.27
0.79
0.72
0.73
1.01
1.01
0.73
0.7
0.7
0.77
0.77
0.77
0.84
0.72
0.76
0.7
0.54
0.75
0.79
0.74
0.7
0.7
0.75
1.2
0.8
0.66
0.87
0.86
0.74
0.58
0.78
0.74
0.73
0.91
0.71
0.71
0.79
0.79
0.71
0.82
0.78
0.7
1.12
0.73
0.91
0.91
0.91
0.91
0.7
0.68
0.73
1.03
0.74
0.98
1.02
1.0
1.02
0.6
0.8
0.97
1.0
0.26
0.26
0.36
0.34
0.34
0.34
0.34
0.34
0.34
0.32
0.31
0.31
0.31
0.31
0.31
0.31
0.31
0.31
0.31
0.33
0.33
0.33
0.33
0.33
0.33
0.33
0.33
0.33
0.33
0.33
1.0
0.77
0.77
0.7
0.9
0.72
0.9
0.72
0.7
0.81
0.81
0.71
0.7
0.71
0.71
0.92
0.76
0.73
0.71
0.7
0.9
0.71
0.7
0.7
0.77
0.71
0.7
0.75
0.83
0.71
0.9
0.6
0.71
0.53
0.71
0.62
0.62
0.9
0.62
0.82
0.66
0.7
0.8
0.8
0.79
0.71
0.7
0.7
0.79
0.7
1.22
1.01
0.73
0.91
0.71
0.83
0.84
0.71
0.71
0.71
0.71
0.71
0.71
0.91
0.9
0.71
0.71
0.72
0.72
0.71
0.81
0.83
0.73
0.56
0.56
0.71
0.7
0.96
0.71
0.7
0.71
0.8
0.95
0.82
0.52
0.82
0.82
0.82
0.8
0.96
0.72
0.62
0.79
0.75
1.08
0.72
0.62
0.73
0.72
0.52
0.83
0.64
0.8
0.74
0.72
0.82
0.73
1.04
0.73
0.73
0.9
0.75
0.79
0.7
0.75
1.02
0.7
0.7
0.7
0.7
0.7
0.7
0.72
0.74
0.84
0.76
0.77
0.76
1.0
1.0
0.9
0.9
0.9
0.9
0.9
0.9
0.78
0.71
0.7
0.7
0.7
0.7
0.7
0.7
0.7
1.0
0.77
1.0
1.0
1.0
0.73
0.79
0.72
0.71
0.74
0.7
0.7
0.79
0.79
0.79
0.71
0.79
0.73
0.63
0.7
0.71
0.84
0.84
1.02
0.72
0.72
0.92
0.74
0.7
0.71
1.05
0.7
0.54
0.73
0.88
0.73
0.72
0.9
0.9
1.03
0.84
1.01
0.77
0.8
0.9
0.73
0.72
0.71
0.7
0.79
0.72
0.7
0.7
0.9
0.71
0.5
0.5
0.74
0.77
0.77
0.8
0.8
0.8
0.8
0.66
0.71
0.71
0.71
0.71
0.72
0.71
0.86
1.19
0.71
0.82
0.71
0.75
0.7
0.8
0.82
0.82
0.82
0.81
0.81
0.76
0.71
0.7
0.7
0.74
0.77
0.77
0.53
0.79
0.73
0.77
0.77
1.01
1.01
0.6
0.76
0.54
0.72
0.72
0.74
1.12

The above histogram of the diamonds' carat ratings shows that carats have a skewed distribution: Many diamonds are small, but there are a number of diamonds in the dataset which are much larger.

  • Extremely skewed distributions can cause problems for some algorithms (e.g., Linear Regression).
  • However, Decision Trees handle skewed distributions very naturally.

Note: When you call display to create a histogram like that above, it will plot using a subsample from the dataset (for efficiency), but you can plot using the full dataset by selecting "Plot over all results". For our dataset, the two plots can actually look very different due to the long-tailed distribution.

We will not examine the label distribution for now. It can be helpful to examine the label distribution, but it is best to do so only on the training set, not on the test set which we will hold out for evaluation. These will be seen in the sequel

You Try! Of course knock youself out visually exploring the dataset more...

display(diamondsDF.select("cut","carat"))
cut carat
Ideal 0.23
Premium 0.21
Good 0.23
Premium 0.29
Good 0.31
Very Good 0.24
Very Good 0.24
Very Good 0.26
Fair 0.22
Very Good 0.23
Good 0.3
Ideal 0.23
Premium 0.22
Ideal 0.31
Premium 0.2
Premium 0.32
Ideal 0.3
Good 0.3
Good 0.3
Very Good 0.3
Good 0.3
Very Good 0.23
Very Good 0.23
Very Good 0.31
Very Good 0.31
Very Good 0.23
Premium 0.24
Very Good 0.3
Very Good 0.23
Very Good 0.23
Very Good 0.23
Very Good 0.23
Very Good 0.23
Very Good 0.23
Very Good 0.23
Good 0.23
Good 0.23
Good 0.31
Very Good 0.26
Ideal 0.33
Ideal 0.33
Ideal 0.33
Good 0.26
Good 0.26
Good 0.32
Premium 0.29
Very Good 0.32
Good 0.32
Very Good 0.25
Very Good 0.29
Very Good 0.24
Ideal 0.23
Ideal 0.32
Premium 0.22
Premium 0.22
Ideal 0.3
Premium 0.3
Very Good 0.3
Very Good 0.3
Good 0.3
Ideal 0.35
Premium 0.3
Ideal 0.3
Ideal 0.3
Premium 0.42
Ideal 0.28
Ideal 0.32
Very Good 0.31
Premium 0.31
Premium 0.24
Very Good 0.24
Very Good 0.3
Premium 0.3
Premium 0.3
Good 0.3
Very Good 0.26
Very Good 0.26
Very Good 0.26
Very Good 0.26
Very Good 0.26
Very Good 0.26
Very Good 0.26
Ideal 0.26
Ideal 0.38
Good 0.26
Premium 0.24
Premium 0.24
Premium 0.24
Premium 0.24
Premium 0.32
Ideal 0.7
Fair 0.86
Ideal 0.7
Very Good 0.71
Very Good 0.78
Good 0.7
Good 0.7
Fair 0.96
Very Good 0.73
Premium 0.8
Very Good 0.75
Premium 0.75
Ideal 0.74
Premium 0.75
Ideal 0.8
Ideal 0.75
Premium 0.8
Ideal 0.74
Ideal 0.81
Ideal 0.59
Ideal 0.8
Ideal 0.74
Premium 0.9
Very Good 0.74
Ideal 0.73
Ideal 0.73
Premium 0.8
Ideal 0.71
Ideal 0.7
Ideal 0.8
Ideal 0.71
Ideal 0.74
Very Good 0.7
Fair 0.7
Fair 0.7
Premium 0.7
Premium 0.91
Very Good 0.61
Fair 0.91
Fair 0.91
Ideal 0.77
Very Good 0.71
Ideal 0.71
Very Good 0.7
Very Good 0.77
Premium 0.63
Very Good 0.71
Premium 0.71
Ideal 0.76
Ideal 0.64
Premium 0.71
Premium 0.71
Very Good 0.7
Very Good 0.7
Ideal 0.71
Good 0.7
Very Good 0.71
Very Good 0.73
Very Good 0.7
Ideal 0.7
Premium 0.71
Ideal 0.74
Premium 0.71
Premium 0.73
Very Good 0.76
Ideal 0.76
Ideal 0.71
Premium 0.73
Premium 0.73
Ideal 0.73
Premium 0.73
Very Good 0.72
Very Good 0.73
Ideal 0.71
Ideal 0.79
Very Good 0.73
Very Good 0.8
Ideal 0.58
Ideal 0.58
Good 0.71
Ideal 0.75
Premium 0.7
Very Good 1.17
Ideal 0.6
Ideal 0.7
Good 0.83
Very Good 0.74
Very Good 0.72
Premium 0.71
Ideal 0.71
Ideal 0.54
Ideal 0.54
Ideal 0.72
Ideal 0.72
Good 0.72
Ideal 0.71
Very Good 0.7
Premium 0.71
Very Good 0.71
Good 0.71
Good 0.71
Ideal 0.7
Premium 0.7
Premium 0.7
Premium 0.7
Premium 0.7
Premium 0.7
Premium 0.7
Ideal 0.7
Premium 0.72
Very Good 0.7
Premium 0.7
Premium 0.7
Good 0.7
Fair 0.98
Premium 0.78
Very Good 0.7
Ideal 0.52
Very Good 0.73
Ideal 0.74
Very Good 0.7
Premium 0.77
Ideal 0.71
Ideal 0.74
Ideal 0.7
Premium 1.01
Ideal 0.77
Ideal 0.78
Very Good 0.72
Very Good 0.53
Ideal 0.76
Good 0.7
Premium 0.7
Very Good 0.75
Ideal 0.72
Premium 0.72
Premium 0.7
Fair 0.84
Premium 0.75
Ideal 0.52
Very Good 0.72
Very Good 0.79
Very Good 0.72
Ideal 0.51
Ideal 0.64
Very Good 0.7
Very Good 0.83
Ideal 0.76
Good 0.71
Good 0.77
Ideal 0.71
Fair 1.01
Premium 1.01
Good 0.77
Good 0.76
Premium 0.76
Premium 0.76
Very Good 1.05
Ideal 0.81
Ideal 0.7
Ideal 0.55
Good 0.81
Premium 0.63
Premium 0.63
Premium 0.77
Fair 1.05
Ideal 0.64
Premium 0.76
Ideal 0.83
Premium 0.71
Premium 0.71
Very Good 0.87
Ideal 0.73
Premium 0.71
Premium 0.71
Ideal 0.71
Premium 0.7
Premium 0.7
Premium 0.76
Ideal 0.7
Very Good 0.79
Very Good 0.7
Good 0.7
Ideal 0.76
Ideal 0.73
Very Good 0.79
Very Good 0.71
Premium 0.81
Ideal 0.81
Good 0.72
Premium 0.72
Premium 0.72
Premium 0.81
Premium 0.72
Premium 1.0
Good 0.73
Very Good 0.81
Very Good 0.81
Premium 0.71
Premium 0.71
Premium 0.71
Ideal 0.57
Ideal 0.51
Ideal 0.72
Ideal 0.74
Ideal 0.74
Fair 0.7
Premium 0.8
Fair 1.01
Very Good 0.8
Ideal 0.77
Very Good 0.83
Ideal 0.82
Ideal 0.78
Very Good 0.6
Good 0.9
Premium 0.7
Very Good 0.9
Ideal 0.83
Ideal 0.83
Very Good 0.83
Premium 0.74
Ideal 0.79
Ideal 0.61
Fair 0.76
Ideal 0.96
Ideal 0.73
Premium 0.73
Ideal 0.75
Premium 0.71
Good 0.71
Good 0.71
Premium 0.71
Premium 1.04
Premium 1.0
Very Good 0.87
Ideal 0.53
Premium 0.72
Premium 0.72
Very Good 0.7
Very Good 0.74
Ideal 0.71
Ideal 0.73
Good 0.7
Premium 0.71
Premium 0.71
Premium 0.71
Premium 0.77
Premium 0.71
Premium 0.78
Very Good 0.71
Ideal 0.91
Very Good 0.71
Very Good 0.71
Very Good 0.8
Very Good 0.7
Ideal 0.72
Very Good 0.72
Ideal 0.82
Ideal 0.7
Ideal 0.72
Ideal 0.72
Fair 0.9
Premium 0.74
Premium 0.74
Ideal 0.73
Fair 0.57
Premium 0.73
Ideal 0.72
Fair 0.74
Good 0.82
Very Good 0.81
Very Good 0.75
Ideal 0.7
Very Good 0.71
Very Good 0.71
Premium 0.93
Very Good 0.8
Very Good 0.7
Fair 1.0
Very Good 0.75
Ideal 0.58
Very Good 0.73
Very Good 0.81
Premium 0.81
Premium 0.71
Fair 1.2
Very Good 0.7
Very Good 0.7
Ideal 0.74
Good 0.7
Good 0.8
Very Good 0.75
Very Good 0.83
Fair 1.0
Fair 0.99
Very Good 0.7
Very Good 0.7
Good 0.7
Very Good 0.7
Premium 0.32
Premium 0.32
Ideal 0.32
Premium 0.32
Very Good 0.32
Ideal 0.32
Premium 0.3
Premium 0.3
Premium 0.3
Very Good 0.3
Good 0.3
Ideal 0.3
Very Good 0.3
Good 0.32
Ideal 0.33
Very Good 0.29
Very Good 0.29
Very Good 0.31
Ideal 0.34
Ideal 0.34
Ideal 0.34
Ideal 0.34
Ideal 0.3
Ideal 0.29
Ideal 0.35
Very Good 0.43
Very Good 0.32
Ideal 0.36
Ideal 0.3
Ideal 0.26
Very Good 0.7
Very Good 0.7
Ideal 0.71
Fair 0.99
Premium 0.73
Ideal 0.51
Premium 0.91
Very Good 0.84
Good 0.91
Premium 0.76
Ideal 0.76
Premium 0.75
Very Good 0.55
Very Good 0.76
Premium 0.74
Ideal 0.7
Ideal 0.7
Ideal 0.7
Ideal 0.7
Fair 0.9
Fair 0.95
Premium 0.89
Premium 0.72
Fair 0.96
Premium 1.02
Very Good 0.78
Ideal 0.61
Good 0.71
Premium 0.78
Ideal 0.87
Ideal 0.83
Premium 0.71
Ideal 0.71
Ideal 0.71
Premium 0.71
Ideal 0.63
Premium 0.71
Premium 0.71
Ideal 0.71
Premium 0.71
Ideal 0.9
Good 0.71
Premium 0.7
Premium 0.7
Premium 0.7
Premium 1.0
Premium 0.86
Ideal 0.8
Ideal 0.7
Premium 0.7
Premium 0.7
Premium 0.7
Fair 1.0
Very Good 0.72
Ideal 0.72
Good 0.7
Good 0.86
Ideal 0.71
Ideal 0.75
Premium 0.73
Ideal 0.53
Premium 0.73
Good 0.73
Premium 0.73
Premium 0.73
Premium 0.73
Very Good 0.73
Premium 0.7
Premium 0.72
Premium 0.72
Premium 0.72
Premium 0.7
Ideal 0.6
Ideal 0.74
Ideal 0.73
Premium 0.71
Premium 0.71
Premium 0.7
Ideal 0.7
Good 0.9
Ideal 0.71
Premium 0.7
Premium 0.7
Ideal 0.7
Good 0.7
Ideal 0.7
Ideal 0.7
Premium 0.7
Premium 0.7
Ideal 0.7
Very Good 0.79
Good 0.9
Premium 0.71
Ideal 0.61
Fair 0.9
Ideal 0.71
Premium 0.71
Ideal 0.77
Good 0.74
Ideal 0.82
Premium 0.82
Premium 0.71
Premium 0.83
Very Good 0.73
Premium 0.83
Premium 1.17
Fair 0.91
Premium 0.73
Good 0.7
Premium 0.9
Premium 0.7
Very Good 0.7
Premium 0.7
Very Good 0.9
Premium 0.78
Ideal 0.96
Very Good 0.7
Good 0.72
Premium 0.79
Ideal 0.7
Ideal 0.7
Ideal 0.7
Premium 1.01
Premium 0.72
Good 0.8
Ideal 0.59
Ideal 0.72
Premium 0.75
Premium 0.8
Very Good 0.71
Very Good 0.77
Ideal 0.97
Ideal 0.53
Ideal 0.53
Ideal 0.8
Premium 0.9
Very Good 0.76
Ideal 0.72
Ideal 0.75
Premium 0.72
Ideal 0.79
Very Good 0.72
Very Good 0.91
Premium 0.71
Premium 0.81
Ideal 0.82
Premium 0.71
Good 0.9
Very Good 0.8
Very Good 0.56
Very Good 0.7
Ideal 0.7
Ideal 0.61
Ideal 0.85
Ideal 0.7
Ideal 0.8
Ideal 0.8
Very Good 0.51
Ideal 0.53
Ideal 0.78
Very Good 0.9
Fair 0.9
Ideal 0.77
Ideal 0.73
Ideal 0.63
Ideal 0.7
Ideal 0.72
Ideal 0.72
Premium 0.75
Very Good 0.82
Good 0.71
Premium 0.7
Ideal 0.7
Ideal 0.71
Very Good 0.76
Fair 0.82
Premium 0.72
Premium 0.7
Premium 0.7
Premium 0.7
Premium 0.7
Very Good 0.7
Premium 0.7
Premium 0.7
Premium 0.7
Very Good 0.74
Ideal 0.71
Ideal 0.7
Ideal 0.71
Premium 0.71
Premium 0.71
Premium 0.71
Ideal 0.7
Ideal 0.73
Ideal 0.7
Ideal 0.7
Premium 0.71
Premium 0.71
Premium 0.79
Premium 0.71
Very Good 0.77
Very Good 0.75
Ideal 0.7
Premium 0.71
Ideal 0.92
Premium 0.83
Premium 0.7
Premium 0.73
Very Good 0.71
Very Good 0.73
Ideal 0.82
Ideal 0.82
Very Good 0.82
Ideal 0.52
Premium 1.0
Fair 0.95
Ideal 0.73
Premium 0.73
Premium 0.73
Ideal 0.8
Premium 0.7
Very Good 0.7
Very Good 0.7
Very Good 0.71
Very Good 0.81
Very Good 0.71
Ideal 0.73
Very Good 0.73
Ideal 0.72
Ideal 0.81
Ideal 0.71
Very Good 0.73
Very Good 0.7
Ideal 1.01
Good 1.01
Ideal 0.79
Very Good 0.7
Very Good 0.7
Good 0.8
Premium 1.27
Ideal 0.79
Very Good 0.72
Ideal 0.73
Fair 1.01
Good 1.01
Ideal 0.73
Ideal 0.7
Good 0.7
Premium 0.77
Premium 0.77
Premium 0.77
Very Good 0.84
Ideal 0.72
Premium 0.76
Very Good 0.7
Ideal 0.54
Fair 0.75
Good 0.79
Very Good 0.74
Very Good 0.7
Ideal 0.7
Ideal 0.75
Very Good 1.2
Very Good 0.8
Ideal 0.66
Very Good 0.87
Premium 0.86
Ideal 0.74
Very Good 0.58
Ideal 0.78
Ideal 0.74
Ideal 0.73
Very Good 0.91
Premium 0.71
Good 0.71
Ideal 0.79
Premium 0.79
Premium 0.71
Premium 0.82
Very Good 0.78
Very Good 0.7
Premium 1.12
Premium 0.73
Fair 0.91
Fair 0.91
Good 0.91
Fair 0.91
Premium 0.7
Premium 0.68
Very Good 0.73
Good 1.03
Premium 0.74
Fair 0.98
Fair 1.02
Fair 1.0
Ideal 1.02
Ideal 0.6
Ideal 0.8
Ideal 0.97
Fair 1.0
Ideal 0.26
Ideal 0.26
Ideal 0.36
Good 0.34
Good 0.34
Good 0.34
Good 0.34
Very Good 0.34
Ideal 0.34
Good 0.32
Ideal 0.31
Ideal 0.31
Ideal 0.31
Ideal 0.31
Ideal 0.31
Ideal 0.31
Ideal 0.31
Ideal 0.31
Ideal 0.31
Premium 0.33
Premium 0.33
Premium 0.33
Ideal 0.33
Premium 0.33
Premium 0.33
Premium 0.33
Ideal 0.33
Ideal 0.33
Good 0.33
Premium 0.33
Fair 1.0
Premium 0.77
Premium 0.77
Good 0.7
Very Good 0.9
Ideal 0.72
Premium 0.9
Premium 0.72
Ideal 0.7
Very Good 0.81
Very Good 0.81
Premium 0.71
Premium 0.7
Premium 0.71
Very Good 0.71
Premium 0.92
Ideal 0.76
Ideal 0.73
Premium 0.71
Good 0.7
Fair 0.9
Fair 0.71
Ideal 0.7
Premium 0.7
Premium 0.77
Ideal 0.71
Premium 0.7
Fair 0.75
Premium 0.83
Very Good 0.71
Very Good 0.9
Ideal 0.6
Premium 0.71
Ideal 0.53
Premium 0.71
Ideal 0.62
Ideal 0.62
Premium 0.9
Fair 0.62
Premium 0.82
Premium 0.66
Very Good 0.7
Very Good 0.8
Very Good 0.8
Premium 0.79
Very Good 0.71
Ideal 0.7
Very Good 0.7
Premium 0.79
Premium 0.7
Premium 1.22
Fair 1.01
Premium 0.73
Good 0.91
Ideal 0.71
Premium 0.83
Premium 0.84
Premium 0.71
Premium 0.71
Premium 0.71
Ideal 0.71
Premium 0.71
Premium 0.71
Premium 0.91
Premium 0.9
Premium 0.71
Premium 0.71
Ideal 0.72
Premium 0.72
Ideal 0.71
Ideal 0.81
Very Good 0.83
Premium 0.73
Very Good 0.56
Very Good 0.56
Ideal 0.71
Ideal 0.7
Premium 0.96
Very Good 0.71
Ideal 0.7
Ideal 0.71
Premium 0.8
Premium 0.95
Ideal 0.82
Ideal 0.52
Ideal 0.82
Ideal 0.82
Premium 0.82
Ideal 0.8
Fair 0.96
Ideal 0.72
Ideal 0.62
Premium 0.79
Very Good 0.75
Premium 1.08
Ideal 0.72
Ideal 0.62
Ideal 0.73
Ideal 0.72
Premium 0.52
Ideal 0.83
Premium 0.64
Ideal 0.8
Ideal 0.74
Ideal 0.72
Ideal 0.82
Premium 0.73
Premium 1.04
Very Good 0.73
Good 0.73
Premium 0.9
Ideal 0.75
Ideal 0.79
Good 0.7
Very Good 0.75
Ideal 1.02
Very Good 0.7
Ideal 0.7
Good 0.7
Ideal 0.7
Ideal 0.7
Very Good 0.7
Very Good 0.72
Ideal 0.74
Good 0.84
Very Good 0.76
Very Good 0.77
Ideal 0.76
Ideal 1.0
Fair 1.0
Fair 0.9
Premium 0.9
Good 0.9
Fair 0.9
Fair 0.9
Premium 0.9
Premium 0.78
Premium 0.71
Premium 0.7
Premium 0.7
Premium 0.7
Ideal 0.7
Ideal 0.7
Ideal 0.7
Premium 0.7
Fair 1.0
Ideal 0.77
Fair 1.0
Fair 1.0
Fair 1.0
Premium 0.73
Premium 0.79
Very Good 0.72
Ideal 0.71
Ideal 0.74
Good 0.7
Good 0.7
Very Good 0.79
Very Good 0.79
Very Good 0.79
Ideal 0.71
Ideal 0.79
Very Good 0.73
Premium 0.63
Premium 0.7
Premium 0.71
Ideal 0.84
Ideal 0.84
Ideal 1.02
Fair 0.72
Ideal 0.72
Very Good 0.92
Very Good 0.74
Ideal 0.7
Very Good 0.71
Premium 1.05
Very Good 0.7
Ideal 0.54
Premium 0.73
Fair 0.88
Premium 0.73
Ideal 0.72
Good 0.9
Fair 0.9
Fair 1.03
Very Good 0.84
Premium 1.01
Ideal 0.77
Fair 0.8
Fair 0.9
Ideal 0.73
Ideal 0.72
Very Good 0.71
Premium 0.7
Ideal 0.79
Very Good 0.72
Very Good 0.7
Very Good 0.7
Good 0.9
Very Good 0.71
Ideal 0.5
Ideal 0.5
Ideal 0.74
Premium 0.77
Premium 0.77
Ideal 0.8
Ideal 0.8
Premium 0.8
Good 0.8
Ideal 0.66
Very Good 0.71
Ideal 0.71
Ideal 0.71
Ideal 0.71
Ideal 0.72
Good 0.71
Ideal 0.86
Fair 1.19
Very Good 0.71
Very Good 0.82
Ideal 0.71
Ideal 0.75
Very Good 0.7
Ideal 0.8
Good 0.82
Very Good 0.82
Premium 0.82
Very Good 0.81
Ideal 0.81
Ideal 0.76
Very Good 0.71
Very Good 0.7
Ideal 0.7
Very Good 0.74
Very Good 0.77
Very Good 0.77
Ideal 0.53
Ideal 0.79
Ideal 0.73
Ideal 0.77
Premium 0.77
Very Good 1.01
Ideal 1.01
Very Good 0.6
Premium 0.76
Ideal 0.54
Ideal 0.72
Good 0.72
Premium 0.74
Premium 1.12

Try scatter plot to see pairwise scatter plots of continuous features.

display(diamondsDF) //Ctrl+Enter 
carat cut color clarity depth table price x y z
0.23 Ideal E SI2 61.5 55.0 326.0 3.95 3.98 2.43
0.21 Premium E SI1 59.8 61.0 326.0 3.89 3.84 2.31
0.23 Good E VS1 56.9 65.0 327.0 4.05 4.07 2.31
0.29 Premium I VS2 62.4 58.0 334.0 4.2 4.23 2.63
0.31 Good J SI2 63.3 58.0 335.0 4.34 4.35 2.75
0.24 Very Good J VVS2 62.8 57.0 336.0 3.94 3.96 2.48
0.24 Very Good I VVS1 62.3 57.0 336.0 3.95 3.98 2.47
0.26 Very Good H SI1 61.9 55.0 337.0 4.07 4.11 2.53
0.22 Fair E VS2 65.1 61.0 337.0 3.87 3.78 2.49
0.23 Very Good H VS1 59.4 61.0 338.0 4.0 4.05 2.39
0.3 Good J SI1 64.0 55.0 339.0 4.25 4.28 2.73
0.23 Ideal J VS1 62.8 56.0 340.0 3.93 3.9 2.46
0.22 Premium F SI1 60.4 61.0 342.0 3.88 3.84 2.33
0.31 Ideal J SI2 62.2 54.0 344.0 4.35 4.37 2.71
0.2 Premium E SI2 60.2 62.0 345.0 3.79 3.75 2.27
0.32 Premium E I1 60.9 58.0 345.0 4.38 4.42 2.68
0.3 Ideal I SI2 62.0 54.0 348.0 4.31 4.34 2.68
0.3 Good J SI1 63.4 54.0 351.0 4.23 4.29 2.7
0.3 Good J SI1 63.8 56.0 351.0 4.23 4.26 2.71
0.3 Very Good J SI1 62.7 59.0 351.0 4.21 4.27 2.66
0.3 Good I SI2 63.3 56.0 351.0 4.26 4.3 2.71
0.23 Very Good E VS2 63.8 55.0 352.0 3.85 3.92 2.48
0.23 Very Good H VS1 61.0 57.0 353.0 3.94 3.96 2.41
0.31 Very Good J SI1 59.4 62.0 353.0 4.39 4.43 2.62
0.31 Very Good J SI1 58.1 62.0 353.0 4.44 4.47 2.59
0.23 Very Good G VVS2 60.4 58.0 354.0 3.97 4.01 2.41
0.24 Premium I VS1 62.5 57.0 355.0 3.97 3.94 2.47
0.3 Very Good J VS2 62.2 57.0 357.0 4.28 4.3 2.67
0.23 Very Good D VS2 60.5 61.0 357.0 3.96 3.97 2.4
0.23 Very Good F VS1 60.9 57.0 357.0 3.96 3.99 2.42
0.23 Very Good F VS1 60.0 57.0 402.0 4.0 4.03 2.41
0.23 Very Good F VS1 59.8 57.0 402.0 4.04 4.06 2.42
0.23 Very Good E VS1 60.7 59.0 402.0 3.97 4.01 2.42
0.23 Very Good E VS1 59.5 58.0 402.0 4.01 4.06 2.4
0.23 Very Good D VS1 61.9 58.0 402.0 3.92 3.96 2.44
0.23 Good F VS1 58.2 59.0 402.0 4.06 4.08 2.37
0.23 Good E VS1 64.1 59.0 402.0 3.83 3.85 2.46
0.31 Good H SI1 64.0 54.0 402.0 4.29 4.31 2.75
0.26 Very Good D VS2 60.8 59.0 403.0 4.13 4.16 2.52
0.33 Ideal I SI2 61.8 55.0 403.0 4.49 4.51 2.78
0.33 Ideal I SI2 61.2 56.0 403.0 4.49 4.5 2.75
0.33 Ideal J SI1 61.1 56.0 403.0 4.49 4.55 2.76
0.26 Good D VS2 65.2 56.0 403.0 3.99 4.02 2.61
0.26 Good D VS1 58.4 63.0 403.0 4.19 4.24 2.46
0.32 Good H SI2 63.1 56.0 403.0 4.34 4.37 2.75
0.29 Premium F SI1 62.4 58.0 403.0 4.24 4.26 2.65
0.32 Very Good H SI2 61.8 55.0 403.0 4.35 4.42 2.71
0.32 Good H SI2 63.8 56.0 403.0 4.36 4.38 2.79
0.25 Very Good E VS2 63.3 60.0 404.0 4.0 4.03 2.54
0.29 Very Good H SI2 60.7 60.0 404.0 4.33 4.37 2.64
0.24 Very Good F SI1 60.9 61.0 404.0 4.02 4.03 2.45
0.23 Ideal G VS1 61.9 54.0 404.0 3.93 3.95 2.44
0.32 Ideal I SI1 60.9 55.0 404.0 4.45 4.48 2.72
0.22 Premium E VS2 61.6 58.0 404.0 3.93 3.89 2.41
0.22 Premium D VS2 59.3 62.0 404.0 3.91 3.88 2.31
0.3 Ideal I SI2 61.0 59.0 405.0 4.3 4.33 2.63
0.3 Premium J SI2 59.3 61.0 405.0 4.43 4.38 2.61
0.3 Very Good I SI1 62.6 57.0 405.0 4.25 4.28 2.67
0.3 Very Good I SI1 63.0 57.0 405.0 4.28 4.32 2.71
0.3 Good I SI1 63.2 55.0 405.0 4.25 4.29 2.7
0.35 Ideal I VS1 60.9 57.0 552.0 4.54 4.59 2.78
0.3 Premium D SI1 62.6 59.0 552.0 4.23 4.27 2.66
0.3 Ideal D SI1 62.5 57.0 552.0 4.29 4.32 2.69
0.3 Ideal D SI1 62.1 56.0 552.0 4.3 4.33 2.68
0.42 Premium I SI2 61.5 59.0 552.0 4.78 4.84 2.96
0.28 Ideal G VVS2 61.4 56.0 553.0 4.19 4.22 2.58
0.32 Ideal I VVS1 62.0 55.3 553.0 4.39 4.42 2.73
0.31 Very Good G SI1 63.3 57.0 553.0 4.33 4.3 2.73
0.31 Premium G SI1 61.8 58.0 553.0 4.35 4.32 2.68
0.24 Premium E VVS1 60.7 58.0 553.0 4.01 4.03 2.44
0.24 Very Good D VVS1 61.5 60.0 553.0 3.97 4.0 2.45
0.3 Very Good H SI1 63.1 56.0 554.0 4.29 4.27 2.7
0.3 Premium H SI1 62.9 59.0 554.0 4.28 4.24 2.68
0.3 Premium H SI1 62.5 57.0 554.0 4.29 4.25 2.67
0.3 Good H SI1 63.7 57.0 554.0 4.28 4.26 2.72
0.26 Very Good F VVS2 59.2 60.0 554.0 4.19 4.22 2.49
0.26 Very Good E VVS2 59.9 58.0 554.0 4.15 4.23 2.51
0.26 Very Good D VVS2 62.4 54.0 554.0 4.08 4.13 2.56
0.26 Very Good D VVS2 62.8 60.0 554.0 4.01 4.05 2.53
0.26 Very Good E VVS1 62.6 59.0 554.0 4.06 4.09 2.55
0.26 Very Good E VVS1 63.4 59.0 554.0 4.0 4.04 2.55
0.26 Very Good D VVS1 62.1 60.0 554.0 4.03 4.12 2.53
0.26 Ideal E VVS2 62.9 58.0 554.0 4.02 4.06 2.54
0.38 Ideal I SI2 61.6 56.0 554.0 4.65 4.67 2.87
0.26 Good E VVS1 57.9 60.0 554.0 4.22 4.25 2.45
0.24 Premium G VVS1 62.3 59.0 554.0 3.95 3.92 2.45
0.24 Premium H VVS1 61.2 58.0 554.0 4.01 3.96 2.44
0.24 Premium H VVS1 60.8 59.0 554.0 4.02 4.0 2.44
0.24 Premium H VVS2 60.7 58.0 554.0 4.07 4.04 2.46
0.32 Premium I SI1 62.9 58.0 554.0 4.35 4.33 2.73
0.7 Ideal E SI1 62.5 57.0 2757.0 5.7 5.72 3.57
0.86 Fair E SI2 55.1 69.0 2757.0 6.45 6.33 3.52
0.7 Ideal G VS2 61.6 56.0 2757.0 5.7 5.67 3.5
0.71 Very Good E VS2 62.4 57.0 2759.0 5.68 5.73 3.56
0.78 Very Good G SI2 63.8 56.0 2759.0 5.81 5.85 3.72
0.7 Good E VS2 57.5 58.0 2759.0 5.85 5.9 3.38
0.7 Good F VS1 59.4 62.0 2759.0 5.71 5.76 3.4
0.96 Fair F SI2 66.3 62.0 2759.0 6.27 5.95 4.07
0.73 Very Good E SI1 61.6 59.0 2760.0 5.77 5.78 3.56
0.8 Premium H SI1 61.5 58.0 2760.0 5.97 5.93 3.66
0.75 Very Good D SI1 63.2 56.0 2760.0 5.8 5.75 3.65
0.75 Premium E SI1 59.9 54.0 2760.0 6.0 5.96 3.58
0.74 Ideal G SI1 61.6 55.0 2760.0 5.8 5.85 3.59
0.75 Premium G VS2 61.7 58.0 2760.0 5.85 5.79 3.59
0.8 Ideal I VS1 62.9 56.0 2760.0 5.94 5.87 3.72
0.75 Ideal G SI1 62.2 55.0 2760.0 5.87 5.8 3.63
0.8 Premium G SI1 63.0 59.0 2760.0 5.9 5.81 3.69
0.74 Ideal I VVS2 62.3 55.0 2761.0 5.77 5.81 3.61
0.81 Ideal F SI2 58.8 57.0 2761.0 6.14 6.11 3.6
0.59 Ideal E VVS2 62.0 55.0 2761.0 5.38 5.43 3.35
0.8 Ideal F SI2 61.4 57.0 2761.0 5.96 6.0 3.67
0.74 Ideal E SI2 62.2 56.0 2761.0 5.8 5.84 3.62
0.9 Premium I VS2 63.0 58.0 2761.0 6.16 6.12 3.87
0.74 Very Good G SI1 62.2 59.0 2762.0 5.73 5.82 3.59
0.73 Ideal F VS2 62.6 56.0 2762.0 5.77 5.74 3.6
0.73 Ideal F VS2 62.7 53.0 2762.0 5.8 5.75 3.62
0.8 Premium F SI2 61.7 58.0 2762.0 5.98 5.94 3.68
0.71 Ideal G VS2 62.4 54.0 2762.0 5.72 5.76 3.58
0.7 Ideal E VS2 60.7 58.0 2762.0 5.73 5.76 3.49
0.8 Ideal F SI2 59.9 59.0 2762.0 6.01 6.07 3.62
0.71 Ideal D SI2 62.3 56.0 2762.0 5.73 5.69 3.56
0.74 Ideal E SI1 62.3 54.0 2762.0 5.8 5.83 3.62
0.7 Very Good F VS2 61.7 63.0 2762.0 5.64 5.61 3.47
0.7 Fair F VS2 64.5 57.0 2762.0 5.57 5.53 3.58
0.7 Fair F VS2 65.3 55.0 2762.0 5.63 5.58 3.66
0.7 Premium F VS2 61.6 60.0 2762.0 5.65 5.59 3.46
0.91 Premium H SI1 61.4 56.0 2763.0 6.09 5.97 3.7
0.61 Very Good D VVS2 59.6 57.0 2763.0 5.56 5.58 3.32
0.91 Fair H SI2 64.4 57.0 2763.0 6.11 6.09 3.93
0.91 Fair H SI2 65.7 60.0 2763.0 6.03 5.99 3.95
0.77 Ideal H VS2 62.0 56.0 2763.0 5.89 5.86 3.64
0.71 Very Good D SI1 63.6 58.0 2764.0 5.64 5.68 3.6
0.71 Ideal D SI1 61.9 59.0 2764.0 5.69 5.72 3.53
0.7 Very Good E VS2 62.6 60.0 2765.0 5.62 5.65 3.53
0.77 Very Good H VS1 61.3 60.0 2765.0 5.88 5.9 3.61
0.63 Premium E VVS1 60.9 60.0 2765.0 5.52 5.55 3.37
0.71 Very Good F VS1 60.1 62.0 2765.0 5.74 5.77 3.46
0.71 Premium F VS1 61.8 59.0 2765.0 5.69 5.73 3.53
0.76 Ideal H SI1 61.2 57.0 2765.0 5.88 5.91 3.61
0.64 Ideal G VVS1 61.9 56.0 2766.0 5.53 5.56 3.43
0.71 Premium G VS2 60.9 57.0 2766.0 5.78 5.75 3.51
0.71 Premium G VS2 59.8 56.0 2766.0 5.89 5.81 3.5
0.7 Very Good D VS2 61.8 55.0 2767.0 5.68 5.72 3.52
0.7 Very Good F VS1 60.0 57.0 2767.0 5.8 5.87 3.5
0.71 Ideal D SI2 61.6 55.0 2767.0 5.74 5.76 3.54
0.7 Good H VVS2 62.1 64.0 2767.0 5.62 5.65 3.5
0.71 Very Good G VS1 63.3 59.0 2768.0 5.52 5.61 3.52
0.73 Very Good D SI1 60.2 56.0 2768.0 5.83 5.87 3.52
0.7 Very Good D SI1 61.1 58.0 2768.0 5.66 5.73 3.48
0.7 Ideal E SI1 60.9 57.0 2768.0 5.73 5.76 3.5
0.71 Premium D SI2 61.7 59.0 2768.0 5.71 5.67 3.51
0.74 Ideal I SI1 61.3 56.0 2769.0 5.82 5.86 3.57
0.71 Premium D VS2 62.5 60.0 2770.0 5.65 5.61 3.52
0.73 Premium G VS2 61.4 59.0 2770.0 5.83 5.76 3.56
0.76 Very Good F SI1 62.9 57.0 2770.0 5.79 5.81 3.65
0.76 Ideal D SI2 62.4 57.0 2770.0 5.78 5.83 3.62
0.71 Ideal F SI1 60.7 56.0 2770.0 5.77 5.8 3.51
0.73 Premium G VS2 60.7 58.0 2770.0 5.87 5.82 3.55
0.73 Premium G VS1 61.5 58.0 2770.0 5.79 5.75 3.55
0.73 Ideal D SI2 59.9 57.0 2770.0 5.92 5.89 3.54
0.73 Premium G VS2 59.2 59.0 2770.0 5.92 5.87 3.49
0.72 Very Good H VVS2 60.3 56.0 2771.0 5.81 5.83 3.51
0.73 Very Good F SI1 61.7 60.0 2771.0 5.79 5.82 3.58
0.71 Ideal G VS2 61.9 57.0 2771.0 5.73 5.77 3.56
0.79 Ideal F SI2 61.9 55.0 2771.0 5.97 5.92 3.68
0.73 Very Good H VVS1 60.4 59.0 2772.0 5.83 5.89 3.54
0.8 Very Good F SI2 61.0 57.0 2772.0 6.01 6.03 3.67
0.58 Ideal G VVS1 61.5 55.0 2772.0 5.39 5.44 3.33
0.58 Ideal F VVS1 61.7 56.0 2772.0 5.33 5.37 3.3
0.71 Good E VS2 59.2 61.0 2772.0 5.8 5.88 3.46
0.75 Ideal D SI2 61.3 56.0 2773.0 5.85 5.89 3.6
0.7 Premium D VS2 58.0 62.0 2773.0 5.87 5.78 3.38
1.17 Very Good J I1 60.2 61.0 2774.0 6.83 6.9 4.13
0.6 Ideal E VS1 61.7 55.0 2774.0 5.41 5.44 3.35
0.7 Ideal E SI1 62.7 55.0 2774.0 5.68 5.74 3.58
0.83 Good I VS2 64.6 54.0 2774.0 5.85 5.88 3.79
0.74 Very Good F VS2 61.3 61.0 2775.0 5.8 5.84 3.57
0.72 Very Good G VS2 63.7 56.4 2776.0 5.62 5.69 3.61
0.71 Premium E VS2 62.7 58.0 2776.0 5.74 5.68 3.58
0.71 Ideal E VS2 62.2 57.0 2776.0 5.79 5.62 3.55
0.54 Ideal E VVS2 61.6 56.0 2776.0 5.25 5.27 3.24
0.54 Ideal E VVS2 61.5 57.0 2776.0 5.24 5.26 3.23
0.72 Ideal G SI1 61.8 56.0 2776.0 5.72 5.75 3.55
0.72 Ideal G SI1 60.7 56.0 2776.0 5.79 5.82 3.53
0.72 Good G VS2 59.7 60.5 2776.0 5.8 5.84 3.47
0.71 Ideal G SI1 60.5 56.0 2776.0 5.8 5.76 3.5
0.7 Very Good D VS1 62.7 58.0 2777.0 5.66 5.73 3.57
0.71 Premium F VS2 62.1 58.0 2777.0 5.67 5.7 3.53
0.71 Very Good F VS2 62.8 57.0 2777.0 5.64 5.69 3.56
0.71 Good F VS2 63.8 58.0 2777.0 5.61 5.64 3.59
0.71 Good F VS2 57.8 60.0 2777.0 5.87 5.9 3.4
0.7 Ideal E VS2 62.1 55.0 2777.0 5.7 5.67 3.53
0.7 Premium E VS2 61.1 60.0 2777.0 5.71 5.64 3.47
0.7 Premium E SI1 60.0 59.0 2777.0 5.79 5.75 3.46
0.7 Premium E SI1 61.2 57.0 2777.0 5.73 5.68 3.49
0.7 Premium E SI1 62.7 59.0 2777.0 5.67 5.63 3.54
0.7 Premium E SI1 61.0 57.0 2777.0 5.73 5.68 3.48
0.7 Premium E SI1 61.0 58.0 2777.0 5.78 5.72 3.51
0.7 Ideal E SI1 61.4 57.0 2777.0 5.76 5.7 3.52
0.72 Premium F SI1 61.8 61.0 2777.0 5.82 5.71 3.56
0.7 Very Good E SI1 59.9 63.0 2777.0 5.76 5.7 3.43
0.7 Premium E SI1 61.3 58.0 2777.0 5.71 5.68 3.49
0.7 Premium E SI1 60.5 58.0 2777.0 5.77 5.74 3.48
0.7 Good E VS2 64.1 59.0 2777.0 5.64 5.59 3.6
0.98 Fair H SI2 67.9 60.0 2777.0 6.05 5.97 4.08
0.78 Premium F SI1 62.4 58.0 2777.0 5.83 5.8 3.63
0.7 Very Good E SI1 63.2 60.0 2777.0 5.6 5.51 3.51
0.52 Ideal F VVS1 61.3 55.0 2778.0 5.19 5.22 3.19
0.73 Very Good H VS2 60.8 56.0 2779.0 5.82 5.84 3.55
0.74 Ideal E SI1 61.7 56.0 2779.0 5.84 5.8 3.59
0.7 Very Good F VS2 63.6 57.0 2780.0 5.61 5.65 3.58
0.77 Premium G VS2 61.2 58.0 2780.0 5.9 5.93 3.62
0.71 Ideal F VS2 62.1 54.0 2780.0 5.68 5.72 3.54
0.74 Ideal G VS1 61.5 55.0 2780.0 5.81 5.86 3.59
0.7 Ideal G VS1 61.4 59.0 2780.0 5.64 5.73 3.49
1.01 Premium F I1 61.8 60.0 2781.0 6.39 6.36 3.94
0.77 Ideal H SI1 62.2 56.0 2781.0 5.83 5.88 3.64
0.78 Ideal H SI1 61.2 56.0 2781.0 5.92 5.99 3.64
0.72 Very Good H VS1 60.6 63.0 2782.0 5.83 5.76 3.51
0.53 Very Good D VVS2 57.5 64.0 2782.0 5.34 5.37 3.08
0.76 Ideal G VS2 61.3 56.0 2782.0 5.9 5.94 3.63
0.7 Good E VS1 57.2 62.0 2782.0 5.81 5.77 3.31
0.7 Premium E VS1 62.9 60.0 2782.0 5.62 5.54 3.51
0.75 Very Good D SI2 63.1 58.0 2782.0 5.78 5.73 3.63
0.72 Ideal D SI1 60.8 57.0 2782.0 5.76 5.75 3.5
0.72 Premium D SI1 62.7 59.0 2782.0 5.73 5.69 3.58
0.7 Premium D SI1 62.8 60.0 2782.0 5.68 5.66 3.56
0.84 Fair G SI1 55.1 67.0 2782.0 6.39 6.2 3.47
0.75 Premium F SI1 61.4 59.0 2782.0 5.88 5.85 3.6
0.52 Ideal F IF 62.2 55.0 2783.0 5.14 5.18 3.21
0.72 Very Good F VS2 63.0 54.0 2784.0 5.69 5.73 3.6
0.79 Very Good H VS1 63.7 56.0 2784.0 5.85 5.92 3.75
0.72 Very Good F VS2 63.6 58.0 2787.0 5.66 5.69 3.61
0.51 Ideal F VVS1 62.0 57.0 2787.0 5.11 5.15 3.18
0.64 Ideal D VS1 61.5 56.0 2787.0 5.54 5.55 3.41
0.7 Very Good H VVS1 60.5 60.0 2788.0 5.74 5.77 3.48
0.83 Very Good I VS1 61.1 60.0 2788.0 6.07 6.1 3.72
0.76 Ideal I VVS2 61.8 56.0 2788.0 5.85 5.87 3.62
0.71 Good D VS2 63.3 56.0 2788.0 5.64 5.68 3.58
0.77 Good G VS1 59.4 64.0 2788.0 5.97 5.92 3.53
0.71 Ideal F SI1 62.5 55.0 2788.0 5.71 5.65 3.55
1.01 Fair E I1 64.5 58.0 2788.0 6.29 6.21 4.03
1.01 Premium H SI2 62.7 59.0 2788.0 6.31 6.22 3.93
0.77 Good F SI1 64.2 52.0 2789.0 5.81 5.77 3.72
0.76 Good E SI1 63.7 54.0 2789.0 5.76 5.85 3.7
0.76 Premium E SI1 60.4 58.0 2789.0 5.92 5.94 3.58
0.76 Premium E SI1 61.8 58.0 2789.0 5.82 5.86 3.61
1.05 Very Good J SI2 63.2 56.0 2789.0 6.49 6.45 4.09
0.81 Ideal G SI2 61.6 56.0 2789.0 5.97 6.01 3.69
0.7 Ideal E SI1 61.6 56.0 2789.0 5.72 5.75 3.53
0.55 Ideal G IF 60.9 57.0 2789.0 5.28 5.3 3.22
0.81 Good G SI2 61.0 61.0 2789.0 5.94 5.99 3.64
0.63 Premium E VVS2 62.1 57.0 2789.0 5.48 5.41 3.38
0.63 Premium E VVS1 60.9 60.0 2789.0 5.55 5.52 3.37
0.77 Premium H VS1 61.3 60.0 2789.0 5.9 5.88 3.61
1.05 Fair J SI2 65.8 59.0 2789.0 6.41 6.27 4.18
0.64 Ideal G IF 61.3 56.0 2790.0 5.54 5.58 3.41
0.76 Premium I VVS1 58.8 59.0 2790.0 6.0 5.94 3.51
0.83 Ideal F SI2 62.3 55.0 2790.0 6.02 6.05 3.76
0.71 Premium F VS1 60.1 62.0 2790.0 5.77 5.74 3.46
0.71 Premium F VS1 61.8 59.0 2790.0 5.73 5.69 3.53
0.87 Very Good I SI1 63.6 55.8 2791.0 6.07 6.1 3.87
0.73 Ideal E SI1 62.2 56.0 2791.0 5.74 5.78 3.58
0.71 Premium E SI1 59.2 59.0 2792.0 5.83 5.86 3.46
0.71 Premium E SI1 61.8 59.0 2792.0 5.7 5.75 3.54
0.71 Ideal E SI1 61.3 55.0 2792.0 5.72 5.77 3.52
0.7 Premium F VS1 62.1 60.0 2792.0 5.71 5.65 3.53
0.7 Premium F VS1 60.7 60.0 2792.0 5.78 5.75 3.5
0.76 Premium H VVS2 59.6 57.0 2792.0 5.91 5.86 3.51
0.7 Ideal F VS1 62.2 56.0 2792.0 5.73 5.68 3.55
0.79 Very Good G SI1 60.6 57.0 2793.0 5.98 6.06 3.65
0.7 Very Good E VS2 62.9 57.0 2793.0 5.66 5.69 3.57
0.7 Good E VS2 64.1 55.0 2793.0 5.6 5.66 3.61
0.76 Ideal I VS2 61.3 56.0 2793.0 5.87 5.91 3.61
0.73 Ideal H VS2 62.7 55.0 2793.0 5.72 5.76 3.6
0.79 Very Good E SI1 63.2 56.0 2794.0 5.91 5.86 3.72
0.71 Very Good E VS2 60.7 56.0 2795.0 5.81 5.82 3.53
0.81 Premium I VVS2 61.9 60.0 2795.0 5.91 5.86 3.64
0.81 Ideal F SI2 62.6 55.0 2795.0 5.92 5.96 3.72
0.72 Good F VS1 60.7 60.0 2795.0 5.74 5.72 3.48
0.72 Premium D SI2 62.0 60.0 2795.0 5.73 5.69 3.54
0.72 Premium I IF 63.0 57.0 2795.0 5.72 5.7 3.6
0.81 Premium H VS2 58.0 59.0 2795.0 6.17 6.13 3.57
0.72 Premium G VS2 62.9 57.0 2795.0 5.73 5.65 3.58
1.0 Premium I SI2 58.2 60.0 2795.0 6.61 6.55 3.83
0.73 Good E SI1 63.2 58.0 2796.0 5.7 5.76 3.62
0.81 Very Good H SI2 61.3 59.0 2797.0 5.94 6.01 3.66
0.81 Very Good E SI1 60.3 60.0 2797.0 6.07 6.1 3.67
0.71 Premium D SI1 62.7 60.0 2797.0 5.67 5.71 3.57
0.71 Premium D SI1 61.3 58.0 2797.0 5.73 5.75 3.52
0.71 Premium D SI1 61.6 60.0 2797.0 5.74 5.69 3.52
0.57 Ideal F VVS2 61.9 55.0 2797.0 5.34 5.35 3.31
0.51 Ideal D VVS1 61.7 56.0 2797.0 5.12 5.16 3.17
0.72 Ideal G VS2 61.9 58.0 2797.0 5.72 5.75 3.55
0.74 Ideal H VS1 61.8 58.0 2797.0 5.77 5.81 3.58
0.74 Ideal H VS1 61.6 56.0 2797.0 5.81 5.82 3.58
0.7 Fair G VVS1 58.8 66.0 2797.0 5.81 5.9 3.44
0.8 Premium F SI2 61.0 57.0 2797.0 6.03 6.01 3.67
1.01 Fair E SI2 67.4 60.0 2797.0 6.19 6.05 4.13
0.8 Very Good H VS2 63.4 60.0 2797.0 5.92 5.82 3.72
0.77 Ideal I VS1 61.5 59.0 2798.0 5.87 5.91 3.62
0.83 Very Good E SI2 58.0 62.0 2799.0 6.19 6.25 3.61
0.82 Ideal F SI2 62.4 54.0 2799.0 5.97 6.02 3.74
0.78 Ideal D SI1 61.9 57.0 2799.0 5.91 5.86 3.64
0.6 Very Good G IF 61.6 56.0 2800.0 5.43 5.46 3.35
0.9 Good I SI2 62.2 59.0 2800.0 6.07 6.11 3.79
0.7 Premium E VS1 62.2 58.0 2800.0 5.6 5.66 3.5
0.9 Very Good I SI2 61.3 56.0 2800.0 6.17 6.23 3.8
0.83 Ideal G SI1 62.3 57.0 2800.0 5.99 6.08 3.76
0.83 Ideal G SI1 61.8 57.0 2800.0 6.03 6.07 3.74
0.83 Very Good H SI1 62.5 59.0 2800.0 5.95 6.02 3.74
0.74 Premium G VS1 62.9 60.0 2800.0 5.74 5.68 3.59
0.79 Ideal I VS1 61.8 59.0 2800.0 5.92 5.95 3.67
0.61 Ideal G IF 62.3 56.0 2800.0 5.43 5.45 3.39
0.76 Fair G VS1 59.0 70.0 2800.0 5.89 5.8 3.46
0.96 Ideal F I1 60.7 55.0 2801.0 6.37 6.41 3.88
0.73 Ideal F VS2 62.5 55.0 2801.0 5.8 5.76 3.61
0.73 Premium F VS2 62.7 58.0 2801.0 5.76 5.7 3.59
0.75 Ideal H SI1 60.4 57.0 2801.0 5.93 5.96 3.59
0.71 Premium F VS2 62.1 58.0 2801.0 5.7 5.67 3.53
0.71 Good F VS2 57.8 60.0 2801.0 5.9 5.87 3.4
0.71 Good F VS2 63.8 58.0 2801.0 5.64 5.61 3.59
0.71 Premium F VS2 62.8 57.0 2801.0 5.69 5.64 3.56
1.04 Premium G I1 62.2 58.0 2801.0 6.46 6.41 4.0
1.0 Premium J SI2 62.3 58.0 2801.0 6.45 6.34 3.98
0.87 Very Good G SI2 59.9 58.0 2802.0 6.19 6.23 3.72
0.53 Ideal F IF 61.9 54.0 2802.0 5.22 5.25 3.24
0.72 Premium E VS2 63.0 55.0 2802.0 5.79 5.61 3.59
0.72 Premium F VS1 62.4 58.0 2802.0 5.83 5.7 3.6
0.7 Very Good F VS2 62.9 58.0 2803.0 5.63 5.65 3.55
0.74 Very Good E SI1 63.5 56.0 2803.0 5.74 5.79 3.66
0.71 Ideal G VS2 61.3 56.0 2803.0 5.75 5.71 3.51
0.73 Ideal E SI1 60.6 54.0 2803.0 5.84 5.89 3.55
0.7 Good G VS1 65.1 58.0 2803.0 5.56 5.59 3.63
0.71 Premium F VS2 62.6 58.0 2803.0 5.7 5.67 3.56
0.71 Premium F VS2 58.0 62.0 2803.0 5.85 5.81 3.38
0.71 Premium G VS1 62.4 61.0 2803.0 5.7 5.65 3.54
0.77 Premium G VS2 61.3 57.0 2803.0 5.93 5.88 3.62
0.71 Premium G VS2 59.9 60.0 2803.0 5.81 5.77 3.47
0.78 Premium G VS2 60.8 58.0 2803.0 6.03 5.95 3.64
0.71 Very Good G VS1 63.5 55.0 2803.0 5.66 5.64 3.59
0.91 Ideal D SI2 62.2 57.0 2803.0 6.21 6.15 3.85
0.71 Very Good E VS2 63.8 58.0 2804.0 5.62 5.66 3.6
0.71 Very Good E VS2 64.0 57.0 2804.0 5.66 5.68 3.63
0.8 Very Good E SI2 62.5 56.0 2804.0 5.88 5.96 3.7
0.7 Very Good D SI1 62.3 58.0 2804.0 5.69 5.73 3.56
0.72 Ideal F VS1 61.7 57.0 2804.0 5.74 5.77 3.55
0.72 Very Good F VS1 62.2 58.0 2804.0 5.75 5.7 3.56
0.82 Ideal H VS2 61.5 56.0 2804.0 6.01 6.08 3.72
0.7 Ideal D SI1 61.0 59.0 2804.0 5.68 5.7 3.47
0.72 Ideal D SI1 62.2 56.0 2804.0 5.74 5.77 3.58
0.72 Ideal D SI1 61.5 54.0 2804.0 5.77 5.8 3.56
0.9 Fair I SI1 67.3 59.0 2804.0 5.93 5.84 3.96
0.74 Premium F VS2 61.7 58.0 2805.0 5.85 5.78 3.59
0.74 Premium F VS2 61.9 56.0 2805.0 5.8 5.77 3.58
0.73 Ideal E SI2 61.8 58.0 2805.0 5.77 5.81 3.58
0.57 Fair E VVS1 58.7 66.0 2805.0 5.34 5.43 3.16
0.73 Premium F VS2 62.5 57.0 2805.0 5.75 5.7 3.58
0.72 Ideal G VS2 62.8 56.0 2805.0 5.74 5.7 3.59
0.74 Fair F VS2 61.1 68.0 2805.0 5.82 5.75 3.53
0.82 Good G VS2 64.0 57.0 2805.0 5.92 5.89 3.78
0.81 Very Good G SI1 62.5 60.0 2806.0 5.89 5.94 3.69
0.75 Very Good H VVS1 60.6 58.0 2806.0 5.85 5.9 3.56
0.7 Ideal F SI1 61.6 55.0 2806.0 5.72 5.74 3.53
0.71 Very Good F VS1 62.2 58.0 2807.0 5.66 5.72 3.54
0.71 Very Good F VS1 60.0 57.0 2807.0 5.84 5.9 3.52
0.93 Premium J SI2 61.9 57.0 2807.0 6.21 6.19 3.84
0.8 Very Good H VS2 62.8 57.0 2808.0 5.87 5.91 3.7
0.7 Very Good F VS1 62.0 57.0 2808.0 5.64 5.71 3.52
1.0 Fair G I1 66.4 59.0 2808.0 6.16 6.09 4.07
0.75 Very Good G VS2 63.4 56.0 2808.0 5.78 5.74 3.65
0.58 Ideal E VVS2 60.9 56.0 2808.0 5.41 5.43 3.3
0.73 Very Good D SI1 63.1 57.0 2808.0 5.74 5.7 3.61
0.81 Very Good F SI1 63.1 59.0 2809.0 5.85 5.79 3.67
0.81 Premium D SI2 59.2 57.0 2809.0 6.15 6.05 3.61
0.71 Premium F SI1 60.7 54.0 2809.0 5.84 5.8 3.53
1.2 Fair F I1 64.6 56.0 2809.0 6.73 6.66 4.33
0.7 Very Good F VS1 61.8 56.0 2810.0 5.63 5.7 3.5
0.7 Very Good F VS1 59.9 60.0 2810.0 5.77 5.84 3.48
0.74 Ideal D SI2 61.7 55.0 2810.0 5.81 5.85 3.6
0.7 Good F VS1 62.8 61.0 2810.0 5.57 5.61 3.51
0.8 Good G SI1 62.7 57.0 2810.0 5.84 5.93 3.69
0.75 Very Good F SI1 63.4 58.0 2811.0 5.72 5.76 3.64
0.83 Very Good D SI1 63.5 54.0 2811.0 5.98 5.95 3.79
1.0 Fair J VS2 65.7 59.0 2811.0 6.14 6.07 4.01
0.99 Fair I SI2 68.1 56.0 2811.0 6.21 6.06 4.18
0.7 Very Good G VS1 63.0 60.0 2812.0 5.57 5.64 3.53
0.7 Very Good F VS2 59.5 58.0 2812.0 5.75 5.85 3.45
0.7 Good E SI1 63.5 59.0 2812.0 5.49 5.53 3.5
0.7 Very Good F VS2 61.7 58.0 2812.0 5.63 5.69 3.49
0.32 Premium I SI1 62.7 58.0 554.0 4.37 4.34 2.73
0.32 Premium I SI1 62.8 58.0 554.0 4.39 4.34 2.74
0.32 Ideal I SI1 62.4 57.0 554.0 4.37 4.35 2.72
0.32 Premium I SI1 61.0 59.0 554.0 4.39 4.36 2.67
0.32 Very Good I SI1 63.1 56.0 554.0 4.39 4.36 2.76
0.32 Ideal I SI1 60.7 57.0 554.0 4.47 4.42 2.7
0.3 Premium H SI1 60.9 59.0 554.0 4.31 4.29 2.62
0.3 Premium H SI1 60.1 55.0 554.0 4.41 4.38 2.64
0.3 Premium H SI1 62.9 58.0 554.0 4.28 4.24 2.68
0.3 Very Good H SI1 63.3 56.0 554.0 4.29 4.27 2.71
0.3 Good H SI1 63.8 55.0 554.0 4.26 4.2 2.7
0.3 Ideal H SI1 62.9 57.0 554.0 4.27 4.22 2.67
0.3 Very Good H SI1 63.4 60.0 554.0 4.25 4.23 2.69
0.32 Good I SI1 63.9 55.0 554.0 4.36 4.34 2.78
0.33 Ideal H SI2 61.4 56.0 554.0 4.85 4.79 2.95
0.29 Very Good E VS1 61.9 55.0 555.0 4.28 4.33 2.66
0.29 Very Good E VS1 62.4 55.0 555.0 4.2 4.25 2.63
0.31 Very Good F SI1 61.8 58.0 555.0 4.32 4.35 2.68
0.34 Ideal H VS2 61.5 56.0 555.0 4.47 4.5 2.76
0.34 Ideal H VS2 60.4 57.0 555.0 4.54 4.57 2.75
0.34 Ideal I VS1 61.8 55.0 555.0 4.48 4.52 2.78
0.34 Ideal I VS1 62.0 56.0 555.0 4.5 4.53 2.8
0.3 Ideal G VS1 62.3 56.0 555.0 4.29 4.31 2.68
0.29 Ideal F VS1 61.6 56.0 555.0 4.26 4.31 2.64
0.35 Ideal G SI1 60.6 56.0 555.0 4.56 4.58 2.77
0.43 Very Good E I1 58.4 62.0 555.0 4.94 5.0 2.9
0.32 Very Good F VS2 61.4 58.0 556.0 4.37 4.42 2.7
0.36 Ideal I VS2 61.9 56.0 556.0 4.54 4.57 2.82
0.3 Ideal G VS2 62.0 56.0 556.0 4.28 4.3 2.66
0.26 Ideal E VS1 61.5 57.0 556.0 4.09 4.12 2.52
0.7 Very Good F VS2 62.3 58.0 2812.0 5.64 5.72 3.54
0.7 Very Good F VS2 60.9 61.0 2812.0 5.66 5.71 3.46
0.71 Ideal D SI1 62.4 57.0 2812.0 5.69 5.72 3.56
0.99 Fair J SI1 55.0 61.0 2812.0 6.72 6.67 3.68
0.73 Premium E VS2 58.6 60.0 2812.0 5.92 5.89 3.46
0.51 Ideal F VVS1 62.0 57.0 2812.0 5.15 5.11 3.18
0.91 Premium G SI2 59.8 58.0 2813.0 6.3 6.29 3.77
0.84 Very Good E SI1 63.4 55.0 2813.0 6.0 5.95 3.79
0.91 Good I VS2 64.3 58.0 2813.0 6.09 6.05 3.9
0.76 Premium E SI1 62.2 59.0 2814.0 5.86 5.81 3.63
0.76 Ideal E SI1 61.7 57.0 2814.0 5.88 5.85 3.62
0.75 Premium E SI1 61.1 59.0 2814.0 5.86 5.83 3.57
0.55 Very Good D VVS1 61.5 56.0 2815.0 5.23 5.27 3.23
0.76 Very Good F SI2 58.5 62.0 2815.0 5.93 6.01 3.49
0.74 Premium G VS1 61.7 58.0 2815.0 5.79 5.81 3.58
0.7 Ideal H SI1 60.4 56.0 2815.0 5.75 5.81 3.49
0.7 Ideal H SI1 61.4 56.0 2815.0 5.7 5.76 3.52
0.7 Ideal H SI1 61.5 55.0 2815.0 5.73 5.79 3.54
0.7 Ideal H SI1 61.4 56.0 2815.0 5.72 5.77 3.53
0.9 Fair J VS2 65.0 56.0 2815.0 6.08 6.04 3.94
0.95 Fair F SI2 56.0 60.0 2815.0 6.62 6.53 3.68
0.89 Premium H SI2 60.2 59.0 2815.0 6.26 6.23 3.76
0.72 Premium E VS2 58.3 58.0 2815.0 5.99 5.92 3.47
0.96 Fair E SI2 53.1 63.0 2815.0 6.73 6.65 3.55
1.02 Premium G I1 60.3 58.0 2815.0 6.55 6.5 3.94
0.78 Very Good I VVS2 61.4 56.0 2816.0 5.91 5.95 3.64
0.61 Ideal G VVS2 60.1 57.0 2816.0 5.52 5.54 3.32
0.71 Good D VS1 63.4 55.0 2816.0 5.61 5.69 3.58
0.78 Premium F SI1 61.5 59.0 2816.0 5.96 5.88 3.64
0.87 Ideal H SI2 62.7 56.0 2816.0 6.16 6.13 3.85
0.83 Ideal H SI1 62.5 55.0 2816.0 6.04 6.0 3.76
0.71 Premium E SI1 61.3 56.0 2817.0 5.78 5.73 3.53
0.71 Ideal I VVS2 60.2 56.0 2817.0 5.84 5.89 3.53
0.71 Ideal E VS2 62.7 57.0 2817.0 5.66 5.64 3.54
0.71 Premium E VS2 62.3 58.0 2817.0 5.69 5.65 3.53
0.63 Ideal F VVS2 61.5 56.0 2817.0 5.48 5.52 3.38
0.71 Premium E SI1 59.2 59.0 2817.0 5.86 5.83 3.46
0.71 Premium E SI1 61.8 59.0 2817.0 5.75 5.7 3.54
0.71 Ideal E SI1 61.3 55.0 2817.0 5.77 5.72 3.52
0.71 Premium E SI1 61.4 58.0 2817.0 5.77 5.73 3.53
0.9 Ideal J VS2 62.8 55.0 2817.0 6.2 6.16 3.88
0.71 Good E SI1 62.8 64.0 2817.0 5.6 5.54 3.5
0.7 Premium E VS2 62.4 61.0 2818.0 5.66 5.63 3.52
0.7 Premium E VS2 59.3 60.0 2818.0 5.78 5.73 3.41
0.7 Premium E VS2 63.0 60.0 2818.0 5.64 5.6 3.54
1.0 Premium H I1 61.3 60.0 2818.0 6.43 6.39 3.93
0.86 Premium F SI2 59.3 62.0 2818.0 6.36 6.22 3.73
0.8 Ideal H SI1 61.0 57.0 2818.0 6.07 6.0 3.68
0.7 Ideal E VS1 62.9 57.0 2818.0 5.66 5.61 3.54
0.7 Premium E VS1 59.6 57.0 2818.0 5.91 5.83 3.5
0.7 Premium F VS2 61.8 60.0 2818.0 5.69 5.64 3.5
0.7 Premium E VS1 62.7 57.0 2818.0 5.68 5.64 3.55
1.0 Fair H SI2 65.3 62.0 2818.0 6.34 6.12 4.08
0.72 Very Good G VS1 63.8 58.0 2819.0 5.64 5.68 3.61
0.72 Ideal H VS1 62.3 56.0 2819.0 5.73 5.77 3.58
0.7 Good F VS1 59.7 63.0 2819.0 5.76 5.79 3.45
0.86 Good F SI2 64.3 60.0 2819.0 5.97 5.95 3.83
0.71 Ideal G VS1 62.9 58.0 2820.0 5.66 5.69 3.57
0.75 Ideal E SI1 62.0 57.0 2821.0 5.8 5.78 3.59
0.73 Premium E VS2 61.6 59.0 2821.0 5.77 5.73 3.54
0.53 Ideal E VVS1 61.9 55.0 2821.0 5.2 5.21 3.22
0.73 Premium E SI1 61.3 58.0 2821.0 5.83 5.76 3.55
0.73 Good E SI1 63.6 57.0 2821.0 5.72 5.7 3.63
0.73 Premium E SI1 59.6 61.0 2821.0 5.92 5.85 3.51
0.73 Premium E SI1 62.2 59.0 2821.0 5.77 5.68 3.56
0.73 Premium D SI1 61.7 55.0 2821.0 5.84 5.82 3.6
0.73 Very Good E SI1 63.2 58.0 2821.0 5.76 5.7 3.62
0.7 Premium E VS1 60.8 60.0 2822.0 5.74 5.71 3.48
0.72 Premium E VS2 60.3 59.0 2822.0 5.84 5.8 3.51
0.72 Premium E VS2 60.9 60.0 2822.0 5.8 5.76 3.52
0.72 Premium E VS2 62.4 59.0 2822.0 5.77 5.7 3.58
0.7 Premium E VS2 60.2 60.0 2822.0 5.73 5.7 3.44
0.6 Ideal F VVS2 62.0 55.0 2822.0 5.37 5.4 3.34
0.74 Ideal I VVS1 60.8 57.0 2822.0 5.85 5.89 3.57
0.73 Ideal F SI1 62.1 55.0 2822.0 5.75 5.78 3.58
0.71 Premium D SI1 62.7 60.0 2822.0 5.71 5.67 3.57
0.71 Premium D SI1 61.3 58.0 2822.0 5.75 5.73 3.52
0.7 Premium D SI1 60.2 60.0 2822.0 5.82 5.75 3.48
0.7 Ideal D SI1 60.7 56.0 2822.0 5.75 5.72 3.48
0.9 Good J VS2 64.0 61.0 2822.0 6.04 6.03 3.86
0.71 Ideal D SI1 60.2 56.0 2822.0 5.86 5.83 3.52
0.7 Premium E VS2 61.5 59.0 2822.0 5.73 5.68 3.51
0.7 Premium E VS2 62.6 56.0 2822.0 5.71 5.66 3.56
0.7 Ideal D SI1 59.7 58.0 2822.0 5.82 5.77 3.46
0.7 Good E SI1 61.4 64.0 2822.0 5.71 5.66 3.49
0.7 Ideal D SI1 62.5 57.0 2822.0 5.62 5.59 3.51
0.7 Ideal D SI1 61.8 56.0 2822.0 5.73 5.63 3.51
0.7 Premium E VS2 60.7 62.0 2822.0 5.72 5.68 3.46
0.7 Premium F VS2 60.6 58.0 2822.0 5.8 5.72 3.49
0.7 Ideal D SI1 61.4 54.0 2822.0 5.75 5.71 3.52
0.79 Very Good D SI2 62.8 59.0 2823.0 5.86 5.9 3.69
0.9 Good I SI1 63.8 57.0 2823.0 6.06 6.13 3.89
0.71 Premium E VS2 62.3 58.0 2823.0 5.71 5.66 3.54
0.61 Ideal E VVS2 61.3 54.0 2823.0 5.51 5.59 3.4
0.9 Fair H SI2 65.8 54.0 2823.0 6.05 5.98 3.96
0.71 Ideal E SI1 60.5 56.0 2823.0 5.77 5.73 3.47
0.71 Premium D VS2 61.2 59.0 2824.0 5.74 5.69 3.5
0.77 Ideal I VVS2 62.1 57.0 2824.0 5.84 5.86 3.63
0.74 Good E VS1 63.1 58.0 2824.0 5.73 5.75 3.62
0.82 Ideal F SI2 62.4 54.0 2824.0 6.02 5.97 3.74
0.82 Premium E SI2 60.8 60.0 2824.0 6.05 6.03 3.67
0.71 Premium G VS1 62.2 59.0 2825.0 5.73 5.66 3.54
0.83 Premium H SI1 60.0 59.0 2825.0 6.08 6.05 3.64
0.73 Very Good G VS1 62.0 57.0 2825.0 5.75 5.8 3.58
0.83 Premium H SI1 62.5 59.0 2825.0 6.02 5.95 3.74
1.17 Premium J I1 60.2 61.0 2825.0 6.9 6.83 4.13
0.91 Fair H SI2 61.3 67.0 2825.0 6.24 6.19 3.81
0.73 Premium E VS1 62.6 60.0 2826.0 5.75 5.68 3.58
0.7 Good E VS1 57.2 59.0 2826.0 5.94 5.88 3.38
0.9 Premium I SI2 62.2 59.0 2826.0 6.11 6.07 3.79
0.7 Premium E VS1 62.2 58.0 2826.0 5.66 5.6 3.5
0.7 Very Good D VS2 63.3 56.0 2826.0 5.6 5.58 3.54
0.7 Premium E VS1 59.4 61.0 2826.0 5.78 5.74 3.42
0.9 Very Good I SI2 63.5 56.0 2826.0 6.17 6.07 3.88
0.78 Premium F SI1 60.8 60.0 2826.0 5.97 5.94 3.62
0.96 Ideal F I1 60.7 55.0 2826.0 6.41 6.37 3.88
0.7 Very Good D SI1 62.3 59.0 2827.0 5.67 5.7 3.54
0.72 Good D VS2 64.0 54.0 2827.0 5.68 5.7 3.64
0.79 Premium H VVS2 62.6 58.0 2827.0 5.96 5.9 3.71
0.7 Ideal H VVS1 61.6 57.0 2827.0 5.69 5.74 3.52
0.7 Ideal H VVS1 62.3 55.0 2827.0 5.66 5.7 3.54
0.7 Ideal D SI2 60.6 57.0 2828.0 5.74 5.77 3.49
1.01 Premium H SI2 61.6 61.0 2828.0 6.39 6.31 3.91
0.72 Premium F VS1 62.2 58.0 2829.0 5.75 5.7 3.56
0.8 Good E SI2 63.7 54.0 2829.0 5.91 5.87 3.75
0.59 Ideal E VVS1 62.0 56.0 2829.0 5.36 5.38 3.33
0.72 Ideal F VS1 61.7 57.0 2829.0 5.77 5.74 3.55
0.75 Premium E SI2 61.9 57.0 2829.0 5.88 5.82 3.62
0.8 Premium E SI2 60.2 57.0 2829.0 6.05 6.01 3.63
0.71 Very Good E VS2 62.7 59.0 2830.0 5.65 5.7 3.56
0.77 Very Good H SI1 61.7 56.0 2830.0 5.84 5.89 3.62
0.97 Ideal F I1 60.7 56.0 2830.0 6.41 6.43 3.9
0.53 Ideal F VVS1 60.9 57.0 2830.0 5.23 5.29 3.19
0.53 Ideal F VVS1 61.8 57.0 2830.0 5.16 5.19 3.2
0.8 Ideal I VS2 62.1 54.4 2830.0 5.94 5.99 3.7
0.9 Premium G SI1 60.6 62.0 2830.0 6.21 6.13 3.74
0.76 Very Good E SI2 60.8 60.0 2831.0 5.89 5.98 3.61
0.72 Ideal E SI1 62.3 57.0 2831.0 5.7 5.76 3.57
0.75 Ideal E SI1 61.4 57.0 2831.0 5.82 5.87 3.59
0.72 Premium E SI1 62.1 58.0 2831.0 5.73 5.76 3.57
0.79 Ideal G SI1 61.8 56.0 2831.0 5.93 5.91 3.66
0.72 Very Good F VS2 62.5 58.0 2832.0 5.71 5.75 3.58
0.91 Very Good I SI2 62.8 61.0 2832.0 6.15 6.18 3.87
0.71 Premium G VVS2 62.1 57.0 2832.0 5.75 5.65 3.54
0.81 Premium G SI1 63.0 60.0 2832.0 5.87 5.81 3.68
0.82 Ideal H SI1 62.5 57.0 2832.0 5.91 5.97 3.71
0.71 Premium F VS1 62.2 58.0 2832.0 5.72 5.66 3.54
0.9 Good J SI1 64.3 63.0 2832.0 6.05 6.01 3.88
0.8 Very Good I VS2 62.0 58.0 2833.0 5.86 5.95 3.66
0.56 Very Good E IF 61.0 59.0 2833.0 5.28 5.34 3.24
0.7 Very Good D VS2 59.6 61.0 2833.0 5.77 5.8 3.45
0.7 Ideal D VS2 61.0 57.0 2833.0 5.74 5.76 3.51
0.61 Ideal F VVS2 61.7 55.0 2833.0 5.45 5.48 3.37
0.85 Ideal H SI2 62.5 57.0 2833.0 6.02 6.07 3.78
0.7 Ideal F SI1 60.7 57.0 2833.0 5.73 5.75 3.49
0.8 Ideal G VS2 62.2 56.0 2834.0 5.94 5.87 3.67
0.8 Ideal H VS2 62.8 57.0 2834.0 5.91 5.87 3.7
0.51 Very Good D VVS1 59.9 58.0 2834.0 5.16 5.19 3.1
0.53 Ideal F VVS1 61.4 57.0 2834.0 5.2 5.23 3.2
0.78 Ideal I VS2 61.8 55.0 2834.0 5.92 5.95 3.67
0.9 Very Good J SI1 63.4 54.0 2834.0 6.17 6.14 3.9
0.9 Fair G SI2 65.3 59.0 2834.0 6.07 6.0 3.94
0.77 Ideal E SI2 60.7 55.0 2834.0 6.01 5.95 3.63
0.73 Ideal F VS1 61.2 56.0 2835.0 5.89 5.81 3.58
0.63 Ideal F VVS2 61.9 57.0 2835.0 5.47 5.51 3.4
0.7 Ideal E VS2 61.5 54.0 2835.0 5.7 5.75 3.52
0.72 Ideal E VS2 62.8 57.0 2835.0 5.71 5.73 3.59
0.72 Ideal E SI1 61.0 57.0 2835.0 5.78 5.8 3.53
0.75 Premium F VS2 59.6 59.0 2835.0 6.04 5.94 3.57
0.82 Very Good H SI1 60.7 56.0 2836.0 6.04 6.06 3.67
0.71 Good E VS2 62.8 60.0 2836.0 5.6 5.65 3.53
0.7 Premium E VS1 62.6 59.0 2837.0 5.69 5.66 3.55
0.7 Ideal E VS1 61.8 56.0 2837.0 5.74 5.69 3.53
0.71 Ideal F SI1 59.8 53.0 2838.0 5.86 5.82 3.49
0.76 Very Good H SI1 60.9 55.0 2838.0 5.92 5.94 3.61
0.82 Fair F SI1 64.9 58.0 2838.0 5.83 5.79 3.77
0.72 Premium F VS1 58.8 60.0 2838.0 5.91 5.89 3.47
0.7 Premium F VS2 62.3 58.0 2838.0 5.72 5.64 3.54
0.7 Premium F VS2 61.7 58.0 2838.0 5.69 5.63 3.49
0.7 Premium G VS1 62.6 55.0 2838.0 5.73 5.64 3.56
0.7 Premium F VS2 59.4 61.0 2838.0 5.83 5.79 3.45
0.7 Very Good E SI1 63.5 59.0 2838.0 5.53 5.49 3.5
0.7 Premium F VS2 60.9 61.0 2838.0 5.71 5.66 3.46
0.7 Premium F VS2 59.5 58.0 2838.0 5.85 5.75 3.45
0.7 Premium G VS1 63.0 60.0 2838.0 5.64 5.57 3.53
0.74 Very Good E SI1 60.0 57.0 2839.0 5.85 5.89 3.52
0.71 Ideal F VS1 61.5 57.0 2839.0 5.74 5.71 3.52
0.7 Ideal F VS1 61.6 54.0 2839.0 5.75 5.72 3.53
0.71 Ideal F VS1 62.1 55.0 2839.0 5.82 5.68 3.57
0.71 Premium F VS1 59.1 61.0 2839.0 5.84 5.81 3.44
0.71 Premium F VS1 59.0 60.0 2839.0 5.82 5.8 3.43
0.71 Premium F VS1 60.5 58.0 2839.0 5.75 5.72 3.47
0.7 Ideal F VS1 62.4 53.0 2839.0 5.73 5.71 3.57
0.73 Ideal G VS2 61.8 54.0 2839.0 5.8 5.82 3.59
0.7 Ideal E VS2 62.1 54.0 2839.0 5.69 5.72 3.54
0.7 Ideal G VS1 61.3 57.0 2839.0 5.71 5.74 3.51
0.71 Premium G VVS2 60.3 58.0 2839.0 5.82 5.78 3.5
0.71 Premium F VS1 59.2 58.0 2839.0 5.87 5.82 3.46
0.79 Premium G VS2 59.3 62.0 2839.0 6.09 6.01 3.59
0.71 Premium F VS1 62.7 59.0 2839.0 5.7 5.62 3.55
0.77 Very Good H VS1 61.0 60.0 2840.0 5.9 5.87 3.59
0.75 Very Good F SI2 59.8 56.0 2840.0 5.85 5.92 3.52
0.7 Ideal F SI1 61.0 56.0 2840.0 5.75 5.8 3.52
0.71 Premium F VS2 59.3 56.0 2840.0 5.88 5.82 3.47
0.92 Ideal D SI2 61.9 56.0 2840.0 6.27 6.2 3.86
0.83 Premium F SI2 61.4 59.0 2840.0 6.08 6.04 3.72
0.7 Premium H VVS1 59.2 60.0 2840.0 5.87 5.78 3.45
0.73 Premium F VS2 60.3 59.0 2841.0 5.9 5.87 3.55
0.71 Very Good D VS1 63.4 55.0 2841.0 5.69 5.61 3.58
0.73 Very Good D SI1 63.9 57.0 2841.0 5.66 5.71 3.63
0.82 Ideal F SI2 61.7 53.0 2841.0 6.0 6.12 3.74
0.82 Ideal F SI2 62.3 56.0 2841.0 5.96 6.02 3.73
0.82 Very Good F SI2 59.7 57.0 2841.0 6.12 6.14 3.66
0.52 Ideal F VVS1 61.2 56.0 2841.0 5.19 5.21 3.18
1.0 Premium F I1 58.9 60.0 2841.0 6.6 6.55 3.87
0.95 Fair G SI1 66.7 56.0 2841.0 6.16 6.03 4.06
0.73 Ideal D SI1 61.4 57.0 2841.0 5.76 5.8 3.55
0.73 Premium F VS2 59.9 59.0 2841.0 5.87 5.77 3.5
0.73 Premium G VS1 61.4 58.0 2841.0 5.82 5.77 3.56
0.8 Ideal I VS1 62.6 54.0 2842.0 5.92 5.96 3.72
0.7 Premium F VS2 58.7 61.0 2842.0 5.8 5.72 3.38
0.7 Very Good E VS2 60.2 62.0 2843.0 5.71 5.75 3.45
0.7 Very Good E VS2 62.7 58.0 2843.0 5.65 5.67 3.55
0.71 Very Good E VS2 59.4 58.0 2843.0 5.76 5.82 3.44
0.81 Very Good F SI2 63.2 58.0 2843.0 5.91 5.92 3.74
0.71 Very Good D SI1 61.5 58.0 2843.0 5.73 5.79 3.54
0.73 Ideal G VVS2 61.3 57.0 2843.0 5.81 5.84 3.57
0.73 Very Good F VS1 61.8 59.0 2843.0 5.73 5.79 3.56
0.72 Ideal E VS2 62.0 57.0 2843.0 5.71 5.74 3.55
0.81 Ideal F SI2 62.1 57.0 2843.0 5.91 5.95 3.68
0.71 Ideal G VVS2 60.7 57.0 2843.0 5.81 5.78 3.52
0.73 Very Good E SI1 57.7 61.0 2844.0 5.92 5.96 3.43
0.7 Very Good E VS1 62.0 59.0 2844.0 5.65 5.68 3.51
1.01 Ideal I I1 61.5 57.0 2844.0 6.45 6.46 3.97
1.01 Good I I1 63.1 57.0 2844.0 6.35 6.39 4.02
0.79 Ideal H VS2 62.5 57.0 2844.0 5.91 5.93 3.7
0.7 Very Good E VS2 61.8 59.0 2845.0 5.65 5.68 3.5
0.7 Very Good E VS2 58.9 60.0 2845.0 5.83 5.85 3.44
0.8 Good H VS2 63.4 60.0 2845.0 5.92 5.82 3.72
1.27 Premium H SI2 59.3 61.0 2845.0 7.12 7.05 4.2
0.79 Ideal D SI1 61.5 56.0 2846.0 5.96 5.91 3.65
0.72 Very Good F VS1 60.2 59.0 2846.0 5.79 5.84 3.5
0.73 Ideal H VVS2 61.6 56.0 2846.0 5.79 5.84 3.58
1.01 Fair H SI2 65.4 59.0 2846.0 6.3 6.26 4.11
1.01 Good H I1 64.2 61.0 2846.0 6.25 6.18 3.99
0.73 Ideal E SI1 59.1 59.0 2846.0 5.92 5.95 3.51
0.7 Ideal E SI1 61.6 57.0 2846.0 5.71 5.76 3.53
0.7 Good F VS2 59.1 61.0 2846.0 5.76 5.84 3.43
0.77 Premium E SI1 62.9 59.0 2846.0 5.84 5.79 3.66
0.77 Premium G VS2 61.3 60.0 2846.0 5.91 5.81 3.59
0.77 Premium G VS1 61.4 58.0 2846.0 5.94 5.89 3.63
0.84 Very Good H SI1 61.2 57.0 2847.0 6.1 6.12 3.74
0.72 Ideal E SI1 60.3 57.0 2847.0 5.83 5.85 3.52
0.76 Premium D SI1 61.1 59.0 2847.0 5.93 5.88 3.61
0.7 Very Good G VVS2 62.9 59.0 2848.0 5.61 5.68 3.55
0.54 Ideal D VVS2 61.5 55.0 2848.0 5.25 5.29 3.24
0.75 Fair D SI2 64.6 57.0 2848.0 5.74 5.72 3.7
0.79 Good E SI1 64.1 54.0 2849.0 5.86 5.84 3.75
0.74 Very Good E VS1 63.1 58.0 2849.0 5.75 5.73 3.62
0.7 Very Good E VS2 61.0 60.0 2850.0 5.74 5.77 3.51
0.7 Ideal F VS2 60.8 59.0 2850.0 5.69 5.79 3.49
0.75 Ideal J SI1 61.5 56.0 2850.0 5.83 5.87 3.6
1.2 Very Good H I1 63.1 60.0 2850.0 6.75 6.67 4.23
0.8 Very Good F SI1 63.4 57.0 2851.0 5.89 5.82 3.71
0.66 Ideal D VS1 62.1 56.0 2851.0 5.54 5.57 3.45
0.87 Very Good F SI2 61.0 63.0 2851.0 6.22 6.07 3.75
0.86 Premium H SI1 62.7 59.0 2851.0 6.04 5.98 3.77
0.74 Ideal F SI1 61.0 57.0 2851.0 5.85 5.81 3.56
0.58 Very Good E IF 60.6 59.0 2852.0 5.37 5.43 3.27
0.78 Ideal I VS1 61.5 57.0 2852.0 5.88 5.92 3.63
0.74 Ideal G SI1 61.3 55.0 2852.0 5.85 5.86 3.59
0.73 Ideal E SI1 62.7 55.0 2852.0 5.7 5.79 3.6
0.91 Very Good I SI1 63.5 57.0 2852.0 6.12 6.07 3.87
0.71 Premium F VS2 62.6 58.0 2853.0 5.67 5.7 3.56
0.71 Good G VS1 63.5 55.0 2853.0 5.64 5.66 3.59
0.79 Ideal D SI2 62.8 57.0 2853.0 5.9 5.85 3.69
0.79 Premium D SI2 60.0 60.0 2853.0 6.07 6.03 3.63
0.71 Premium E SI1 62.7 58.0 2853.0 5.73 5.66 3.57
0.82 Premium I VS1 61.9 58.0 2853.0 5.99 5.97 3.7
0.78 Very Good H VS1 61.9 57.1 2854.0 5.87 5.95 3.66
0.7 Very Good E VS1 62.4 56.0 2854.0 5.64 5.7 3.54
1.12 Premium H I1 59.1 61.0 2854.0 6.78 6.75 4.0
0.73 Premium E VS2 62.0 57.0 2854.0 5.86 5.76 3.6
0.91 Fair J VS2 64.4 62.0 2854.0 6.06 6.03 3.89
0.91 Fair J VS2 65.4 60.0 2854.0 6.04 6.0 3.94
0.91 Good J VS2 64.2 58.0 2854.0 6.12 6.09 3.92
0.91 Fair H SI1 65.8 58.0 2854.0 6.04 6.01 3.96
0.7 Premium E VS1 58.4 59.0 2854.0 5.91 5.83 3.43
0.68 Premium F VVS2 61.7 57.0 2854.0 5.67 5.64 3.49
0.73 Very Good F VS2 62.5 57.0 2855.0 5.7 5.75 3.58
1.03 Good J SI1 63.6 57.0 2855.0 6.38 6.29 4.03
0.74 Premium D VS2 62.4 57.0 2855.0 5.8 5.74 3.6
0.98 Fair E SI2 53.3 67.0 2855.0 6.82 6.74 3.61
1.02 Fair I SI1 53.0 63.0 2856.0 6.84 6.77 3.66
1.0 Fair G SI2 67.8 61.0 2856.0 5.96 5.9 4.02
1.02 Ideal H SI2 61.6 55.0 2856.0 6.49 6.43 3.98
0.6 Ideal F VVS2 60.8 57.0 2856.0 5.44 5.49 3.32
0.8 Ideal G SI2 61.6 56.0 2856.0 5.97 6.01 3.69
0.97 Ideal F I1 60.7 56.0 2856.0 6.43 6.41 3.9
1.0 Fair I SI1 67.9 62.0 2856.0 6.19 6.03 4.15
0.26 Ideal E VS1 62.3 57.0 556.0 4.05 4.08 2.53
0.26 Ideal E VS1 62.1 56.0 556.0 4.09 4.12 2.55
0.36 Ideal H SI1 61.9 55.0 556.0 4.57 4.59 2.83
0.34 Good G VS2 57.5 61.0 556.0 4.6 4.66 2.66
0.34 Good E SI1 63.3 57.0 556.0 4.44 4.47 2.82
0.34 Good E SI1 63.5 55.0 556.0 4.44 4.47 2.83
0.34 Good E SI1 63.4 55.0 556.0 4.44 4.46 2.82
0.34 Very Good G VS2 59.6 62.0 556.0 4.54 4.56 2.71
0.34 Ideal E SI1 62.2 54.0 556.0 4.47 4.5 2.79
0.32 Good E VS2 64.1 54.0 556.0 4.34 4.37 2.79
0.31 Ideal I VVS1 61.6 55.0 557.0 4.36 4.41 2.7
0.31 Ideal I VVS1 61.3 56.0 557.0 4.36 4.38 2.68
0.31 Ideal I VVS1 62.3 54.0 557.0 4.37 4.4 2.73
0.31 Ideal I VVS1 62.0 54.0 557.0 4.37 4.4 2.72
0.31 Ideal I VVS1 62.7 53.0 557.0 4.33 4.35 2.72
0.31 Ideal I VVS1 62.2 53.0 557.0 4.36 4.38 2.72
0.31 Ideal G VS2 62.2 53.6 557.0 4.32 4.35 2.7
0.31 Ideal H VS1 61.6 54.8 557.0 4.35 4.37 2.69
0.31 Ideal H VS1 61.8 54.2 557.0 4.33 4.37 2.69
0.33 Premium G SI2 59.4 59.0 557.0 4.52 4.5 2.68
0.33 Premium F SI2 62.3 58.0 557.0 4.43 4.4 2.75
0.33 Premium G SI2 62.6 58.0 557.0 4.42 4.4 2.76
0.33 Ideal G SI2 61.9 56.0 557.0 4.45 4.41 2.74
0.33 Premium F SI2 63.0 58.0 557.0 4.42 4.4 2.78
0.33 Premium J VS1 62.8 58.0 557.0 4.41 4.38 2.76
0.33 Premium J VS1 61.5 61.0 557.0 4.46 4.39 2.72
0.33 Ideal J VS1 62.1 55.0 557.0 4.44 4.41 2.75
0.33 Ideal I SI1 63.0 57.0 557.0 4.39 4.37 2.76
0.33 Good I SI1 63.6 53.0 557.0 4.43 4.4 2.81
0.33 Premium I SI1 60.4 59.0 557.0 4.54 4.5 2.73
1.0 Fair H SI2 66.1 56.0 2856.0 6.21 5.97 4.04
0.77 Premium F SI1 60.8 59.0 2856.0 5.92 5.86 3.58
0.77 Premium F SI1 61.0 58.0 2856.0 5.94 5.9 3.61
0.7 Good E VVS2 60.1 63.0 2857.0 5.68 5.71 3.42
0.9 Very Good G SI2 63.1 58.0 2857.0 6.08 6.02 3.82
0.72 Ideal E SI1 62.3 57.0 2857.0 5.76 5.7 3.57
0.9 Premium I VS2 61.9 59.0 2857.0 6.2 6.14 3.82
0.72 Premium E SI1 62.1 58.0 2857.0 5.76 5.73 3.57
0.7 Ideal G VVS2 62.1 56.0 2858.0 5.63 5.71 3.52
0.81 Very Good F SI1 61.3 57.0 2858.0 6.02 6.05 3.7
0.81 Very Good F SI1 61.7 57.0 2858.0 6.0 6.05 3.72
0.71 Premium E VS2 61.0 60.0 2858.0 5.76 5.69 3.49
0.7 Premium E VS2 61.4 59.0 2858.0 5.73 5.7 3.51
0.71 Premium E VS2 61.5 60.0 2858.0 5.76 5.68 3.52
0.71 Very Good E VS2 63.5 59.0 2858.0 5.68 5.59 3.58
0.92 Premium J SI1 62.9 58.0 2858.0 6.22 6.18 3.9
0.76 Ideal E SI1 62.7 54.0 2858.0 5.88 5.83 3.67
0.73 Ideal D SI1 61.5 56.0 2858.0 5.84 5.8 3.58
0.71 Premium D VS2 60.4 62.0 2858.0 5.74 5.72 3.46
0.7 Good E VVS2 63.6 62.0 2858.0 5.61 5.58 3.56
0.9 Fair G SI2 64.5 56.0 2858.0 6.06 6.0 3.89
0.71 Fair D VS2 56.9 65.0 2858.0 5.89 5.84 3.34
0.7 Ideal D VS2 61.0 57.0 2859.0 5.76 5.74 3.51
0.7 Premium D VS2 62.4 56.0 2859.0 5.72 5.66 3.55
0.77 Premium F VS1 60.9 60.0 2859.0 5.91 5.88 3.59
0.71 Ideal G VS1 61.5 56.0 2859.0 5.74 5.78 3.54
0.7 Premium D VS2 59.6 61.0 2859.0 5.8 5.77 3.45
0.75 Fair F VS1 55.8 70.0 2859.0 6.09 5.98 3.37
0.83 Premium E SI2 59.2 60.0 2859.0 6.17 6.12 3.64
0.71 Very Good F VS2 61.3 61.0 2860.0 5.68 5.73 3.5
0.9 Very Good J SI2 63.6 58.0 2860.0 6.07 6.1 3.87
0.6 Ideal E VVS2 61.9 54.9 2860.0 5.41 5.44 3.35
0.71 Premium D VS1 62.9 57.0 2860.0 5.66 5.6 3.54
0.53 Ideal F VVS1 61.4 57.0 2860.0 5.23 5.2 3.2
0.71 Premium D SI1 60.7 58.0 2861.0 5.95 5.78 3.56
0.62 Ideal G VVS2 61.6 56.0 2861.0 5.45 5.48 3.37
0.62 Ideal G VVS2 61.6 56.0 2861.0 5.48 5.51 3.38
0.9 Premium I SI1 63.0 58.0 2861.0 6.09 6.01 3.81
0.62 Fair F IF 60.1 61.0 2861.0 5.53 5.56 3.33
0.82 Premium E SI2 61.7 59.0 2861.0 6.01 5.98 3.7
0.66 Premium D VS1 61.0 58.0 2861.0 5.67 5.57 3.43
0.7 Very Good D SI1 62.5 55.0 2862.0 5.67 5.72 3.56
0.8 Very Good F SI1 62.6 58.0 2862.0 5.9 5.92 3.7
0.8 Very Good D SI2 62.5 59.0 2862.0 5.88 5.92 3.69
0.79 Premium F SI1 62.3 54.0 2862.0 5.97 5.91 3.7
0.71 Very Good F VVS1 63.2 60.0 2862.0 5.65 5.61 3.56
0.7 Ideal H VS2 61.1 57.0 2862.0 5.71 5.74 3.5
0.7 Very Good E VS2 58.7 63.0 2862.0 5.73 5.69 3.35
0.79 Premium H VS1 60.0 60.0 2862.0 6.07 5.99 3.64
0.7 Premium E VS2 59.5 59.0 2862.0 5.82 5.77 3.45
1.22 Premium E I1 60.9 57.0 2862.0 6.93 6.88 4.21
1.01 Fair E SI2 67.6 57.0 2862.0 6.21 6.11 4.18
0.73 Premium E VS2 62.5 61.0 2862.0 5.78 5.64 3.59
0.91 Good I VS2 64.3 58.0 2863.0 6.05 6.09 3.9
0.71 Ideal D SI1 60.8 56.0 2863.0 5.8 5.77 3.52
0.83 Premium G SI1 62.3 58.0 2863.0 6.01 5.97 3.73
0.84 Premium F SI2 62.3 59.0 2863.0 6.06 6.01 3.76
0.71 Premium D SI1 61.0 61.0 2863.0 5.82 5.75 3.53
0.71 Premium D SI1 59.7 59.0 2863.0 5.82 5.8 3.47
0.71 Premium D SI1 61.7 56.0 2863.0 5.8 5.68 3.54
0.71 Ideal D SI1 61.7 57.0 2863.0 5.75 5.7 3.53
0.71 Premium D SI1 61.4 58.0 2863.0 5.79 5.75 3.54
0.71 Premium D SI1 60.6 58.0 2863.0 5.79 5.77 3.5
0.91 Premium J SI1 59.5 62.0 2863.0 6.4 6.18 3.74
0.9 Premium J VS2 59.8 62.0 2863.0 6.24 6.21 3.72
0.71 Premium H VVS2 61.5 62.0 2863.0 5.74 5.68 3.51
0.71 Premium E SI1 59.1 61.0 2863.0 5.84 5.8 3.44
0.72 Ideal F VS2 59.5 57.0 2863.0 5.91 5.86 3.5
0.72 Premium E SI1 60.9 60.0 2863.0 5.78 5.74 3.51
0.71 Ideal E VS2 61.0 55.0 2863.0 5.79 5.75 3.52
0.81 Ideal E SI2 60.3 57.0 2864.0 6.07 6.04 3.65
0.83 Very Good I VS2 61.6 58.0 2865.0 6.05 6.07 3.73
0.73 Premium D SI1 60.8 55.0 2865.0 5.87 5.81 3.55
0.56 Very Good D VVS1 62.0 56.0 2866.0 5.25 5.3 3.27
0.56 Very Good D VVS1 61.8 55.0 2866.0 5.27 5.31 3.27
0.71 Ideal E VS1 62.2 55.0 2866.0 5.74 5.7 3.56
0.7 Ideal H VVS1 62.3 58.0 2866.0 5.66 5.7 3.54
0.96 Premium I SI1 61.3 58.0 2866.0 6.39 6.3 3.89
0.71 Very Good H VVS1 62.9 57.0 2867.0 5.67 5.69 3.57
0.7 Ideal D VS2 62.4 57.0 2867.0 5.68 5.61 3.52
0.71 Ideal H VVS1 60.4 57.0 2867.0 5.78 5.81 3.5
0.8 Premium H VS2 61.2 53.0 2867.0 6.05 5.98 3.68
0.95 Premium F SI2 58.4 57.0 2867.0 6.49 6.41 3.77
0.82 Ideal F SI2 62.3 56.0 2867.0 5.99 5.95 3.72
0.52 Ideal F VVS1 61.2 56.0 2867.0 5.21 5.19 3.18
0.82 Ideal F SI2 61.7 53.0 2867.0 6.12 6.0 3.74
0.82 Ideal F SI2 62.3 56.0 2867.0 6.02 5.96 3.73
0.82 Premium F SI2 59.7 57.0 2867.0 6.14 6.12 3.66
0.8 Ideal G SI1 61.3 57.0 2867.0 5.96 5.91 3.64
0.96 Fair F SI2 68.2 61.0 2867.0 6.07 5.88 4.1
0.72 Ideal I VS1 62.4 55.0 2868.0 5.72 5.75 3.58
0.62 Ideal G IF 60.5 57.0 2868.0 5.52 5.56 3.35
0.79 Premium E SI2 61.0 58.0 2868.0 5.96 5.9 3.62
0.75 Very Good E SI1 63.1 56.0 2868.0 5.78 5.7 3.62
1.08 Premium D I1 61.9 60.0 2869.0 6.55 6.48 4.03
0.72 Ideal E SI1 60.8 55.0 2869.0 5.77 5.84 3.53
0.62 Ideal G IF 61.8 56.0 2869.0 5.43 5.47 3.37
0.73 Ideal G VVS2 61.3 57.0 2869.0 5.84 5.81 3.57
0.72 Ideal H VVS2 60.9 57.0 2869.0 5.79 5.77 3.52
0.52 Premium F VVS2 61.8 60.0 2870.0 5.16 5.13 3.18
0.83 Ideal E SI2 62.2 57.0 2870.0 6.0 6.05 3.75
0.64 Premium E VVS2 62.1 58.0 2870.0 5.56 5.51 3.44
0.8 Ideal G SI1 62.5 57.0 2870.0 5.94 5.9 3.7
0.74 Ideal H SI1 62.1 56.0 2870.0 5.77 5.83 3.6
0.72 Ideal F SI1 61.5 56.0 2870.0 5.72 5.79 3.54
0.82 Ideal H VS2 59.5 57.0 2870.0 6.12 6.09 3.63
0.73 Premium E VS1 61.3 59.0 2870.0 5.81 5.78 3.55
1.04 Premium I I1 61.6 61.0 2870.0 6.47 6.45 3.98
0.73 Very Good E SI1 61.3 58.0 2871.0 5.76 5.83 3.55
0.73 Good E SI1 63.6 57.0 2871.0 5.7 5.72 3.63
0.9 Premium J SI1 62.8 59.0 2871.0 6.13 6.03 3.82
0.75 Ideal I SI1 61.8 55.0 2871.0 5.83 5.85 3.61
0.79 Ideal G SI1 62.6 55.0 2871.0 5.91 5.95 3.71
0.7 Good D SI1 62.5 56.7 2872.0 5.59 5.62 3.51
0.75 Very Good D SI1 60.7 55.0 2872.0 5.87 5.92 3.58
1.02 Ideal I I1 61.7 56.0 2872.0 6.44 6.49 3.99
0.7 Very Good G SI2 59.0 62.0 2872.0 5.79 5.81 3.42
0.7 Ideal D SI1 61.8 56.0 2872.0 5.63 5.73 3.51
0.7 Good E SI1 61.4 64.0 2872.0 5.66 5.71 3.49
0.7 Ideal D SI1 61.4 54.0 2872.0 5.71 5.75 3.52
0.7 Ideal D SI1 60.7 56.0 2872.0 5.72 5.75 3.48
0.7 Very Good D SI1 60.2 60.0 2872.0 5.75 5.82 3.48
0.72 Very Good E VS2 58.3 57.0 2872.0 5.89 5.94 3.45
0.74 Ideal E SI1 62.3 58.0 2872.0 5.74 5.78 3.59
0.84 Good G SI1 65.1 55.0 2872.0 5.88 5.97 3.86
0.76 Very Good F VS2 62.0 58.0 2873.0 5.8 5.86 3.62
0.77 Very Good E SI1 63.2 58.0 2873.0 5.8 5.84 3.68
0.76 Ideal E SI2 62.8 56.0 2873.0 5.78 5.82 3.64
1.0 Ideal I SI2 61.7 56.0 2873.0 6.45 6.41 3.97
1.0 Fair H SI1 65.5 62.0 2873.0 6.14 6.07 4.0
0.9 Fair I SI1 65.7 58.0 2873.0 6.03 6.0 3.95
0.9 Premium J SI1 61.8 58.0 2873.0 6.16 6.13 3.8
0.9 Good J SI1 64.0 61.0 2873.0 6.0 5.96 3.83
0.9 Fair I SI1 65.3 61.0 2873.0 5.98 5.94 3.89
0.9 Fair I SI1 65.8 56.0 2873.0 6.01 5.96 3.94
0.9 Premium J SI1 60.9 61.0 2873.0 6.26 6.22 3.8
0.78 Premium F VS2 62.6 58.0 2874.0 5.91 5.82 3.67
0.71 Premium D VS2 61.2 59.0 2874.0 5.69 5.74 3.5
0.7 Premium F VS1 59.0 59.0 2874.0 5.79 5.77 3.41
0.7 Premium F VS1 60.8 62.0 2874.0 5.71 5.67 3.46
0.7 Premium G VVS2 61.8 58.0 2874.0 5.67 5.63 3.49
0.7 Ideal F VS1 61.0 55.0 2874.0 5.77 5.73 3.51
0.7 Ideal F VS1 61.6 55.0 2874.0 5.75 5.71 3.53
0.7 Ideal F VS1 62.4 56.0 2874.0 5.69 5.65 3.54
0.7 Premium G VVS2 62.9 59.0 2874.0 5.68 5.61 3.55
1.0 Fair H SI2 67.7 60.0 2875.0 6.11 5.98 4.09
0.77 Ideal H SI1 62.4 56.0 2875.0 5.84 5.9 3.66
1.0 Fair J VS1 65.5 55.0 2875.0 6.3 6.25 4.11
1.0 Fair I SI1 66.3 61.0 2875.0 6.08 6.03 4.01
1.0 Fair H SI2 69.5 55.0 2875.0 6.17 6.1 4.26
0.73 Premium E VS1 62.6 60.0 2876.0 5.68 5.75 3.58
0.79 Premium E VS2 60.6 53.0 2876.0 6.04 5.98 3.64
0.72 Very Good H VS1 62.2 54.0 2877.0 5.74 5.76 3.57
0.71 Ideal E VS1 62.4 56.0 2877.0 5.75 5.7 3.57
0.74 Ideal G VS2 62.3 55.0 2877.0 5.8 5.83 3.62
0.7 Good H VVS1 62.7 56.0 2877.0 5.6 5.66 3.53
0.7 Good F VS1 59.1 62.0 2877.0 5.82 5.86 3.44
0.79 Very Good F SI1 62.8 59.0 2878.0 5.86 5.89 3.69
0.79 Very Good F SI1 62.7 60.0 2878.0 5.82 5.89 3.67
0.79 Very Good D SI2 59.7 58.0 2878.0 6.0 6.07 3.6
0.71 Ideal I VS2 61.5 55.0 2878.0 5.76 5.78 3.55
0.79 Ideal F SI1 62.8 56.0 2878.0 5.88 5.9 3.7
0.73 Very Good F SI1 61.4 56.0 2879.0 5.81 5.86 3.58
0.63 Premium E IF 60.3 62.0 2879.0 5.55 5.53 3.34
0.7 Premium F VS1 60.4 60.0 2879.0 5.73 5.7 3.45
0.71 Premium F VS1 62.7 58.0 2879.0 5.71 5.67 3.57
0.84 Ideal G SI2 61.0 56.0 2879.0 6.13 6.1 3.73
0.84 Ideal G SI2 62.3 55.0 2879.0 6.08 6.03 3.77
1.02 Ideal J SI2 60.3 54.0 2879.0 6.53 6.5 3.93
0.72 Fair F VS1 56.9 69.0 2879.0 5.93 5.77 3.33
0.72 Ideal F VS1 62.0 56.0 2879.0 5.76 5.73 3.56
0.92 Very Good J SI2 58.7 61.0 2880.0 6.34 6.43 3.75
0.74 Very Good D SI1 63.9 57.0 2880.0 5.72 5.74 3.66
0.7 Ideal H VVS1 62.0 55.0 2881.0 5.74 5.71 3.55
0.71 Very Good E VS2 60.0 59.0 2881.0 5.84 5.83 3.5
1.05 Premium H I1 62.0 59.0 2881.0 6.5 6.47 4.02
0.7 Very Good H IF 62.8 56.0 2882.0 5.62 5.65 3.54
0.54 Ideal F VVS1 61.8 56.0 2882.0 5.23 5.26 3.24
0.73 Premium F VS2 59.9 58.0 2882.0 5.87 5.84 3.51
0.88 Fair F SI1 56.6 65.0 2882.0 6.39 6.32 3.6
0.73 Premium F VS2 58.7 57.0 2882.0 5.97 5.92 3.49
0.72 Ideal D SI1 61.8 56.0 2883.0 5.75 5.81 3.57
0.9 Good H SI2 62.7 64.0 2883.0 6.09 6.0 3.79
0.9 Fair H SI2 65.0 61.0 2883.0 6.01 5.96 3.89
1.03 Fair I SI2 65.3 55.0 2884.0 6.32 6.27 4.11
0.84 Very Good F SI1 63.8 57.0 2885.0 5.95 6.0 3.81
1.01 Premium I SI1 62.7 60.0 2885.0 6.36 6.27 3.96
0.77 Ideal D SI2 61.5 55.0 2885.0 5.9 5.93 3.64
0.8 Fair E SI1 56.3 63.0 2885.0 6.22 6.14 3.48
0.9 Fair D SI2 66.9 57.0 2885.0 6.02 5.9 3.99
0.73 Ideal E SI1 61.4 56.0 2886.0 5.79 5.81 3.56
0.72 Ideal E SI1 62.7 55.0 2886.0 5.64 5.69 3.55
0.71 Very Good D SI1 62.4 54.0 2887.0 5.71 5.79 3.59
0.7 Premium E VS1 62.6 59.0 2887.0 5.66 5.69 3.55
0.79 Ideal I VS1 61.7 59.0 2888.0 5.93 5.96 3.67
0.72 Very Good G VVS2 62.5 58.0 2889.0 5.68 5.72 3.56
0.7 Very Good E VS2 63.5 54.0 2889.0 5.62 5.66 3.58
0.7 Very Good F VS1 62.2 58.0 2889.0 5.64 5.75 3.54
0.9 Good H SI2 63.5 58.0 2889.0 6.09 6.14 3.88
0.71 Very Good F VS1 62.8 56.0 2889.0 5.69 5.72 3.58
0.5 Ideal E VVS2 62.2 54.0 2889.0 5.08 5.12 3.17
0.5 Ideal E VVS2 62.2 54.0 2889.0 5.09 5.11 3.17
0.74 Ideal F SI1 61.2 56.0 2889.0 5.83 5.87 3.58
0.77 Premium F VS2 61.8 56.0 2889.0 5.94 5.9 3.66
0.77 Premium E SI1 59.8 61.0 2889.0 5.99 5.95 3.57
0.8 Ideal F SI1 61.5 54.0 2890.0 6.07 6.0 3.71
0.8 Ideal F SI1 62.4 57.0 2890.0 5.9 5.87 3.67
0.8 Premium F SI1 61.5 60.0 2890.0 5.97 5.94 3.66
0.8 Good F SI1 63.8 59.0 2890.0 5.87 5.83 3.73
0.66 Ideal G VVS1 61.5 56.0 2890.0 5.61 5.58 3.44
0.71 Very Good E VS2 61.2 58.0 2891.0 5.71 5.79 3.52
0.71 Ideal F VS2 61.2 56.0 2891.0 5.73 5.77 3.52
0.71 Ideal E VS2 61.6 56.0 2891.0 5.74 5.76 3.54
0.71 Ideal E VS2 62.7 56.0 2891.0 5.71 5.75 3.59
0.72 Ideal D SI1 61.1 56.0 2891.0 5.78 5.81 3.54
0.71 Good D VS2 62.3 61.0 2891.0 5.7 5.73 3.56
0.86 Ideal H SI2 61.8 55.0 2892.0 6.12 6.14 3.79
1.19 Fair H I1 65.1 59.0 2892.0 6.62 6.55 4.29
0.71 Very Good F VS1 62.6 55.0 2893.0 5.66 5.71 3.56
0.82 Very Good G SI2 62.5 56.0 2893.0 5.99 6.04 3.76
0.71 Ideal G VVS2 61.5 57.0 2893.0 5.73 5.75 3.53
0.75 Ideal F VS2 62.5 57.0 2893.0 5.78 5.83 3.63
0.7 Very Good H VVS1 59.2 60.0 2893.0 5.87 5.78 3.45
0.8 Ideal G SI2 62.5 55.0 2893.0 5.89 5.92 3.69
0.82 Good G SI2 59.9 62.0 2893.0 6.02 6.04 3.61
0.82 Very Good G SI1 63.4 55.0 2893.0 6.0 5.93 3.78
0.82 Premium G SI1 59.9 59.0 2893.0 6.09 6.06 3.64
0.81 Very Good E SI2 62.4 57.0 2894.0 5.91 5.99 3.71
0.81 Ideal G SI2 62.2 57.0 2894.0 5.96 6.0 3.72
0.76 Ideal F SI1 61.4 56.0 2894.0 5.88 5.92 3.62
0.71 Very Good G VS2 60.9 56.0 2895.0 5.75 5.78 3.51
0.7 Very Good F VS1 61.8 59.0 2895.0 5.66 5.76 3.53
0.7 Ideal G VVS2 62.1 53.0 2895.0 5.71 5.75 3.56
0.74 Very Good G VS1 59.8 58.0 2896.0 5.85 5.89 3.51
0.77 Very Good G VS2 61.3 60.0 2896.0 5.81 5.91 3.59
0.77 Very Good G VS2 58.3 63.0 2896.0 6.0 6.05 3.51
0.53 Ideal F VVS1 61.6 56.0 2896.0 5.18 5.24 3.21
0.79 Ideal D SI1 61.5 56.0 2896.0 5.91 5.96 3.65
0.73 Ideal E SI2 61.5 55.0 2896.0 5.82 5.86 3.59
0.77 Ideal D SI2 62.1 56.0 2896.0 5.83 5.89 3.64
0.77 Premium E SI1 60.9 58.0 2896.0 5.94 5.88 3.6
1.01 Very Good I I1 63.1 57.0 2896.0 6.39 6.35 4.02
1.01 Ideal I I1 61.5 57.0 2896.0 6.46 6.45 3.97
0.6 Very Good D VVS2 60.6 57.0 2897.0 5.48 5.51 3.33
0.76 Premium E SI1 61.1 58.0 2897.0 5.91 5.85 3.59
0.54 Ideal D VVS2 61.4 52.0 2897.0 5.3 5.34 3.26
0.72 Ideal E SI1 62.5 55.0 2897.0 5.69 5.74 3.57
0.72 Good F VS1 59.4 61.0 2897.0 5.82 5.89 3.48
0.74 Premium D VS2 61.8 58.0 2897.0 5.81 5.77 3.58
1.12 Premium J SI2 60.6 59.0 2898.0 6.68 6.61 4.03

Note that columns of type string are not in the scatter plot!

diamondsDF.printSchema // Ctrl+Enter
root
 |-- carat: double (nullable = true)
 |-- cut: string (nullable = true)
 |-- color: string (nullable = true)
 |-- clarity: string (nullable = true)
 |-- depth: double (nullable = true)
 |-- table: double (nullable = true)
 |-- price: double (nullable = true)
 |-- x: double (nullable = true)
 |-- y: double (nullable = true)
 |-- z: double (nullable = true)

Let us run through some basic inteactive SQL queries next

  • HiveQL supports =, <, >, <=, >= and != operators. It also supports LIKE operator for fuzzy matching of Strings
  • Enclose Strings in single quotes
  • Multiple conditions can be combined using and and or
  • Enclose conditions in () for precedence
  • ...
  • ...

Why do I need to learn interactive SQL queries?

Such queries in the widely known declarative SQL language can help us explore the data and thereby inform the modeling process!!!

Using DataFrame API, we can apply a filter after select to transform the DataFrame diamondsDF to the new DataFrame diamondsDColoredDF.

Below, $ is an alias for column.

Let as select the columns named carat, colour, price where color value is equal to D.

val diamondsDColoredDF = diamondsDF.select("carat", "color", "price").filter($"color" === "D") // Shift+Enter
diamondsDColoredDF: org.apache.spark.sql.Dataset[org.apache.spark.sql.Row] = [carat: double, color: string ... 1 more field]
diamondsDColoredDF.show(10) // Ctrl+Enter
+-----+-----+-----+
|carat|color|price|
+-----+-----+-----+
| 0.23|    D|357.0|
| 0.23|    D|402.0|
| 0.26|    D|403.0|
| 0.26|    D|403.0|
| 0.26|    D|403.0|
| 0.22|    D|404.0|
|  0.3|    D|552.0|
|  0.3|    D|552.0|
|  0.3|    D|552.0|
| 0.24|    D|553.0|
+-----+-----+-----+
only showing top 10 rows

As you can see all the colors are now 'D'. But to really confirm this we can do the following for fun:

diamondsDColoredDF.select("color").distinct().show
+-----+
|color|
+-----+
|    D|
+-----+

Let's try to do the same in SQL for those who know SQL from before.

First we need to see if the table is registerd (not just the DataFrame), and if not we ened to register our DataFrame as a temporary table.

sqlContext.tables.show() // Ctrl+Enter to see available tables
+--------+--------------------+-----------+
|database|           tableName|isTemporary|
+--------+--------------------+-----------+
| default|          all_prices|      false|
| default|bitcoin_normed_wi...|      false|
| default|bitcoin_reversals...|      false|
| default|        countrycodes|      false|
| default|  gold_normed_window|      false|
| default|gold_reversals_wi...|      false|
| default|ltcar_locations_2...|      false|
| default|            magellan|      false|
| default|       mobile_sample|      false|
| default|   oil_normed_window|      false|
| default|oil_reversals_window|      false|
| default|oil_reversals_win...|      false|
| default|    over300all_2_txt|      false|
| default|              person|      false|
| default|            personer|      false|
| default|             persons|      false|
| default|        simple_range|      false|
| default|  social_media_usage|      false|
| default|social_media_usag...|      false|
| default|voronoi20191213up...|      false|
+--------+--------------------+-----------+
only showing top 20 rows

Looks like diamonds is already there (if not just execute the following cell).

diamondsDF.createOrReplaceTempView("diamonds")
sqlContext.tables.show() // Ctrl+Enter to see available tables
+--------+--------------------+-----------+
|database|           tableName|isTemporary|
+--------+--------------------+-----------+
| default|          all_prices|      false|
| default|bitcoin_normed_wi...|      false|
| default|bitcoin_reversals...|      false|
| default|        countrycodes|      false|
| default|  gold_normed_window|      false|
| default|gold_reversals_wi...|      false|
| default|ltcar_locations_2...|      false|
| default|            magellan|      false|
| default|       mobile_sample|      false|
| default|   oil_normed_window|      false|
| default|oil_reversals_window|      false|
| default|oil_reversals_win...|      false|
| default|    over300all_2_txt|      false|
| default|              person|      false|
| default|            personer|      false|
| default|             persons|      false|
| default|        simple_range|      false|
| default|  social_media_usage|      false|
| default|social_media_usag...|      false|
| default|voronoi20191213up...|      false|
+--------+--------------------+-----------+
only showing top 20 rows
-- Shift+Enter to do the same in SQL
select carat, color, price from diamonds where color='D'
carat color price
0.23 D 357.0
0.23 D 402.0
0.26 D 403.0
0.26 D 403.0
0.26 D 403.0
0.22 D 404.0
0.3 D 552.0
0.3 D 552.0
0.3 D 552.0
0.24 D 553.0
0.26 D 554.0
0.26 D 554.0
0.26 D 554.0
0.75 D 2760.0
0.71 D 2762.0
0.61 D 2763.0
0.71 D 2764.0
0.71 D 2764.0
0.7 D 2767.0
0.71 D 2767.0
0.73 D 2768.0
0.7 D 2768.0
0.71 D 2768.0
0.71 D 2770.0
0.76 D 2770.0
0.73 D 2770.0
0.75 D 2773.0
0.7 D 2773.0
0.7 D 2777.0
0.53 D 2782.0
0.75 D 2782.0
0.72 D 2782.0
0.72 D 2782.0
0.7 D 2782.0
0.64 D 2787.0
0.71 D 2788.0
0.72 D 2795.0
0.71 D 2797.0
0.71 D 2797.0
0.71 D 2797.0
0.51 D 2797.0
0.78 D 2799.0
0.91 D 2803.0
0.7 D 2804.0
0.7 D 2804.0
0.72 D 2804.0
0.72 D 2804.0
0.73 D 2808.0
0.81 D 2809.0
0.74 D 2810.0
0.83 D 2811.0
0.71 D 2812.0
0.55 D 2815.0
0.71 D 2816.0
0.73 D 2821.0
0.71 D 2822.0
0.71 D 2822.0
0.7 D 2822.0
0.7 D 2822.0
0.71 D 2822.0
0.7 D 2822.0
0.7 D 2822.0
0.7 D 2822.0
0.7 D 2822.0
0.79 D 2823.0
0.71 D 2824.0
0.7 D 2826.0
0.7 D 2827.0
0.72 D 2827.0
0.7 D 2828.0
0.7 D 2833.0
0.7 D 2833.0
0.51 D 2834.0
0.92 D 2840.0
0.71 D 2841.0
0.73 D 2841.0
0.73 D 2841.0
0.71 D 2843.0
0.79 D 2846.0
0.76 D 2847.0
0.54 D 2848.0
0.75 D 2848.0
0.66 D 2851.0
0.79 D 2853.0
0.79 D 2853.0
0.74 D 2855.0
0.73 D 2858.0
0.71 D 2858.0
0.71 D 2858.0
0.7 D 2859.0
0.7 D 2859.0
0.7 D 2859.0
0.71 D 2860.0
0.71 D 2861.0
0.66 D 2861.0
0.7 D 2862.0
0.8 D 2862.0
0.71 D 2863.0
0.71 D 2863.0
0.71 D 2863.0
0.71 D 2863.0
0.71 D 2863.0
0.71 D 2863.0
0.71 D 2863.0
0.73 D 2865.0
0.56 D 2866.0
0.56 D 2866.0
0.7 D 2867.0
1.08 D 2869.0
0.7 D 2872.0
0.75 D 2872.0
0.7 D 2872.0
0.7 D 2872.0
0.7 D 2872.0
0.7 D 2872.0
0.71 D 2874.0
0.79 D 2878.0
0.74 D 2880.0
0.72 D 2883.0
0.77 D 2885.0
0.9 D 2885.0
0.71 D 2887.0
0.72 D 2891.0
0.71 D 2891.0
0.79 D 2896.0
0.77 D 2896.0
0.6 D 2897.0
0.54 D 2897.0
0.74 D 2897.0
0.75 D 2898.0
0.77 D 2898.0
0.72 D 2900.0
0.75 D 2903.0
0.75 D 2903.0
0.72 D 2903.0
0.72 D 2903.0
0.79 D 2904.0
0.53 D 2905.0
0.74 D 2906.0
0.32 D 558.0
0.7 D 2909.0
0.7 D 2909.0
0.71 D 2910.0
0.7 D 2913.0
0.71 D 2913.0
0.71 D 2913.0
0.71 D 2913.0
0.71 D 2913.0
0.71 D 2913.0
0.83 D 2918.0
0.71 D 2921.0
0.77 D 2922.0
0.77 D 2923.0
0.8 D 2925.0
0.81 D 2926.0
0.7 D 2928.0
0.59 D 2933.0
0.75 D 2933.0
0.71 D 2934.0
0.7 D 2936.0
0.77 D 2939.0
0.76 D 2942.0
0.73 D 2943.0
0.57 D 2945.0
0.78 D 2945.0
0.73 D 2947.0
0.73 D 2947.0
0.77 D 2949.0
0.71 D 2950.0
0.72 D 2951.0
0.72 D 2954.0
0.72 D 2954.0
0.75 D 2954.0
0.82 D 2954.0
0.7 D 2956.0
0.56 D 2959.0
0.7 D 2960.0
0.7 D 2960.0
0.7 D 2960.0
0.63 D 2962.0
0.71 D 2964.0
0.71 D 2968.0
0.77 D 2973.0
1.0 D 2974.0
0.76 D 2977.0
0.7 D 2980.0
0.7 D 2985.0
0.74 D 2987.0
0.83 D 2990.0
0.7 D 2991.0
0.72 D 2993.0
0.81 D 2994.0
0.73 D 2995.0
0.77 D 2996.0
0.7 D 2998.0
0.7 D 2999.0
0.72 D 3001.0
0.7 D 3001.0
0.7 D 3001.0
0.7 D 3001.0
0.71 D 3002.0
1.01 D 3003.0
0.65 D 3003.0
0.92 D 3004.0
0.55 D 3006.0
0.76 D 3007.0
0.7 D 3008.0
0.8 D 3011.0
0.77 D 3011.0
0.9 D 3013.0
0.73 D 3014.0
0.72 D 3016.0
0.5 D 3017.0
0.78 D 3019.0
0.71 D 3020.0
0.75 D 3024.0
0.75 D 3024.0
0.65 D 3025.0
0.71 D 3033.0
0.7 D 3033.0
0.7 D 3033.0
0.7 D 3033.0
0.78 D 3035.0
0.71 D 3035.0
0.74 D 3036.0
0.61 D 3036.0
0.77 D 3040.0
0.71 D 3045.0
0.72 D 3045.0
0.75 D 3046.0
0.73 D 3047.0
0.75 D 3048.0
0.72 D 3048.0
0.72 D 3048.0
0.66 D 3049.0
0.62 D 3050.0
0.7 D 3052.0
0.7 D 3053.0
0.7 D 3054.0
0.65 D 3056.0
0.92 D 3057.0
0.79 D 3058.0
0.72 D 3062.0
0.85 D 3066.0
0.7 D 3073.0
0.72 D 3075.0
0.72 D 3075.0
0.7 D 3075.0
0.76 D 3075.0
0.71 D 3077.0
0.71 D 3077.0
0.75 D 3078.0
0.83 D 3078.0
0.91 D 3079.0
0.79 D 3081.0
0.7 D 3082.0
0.8 D 3082.0
0.71 D 3084.0
0.75 D 3085.0
0.7 D 3087.0
0.7 D 3087.0
0.7 D 3087.0
0.74 D 3087.0
0.71 D 3090.0
0.71 D 3090.0
0.7 D 3092.0
0.7 D 3092.0
0.7 D 3092.0
0.7 D 3093.0
0.71 D 3096.0
0.71 D 3096.0
0.53 D 3097.0
0.72 D 3099.0
0.72 D 3102.0
0.66 D 3103.0
0.78 D 3103.0
0.75 D 3105.0
0.7 D 3107.0
0.79 D 3112.0
0.94 D 3125.0
0.57 D 3126.0
0.57 D 3126.0
0.7 D 3129.0
0.7 D 3131.0
0.71 D 3131.0
0.71 D 3135.0
0.71 D 3135.0
0.8 D 3135.0
0.81 D 3135.0
0.71 D 3136.0
0.71 D 3137.0
0.74 D 3138.0
0.72 D 3139.0
0.54 D 3139.0
0.73 D 3140.0
0.71 D 3145.0
0.84 D 3145.0
0.78 D 3145.0
0.75 D 3152.0
0.9 D 3153.0
0.71 D 3153.0
0.58 D 3154.0
0.8 D 3154.0
0.77 D 3158.0
0.82 D 3159.0
0.77 D 3160.0
0.81 D 3160.0
0.71 D 3161.0
0.71 D 3161.0
0.71 D 3161.0
0.77 D 3166.0
0.8 D 3173.0
0.72 D 3176.0
0.74 D 3177.0
0.72 D 3179.0
0.72 D 3179.0
0.72 D 3179.0
0.81 D 3179.0
0.73 D 3182.0
0.73 D 3182.0
0.7 D 3183.0
0.79 D 3185.0
0.73 D 3189.0
0.73 D 3189.0
0.71 D 3192.0
0.7 D 3193.0
0.54 D 3194.0
0.73 D 3195.0
0.8 D 3195.0
0.7 D 3199.0
0.71 D 3203.0
0.9 D 3205.0
0.9 D 3205.0
0.9 D 3205.0
0.9 D 3205.0
0.9 D 3205.0
0.9 D 3205.0
0.9 D 3205.0
0.72 D 3205.0
0.58 D 3206.0
0.83 D 3207.0
0.7 D 3208.0
0.79 D 3209.0
0.8 D 3210.0
0.7 D 3210.0
0.71 D 3212.0
0.78 D 3214.0
0.7 D 3214.0
0.95 D 3214.0
0.71 D 3217.0
0.71 D 3217.0
0.71 D 3217.0
0.52 D 3218.0
0.72 D 3219.0
0.72 D 3219.0
0.71 D 3222.0
0.71 D 3222.0
0.51 D 3223.0
0.8 D 3226.0
0.65 D 3228.0
0.7 D 3229.0
0.7 D 3229.0
0.7 D 3231.0
0.59 D 3234.0
0.71 D 3234.0
0.72 D 3236.0
0.7 D 3239.0
0.7 D 3239.0
0.7 D 3239.0
0.77 D 3241.0
0.79 D 3242.0
0.71 D 3245.0
0.84 D 3246.0
0.25 D 563.0
0.26 D 564.0
0.31 D 565.0
0.31 D 565.0
0.7 D 3247.0
0.52 D 3247.0
0.76 D 3248.0
0.73 D 3250.0
0.77 D 3251.0
0.71 D 3252.0
0.78 D 3253.0
0.73 D 3255.0
0.78 D 3258.0
0.9 D 3262.0
0.71 D 3262.0
0.84 D 3265.0
0.81 D 3266.0
0.7 D 3267.0
0.56 D 3270.0
0.79 D 3270.0
0.72 D 3275.0
0.92 D 3277.0
0.7 D 3278.0
0.52 D 3284.0
0.86 D 3284.0
0.7 D 3287.0
0.7 D 3287.0
0.77 D 3291.0
0.76 D 3293.0
0.74 D 3294.0
0.7 D 3296.0
0.91 D 3298.0
0.78 D 3298.0
0.78 D 3298.0
0.71 D 3299.0
1.0 D 3304.0
1.0 D 3304.0
1.0 D 3304.0
0.76 D 3306.0
0.76 D 3306.0
0.53 D 3307.0
0.73 D 3308.0
0.77 D 3309.0
0.31 D 565.0
0.31 D 565.0
0.8 D 3312.0
0.7 D 3312.0
0.8 D 3312.0
0.9 D 3312.0
0.9 D 3312.0
0.7 D 3312.0
0.9 D 3312.0
0.71 D 3316.0
0.73 D 3319.0
0.52 D 3321.0
0.71 D 3321.0
0.71 D 3321.0
0.72 D 3322.0
0.81 D 3324.0
0.78 D 3326.0
0.79 D 3328.0
0.71 D 3332.0
0.71 D 3333.0
0.92 D 3335.0
0.7 D 3335.0
0.61 D 3336.0
1.01 D 3337.0
0.77 D 3345.0
0.53 D 3346.0
0.73 D 3346.0
0.83 D 3347.0
0.91 D 3349.0
0.77 D 3351.0
0.76 D 3352.0
0.74 D 3353.0
0.76 D 3353.0
0.81 D 3353.0
0.82 D 3357.0
0.91 D 3357.0
0.7 D 3360.0
0.7 D 3361.0
0.7 D 3365.0
0.74 D 3365.0
0.71 D 3366.0
0.69 D 3369.0
0.9 D 3371.0
0.9 D 3371.0
0.71 D 3372.0
0.52 D 3373.0
0.7 D 3375.0
0.72 D 3375.0
0.5 D 3378.0
0.5 D 3378.0
0.6 D 3382.0
0.27 D 567.0
0.31 D 567.0
0.33 D 567.0
0.33 D 567.0
0.33 D 567.0
0.3 D 568.0
0.9 D 3382.0
0.95 D 3384.0
0.76 D 3384.0
0.78 D 3389.0
0.88 D 3390.0
0.61 D 3397.0
0.85 D 3398.0
0.76 D 3401.0
0.91 D 3403.0
0.71 D 3406.0
0.71 D 3406.0
0.91 D 3408.0
0.7 D 3410.0
0.73 D 3411.0
0.73 D 3412.0
0.8 D 3419.0
0.7 D 3419.0
0.96 D 3419.0
0.96 D 3419.0
0.71 D 3420.0
0.9 D 3425.0
0.7 D 3425.0
0.77 D 3428.0
0.77 D 3428.0
0.77 D 3428.0
0.77 D 3428.0
0.79 D 3432.0
0.73 D 3440.0
0.8 D 3441.0
0.53 D 3442.0
0.77 D 3442.0
0.76 D 3443.0
0.76 D 3443.0
0.51 D 3446.0
0.51 D 3446.0
0.7 D 3448.0
0.72 D 3450.0
0.3 D 568.0
0.74 D 3454.0
0.78 D 3454.0
0.7 D 3454.0
0.75 D 3456.0
0.72 D 3459.0
0.74 D 3461.0
0.81 D 3462.0
0.91 D 3463.0
0.7 D 3463.0
0.73 D 3464.0
0.56 D 3465.0
0.71 D 3465.0
0.73 D 3467.0
0.55 D 3468.0
0.55 D 3468.0
0.55 D 3468.0
0.7 D 3471.0
0.7 D 3471.0
0.7 D 3471.0
0.9 D 3473.0
0.9 D 3473.0
0.9 D 3473.0
0.9 D 3473.0
0.9 D 3473.0
0.78 D 3473.0
0.74 D 3476.0
0.7 D 3477.0
0.71 D 3479.0
0.96 D 3480.0
0.74 D 3487.0
0.77 D 3489.0
0.77 D 3489.0
0.72 D 3493.0
0.54 D 3494.0
0.72 D 3495.0
0.56 D 3496.0
0.74 D 3498.0
0.7 D 3501.0
0.8 D 3502.0
0.71 D 3502.0
0.71 D 3502.0
0.71 D 3502.0
0.9 D 3505.0
0.55 D 3509.0
0.73 D 3509.0
0.91 D 3511.0
0.74 D 3517.0
0.53 D 3517.0
0.71 D 3518.0
0.72 D 3522.0
0.71 D 3524.0
0.73 D 3528.0
0.7 D 3529.0
0.32 D 571.0
0.31 D 571.0
0.31 D 571.0
0.31 D 571.0
0.31 D 571.0
0.31 D 571.0
0.31 D 571.0
0.31 D 571.0
0.31 D 571.0
0.31 D 571.0
0.31 D 571.0
0.31 D 571.0
0.31 D 571.0
0.31 D 571.0
0.31 D 571.0
0.31 D 571.0
0.31 D 571.0
0.31 D 571.0
0.9 D 3534.0
0.9 D 3534.0
0.9 D 3534.0
0.9 D 3534.0
0.78 D 3534.0
0.7 D 3535.0
0.93 D 3540.0
0.71 D 3540.0
0.72 D 3543.0
0.72 D 3550.0
0.92 D 3550.0
0.72 D 3554.0
0.83 D 3556.0
0.83 D 3556.0
0.73 D 3557.0
0.7 D 3561.0
0.75 D 3562.0
0.8 D 3564.0
0.9 D 3567.0
0.7 D 3567.0
0.9 D 3568.0
0.72 D 3568.0
1.0 D 3569.0
0.72 D 3570.0
0.6 D 3570.0
0.91 D 3573.0
0.71 D 3576.0
0.9 D 3578.0
0.9 D 3579.0
0.76 D 3581.0
0.71 D 3582.0
0.97 D 3585.0
1.11 D 3589.0
0.82 D 3593.0
0.78 D 3595.0
0.8 D 3597.0
0.72 D 3601.0
1.01 D 3604.0
0.9 D 3604.0
1.01 D 3605.0
0.79 D 3605.0
1.03 D 3607.0
0.31 D 571.0
0.31 D 571.0
0.31 D 571.0
0.31 D 571.0
0.31 D 571.0
0.31 D 571.0
0.31 D 571.0
0.31 D 571.0
0.31 D 571.0
0.31 D 571.0
0.31 D 571.0
0.31 D 571.0
0.31 D 571.0
0.31 D 571.0
0.31 D 571.0
0.31 D 571.0
0.31 D 571.0
0.31 D 571.0
0.31 D 571.0
0.31 D 571.0
0.31 D 571.0
0.31 D 571.0
0.31 D 571.0
0.31 D 571.0
0.31 D 571.0
0.92 D 3613.0
0.73 D 3615.0
0.7 D 3618.0
0.7 D 3618.0
0.71 D 3618.0
0.72 D 3619.0
0.73 D 3620.0
0.7 D 3622.0
0.7 D 3622.0
0.72 D 3622.0
0.72 D 3622.0
0.75 D 3625.0
0.61 D 3625.0
0.72 D 3629.0
0.9 D 3632.0
0.94 D 3634.0
1.0 D 3634.0
1.0 D 3634.0
1.0 D 3634.0
1.0 D 3634.0
0.9 D 3643.0
0.77 D 3643.0
1.16 D 3644.0
0.77 D 3644.0
1.11 D 3655.0
0.91 D 3660.0
0.87 D 3664.0
0.7 D 3668.0
0.78 D 3668.0
0.74 D 3668.0
0.85 D 3669.0
0.71 D 3670.0
1.01 D 3671.0
1.01 D 3671.0
0.78 D 3672.0
0.73 D 3673.0
0.71 D 3674.0
0.71 D 3674.0
1.03 D 3675.0
0.75 D 3679.0
0.31 D 571.0
0.31 D 571.0
0.31 D 571.0
0.8 D 3682.0
0.84 D 3685.0
0.9 D 3689.0
0.9 D 3689.0
0.9 D 3689.0
0.9 D 3689.0
0.9 D 3689.0
0.9 D 3689.0
0.71 D 3690.0
0.94 D 3691.0
0.75 D 3696.0
0.9 D 3706.0
0.92 D 3707.0
0.86 D 3709.0
1.16 D 3711.0
0.75 D 3712.0
0.71 D 3716.0
0.71 D 3718.0
0.77 D 3721.0
0.72 D 3722.0
0.91 D 3730.0
0.91 D 3730.0
0.91 D 3730.0
0.58 D 3732.0
0.76 D 3732.0
0.73 D 3735.0
0.78 D 3736.0
0.7 D 3737.0
0.9 D 3740.0
0.9 D 3740.0
0.9 D 3740.0
0.9 D 3740.0
0.58 D 3741.0
0.87 D 3742.0
1.09 D 3742.0
1.03 D 3743.0
1.03 D 3743.0
0.93 D 3744.0
0.74 D 3746.0
0.3 D 574.0
0.9 D 3751.0
0.7 D 3752.0
0.9 D 3755.0
0.9 D 3755.0
0.77 D 3755.0
0.61 D 3758.0
0.78 D 3763.0
0.91 D 3763.0
1.0 D 3767.0
1.02 D 3769.0
1.02 D 3773.0
0.83 D 3774.0
1.04 D 3780.0
1.04 D 3780.0
0.9 D 3780.0
1.04 D 3780.0
1.5 D 3780.0
0.91 D 3781.0
0.91 D 3781.0
0.77 D 3787.0
0.7 D 3788.0
0.9 D 3789.0
0.59 D 3791.0
0.91 D 3796.0
0.79 D 3798.0
0.9 D 3798.0
0.9 D 3798.0
0.9 D 3798.0
0.71 D 3799.0
0.78 D 3800.0
0.71 D 3801.0
0.9 D 3806.0
0.9 D 3806.0
0.9 D 3806.0
0.84 D 3809.0
0.78 D 3811.0
0.74 D 3812.0
0.53 D 3812.0
0.93 D 3812.0
0.9 D 3812.0
0.9 D 3812.0
0.9 D 3812.0
0.93 D 3812.0
0.74 D 3813.0
1.18 D 3816.0
0.84 D 3816.0
1.05 D 3816.0
0.79 D 3818.0
0.9 D 3818.0
0.3 D 574.0
0.3 D 574.0
0.3 D 574.0
0.3 D 574.0
0.3 D 574.0
0.3 D 574.0
0.3 D 574.0
0.3 D 574.0
0.3 D 574.0
0.3 D 574.0
0.3 D 574.0
0.3 D 574.0
0.85 D 3821.0
0.92 D 3823.0
0.53 D 3827.0
0.91 D 3828.0
0.63 D 3832.0
0.91 D 3837.0
0.77 D 3837.0
0.71 D 3838.0
1.02 D 3838.0
1.02 D 3839.0
0.93 D 3839.0
0.7 D 3840.0
1.02 D 3842.0
0.92 D 3843.0
0.9 D 3847.0
0.91 D 3848.0
0.91 D 3848.0
0.91 D 3848.0
0.6 D 3850.0
0.81 D 3852.0
0.91 D 3855.0
0.73 D 3856.0
0.71 D 3856.0
0.74 D 3858.0
0.94 D 3862.0
0.78 D 3864.0
1.17 D 3866.0
0.9 D 3871.0
1.01 D 3871.0
0.87 D 3873.0
0.92 D 3877.0
0.71 D 3877.0
0.9 D 3880.0
0.9 D 3880.0
0.9 D 3880.0
0.93 D 3880.0
1.13 D 3883.0
1.18 D 3886.0
0.91 D 3889.0
0.3 D 574.0
0.3 D 574.0
0.3 D 574.0
0.3 D 574.0
0.3 D 574.0
0.3 D 574.0
0.25 D 575.0
0.27 D 575.0
0.25 D 575.0
1.09 D 3890.0
0.92 D 3891.0
1.0 D 3894.0
0.76 D 3894.0
0.72 D 3896.0
1.18 D 3899.0
1.02 D 3909.0
1.02 D 3909.0
0.91 D 3910.0
0.91 D 3911.0
0.66 D 3915.0
0.92 D 3916.0
0.9 D 3918.0
0.7 D 3920.0
0.78 D 3923.0
0.9 D 3931.0
1.01 D 3932.0
0.83 D 3933.0
0.92 D 3936.0
0.73 D 3937.0
0.91 D 3943.0
0.9 D 3945.0
0.91 D 3949.0
1.14 D 3950.0
0.76 D 3950.0
0.71 D 3952.0
0.91 D 3958.0
1.01 D 3959.0
0.75 D 3961.0
1.09 D 3961.0
0.88 D 3962.0
1.0 D 3965.0
1.0 D 3965.0
1.0 D 3965.0
1.0 D 3965.0
1.0 D 3965.0
0.33 D 575.0
1.0 D 3965.0
0.77 D 3966.0
0.62 D 3968.0
1.02 D 3971.0
0.9 D 3975.0
0.9 D 3975.0
1.23 D 3977.0
0.77 D 3980.0
0.73 D 3980.0
0.83 D 3984.0
0.9 D 3989.0
0.96 D 3989.0
0.9 D 3990.0
0.93 D 3990.0
0.83 D 3990.0
0.92 D 3997.0
0.9 D 3997.0
0.9 D 3997.0
0.9 D 3997.0
0.9 D 3997.0
0.9 D 3997.0
0.7 D 4003.0
1.01 D 4004.0
0.75 D 4007.0
0.9 D 4007.0
0.9 D 4007.0
0.87 D 4012.0
0.71 D 4014.0
0.7 D 4022.0
0.65 D 4022.0
1.14 D 4022.0
0.56 D 4025.0
0.71 D 4029.0
0.71 D 4029.0
0.71 D 4029.0
0.71 D 4029.0
0.71 D 4029.0
0.57 D 4032.0
0.77 D 4037.0
0.77 D 4039.0
0.74 D 4040.0
0.91 D 4041.0
0.54 D 4042.0
1.02 D 4044.0
1.02 D 4044.0
1.02 D 4044.0
0.72 D 4047.0
1.23 D 4050.0
0.91 D 4051.0
0.91 D 4051.0
0.91 D 4051.0
0.96 D 4060.0
1.01 D 4064.0
1.0 D 4065.0
0.91 D 4067.0
0.9 D 4068.0
0.9 D 4068.0
0.9 D 4068.0
0.9 D 4068.0
0.9 D 4068.0
0.9 D 4068.0
1.12 D 4071.0
1.01 D 4072.0
0.9 D 4078.0
0.9 D 4078.0
0.9 D 4078.0
0.72 D 4082.0
0.72 D 4082.0
0.64 D 4084.0
0.92 D 4086.0
0.81 D 4087.0
0.7 D 4095.0
0.92 D 4096.0
0.92 D 4096.0
0.25 D 410.0
0.23 D 411.0
0.27 D 413.0
0.3 D 413.0
0.3 D 413.0
0.23 D 577.0
0.91 D 4107.0
0.91 D 4107.0
0.87 D 4108.0
0.91 D 4113.0
0.82 D 4113.0
0.9 D 4114.0
0.73 D 4116.0
0.9 D 4117.0
1.01 D 4118.0
0.9 D 4120.0
0.91 D 4123.0
0.91 D 4123.0
0.91 D 4123.0
1.04 D 4123.0
0.9 D 4128.0
0.9 D 4130.0
0.9 D 4133.0
0.73 D 4134.0
0.73 D 4134.0
0.82 D 4135.0
0.82 D 4135.0
1.12 D 4139.0
0.93 D 4140.0
0.93 D 4140.0
0.92 D 4150.0
0.76 D 4150.0
1.0 D 4155.0
1.06 D 4155.0
0.92 D 4158.0
0.92 D 4158.0
0.83 D 4159.0
0.59 D 4161.0
0.93 D 4165.0
0.91 D 4165.0
0.9 D 4167.0
0.92 D 4168.0
0.92 D 4168.0
1.19 D 4168.0
0.8 D 4170.0
0.6 D 4172.0
1.03 D 4177.0
0.9 D 4178.0

Alternatively, one could just write the SQL statement in scala to create a new DataFrame diamondsDColoredDF_FromTable from the table diamonds and display it, as follows:

val diamondsDColoredDF_FromTable = sqlContext.sql("select carat, color, price from diamonds where color='D'") // Shift+Enter
diamondsDColoredDF_FromTable: org.apache.spark.sql.DataFrame = [carat: double, color: string ... 1 more field]
// or if you like use upper case for SQL then this is equivalent
val diamondsDColoredDF_FromTable = sqlContext.sql("SELECT carat, color, price FROM diamonds WHERE color='D'") // Shift+Enter
diamondsDColoredDF_FromTable: org.apache.spark.sql.DataFrame = [carat: double, color: string ... 1 more field]
// from version 2.x onwards you can call from SparkSession, the pre-made spark in spark-shell or databricks notebook
val diamondsDColoredDF_FromTable = spark.sql("SELECT carat, color, price FROM diamonds WHERE color='D'") // Shift+Enter
diamondsDColoredDF_FromTable: org.apache.spark.sql.DataFrame = [carat: double, color: string ... 1 more field]
display(diamondsDColoredDF_FromTable) // Ctrl+Enter to see the same DF!
carat color price
0.23 D 357.0
0.23 D 402.0
0.26 D 403.0
0.26 D 403.0
0.26 D 403.0
0.22 D 404.0
0.3 D 552.0
0.3 D 552.0
0.3 D 552.0
0.24 D 553.0
0.26 D 554.0
0.26 D 554.0
0.26 D 554.0
0.75 D 2760.0
0.71 D 2762.0
0.61 D 2763.0
0.71 D 2764.0
0.71 D 2764.0
0.7 D 2767.0
0.71 D 2767.0
0.73 D 2768.0
0.7 D 2768.0
0.71 D 2768.0
0.71 D 2770.0
0.76 D 2770.0
0.73 D 2770.0
0.75 D 2773.0
0.7 D 2773.0
0.7 D 2777.0
0.53 D 2782.0
0.75 D 2782.0
0.72 D 2782.0
0.72 D 2782.0
0.7 D 2782.0
0.64 D 2787.0
0.71 D 2788.0
0.72 D 2795.0
0.71 D 2797.0
0.71 D 2797.0
0.71 D 2797.0
0.51 D 2797.0
0.78 D 2799.0
0.91 D 2803.0
0.7 D 2804.0
0.7 D 2804.0
0.72 D 2804.0
0.72 D 2804.0
0.73 D 2808.0
0.81 D 2809.0
0.74 D 2810.0
0.83 D 2811.0
0.71 D 2812.0
0.55 D 2815.0
0.71 D 2816.0
0.73 D 2821.0
0.71 D 2822.0
0.71 D 2822.0
0.7 D 2822.0
0.7 D 2822.0
0.71 D 2822.0
0.7 D 2822.0
0.7 D 2822.0
0.7 D 2822.0
0.7 D 2822.0
0.79 D 2823.0
0.71 D 2824.0
0.7 D 2826.0
0.7 D 2827.0
0.72 D 2827.0
0.7 D 2828.0
0.7 D 2833.0
0.7 D 2833.0
0.51 D 2834.0
0.92 D 2840.0
0.71 D 2841.0
0.73 D 2841.0
0.73 D 2841.0
0.71 D 2843.0
0.79 D 2846.0
0.76 D 2847.0
0.54 D 2848.0
0.75 D 2848.0
0.66 D 2851.0
0.79 D 2853.0
0.79 D 2853.0
0.74 D 2855.0
0.73 D 2858.0
0.71 D 2858.0
0.71 D 2858.0
0.7 D 2859.0
0.7 D 2859.0
0.7 D 2859.0
0.71 D 2860.0
0.71 D 2861.0
0.66 D 2861.0
0.7 D 2862.0
0.8 D 2862.0
0.71 D 2863.0
0.71 D 2863.0
0.71 D 2863.0
0.71 D 2863.0
0.71 D 2863.0
0.71 D 2863.0
0.71 D 2863.0
0.73 D 2865.0
0.56 D 2866.0
0.56 D 2866.0
0.7 D 2867.0
1.08 D 2869.0
0.7 D 2872.0
0.75 D 2872.0
0.7 D 2872.0
0.7 D 2872.0
0.7 D 2872.0
0.7 D 2872.0
0.71 D 2874.0
0.79 D 2878.0
0.74 D 2880.0
0.72 D 2883.0
0.77 D 2885.0
0.9 D 2885.0
0.71 D 2887.0
0.72 D 2891.0
0.71 D 2891.0
0.79 D 2896.0
0.77 D 2896.0
0.6 D 2897.0
0.54 D 2897.0
0.74 D 2897.0
0.75 D 2898.0
0.77 D 2898.0
0.72 D 2900.0
0.75 D 2903.0
0.75 D 2903.0
0.72 D 2903.0
0.72 D 2903.0
0.79 D 2904.0
0.53 D 2905.0
0.74 D 2906.0
0.32 D 558.0
0.7 D 2909.0
0.7 D 2909.0
0.71 D 2910.0
0.7 D 2913.0
0.71 D 2913.0
0.71 D 2913.0
0.71 D 2913.0
0.71 D 2913.0
0.71 D 2913.0
0.83 D 2918.0
0.71 D 2921.0
0.77 D 2922.0
0.77 D 2923.0
0.8 D 2925.0
0.81 D 2926.0
0.7 D 2928.0
0.59 D 2933.0
0.75 D 2933.0
0.71 D 2934.0
0.7 D 2936.0
0.77 D 2939.0
0.76 D 2942.0
0.73 D 2943.0
0.57 D 2945.0
0.78 D 2945.0
0.73 D 2947.0
0.73 D 2947.0
0.77 D 2949.0
0.71 D 2950.0
0.72 D 2951.0
0.72 D 2954.0
0.72 D 2954.0
0.75 D 2954.0
0.82 D 2954.0
0.7 D 2956.0
0.56 D 2959.0
0.7 D 2960.0
0.7 D 2960.0
0.7 D 2960.0
0.63 D 2962.0
0.71 D 2964.0
0.71 D 2968.0
0.77 D 2973.0
1.0 D 2974.0
0.76 D 2977.0
0.7 D 2980.0
0.7 D 2985.0
0.74 D 2987.0
0.83 D 2990.0
0.7 D 2991.0
0.72 D 2993.0
0.81 D 2994.0
0.73 D 2995.0
0.77 D 2996.0
0.7 D 2998.0
0.7 D 2999.0
0.72 D 3001.0
0.7 D 3001.0
0.7 D 3001.0
0.7 D 3001.0
0.71 D 3002.0
1.01 D 3003.0
0.65 D 3003.0
0.92 D 3004.0
0.55 D 3006.0
0.76 D 3007.0
0.7 D 3008.0
0.8 D 3011.0
0.77 D 3011.0
0.9 D 3013.0
0.73 D 3014.0
0.72 D 3016.0
0.5 D 3017.0
0.78 D 3019.0
0.71 D 3020.0
0.75 D 3024.0
0.75 D 3024.0
0.65 D 3025.0
0.71 D 3033.0
0.7 D 3033.0
0.7 D 3033.0
0.7 D 3033.0
0.78 D 3035.0
0.71 D 3035.0
0.74 D 3036.0
0.61 D 3036.0
0.77 D 3040.0
0.71 D 3045.0
0.72 D 3045.0
0.75 D 3046.0
0.73 D 3047.0
0.75 D 3048.0
0.72 D 3048.0
0.72 D 3048.0
0.66 D 3049.0
0.62 D 3050.0
0.7 D 3052.0
0.7 D 3053.0
0.7 D 3054.0
0.65 D 3056.0
0.92 D 3057.0
0.79 D 3058.0
0.72 D 3062.0
0.85 D 3066.0
0.7 D 3073.0
0.72 D 3075.0
0.72 D 3075.0
0.7 D 3075.0
0.76 D 3075.0
0.71 D 3077.0
0.71 D 3077.0
0.75 D 3078.0
0.83 D 3078.0
0.91 D 3079.0
0.79 D 3081.0
0.7 D 3082.0
0.8 D 3082.0
0.71 D 3084.0
0.75 D 3085.0
0.7 D 3087.0
0.7 D 3087.0
0.7 D 3087.0
0.74 D 3087.0
0.71 D 3090.0
0.71 D 3090.0
0.7 D 3092.0
0.7 D 3092.0
0.7 D 3092.0
0.7 D 3093.0
0.71 D 3096.0
0.71 D 3096.0
0.53 D 3097.0
0.72 D 3099.0
0.72 D 3102.0
0.66 D 3103.0
0.78 D 3103.0
0.75 D 3105.0
0.7 D 3107.0
0.79 D 3112.0
0.94 D 3125.0
0.57 D 3126.0
0.57 D 3126.0
0.7 D 3129.0
0.7 D 3131.0
0.71 D 3131.0
0.71 D 3135.0
0.71 D 3135.0
0.8 D 3135.0
0.81 D 3135.0
0.71 D 3136.0
0.71 D 3137.0
0.74 D 3138.0
0.72 D 3139.0
0.54 D 3139.0
0.73 D 3140.0
0.71 D 3145.0
0.84 D 3145.0
0.78 D 3145.0
0.75 D 3152.0
0.9 D 3153.0
0.71 D 3153.0
0.58 D 3154.0
0.8 D 3154.0
0.77 D 3158.0
0.82 D 3159.0
0.77 D 3160.0
0.81 D 3160.0
0.71 D 3161.0
0.71 D 3161.0
0.71 D 3161.0
0.77 D 3166.0
0.8 D 3173.0
0.72 D 3176.0
0.74 D 3177.0
0.72 D 3179.0
0.72 D 3179.0
0.72 D 3179.0
0.81 D 3179.0
0.73 D 3182.0
0.73 D 3182.0
0.7 D 3183.0
0.79 D 3185.0
0.73 D 3189.0
0.73 D 3189.0
0.71 D 3192.0
0.7 D 3193.0
0.54 D 3194.0
0.73 D 3195.0
0.8 D 3195.0
0.7 D 3199.0
0.71 D 3203.0
0.9 D 3205.0
0.9 D 3205.0
0.9 D 3205.0
0.9 D 3205.0
0.9 D 3205.0
0.9 D 3205.0
0.9 D 3205.0
0.72 D 3205.0
0.58 D 3206.0
0.83 D 3207.0
0.7 D 3208.0
0.79 D 3209.0
0.8 D 3210.0
0.7 D 3210.0
0.71 D 3212.0
0.78 D 3214.0
0.7 D 3214.0
0.95 D 3214.0
0.71 D 3217.0
0.71 D 3217.0
0.71 D 3217.0
0.52 D 3218.0
0.72 D 3219.0
0.72 D 3219.0
0.71 D 3222.0
0.71 D 3222.0
0.51 D 3223.0
0.8 D 3226.0
0.65 D 3228.0
0.7 D 3229.0
0.7 D 3229.0
0.7 D 3231.0
0.59 D 3234.0
0.71 D 3234.0
0.72 D 3236.0
0.7 D 3239.0
0.7 D 3239.0
0.7 D 3239.0
0.77 D 3241.0
0.79 D 3242.0
0.71 D 3245.0
0.84 D 3246.0
0.25 D 563.0
0.26 D 564.0
0.31 D 565.0
0.31 D 565.0
0.7 D 3247.0
0.52 D 3247.0
0.76 D 3248.0
0.73 D 3250.0
0.77 D 3251.0
0.71 D 3252.0
0.78 D 3253.0
0.73 D 3255.0
0.78 D 3258.0
0.9 D 3262.0
0.71 D 3262.0
0.84 D 3265.0
0.81 D 3266.0
0.7 D 3267.0
0.56 D 3270.0
0.79 D 3270.0
0.72 D 3275.0
0.92 D 3277.0
0.7 D 3278.0
0.52 D 3284.0
0.86 D 3284.0
0.7 D 3287.0
0.7 D 3287.0
0.77 D 3291.0
0.76 D 3293.0
0.74 D 3294.0
0.7 D 3296.0
0.91 D 3298.0
0.78 D 3298.0
0.78 D 3298.0
0.71 D 3299.0
1.0 D 3304.0
1.0 D 3304.0
1.0 D 3304.0
0.76 D 3306.0
0.76 D 3306.0
0.53 D 3307.0
0.73 D 3308.0
0.77 D 3309.0
0.31 D 565.0
0.31 D 565.0
0.8 D 3312.0
0.7 D 3312.0
0.8 D 3312.0
0.9 D 3312.0
0.9 D 3312.0
0.7 D 3312.0
0.9 D 3312.0
0.71 D 3316.0
0.73 D 3319.0
0.52 D 3321.0
0.71 D 3321.0
0.71 D 3321.0
0.72 D 3322.0
0.81 D 3324.0
0.78 D 3326.0
0.79 D 3328.0
0.71 D 3332.0
0.71 D 3333.0
0.92 D 3335.0
0.7 D 3335.0
0.61 D 3336.0
1.01 D 3337.0
0.77 D 3345.0
0.53 D 3346.0
0.73 D 3346.0
0.83 D 3347.0
0.91 D 3349.0
0.77 D 3351.0
0.76 D 3352.0
0.74 D 3353.0
0.76 D 3353.0
0.81 D 3353.0
0.82 D 3357.0
0.91 D 3357.0
0.7 D 3360.0
0.7 D 3361.0
0.7 D 3365.0
0.74 D 3365.0
0.71 D 3366.0
0.69 D 3369.0
0.9 D 3371.0
0.9 D 3371.0
0.71 D 3372.0
0.52 D 3373.0
0.7 D 3375.0
0.72 D 3375.0
0.5 D 3378.0
0.5 D 3378.0
0.6 D 3382.0
0.27 D 567.0
0.31 D 567.0
0.33 D 567.0
0.33 D 567.0
0.33 D 567.0
0.3 D 568.0
0.9 D 3382.0
0.95 D 3384.0
0.76 D 3384.0
0.78 D 3389.0
0.88 D 3390.0
0.61 D 3397.0
0.85 D 3398.0
0.76 D 3401.0
0.91 D 3403.0
0.71 D 3406.0
0.71 D 3406.0
0.91 D 3408.0
0.7 D 3410.0
0.73 D 3411.0
0.73 D 3412.0
0.8 D 3419.0
0.7 D 3419.0
0.96 D 3419.0
0.96 D 3419.0
0.71 D 3420.0
0.9 D 3425.0
0.7 D 3425.0
0.77 D 3428.0
0.77 D 3428.0
0.77 D 3428.0
0.77 D 3428.0
0.79 D 3432.0
0.73 D 3440.0
0.8 D 3441.0
0.53 D 3442.0
0.77 D 3442.0
0.76 D 3443.0
0.76 D 3443.0
0.51 D 3446.0
0.51 D 3446.0
0.7 D 3448.0
0.72 D 3450.0
0.3 D 568.0
0.74 D 3454.0
0.78 D 3454.0
0.7 D 3454.0
0.75 D 3456.0
0.72 D 3459.0
0.74 D 3461.0
0.81 D 3462.0
0.91 D 3463.0
0.7 D 3463.0
0.73 D 3464.0
0.56 D 3465.0
0.71 D 3465.0
0.73 D 3467.0
0.55 D 3468.0
0.55 D 3468.0
0.55 D 3468.0
0.7 D 3471.0
0.7 D 3471.0
0.7 D 3471.0
0.9 D 3473.0
0.9 D 3473.0
0.9 D 3473.0
0.9 D 3473.0
0.9 D 3473.0
0.78 D 3473.0
0.74 D 3476.0
0.7 D 3477.0
0.71 D 3479.0
0.96 D 3480.0
0.74 D 3487.0
0.77 D 3489.0
0.77 D 3489.0
0.72 D 3493.0
0.54 D 3494.0
0.72 D 3495.0
0.56 D 3496.0
0.74 D 3498.0
0.7 D 3501.0
0.8 D 3502.0
0.71 D 3502.0
0.71 D 3502.0
0.71 D 3502.0
0.9 D 3505.0
0.55 D 3509.0
0.73 D 3509.0
0.91 D 3511.0
0.74 D 3517.0
0.53 D 3517.0
0.71 D 3518.0
0.72 D 3522.0
0.71 D 3524.0
0.73 D 3528.0
0.7 D 3529.0
0.32 D 571.0
0.31 D 571.0
0.31 D 571.0
0.31 D 571.0
0.31 D 571.0
0.31 D 571.0
0.31 D 571.0
0.31 D 571.0
0.31 D 571.0
0.31 D 571.0
0.31 D 571.0
0.31 D 571.0
0.31 D 571.0
0.31 D 571.0
0.31 D 571.0
0.31 D 571.0
0.31 D 571.0
0.31 D 571.0
0.9 D 3534.0
0.9 D 3534.0
0.9 D 3534.0
0.9 D 3534.0
0.78 D 3534.0
0.7 D 3535.0
0.93 D 3540.0
0.71 D 3540.0
0.72 D 3543.0
0.72 D 3550.0
0.92 D 3550.0
0.72 D 3554.0
0.83 D 3556.0
0.83 D 3556.0
0.73 D 3557.0
0.7 D 3561.0
0.75 D 3562.0
0.8 D 3564.0
0.9 D 3567.0
0.7 D 3567.0
0.9 D 3568.0
0.72 D 3568.0
1.0 D 3569.0
0.72 D 3570.0
0.6 D 3570.0
0.91 D 3573.0
0.71 D 3576.0
0.9 D 3578.0
0.9 D 3579.0
0.76 D 3581.0
0.71 D 3582.0
0.97 D 3585.0
1.11 D 3589.0
0.82 D 3593.0
0.78 D 3595.0
0.8 D 3597.0
0.72 D 3601.0
1.01 D 3604.0
0.9 D 3604.0
1.01 D 3605.0
0.79 D 3605.0
1.03 D 3607.0
0.31 D 571.0
0.31 D 571.0
0.31 D 571.0
0.31 D 571.0
0.31 D 571.0
0.31 D 571.0
0.31 D 571.0
0.31 D 571.0
0.31 D 571.0
0.31 D 571.0
0.31 D 571.0
0.31 D 571.0
0.31 D 571.0
0.31 D 571.0
0.31 D 571.0
0.31 D 571.0
0.31 D 571.0
0.31 D 571.0
0.31 D 571.0
0.31 D 571.0
0.31 D 571.0
0.31 D 571.0
0.31 D 571.0
0.31 D 571.0
0.31 D 571.0
0.92 D 3613.0
0.73 D 3615.0
0.7 D 3618.0
0.7 D 3618.0
0.71 D 3618.0
0.72 D 3619.0
0.73 D 3620.0
0.7 D 3622.0
0.7 D 3622.0
0.72 D 3622.0
0.72 D 3622.0
0.75 D 3625.0
0.61 D 3625.0
0.72 D 3629.0
0.9 D 3632.0
0.94 D 3634.0
1.0 D 3634.0
1.0 D 3634.0
1.0 D 3634.0
1.0 D 3634.0
0.9 D 3643.0
0.77 D 3643.0
1.16 D 3644.0
0.77 D 3644.0
1.11 D 3655.0
0.91 D 3660.0
0.87 D 3664.0
0.7 D 3668.0
0.78 D 3668.0
0.74 D 3668.0
0.85 D 3669.0
0.71 D 3670.0
1.01 D 3671.0
1.01 D 3671.0
0.78 D 3672.0
0.73 D 3673.0
0.71 D 3674.0
0.71 D 3674.0
1.03 D 3675.0
0.75 D 3679.0
0.31 D 571.0
0.31 D 571.0
0.31 D 571.0
0.8 D 3682.0
0.84 D 3685.0
0.9 D 3689.0
0.9 D 3689.0
0.9 D 3689.0
0.9 D 3689.0
0.9 D 3689.0
0.9 D 3689.0
0.71 D 3690.0
0.94 D 3691.0
0.75 D 3696.0
0.9 D 3706.0
0.92 D 3707.0
0.86 D 3709.0
1.16 D 3711.0
0.75 D 3712.0
0.71 D 3716.0
0.71 D 3718.0
0.77 D 3721.0
0.72 D 3722.0
0.91 D 3730.0
0.91 D 3730.0
0.91 D 3730.0
0.58 D 3732.0
0.76 D 3732.0
0.73 D 3735.0
0.78 D 3736.0
0.7 D 3737.0
0.9 D 3740.0
0.9 D 3740.0
0.9 D 3740.0
0.9 D 3740.0
0.58 D 3741.0
0.87 D 3742.0
1.09 D 3742.0
1.03 D 3743.0
1.03 D 3743.0
0.93 D 3744.0
0.74 D 3746.0
0.3 D 574.0
0.9 D 3751.0
0.7 D 3752.0
0.9 D 3755.0
0.9 D 3755.0
0.77 D 3755.0
0.61 D 3758.0
0.78 D 3763.0
0.91 D 3763.0
1.0 D 3767.0
1.02 D 3769.0
1.02 D 3773.0
0.83 D 3774.0
1.04 D 3780.0
1.04 D 3780.0
0.9 D 3780.0
1.04 D 3780.0
1.5 D 3780.0
0.91 D 3781.0
0.91 D 3781.0
0.77 D 3787.0
0.7 D 3788.0
0.9 D 3789.0
0.59 D 3791.0
0.91 D 3796.0
0.79 D 3798.0
0.9 D 3798.0
0.9 D 3798.0
0.9 D 3798.0
0.71 D 3799.0
0.78 D 3800.0
0.71 D 3801.0
0.9 D 3806.0
0.9 D 3806.0
0.9 D 3806.0
0.84 D 3809.0
0.78 D 3811.0
0.74 D 3812.0
0.53 D 3812.0
0.93 D 3812.0
0.9 D 3812.0
0.9 D 3812.0
0.9 D 3812.0
0.93 D 3812.0
0.74 D 3813.0
1.18 D 3816.0
0.84 D 3816.0
1.05 D 3816.0
0.79 D 3818.0
0.9 D 3818.0
0.3 D 574.0
0.3 D 574.0
0.3 D 574.0
0.3 D 574.0
0.3 D 574.0
0.3 D 574.0
0.3 D 574.0
0.3 D 574.0
0.3 D 574.0
0.3 D 574.0
0.3 D 574.0
0.3 D 574.0
0.85 D 3821.0
0.92 D 3823.0
0.53 D 3827.0
0.91 D 3828.0
0.63 D 3832.0
0.91 D 3837.0
0.77 D 3837.0
0.71 D 3838.0
1.02 D 3838.0
1.02 D 3839.0
0.93 D 3839.0
0.7 D 3840.0
1.02 D 3842.0
0.92 D 3843.0
0.9 D 3847.0
0.91 D 3848.0
0.91 D 3848.0
0.91 D 3848.0
0.6 D 3850.0
0.81 D 3852.0
0.91 D 3855.0
0.73 D 3856.0
0.71 D 3856.0
0.74 D 3858.0
0.94 D 3862.0
0.78 D 3864.0
1.17 D 3866.0
0.9 D 3871.0
1.01 D 3871.0
0.87 D 3873.0
0.92 D 3877.0
0.71 D 3877.0
0.9 D 3880.0
0.9 D 3880.0
0.9 D 3880.0
0.93 D 3880.0
1.13 D 3883.0
1.18 D 3886.0
0.91 D 3889.0
0.3 D 574.0
0.3 D 574.0
0.3 D 574.0
0.3 D 574.0
0.3 D 574.0
0.3 D 574.0
0.25 D 575.0
0.27 D 575.0
0.25 D 575.0
1.09 D 3890.0
0.92 D 3891.0
1.0 D 3894.0
0.76 D 3894.0
0.72 D 3896.0
1.18 D 3899.0
1.02 D 3909.0
1.02 D 3909.0
0.91 D 3910.0
0.91 D 3911.0
0.66 D 3915.0
0.92 D 3916.0
0.9 D 3918.0
0.7 D 3920.0
0.78 D 3923.0
0.9 D 3931.0
1.01 D 3932.0
0.83 D 3933.0
0.92 D 3936.0
0.73 D 3937.0
0.91 D 3943.0
0.9 D 3945.0
0.91 D 3949.0
1.14 D 3950.0
0.76 D 3950.0
0.71 D 3952.0
0.91 D 3958.0
1.01 D 3959.0
0.75 D 3961.0
1.09 D 3961.0
0.88 D 3962.0
1.0 D 3965.0
1.0 D 3965.0
1.0 D 3965.0
1.0 D 3965.0
1.0 D 3965.0
0.33 D 575.0
1.0 D 3965.0
0.77 D 3966.0
0.62 D 3968.0
1.02 D 3971.0
0.9 D 3975.0
0.9 D 3975.0
1.23 D 3977.0
0.77 D 3980.0
0.73 D 3980.0
0.83 D 3984.0
0.9 D 3989.0
0.96 D 3989.0
0.9 D 3990.0
0.93 D 3990.0
0.83 D 3990.0
0.92 D 3997.0
0.9 D 3997.0
0.9 D 3997.0
0.9 D 3997.0
0.9 D 3997.0
0.9 D 3997.0
0.7 D 4003.0
1.01 D 4004.0
0.75 D 4007.0
0.9 D 4007.0
0.9 D 4007.0
0.87 D 4012.0
0.71 D 4014.0
0.7 D 4022.0
0.65 D 4022.0
1.14 D 4022.0
0.56 D 4025.0
0.71 D 4029.0
0.71 D 4029.0
0.71 D 4029.0
0.71 D 4029.0
0.71 D 4029.0
0.57 D 4032.0
0.77 D 4037.0
0.77 D 4039.0
0.74 D 4040.0
0.91 D 4041.0
0.54 D 4042.0
1.02 D 4044.0
1.02 D 4044.0
1.02 D 4044.0
0.72 D 4047.0
1.23 D 4050.0
0.91 D 4051.0
0.91 D 4051.0
0.91 D 4051.0
0.96 D 4060.0
1.01 D 4064.0
1.0 D 4065.0
0.91 D 4067.0
0.9 D 4068.0
0.9 D 4068.0
0.9 D 4068.0
0.9 D 4068.0
0.9 D 4068.0
0.9 D 4068.0
1.12 D 4071.0
1.01 D 4072.0
0.9 D 4078.0
0.9 D 4078.0
0.9 D 4078.0
0.72 D 4082.0
0.72 D 4082.0
0.64 D 4084.0
0.92 D 4086.0
0.81 D 4087.0
0.7 D 4095.0
0.92 D 4096.0
0.92 D 4096.0
0.25 D 410.0
0.23 D 411.0
0.27 D 413.0
0.3 D 413.0
0.3 D 413.0
0.23 D 577.0
0.91 D 4107.0
0.91 D 4107.0
0.87 D 4108.0
0.91 D 4113.0
0.82 D 4113.0
0.9 D 4114.0
0.73 D 4116.0
0.9 D 4117.0
1.01 D 4118.0
0.9 D 4120.0
0.91 D 4123.0
0.91 D 4123.0
0.91 D 4123.0
1.04 D 4123.0
0.9 D 4128.0
0.9 D 4130.0
0.9 D 4133.0
0.73 D 4134.0
0.73 D 4134.0
0.82 D 4135.0
0.82 D 4135.0
1.12 D 4139.0
0.93 D 4140.0
0.93 D 4140.0
0.92 D 4150.0
0.76 D 4150.0
1.0 D 4155.0
1.06 D 4155.0
0.92 D 4158.0
0.92 D 4158.0
0.83 D 4159.0
0.59 D 4161.0
0.93 D 4165.0
0.91 D 4165.0
0.9 D 4167.0
0.92 D 4168.0
0.92 D 4168.0
1.19 D 4168.0
0.8 D 4170.0
0.6 D 4172.0
1.03 D 4177.0
0.9 D 4178.0
// You can also use the familiar wildchard character '%' when matching Strings
display(spark.sql("SELECT * FROM diamonds WHERE clarity LIKE 'V%'"))
carat cut color clarity depth table price x y z
0.23 Good E VS1 56.9 65.0 327.0 4.05 4.07 2.31
0.29 Premium I VS2 62.4 58.0 334.0 4.2 4.23 2.63
0.24 Very Good J VVS2 62.8 57.0 336.0 3.94 3.96 2.48
0.24 Very Good I VVS1 62.3 57.0 336.0 3.95 3.98 2.47
0.22 Fair E VS2 65.1 61.0 337.0 3.87 3.78 2.49
0.23 Very Good H VS1 59.4 61.0 338.0 4.0 4.05 2.39
0.23 Ideal J VS1 62.8 56.0 340.0 3.93 3.9 2.46
0.23 Very Good E VS2 63.8 55.0 352.0 3.85 3.92 2.48
0.23 Very Good H VS1 61.0 57.0 353.0 3.94 3.96 2.41
0.23 Very Good G VVS2 60.4 58.0 354.0 3.97 4.01 2.41
0.24 Premium I VS1 62.5 57.0 355.0 3.97 3.94 2.47
0.3 Very Good J VS2 62.2 57.0 357.0 4.28 4.3 2.67
0.23 Very Good D VS2 60.5 61.0 357.0 3.96 3.97 2.4
0.23 Very Good F VS1 60.9 57.0 357.0 3.96 3.99 2.42
0.23 Very Good F VS1 60.0 57.0 402.0 4.0 4.03 2.41
0.23 Very Good F VS1 59.8 57.0 402.0 4.04 4.06 2.42
0.23 Very Good E VS1 60.7 59.0 402.0 3.97 4.01 2.42
0.23 Very Good E VS1 59.5 58.0 402.0 4.01 4.06 2.4
0.23 Very Good D VS1 61.9 58.0 402.0 3.92 3.96 2.44
0.23 Good F VS1 58.2 59.0 402.0 4.06 4.08 2.37
0.23 Good E VS1 64.1 59.0 402.0 3.83 3.85 2.46
0.26 Very Good D VS2 60.8 59.0 403.0 4.13 4.16 2.52
0.26 Good D VS2 65.2 56.0 403.0 3.99 4.02 2.61
0.26 Good D VS1 58.4 63.0 403.0 4.19 4.24 2.46
0.25 Very Good E VS2 63.3 60.0 404.0 4.0 4.03 2.54
0.23 Ideal G VS1 61.9 54.0 404.0 3.93 3.95 2.44
0.22 Premium E VS2 61.6 58.0 404.0 3.93 3.89 2.41
0.22 Premium D VS2 59.3 62.0 404.0 3.91 3.88 2.31
0.35 Ideal I VS1 60.9 57.0 552.0 4.54 4.59 2.78
0.28 Ideal G VVS2 61.4 56.0 553.0 4.19 4.22 2.58
0.32 Ideal I VVS1 62.0 55.3 553.0 4.39 4.42 2.73
0.24 Premium E VVS1 60.7 58.0 553.0 4.01 4.03 2.44
0.24 Very Good D VVS1 61.5 60.0 553.0 3.97 4.0 2.45
0.26 Very Good F VVS2 59.2 60.0 554.0 4.19 4.22 2.49
0.26 Very Good E VVS2 59.9 58.0 554.0 4.15 4.23 2.51
0.26 Very Good D VVS2 62.4 54.0 554.0 4.08 4.13 2.56
0.26 Very Good D VVS2 62.8 60.0 554.0 4.01 4.05 2.53
0.26 Very Good E VVS1 62.6 59.0 554.0 4.06 4.09 2.55
0.26 Very Good E VVS1 63.4 59.0 554.0 4.0 4.04 2.55
0.26 Very Good D VVS1 62.1 60.0 554.0 4.03 4.12 2.53
0.26 Ideal E VVS2 62.9 58.0 554.0 4.02 4.06 2.54
0.26 Good E VVS1 57.9 60.0 554.0 4.22 4.25 2.45
0.24 Premium G VVS1 62.3 59.0 554.0 3.95 3.92 2.45
0.24 Premium H VVS1 61.2 58.0 554.0 4.01 3.96 2.44
0.24 Premium H VVS1 60.8 59.0 554.0 4.02 4.0 2.44
0.24 Premium H VVS2 60.7 58.0 554.0 4.07 4.04 2.46
0.7 Ideal G VS2 61.6 56.0 2757.0 5.7 5.67 3.5
0.71 Very Good E VS2 62.4 57.0 2759.0 5.68 5.73 3.56
0.7 Good E VS2 57.5 58.0 2759.0 5.85 5.9 3.38
0.7 Good F VS1 59.4 62.0 2759.0 5.71 5.76 3.4
0.75 Premium G VS2 61.7 58.0 2760.0 5.85 5.79 3.59
0.8 Ideal I VS1 62.9 56.0 2760.0 5.94 5.87 3.72
0.74 Ideal I VVS2 62.3 55.0 2761.0 5.77 5.81 3.61
0.59 Ideal E VVS2 62.0 55.0 2761.0 5.38 5.43 3.35
0.9 Premium I VS2 63.0 58.0 2761.0 6.16 6.12 3.87
0.73 Ideal F VS2 62.6 56.0 2762.0 5.77 5.74 3.6
0.73 Ideal F VS2 62.7 53.0 2762.0 5.8 5.75 3.62
0.71 Ideal G VS2 62.4 54.0 2762.0 5.72 5.76 3.58
0.7 Ideal E VS2 60.7 58.0 2762.0 5.73 5.76 3.49
0.7 Very Good F VS2 61.7 63.0 2762.0 5.64 5.61 3.47
0.7 Fair F VS2 64.5 57.0 2762.0 5.57 5.53 3.58
0.7 Fair F VS2 65.3 55.0 2762.0 5.63 5.58 3.66
0.7 Premium F VS2 61.6 60.0 2762.0 5.65 5.59 3.46
0.61 Very Good D VVS2 59.6 57.0 2763.0 5.56 5.58 3.32
0.77 Ideal H VS2 62.0 56.0 2763.0 5.89 5.86 3.64
0.7 Very Good E VS2 62.6 60.0 2765.0 5.62 5.65 3.53
0.77 Very Good H VS1 61.3 60.0 2765.0 5.88 5.9 3.61
0.63 Premium E VVS1 60.9 60.0 2765.0 5.52 5.55 3.37
0.71 Very Good F VS1 60.1 62.0 2765.0 5.74 5.77 3.46
0.71 Premium F VS1 61.8 59.0 2765.0 5.69 5.73 3.53
0.64 Ideal G VVS1 61.9 56.0 2766.0 5.53 5.56 3.43
0.71 Premium G VS2 60.9 57.0 2766.0 5.78 5.75 3.51
0.71 Premium G VS2 59.8 56.0 2766.0 5.89 5.81 3.5
0.7 Very Good D VS2 61.8 55.0 2767.0 5.68 5.72 3.52
0.7 Very Good F VS1 60.0 57.0 2767.0 5.8 5.87 3.5
0.7 Good H VVS2 62.1 64.0 2767.0 5.62 5.65 3.5
0.71 Very Good G VS1 63.3 59.0 2768.0 5.52 5.61 3.52
0.71 Premium D VS2 62.5 60.0 2770.0 5.65 5.61 3.52
0.73 Premium G VS2 61.4 59.0 2770.0 5.83 5.76 3.56
0.73 Premium G VS2 60.7 58.0 2770.0 5.87 5.82 3.55
0.73 Premium G VS1 61.5 58.0 2770.0 5.79 5.75 3.55
0.73 Premium G VS2 59.2 59.0 2770.0 5.92 5.87 3.49
0.72 Very Good H VVS2 60.3 56.0 2771.0 5.81 5.83 3.51
0.71 Ideal G VS2 61.9 57.0 2771.0 5.73 5.77 3.56
0.73 Very Good H VVS1 60.4 59.0 2772.0 5.83 5.89 3.54
0.58 Ideal G VVS1 61.5 55.0 2772.0 5.39 5.44 3.33
0.58 Ideal F VVS1 61.7 56.0 2772.0 5.33 5.37 3.3
0.71 Good E VS2 59.2 61.0 2772.0 5.8 5.88 3.46
0.7 Premium D VS2 58.0 62.0 2773.0 5.87 5.78 3.38
0.6 Ideal E VS1 61.7 55.0 2774.0 5.41 5.44 3.35
0.83 Good I VS2 64.6 54.0 2774.0 5.85 5.88 3.79
0.74 Very Good F VS2 61.3 61.0 2775.0 5.8 5.84 3.57
0.72 Very Good G VS2 63.7 56.4 2776.0 5.62 5.69 3.61
0.71 Premium E VS2 62.7 58.0 2776.0 5.74 5.68 3.58
0.71 Ideal E VS2 62.2 57.0 2776.0 5.79 5.62 3.55
0.54 Ideal E VVS2 61.6 56.0 2776.0 5.25 5.27 3.24
0.54 Ideal E VVS2 61.5 57.0 2776.0 5.24 5.26 3.23
0.72 Good G VS2 59.7 60.5 2776.0 5.8 5.84 3.47
0.7 Very Good D VS1 62.7 58.0 2777.0 5.66 5.73 3.57
0.71 Premium F VS2 62.1 58.0 2777.0 5.67 5.7 3.53
0.71 Very Good F VS2 62.8 57.0 2777.0 5.64 5.69 3.56
0.71 Good F VS2 63.8 58.0 2777.0 5.61 5.64 3.59
0.71 Good F VS2 57.8 60.0 2777.0 5.87 5.9 3.4
0.7 Ideal E VS2 62.1 55.0 2777.0 5.7 5.67 3.53
0.7 Premium E VS2 61.1 60.0 2777.0 5.71 5.64 3.47
0.7 Good E VS2 64.1 59.0 2777.0 5.64 5.59 3.6
0.52 Ideal F VVS1 61.3 55.0 2778.0 5.19 5.22 3.19
0.73 Very Good H VS2 60.8 56.0 2779.0 5.82 5.84 3.55
0.7 Very Good F VS2 63.6 57.0 2780.0 5.61 5.65 3.58
0.77 Premium G VS2 61.2 58.0 2780.0 5.9 5.93 3.62
0.71 Ideal F VS2 62.1 54.0 2780.0 5.68 5.72 3.54
0.74 Ideal G VS1 61.5 55.0 2780.0 5.81 5.86 3.59
0.7 Ideal G VS1 61.4 59.0 2780.0 5.64 5.73 3.49
0.72 Very Good H VS1 60.6 63.0 2782.0 5.83 5.76 3.51
0.53 Very Good D VVS2 57.5 64.0 2782.0 5.34 5.37 3.08
0.76 Ideal G VS2 61.3 56.0 2782.0 5.9 5.94 3.63
0.7 Good E VS1 57.2 62.0 2782.0 5.81 5.77 3.31
0.7 Premium E VS1 62.9 60.0 2782.0 5.62 5.54 3.51
0.72 Very Good F VS2 63.0 54.0 2784.0 5.69 5.73 3.6
0.79 Very Good H VS1 63.7 56.0 2784.0 5.85 5.92 3.75
0.72 Very Good F VS2 63.6 58.0 2787.0 5.66 5.69 3.61
0.51 Ideal F VVS1 62.0 57.0 2787.0 5.11 5.15 3.18
0.64 Ideal D VS1 61.5 56.0 2787.0 5.54 5.55 3.41
0.7 Very Good H VVS1 60.5 60.0 2788.0 5.74 5.77 3.48
0.83 Very Good I VS1 61.1 60.0 2788.0 6.07 6.1 3.72
0.76 Ideal I VVS2 61.8 56.0 2788.0 5.85 5.87 3.62
0.71 Good D VS2 63.3 56.0 2788.0 5.64 5.68 3.58
0.77 Good G VS1 59.4 64.0 2788.0 5.97 5.92 3.53
0.63 Premium E VVS2 62.1 57.0 2789.0 5.48 5.41 3.38
0.63 Premium E VVS1 60.9 60.0 2789.0 5.55 5.52 3.37
0.77 Premium H VS1 61.3 60.0 2789.0 5.9 5.88 3.61
0.76 Premium I VVS1 58.8 59.0 2790.0 6.0 5.94 3.51
0.71 Premium F VS1 60.1 62.0 2790.0 5.77 5.74 3.46
0.71 Premium F VS1 61.8 59.0 2790.0 5.73 5.69 3.53
0.7 Premium F VS1 62.1 60.0 2792.0 5.71 5.65 3.53
0.7 Premium F VS1 60.7 60.0 2792.0 5.78 5.75 3.5
0.76 Premium H VVS2 59.6 57.0 2792.0 5.91 5.86 3.51
0.7 Ideal F VS1 62.2 56.0 2792.0 5.73 5.68 3.55
0.7 Very Good E VS2 62.9 57.0 2793.0 5.66 5.69 3.57
0.7 Good E VS2 64.1 55.0 2793.0 5.6 5.66 3.61
0.76 Ideal I VS2 61.3 56.0 2793.0 5.87 5.91 3.61
0.73 Ideal H VS2 62.7 55.0 2793.0 5.72 5.76 3.6
0.71 Very Good E VS2 60.7 56.0 2795.0 5.81 5.82 3.53
0.81 Premium I VVS2 61.9 60.0 2795.0 5.91 5.86 3.64
0.72 Good F VS1 60.7 60.0 2795.0 5.74 5.72 3.48
0.81 Premium H VS2 58.0 59.0 2795.0 6.17 6.13 3.57
0.72 Premium G VS2 62.9 57.0 2795.0 5.73 5.65 3.58
0.57 Ideal F VVS2 61.9 55.0 2797.0 5.34 5.35 3.31
0.51 Ideal D VVS1 61.7 56.0 2797.0 5.12 5.16 3.17
0.72 Ideal G VS2 61.9 58.0 2797.0 5.72 5.75 3.55
0.74 Ideal H VS1 61.8 58.0 2797.0 5.77 5.81 3.58
0.74 Ideal H VS1 61.6 56.0 2797.0 5.81 5.82 3.58
0.7 Fair G VVS1 58.8 66.0 2797.0 5.81 5.9 3.44
0.8 Very Good H VS2 63.4 60.0 2797.0 5.92 5.82 3.72
0.77 Ideal I VS1 61.5 59.0 2798.0 5.87 5.91 3.62
0.7 Premium E VS1 62.2 58.0 2800.0 5.6 5.66 3.5
0.74 Premium G VS1 62.9 60.0 2800.0 5.74 5.68 3.59
0.79 Ideal I VS1 61.8 59.0 2800.0 5.92 5.95 3.67
0.76 Fair G VS1 59.0 70.0 2800.0 5.89 5.8 3.46
0.73 Ideal F VS2 62.5 55.0 2801.0 5.8 5.76 3.61
0.73 Premium F VS2 62.7 58.0 2801.0 5.76 5.7 3.59
0.71 Premium F VS2 62.1 58.0 2801.0 5.7 5.67 3.53
0.71 Good F VS2 57.8 60.0 2801.0 5.9 5.87 3.4
0.71 Good F VS2 63.8 58.0 2801.0 5.64 5.61 3.59
0.71 Premium F VS2 62.8 57.0 2801.0 5.69 5.64 3.56
0.72 Premium E VS2 63.0 55.0 2802.0 5.79 5.61 3.59
0.72 Premium F VS1 62.4 58.0 2802.0 5.83 5.7 3.6
0.7 Very Good F VS2 62.9 58.0 2803.0 5.63 5.65 3.55
0.71 Ideal G VS2 61.3 56.0 2803.0 5.75 5.71 3.51
0.7 Good G VS1 65.1 58.0 2803.0 5.56 5.59 3.63
0.71 Premium F VS2 62.6 58.0 2803.0 5.7 5.67 3.56
0.71 Premium F VS2 58.0 62.0 2803.0 5.85 5.81 3.38
0.71 Premium G VS1 62.4 61.0 2803.0 5.7 5.65 3.54
0.77 Premium G VS2 61.3 57.0 2803.0 5.93 5.88 3.62
0.71 Premium G VS2 59.9 60.0 2803.0 5.81 5.77 3.47
0.78 Premium G VS2 60.8 58.0 2803.0 6.03 5.95 3.64
0.71 Very Good G VS1 63.5 55.0 2803.0 5.66 5.64 3.59
0.71 Very Good E VS2 63.8 58.0 2804.0 5.62 5.66 3.6
0.71 Very Good E VS2 64.0 57.0 2804.0 5.66 5.68 3.63
0.72 Ideal F VS1 61.7 57.0 2804.0 5.74 5.77 3.55
0.72 Very Good F VS1 62.2 58.0 2804.0 5.75 5.7 3.56
0.82 Ideal H VS2 61.5 56.0 2804.0 6.01 6.08 3.72
0.74 Premium F VS2 61.7 58.0 2805.0 5.85 5.78 3.59
0.74 Premium F VS2 61.9 56.0 2805.0 5.8 5.77 3.58
0.57 Fair E VVS1 58.7 66.0 2805.0 5.34 5.43 3.16
0.73 Premium F VS2 62.5 57.0 2805.0 5.75 5.7 3.58
0.72 Ideal G VS2 62.8 56.0 2805.0 5.74 5.7 3.59
0.74 Fair F VS2 61.1 68.0 2805.0 5.82 5.75 3.53
0.82 Good G VS2 64.0 57.0 2805.0 5.92 5.89 3.78
0.75 Very Good H VVS1 60.6 58.0 2806.0 5.85 5.9 3.56
0.71 Very Good F VS1 62.2 58.0 2807.0 5.66 5.72 3.54
0.71 Very Good F VS1 60.0 57.0 2807.0 5.84 5.9 3.52
0.8 Very Good H VS2 62.8 57.0 2808.0 5.87 5.91 3.7
0.7 Very Good F VS1 62.0 57.0 2808.0 5.64 5.71 3.52
0.75 Very Good G VS2 63.4 56.0 2808.0 5.78 5.74 3.65
0.58 Ideal E VVS2 60.9 56.0 2808.0 5.41 5.43 3.3
0.7 Very Good F VS1 61.8 56.0 2810.0 5.63 5.7 3.5
0.7 Very Good F VS1 59.9 60.0 2810.0 5.77 5.84 3.48
0.7 Good F VS1 62.8 61.0 2810.0 5.57 5.61 3.51
1.0 Fair J VS2 65.7 59.0 2811.0 6.14 6.07 4.01
0.7 Very Good G VS1 63.0 60.0 2812.0 5.57 5.64 3.53
0.7 Very Good F VS2 59.5 58.0 2812.0 5.75 5.85 3.45
0.7 Very Good F VS2 61.7 58.0 2812.0 5.63 5.69 3.49
0.29 Very Good E VS1 61.9 55.0 555.0 4.28 4.33 2.66
0.29 Very Good E VS1 62.4 55.0 555.0 4.2 4.25 2.63
0.34 Ideal H VS2 61.5 56.0 555.0 4.47 4.5 2.76
0.34 Ideal H VS2 60.4 57.0 555.0 4.54 4.57 2.75
0.34 Ideal I VS1 61.8 55.0 555.0 4.48 4.52 2.78
0.34 Ideal I VS1 62.0 56.0 555.0 4.5 4.53 2.8
0.3 Ideal G VS1 62.3 56.0 555.0 4.29 4.31 2.68
0.29 Ideal F VS1 61.6 56.0 555.0 4.26 4.31 2.64
0.32 Very Good F VS2 61.4 58.0 556.0 4.37 4.42 2.7
0.36 Ideal I VS2 61.9 56.0 556.0 4.54 4.57 2.82
0.3 Ideal G VS2 62.0 56.0 556.0 4.28 4.3 2.66
0.26 Ideal E VS1 61.5 57.0 556.0 4.09 4.12 2.52
0.7 Very Good F VS2 62.3 58.0 2812.0 5.64 5.72 3.54
0.7 Very Good F VS2 60.9 61.0 2812.0 5.66 5.71 3.46
0.73 Premium E VS2 58.6 60.0 2812.0 5.92 5.89 3.46
0.51 Ideal F VVS1 62.0 57.0 2812.0 5.15 5.11 3.18
0.91 Good I VS2 64.3 58.0 2813.0 6.09 6.05 3.9
0.55 Very Good D VVS1 61.5 56.0 2815.0 5.23 5.27 3.23
0.74 Premium G VS1 61.7 58.0 2815.0 5.79 5.81 3.58
0.9 Fair J VS2 65.0 56.0 2815.0 6.08 6.04 3.94
0.72 Premium E VS2 58.3 58.0 2815.0 5.99 5.92 3.47
0.78 Very Good I VVS2 61.4 56.0 2816.0 5.91 5.95 3.64
0.61 Ideal G VVS2 60.1 57.0 2816.0 5.52 5.54 3.32
0.71 Good D VS1 63.4 55.0 2816.0 5.61 5.69 3.58
0.71 Ideal I VVS2 60.2 56.0 2817.0 5.84 5.89 3.53
0.71 Ideal E VS2 62.7 57.0 2817.0 5.66 5.64 3.54
0.71 Premium E VS2 62.3 58.0 2817.0 5.69 5.65 3.53
0.63 Ideal F VVS2 61.5 56.0 2817.0 5.48 5.52 3.38
0.9 Ideal J VS2 62.8 55.0 2817.0 6.2 6.16 3.88
0.7 Premium E VS2 62.4 61.0 2818.0 5.66 5.63 3.52
0.7 Premium E VS2 59.3 60.0 2818.0 5.78 5.73 3.41
0.7 Premium E VS2 63.0 60.0 2818.0 5.64 5.6 3.54
0.7 Ideal E VS1 62.9 57.0 2818.0 5.66 5.61 3.54
0.7 Premium E VS1 59.6 57.0 2818.0 5.91 5.83 3.5
0.7 Premium F VS2 61.8 60.0 2818.0 5.69 5.64 3.5
0.7 Premium E VS1 62.7 57.0 2818.0 5.68 5.64 3.55
0.72 Very Good G VS1 63.8 58.0 2819.0 5.64 5.68 3.61
0.72 Ideal H VS1 62.3 56.0 2819.0 5.73 5.77 3.58
0.7 Good F VS1 59.7 63.0 2819.0 5.76 5.79 3.45
0.71 Ideal G VS1 62.9 58.0 2820.0 5.66 5.69 3.57
0.73 Premium E VS2 61.6 59.0 2821.0 5.77 5.73 3.54
0.53 Ideal E VVS1 61.9 55.0 2821.0 5.2 5.21 3.22
0.7 Premium E VS1 60.8 60.0 2822.0 5.74 5.71 3.48
0.72 Premium E VS2 60.3 59.0 2822.0 5.84 5.8 3.51
0.72 Premium E VS2 60.9 60.0 2822.0 5.8 5.76 3.52
0.72 Premium E VS2 62.4 59.0 2822.0 5.77 5.7 3.58
0.7 Premium E VS2 60.2 60.0 2822.0 5.73 5.7 3.44
0.6 Ideal F VVS2 62.0 55.0 2822.0 5.37 5.4 3.34
0.74 Ideal I VVS1 60.8 57.0 2822.0 5.85 5.89 3.57
0.9 Good J VS2 64.0 61.0 2822.0 6.04 6.03 3.86
0.7 Premium E VS2 61.5 59.0 2822.0 5.73 5.68 3.51
0.7 Premium E VS2 62.6 56.0 2822.0 5.71 5.66 3.56
0.7 Premium E VS2 60.7 62.0 2822.0 5.72 5.68 3.46
0.7 Premium F VS2 60.6 58.0 2822.0 5.8 5.72 3.49
0.71 Premium E VS2 62.3 58.0 2823.0 5.71 5.66 3.54
0.61 Ideal E VVS2 61.3 54.0 2823.0 5.51 5.59 3.4
0.71 Premium D VS2 61.2 59.0 2824.0 5.74 5.69 3.5
0.77 Ideal I VVS2 62.1 57.0 2824.0 5.84 5.86 3.63
0.74 Good E VS1 63.1 58.0 2824.0 5.73 5.75 3.62
0.71 Premium G VS1 62.2 59.0 2825.0 5.73 5.66 3.54
0.73 Very Good G VS1 62.0 57.0 2825.0 5.75 5.8 3.58
0.73 Premium E VS1 62.6 60.0 2826.0 5.75 5.68 3.58
0.7 Good E VS1 57.2 59.0 2826.0 5.94 5.88 3.38
0.7 Premium E VS1 62.2 58.0 2826.0 5.66 5.6 3.5
0.7 Very Good D VS2 63.3 56.0 2826.0 5.6 5.58 3.54
0.7 Premium E VS1 59.4 61.0 2826.0 5.78 5.74 3.42
0.72 Good D VS2 64.0 54.0 2827.0 5.68 5.7 3.64
0.79 Premium H VVS2 62.6 58.0 2827.0 5.96 5.9 3.71
0.7 Ideal H VVS1 61.6 57.0 2827.0 5.69 5.74 3.52
0.7 Ideal H VVS1 62.3 55.0 2827.0 5.66 5.7 3.54
0.72 Premium F VS1 62.2 58.0 2829.0 5.75 5.7 3.56
0.59 Ideal E VVS1 62.0 56.0 2829.0 5.36 5.38 3.33
0.72 Ideal F VS1 61.7 57.0 2829.0 5.77 5.74 3.55
0.71 Very Good E VS2 62.7 59.0 2830.0 5.65 5.7 3.56
0.53 Ideal F VVS1 60.9 57.0 2830.0 5.23 5.29 3.19
0.53 Ideal F VVS1 61.8 57.0 2830.0 5.16 5.19 3.2
0.8 Ideal I VS2 62.1 54.4 2830.0 5.94 5.99 3.7
0.72 Very Good F VS2 62.5 58.0 2832.0 5.71 5.75 3.58
0.71 Premium G VVS2 62.1 57.0 2832.0 5.75 5.65 3.54
0.71 Premium F VS1 62.2 58.0 2832.0 5.72 5.66 3.54
0.8 Very Good I VS2 62.0 58.0 2833.0 5.86 5.95 3.66
0.7 Very Good D VS2 59.6 61.0 2833.0 5.77 5.8 3.45
0.7 Ideal D VS2 61.0 57.0 2833.0 5.74 5.76 3.51
0.61 Ideal F VVS2 61.7 55.0 2833.0 5.45 5.48 3.37
0.8 Ideal G VS2 62.2 56.0 2834.0 5.94 5.87 3.67
0.8 Ideal H VS2 62.8 57.0 2834.0 5.91 5.87 3.7
0.51 Very Good D VVS1 59.9 58.0 2834.0 5.16 5.19 3.1
0.53 Ideal F VVS1 61.4 57.0 2834.0 5.2 5.23 3.2
0.78 Ideal I VS2 61.8 55.0 2834.0 5.92 5.95 3.67
0.73 Ideal F VS1 61.2 56.0 2835.0 5.89 5.81 3.58
0.63 Ideal F VVS2 61.9 57.0 2835.0 5.47 5.51 3.4
0.7 Ideal E VS2 61.5 54.0 2835.0 5.7 5.75 3.52
0.72 Ideal E VS2 62.8 57.0 2835.0 5.71 5.73 3.59
0.75 Premium F VS2 59.6 59.0 2835.0 6.04 5.94 3.57
0.71 Good E VS2 62.8 60.0 2836.0 5.6 5.65 3.53
0.7 Premium E VS1 62.6 59.0 2837.0 5.69 5.66 3.55
0.7 Ideal E VS1 61.8 56.0 2837.0 5.74 5.69 3.53
0.72 Premium F VS1 58.8 60.0 2838.0 5.91 5.89 3.47
0.7 Premium F VS2 62.3 58.0 2838.0 5.72 5.64 3.54
0.7 Premium F VS2 61.7 58.0 2838.0 5.69 5.63 3.49
0.7 Premium G VS1 62.6 55.0 2838.0 5.73 5.64 3.56
0.7 Premium F VS2 59.4 61.0 2838.0 5.83 5.79 3.45
0.7 Premium F VS2 60.9 61.0 2838.0 5.71 5.66 3.46
0.7 Premium F VS2 59.5 58.0 2838.0 5.85 5.75 3.45
0.7 Premium G VS1 63.0 60.0 2838.0 5.64 5.57 3.53
0.71 Ideal F VS1 61.5 57.0 2839.0 5.74 5.71 3.52
0.7 Ideal F VS1 61.6 54.0 2839.0 5.75 5.72 3.53
0.71 Ideal F VS1 62.1 55.0 2839.0 5.82 5.68 3.57
0.71 Premium F VS1 59.1 61.0 2839.0 5.84 5.81 3.44
0.71 Premium F VS1 59.0 60.0 2839.0 5.82 5.8 3.43
0.71 Premium F VS1 60.5 58.0 2839.0 5.75 5.72 3.47
0.7 Ideal F VS1 62.4 53.0 2839.0 5.73 5.71 3.57
0.73 Ideal G VS2 61.8 54.0 2839.0 5.8 5.82 3.59
0.7 Ideal E VS2 62.1 54.0 2839.0 5.69 5.72 3.54
0.7 Ideal G VS1 61.3 57.0 2839.0 5.71 5.74 3.51
0.71 Premium G VVS2 60.3 58.0 2839.0 5.82 5.78 3.5
0.71 Premium F VS1 59.2 58.0 2839.0 5.87 5.82 3.46
0.79 Premium G VS2 59.3 62.0 2839.0 6.09 6.01 3.59
0.71 Premium F VS1 62.7 59.0 2839.0 5.7 5.62 3.55
0.77 Very Good H VS1 61.0 60.0 2840.0 5.9 5.87 3.59
0.71 Premium F VS2 59.3 56.0 2840.0 5.88 5.82 3.47
0.7 Premium H VVS1 59.2 60.0 2840.0 5.87 5.78 3.45
0.73 Premium F VS2 60.3 59.0 2841.0 5.9 5.87 3.55
0.71 Very Good D VS1 63.4 55.0 2841.0 5.69 5.61 3.58
0.52 Ideal F VVS1 61.2 56.0 2841.0 5.19 5.21 3.18
0.73 Premium F VS2 59.9 59.0 2841.0 5.87 5.77 3.5
0.73 Premium G VS1 61.4 58.0 2841.0 5.82 5.77 3.56
0.8 Ideal I VS1 62.6 54.0 2842.0 5.92 5.96 3.72
0.7 Premium F VS2 58.7 61.0 2842.0 5.8 5.72 3.38
0.7 Very Good E VS2 60.2 62.0 2843.0 5.71 5.75 3.45
0.7 Very Good E VS2 62.7 58.0 2843.0 5.65 5.67 3.55
0.71 Very Good E VS2 59.4 58.0 2843.0 5.76 5.82 3.44
0.73 Ideal G VVS2 61.3 57.0 2843.0 5.81 5.84 3.57
0.73 Very Good F VS1 61.8 59.0 2843.0 5.73 5.79 3.56
0.72 Ideal E VS2 62.0 57.0 2843.0 5.71 5.74 3.55
0.71 Ideal G VVS2 60.7 57.0 2843.0 5.81 5.78 3.52
0.7 Very Good E VS1 62.0 59.0 2844.0 5.65 5.68 3.51
0.79 Ideal H VS2 62.5 57.0 2844.0 5.91 5.93 3.7
0.7 Very Good E VS2 61.8 59.0 2845.0 5.65 5.68 3.5
0.7 Very Good E VS2 58.9 60.0 2845.0 5.83 5.85 3.44
0.8 Good H VS2 63.4 60.0 2845.0 5.92 5.82 3.72
0.72 Very Good F VS1 60.2 59.0 2846.0 5.79 5.84 3.5
0.73 Ideal H VVS2 61.6 56.0 2846.0 5.79 5.84 3.58
0.7 Good F VS2 59.1 61.0 2846.0 5.76 5.84 3.43
0.77 Premium G VS2 61.3 60.0 2846.0 5.91 5.81 3.59
0.77 Premium G VS1 61.4 58.0 2846.0 5.94 5.89 3.63
0.7 Very Good G VVS2 62.9 59.0 2848.0 5.61 5.68 3.55
0.54 Ideal D VVS2 61.5 55.0 2848.0 5.25 5.29 3.24
0.74 Very Good E VS1 63.1 58.0 2849.0 5.75 5.73 3.62
0.7 Very Good E VS2 61.0 60.0 2850.0 5.74 5.77 3.51
0.7 Ideal F VS2 60.8 59.0 2850.0 5.69 5.79 3.49
0.66 Ideal D VS1 62.1 56.0 2851.0 5.54 5.57 3.45
0.78 Ideal I VS1 61.5 57.0 2852.0 5.88 5.92 3.63
0.71 Premium F VS2 62.6 58.0 2853.0 5.67 5.7 3.56
0.71 Good G VS1 63.5 55.0 2853.0 5.64 5.66 3.59
0.82 Premium I VS1 61.9 58.0 2853.0 5.99 5.97 3.7
0.78 Very Good H VS1 61.9 57.1 2854.0 5.87 5.95 3.66
0.7 Very Good E VS1 62.4 56.0 2854.0 5.64 5.7 3.54
0.73 Premium E VS2 62.0 57.0 2854.0 5.86 5.76 3.6
0.91 Fair J VS2 64.4 62.0 2854.0 6.06 6.03 3.89
0.91 Fair J VS2 65.4 60.0 2854.0 6.04 6.0 3.94
0.91 Good J VS2 64.2 58.0 2854.0 6.12 6.09 3.92
0.7 Premium E VS1 58.4 59.0 2854.0 5.91 5.83 3.43
0.68 Premium F VVS2 61.7 57.0 2854.0 5.67 5.64 3.49
0.73 Very Good F VS2 62.5 57.0 2855.0 5.7 5.75 3.58
0.74 Premium D VS2 62.4 57.0 2855.0 5.8 5.74 3.6
0.6 Ideal F VVS2 60.8 57.0 2856.0 5.44 5.49 3.32
0.26 Ideal E VS1 62.3 57.0 556.0 4.05 4.08 2.53
0.26 Ideal E VS1 62.1 56.0 556.0 4.09 4.12 2.55
0.34 Good G VS2 57.5 61.0 556.0 4.6 4.66 2.66
0.34 Very Good G VS2 59.6 62.0 556.0 4.54 4.56 2.71
0.32 Good E VS2 64.1 54.0 556.0 4.34 4.37 2.79
0.31 Ideal I VVS1 61.6 55.0 557.0 4.36 4.41 2.7
0.31 Ideal I VVS1 61.3 56.0 557.0 4.36 4.38 2.68
0.31 Ideal I VVS1 62.3 54.0 557.0 4.37 4.4 2.73
0.31 Ideal I VVS1 62.0 54.0 557.0 4.37 4.4 2.72
0.31 Ideal I VVS1 62.7 53.0 557.0 4.33 4.35 2.72
0.31 Ideal I VVS1 62.2 53.0 557.0 4.36 4.38 2.72
0.31 Ideal G VS2 62.2 53.6 557.0 4.32 4.35 2.7
0.31 Ideal H VS1 61.6 54.8 557.0 4.35 4.37 2.69
0.31 Ideal H VS1 61.8 54.2 557.0 4.33 4.37 2.69
0.33 Premium J VS1 62.8 58.0 557.0 4.41 4.38 2.76
0.33 Premium J VS1 61.5 61.0 557.0 4.46 4.39 2.72
0.33 Ideal J VS1 62.1 55.0 557.0 4.44 4.41 2.75
0.7 Good E VVS2 60.1 63.0 2857.0 5.68 5.71 3.42
0.9 Premium I VS2 61.9 59.0 2857.0 6.2 6.14 3.82
0.7 Ideal G VVS2 62.1 56.0 2858.0 5.63 5.71 3.52
0.71 Premium E VS2 61.0 60.0 2858.0 5.76 5.69 3.49
0.7 Premium E VS2 61.4 59.0 2858.0 5.73 5.7 3.51
0.71 Premium E VS2 61.5 60.0 2858.0 5.76 5.68 3.52
0.71 Very Good E VS2 63.5 59.0 2858.0 5.68 5.59 3.58
0.71 Premium D VS2 60.4 62.0 2858.0 5.74 5.72 3.46
0.7 Good E VVS2 63.6 62.0 2858.0 5.61 5.58 3.56
0.71 Fair D VS2 56.9 65.0 2858.0 5.89 5.84 3.34
0.7 Ideal D VS2 61.0 57.0 2859.0 5.76 5.74 3.51
0.7 Premium D VS2 62.4 56.0 2859.0 5.72 5.66 3.55
0.77 Premium F VS1 60.9 60.0 2859.0 5.91 5.88 3.59
0.71 Ideal G VS1 61.5 56.0 2859.0 5.74 5.78 3.54
0.7 Premium D VS2 59.6 61.0 2859.0 5.8 5.77 3.45
0.75 Fair F VS1 55.8 70.0 2859.0 6.09 5.98 3.37
0.71 Very Good F VS2 61.3 61.0 2860.0 5.68 5.73 3.5
0.6 Ideal E VVS2 61.9 54.9 2860.0 5.41 5.44 3.35
0.71 Premium D VS1 62.9 57.0 2860.0 5.66 5.6 3.54
0.53 Ideal F VVS1 61.4 57.0 2860.0 5.23 5.2 3.2
0.62 Ideal G VVS2 61.6 56.0 2861.0 5.45 5.48 3.37
0.62 Ideal G VVS2 61.6 56.0 2861.0 5.48 5.51 3.38
0.66 Premium D VS1 61.0 58.0 2861.0 5.67 5.57 3.43
0.71 Very Good F VVS1 63.2 60.0 2862.0 5.65 5.61 3.56
0.7 Ideal H VS2 61.1 57.0 2862.0 5.71 5.74 3.5
0.7 Very Good E VS2 58.7 63.0 2862.0 5.73 5.69 3.35
0.79 Premium H VS1 60.0 60.0 2862.0 6.07 5.99 3.64
0.7 Premium E VS2 59.5 59.0 2862.0 5.82 5.77 3.45
0.73 Premium E VS2 62.5 61.0 2862.0 5.78 5.64 3.59
0.91 Good I VS2 64.3 58.0 2863.0 6.05 6.09 3.9
0.9 Premium J VS2 59.8 62.0 2863.0 6.24 6.21 3.72
0.71 Premium H VVS2 61.5 62.0 2863.0 5.74 5.68 3.51
0.72 Ideal F VS2 59.5 57.0 2863.0 5.91 5.86 3.5
0.71 Ideal E VS2 61.0 55.0 2863.0 5.79 5.75 3.52
0.83 Very Good I VS2 61.6 58.0 2865.0 6.05 6.07 3.73
0.56 Very Good D VVS1 62.0 56.0 2866.0 5.25 5.3 3.27
0.56 Very Good D VVS1 61.8 55.0 2866.0 5.27 5.31 3.27
0.71 Ideal E VS1 62.2 55.0 2866.0 5.74 5.7 3.56
0.7 Ideal H VVS1 62.3 58.0 2866.0 5.66 5.7 3.54
0.71 Very Good H VVS1 62.9 57.0 2867.0 5.67 5.69 3.57
0.7 Ideal D VS2 62.4 57.0 2867.0 5.68 5.61 3.52
0.71 Ideal H VVS1 60.4 57.0 2867.0 5.78 5.81 3.5
0.8 Premium H VS2 61.2 53.0 2867.0 6.05 5.98 3.68
0.52 Ideal F VVS1 61.2 56.0 2867.0 5.21 5.19 3.18
0.72 Ideal I VS1 62.4 55.0 2868.0 5.72 5.75 3.58
0.73 Ideal G VVS2 61.3 57.0 2869.0 5.84 5.81 3.57
0.72 Ideal H VVS2 60.9 57.0 2869.0 5.79 5.77 3.52
0.52 Premium F VVS2 61.8 60.0 2870.0 5.16 5.13 3.18
0.64 Premium E VVS2 62.1 58.0 2870.0 5.56 5.51 3.44
0.82 Ideal H VS2 59.5 57.0 2870.0 6.12 6.09 3.63
0.73 Premium E VS1 61.3 59.0 2870.0 5.81 5.78 3.55
0.72 Very Good E VS2 58.3 57.0 2872.0 5.89 5.94 3.45
0.76 Very Good F VS2 62.0 58.0 2873.0 5.8 5.86 3.62
0.78 Premium F VS2 62.6 58.0 2874.0 5.91 5.82 3.67
0.71 Premium D VS2 61.2 59.0 2874.0 5.69 5.74 3.5
0.7 Premium F VS1 59.0 59.0 2874.0 5.79 5.77 3.41
0.7 Premium F VS1 60.8 62.0 2874.0 5.71 5.67 3.46
0.7 Premium G VVS2 61.8 58.0 2874.0 5.67 5.63 3.49
0.7 Ideal F VS1 61.0 55.0 2874.0 5.77 5.73 3.51
0.7 Ideal F VS1 61.6 55.0 2874.0 5.75 5.71 3.53
0.7 Ideal F VS1 62.4 56.0 2874.0 5.69 5.65 3.54
0.7 Premium G VVS2 62.9 59.0 2874.0 5.68 5.61 3.55
1.0 Fair J VS1 65.5 55.0 2875.0 6.3 6.25 4.11
0.73 Premium E VS1 62.6 60.0 2876.0 5.68 5.75 3.58
0.79 Premium E VS2 60.6 53.0 2876.0 6.04 5.98 3.64
0.72 Very Good H VS1 62.2 54.0 2877.0 5.74 5.76 3.57
0.71 Ideal E VS1 62.4 56.0 2877.0 5.75 5.7 3.57
0.74 Ideal G VS2 62.3 55.0 2877.0 5.8 5.83 3.62
0.7 Good H VVS1 62.7 56.0 2877.0 5.6 5.66 3.53
0.7 Good F VS1 59.1 62.0 2877.0 5.82 5.86 3.44
0.71 Ideal I VS2 61.5 55.0 2878.0 5.76 5.78 3.55
0.7 Premium F VS1 60.4 60.0 2879.0 5.73 5.7 3.45
0.71 Premium F VS1 62.7 58.0 2879.0 5.71 5.67 3.57
0.72 Fair F VS1 56.9 69.0 2879.0 5.93 5.77 3.33
0.72 Ideal F VS1 62.0 56.0 2879.0 5.76 5.73 3.56
0.7 Ideal H VVS1 62.0 55.0 2881.0 5.74 5.71 3.55
0.71 Very Good E VS2 60.0 59.0 2881.0 5.84 5.83 3.5
0.54 Ideal F VVS1 61.8 56.0 2882.0 5.23 5.26 3.24
0.73 Premium F VS2 59.9 58.0 2882.0 5.87 5.84 3.51
0.73 Premium F VS2 58.7 57.0 2882.0 5.97 5.92 3.49
0.7 Premium E VS1 62.6 59.0 2887.0 5.66 5.69 3.55
0.79 Ideal I VS1 61.7 59.0 2888.0 5.93 5.96 3.67
0.72 Very Good G VVS2 62.5 58.0 2889.0 5.68 5.72 3.56
0.7 Very Good E VS2 63.5 54.0 2889.0 5.62 5.66 3.58
0.7 Very Good F VS1 62.2 58.0 2889.0 5.64 5.75 3.54
0.71 Very Good F VS1 62.8 56.0 2889.0 5.69 5.72 3.58
0.5 Ideal E VVS2 62.2 54.0 2889.0 5.08 5.12 3.17
0.5 Ideal E VVS2 62.2 54.0 2889.0 5.09 5.11 3.17
0.77 Premium F VS2 61.8 56.0 2889.0 5.94 5.9 3.66
0.66 Ideal G VVS1 61.5 56.0 2890.0 5.61 5.58 3.44
0.71 Very Good E VS2 61.2 58.0 2891.0 5.71 5.79 3.52
0.71 Ideal F VS2 61.2 56.0 2891.0 5.73 5.77 3.52
0.71 Ideal E VS2 61.6 56.0 2891.0 5.74 5.76 3.54
0.71 Ideal E VS2 62.7 56.0 2891.0 5.71 5.75 3.59
0.71 Good D VS2 62.3 61.0 2891.0 5.7 5.73 3.56
0.71 Very Good F VS1 62.6 55.0 2893.0 5.66 5.71 3.56
0.71 Ideal G VVS2 61.5 57.0 2893.0 5.73 5.75 3.53
0.75 Ideal F VS2 62.5 57.0 2893.0 5.78 5.83 3.63
0.7 Very Good H VVS1 59.2 60.0 2893.0 5.87 5.78 3.45
0.71 Very Good G VS2 60.9 56.0 2895.0 5.75 5.78 3.51
0.7 Very Good F VS1 61.8 59.0 2895.0 5.66 5.76 3.53
0.7 Ideal G VVS2 62.1 53.0 2895.0 5.71 5.75 3.56
0.74 Very Good G VS1 59.8 58.0 2896.0 5.85 5.89 3.51
0.77 Very Good G VS2 61.3 60.0 2896.0 5.81 5.91 3.59
0.77 Very Good G VS2 58.3 63.0 2896.0 6.0 6.05 3.51
0.53 Ideal F VVS1 61.6 56.0 2896.0 5.18 5.24 3.21
0.6 Very Good D VVS2 60.6 57.0 2897.0 5.48 5.51 3.33
0.54 Ideal D VVS2 61.4 52.0 2897.0 5.3 5.34 3.26
0.72 Good F VS1 59.4 61.0 2897.0 5.82 5.89 3.48
0.74 Premium D VS2 61.8 58.0 2897.0 5.81 5.77 3.58
0.7 Good G VVS1 59.9 61.0 2899.0 5.75 5.81 3.46
0.72 Premium D VS1 62.7 58.0 2900.0 5.68 5.65 3.55
0.74 Ideal E VS2 61.9 57.0 2901.0 5.81 5.78 3.59
0.73 Premium E VS2 62.0 60.0 2902.0 5.76 5.73 3.56
0.73 Ideal E VS2 61.4 55.0 2902.0 5.82 5.8 3.57
0.71 Fair E VS2 64.6 59.0 2902.0 5.62 5.59 3.62
0.71 Premium E VS2 59.6 60.0 2902.0 5.85 5.8 3.47
0.72 Premium E VS2 61.1 59.0 2903.0 5.8 5.75 3.53
0.7 Very Good E VS1 58.4 59.0 2904.0 5.83 5.91 3.43
0.62 Ideal E VVS2 62.0 56.0 2904.0 5.48 5.52 3.41
0.7 Very Good G VVS2 59.3 62.0 2905.0 5.78 5.82 3.44
0.7 Very Good G VVS2 63.4 59.0 2905.0 5.62 5.64 3.57
0.7 Very Good G VVS2 63.3 59.0 2905.0 5.59 5.62 3.55
0.71 Very Good G VS2 62.1 58.0 2905.0 5.65 5.71 3.53
0.86 Very Good I VS1 61.2 58.0 2905.0 6.1 6.16 3.75
0.53 Ideal D VVS1 62.5 54.0 2905.0 5.16 5.21 3.24
0.74 Very Good D VS2 62.4 57.0 2906.0 5.74 5.8 3.6
0.8 Ideal I VS1 62.2 58.0 2906.0 5.92 5.95 3.69
0.61 Ideal E VVS2 62.4 53.9 2907.0 5.42 5.43 3.38
0.61 Ideal E VVS2 62.4 53.6 2907.0 5.42 5.45 3.39
0.61 Ideal E VVS2 62.1 54.2 2907.0 5.43 5.45 3.38
0.72 Ideal H VVS1 62.8 57.0 2907.0 5.68 5.72 3.58
0.7 Ideal F VS2 62.3 53.0 2907.0 5.69 5.73 3.56
0.71 Ideal F VS1 61.9 56.0 2907.0 5.7 5.74 3.54
0.25 Premium F VS1 61.2 59.0 558.0 4.05 4.02 2.47
0.25 Good F VS1 63.6 57.0 558.0 4.04 4.01 2.56
0.25 Premium E VS1 60.7 59.0 558.0 4.13 4.11 2.5
0.25 Premium E VS1 61.5 60.0 558.0 4.04 4.02 2.48
0.31 Premium I VS2 60.8 58.0 558.0 4.37 4.34 2.65
0.31 Premium I VS2 59.8 60.0 558.0 4.42 4.38 2.63
0.31 Very Good I VS2 63.2 55.0 558.0 4.4 4.3 2.75
0.31 Premium I VS2 62.3 57.0 558.0 4.35 4.32 2.7
0.31 Premium I VS2 60.8 60.0 558.0 4.42 4.37 2.67
0.31 Ideal I VS2 59.9 57.0 558.0 4.4 4.38 2.63
0.31 Premium I VS2 59.9 60.0 558.0 4.44 4.41 2.65
0.31 Premium I VS2 61.1 58.0 558.0 4.38 4.36 2.67
0.31 Premium I VS2 60.7 61.0 558.0 4.34 4.32 2.63
0.31 Very Good I VS2 63.1 54.0 558.0 4.34 4.31 2.73
0.31 Premium I VS2 62.3 60.0 558.0 4.32 4.31 2.69
0.73 Ideal I VS1 61.5 55.0 2908.0 5.8 5.84 3.58
0.7 Premium D VS2 61.0 60.0 2909.0 5.75 5.7 3.49
0.7 Premium D VS2 60.9 57.0 2909.0 5.71 5.69 3.47
0.71 Ideal H VS1 61.2 56.0 2909.0 5.76 5.81 3.54
0.71 Ideal H VS1 61.9 56.0 2909.0 5.7 5.74 3.54
0.71 Very Good D VS1 62.9 57.0 2910.0 5.6 5.66 3.54
0.59 Ideal E VVS2 61.1 57.0 2911.0 5.39 5.41 3.3
0.71 Ideal G VS2 60.6 56.0 2911.0 5.76 5.8 3.5
0.77 Good F VS2 60.3 61.0 2911.0 5.89 5.96 3.57
0.73 Good E VS2 64.2 54.0 2912.0 5.68 5.72 3.66
0.7 Good E VS2 58.7 63.0 2912.0 5.69 5.73 3.35
0.73 Good E VS2 63.2 56.0 2912.0 5.75 5.76 3.64
0.7 Very Good D VS2 60.7 60.0 2913.0 5.72 5.74 3.48
0.83 Very Good I VS2 62.0 55.0 2915.0 6.03 6.06 3.74
0.71 Ideal F VS2 62.2 56.0 2915.0 5.74 5.71 3.56
0.73 Very Good H VS1 60.8 57.0 2916.0 5.8 5.83 3.54
0.74 Premium F VS1 62.5 60.0 2917.0 5.78 5.74 3.6
0.7 Ideal E VS2 62.5 58.0 2917.0 5.63 5.67 3.53
0.71 Ideal F VS2 61.2 56.0 2917.0 5.77 5.73 3.52
0.71 Very Good F VS2 59.5 58.0 2918.0 5.82 5.87 3.48
0.8 Very Good H VS2 61.2 53.0 2918.0 5.98 6.05 3.68
0.71 Ideal H VVS1 62.1 54.0 2918.0 5.7 5.76 3.56
0.72 Ideal I VS2 61.8 55.0 2918.0 5.75 5.79 3.56
0.72 Very Good G VS1 60.5 57.0 2919.0 5.8 5.83 3.52
0.73 Premium G VVS2 62.2 56.0 2919.0 5.79 5.75 3.59
0.7 Good F VS1 63.8 58.0 2919.0 5.61 5.58 3.57
0.73 Ideal H VS1 61.9 55.0 2919.0 5.79 5.76 3.58
0.73 Ideal G VVS2 61.9 55.0 2919.0 5.83 5.77 3.59
0.71 Premium E VS1 59.7 57.0 2920.0 5.87 5.78 3.48
0.71 Premium F VS1 59.1 59.0 2920.0 5.88 5.83 3.46
0.71 Ideal F VS1 62.6 55.0 2920.0 5.71 5.67 3.56
0.74 Very Good H VVS2 60.5 60.0 2921.0 5.79 5.81 3.51
0.71 Very Good E VS2 59.9 59.0 2921.0 5.77 5.81 3.47
0.71 Very Good E VS2 60.7 60.0 2921.0 5.75 5.78 3.5
0.65 Ideal F VVS2 61.3 56.0 2921.0 5.58 5.61 3.43
0.9 Fair I VS2 64.1 66.0 2921.0 6.04 5.98 3.85
0.71 Very Good E VS2 63.7 58.0 2922.0 5.6 5.64 3.58
0.71 Very Good E VS2 63.3 59.0 2922.0 5.62 5.66 3.57
0.68 Very Good F VS1 59.7 57.0 2922.0 5.79 5.76 3.45
0.53 Ideal F VVS1 61.6 56.0 2922.0 5.24 5.18 3.21
0.72 Very Good E VS2 63.0 57.0 2923.0 5.69 5.73 3.6
0.72 Very Good E VS2 63.2 58.0 2923.0 5.67 5.72 3.6
0.71 Ideal E VS1 62.4 54.0 2923.0 5.71 5.74 3.57
0.9 Premium I VS2 58.7 60.0 2923.0 6.35 6.28 3.7
0.7 Ideal I VS1 61.5 56.0 2924.0 5.71 5.75 3.52
0.7 Very Good F VS1 64.5 58.0 2925.0 5.55 5.59 3.59
0.77 Very Good H VS1 63.3 57.0 2927.0 5.79 5.83 3.68
0.7 Very Good F VS2 61.3 54.0 2928.0 5.72 5.76 3.52
0.7 Very Good D VS2 60.8 59.0 2928.0 5.67 5.71 3.46
0.8 Very Good G VS2 61.1 57.0 2929.0 6.01 6.07 3.69
0.7 Ideal G VS2 61.8 57.0 2929.0 5.68 5.71 3.52
0.71 Very Good E VS2 61.3 60.0 2930.0 5.74 5.71 3.51
0.7 Premium E VS1 60.3 58.0 2930.0 5.7 5.74 3.45
0.7 Ideal E VS1 62.3 54.0 2930.0 5.67 5.72 3.55
0.71 Ideal F VS2 62.3 57.0 2930.0 5.69 5.74 3.56
0.71 Ideal G VS1 62.7 57.0 2930.0 5.69 5.73 3.58
0.71 Ideal G VS1 62.6 57.0 2930.0 5.67 5.7 3.56
0.71 Ideal G VVS1 61.7 57.0 2930.0 5.75 5.7 3.53
0.7 Very Good G VVS2 60.8 57.0 2931.0 5.72 5.76 3.49
0.72 Very Good F VS2 63.3 57.0 2931.0 5.69 5.72 3.61
0.72 Ideal F VS2 61.8 59.0 2931.0 5.71 5.74 3.54
0.7 Premium G VVS1 62.0 61.0 2932.0 5.71 5.62 3.51
0.7 Premium F VVS2 61.0 57.0 2932.0 5.8 5.71 3.51
0.7 Very Good F VVS2 63.2 58.0 2932.0 5.66 5.6 3.56
0.72 Very Good G VVS2 62.2 57.0 2933.0 5.67 5.72 3.54
0.59 Very Good D VVS2 60.6 59.0 2933.0 5.44 5.49 3.31
0.73 Premium F VS2 59.9 58.0 2933.0 5.84 5.87 3.51
0.75 Ideal F VS2 62.3 57.0 2933.0 5.81 5.87 3.64
0.8 Premium H VS1 62.0 60.0 2935.0 5.92 5.86 3.65
0.7 Very Good G VVS2 61.8 60.0 2936.0 5.63 5.69 3.5
0.74 Ideal F VS2 60.5 59.0 2936.0 5.81 5.86 3.53
0.76 Premium G VS1 59.6 57.0 2937.0 6.01 5.91 3.55
0.71 Very Good H VVS1 62.7 57.0 2938.0 5.66 5.72 3.57
0.71 Very Good H VVS1 62.7 59.0 2938.0 5.65 5.67 3.55
0.73 Very Good F VS2 62.7 58.0 2939.0 5.73 5.75 3.6
0.73 Very Good G VS1 60.7 57.0 2939.0 5.76 5.83 3.52
0.73 Ideal F VS2 62.7 58.0 2939.0 5.72 5.77 3.6
0.75 Ideal G VS2 60.6 55.0 2939.0 5.93 5.91 3.59
0.81 Ideal I VS2 61.8 56.0 2939.0 6.02 5.99 3.71
0.82 Premium H VS2 62.6 59.0 2939.0 5.99 5.93 3.73
0.7 Good F VVS2 63.1 57.0 2940.0 5.59 5.66 3.55
0.7 Very Good F VVS2 62.6 59.0 2940.0 5.6 5.64 3.52
0.7 Ideal F VS1 61.2 54.0 2940.0 5.92 5.64 3.54
0.75 Fair E VS2 56.0 67.0 2940.0 6.18 6.08 3.43
0.75 Ideal E VS2 61.6 57.0 2940.0 5.84 5.81 3.59
0.7 Ideal E VS2 61.5 56.0 2940.0 5.73 5.68 3.51
0.71 Premium F VS1 61.1 58.0 2942.0 5.76 5.72 3.51
0.7 Ideal F VS2 60.8 56.0 2942.0 5.78 5.79 3.52
0.72 Ideal F VS2 62.0 56.0 2943.0 5.77 5.75 3.57
0.74 Very Good H VVS2 61.3 58.0 2944.0 5.8 5.85 3.57
0.57 Very Good D VVS1 60.4 57.0 2945.0 5.39 5.44 3.27
0.79 Very Good H VS2 61.5 55.0 2945.0 5.89 5.94 3.64
0.71 Very Good E VS1 63.3 59.0 2946.0 5.64 5.67 3.58
0.71 Very Good E VS1 62.7 57.0 2946.0 5.69 5.73 3.58
0.72 Ideal H VVS1 62.2 56.0 2946.0 5.72 5.75 3.57
0.72 Ideal H VVS1 62.5 57.0 2946.0 5.7 5.73 3.57
0.78 Very Good H VS1 61.7 56.0 2947.0 5.92 5.94 3.66
0.76 Ideal E VS1 62.1 57.0 2947.0 5.82 5.87 3.63
0.73 Premium D VS2 60.9 59.0 2947.0 5.82 5.77 3.53
0.7 Ideal H VVS1 61.2 57.0 2947.0 5.69 5.72 3.49
0.7 Ideal H VVS1 60.5 58.0 2947.0 5.76 5.81 3.5
0.74 Ideal I VS1 62.0 56.0 2947.0 5.79 5.82 3.6
0.74 Ideal I VS1 61.1 57.0 2947.0 5.83 5.86 3.57
0.82 Good H VS2 62.4 54.0 2947.0 5.97 6.04 3.75
0.73 Ideal G VS1 61.7 55.0 2948.0 5.8 5.84 3.59
0.72 Very Good E VS2 63.0 56.0 2949.0 5.66 5.73 3.59
0.72 Ideal H VS1 62.3 55.0 2949.0 5.72 5.74 3.57
0.81 Very Good I VS1 62.7 58.0 2950.0 5.9 5.96 3.72
0.71 Ideal G VS1 62.4 57.0 2950.0 5.68 5.73 3.56
0.71 Premium D VS2 62.1 60.0 2950.0 5.72 5.68 3.54
0.54 Ideal F VVS1 61.6 55.0 2951.0 5.27 5.28 3.25
0.72 Very Good D VS1 62.7 58.0 2951.0 5.65 5.68 3.55
0.7 Very Good E VS2 62.4 58.0 2952.0 5.66 5.68 3.54
0.7 Very Good E VS2 63.4 59.0 2952.0 5.63 5.67 3.58
0.7 Very Good E VS2 61.8 59.0 2952.0 5.63 5.67 3.49
0.7 Very Good E VS1 61.3 60.0 2952.0 5.68 5.7 3.49
0.72 Ideal G VS2 61.5 55.0 2952.0 5.76 5.79 3.55
0.72 Ideal G VS2 61.4 55.0 2952.0 5.76 5.8 3.55
0.7 Ideal E VS2 61.9 58.0 2952.0 5.7 5.73 3.54
0.7 Ideal E VS2 62.6 57.0 2952.0 5.63 5.68 3.54
0.7 Ideal E VS2 62.1 55.0 2952.0 5.71 5.75 3.56
0.7 Good E VS1 61.0 61.0 2952.0 5.69 5.72 3.48
0.8 Very Good H VS2 59.1 59.0 2953.0 6.02 6.07 3.57
0.79 Premium F VS2 63.0 59.0 2953.0 5.84 5.8 3.66
0.75 Good F VS1 64.4 59.0 2953.0 5.67 5.72 3.66
0.71 Very Good E VS2 59.6 60.0 2954.0 5.8 5.85 3.47
0.72 Premium E VS2 61.1 59.0 2954.0 5.75 5.8 3.53
0.76 Ideal G VS2 61.7 54.0 2954.0 5.88 5.92 3.64
0.89 Premium I VS1 62.2 62.0 2955.0 6.14 6.02 3.78
0.7 Very Good F VS2 62.4 57.0 2956.0 5.67 5.71 3.55
0.74 Very Good H VS1 61.4 56.0 2956.0 5.81 5.84 3.57
0.74 Very Good H VS1 62.3 56.0 2956.0 5.75 5.78 3.59
0.7 Ideal F VS2 60.8 57.0 2956.0 5.75 5.77 3.5
0.71 Good F VVS2 58.2 60.0 2956.0 5.89 5.94 3.44
0.7 Premium D VS1 60.4 58.0 2956.0 5.78 5.71 3.47
0.72 Ideal F VS2 62.6 56.0 2956.0 5.75 5.72 3.59
0.72 Ideal F VS2 62.2 56.0 2956.0 5.75 5.73 3.57
0.72 Ideal H VVS1 62.0 55.0 2958.0 5.74 5.77 3.57
0.79 Ideal I VS1 62.2 57.0 2958.0 5.89 5.94 3.68
0.72 Good G VS1 58.0 57.8 2958.0 5.85 5.87 3.4
0.56 Very Good D VVS1 60.1 58.0 2959.0 5.36 5.42 3.24
0.7 Very Good F VS1 60.1 58.0 2959.0 5.73 5.79 3.46
0.79 Premium G VS2 62.3 58.0 2959.0 5.92 5.89 3.68
0.74 Fair G VVS2 65.2 58.0 2959.0 5.7 5.6 3.69
0.71 Very Good H VVS2 61.8 56.0 2960.0 5.7 5.73 3.53
0.7 Very Good D VS2 63.0 56.0 2960.0 5.61 5.69 3.56
0.7 Good D VS2 63.4 57.0 2960.0 5.6 5.67 3.57
0.7 Ideal D VS2 61.3 57.0 2960.0 5.72 5.76 3.52
0.76 Ideal F VS2 62.6 56.0 2960.0 5.82 5.78 3.63
0.72 Ideal G VS2 61.3 56.0 2960.0 5.77 5.81 3.55
0.71 Good F VVS2 58.9 61.0 2960.0 5.8 5.9 3.44
0.74 Ideal G VS1 61.8 55.0 2960.0 5.85 5.8 3.6
0.77 Very Good H VS1 62.8 58.0 2961.0 5.75 5.78 3.62
0.74 Ideal H VVS2 61.2 57.0 2961.0 5.79 5.85 3.56
0.72 Premium E VS1 61.5 60.0 2961.0 5.79 5.75 3.55
0.73 Premium F VS1 61.9 56.0 2961.0 5.81 5.76 3.58
0.73 Premium F VS1 62.7 56.0 2961.0 5.75 5.73 3.6
0.63 Ideal F VVS2 62.3 56.0 2961.0 5.48 5.5 3.42
0.72 Ideal H VS1 61.1 57.0 2961.0 5.8 5.82 3.55
0.71 Premium F VS1 62.1 53.0 2961.0 5.77 5.7 3.56
0.75 Premium H VS1 61.9 61.0 2961.0 5.85 5.82 3.61
0.63 Ideal D VVS2 62.6 56.0 2962.0 5.47 5.49 3.43
0.72 Ideal E VS2 62.0 56.0 2962.0 5.73 5.76 3.56
0.71 Ideal G VS1 62.2 56.0 2962.0 5.69 5.72 3.55
0.71 Ideal E VS1 62.1 53.0 2963.0 5.76 5.73 3.57
0.71 Very Good E VS2 62.9 57.0 2964.0 5.68 5.7 3.58
0.7 Good E VS1 63.6 58.0 2964.0 5.61 5.56 3.55
0.7 Fair E VS1 64.5 57.0 2964.0 5.59 5.55 3.59
0.9 Fair J VS1 65.4 60.0 2964.0 6.02 5.93 3.91
0.9 Premium J VS1 62.1 62.0 2964.0 6.12 6.05 3.78
0.9 Fair J VS1 64.6 58.0 2964.0 6.12 6.06 3.93
0.71 Ideal I VS1 61.8 56.0 2965.0 5.68 5.72 3.52
0.71 Ideal I VS1 61.6 56.0 2965.0 5.71 5.75 3.53
0.71 Ideal I VS1 61.3 57.0 2965.0 5.73 5.76 3.52
0.71 Ideal I VS1 61.5 56.0 2965.0 5.72 5.76 3.52
0.73 Very Good G VS2 62.1 59.0 2966.0 5.68 5.73 3.54
0.7 Ideal I VVS1 61.8 56.0 2966.0 5.69 5.73 3.53
0.7 Very Good E VS1 61.3 56.0 2967.0 5.68 5.71 3.49
0.7 Very Good E VS1 61.5 56.0 2967.0 5.69 5.75 3.52
0.79 Ideal H VS2 62.0 56.0 2967.0 5.91 5.93 3.67
0.3 Very Good H VVS2 62.0 56.0 559.0 4.28 4.3 2.66
0.31 Very Good G VS2 62.6 56.0 559.0 4.33 4.37 2.72
0.31 Very Good G VS2 61.4 55.0 559.0 4.38 4.41 2.69
0.31 Very Good G VS2 60.9 57.0 559.0 4.37 4.39 2.67
0.24 Ideal G VVS1 62.4 56.0 559.0 3.97 3.99 2.48
0.24 Ideal G VVS1 62.1 56.0 559.0 3.97 4.0 2.47
0.24 Ideal G VVS1 62.2 56.0 559.0 4.0 4.04 2.5
0.24 Ideal G VVS1 62.0 55.0 559.0 4.01 4.03 2.49
0.24 Ideal G VVS1 62.0 56.0 559.0 3.97 4.01 2.47
0.32 Ideal G VS1 62.3 55.0 559.0 4.39 4.41 2.74
0.32 Ideal G VS1 61.8 55.0 559.0 4.42 4.45 2.74
0.25 Very Good E VVS2 62.0 56.0 560.0 4.05 4.08 2.52
0.25 Very Good E VVS1 61.5 56.0 560.0 4.06 4.08 2.5
0.32 Ideal G VS2 61.6 54.0 560.0 4.4 4.43 2.72
0.32 Premium H VS1 60.2 58.0 561.0 4.43 4.47 2.68
0.32 Ideal H VS1 61.5 57.0 561.0 4.4 4.42 2.71
0.71 Premium D VS2 58.7 61.0 2968.0 5.88 5.85 3.44
0.8 Ideal G VS2 61.2 57.0 2969.0 6.02 6.07 3.7
0.52 Premium E VVS2 60.1 58.0 2970.0 5.23 5.18 3.13
0.72 Very Good G VS1 60.6 56.0 2970.0 5.84 5.87 3.55
0.7 Good F VS1 63.8 58.0 2970.0 5.58 5.61 3.57
0.78 Premium E VS2 62.6 57.0 2970.0 5.91 5.85 3.68
0.78 Ideal H VS2 61.6 56.0 2970.0 5.94 5.91 3.64
0.76 Ideal G VS1 59.4 57.0 2972.0 5.99 6.03 3.57
0.7 Ideal G VS1 61.7 56.0 2972.0 5.64 5.71 3.5
0.81 Premium H VS1 62.6 58.0 2972.0 5.96 5.9 3.71
0.75 Ideal G VS1 62.3 57.0 2973.0 5.83 5.86 3.64
0.7 Ideal E VS1 60.5 56.0 2973.0 5.74 5.79 3.49
0.7 Good E VS1 59.8 62.0 2973.0 5.74 5.8 3.45
0.71 Ideal G VS2 59.5 57.0 2974.0 5.81 5.8 3.46
0.7 Very Good F VS1 62.1 57.0 2975.0 5.69 5.72 3.54
0.7 Premium F VVS2 62.2 58.0 2975.0 5.72 5.66 3.54
0.83 Ideal H VS2 61.3 54.0 2975.0 6.1 6.06 3.73
0.71 Very Good G VVS2 60.8 58.0 2977.0 5.75 5.77 3.5
0.76 Premium D VS2 60.9 58.0 2977.0 5.9 5.85 3.58
0.54 Ideal F VVS1 61.6 55.0 2977.0 5.28 5.27 3.25
0.71 Ideal G VVS2 62.5 58.0 2978.0 5.7 5.73 3.57
0.7 Ideal E VS1 61.3 54.0 2978.0 5.77 5.83 3.54
0.72 Ideal H VVS1 59.9 59.0 2979.0 5.76 5.82 3.47
0.7 Ideal E VS2 61.7 56.0 2979.0 5.74 5.71 3.53
0.7 Ideal E VS2 61.5 57.0 2980.0 5.67 5.78 3.52
0.7 Ideal E VS2 62.2 55.0 2981.0 5.67 5.71 3.54
0.71 Ideal G VVS1 61.7 57.0 2982.0 5.7 5.75 3.53
0.71 Ideal E VS2 59.5 57.0 2982.0 5.86 5.83 3.48
0.71 Very Good G VS1 60.8 63.0 2982.0 5.76 5.68 3.48
0.71 Premium E VS2 62.6 58.0 2982.0 5.72 5.68 3.57
0.74 Ideal E VS2 62.7 54.0 2984.0 5.8 5.77 3.63
0.9 Very Good J VS2 63.1 57.0 2984.0 6.12 6.06 3.84
0.7 Very Good D VS2 63.1 56.0 2985.0 5.62 5.69 3.57
0.82 Premium H VS1 62.3 60.0 2985.0 5.97 5.94 3.71
0.77 Very Good G VS1 62.8 58.0 2986.0 5.78 5.84 3.65
0.8 Ideal I VS1 61.9 54.1 2986.0 5.92 5.98 3.69
0.82 Ideal I VS1 61.6 57.0 2986.0 6.0 6.05 3.71
0.7 Ideal G VS1 61.3 59.0 2987.0 5.68 5.7 3.49
0.72 Ideal F VS2 62.1 54.0 2989.0 5.76 5.8 3.59
0.76 Very Good G VS2 62.1 54.0 2990.0 5.88 5.94 3.67
0.72 Very Good E VS2 62.9 57.0 2990.0 5.68 5.73 3.59
0.57 Good E VVS1 59.1 65.0 2990.0 5.34 5.43 3.18
0.75 Ideal G VS2 60.6 55.0 2991.0 5.91 5.93 3.59
0.7 Ideal D VS2 60.3 60.0 2991.0 5.71 5.76 3.46
0.7 Very Good E VS2 62.8 56.0 2992.0 5.66 5.68 3.56
0.75 Ideal H VVS2 62.0 55.1 2992.0 5.83 5.85 3.62
0.69 Very Good F VVS2 61.5 60.0 2993.0 5.64 5.67 3.48
0.7 Ideal G VVS2 63.0 55.0 2993.0 5.65 5.69 3.57
0.7 Ideal F VS1 62.4 55.0 2993.0 5.65 5.7 3.54
0.71 Very Good F VS2 59.6 56.0 2994.0 5.84 5.88 3.49
0.71 Very Good G VS1 59.3 55.0 2994.0 5.88 5.95 3.51
0.81 Very Good G VS2 63.1 58.0 2994.0 5.88 5.84 3.7
0.81 Premium G VS2 62.0 58.0 2994.0 5.95 5.92 3.68
0.7 Ideal G VS1 60.9 56.0 2995.0 5.76 5.8 3.52
0.88 Very Good I VS1 63.3 55.0 2996.0 6.11 6.06 3.85
0.74 Ideal I VS2 61.9 55.0 2997.0 5.8 5.83 3.6
0.7 Ideal D VS2 62.8 57.0 2998.0 5.69 5.75 3.59
0.72 Ideal H VS1 61.4 56.0 2998.0 5.79 5.81 3.56
0.7 Ideal F VS1 61.6 57.0 2998.0 5.7 5.73 3.52
1.01 Fair J VVS2 66.0 56.0 2998.0 6.29 6.22 4.13
0.85 Fair G VS1 57.7 67.0 2998.0 6.26 6.19 3.59
0.7 Very Good D VS2 59.7 59.0 2999.0 5.82 5.78 3.46
0.73 Very Good G VS1 62.4 58.1 2999.0 5.71 5.75 3.58
0.7 Premium G VVS2 60.6 60.0 2999.0 5.77 5.69 3.47
0.74 Premium E VS1 62.7 58.0 2999.0 5.83 5.74 3.63
0.74 Premium E VS1 60.9 62.0 2999.0 5.83 5.8 3.54
0.7 Premium G VVS2 60.2 61.0 2999.0 5.74 5.66 3.43
0.93 Good J VS2 63.6 61.0 3000.0 6.16 6.08 3.89
0.7 Premium D VS1 61.6 61.0 3001.0 5.66 5.61 3.47
0.7 Good D VS1 63.6 60.0 3001.0 5.61 5.52 3.54
0.7 Very Good D VS1 63.4 59.0 3001.0 5.58 5.55 3.53
0.6 Ideal G VVS1 62.1 56.0 3001.0 5.42 5.43 3.37
0.75 Very Good H VVS2 60.6 57.0 3002.0 5.86 5.89 3.56
0.71 Premium D VS2 62.1 60.0 3002.0 5.68 5.72 3.54
0.72 Good F VS1 63.8 58.0 3002.0 5.68 5.63 3.61
0.72 Ideal G VVS2 61.6 55.0 3002.0 5.78 5.77 3.56
0.8 Premium G VS2 60.6 59.0 3002.0 6.02 5.97 3.63
0.73 Fair F VS1 58.6 66.0 3002.0 5.92 5.88 3.46
0.65 Premium D VVS2 59.9 58.0 3003.0 5.69 5.63 3.39
0.7 Ideal H VS1 61.7 55.0 3004.0 5.69 5.72 3.52
0.61 Ideal E VS1 61.3 54.0 3004.0 5.53 5.5 3.38
0.55 Ideal F VVS1 61.2 54.0 3005.0 5.3 5.35 3.26
0.72 Ideal I VS1 60.4 56.0 3005.0 5.8 5.86 3.52
0.73 Ideal H VVS1 61.6 57.0 3005.0 5.81 5.78 3.57
0.71 Premium E VS1 61.1 58.0 3006.0 5.8 5.76 3.53
0.71 Very Good E VS1 63.2 60.0 3006.0 5.63 5.6 3.55
0.55 Premium D VVS1 60.3 59.0 3006.0 5.34 5.3 3.21
0.71 Ideal I VVS2 60.7 57.0 3007.0 5.76 5.8 3.51
0.71 Ideal I VVS2 60.4 57.0 3007.0 5.78 5.81 3.5
0.7 Premium E VVS2 62.7 53.0 3007.0 5.65 5.61 3.53
0.7 Very Good D VS1 60.4 58.0 3008.0 5.71 5.78 3.47
0.61 Ideal G VVS1 61.2 56.0 3008.0 5.46 5.48 3.35
0.7 Ideal F VS2 61.3 57.0 3008.0 5.7 5.76 3.51
0.82 Premium H VS1 62.5 59.0 3008.0 5.96 5.94 3.72
0.71 Very Good E VS1 63.7 58.0 3009.0 5.63 5.68 3.6
0.71 Very Good E VS1 62.1 57.0 3009.0 5.67 5.69 3.53
0.71 Very Good E VS1 63.4 58.0 3009.0 5.64 5.68 3.59
0.8 Ideal I VS1 60.7 59.0 3010.0 5.98 6.02 3.64
0.73 Very Good G VS1 60.7 55.0 3011.0 5.87 5.89 3.57
0.61 Ideal E VVS2 62.0 54.0 3011.0 5.43 5.47 3.38
0.7 Ideal F VS2 61.9 55.0 3011.0 5.7 5.74 3.54
0.7 Ideal F VS2 61.8 57.0 3011.0 5.67 5.75 3.53
0.7 Ideal F VS2 62.7 55.0 3011.0 5.66 5.69 3.56
0.7 Ideal F VS2 61.4 58.0 3011.0 5.7 5.73 3.51
0.78 Very Good G VS2 61.3 60.0 3012.0 5.89 5.96 3.63
0.72 Ideal G VS2 61.7 56.0 3012.0 5.74 5.78 3.55
0.75 Premium F VS2 61.6 58.0 3013.0 5.84 5.89 3.61
0.71 Very Good F VS1 62.1 53.0 3013.0 5.7 5.77 3.56
0.71 Ideal F VS1 61.1 57.0 3013.0 5.76 5.82 3.54
0.71 Ideal H VVS1 61.8 56.0 3014.0 5.7 5.75 3.54
0.78 Ideal H VVS2 61.7 55.0 3015.0 5.9 5.94 3.65
0.72 Very Good D VS2 62.1 59.0 3016.0 5.7 5.73 3.55
0.7 Premium E VS1 61.8 58.0 3016.0 5.71 5.75 3.54
0.7 Ideal E VS1 62.7 57.0 3016.0 5.65 5.7 3.56
0.76 Ideal H VS2 61.9 55.0 3016.0 5.85 5.88 3.64
0.7 Very Good G VS1 60.1 60.0 3017.0 5.73 5.76 3.45
0.71 Very Good F VS1 61.8 60.0 3017.0 5.66 5.7 3.51
0.7 Ideal G VS1 61.1 56.0 3017.0 5.72 5.74 3.5
0.5 Good D VVS2 62.4 64.0 3017.0 5.03 5.06 3.14
0.7 Good F VVS1 63.2 58.0 3018.0 5.58 5.62 3.54
0.7 Premium F VVS2 62.5 59.0 3018.0 5.68 5.61 3.53
0.71 Ideal F VVS2 62.6 56.0 3018.0 5.7 5.65 3.55
0.72 Ideal H VS2 61.2 57.0 3018.0 5.79 5.77 3.54
0.7 Good E VS1 60.2 61.0 3018.0 5.71 5.75 3.45
0.77 Premium F VS2 62.4 59.0 3018.0 5.85 5.81 3.64
0.7 Premium F VVS2 62.2 56.0 3018.0 5.72 5.63 3.53
0.71 Ideal D VS2 60.4 53.0 3020.0 5.81 5.85 3.52
0.65 Ideal E VVS2 62.1 57.0 3023.0 5.55 5.6 3.46
0.75 Premium E VS2 62.1 57.0 3024.0 5.9 5.79 3.63
0.9 Very Good J VS2 63.1 59.0 3024.0 6.09 6.05 3.83
0.9 Good J VS2 63.9 58.0 3024.0 6.15 6.08 3.91
0.72 Premium E VS2 60.4 61.0 3024.0 5.79 5.76 3.49
0.72 Premium E VS2 62.5 59.0 3024.0 5.73 5.7 3.57
0.72 Very Good G VS1 60.1 63.0 3024.0 5.86 5.82 3.51
0.65 Very Good D VVS2 57.7 60.0 3025.0 5.69 5.74 3.3
0.7 Very Good G VS2 61.8 55.0 3026.0 5.69 5.74 3.53
0.59 Ideal E VVS2 61.8 57.0 3026.0 5.35 5.4 3.32
0.71 Ideal E VS2 62.3 56.0 3026.0 5.7 5.73 3.56
0.83 Ideal H VS2 61.3 54.0 3027.0 6.06 6.1 3.73
0.77 Good H VVS2 57.9 61.0 3027.0 6.07 6.01 3.5
0.7 Very Good F VVS2 58.5 60.0 3028.0 5.82 5.94 3.44
0.8 Ideal H VS2 62.1 54.0 3030.0 5.96 5.99 3.71
0.74 Ideal H VS1 61.6 55.0 3030.0 5.79 5.83 3.58
0.77 Fair F VS1 66.8 57.0 3031.0 5.66 5.76 3.82
0.72 Premium G VS1 58.9 58.0 3032.0 5.93 5.85 3.47
0.55 Ideal F VVS1 61.2 54.0 3032.0 5.35 5.3 3.26
0.71 Very Good D VS2 63.0 57.0 3033.0 5.67 5.7 3.58
0.73 Ideal G VS1 61.6 57.0 3033.0 5.76 5.79 3.56
0.7 Good D VS2 64.1 59.0 3033.0 5.56 5.49 3.54
0.7 Very Good D VS2 63.2 60.0 3033.0 5.61 5.56 3.53
0.7 Good D VS2 63.9 58.0 3033.0 5.62 5.58 3.58
0.92 Fair I VS2 64.4 58.0 3033.0 6.13 6.1 3.94
0.7 Ideal G VS1 61.4 57.0 3034.0 5.7 5.73 3.51
0.72 Very Good E VS2 63.8 57.0 3035.0 5.66 5.69 3.62
0.71 Ideal E VS2 59.5 57.0 3035.0 5.83 5.86 3.48
0.72 Ideal G VS1 62.4 59.0 3035.0 5.71 5.74 3.57
0.8 Very Good H VVS2 62.9 56.0 3036.0 5.9 5.96 3.73
0.74 Ideal E VS2 62.6 56.0 3036.0 5.73 5.81 3.61
0.61 Ideal D VVS2 62.4 58.0 3036.0 5.38 5.42 3.37
0.7 Very Good G VVS1 63.3 57.0 3037.0 5.59 5.63 3.55
0.32 Premium G VS2 60.5 58.0 561.0 4.41 4.42 2.67
0.32 Premium G VS2 62.5 60.0 561.0 4.32 4.38 2.72
0.32 Ideal G VS2 61.4 56.0 561.0 4.37 4.39 2.69
0.32 Premium G VS2 59.8 59.0 561.0 4.48 4.52 2.69
0.32 Premium I VVS2 60.7 59.0 561.0 4.4 4.43 2.68
0.32 Very Good G VS2 60.2 57.0 561.0 4.42 4.45 2.67
0.32 Good G VS2 63.3 54.0 561.0 4.36 4.39 2.77
0.32 Good H VS1 63.1 57.0 561.0 4.34 4.37 2.75
0.32 Ideal G VS2 61.4 55.0 561.0 4.4 4.46 2.72
0.32 Ideal G VS2 59.8 57.0 561.0 4.43 4.46 2.66
0.32 Ideal G VS2 61.7 57.0 561.0 4.38 4.4 2.71
0.32 Premium H VS1 62.3 58.0 561.0 4.34 4.39 2.72
0.32 Very Good H VS1 63.0 57.0 561.0 4.32 4.35 2.73
0.32 Premium G VS2 61.9 58.0 561.0 4.36 4.43 2.72
0.32 Good G VS2 63.1 57.0 561.0 4.3 4.35 2.73
0.32 Very Good H VS1 63.0 57.0 561.0 4.37 4.39 2.76
0.32 Ideal G VS2 61.8 57.0 561.0 4.37 4.4 2.71
0.32 Very Good H VS1 61.7 58.0 561.0 4.37 4.41 2.71
0.32 Premium H VS1 61.7 58.0 561.0 4.38 4.44 2.72
0.32 Ideal G VS2 61.8 55.0 561.0 4.41 4.42 2.73
0.32 Premium G VS2 61.7 60.0 561.0 4.32 4.4 2.69
0.32 Very Good G VS2 62.6 58.0 561.0 4.37 4.39 2.74
0.32 Premium G VS2 62.3 58.0 561.0 4.36 4.41 2.73
0.32 Ideal G VS2 61.6 57.0 561.0 4.39 4.41 2.71
0.32 Ideal H VS1 61.9 55.0 561.0 4.4 4.42 2.73
0.32 Ideal H VS1 60.2 56.0 561.0 4.44 4.49 2.69
0.76 Ideal H VS2 61.4 57.0 3038.0 5.85 5.88 3.6
0.7 Ideal H VS2 61.5 56.0 3038.0 5.71 5.73 3.52
0.7 Very Good G VVS2 61.0 59.0 3039.0 5.67 5.7 3.47
0.7 Fair F VS1 64.9 59.0 3039.0 5.56 5.59 3.62
0.73 Ideal G VS1 61.8 57.0 3041.0 5.78 5.81 3.58
0.71 Ideal F VS1 62.7 57.0 3041.0 5.66 5.7 3.56
0.71 Ideal F VS1 61.7 55.0 3041.0 5.73 5.77 3.55
0.81 Good I VS1 59.4 56.0 3042.0 5.97 6.11 3.59
0.71 Ideal G VVS2 62.5 57.0 3042.0 5.73 5.7 3.57
0.72 Very Good G VVS2 60.4 58.0 3043.0 5.77 5.82 3.5
0.71 Very Good F VS1 62.2 55.0 3045.0 5.68 5.74 3.56
0.71 Very Good F VS1 61.2 57.0 3045.0 5.73 5.77 3.52
0.71 Very Good D VS2 62.8 56.0 3045.0 5.67 5.7 3.57
0.72 Premium D VS2 60.2 60.0 3045.0 5.76 5.81 3.48
0.7 Good G VVS2 61.1 61.0 3046.0 5.67 5.69 3.47
0.73 Fair D VS1 66.0 54.0 3047.0 5.56 5.66 3.7
0.72 Good E VS1 57.9 60.0 3048.0 5.97 5.91 3.44
0.72 Very Good E VS1 63.1 56.0 3048.0 5.7 5.65 3.58
0.9 Ideal J VS1 62.6 55.0 3048.0 6.13 6.11 3.83
0.66 Ideal D VVS2 61.6 57.0 3049.0 5.64 5.57 3.45
0.62 Very Good D VVS2 58.1 63.0 3050.0 5.59 5.66 3.27
0.7 Very Good D VS2 62.5 55.0 3052.0 5.65 5.71 3.55
0.77 Ideal F VS2 61.2 57.0 3052.0 5.93 5.97 3.64
0.7 Very Good G VVS2 60.2 61.0 3052.0 5.66 5.74 3.43
0.7 Very Good D VS2 62.6 58.0 3053.0 5.67 5.7 3.56
0.71 Very Good E VS2 59.9 59.0 3053.0 5.79 5.83 3.48
0.7 Very Good F VS1 62.8 59.0 3053.0 5.65 5.69 3.56
0.71 Ideal E VS2 60.9 56.0 3053.0 5.77 5.83 3.53
0.79 Premium G VS1 62.3 56.0 3053.0 5.94 5.87 3.68
0.79 Premium G VS1 61.3 59.0 3053.0 5.97 5.91 3.64
0.7 Very Good D VS1 62.9 60.0 3054.0 5.62 5.67 3.55
0.65 Very Good D VVS2 59.9 58.0 3056.0 5.63 5.69 3.39
0.61 Ideal E VVS2 60.8 56.0 3056.0 5.5 5.47 3.34
0.57 Ideal F VVS1 61.1 55.0 3057.0 5.36 5.44 3.3
0.76 Good F VS1 59.9 61.0 3057.0 5.89 5.98 3.56
0.91 Premium J VS2 61.6 58.0 3058.0 6.28 6.23 3.85
0.72 Very Good F VS1 62.1 59.0 3059.0 5.69 5.74 3.55
0.71 Very Good E VS1 61.8 56.0 3059.0 5.74 5.78 3.56
0.74 Very Good H VVS1 62.4 57.0 3061.0 5.76 5.81 3.61
0.7 Very Good E VS1 61.1 55.0 3061.0 5.72 5.77 3.51
0.71 Very Good E VS1 63.3 56.0 3061.0 5.64 5.68 3.58
0.71 Fair G VVS1 62.8 57.0 3062.0 5.67 5.57 3.53
0.7 Premium F VVS2 58.7 60.0 3062.0 5.8 5.75 3.39
0.71 Premium E VS2 62.2 59.0 3062.0 5.71 5.61 3.52
0.71 Premium E VS2 62.0 61.0 3062.0 5.71 5.65 3.52
0.93 Premium J VS1 60.3 58.0 3062.0 6.37 6.31 3.82
0.7 Very Good E VS1 62.2 57.0 3063.0 5.63 5.68 3.52
0.7 Very Good E VS1 62.5 56.0 3063.0 5.64 5.68 3.54
0.7 Good E VS1 59.4 61.0 3063.0 5.79 5.83 3.45
0.71 Very Good E VS1 63.3 59.0 3064.0 5.64 5.68 3.58
0.76 Premium E VS2 61.7 62.0 3064.0 5.85 5.82 3.6
0.7 Ideal F VS2 61.4 56.0 3064.0 5.72 5.75 3.52
0.7 Ideal F VS2 61.6 55.0 3064.0 5.72 5.75 3.53
0.72 Very Good E VS2 63.0 58.0 3065.0 5.69 5.73 3.6
0.7 Ideal G VS1 61.5 56.0 3065.0 5.7 5.75 3.52
0.77 Ideal I VS1 61.4 56.0 3066.0 5.9 5.93 3.63
0.71 Ideal F VS1 62.0 57.0 3066.0 5.7 5.75 3.55
0.71 Ideal F VS1 62.1 57.0 3066.0 5.73 5.76 3.57
0.73 Very Good E VS2 63.1 55.0 3066.0 5.77 5.71 3.62
0.7 Very Good E VS1 63.4 60.0 3068.0 5.63 5.66 3.58
0.7 Ideal E VS2 62.6 56.0 3068.0 5.65 5.69 3.55
0.85 Very Good I VS2 60.0 57.0 3070.0 6.1 6.16 3.68
0.82 Ideal I VS1 61.6 56.0 3071.0 6.05 6.01 3.72
0.71 Good G VVS1 62.7 61.0 3072.0 5.64 5.68 3.55
0.7 Very Good G VVS1 63.1 56.0 3073.0 5.64 5.67 3.57
0.7 Ideal G VVS1 61.6 55.0 3073.0 5.72 5.75 3.53
0.75 Ideal G VS2 61.6 55.0 3073.0 5.86 5.89 3.62
0.71 Ideal E VS2 62.2 57.0 3073.0 5.69 5.73 3.55
0.62 Premium E VVS1 61.9 59.0 3073.0 5.62 5.5 3.44
0.7 Good D VS2 58.0 65.0 3073.0 5.81 5.73 3.39
0.78 Very Good G VS2 61.7 58.0 3074.0 5.87 5.92 3.64
0.9 Fair I VVS2 67.0 56.0 3074.0 5.91 5.83 3.93
0.77 Ideal H VS1 61.4 55.0 3074.0 5.89 5.93 3.63
0.72 Very Good D VS2 61.8 58.0 3075.0 5.73 5.76 3.55
0.72 Very Good D VS2 62.6 59.0 3075.0 5.69 5.72 3.57
0.72 Ideal H VVS1 62.2 57.0 3075.0 5.72 5.75 3.57
0.76 Ideal I VS2 61.7 56.0 3075.0 5.87 5.9 3.63
0.73 Ideal E VS2 62.7 56.0 3077.0 5.75 5.8 3.62
0.71 Fair D VS2 64.7 58.0 3077.0 5.61 5.58 3.62
0.71 Premium D VS2 60.3 62.0 3077.0 5.76 5.69 3.45
0.72 Premium E VS2 62.5 59.0 3078.0 5.7 5.73 3.57
0.76 Ideal E VS2 61.3 56.0 3079.0 5.79 5.83 3.56
// Combining conditions
display(spark.sql("SELECT * FROM diamonds WHERE clarity LIKE 'V%' AND price > 10000"))
carat cut color clarity depth table price x y z
1.7 Ideal J VS2 60.5 58.0 10002.0 7.73 7.74 4.68
1.03 Ideal E VVS2 60.6 59.0 10003.0 6.5 6.53 3.95
1.23 Very Good G VVS2 60.6 55.0 10004.0 6.93 7.02 4.23
1.25 Ideal F VS2 61.6 55.0 10006.0 6.93 6.96 4.28
1.21 Very Good F VS1 62.3 58.0 10009.0 6.76 6.85 4.24
1.51 Premium I VS2 59.9 60.0 10010.0 7.42 7.36 4.43
1.05 Ideal F VVS2 60.5 55.0 10011.0 6.67 6.58 4.01
1.6 Ideal J VS1 62.0 53.0 10011.0 7.57 7.56 4.69
1.35 Premium G VS1 62.1 59.0 10012.0 7.06 7.02 4.37
1.53 Premium I VS2 62.0 58.0 10013.0 7.36 7.41 4.58
1.13 Ideal F VS1 60.9 57.0 10016.0 6.73 6.76 4.11
1.21 Premium F VS1 62.6 59.0 10018.0 6.81 6.76 4.25
1.01 Very Good F VVS1 62.9 57.0 10019.0 6.35 6.41 4.01
1.04 Ideal E VVS2 62.9 55.0 10019.0 6.47 6.51 4.08
1.26 Very Good G VVS2 60.9 56.0 10020.0 6.95 7.01 4.25
1.5 Very Good H VS2 60.9 59.0 10023.0 7.37 7.43 4.51
1.12 Premium F VVS2 62.4 59.0 10028.0 6.58 6.66 4.13
1.27 Premium F VS1 60.3 58.0 10028.0 7.06 7.04 4.25
1.52 Very Good I VS1 62.9 59.9 10032.0 7.27 7.31 4.59
1.24 Premium F VS1 62.5 58.0 10033.0 6.87 6.83 4.28
1.23 Very Good F VS1 62.0 59.0 10035.0 6.84 6.87 4.25
1.5 Good G VS1 63.6 57.0 10036.0 7.23 7.14 4.57
1.22 Ideal G VVS2 62.3 56.0 10038.0 6.81 6.84 4.25
1.3 Ideal G VS1 62.0 55.0 10038.0 6.98 7.02 4.34
1.59 Premium I VS2 60.2 60.0 10039.0 7.58 7.61 4.57
1.83 Premium I VS2 60.5 60.0 10043.0 7.93 7.86 4.78
1.07 Ideal E VVS2 61.4 56.0 10043.0 6.65 6.55 4.05
1.51 Very Good H VS1 61.5 54.0 10045.0 7.34 7.42 4.54
1.08 Ideal F VVS2 61.6 57.0 10046.0 6.57 6.6 4.06
1.0 Premium D VVS2 61.6 60.0 10046.0 6.41 6.36 3.93
1.03 Ideal F VVS2 61.1 57.0 10049.0 6.51 6.54 3.99
1.52 Very Good I VS2 62.3 58.0 10051.0 7.32 7.28 4.55
1.08 Ideal F VVS2 62.1 55.0 10052.0 6.57 6.6 4.09
1.2 Premium G VVS2 62.8 59.0 10053.0 6.72 6.65 4.2
1.2 Premium E VS1 60.7 57.0 10053.0 6.89 6.81 4.16
1.2 Premium G VVS2 61.2 58.0 10053.0 6.88 6.84 4.2
1.71 Premium I VS1 60.3 62.0 10055.0 7.76 7.7 4.66
1.0 Ideal F VVS1 62.3 53.0 10058.0 6.37 6.43 3.99
1.07 Ideal F VVS2 62.3 57.0 10061.0 6.56 6.58 4.09
1.66 Premium J VVS2 62.6 59.0 10062.0 7.58 7.54 4.73
1.2 Premium F VVS2 60.5 60.0 10064.0 6.98 6.87 4.19
1.11 Very Good F VVS1 62.5 59.0 10069.0 6.59 6.63 4.13
1.34 Ideal G VS1 62.7 57.0 10070.0 7.1 7.04 4.43
1.31 Premium G VS1 61.5 59.0 10071.0 7.06 7.0 4.32
1.31 Ideal G VS1 62.2 56.0 10071.0 7.05 7.01 4.37
1.31 Ideal G VS1 61.5 57.0 10071.0 7.06 7.02 4.33
1.53 Very Good H VS1 59.5 63.0 10076.0 7.51 7.44 4.45
1.26 Premium F VS1 62.7 58.0 10076.0 6.93 6.86 4.32
1.73 Ideal J VS2 63.0 57.0 10076.0 7.64 7.6 4.8
1.19 Ideal D VS1 61.1 57.0 10079.0 6.84 6.87 4.19
1.5 Ideal I VS1 61.3 57.0 10080.0 7.35 7.32 4.5
1.5 Premium I VS1 62.7 59.0 10080.0 7.3 7.25 4.56
1.5 Ideal H VS1 61.3 55.0 10080.0 7.37 7.34 4.51
1.21 Premium D VS1 60.2 59.0 10083.0 6.89 6.86 4.14
1.71 Premium H VS2 59.2 61.0 10084.0 7.83 7.77 4.62
1.82 Very Good J VS1 62.2 56.0 10090.0 7.83 7.96 4.91
1.51 Very Good H VS2 61.9 57.0 10090.0 7.32 7.36 4.54
1.3 Ideal F VS2 62.2 56.0 10090.0 6.98 6.94 4.33
1.3 Premium F VS2 60.4 59.0 10090.0 7.12 7.06 4.28
1.5 Very Good I VVS2 63.3 58.0 10090.0 7.27 7.24 4.59
1.57 Ideal I VS2 61.5 56.0 10093.0 7.56 7.49 4.63
1.07 Ideal F VVS2 60.3 55.0 10093.0 6.65 6.68 4.02
1.31 Very Good E VS2 63.1 56.0 10094.0 6.95 6.9 4.37
1.33 Good G VS1 62.8 60.0 10096.0 6.87 6.92 4.33
1.53 Premium I VS1 61.2 59.0 10098.0 7.39 7.41 4.53
1.61 Ideal I VS2 62.5 57.0 10098.0 7.49 7.43 4.66
1.31 Ideal G VS1 61.9 56.0 10099.0 7.03 7.13 4.38
1.22 Ideal F VS1 62.3 57.0 10100.0 6.83 6.79 4.24
1.07 Ideal E VVS2 61.7 57.0 10104.0 6.55 6.61 4.06
1.59 Very Good I VS2 60.5 63.0 10106.0 7.52 7.45 4.53
1.22 Premium G VVS2 62.0 58.0 10111.0 6.9 6.85 4.26
1.09 Premium E VVS2 59.9 59.0 10111.0 6.73 6.7 4.02
1.58 Very Good I VS1 61.8 57.0 10112.0 7.5 7.56 4.64
1.0 Very Good D VVS2 61.7 58.0 10113.0 6.37 6.41 3.94
1.23 Ideal G VVS1 63.2 56.0 10113.0 6.78 6.83 4.3
1.25 Ideal D VS2 62.6 56.0 10114.0 6.87 6.84 4.29
1.17 Premium D VS1 61.7 59.0 10115.0 6.77 6.72 4.16
1.28 Ideal G VS1 62.1 57.0 10126.0 6.91 6.94 4.3
1.43 Ideal H VVS2 61.6 54.0 10129.0 7.25 7.29 4.48
1.51 Good H VS1 59.9 61.0 10129.0 7.34 7.39 4.41
1.52 Very Good I VS2 61.7 55.0 10130.0 7.39 7.32 4.54
1.04 Very Good D VVS2 60.8 58.0 10130.0 6.49 6.53 3.96
1.07 Ideal E VVS2 62.3 56.0 10133.0 6.51 6.61 4.09
1.5 Good F VS2 64.0 56.0 10134.0 7.18 7.13 4.64
1.0 Premium E VVS1 60.3 54.0 10134.0 6.59 6.47 3.94
1.21 Premium E VS1 60.3 58.0 10137.0 6.95 6.91 4.18
1.24 Ideal F VS1 61.5 54.0 10138.0 6.93 6.89 4.25
1.24 Ideal F VS1 60.9 54.0 10138.0 6.98 6.95 4.24
1.11 Very Good F VVS1 59.7 55.0 10141.0 6.77 6.82 4.06
1.1 Ideal D VS1 61.9 56.0 10144.0 6.58 6.61 4.09
1.01 Premium D VVS2 60.2 58.0 10147.0 6.57 6.51 3.94
1.31 Ideal G VS1 60.5 57.0 10155.0 7.1 7.14 4.31
1.2 Premium D VS2 61.1 58.0 10161.0 6.85 6.83 4.18
1.5 Very Good I VS1 62.2 59.0 10164.0 7.27 7.3 4.53
1.54 Premium I VS1 61.6 58.0 10164.0 7.39 7.42 4.56
1.54 Good I VS1 63.6 60.0 10164.0 7.3 7.33 4.65
1.5 Ideal I VS1 62.0 54.0 10164.0 7.32 7.38 4.56
1.67 Very Good I VS2 60.7 60.0 10165.0 7.61 7.68 4.64
1.7 Very Good J VS1 62.9 58.0 10165.0 7.54 7.67 4.79
1.53 Ideal I VS1 60.2 60.0 10171.0 7.51 7.48 4.51
1.2 Very Good F VVS2 63.8 58.0 10173.0 6.67 6.69 4.26
1.21 Ideal F VS2 61.5 54.0 10177.0 6.88 6.89 4.24
1.01 Good G VS2 63.6 56.0 10181.0 6.31 6.24 3.99
1.24 Very Good E VS1 62.0 58.0 10185.0 6.9 6.96 4.3
1.51 Ideal H VS1 61.2 58.0 10186.0 7.36 7.42 4.52
1.35 Ideal G VS1 61.5 56.0 10193.0 7.12 7.15 4.39
1.53 Premium I VS2 62.0 58.0 10196.0 7.41 7.36 4.58
1.09 Ideal F VVS2 62.0 56.0 10196.0 6.63 6.6 4.1
1.01 Ideal F VVS1 60.5 60.0 10197.0 6.45 6.47 3.91
1.58 Ideal I VS2 61.4 55.0 10197.0 7.49 7.55 4.62
1.24 Premium G VVS2 59.9 60.0 10202.0 6.98 7.0 4.19
1.24 Very Good E VS1 59.9 61.0 10202.0 6.96 6.99 4.18
1.27 Premium G VVS2 61.0 58.0 10203.0 6.96 7.01 4.26
1.08 Very Good F VVS1 61.0 58.0 10204.0 6.64 6.61 4.04
1.5 Very Good H VS2 63.4 57.0 10206.0 7.27 7.2 4.59
1.5 Fair H VS2 65.2 58.0 10206.0 7.12 7.06 4.62
1.57 Ideal I VS1 62.3 57.0 10209.0 7.44 7.48 4.65
1.12 Premium F VVS2 62.4 59.0 10211.0 6.66 6.58 4.13
1.52 Ideal I VS1 62.9 60.0 10214.0 7.31 7.27 4.59
1.2 Ideal E VS2 61.3 56.0 10214.0 6.89 6.84 4.21
1.51 Very Good I VS1 61.1 61.0 10215.0 7.32 7.37 4.49
1.01 Premium D VVS2 62.4 60.0 10221.0 6.31 6.36 3.95
1.3 Ideal G VS1 62.0 55.0 10221.0 7.02 6.98 4.34
1.22 Ideal G VVS2 62.3 56.0 10221.0 6.84 6.81 4.25
1.07 Ideal E VVS2 61.3 56.0 10222.0 6.53 6.6 4.02
1.59 Premium I VS2 60.2 60.0 10222.0 7.61 7.58 4.57
1.53 Premium H VS2 59.3 59.0 10224.0 7.53 7.59 4.48
1.53 Premium H VS2 59.8 58.0 10224.0 7.49 7.52 4.49
1.51 Very Good H VS2 62.8 58.0 10225.0 7.21 7.28 4.55
1.37 Very Good G VS1 58.3 60.0 10226.0 7.3 7.35 4.27
1.21 Ideal G VVS1 60.2 57.0 10232.0 7.02 6.94 4.2
1.12 Ideal F VVS2 60.4 56.0 10236.0 6.82 6.78 4.11
1.6 Very Good I VS1 62.3 59.0 10238.0 7.46 7.51 4.66
1.16 Ideal D VS1 61.2 58.0 10241.0 6.73 6.76 4.13
1.31 Ideal F VS2 59.6 60.0 10243.0 7.06 7.16 4.24
1.35 Ideal G VS1 62.2 57.0 10244.0 7.09 7.05 4.4
1.21 Premium F VS1 61.9 58.0 10245.0 6.82 6.76 4.2
1.09 Ideal F VVS2 62.1 56.0 10246.0 6.55 6.59 4.08
1.34 Ideal H VVS1 62.1 56.0 10255.0 7.05 7.11 4.4
1.5 Good H VS1 63.4 59.0 10256.0 7.2 7.29 4.59
1.21 Ideal G VVS2 60.6 56.0 10256.0 6.9 6.89 4.18
1.14 Premium E VVS2 59.7 58.0 10258.0 6.83 6.91 4.1
1.07 Ideal D VVS2 61.3 58.0 10266.0 6.55 6.64 4.04
1.23 Very Good F VS1 60.8 58.0 10276.0 6.9 6.94 4.21
1.57 Ideal I VS2 62.7 56.0 10278.0 7.36 7.4 4.63
1.53 Premium I VS1 61.2 59.0 10282.0 7.41 7.39 4.53
1.21 Very Good F VS2 60.1 58.0 10283.0 6.85 6.92 4.14
1.01 Ideal E VVS2 61.4 56.0 10283.0 6.49 6.45 3.97
1.26 Very Good F VS1 62.5 58.0 10284.0 6.83 6.94 4.3
1.25 Very Good E VS1 61.5 59.0 10285.0 6.91 6.95 4.26
1.55 Ideal I VS1 61.2 55.0 10286.0 7.49 7.47 4.58
1.07 Ideal E VVS2 61.7 57.0 10288.0 6.61 6.55 4.06
1.5 Good H VS2 63.6 58.0 10291.0 7.22 7.27 4.61
1.5 Good H VS2 61.2 61.0 10291.0 7.25 7.32 4.46
1.5 Premium H VS2 62.2 58.0 10291.0 7.27 7.36 4.55
1.5 Premium H VS2 60.8 59.0 10291.0 7.34 7.36 4.47
1.08 Ideal E VS1 61.7 55.0 10292.0 6.58 6.61 4.07
1.21 Ideal G VVS1 61.0 57.0 10295.0 6.87 6.93 4.21
1.08 Ideal E VVS2 62.5 57.0 10300.0 6.52 6.57 4.09
1.52 Premium I VS1 60.6 57.0 10300.0 7.51 7.44 4.53
1.46 Ideal G VS2 62.3 56.0 10302.0 7.28 7.2 4.51
1.26 Premium G VVS2 62.7 58.0 10302.0 6.95 6.86 4.33
1.23 Ideal G VVS2 60.3 57.0 10304.0 6.98 6.97 4.21
1.23 Ideal G VVS2 61.0 57.0 10304.0 6.93 6.9 4.22
1.12 Ideal G VVS2 61.5 57.0 10305.0 6.65 6.67 4.1
1.18 Ideal G VVS2 61.3 55.0 10308.0 6.86 6.81 4.19
1.59 Very Good I VS2 61.1 58.6 10309.0 7.49 7.53 4.59
1.71 Ideal J VS1 62.4 56.0 10309.0 7.59 7.63 4.75
1.4 Ideal G VS2 61.7 56.0 10311.0 7.2 7.25 4.46
1.86 Ideal J VS2 62.6 56.0 10312.0 7.95 7.87 4.95
1.08 Ideal E VVS2 61.8 56.0 10313.0 6.55 6.59 4.06
1.09 Ideal E VVS2 61.6 56.0 10314.0 6.6 6.64 4.08
1.04 Premium D VVS2 60.8 58.0 10314.0 6.53 6.49 3.96
1.15 Ideal F VS1 61.1 55.0 10316.0 6.76 6.82 4.15
1.23 Ideal G VVS2 62.2 55.0 10317.0 6.86 6.9 4.28
1.23 Ideal G VVS2 62.7 56.0 10317.0 6.81 6.84 4.28
1.51 Very Good H VS2 63.0 57.0 10319.0 7.25 7.3 4.58
1.27 Very Good G VVS2 61.5 58.0 10321.0 6.9 6.96 4.26
1.13 Ideal F VVS2 61.4 56.0 10327.0 6.77 6.72 4.14
1.1 Very Good F VVS1 62.2 59.0 10329.0 6.56 6.69 4.12
1.56 Premium H VS2 62.4 58.0 10331.0 7.44 7.39 4.63
1.09 Ideal E VVS2 60.9 56.0 10333.0 6.66 6.7 4.07
1.56 Ideal I VS2 61.8 56.0 10333.0 7.41 7.45 4.59
1.7 Premium H VS2 60.6 58.0 10333.0 7.72 7.65 4.66
1.7 Ideal H VS2 62.8 55.0 10333.0 7.61 7.54 4.76
1.7 Premium H VS2 59.0 58.0 10333.0 7.77 7.72 4.57
1.21 Ideal D VS1 61.0 57.0 10335.0 6.88 6.85 4.19
1.57 Very Good I VS1 59.7 61.0 10336.0 7.51 7.62 4.52
1.7 Premium H VVS2 61.4 58.0 10337.0 7.66 7.62 4.69
1.58 Ideal I VS2 61.1 55.0 10338.0 7.48 7.53 4.59
1.42 Premium F VS1 58.4 59.0 10338.0 7.36 7.32 4.29
1.29 Premium F VS1 60.9 58.0 10341.0 6.97 7.01 4.26
1.01 Premium F VVS1 60.8 58.0 10341.0 6.55 6.48 3.96
1.27 Ideal G VVS2 62.4 53.3 10342.0 6.94 6.95 4.33
1.17 Ideal G VVS1 61.7 57.0 10342.0 6.84 6.9 4.13
1.23 Ideal F VS2 62.4 54.0 10342.0 6.84 6.87 4.28
1.6 Very Good I VS2 60.0 58.0 10346.0 7.61 7.68 4.59
1.54 Premium I VS1 61.6 58.0 10349.0 7.42 7.39 4.56
1.54 Very Good I VS1 61.1 63.0 10349.0 7.43 7.36 4.52
1.54 Good I VS1 63.6 60.0 10349.0 7.33 7.3 4.65
1.04 Premium E VVS1 62.5 59.0 10350.0 6.41 6.46 4.02
1.1 Ideal D VS2 61.7 56.0 10350.0 6.63 6.67 4.1
1.4 Very Good G VS2 60.1 62.0 10351.0 7.16 7.25 4.33
1.17 Ideal F VVS2 61.9 54.0 10351.0 6.76 6.82 4.2
1.21 Ideal E VS1 62.4 57.0 10351.0 6.75 6.83 4.24
1.21 Ideal D VS2 60.9 60.0 10353.0 6.91 6.86 4.19
1.58 Ideal J VVS1 61.5 56.0 10357.0 7.48 7.5 4.61
1.31 Ideal G VS1 62.0 58.0 10359.0 6.97 7.02 4.34
1.62 Premium I VS1 61.7 59.0 10362.0 7.55 7.47 4.63
1.26 Premium E VS1 60.7 58.0 10367.0 7.02 7.04 4.27
1.26 Ideal G VVS2 60.7 56.0 10367.0 7.03 7.05 4.27
1.2 Ideal F VS1 62.1 58.0 10367.0 6.78 6.84 4.23
1.26 Ideal G VS1 62.2 54.0 10371.0 6.89 6.98 4.31
1.4 Very Good G VS2 62.2 61.0 10378.0 7.09 7.13 4.42
1.21 Ideal G VVS2 62.0 56.0 10378.0 6.8 6.84 4.23
1.35 Ideal G VS1 61.5 56.0 10378.0 7.15 7.12 4.39
1.55 Ideal I VS1 61.5 54.0 10384.0 7.42 7.58 4.61
1.02 Ideal E VVS2 61.5 57.0 10384.0 6.55 6.5 4.01
1.1 Premium E VVS2 60.1 58.0 10387.0 6.69 6.76 4.04
1.24 Premium E VS1 59.9 61.0 10388.0 6.99 6.96 4.18
1.24 Premium G VVS2 59.9 60.0 10388.0 7.0 6.98 4.19
1.27 Premium G VVS2 61.0 58.0 10389.0 7.01 6.96 4.26
1.52 Ideal I VS1 60.7 60.0 10392.0 7.4 7.42 4.5
1.24 Ideal G VVS2 61.4 58.0 10395.0 6.88 6.92 4.24
1.13 Ideal G VVS2 61.6 57.0 10396.0 6.66 6.75 4.13
1.7 Premium I VS2 61.9 56.0 10396.0 7.55 7.5 4.66
1.56 Ideal I VS1 61.2 59.0 10399.0 7.43 7.5 4.57
2.01 Premium J VS2 58.6 61.0 10401.0 8.18 8.14 4.78
1.51 Premium I VS1 61.1 61.0 10401.0 7.37 7.32 4.49
1.5 Ideal I VS1 62.2 54.0 10406.0 7.33 7.4 4.57
1.01 Premium D VVS2 62.4 60.0 10407.0 6.36 6.31 3.95
1.31 Very Good D VS2 59.5 61.0 10409.0 7.16 7.19 4.27
1.1 Ideal D VVS2 61.0 56.0 10410.0 6.67 6.73 4.09
1.51 Premium H VS2 60.6 58.0 10411.0 7.42 7.36 4.48
1.11 Ideal F VVS1 61.9 57.0 10412.0 6.62 6.66 4.11
1.37 Premium G VS1 58.3 60.0 10412.0 7.35 7.3 4.27
1.35 Premium G VS1 61.0 59.0 10415.0 7.15 7.09 4.34
1.55 Premium I VS1 58.2 60.0 10416.0 7.69 7.59 4.45
1.43 Very Good G VS2 62.2 58.0 10419.0 7.15 7.18 4.46
1.32 Premium F VS2 60.9 59.0 10423.0 7.12 7.06 4.32
1.56 Premium H VS2 62.2 58.0 10424.0 7.41 7.44 4.62
1.5 Very Good H VS2 58.9 59.0 10424.0 7.34 7.43 4.35
1.4 Ideal G VS2 62.1 55.0 10427.0 7.12 7.05 4.4
1.58 Premium I VS1 61.1 59.0 10428.0 7.44 7.52 4.57
1.52 Good H VS2 63.3 57.0 10428.0 7.32 7.33 4.64
1.71 Very Good J VS1 61.9 59.0 10428.0 7.6 7.69 4.73
1.75 Premium J VS1 62.2 59.0 10429.0 7.7 7.74 4.8
1.21 Very Good E VS1 60.0 58.0 10430.0 6.89 6.97 4.16
1.34 Premium F VS2 61.1 58.0 10431.0 7.12 7.05 4.33
1.23 Very Good G VVS1 61.3 57.0 10435.0 6.88 6.96 4.24
1.19 Ideal F VVS2 61.7 56.0 10436.0 6.82 6.85 4.22
1.29 Premium F VS1 59.3 60.0 10437.0 7.14 7.1 4.22
1.16 Ideal F VVS2 61.8 56.7 10439.0 6.7 6.78 4.16
1.25 Ideal D VS1 61.7 56.0 10441.0 6.92 7.01 4.3
1.11 Ideal D VS2 61.2 57.0 10443.0 6.69 6.71 4.1
1.23 Ideal G VVS2 61.3 56.0 10445.0 6.89 6.91 4.23
1.14 Premium E VVS2 59.7 58.0 10446.0 6.91 6.83 4.1
1.57 Very Good I VS2 60.3 58.0 10447.0 7.58 7.55 4.56
1.19 Ideal F VS1 60.5 57.0 10449.0 6.82 6.88 4.15
1.01 Very Good D VVS2 62.5 59.0 10453.0 6.34 6.4 3.98
1.07 Ideal E VVS2 61.8 54.0 10453.0 6.56 6.61 4.07
1.2 Ideal G VVS2 61.1 56.0 10454.0 6.88 6.91 4.21
1.36 Ideal G VS1 61.0 56.0 10455.0 7.13 7.11 4.34
1.71 Premium H VS1 62.1 59.0 10457.0 7.63 7.55 4.71
1.57 Ideal I VS2 62.8 57.0 10462.0 7.46 7.37 4.66
1.09 Very Good F VVS1 61.4 58.0 10463.0 6.6 6.65 4.07
1.0 Very Good E VVS1 62.7 54.0 10463.0 6.36 6.39 4.0
1.2 Ideal G VVS2 62.2 53.0 10463.0 6.8 6.84 4.24
1.59 Premium I VS2 62.9 56.0 10471.0 7.48 7.43 4.69
1.35 Ideal G VS1 60.9 54.0 10471.0 7.18 7.15 4.36
1.39 Very Good G VS2 62.6 56.0 10476.0 7.08 7.11 4.44
1.72 Very Good J VS2 60.9 61.0 10477.0 7.77 7.79 4.74
1.5 Good H VS2 63.6 58.0 10478.0 7.27 7.22 4.61
1.5 Premium H VS2 61.2 61.0 10478.0 7.32 7.25 4.46
1.63 Premium I VS2 61.0 60.0 10479.0 7.62 7.59 4.64
1.56 Ideal I VVS2 62.0 56.0 10481.0 7.39 7.42 4.6
1.21 Ideal G VVS1 61.0 57.0 10482.0 6.93 6.87 4.21
1.5 Premium I VVS2 60.2 58.0 10483.0 7.5 7.34 4.47
1.21 Ideal G VVS1 61.4 58.0 10483.0 6.85 6.89 4.22
1.24 Premium G VVS1 60.4 59.0 10485.0 7.02 7.01 4.24
1.1 Ideal F VVS2 61.2 56.0 10487.0 6.66 6.74 4.1
1.22 Premium F VVS2 63.0 56.0 10494.0 6.88 6.76 4.3
1.6 Very Good I VS2 60.0 60.0 10497.0 7.55 7.59 4.54
1.52 Very Good H VS1 59.8 57.0 10497.0 7.47 7.55 4.49
1.01 Very Good E VVS1 63.2 54.0 10498.0 6.41 6.31 4.02
1.01 Very Good D VVS2 59.8 57.0 10499.0 6.49 6.58 3.91
1.55 Premium G VS2 60.5 60.0 10499.0 7.49 7.46 4.52
1.62 Ideal I VS2 62.1 55.0 10501.0 7.53 7.58 4.69
1.21 Ideal F VS1 61.8 56.0 10504.0 6.84 6.86 4.23
1.19 Ideal G VVS1 61.6 59.0 10508.0 6.78 6.79 4.18
1.14 Ideal E VVS2 62.3 55.0 10512.0 6.68 6.71 4.17
1.54 Very Good I VVS2 62.7 57.0 10518.0 7.35 7.43 4.63
1.56 Very Good I VS1 58.2 59.0 10523.0 7.65 7.7 4.47
1.29 Premium F VS1 60.9 58.0 10530.0 7.01 6.97 4.26
1.71 Ideal H VS2 63.0 57.0 10534.0 7.57 7.53 4.76
1.28 Ideal G VS1 62.1 56.0 10537.0 6.97 6.94 4.32
1.22 Ideal D VS2 61.7 56.0 10538.0 6.89 6.86 4.24
1.04 Premium E VVS1 62.5 59.0 10539.0 6.46 6.41 4.02
1.33 Ideal G VS1 62.0 55.0 10539.0 7.12 7.07 4.4
1.15 Ideal D VS2 61.0 57.0 10546.0 6.76 6.78 4.13
1.03 Premium F VVS1 59.7 60.0 10546.0 6.63 6.57 3.94
1.55 Very Good H VS2 63.3 56.0 10546.0 7.38 7.32 4.65
1.51 Premium H VS2 61.5 58.0 10548.0 7.45 7.32 4.45
1.51 Premium H VS2 63.0 58.0 10548.0 7.34 7.27 4.6
1.5 Very Good I VVS2 59.7 60.0 10551.0 7.46 7.62 4.5
1.51 Ideal G VS2 62.8 57.0 10553.0 7.33 7.26 4.58
1.51 Fair G VS2 58.1 67.0 10553.0 7.59 7.49 4.38
1.21 Ideal E VS2 61.8 53.0 10556.0 6.86 6.9 4.25
1.31 Ideal G VS1 62.0 53.0 10556.0 7.06 7.07 4.37
1.26 Ideal G VVS2 60.7 56.0 10556.0 7.05 7.03 4.27
1.5 Very Good H VS2 61.6 55.0 10558.0 7.37 7.43 4.56
1.53 Ideal I VS1 61.5 55.0 10560.0 7.4 7.42 4.56
1.64 Ideal I VS2 62.5 56.0 10562.0 7.58 7.52 4.72
1.0 Fair D VVS2 61.1 57.0 10562.0 6.37 6.3 3.87
1.01 Good E VVS1 63.1 59.0 10567.0 6.31 6.34 3.99
1.21 Ideal G VVS2 61.3 57.0 10568.0 6.83 6.85 4.19
1.23 Ideal G VVS1 61.8 56.0 10572.0 6.8 6.89 4.24
1.32 Very Good F VS1 60.3 57.0 10575.0 7.08 7.12 4.28
1.51 Ideal I VS1 61.6 56.0 10576.0 7.37 7.43 4.56
1.1 Premium E VVS2 60.1 58.0 10577.0 6.76 6.69 4.04
1.0 Ideal F VVS2 61.3 53.0 10577.0 6.44 6.48 3.96
1.02 Premium D VVS2 61.4 58.0 10580.0 6.43 6.46 3.96
1.2 Premium F VVS2 62.8 60.0 10580.0 6.79 6.74 4.25
1.23 Ideal F VVS2 63.0 55.0 10580.0 6.89 6.79 4.31
1.79 Premium J VS2 62.5 60.0 10581.0 7.76 7.69 4.83
1.41 Ideal G VS2 60.8 55.0 10581.0 7.27 7.2 4.4
1.5 Premium H VS1 59.3 61.0 10584.0 7.53 7.47 4.45
1.5 Very Good H VS1 63.4 56.0 10584.0 7.29 7.25 4.61
1.7 Ideal J VS1 60.9 58.0 10589.0 7.73 7.68 4.69
1.69 Premium I VS2 62.4 58.0 10600.0 7.66 7.53 4.74
1.53 Very Good I VS1 62.8 55.0 10602.0 7.35 7.4 4.63
1.35 Ideal D VS2 61.3 57.0 10602.0 7.09 7.13 4.36
1.11 Ideal F VVS1 61.9 57.0 10602.0 6.66 6.62 4.11
1.03 Ideal D VS1 61.7 57.0 10607.0 6.45 6.48 3.99
1.58 Very Good H VS2 61.4 60.0 10608.0 7.49 7.44 4.58
1.43 Premium G VS2 62.2 58.0 10609.0 7.18 7.15 4.46
1.23 Very Good F VS1 59.3 59.0 10609.0 6.98 7.01 4.15
1.29 Ideal G VVS2 62.4 57.0 10614.0 6.94 6.97 4.34
1.52 Very Good H VS2 63.3 57.0 10618.0 7.33 7.32 4.64
1.58 Premium I VS1 61.1 59.0 10618.0 7.52 7.44 4.57
1.03 Ideal F VVS1 62.1 56.3 10619.0 6.43 6.5 4.02
1.75 Premium J VS1 62.2 59.0 10619.0 7.74 7.7 4.8
1.22 Ideal E VS1 62.4 54.0 10622.0 6.77 6.88 4.26
1.52 Fair G VS2 55.2 66.0 10623.0 7.72 7.67 4.26
1.51 Premium I VVS2 61.1 60.0 10623.0 7.33 7.36 4.49
1.4 Premium F VS2 61.5 60.0 10625.0 7.25 7.18 4.44
1.51 Good H VS2 63.5 60.0 10628.0 7.24 7.27 4.61
1.32 Ideal G VS1 62.4 53.0 10631.0 7.03 7.08 4.4
1.25 Ideal G VVS2 61.5 55.0 10636.0 6.92 6.94 4.26
1.25 Ideal G VVS2 62.5 54.0 10636.0 6.88 6.93 4.31
1.35 Ideal H VVS1 61.9 57.0 10639.0 7.06 7.09 4.38
2.0 Good I VS2 64.0 60.0 10640.0 7.9 7.83 5.04
1.25 Premium E VS2 61.5 58.0 10640.0 6.98 6.91 4.27
1.52 Premium G VS1 58.8 61.0 10640.0 7.54 7.45 4.41
1.02 Very Good E VVS1 60.2 60.0 10641.0 6.39 6.54 3.89
1.52 Ideal I VS1 62.4 57.0 10641.0 7.32 7.36 4.58
1.15 Ideal G VVS1 62.7 56.0 10644.0 6.69 6.67 4.19
1.5 Fair G VS2 66.2 53.0 10644.0 7.12 7.08 4.7
1.5 Ideal I VVS2 61.7 55.0 10646.0 7.32 7.39 4.54
1.23 Ideal G VVS2 60.8 57.0 10646.0 6.89 6.92 4.2
1.23 Very Good F VVS2 58.5 59.0 10650.0 6.98 7.07 4.11
1.5 Very Good H VS1 63.4 59.0 10652.0 7.13 7.2 4.54
1.3 Very Good G VS1 62.5 58.0 10654.0 6.9 6.95 4.33
1.51 Premium H VS1 59.6 60.0 10655.0 7.5 7.41 4.44
1.51 Ideal I VS1 63.2 57.0 10655.0 7.4 7.28 4.62
1.35 Ideal G VS1 62.7 57.0 10656.0 7.02 7.07 4.42
1.27 Ideal F VS2 61.0 54.0 10656.0 7.0 7.02 4.28
1.79 Ideal J VS2 61.8 56.0 10658.0 7.74 7.85 4.82
1.7 Ideal I VS2 61.1 57.0 10662.0 7.7 7.66 4.69
1.7 Premium I VS2 62.7 58.0 10662.0 7.57 7.52 4.73
1.5 Premium H VS2 60.6 61.0 10668.0 7.34 7.31 4.44
1.62 Premium I VS2 60.1 59.0 10669.0 7.63 7.6 4.58
1.26 Premium F VS1 62.0 58.0 10669.0 6.95 6.88 4.29
1.5 Good G VS2 63.7 57.0 10669.0 7.29 7.25 4.63
1.02 Very Good E VVS1 62.2 57.0 10672.0 6.4 6.59 4.04
1.5 Very Good H VS1 60.7 58.0 10681.0 7.35 7.42 4.48
1.75 Very Good J VS1 61.5 59.0 10681.0 7.75 7.83 4.79
1.17 Very Good D VS1 60.5 57.0 10681.0 6.79 6.86 4.13
1.68 Ideal J VS1 61.1 57.0 10681.0 7.64 7.7 4.69
1.57 Very Good I VS1 62.7 58.0 10682.0 7.41 7.43 4.65
1.21 Ideal G VS1 61.3 57.0 10685.0 6.82 6.87 4.2
1.21 Ideal G VS1 61.6 57.0 10685.0 6.85 6.87 4.22
1.21 Ideal G VS1 61.8 57.0 10685.0 6.81 6.86 4.23
1.51 Premium H VS2 60.7 58.0 10685.0 7.45 7.4 4.51
1.51 Ideal H VS2 62.6 57.0 10685.0 7.37 7.33 4.6
1.66 Ideal I VS1 61.0 55.0 10691.0 7.67 7.64 4.67
1.5 Good H VS2 63.9 60.0 10692.0 7.17 7.22 4.6
1.5 Ideal H VS2 62.2 57.0 10692.0 7.27 7.33 4.54
1.01 Very Good E VVS1 61.6 58.0 10693.0 6.45 6.57 4.01
1.01 Good E VVS1 63.1 57.0 10696.0 6.36 6.39 4.02
1.1 Very Good F VVS1 59.8 54.0 10701.0 6.74 6.77 4.04
1.56 Premium H VS2 62.5 59.0 10702.0 7.3 7.33 4.57
1.54 Premium H VS2 61.8 59.0 10702.0 7.35 7.4 4.56
1.14 Ideal E VVS2 62.3 55.0 10703.0 6.71 6.68 4.17
1.01 Very Good E VVS1 62.3 56.0 10704.0 6.41 6.47 4.01
1.21 Premium D VS1 62.6 56.0 10706.0 6.83 6.74 4.25
1.26 Ideal F VS2 61.5 56.0 10709.0 6.97 7.01 4.3
1.04 Ideal D VS1 61.6 57.0 10709.0 6.52 6.56 4.03
1.55 Very Good I VS1 59.0 58.0 10711.0 7.56 7.63 4.48
1.31 Ideal E VS1 61.7 55.0 10711.0 7.11 7.05 4.37
1.28 Premium G VVS2 62.1 58.0 10716.0 6.96 6.91 4.31
1.22 Very Good F VVS2 60.7 62.0 10719.0 6.81 6.84 4.14
1.26 Ideal G VVS1 62.1 57.0 10720.0 6.91 7.01 4.32
1.24 Ideal G VVS1 62.3 56.0 10724.0 6.86 6.89 4.28
1.29 Ideal F VS1 62.3 54.4 10727.0 6.93 7.0 4.34
1.53 Premium I VS1 62.4 59.0 10729.0 7.3 7.34 4.57
1.23 Very Good E VS1 61.5 58.0 10730.0 6.88 6.93 4.25
1.01 Premium D VVS2 62.4 58.0 10732.0 6.39 6.44 4.0
1.33 Ideal G VS1 59.9 57.3 10732.0 7.16 7.21 4.3
1.42 Ideal G VS2 62.6 57.0 10735.0 7.19 7.15 4.49
1.67 Premium I VS2 61.9 56.0 10736.0 7.64 7.6 4.72
1.13 Ideal F VVS2 62.1 54.0 10742.0 6.67 6.71 4.16
1.51 Premium H VS1 62.4 60.0 10743.0 7.27 7.34 4.56
1.5 Premium I VVS2 59.7 60.0 10744.0 7.62 7.46 4.5
1.19 Ideal F VVS2 60.8 57.0 10748.0 6.85 6.83 4.16
1.5 Premium H VS2 61.6 55.0 10750.0 7.43 7.37 4.56
1.0 Fair D VVS1 56.7 68.0 10752.0 6.66 6.64 3.77
1.0 Premium D VVS1 62.9 58.0 10752.0 6.34 6.28 3.97
1.59 Ideal I VS2 61.7 57.0 10752.0 7.47 7.52 4.62
1.15 Ideal F VVS2 62.7 57.0 10757.0 6.69 6.65 4.18
1.54 Premium H VS2 61.9 61.0 10758.0 7.41 7.33 4.56
1.01 Very Good E VVS1 63.1 59.0 10760.0 6.34 6.31 3.99
1.01 Ideal D VVS2 61.7 57.0 10761.0 6.43 6.47 3.98
1.51 Ideal H VS2 62.5 55.0 10763.0 7.29 7.34 4.57
1.51 Good H VS2 64.2 58.5 10763.0 7.16 7.22 4.62
1.3 Premium G VVS2 60.2 58.0 10763.0 7.17 7.08 4.29
1.12 Ideal F VVS2 61.3 57.0 10764.0 6.67 6.7 4.1
1.54 Good H VS1 63.1 57.0 10769.0 7.34 7.4 4.65
1.12 Ideal E VVS2 60.6 57.0 10769.0 6.77 6.66 4.07
1.34 Very Good F VS1 59.7 60.0 10771.0 7.17 7.2 4.29
1.02 Premium D VVS2 61.4 58.0 10773.0 6.46 6.43 3.96
1.51 Premium H VS1 62.1 58.0 10775.0 7.32 7.26 4.53
1.71 Ideal J VS1 62.4 56.0 10778.0 7.63 7.59 4.75
1.29 Very Good E VS1 61.8 57.0 10780.0 6.99 6.93 4.3
1.14 Ideal F VVS2 61.9 57.0 10786.0 6.73 6.68 4.15
1.2 Ideal G VVS1 63.1 56.0 10786.0 6.74 6.7 4.24
1.26 Ideal G VVS1 61.1 57.0 10787.0 7.02 6.98 4.28
1.01 Ideal E VVS1 61.8 54.0 10789.0 6.43 6.49 3.99
1.4 Very Good F VS2 60.0 58.0 10790.0 7.24 7.29 4.36
1.53 Ideal I VS1 62.8 55.0 10796.0 7.4 7.35 4.63
1.35 Ideal D VS2 61.3 57.0 10796.0 7.13 7.09 4.36
1.01 Ideal D VVS2 60.6 56.0 10797.0 6.53 6.48 3.94
1.01 Premium D VVS2 61.6 58.0 10797.0 6.44 6.39 3.95
1.86 Very Good J VS2 62.5 55.0 10798.0 7.81 7.9 4.91
1.01 Very Good D VVS2 63.0 56.0 10800.0 6.35 6.41 4.02
1.68 Ideal I VS2 62.1 57.0 10800.0 7.6 7.54 4.7
1.29 Premium F VS1 61.3 61.0 10801.0 7.05 6.95 4.29
1.67 Ideal I VS1 61.2 57.0 10802.0 7.69 7.66 4.7
1.04 Ideal F VVS2 61.6 57.0 10804.0 6.47 6.51 4.0
1.27 Ideal G VVS1 61.7 56.0 10805.0 6.9 7.03 4.3
1.29 Ideal G VVS2 62.4 57.0 10807.0 6.97 6.94 4.34
1.7 Very Good J VVS1 62.9 60.0 10808.0 7.56 7.61 4.77
1.16 Premium F VVS1 60.4 60.0 10809.0 6.9 6.77 4.13
1.2 Ideal G VVS2 62.3 54.0 10812.0 6.82 6.86 4.26
1.32 Premium F VS2 62.2 58.0 10814.0 7.03 6.95 4.35
1.51 Premium I VVS2 61.1 60.0 10817.0 7.36 7.33 4.49
1.51 Good H VS1 58.2 58.0 10819.0 7.49 7.56 4.38
1.51 Ideal I VVS2 61.6 58.0 10821.0 7.35 7.4 4.54
1.71 Ideal J VS1 61.6 57.0 10821.0 7.67 7.62 4.71
1.51 Very Good H VS2 63.5 60.0 10822.0 7.27 7.24 4.61
1.22 Ideal E VS1 61.8 56.0 10823.0 6.84 6.88 4.24
1.6 Very Good I VS1 58.6 58.0 10824.0 7.66 7.72 4.51
1.52 Premium I VVS2 61.6 58.0 10824.0 7.37 7.41 4.55
1.51 Premium I VVS1 61.0 61.0 10824.0 7.47 7.34 4.52
1.04 Ideal F VVS1 61.6 55.0 10825.0 6.5 6.53 4.02
1.55 Premium H VS2 62.7 57.0 10827.0 7.42 7.36 4.63
1.53 Premium H VS2 59.9 60.0 10827.0 7.5 7.46 4.48
1.23 Very Good F VVS2 62.4 58.0 10835.0 6.74 6.89 4.26
1.68 Ideal G VS2 62.1 55.0 10838.0 7.64 7.59 4.73
1.28 Ideal G VVS2 61.2 54.0 10839.0 7.03 7.06 4.31
1.23 Premium F VVS2 58.5 59.0 10844.0 7.07 6.98 4.11
1.2 Ideal G VVS2 61.2 56.0 10846.0 6.92 6.89 4.23
1.35 Ideal G VS1 62.7 57.0 10850.0 7.07 7.02 4.42
1.16 Ideal F VVS2 60.5 58.0 10850.0 6.77 6.85 4.12
1.38 Premium G VS1 62.4 58.0 10850.0 7.14 7.09 4.44
1.52 Ideal I VS1 61.8 55.0 10851.0 7.38 7.41 4.57
1.01 Ideal E VVS2 61.7 55.0 10852.0 6.44 6.49 3.99
1.23 Ideal G VS1 61.6 57.0 10859.0 6.84 6.9 4.23
1.01 Ideal D VVS2 61.1 57.0 10860.0 6.47 6.49 3.96
1.1 Ideal F VVS1 62.7 57.0 10861.0 6.57 6.63 4.14
1.51 Ideal H VS2 61.4 58.0 10861.0 7.36 7.42 4.54
1.02 Premium E VVS1 62.2 57.0 10867.0 6.59 6.4 4.04
1.55 Ideal I VS1 61.9 55.0 10869.0 7.43 7.46 4.61
1.05 Ideal F VVS1 61.6 55.0 10872.0 6.57 6.53 4.04
1.5 Very Good I VVS2 62.0 53.0 10873.0 7.3 7.34 4.54
1.05 Ideal F VVS2 62.1 55.0 10874.0 6.54 6.56 4.07
1.14 Premium F VVS1 60.8 58.0 10878.0 6.79 6.74 4.11
1.57 Premium H VS1 61.7 58.0 10880.0 7.47 7.56 4.64
1.26 Ideal G VVS2 60.9 57.0 10886.0 6.98 7.01 4.26
1.5 Good H VS2 63.9 60.0 10886.0 7.22 7.17 4.6
1.5 Ideal H VS2 62.2 57.0 10886.0 7.33 7.27 4.54
1.01 Ideal E VVS1 62.2 57.0 10887.0 6.38 6.44 3.99
1.22 Ideal G VVS1 61.1 56.0 10888.0 6.91 6.94 4.23
1.01 Premium E VVS1 61.6 58.0 10888.0 6.57 6.45 4.01
1.53 Premium H VS2 60.1 58.0 10889.0 7.57 7.6 4.71
1.2 Very Good F VVS2 62.9 59.0 10891.0 6.72 6.76 4.24
1.01 Very Good E VVS1 63.1 57.0 10891.0 6.39 6.36 4.02
1.54 Premium H VS2 61.8 59.0 10897.0 7.4 7.35 4.56
1.5 Ideal I VVS2 60.3 57.0 10907.0 7.43 7.47 4.49
1.16 Ideal D VS2 61.8 55.0 10907.0 6.72 6.75 4.16
1.7 Premium I VS2 62.4 58.0 10910.0 7.61 7.56 4.73
1.62 Very Good H VS2 62.6 58.0 10912.0 7.57 7.45 4.7
1.05 Very Good E VVS1 59.5 60.0 10915.0 6.56 6.66 3.93
1.26 Ideal G VVS1 62.1 57.0 10916.0 7.01 6.91 4.32
1.32 Ideal E VS2 62.0 56.0 10919.0 7.02 7.07 4.37
1.5 Ideal H VS2 62.3 56.0 10920.0 7.34 7.29 4.56
1.58 Ideal I VS1 62.4 54.0 10920.0 7.43 7.46 4.64
1.26 Very Good G VVS1 60.2 59.0 10922.0 6.98 7.07 4.23
1.29 Ideal F VS1 62.3 54.0 10923.0 7.0 6.93 4.34
1.53 Premium I VS1 62.4 59.0 10924.0 7.34 7.3 4.57
1.37 Ideal G VS1 62.2 55.0 10927.0 7.09 7.12 4.42
1.01 Premium D VVS2 62.4 58.0 10927.0 6.44 6.39 4.0
1.36 Premium F VS2 59.3 60.0 10929.0 7.23 7.2 4.28
1.7 Premium I VS2 62.0 59.0 10929.0 7.6 7.55 4.7
1.3 Premium F VS1 61.0 59.0 10930.0 7.05 7.08 4.31
1.57 Premium H VS1 60.5 61.0 10930.0 7.6 7.51 4.57
1.2 Very Good F VVS2 61.4 60.0 10931.0 6.79 6.83 4.18
1.5 Ideal I VVS2 60.7 60.0 10931.0 7.35 7.4 4.48
1.14 Ideal G VVS1 61.1 58.0 10933.0 6.74 6.77 4.13
1.7 Very Good I VS2 63.0 58.0 10935.0 7.52 7.65 4.78
1.51 Premium H VS1 62.4 60.0 10939.0 7.34 7.27 4.56
1.71 Ideal J VVS2 61.6 59.0 10945.0 7.65 7.68 4.72
1.24 Ideal G VS1 61.8 55.0 10946.0 6.85 6.89 4.25
1.12 Ideal F VVS2 62.2 55.0 10949.0 6.64 6.68 4.14
1.51 Very Good H VS2 63.2 57.0 10950.0 7.18 7.32 4.58
1.51 Premium H VS2 60.2 60.0 10951.0 7.5 7.38 4.48
1.01 Ideal E VVS1 62.0 57.0 10954.0 6.39 6.45 3.98
1.5 Premium I VVS1 62.4 60.0 10956.0 7.29 7.32 4.56
1.31 Ideal F VS1 61.9 54.0 10957.0 7.03 7.06 4.35
1.5 Very Good H VS2 62.8 57.0 10959.0 7.25 7.3 4.57
1.51 Ideal H VS2 64.2 59.0 10959.0 7.22 7.16 4.62
1.51 Ideal H VS2 62.5 55.0 10959.0 7.34 7.29 4.57
1.25 Very Good G VVS1 60.6 60.0 10962.0 6.92 6.95 4.2
1.35 Premium F VS2 61.9 58.0 10962.0 7.06 7.02 4.36
1.57 Ideal I VS2 60.4 58.0 10964.0 7.55 7.59 4.57
1.02 Ideal D VVS2 62.0 56.0 10967.0 6.43 6.48 4.0
1.44 Very Good D VS2 63.1 56.0 10967.0 7.15 7.12 4.5
1.52 Ideal I VVS1 61.9 56.0 10968.0 7.34 7.37 4.55
1.32 Premium F VS1 61.7 59.0 10977.0 6.95 6.99 4.3
1.54 Premium H VS2 61.0 60.0 10977.0 7.42 7.46 4.54
1.2 Good F VVS1 63.6 57.0 10982.0 6.71 6.74 4.28
1.25 Ideal G VVS2 62.2 55.0 10983.0 6.87 6.93 4.29
1.04 Premium D VVS2 61.1 60.0 10984.0 6.54 6.51 3.99
1.01 Very Good E VVS1 63.6 55.0 10993.0 6.35 6.39 4.05
1.66 Premium H VS2 62.3 58.0 10993.0 7.62 7.57 4.73
1.5 Good F VS2 63.6 59.0 10995.0 7.13 7.22 4.56
1.25 Ideal G VVS1 61.3 56.0 10996.0 6.95 6.98 4.28
1.27 Ideal G VVS1 61.7 56.0 11002.0 7.03 6.9 4.3
1.22 Ideal G VVS2 60.0 58.0 11003.0 6.95 6.99 4.18
1.7 Premium J VVS1 62.9 60.0 11005.0 7.61 7.56 4.77
1.5 Premium I VVS2 61.4 58.0 11007.0 7.38 7.32 4.51
1.5 Fair H VS1 61.7 60.0 11007.0 7.37 7.27 4.52
1.5 Premium H VS1 61.1 58.0 11007.0 7.46 7.37 4.53
1.41 Ideal G VS1 60.4 57.0 11009.0 7.22 7.31 4.39
1.37 Ideal G VS1 61.1 55.0 11009.0 7.2 7.16 4.39
1.41 Very Good F VS1 63.4 63.0 11010.0 7.13 6.97 4.47
1.01 Ideal D VVS2 61.5 57.0 11015.0 6.43 6.48 3.97
1.59 Very Good I VS1 61.2 57.8 11018.0 7.5 7.52 4.6
1.63 Ideal I VS2 61.9 54.3 11019.0 7.54 7.58 4.68
1.21 Ideal D VS1 60.1 60.0 11019.0 6.92 6.99 4.18
1.2 Premium F VVS2 62.2 58.0 11021.0 6.83 6.78 4.23
1.52 Premium I VVS2 61.6 58.0 11021.0 7.41 7.37 4.55
1.6 Premium I VS1 58.6 58.0 11021.0 7.72 7.66 4.51
1.33 Good D VS2 63.6 58.0 11023.0 7.02 6.91 4.43
1.5 Very Good H VS2 60.7 61.0 11025.0 7.34 7.41 4.48
1.02 Ideal E VVS1 61.4 57.0 11028.0 6.39 6.47 3.95
1.44 Premium G VS2 59.5 61.0 11032.0 7.38 7.3 4.37
1.71 Premium G VS2 61.3 58.0 11032.0 7.64 7.6 4.67
1.52 Ideal I VVS1 62.3 55.0 11033.0 7.32 7.37 4.58
1.56 Very Good H VS2 63.1 60.0 11039.0 7.43 7.34 4.66
1.23 Ideal F VVS2 61.8 56.0 11040.0 6.84 6.89 4.24
1.61 Ideal H VS2 61.4 57.0 11045.0 7.52 7.57 4.63
1.55 Premium H VS2 60.4 60.0 11048.0 7.39 7.44 4.48
1.74 Premium J VS1 62.5 58.0 11050.0 7.67 7.65 4.79
1.13 Ideal F VVS1 61.7 56.0 11051.0 6.68 6.75 4.14
1.2 Ideal D VS1 61.0 59.0 11053.0 6.79 6.85 4.16
1.01 Ideal D VVS2 61.1 57.0 11057.0 6.49 6.47 3.96
2.02 Premium I VS2 61.2 60.0 11059.0 8.22 8.13 5.0
1.47 Very Good G VS2 62.7 56.0 11060.0 7.15 7.18 4.49
1.3 Premium G VVS1 60.5 60.0 11061.0 7.01 7.05 4.25
1.02 Premium E VVS1 61.5 59.0 11062.0 6.41 6.46 3.96
1.7 Premium I VS2 61.7 59.0 11062.0 7.63 7.57 4.69
1.52 Premium H VS2 61.1 59.0 11066.0 7.45 7.38 4.53
1.55 Ideal I VS1 61.9 55.0 11067.0 7.46 7.43 4.61
1.51 Very Good H VS2 60.9 57.0 11068.0 7.39 7.43 4.51
1.3 Ideal G VVS2 62.0 57.0 11073.0 6.96 7.03 4.34
1.11 Ideal E VVS2 62.9 55.0 11074.0 6.56 6.62 4.14
1.51 Very Good H VS2 60.9 54.0 11077.0 7.38 7.41 4.5
1.32 Ideal G VS1 61.7 56.0 11079.0 7.03 7.11 4.36
1.23 Very Good G VVS1 61.2 55.8 11081.0 6.9 6.94 4.23
1.01 Ideal D VVS2 62.3 53.0 11082.0 6.4 6.47 4.02
1.69 Ideal I VS2 61.7 56.0 11086.0 7.65 7.71 4.74
1.53 Premium H VS2 60.1 58.0 11087.0 7.6 7.57 4.71
1.51 Very Good I VVS1 63.0 59.0 11088.0 7.24 7.3 4.58
1.5 Premium H VS1 62.1 59.0 11088.0 7.27 7.31 4.53
1.5 Premium H VS1 59.9 60.0 11088.0 7.39 7.44 4.44
1.25 Premium E VS1 61.5 59.0 11088.0 6.95 6.91 4.26
1.25 Ideal G VVS2 62.0 59.0 11089.0 6.88 6.96 4.29
1.5 Premium H VS2 62.4 59.0 11092.0 7.29 7.32 4.56
1.52 Very Good G VS2 62.4 56.0 11093.0 7.26 7.36 4.56
1.18 Ideal E VS1 61.4 57.0 11104.0 6.77 6.81 4.17
1.53 Premium H VS2 62.7 56.0 11104.0 7.39 7.31 4.61
1.52 Premium H VS2 59.4 59.0 11105.0 7.45 7.49 4.44
1.31 Very Good G VVS2 62.2 59.0 11108.0 6.91 6.98 4.32
1.41 Ideal G VS1 60.4 57.0 11109.0 7.31 7.22 4.39
1.47 Premium G VS2 62.8 57.0 11113.0 7.27 7.22 4.55
1.62 Ideal H VS2 62.4 57.0 11114.0 7.48 7.53 4.68
1.13 Ideal E VVS2 61.4 57.0 11115.0 6.69 6.74 4.12
1.13 Ideal E VVS2 61.6 56.0 11115.0 6.69 6.71 4.13
2.01 Good I VS2 59.0 64.0 11115.0 8.25 8.19 4.85
1.34 Premium G VVS2 61.3 58.0 11118.0 7.16 7.1 4.37
1.59 Ideal I VS1 61.2 58.0 11119.0 7.52 7.5 4.6
1.16 Ideal F VVS2 60.5 57.0 11120.0 6.8 6.86 4.13
1.53 Premium H VS1 59.4 59.0 11127.0 7.58 7.51 4.48
1.02 Ideal E VVS1 61.3 57.0 11128.0 6.47 6.54 3.99
1.24 Very Good F VVS2 62.0 55.0 11130.0 6.88 6.95 4.29
1.3 Premium F VS1 62.0 58.0 11130.0 6.94 6.9 4.29
1.1 Ideal D VVS2 62.0 57.0 11132.0 6.57 6.62 4.09
1.1 Very Good D VVS2 61.7 56.0 11132.0 6.64 6.65 4.1
1.36 Ideal F VS1 61.4 57.0 11132.0 7.25 7.09 4.4
1.51 Ideal H VS2 62.5 56.0 11133.0 7.37 7.33 4.57
1.7 Very Good H VVS2 63.2 56.0 11133.0 7.59 7.56 4.79
1.31 Ideal F VS1 61.7 56.0 11136.0 7.02 7.04 4.34
1.23 Ideal F VVS2 61.8 56.0 11141.0 6.89 6.84 4.24
1.31 Ideal G VVS1 61.1 57.0 11146.0 7.01 7.06 4.3
1.57 Premium I VVS2 60.7 58.0 11146.0 7.54 7.51 4.57
1.28 Ideal G VVS1 62.1 56.0 11147.0 6.93 6.96 4.31
1.55 Premium H VS2 60.4 60.0 11149.0 7.44 7.39 4.48
1.77 Ideal J VS1 62.2 56.0 11150.0 7.77 7.73 4.82
1.62 Ideal I VS1 60.8 56.0 11152.0 7.56 7.61 4.62
1.52 Ideal H VS1 62.3 55.0 11154.0 7.36 7.32 4.57
1.01 Ideal E VVS1 62.0 57.0 11154.0 6.45 6.39 3.98
1.5 Premium I VVS1 62.4 60.0 11155.0 7.32 7.29 4.56
1.66 Ideal I VS2 62.3 54.0 11156.0 7.58 7.61 4.73
1.5 Ideal H VS2 62.8 57.0 11159.0 7.3 7.25 4.57
1.51 Very Good G VS2 59.3 58.0 11161.0 7.32 7.49 4.39
1.51 Very Good H VS1 61.8 59.0 11161.0 7.27 7.32 4.51
1.51 Premium H VS1 61.0 60.0 11161.0 7.29 7.34 4.46
1.71 Premium H VS1 58.1 62.0 11161.0 8.02 7.84 4.61
1.3 Premium G VVS1 60.5 60.0 11162.0 7.05 7.01 4.25
1.02 Premium E VVS1 61.5 59.0 11163.0 6.46 6.41 3.96
1.51 Very Good H VS2 62.8 60.0 11166.0 7.25 7.28 4.56
1.08 Ideal E VVS2 61.0 56.0 11166.0 6.64 6.67 4.06
1.02 Ideal D VVS2 62.0 56.0 11167.0 6.48 6.43 4.0
1.52 Ideal I VVS1 61.9 56.0 11168.0 7.37 7.34 4.55
1.6 Very Good I VVS1 59.8 56.0 11170.0 7.6 7.68 4.57
1.23 Very Good E VVS2 60.4 62.0 11175.0 6.88 6.93 4.17
1.21 Premium F VVS1 63.0 59.0 11175.0 6.75 6.7 4.24
1.3 Ideal G VVS2 62.0 57.0 11175.0 7.03 6.96 4.34
1.76 Premium J VS1 62.0 58.0 11177.0 7.74 7.7 4.79
1.54 Premium H VS2 61.0 60.0 11177.0 7.46 7.42 4.54
1.32 Premium F VS1 61.7 59.0 11177.0 6.99 6.95 4.3
1.53 Very Good H VS2 62.2 58.0 11178.0 7.3 7.34 4.55
1.2 Good F VVS1 63.6 57.0 11182.0 6.74 6.71 4.28
1.23 Ideal G VVS1 61.2 56.0 11182.0 6.94 6.9 4.23
1.18 Ideal E VVS2 61.6 58.0 11184.0 6.78 6.82 4.19
1.51 Premium H VS2 61.2 58.0 11188.0 7.4 7.36 4.52
1.5 Premium G VS2 59.9 58.0 11189.0 7.38 7.34 4.41
1.5 Premium H VS1 62.1 59.0 11189.0 7.31 7.27 4.53
1.5 Premium G VS2 58.4 58.0 11189.0 7.54 7.5 4.39
1.5 Premium H VS1 59.9 60.0 11189.0 7.44 7.39 4.44
1.7 Very Good I VS2 58.5 59.0 11190.0 7.81 7.89 4.59
1.5 Premium H VS2 62.4 59.0 11194.0 7.32 7.29 4.56
1.74 Ideal J VS1 61.1 56.0 11194.0 7.85 7.79 4.78
1.05 Ideal D VVS2 61.9 54.0 11196.0 6.53 6.55 4.05
1.29 Ideal F VS1 60.7 57.0 11197.0 7.08 7.05 4.29
1.38 Ideal F VS2 61.9 55.0 11205.0 7.16 7.12 4.42
1.52 Premium H VS2 59.4 59.0 11206.0 7.49 7.45 4.44
1.52 Ideal H VS2 60.7 56.0 11206.0 7.49 7.41 4.52
1.14 Ideal E VVS2 61.6 57.0 11206.0 6.68 6.73 4.13
1.27 Ideal G VS1 61.2 57.0 11206.0 6.97 6.98 4.27
2.04 Premium J VS2 60.9 59.0 11209.0 8.25 8.21 5.01
1.06 Ideal D VVS2 61.1 56.0 11209.0 6.58 6.59 4.02
1.28 Very Good G VVS1 60.3 59.0 11214.0 6.99 7.03 4.23
1.5 Very Good G VS1 59.1 62.0 11216.0 7.38 7.42 4.37
1.62 Ideal I VS2 61.8 55.0 11217.0 7.56 7.59 4.68
1.26 Ideal G VVS1 61.7 56.0 11218.0 6.96 6.98 4.3
1.5 Very Good H VS2 60.0 62.0 11220.0 7.38 7.41 4.44
1.72 Ideal I VS2 62.8 57.0 11226.0 7.69 7.63 4.81
1.14 Ideal F VVS1 60.1 60.0 11226.0 6.79 6.83 4.09
1.24 Premium F VVS2 62.0 55.0 11231.0 6.95 6.88 4.29
1.1 Ideal D VVS2 62.0 57.0 11233.0 6.62 6.57 4.09
1.1 Premium D VVS2 61.7 56.0 11233.0 6.65 6.64 4.1
1.24 Very Good F VVS2 59.0 58.0 11234.0 6.98 7.03 4.13
1.52 Good G VS2 63.3 57.0 11235.0 7.27 7.32 4.62
1.31 Ideal F VS1 61.7 56.0 11237.0 7.04 7.02 4.34
1.71 Premium I VS1 60.6 57.0 11246.0 7.82 7.73 4.71
1.31 Ideal G VVS1 61.1 57.0 11247.0 7.06 7.01 4.3
1.28 Ideal G VVS1 62.1 56.0 11248.0 6.96 6.93 4.31
1.39 Premium E VS2 61.7 59.0 11248.0 7.13 7.09 4.39
1.71 Good I VS2 58.0 60.0 11250.0 7.85 7.9 4.57
1.5 Premium F VS2 61.1 59.0 11255.0 7.37 7.35 4.5
1.31 Premium G VVS2 59.6 61.0 11255.0 7.23 7.14 4.28
1.7 Ideal I VS2 61.7 56.0 11257.0 7.64 7.72 4.74
1.7 Premium I VS2 61.7 59.0 11257.0 7.63 7.68 4.72
1.7 Premium I VS2 61.2 59.0 11257.0 7.55 7.62 4.64
1.58 Ideal H VS2 62.7 56.0 11262.0 7.39 7.44 4.65
1.4 Very Good G VS1 62.6 58.0 11262.0 7.03 7.07 4.41
1.51 Premium H VS1 61.8 59.0 11263.0 7.32 7.27 4.51
1.51 Very Good H VS1 63.2 60.0 11263.0 7.23 7.17 4.55
1.51 Premium H VS1 61.0 60.0 11263.0 7.34 7.29 4.46
1.51 Fair H VS1 58.0 67.0 11263.0 7.63 7.57 4.41
1.24 Very Good F VVS2 63.6 56.0 11268.0 6.75 6.8 4.31
1.51 Premium H VS2 62.9 59.0 11268.0 7.31 7.25 4.58
1.51 Premium H VS2 62.8 60.0 11268.0 7.28 7.25 4.56
1.41 Premium E VS2 61.3 58.0 11269.0 7.29 7.25 4.46
1.57 Premium H VS1 59.8 60.0 11272.0 7.63 7.56 4.54
1.52 Ideal H VS2 62.4 58.0 11272.0 7.3 7.37 4.58
1.04 Premium E VVS1 60.9 58.0 11279.0 6.53 6.61 4.0
1.53 Premium H VS2 62.2 58.0 11280.0 7.34 7.3 4.55
1.38 Very Good F VS1 61.4 61.0 11286.0 7.1 7.14 4.37
1.5 Good G VS2 59.0 58.0 11294.0 7.41 7.45 4.38
1.5 Ideal H VS1 62.3 54.7 11296.0 7.29 7.33 4.55
1.2 Premium E VVS2 62.1 58.0 11301.0 6.76 6.7 4.18
1.43 Ideal G VS1 59.9 57.0 11302.0 7.35 7.3 4.39
1.5 Very Good H VVS2 62.7 58.0 11303.0 7.21 7.24 4.53
1.64 Ideal I VS1 60.5 57.0 11305.0 7.68 7.62 4.64
1.37 Ideal E VS2 60.3 54.0 11314.0 7.26 7.2 4.36
2.0 Fair I VS1 58.5 68.0 11322.0 8.26 8.15 4.8
1.5 Premium H VS2 60.0 62.0 11322.0 7.41 7.38 4.44
1.26 Ideal E VS1 61.2 56.0 11323.0 6.97 6.93 4.25
1.8 Premium J VS1 58.2 61.0 11329.0 8.07 7.95 4.66
1.11 Very Good E VVS1 60.2 59.0 11330.0 6.67 6.79 4.05
1.59 Very Good H VS2 60.7 61.1 11333.0 7.5 7.56 4.57
1.59 Premium H VS2 62.1 58.0 11333.0 7.42 7.48 4.63
1.52 Ideal H VS2 61.8 55.1 11333.0 7.38 7.42 4.58
1.03 Premium D VVS2 60.1 58.0 11335.0 6.55 6.6 3.95
1.52 Very Good G VS2 63.3 57.0 11338.0 7.32 7.27 4.62
1.56 Premium H VS1 62.0 57.0 11345.0 7.48 7.43 4.62
1.41 Premium G VS1 62.1 59.0 11347.0 7.1 7.06 4.4
1.5 Premium H VS2 61.8 59.0 11360.0 7.3 7.35 4.53
1.7 Premium I VS2 61.2 59.0 11360.0 7.62 7.55 4.64
1.7 Premium I VS2 61.7 59.0 11360.0 7.68 7.63 4.72
1.7 Ideal I VS2 61.7 56.0 11360.0 7.72 7.64 4.74
1.72 Premium I VS2 58.3 61.0 11360.0 7.91 7.87 4.6
1.55 Good H VS2 61.0 61.0 11364.0 7.42 7.47 4.54
1.58 Ideal H VS2 62.7 56.0 11365.0 7.44 7.39 4.65
1.4 Premium F VS2 60.7 58.0 11368.0 7.26 7.17 4.38
1.7 Premium I VS2 60.5 61.0 11369.0 7.68 7.65 4.64
1.52 Ideal H VS2 61.8 54.0 11379.0 7.42 7.43 4.59
1.52 Ideal H VS2 61.9 55.0 11379.0 7.38 7.43 4.58
1.53 Very Good F VS1 63.2 58.0 11379.0 7.33 7.3 4.62
1.58 Premium G VS1 60.8 58.0 11380.0 7.58 7.52 4.59
1.04 Premium E VVS1 60.9 58.0 11382.0 6.61 6.53 4.0
1.23 Very Good F VVS2 62.2 58.0 11382.0 6.81 6.86 4.25
1.13 Ideal E VVS2 60.1 59.0 11387.0 6.77 6.81 4.08
1.31 Premium G VVS2 62.7 59.0 11389.0 6.96 6.92 4.35
1.71 Very Good I VS2 63.4 59.0 11389.0 7.53 7.45 4.75
1.52 Very Good J VVS2 62.1 60.0 11392.0 7.33 7.36 4.56
1.3 Premium F VS1 62.5 58.0 11392.0 6.97 6.94 4.35
1.21 Premium E VVS2 61.9 58.0 11395.0 6.84 6.79 4.22
1.01 Ideal E VVS2 61.7 57.0 11400.0 6.42 6.44 3.97
1.67 Premium I VS1 61.1 58.0 11400.0 7.69 7.6 4.67
1.12 Ideal F VVS1 61.0 56.0 11403.0 6.71 6.74 4.1
1.52 Ideal F VS2 62.3 55.0 11405.0 7.37 7.33 4.58
1.83 Ideal J VS2 62.0 56.0 11406.0 7.84 7.9 4.88
1.33 Ideal D VS2 62.8 56.0 11409.0 7.08 7.03 4.43
1.53 Premium H VS1 60.8 59.0 11413.0 7.41 7.36 4.49
1.57 Premium H VS2 62.2 58.0 11415.0 7.45 7.4 4.62
1.18 Ideal F VVS2 60.6 55.0 11415.0 6.84 6.88 4.16
1.23 Ideal G VVS1 61.4 55.0 11417.0 6.89 6.93 4.24
1.32 Ideal G VVS2 62.3 57.0 11419.0 6.96 7.04 4.36
1.58 Ideal H VS2 63.0 56.0 11419.0 7.39 7.46 4.68
1.28 Ideal E VS1 61.7 57.0 11419.0 6.93 6.97 4.29
1.25 Ideal E VS2 60.7 56.0 11422.0 6.97 6.99 4.24
1.62 Very Good H VS2 59.6 59.0 11427.0 7.59 7.67 4.55
1.55 Premium H VS2 61.7 59.0 11428.0 7.44 7.4 4.58
1.18 Very Good F VVS2 60.1 58.0 11430.0 6.88 6.92 4.15
1.23 Premium F VVS2 61.6 59.0 11430.0 6.8 6.9 4.22
1.55 Ideal I VVS2 61.3 59.0 11430.0 7.41 7.46 4.56
1.53 Ideal I VS2 61.5 56.0 11434.0 7.39 7.44 4.55
1.42 Premium G VS1 62.1 56.0 11434.0 7.27 7.22 4.5
1.51 Premium H VS2 60.2 60.0 11435.0 7.33 7.35 4.42
1.51 Premium H VS2 62.3 59.0 11435.0 7.28 7.32 4.55
1.51 Ideal H VS2 62.2 57.0 11435.0 7.29 7.33 4.55
1.58 Ideal I VVS1 61.9 55.0 11435.0 7.47 7.51 4.64
1.2 Premium E VVS2 61.4 56.0 11435.0 6.94 6.83 4.23
1.59 Premium H VS2 62.1 58.0 11437.0 7.48 7.42 4.63
1.59 Ideal H VS2 60.7 61.0 11437.0 7.56 7.5 4.57
1.01 Ideal D VVS2 61.9 56.0 11442.0 6.41 6.45 3.98
1.42 Very Good G VS1 62.7 55.0 11452.0 7.11 7.17 4.48
1.53 Very Good H VS2 60.9 63.0 11452.0 7.37 7.41 4.5
1.21 Very Good F VVS2 61.0 58.0 11455.0 6.89 6.92 4.21
1.71 Ideal I VS2 60.5 56.0 11455.0 7.71 7.73 4.67
1.2 Ideal F VVS1 62.0 56.0 11455.0 6.76 6.82 4.21
1.72 Premium H VS2 59.5 60.0 11455.0 7.79 7.75 4.62
1.2 Very Good E VVS2 63.2 56.0 11456.0 6.73 6.78 4.27
1.31 Ideal G VVS2 59.2 59.0 11459.0 7.12 7.18 4.23
1.5 Premium H VS2 61.8 59.0 11464.0 7.35 7.3 4.53
1.23 Ideal G VVS1 59.5 57.0 11469.0 7.0 6.98 4.16
1.57 Premium H VS2 61.0 59.0 11470.0 7.5 7.54 4.59
1.13 Ideal D VS1 61.2 57.0 11477.0 6.7 6.72 4.1
1.28 Very Good G VVS2 59.5 56.0 11478.0 7.12 7.16 4.25
1.01 Premium D VVS1 59.3 59.0 11480.0 6.56 6.53 3.88
1.51 Premium G VS2 60.4 58.0 11480.0 7.38 7.43 4.47
1.7 Premium I VVS2 61.8 61.0 11481.0 7.57 7.5 4.66
1.54 Ideal G VS2 61.5 57.0 11487.0 7.44 7.41 4.57
1.6 Premium G VS2 62.2 59.0 11489.0 7.45 7.48 4.64
1.58 Premium H VS1 61.7 59.0 11491.0 7.48 7.42 4.6
1.43 Ideal H VS1 62.0 55.0 11498.0 7.21 7.28 4.49
2.07 Ideal J VS2 62.2 56.0 11500.0 8.2 8.16 5.09
1.24 Ideal G VVS1 62.1 56.0 11503.0 6.86 6.91 4.28
1.5 Good G VS2 58.8 64.0 11508.0 7.43 7.4 4.36
1.5 Good G VS2 63.8 56.0 11508.0 7.2 7.15 4.58
1.6 Premium H VS2 62.6 58.0 11508.0 7.5 7.45 4.68
1.04 Ideal D VVS2 60.9 57.0 11511.0 6.54 6.6 4.0
1.51 Very Good I VVS1 62.0 58.0 11512.0 7.27 7.31 4.52
1.51 Good H VS1 59.1 58.0 11512.0 7.48 7.52 4.43
1.7 Very Good I VS1 60.6 59.0 11514.0 7.64 7.67 4.64
1.52 Premium H VS1 61.4 58.0 11516.0 7.3 7.44 4.55
1.7 Premium I VS2 61.4 59.0 11519.0 7.6 7.68 4.69
1.7 Premium I VS2 60.7 59.0 11519.0 7.63 7.7 4.65
1.4 Good G VVS2 63.4 59.0 11519.0 7.04 7.12 4.49
1.5 Good G VS1 63.8 59.0 11524.0 7.16 7.22 4.59
2.01 Good J VS1 63.7 59.0 11526.0 7.93 7.86 5.03
1.58 Ideal I VVS1 61.7 53.0 11526.0 7.52 7.53 4.65
1.54 Very Good H VS2 62.1 62.0 11527.0 7.31 7.38 4.56
1.38 Ideal G VS1 62.2 54.0 11527.0 7.18 7.14 4.45
1.2 Ideal G VVS2 60.9 56.0 11530.0 6.86 6.91 4.19
1.2 Ideal G VVS2 61.0 56.0 11530.0 6.86 6.88 4.19
1.56 Ideal H VS2 61.5 56.0 11531.0 7.46 7.5 4.6
1.31 Ideal G VVS2 60.9 56.0 11531.0 7.17 7.07 4.32
1.51 Very Good G VS2 62.1 57.0 11532.0 7.37 7.32 4.5
1.23 Premium F VVS2 61.6 59.0 11534.0 6.9 6.8 4.22
1.63 Premium I VS1 61.1 57.0 11534.0 7.7 7.58 4.67
1.03 Premium E VVS1 58.8 59.0 11538.0 6.63 6.6 3.89
1.51 Ideal H VS2 62.2 57.0 11540.0 7.33 7.29 4.55
1.51 Premium H VS2 60.2 60.0 11540.0 7.35 7.33 4.42
1.51 Premium H VS2 62.3 59.0 11540.0 7.32 7.28 4.55
1.51 Ideal H VS2 60.6 57.0 11540.0 7.46 7.45 4.52
1.14 Premium F VVS1 59.4 59.0 11549.0 6.87 6.8 4.06
1.1 Ideal D VVS2 62.2 57.0 11550.0 6.58 6.54 4.08
1.5 Ideal H VS1 61.0 56.8 11557.0 7.36 7.4 4.5
1.53 Very Good H VS2 60.9 63.0 11557.0 7.41 7.37 4.5
1.71 Ideal I VS2 60.5 56.0 11559.0 7.73 7.71 4.67
1.55 Premium H VS1 62.6 58.0 11562.0 7.4 7.34 4.61
1.55 Premium H VS2 60.7 58.0 11567.0 7.51 7.47 4.55
1.21 Ideal G VVS1 61.5 56.0 11572.0 6.83 6.89 4.22
1.02 Ideal D VVS2 62.2 59.0 11573.0 6.41 6.46 4.0
1.55 Ideal I VS2 61.8 55.0 11574.0 7.4 7.44 4.58
1.57 Premium H VS2 61.0 59.0 11575.0 7.54 7.5 4.59
2.09 Good J VS2 57.2 64.0 11576.0 8.51 8.46 4.85
1.5 Good G VS2 63.3 62.0 11577.0 7.08 7.2 4.52
1.51 Ideal H VS1 62.6 56.0 11580.0 7.28 7.32 4.57
1.28 Ideal F VS1 61.7 55.0 11580.0 7.01 6.98 4.32
1.4 Very Good G VS1 59.9 56.0 11584.0 7.31 7.34 4.39
1.32 Ideal G VS1 61.7 56.0 11584.0 7.04 7.07 4.35
1.62 Ideal I VVS2 62.7 54.0 11587.0 7.47 7.52 4.7
1.44 Premium G VS1 61.8 57.0 11588.0 7.21 7.09 4.42
1.25 Very Good G VVS1 60.2 58.0 11589.0 6.97 7.04 4.22
1.24 Ideal G VVS2 61.1 56.0 11601.0 6.94 6.97 4.25
1.28 Very Good F VVS2 62.0 57.0 11602.0 6.92 7.01 4.32
1.55 Very Good H VS2 61.3 61.0 11602.0 7.39 7.46 4.55
1.57 Very Good H VS1 62.8 60.0 11605.0 7.36 7.44 4.65
1.7 Ideal H VS2 62.3 57.0 11605.0 7.68 7.65 4.78
1.75 Ideal J VVS2 62.0 55.0 11609.0 7.7 7.73 4.78
1.03 Ideal E VVS2 61.7 56.0 11611.0 6.49 6.51 4.01
1.76 Ideal I VS1 62.0 57.0 11616.0 7.71 7.74 4.79
1.52 Premium H VS1 61.4 58.0 11621.0 7.44 7.3 4.55
1.4 Very Good G VVS2 63.4 59.0 11624.0 7.12 7.04 4.49
1.61 Very Good H VS2 59.4 58.0 11627.0 7.64 7.74 4.57
1.54 Premium H VS2 62.1 62.0 11632.0 7.38 7.31 4.56
1.3 Ideal G VVS2 62.4 56.1 11633.0 6.97 7.02 4.36
1.56 Ideal H VS2 61.5 56.0 11636.0 7.5 7.46 4.6
1.51 Very Good G VS2 61.5 59.0 11640.0 7.34 7.38 4.53
1.51 Good G VS2 64.2 54.0 11640.0 7.18 7.27 4.64
1.34 Ideal G VVS1 62.2 56.0 11640.0 7.11 7.04 4.4
1.13 Ideal E VVS1 61.5 56.0 11641.0 6.68 6.71 4.12
1.41 Premium E VS2 62.7 56.0 11644.0 7.18 7.1 4.48
1.37 Ideal F VS1 59.6 57.0 11649.0 7.28 7.22 4.32
1.11 Very Good F VVS2 59.4 58.0 11650.0 6.74 6.79 4.02
1.45 Premium F VS2 61.1 58.0 11650.0 7.31 7.23 4.44
1.54 Ideal I VS1 61.5 56.0 11651.0 7.42 7.47 4.58
1.5 Very Good H VVS1 63.8 57.0 11654.0 7.17 7.21 4.59
1.14 Premium F VVS1 59.4 59.0 11654.0 6.87 6.8 4.06
1.1 Ideal D VVS2 62.2 57.0 11654.0 6.58 6.54 4.08
1.5 Premium I VS2 61.4 58.0 11655.0 7.28 7.24 4.46
1.01 Ideal D VVS1 62.5 55.0 11661.0 6.39 6.44 4.01
1.54 Premium H VS2 61.9 59.0 11663.0 7.31 7.33 4.53
1.45 Very Good F VS2 62.6 58.0 11667.0 7.12 7.2 4.48
1.03 Very Good D VVS2 62.7 58.0 11677.0 6.39 6.43 4.02
1.52 Very Good H VS2 59.7 55.0 11681.0 7.45 7.42 4.44
1.51 Ideal H VS1 62.6 56.0 11686.0 7.32 7.28 4.57
1.5 Very Good H VVS1 60.9 61.0 11688.0 7.36 7.39 4.49
1.51 Ideal H VS2 61.9 59.0 11696.0 7.29 7.34 4.53
1.52 Good F VS2 64.2 59.0 11696.0 7.16 7.2 4.61
1.55 Very Good G VS2 63.1 57.0 11703.0 7.36 7.31 4.63
1.54 Very Good G VS2 63.0 59.0 11708.0 7.3 7.36 4.62
1.55 Premium H VS2 61.3 61.0 11708.0 7.46 7.39 4.55
1.71 Ideal J VS1 62.1 55.0 11711.0 7.73 7.65 4.78
1.57 Premium G VS2 62.7 60.0 11711.0 7.46 7.38 4.65
1.57 Premium H VS1 62.8 60.0 11711.0 7.44 7.36 4.65
1.24 Very Good F VVS1 62.7 61.0 11716.0 6.75 6.84 4.26
1.16 Premium G VVS1 61.6 55.0 11717.0 6.85 6.72 4.18
1.53 Very Good H VS2 62.5 61.0 11722.0 7.28 7.38 4.58
1.76 Ideal I VS1 62.0 57.0 11722.0 7.74 7.71 4.79
1.22 Premium F VVS1 61.9 58.0 11723.0 6.81 6.85 4.23
1.39 Ideal E VS2 60.8 57.0 11726.0 7.24 7.21 4.39
1.22 Ideal F VVS2 61.9 53.0 11730.0 6.9 6.92 4.28
1.14 Very Good F VVS1 62.2 55.9 11737.0 6.67 6.69 4.16
1.55 Premium H VS2 60.7 59.0 11738.0 7.46 7.5 4.54
1.51 Fair G VS1 64.9 55.0 11739.0 7.25 7.14 4.67
1.51 Good G VS2 64.2 54.0 11746.0 7.27 7.18 4.64
1.51 Premium G VS2 58.1 61.0 11746.0 7.57 7.54 4.39
1.5 Very Good H VVS2 62.9 59.0 11748.0 7.26 7.31 4.58
1.02 Ideal D VVS2 61.0 56.0 11765.0 6.52 6.55 3.99
1.01 Ideal D VVS1 62.5 55.0 11767.0 6.44 6.39 4.01
1.54 Premium H VS2 61.9 59.0 11769.0 7.33 7.31 4.53
1.36 Very Good G VVS2 60.8 60.0 11774.0 7.12 7.16 4.34
1.7 Premium I VVS2 62.1 59.0 11775.0 7.6 7.53 4.7
1.7 Premium I VS2 62.2 58.0 11781.0 7.6 7.65 4.74
1.58 Premium G VS2 58.2 58.0 11786.0 7.68 7.64 4.46
2.0 Premium J VS1 62.0 62.0 11793.0 8.02 7.91 4.94
1.36 Ideal D VS2 62.1 55.0 11793.0 7.18 7.13 4.44
1.54 Premium D VS2 59.4 59.0 11795.0 7.61 7.55 4.5
1.6 Premium H VS2 62.1 60.0 11796.0 7.51 7.44 4.64
1.27 Premium F VVS2 61.3 60.0 11797.0 6.9 6.99 4.26
1.56 Very Good H VS1 63.9 57.0 11804.0 7.3 7.37 4.69
1.52 Good H VS1 63.5 60.0 11804.0 7.24 7.28 4.61
1.54 Premium G VS2 63.0 59.0 11815.0 7.36 7.3 4.62
1.06 Very Good D VVS1 61.8 57.0 11815.0 6.49 6.52 4.03
1.41 Ideal G VS1 62.6 56.0 11817.0 7.15 7.2 4.49
1.65 Very Good H VS1 62.0 56.0 11823.0 7.53 7.59 4.68
1.51 Good H VVS2 63.1 59.0 11826.0 7.26 7.28 4.59
1.61 Ideal I VS2 61.7 56.0 11826.0 7.52 7.62 4.67
1.22 Premium F VVS1 61.9 58.0 11830.0 6.85 6.81 4.23
1.06 Ideal D VVS2 62.0 57.0 11837.0 6.52 6.54 4.05
1.2 Ideal E VVS2 62.2 57.0 11839.0 6.81 6.77 4.22
1.27 Ideal E VS1 61.8 57.0 11840.0 6.94 6.98 4.3
1.73 Very Good I VS1 63.4 58.0 11843.0 7.57 7.6 4.81
1.14 Ideal F VVS1 62.2 56.0 11844.0 6.69 6.67 4.16
1.55 Premium H VS2 60.7 59.0 11846.0 7.5 7.46 4.54
1.36 Ideal G VVS2 61.1 57.0 11848.0 7.14 7.2 4.38
1.71 Very Good I VS2 62.8 59.0 11850.0 7.52 7.58 4.74
1.46 Good E VS2 63.9 57.0 11851.0 7.06 7.12 4.53
1.3 Ideal G VVS2 60.9 57.0 11853.0 7.04 7.11 4.31
1.52 Premium H VS1 58.4 59.0 11853.0 7.55 7.52 4.4
1.5 Premium H VVS2 62.9 59.0 11855.0 7.31 7.26 4.58
1.51 Ideal H VS1 60.8 57.0 11856.0 7.43 7.46 4.53
1.42 Premium G VS1 61.7 55.0 11861.0 7.29 7.24 4.48
1.67 Very Good I VS1 61.6 59.1 11867.0 7.61 7.64 4.7
1.73 Premium G VS1 61.6 60.0 11867.0 7.67 7.62 4.71
1.35 Premium G VVS2 60.2 59.0 11868.0 7.2 7.16 4.32
1.55 Ideal I VVS1 62.1 56.0 11869.0 7.36 7.43 4.59
1.22 Ideal F VVS2 62.2 54.0 11870.0 6.83 6.87 4.26
1.5 Very Good I VS1 63.3 54.0 11879.0 7.26 7.33 4.62
1.22 Ideal F VVS2 62.7 54.0 11880.0 6.79 6.84 4.27
1.2 Ideal E VVS2 61.5 57.0 11883.0 6.79 6.89 4.21
1.17 Ideal F VVS1 62.1 57.0 11886.0 6.82 6.73 4.21
1.7 Premium I VS2 62.2 58.0 11888.0 7.65 7.6 4.74
1.52 Very Good H VVS2 63.0 60.0 11904.0 7.25 7.3 4.58
1.27 Premium F VVS2 61.3 60.0 11905.0 6.99 6.9 4.26
1.18 Ideal E VVS2 61.5 57.0 11906.0 6.8 6.75 4.17
1.52 Very Good H VS1 63.5 60.0 11912.0 7.28 7.24 4.61
1.2 Very Good F VVS1 59.8 63.0 11913.0 6.82 6.8 4.07
1.7 Good I VS1 58.0 60.0 11921.0 7.84 7.88 4.56
1.56 Ideal H VS2 61.6 57.0 11922.0 7.51 7.45 4.61
1.01 Very Good D VVS1 63.9 56.0 11923.0 6.32 6.36 4.05
1.51 Very Good G VS2 62.8 57.0 11923.0 7.25 7.3 4.57
1.23 Ideal F VVS2 61.9 55.0 11927.0 6.92 6.89 4.27
1.51 Very Good H VVS2 63.1 59.0 11934.0 7.28 7.26 4.59
1.5 Very Good E VS2 61.9 57.0 11939.0 7.31 7.38 4.55
1.36 Very Good F VS1 62.7 60.0 11946.0 7.05 7.02 4.41
1.52 Ideal H VS1 60.1 60.0 11946.0 7.54 7.51 4.52
2.01 Good I VS2 64.3 60.0 11954.0 7.91 7.86 5.07
1.34 Ideal G VVS2 62.0 55.0 11955.0 7.02 7.08 4.37
1.36 Ideal G VVS2 61.1 57.0 11956.0 7.2 7.14 4.38
1.71 Premium I VS2 62.8 59.0 11958.0 7.58 7.52 4.74
1.63 Ideal I VS2 61.8 56.0 11963.0 7.56 7.59 4.68
2.0 Fair J VS2 65.4 58.0 11966.0 7.96 7.75 5.14
2.0 Premium J VS2 62.9 60.0 11966.0 7.99 7.95 5.01
1.51 Ideal H VS1 62.3 57.0 11967.0 7.34 7.29 4.55
2.24 Premium J VS1 60.9 58.0 11970.0 8.46 8.41 5.14
1.27 Ideal F VS1 61.6 55.0 11973.0 6.97 7.03 4.31
1.31 Ideal G VVS2 61.3 58.0 11975.0 7.03 7.07 4.32
1.67 Ideal I VS1 61.6 59.0 11975.0 7.64 7.61 4.7
1.52 Premium H VVS2 61.2 58.0 11979.0 7.48 7.41 4.56
1.53 Premium H VVS2 60.4 60.0 11982.0 7.41 7.46 4.49
1.52 Very Good G VS2 63.4 58.0 11986.0 7.31 7.24 4.61
1.57 Ideal H VS2 61.8 55.0 12004.0 7.45 7.49 4.62
1.5 Very Good G VS1 63.4 59.0 12005.0 7.25 7.19 4.58
1.31 Ideal G VS1 61.6 57.0 12008.0 6.99 7.04 4.32
1.52 Premium H VVS2 63.0 60.0 12013.0 7.3 7.25 4.58
1.5 Very Good G VS2 60.5 57.0 12014.0 7.39 7.43 4.48
1.11 Ideal D VVS2 63.0 57.0 12016.0 6.58 6.65 4.17
1.7 Ideal I VS1 63.0 55.0 12030.0 7.75 7.54 4.76
1.7 Premium I VS1 58.0 60.0 12030.0 7.88 7.84 4.56
1.07 Ideal E VVS2 61.3 56.0 12031.0 6.57 6.62 4.04
1.02 Ideal E VVS1 62.2 58.0 12035.0 6.42 6.44 4.0
1.22 Ideal E VVS2 63.0 55.0 12036.0 6.83 6.78 4.29
1.52 Very Good G VS2 62.9 60.0 12038.0 7.28 7.31 4.59
1.52 Premium H VS1 60.6 58.0 12047.0 7.46 7.39 4.5
1.59 Ideal H VS1 61.8 57.0 12047.0 7.42 7.49 4.61
1.06 Ideal D VVS2 62.0 56.0 12053.0 6.53 6.57 4.06
1.5 Ideal H VS1 61.2 56.0 12055.0 7.39 7.4 4.52
1.24 Ideal F VVS2 62.0 57.0 12059.0 6.86 6.91 4.27
1.54 Ideal H VVS2 62.6 56.0 12061.0 7.35 7.42 4.62
1.28 Ideal F VS1 61.5 55.0 12061.0 6.98 7.01 4.3
1.26 Ideal G VVS2 61.4 55.0 12066.0 6.96 6.99 4.29
1.51 Premium G VS2 61.4 58.0 12068.0 7.4 7.3 4.51
// selecting a subset of fields
display(spark.sql("SELECT carat, clarity, price FROM diamonds WHERE color = 'D'"))
carat clarity price
0.23 VS2 357.0
0.23 VS1 402.0
0.26 VS2 403.0
0.26 VS2 403.0
0.26 VS1 403.0
0.22 VS2 404.0
0.3 SI1 552.0
0.3 SI1 552.0
0.3 SI1 552.0
0.24 VVS1 553.0
0.26 VVS2 554.0
0.26 VVS2 554.0
0.26 VVS1 554.0
0.75 SI1 2760.0
0.71 SI2 2762.0
0.61 VVS2 2763.0
0.71 SI1 2764.0
0.71 SI1 2764.0
0.7 VS2 2767.0
0.71 SI2 2767.0
0.73 SI1 2768.0
0.7 SI1 2768.0
0.71 SI2 2768.0
0.71 VS2 2770.0
0.76 SI2 2770.0
0.73 SI2 2770.0
0.75 SI2 2773.0
0.7 VS2 2773.0
0.7 VS1 2777.0
0.53 VVS2 2782.0
0.75 SI2 2782.0
0.72 SI1 2782.0
0.72 SI1 2782.0
0.7 SI1 2782.0
0.64 VS1 2787.0
0.71 VS2 2788.0
0.72 SI2 2795.0
0.71 SI1 2797.0
0.71 SI1 2797.0
0.71 SI1 2797.0
0.51 VVS1 2797.0
0.78 SI1 2799.0
0.91 SI2 2803.0
0.7 SI1 2804.0
0.7 SI1 2804.0
0.72 SI1 2804.0
0.72 SI1 2804.0
0.73 SI1 2808.0
0.81 SI2 2809.0
0.74 SI2 2810.0
0.83 SI1 2811.0
0.71 SI1 2812.0
0.55 VVS1 2815.0
0.71 VS1 2816.0
0.73 SI1 2821.0
0.71 SI1 2822.0
0.71 SI1 2822.0
0.7 SI1 2822.0
0.7 SI1 2822.0
0.71 SI1 2822.0
0.7 SI1 2822.0
0.7 SI1 2822.0
0.7 SI1 2822.0
0.7 SI1 2822.0
0.79 SI2 2823.0
0.71 VS2 2824.0
0.7 VS2 2826.0
0.7 SI1 2827.0
0.72 VS2 2827.0
0.7 SI2 2828.0
0.7 VS2 2833.0
0.7 VS2 2833.0
0.51 VVS1 2834.0
0.92 SI2 2840.0
0.71 VS1 2841.0
0.73 SI1 2841.0
0.73 SI1 2841.0
0.71 SI1 2843.0
0.79 SI1 2846.0
0.76 SI1 2847.0
0.54 VVS2 2848.0
0.75 SI2 2848.0
0.66 VS1 2851.0
0.79 SI2 2853.0
0.79 SI2 2853.0
0.74 VS2 2855.0
0.73 SI1 2858.0
0.71 VS2 2858.0
0.71 VS2 2858.0
0.7 VS2 2859.0
0.7 VS2 2859.0
0.7 VS2 2859.0
0.71 VS1 2860.0
0.71 SI1 2861.0
0.66 VS1 2861.0
0.7 SI1 2862.0
0.8 SI2 2862.0
0.71 SI1 2863.0
0.71 SI1 2863.0
0.71 SI1 2863.0
0.71 SI1 2863.0
0.71 SI1 2863.0
0.71 SI1 2863.0
0.71 SI1 2863.0
0.73 SI1 2865.0
0.56 VVS1 2866.0
0.56 VVS1 2866.0
0.7 VS2 2867.0
1.08 I1 2869.0
0.7 SI1 2872.0
0.75 SI1 2872.0
0.7 SI1 2872.0
0.7 SI1 2872.0
0.7 SI1 2872.0
0.7 SI1 2872.0
0.71 VS2 2874.0
0.79 SI2 2878.0
0.74 SI1 2880.0
0.72 SI1 2883.0
0.77 SI2 2885.0
0.9 SI2 2885.0
0.71 SI1 2887.0
0.72 SI1 2891.0
0.71 VS2 2891.0
0.79 SI1 2896.0
0.77 SI2 2896.0
0.6 VVS2 2897.0
0.54 VVS2 2897.0
0.74 VS2 2897.0
0.75 SI1 2898.0
0.77 SI1 2898.0
0.72 VS1 2900.0
0.75 SI1 2903.0
0.75 SI1 2903.0
0.72 SI1 2903.0
0.72 SI1 2903.0
0.79 SI2 2904.0
0.53 VVS1 2905.0
0.74 VS2 2906.0
0.32 SI1 558.0
0.7 VS2 2909.0
0.7 VS2 2909.0
0.71 VS1 2910.0
0.7 VS2 2913.0
0.71 SI1 2913.0
0.71 SI1 2913.0
0.71 SI1 2913.0
0.71 SI1 2913.0
0.71 SI1 2913.0
0.83 SI2 2918.0
0.71 SI1 2921.0
0.77 SI2 2922.0
0.77 SI2 2923.0
0.8 SI1 2925.0
0.81 SI2 2926.0
0.7 VS2 2928.0
0.59 VVS2 2933.0
0.75 SI2 2933.0
0.71 SI2 2934.0
0.7 SI2 2936.0
0.77 SI1 2939.0
0.76 SI1 2942.0
0.73 SI1 2943.0
0.57 VVS1 2945.0
0.78 SI1 2945.0
0.73 VS2 2947.0
0.73 SI1 2947.0
0.77 SI1 2949.0
0.71 VS2 2950.0
0.72 VS1 2951.0
0.72 SI1 2954.0
0.72 SI1 2954.0
0.75 SI1 2954.0
0.82 SI1 2954.0
0.7 VS1 2956.0
0.56 VVS1 2959.0
0.7 VS2 2960.0
0.7 VS2 2960.0
0.7 VS2 2960.0
0.63 VVS2 2962.0
0.71 SI1 2964.0
0.71 VS2 2968.0
0.77 SI2 2973.0
1.0 SI2 2974.0
0.76 VS2 2977.0
0.7 SI1 2980.0
0.7 VS2 2985.0
0.74 SI1 2987.0
0.83 SI1 2990.0
0.7 VS2 2991.0
0.72 SI1 2993.0
0.81 SI2 2994.0
0.73 SI1 2995.0
0.77 SI1 2996.0
0.7 VS2 2998.0
0.7 VS2 2999.0
0.72 SI1 3001.0
0.7 VS1 3001.0
0.7 VS1 3001.0
0.7 VS1 3001.0
0.71 VS2 3002.0
1.01 SI2 3003.0
0.65 VVS2 3003.0
0.92 SI2 3004.0
0.55 VVS1 3006.0
0.76 SI1 3007.0
0.7 VS1 3008.0
0.8 SI1 3011.0
0.77 SI2 3011.0
0.9 SI1 3013.0
0.73 SI1 3014.0
0.72 VS2 3016.0
0.5 VVS2 3017.0
0.78 SI1 3019.0
0.71 VS2 3020.0
0.75 SI1 3024.0
0.75 SI1 3024.0
0.65 VVS2 3025.0
0.71 VS2 3033.0
0.7 VS2 3033.0
0.7 VS2 3033.0
0.7 VS2 3033.0
0.78 SI1 3035.0
0.71 SI1 3035.0
0.74 SI1 3036.0
0.61 VVS2 3036.0
0.77 SI1 3040.0
0.71 VS2 3045.0
0.72 VS2 3045.0
0.75 SI1 3046.0
0.73 VS1 3047.0
0.75 SI1 3048.0
0.72 SI1 3048.0
0.72 SI1 3048.0
0.66 VVS2 3049.0
0.62 VVS2 3050.0
0.7 VS2 3052.0
0.7 VS2 3053.0
0.7 VS1 3054.0
0.65 VVS2 3056.0
0.92 SI2 3057.0
0.79 SI1 3058.0
0.72 SI1 3062.0
0.85 SI2 3066.0
0.7 VS2 3073.0
0.72 VS2 3075.0
0.72 VS2 3075.0
0.7 SI1 3075.0
0.76 SI1 3075.0
0.71 VS2 3077.0
0.71 VS2 3077.0
0.75 SI1 3078.0
0.83 SI2 3078.0
0.91 SI2 3079.0
0.79 SI2 3081.0
0.7 VS2 3082.0
0.8 SI2 3082.0
0.71 VS2 3084.0
0.75 SI1 3085.0
0.7 VS2 3087.0
0.7 VS2 3087.0
0.7 VS2 3087.0
0.74 VS2 3087.0
0.71 VS1 3090.0
0.71 VS1 3090.0
0.7 VS2 3092.0
0.7 VS2 3092.0
0.7 VS2 3092.0
0.7 VS1 3093.0
0.71 VS2 3096.0
0.71 VS2 3096.0
0.53 VVS1 3097.0
0.72 VS2 3099.0
0.72 SI1 3102.0
0.66 VVS2 3103.0
0.78 SI1 3103.0
0.75 SI1 3105.0
0.7 VS1 3107.0
0.79 SI1 3112.0
0.94 SI2 3125.0
0.57 VVS1 3126.0
0.57 VVS1 3126.0
0.7 VS2 3129.0
0.7 VS2 3131.0
0.71 VS2 3131.0
0.71 VS2 3135.0
0.71 VS2 3135.0
0.8 VS2 3135.0
0.81 SI1 3135.0
0.71 VS1 3136.0
0.71 VS2 3137.0
0.74 SI1 3138.0
0.72 VS2 3139.0
0.54 VVS1 3139.0
0.73 SI1 3140.0
0.71 VS1 3145.0
0.84 SI2 3145.0
0.78 SI1 3145.0
0.75 SI1 3152.0
0.9 SI2 3153.0
0.71 VS2 3153.0
0.58 VVS1 3154.0
0.8 SI2 3154.0
0.77 SI1 3158.0
0.82 SI2 3159.0
0.77 SI1 3160.0
0.81 SI2 3160.0
0.71 VS2 3161.0
0.71 VS2 3161.0
0.71 VS2 3161.0
0.77 SI1 3166.0
0.8 SI2 3173.0
0.72 SI2 3176.0
0.74 VS2 3177.0
0.72 VS2 3179.0
0.72 VS2 3179.0
0.72 VS2 3179.0
0.81 SI1 3179.0
0.73 VS2 3182.0
0.73 VS2 3182.0
0.7 VS1 3183.0
0.79 SI1 3185.0
0.73 SI1 3189.0
0.73 SI1 3189.0
0.71 VS1 3192.0
0.7 VS1 3193.0
0.54 VVS1 3194.0
0.73 SI1 3195.0
0.8 SI1 3195.0
0.7 SI1 3199.0
0.71 VS2 3203.0
0.9 SI2 3205.0
0.9 SI2 3205.0
0.9 SI2 3205.0
0.9 SI2 3205.0
0.9 SI2 3205.0
0.9 SI2 3205.0
0.9 SI2 3205.0
0.72 VS2 3205.0
0.58 VVS1 3206.0
0.83 SI2 3207.0
0.7 VS1 3208.0
0.79 SI1 3209.0
0.8 SI2 3210.0
0.7 VVS2 3210.0
0.71 VS2 3212.0
0.78 SI1 3214.0
0.7 VS1 3214.0
0.95 SI2 3214.0
0.71 VS2 3217.0
0.71 VS2 3217.0
0.71 VS2 3217.0
0.52 VVS1 3218.0
0.72 VS2 3219.0
0.72 VS2 3219.0
0.71 VS2 3222.0
0.71 VS2 3222.0
0.51 VVS2 3223.0
0.8 SI1 3226.0
0.65 VVS2 3228.0
0.7 VS1 3229.0
0.7 VS1 3229.0
0.7 VS1 3231.0
0.59 VVS1 3234.0
0.71 VS2 3234.0
0.72 VS2 3236.0
0.7 VS1 3239.0
0.7 VS1 3239.0
0.7 VS1 3239.0
0.77 SI1 3241.0
0.79 SI1 3242.0
0.71 VS2 3245.0
0.84 SI2 3246.0
0.25 VS1 563.0
0.26 VVS2 564.0
0.31 SI1 565.0
0.31 SI1 565.0
0.7 VS1 3247.0
0.52 VVS1 3247.0
0.76 VS2 3248.0
0.73 VS2 3250.0
0.77 SI1 3251.0
0.71 SI1 3252.0
0.78 SI1 3253.0
0.73 VS2 3255.0
0.78 SI1 3258.0
0.9 SI2 3262.0
0.71 SI1 3262.0
0.84 SI1 3265.0
0.81 SI1 3266.0
0.7 VVS2 3267.0
0.56 VVS1 3270.0
0.79 SI1 3270.0
0.72 VS2 3275.0
0.92 SI2 3277.0
0.7 VS1 3278.0
0.52 VVS2 3284.0
0.86 SI2 3284.0
0.7 VS1 3287.0
0.7 VS1 3287.0
0.77 VS2 3291.0
0.76 VS2 3293.0
0.74 VS2 3294.0
0.7 VVS2 3296.0
0.91 SI2 3298.0
0.78 VS2 3298.0
0.78 VS2 3298.0
0.71 VS2 3299.0
1.0 SI2 3304.0
1.0 SI2 3304.0
1.0 SI2 3304.0
0.76 VS2 3306.0
0.76 SI1 3306.0
0.53 VVS1 3307.0
0.73 VS2 3308.0
0.77 SI1 3309.0
0.31 SI1 565.0
0.31 SI1 565.0
0.8 SI1 3312.0
0.7 VVS2 3312.0
0.8 SI1 3312.0
0.9 SI2 3312.0
0.9 SI2 3312.0
0.7 VVS2 3312.0
0.9 SI2 3312.0
0.71 SI1 3316.0
0.73 VS2 3319.0
0.52 VVS1 3321.0
0.71 VS2 3321.0
0.71 VS2 3321.0
0.72 SI1 3322.0
0.81 SI1 3324.0
0.78 SI1 3326.0
0.79 SI1 3328.0
0.71 VS1 3332.0
0.71 VS1 3333.0
0.92 SI2 3335.0
0.7 VS1 3335.0
0.61 VVS2 3336.0
1.01 SI2 3337.0
0.77 SI1 3345.0
0.53 VVS2 3346.0
0.73 VS2 3346.0
0.83 SI1 3347.0
0.91 SI2 3349.0
0.77 VS2 3351.0
0.76 VS2 3352.0
0.74 VS2 3353.0
0.76 VS1 3353.0
0.81 SI1 3353.0
0.82 SI2 3357.0
0.91 SI1 3357.0
0.7 VS2 3360.0
0.7 VS1 3361.0
0.7 VS1 3365.0
0.74 VS1 3365.0
0.71 VS2 3366.0
0.69 VVS2 3369.0
0.9 SI2 3371.0
0.9 SI2 3371.0
0.71 VS2 3372.0
0.52 VVS1 3373.0
0.7 VS1 3375.0
0.72 VS1 3375.0
0.5 IF 3378.0
0.5 IF 3378.0
0.6 VVS2 3382.0
0.27 VS2 567.0
0.31 VS2 567.0
0.33 SI1 567.0
0.33 SI1 567.0
0.33 SI1 567.0
0.3 VS2 568.0
0.9 SI1 3382.0
0.95 SI2 3384.0
0.76 VS2 3384.0
0.78 SI1 3389.0
0.88 SI2 3390.0
0.61 VVS2 3397.0
0.85 SI2 3398.0
0.76 VS2 3401.0
0.91 SI2 3403.0
0.71 VS1 3406.0
0.71 VS1 3406.0
0.91 SI2 3408.0
0.7 VS1 3410.0
0.73 VS2 3411.0
0.73 VS2 3412.0
0.8 VS2 3419.0
0.7 VS1 3419.0
0.96 SI2 3419.0
0.96 SI2 3419.0
0.71 VS1 3420.0
0.9 SI2 3425.0
0.7 VS1 3425.0
0.77 VS2 3428.0
0.77 VS2 3428.0
0.77 VS2 3428.0
0.77 VS2 3428.0
0.79 SI1 3432.0
0.73 VS2 3440.0
0.8 SI1 3441.0
0.53 VVS1 3442.0
0.77 VS2 3442.0
0.76 VS2 3443.0
0.76 VS2 3443.0
0.51 IF 3446.0
0.51 IF 3446.0
0.7 VS2 3448.0
0.72 VS2 3450.0
0.3 VS2 568.0
0.74 VS2 3454.0
0.78 SI2 3454.0
0.7 SI1 3454.0
0.75 VS2 3456.0
0.72 VVS2 3459.0
0.74 VS1 3461.0
0.81 SI1 3462.0
0.91 SI2 3463.0
0.7 VS1 3463.0
0.73 VS2 3464.0
0.56 VVS1 3465.0
0.71 VS1 3465.0
0.73 VS2 3467.0
0.55 VVS2 3468.0
0.55 VVS2 3468.0
0.55 VVS2 3468.0
0.7 VS1 3471.0
0.7 SI1 3471.0
0.7 SI1 3471.0
0.9 SI2 3473.0
0.9 SI2 3473.0
0.9 SI2 3473.0
0.9 SI2 3473.0
0.9 SI2 3473.0
0.78 VS2 3473.0
0.74 VS2 3476.0
0.7 VS1 3477.0
0.71 VS1 3479.0
0.96 SI2 3480.0
0.74 VS2 3487.0
0.77 VS2 3489.0
0.77 VS2 3489.0
0.72 VS2 3493.0
0.54 VVS1 3494.0
0.72 VS2 3495.0
0.56 VVS1 3496.0
0.74 VS2 3498.0
0.7 VS1 3501.0
0.8 SI1 3502.0
0.71 SI1 3502.0
0.71 SI1 3502.0
0.71 SI1 3502.0
0.9 SI1 3505.0
0.55 IF 3509.0
0.73 VS1 3509.0
0.91 SI2 3511.0
0.74 SI1 3517.0
0.53 IF 3517.0
0.71 VS1 3518.0
0.72 VS1 3522.0
0.71 VS1 3524.0
0.73 VS2 3528.0
0.7 VS1 3529.0
0.32 SI2 571.0
0.31 SI1 571.0
0.31 SI1 571.0
0.31 SI1 571.0
0.31 SI1 571.0
0.31 SI1 571.0
0.31 SI1 571.0
0.31 SI1 571.0
0.31 SI1 571.0
0.31 SI1 571.0
0.31 SI1 571.0
0.31 SI1 571.0
0.31 SI1 571.0
0.31 SI1 571.0
0.31 SI1 571.0
0.31 SI1 571.0
0.31 SI1 571.0
0.31 SI1 571.0
0.9 SI2 3534.0
0.9 SI2 3534.0
0.9 SI2 3534.0
0.9 SI2 3534.0
0.78 VS2 3534.0
0.7 VS1 3535.0
0.93 SI2 3540.0
0.71 VS2 3540.0
0.72 VS2 3543.0
0.72 SI1 3550.0
0.92 SI2 3550.0
0.72 VS1 3554.0
0.83 SI1 3556.0
0.83 SI1 3556.0
0.73 VS1 3557.0
0.7 VS2 3561.0
0.75 VS2 3562.0
0.8 SI1 3564.0
0.9 SI1 3567.0
0.7 VS1 3567.0
0.9 SI1 3568.0
0.72 SI1 3568.0
1.0 SI2 3569.0
0.72 VS1 3570.0
0.6 VVS1 3570.0
0.91 SI2 3573.0
0.71 VS1 3576.0
0.9 SI2 3578.0
0.9 SI2 3579.0
0.76 VS2 3581.0
0.71 VS1 3582.0
0.97 SI2 3585.0
1.11 I1 3589.0
0.82 SI1 3593.0
0.78 VS2 3595.0
0.8 SI1 3597.0
0.72 VS1 3601.0
1.01 SI2 3604.0
0.9 VS2 3604.0
1.01 SI2 3605.0
0.79 SI1 3605.0
1.03 SI2 3607.0
0.31 SI1 571.0
0.31 SI1 571.0
0.31 SI1 571.0
0.31 SI1 571.0
0.31 SI1 571.0
0.31 SI1 571.0
0.31 SI1 571.0
0.31 SI1 571.0
0.31 SI1 571.0
0.31 SI1 571.0
0.31 SI1 571.0
0.31 SI1 571.0
0.31 SI1 571.0
0.31 SI1 571.0
0.31 SI1 571.0
0.31 SI1 571.0
0.31 SI1 571.0
0.31 SI1 571.0
0.31 SI1 571.0
0.31 SI1 571.0
0.31 SI1 571.0
0.31 SI1 571.0
0.31 SI1 571.0
0.31 SI1 571.0
0.31 SI1 571.0
0.92 SI2 3613.0
0.73 SI1 3615.0
0.7 VS1 3618.0
0.7 VS1 3618.0
0.71 VVS2 3618.0
0.72 VS1 3619.0
0.73 VS1 3620.0
0.7 VVS2 3622.0
0.7 VVS2 3622.0
0.72 VS1 3622.0
0.72 VS1 3622.0
0.75 VS2 3625.0
0.61 VVS1 3625.0
0.72 VS1 3629.0
0.9 SI2 3632.0
0.94 SI2 3634.0
1.0 SI2 3634.0
1.0 SI2 3634.0
1.0 SI2 3634.0
1.0 SI2 3634.0
0.9 SI2 3643.0
0.77 VS1 3643.0
1.16 I1 3644.0
0.77 VS1 3644.0
1.11 I1 3655.0
0.91 SI2 3660.0
0.87 SI1 3664.0
0.7 VS2 3668.0
0.78 VS2 3668.0
0.74 VS2 3668.0
0.85 SI1 3669.0
0.71 VVS2 3670.0
1.01 SI2 3671.0
1.01 SI2 3671.0
0.78 VS2 3672.0
0.73 VS2 3673.0
0.71 SI1 3674.0
0.71 SI1 3674.0
1.03 SI2 3675.0
0.75 VS2 3679.0
0.31 SI1 571.0
0.31 SI1 571.0
0.31 SI1 571.0
0.8 SI2 3682.0
0.84 SI1 3685.0
0.9 SI1 3689.0
0.9 SI1 3689.0
0.9 SI1 3689.0
0.9 SI1 3689.0
0.9 SI1 3689.0
0.9 SI1 3689.0
0.71 VS1 3690.0
0.94 SI2 3691.0
0.75 VS1 3696.0
0.9 SI2 3706.0
0.92 SI2 3707.0
0.86 SI1 3709.0
1.16 I1 3711.0
0.75 SI1 3712.0
0.71 VS1 3716.0
0.71 VS1 3718.0
0.77 VS2 3721.0
0.72 SI1 3722.0
0.91 SI1 3730.0
0.91 SI1 3730.0
0.91 SI1 3730.0
0.58 VVS1 3732.0
0.76 SI1 3732.0
0.73 VS2 3735.0
0.78 VS2 3736.0
0.7 VVS2 3737.0
0.9 SI2 3740.0
0.9 SI2 3740.0
0.9 SI2 3740.0
0.9 SI2 3740.0
0.58 VVS1 3741.0
0.87 SI1 3742.0
1.09 SI2 3742.0
1.03 SI2 3743.0
1.03 SI2 3743.0
0.93 SI2 3744.0
0.74 VS1 3746.0
0.3 SI2 574.0
0.9 SI1 3751.0
0.7 VS1 3752.0
0.9 SI1 3755.0
0.9 SI1 3755.0
0.77 VS2 3755.0
0.61 VVS2 3758.0
0.78 VS2 3763.0
0.91 SI2 3763.0
1.0 SI2 3767.0
1.02 I1 3769.0
1.02 SI2 3773.0
0.83 SI2 3774.0
1.04 SI2 3780.0
1.04 SI2 3780.0
0.9 SI2 3780.0
1.04 SI2 3780.0
1.5 I1 3780.0
0.91 SI2 3781.0
0.91 SI2 3781.0
0.77 VS2 3787.0
0.7 VS2 3788.0
0.9 SI2 3789.0
0.59 VVS1 3791.0
0.91 SI1 3796.0
0.79 VS1 3798.0
0.9 SI2 3798.0
0.9 SI2 3798.0
0.9 SI2 3798.0
0.71 VVS2 3799.0
0.78 VS1 3800.0
0.71 VS1 3801.0
0.9 SI2 3806.0
0.9 SI2 3806.0
0.9 SI2 3806.0
0.84 SI1 3809.0
0.78 VS2 3811.0
0.74 VS1 3812.0
0.53 IF 3812.0
0.93 SI1 3812.0
0.9 SI1 3812.0
0.9 SI1 3812.0
0.9 SI1 3812.0
0.93 SI1 3812.0
0.74 VS1 3813.0
1.18 I1 3816.0
0.84 SI1 3816.0
1.05 SI2 3816.0
0.79 VS2 3818.0
0.9 SI2 3818.0
0.3 SI2 574.0
0.3 SI2 574.0
0.3 SI2 574.0
0.3 SI2 574.0
0.3 SI2 574.0
0.3 SI2 574.0
0.3 SI2 574.0
0.3 SI2 574.0
0.3 SI2 574.0
0.3 SI2 574.0
0.3 SI2 574.0
0.3 SI2 574.0
0.85 VS2 3821.0
0.92 SI2 3823.0
0.53 IF 3827.0
0.91 SI2 3828.0
0.63 IF 3832.0
0.91 SI2 3837.0
0.77 VS2 3837.0
0.71 VS2 3838.0
1.02 I1 3838.0
1.02 SI2 3839.0
0.93 SI2 3839.0
0.7 VS1 3840.0
1.02 SI2 3842.0
0.92 SI2 3843.0
0.9 SI2 3847.0
0.91 SI2 3848.0
0.91 SI2 3848.0
0.91 SI2 3848.0
0.6 VVS1 3850.0
0.81 SI1 3852.0
0.91 SI1 3855.0
0.73 VS1 3856.0
0.71 VVS2 3856.0
0.74 VS2 3858.0
0.94 SI2 3862.0
0.78 VS2 3864.0
1.17 SI2 3866.0
0.9 SI2 3871.0
1.01 SI2 3871.0
0.87 VS2 3873.0
0.92 SI2 3877.0
0.71 VVS2 3877.0
0.9 SI1 3880.0
0.9 SI1 3880.0
0.9 SI1 3880.0
0.93 SI1 3880.0
1.13 SI2 3883.0
1.18 I1 3886.0
0.91 SI2 3889.0
0.3 SI2 574.0
0.3 SI2 574.0
0.3 SI2 574.0
0.3 SI2 574.0
0.3 SI2 574.0
0.3 SI2 574.0
0.25 VVS2 575.0
0.27 VVS2 575.0
0.25 VVS2 575.0
1.09 SI2 3890.0
0.92 SI2 3891.0
1.0 SI2 3894.0
0.76 VS1 3894.0
0.72 VS1 3896.0
1.18 SI2 3899.0
1.02 SI2 3909.0
1.02 SI2 3909.0
0.91 SI2 3910.0
0.91 SI2 3911.0
0.66 VVS2 3915.0
0.92 SI2 3916.0
0.9 SI2 3918.0
0.7 VVS1 3920.0
0.78 VS1 3923.0
0.9 VS2 3931.0
1.01 SI2 3932.0
0.83 SI1 3933.0
0.92 SI2 3936.0
0.73 VS1 3937.0
0.91 SI2 3943.0
0.9 SI1 3945.0
0.91 SI2 3949.0
1.14 I1 3950.0
0.76 VS1 3950.0
0.71 VVS1 3952.0
0.91 SI2 3958.0
1.01 SI2 3959.0
0.75 VS1 3961.0
1.09 SI2 3961.0
0.88 SI2 3962.0
1.0 SI2 3965.0
1.0 SI2 3965.0
1.0 SI2 3965.0
1.0 SI2 3965.0
1.0 SI2 3965.0
0.33 SI1 575.0
1.0 SI2 3965.0
0.77 VS1 3966.0
0.62 VVS1 3968.0
1.02 SI2 3971.0
0.9 SI2 3975.0
0.9 SI2 3975.0
1.23 I1 3977.0
0.77 VS2 3980.0
0.73 VS2 3980.0
0.83 VS1 3984.0
0.9 SI2 3989.0
0.96 SI2 3989.0
0.9 SI2 3990.0
0.93 SI2 3990.0
0.83 SI1 3990.0
0.92 SI2 3997.0
0.9 SI1 3997.0
0.9 SI1 3997.0
0.9 SI1 3997.0
0.9 SI1 3997.0
0.9 SI1 3997.0
0.7 VS1 4003.0
1.01 SI2 4004.0
0.75 VS1 4007.0
0.9 SI2 4007.0
0.9 SI2 4007.0
0.87 SI2 4012.0
0.71 VVS2 4014.0
0.7 VVS2 4022.0
0.65 VVS1 4022.0
1.14 I1 4022.0
0.56 IF 4025.0
0.71 VS2 4029.0
0.71 VS2 4029.0
0.71 VS2 4029.0
0.71 VS2 4029.0
0.71 VS2 4029.0
0.57 IF 4032.0
0.77 VS1 4037.0
0.77 VS1 4039.0
0.74 VVS2 4040.0
0.91 SI1 4041.0
0.54 VVS1 4042.0
1.02 SI2 4044.0
1.02 SI2 4044.0
1.02 SI2 4044.0
0.72 VS1 4047.0
1.23 I1 4050.0
0.91 SI2 4051.0
0.91 SI2 4051.0
0.91 SI2 4051.0
0.96 SI2 4060.0
1.01 SI2 4064.0
1.0 SI2 4065.0
0.91 SI2 4067.0
0.9 SI1 4068.0
0.9 SI1 4068.0
0.9 SI1 4068.0
0.9 SI1 4068.0
0.9 SI1 4068.0
0.9 SI1 4068.0
1.12 SI2 4071.0
1.01 SI2 4072.0
0.9 SI2 4078.0
0.9 SI2 4078.0
0.9 SI2 4078.0
0.72 VS2 4082.0
0.72 VS2 4082.0
0.64 VVS1 4084.0
0.92 SI1 4086.0
0.81 VS2 4087.0
0.7 VS1 4095.0
0.92 SI2 4096.0
0.92 SI2 4096.0
0.25 VS1 410.0
0.23 VS2 411.0
0.27 VS1 413.0
0.3 SI2 413.0
0.3 SI2 413.0
0.23 VS2 577.0
0.91 VS2 4107.0
0.91 VS2 4107.0
0.87 SI1 4108.0
0.91 SI1 4113.0
0.82 SI1 4113.0
0.9 SI2 4114.0
0.73 VS1 4116.0
0.9 SI1 4117.0
1.01 SI1 4118.0
0.9 SI1 4120.0
0.91 SI2 4123.0
0.91 SI2 4123.0
0.91 SI2 4123.0
1.04 SI2 4123.0
0.9 VS2 4128.0
0.9 SI1 4130.0
0.9 SI2 4133.0
0.73 VS2 4134.0
0.73 VS2 4134.0
0.82 SI1 4135.0
0.82 SI1 4135.0
1.12 I1 4139.0
0.93 SI2 4140.0
0.93 SI2 4140.0
0.92 SI2 4150.0
0.76 VVS2 4150.0
1.0 SI1 4155.0
1.06 SI2 4155.0
0.92 SI1 4158.0
0.92 SI1 4158.0
0.83 SI1 4159.0
0.59 IF 4161.0
0.93 SI2 4165.0
0.91 SI1 4165.0
0.9 SI2 4167.0
0.92 SI2 4168.0
0.92 SI2 4168.0
1.19 SI2 4168.0
0.8 VS2 4170.0
0.6 VVS1 4172.0
1.03 SI2 4177.0
0.9 SI1 4178.0
//renaming a field using as
display(spark.sql("SELECT carat AS carrot, clarity, price FROM diamonds"))
carrot clarity price
0.23 SI2 326.0
0.21 SI1 326.0
0.23 VS1 327.0
0.29 VS2 334.0
0.31 SI2 335.0
0.24 VVS2 336.0
0.24 VVS1 336.0
0.26 SI1 337.0
0.22 VS2 337.0
0.23 VS1 338.0
0.3 SI1 339.0
0.23 VS1 340.0
0.22 SI1 342.0
0.31 SI2 344.0
0.2 SI2 345.0
0.32 I1 345.0
0.3 SI2 348.0
0.3 SI1 351.0
0.3 SI1 351.0
0.3 SI1 351.0
0.3 SI2 351.0
0.23 VS2 352.0
0.23 VS1 353.0
0.31 SI1 353.0
0.31 SI1 353.0
0.23 VVS2 354.0
0.24 VS1 355.0
0.3 VS2 357.0
0.23 VS2 357.0
0.23 VS1 357.0
0.23 VS1 402.0
0.23 VS1 402.0
0.23 VS1 402.0
0.23 VS1 402.0
0.23 VS1 402.0
0.23 VS1 402.0
0.23 VS1 402.0
0.31 SI1 402.0
0.26 VS2 403.0
0.33 SI2 403.0
0.33 SI2 403.0
0.33 SI1 403.0
0.26 VS2 403.0
0.26 VS1 403.0
0.32 SI2 403.0
0.29 SI1 403.0
0.32 SI2 403.0
0.32 SI2 403.0
0.25 VS2 404.0
0.29 SI2 404.0
0.24 SI1 404.0
0.23 VS1 404.0
0.32 SI1 404.0
0.22 VS2 404.0
0.22 VS2 404.0
0.3 SI2 405.0
0.3 SI2 405.0
0.3 SI1 405.0
0.3 SI1 405.0
0.3 SI1 405.0
0.35 VS1 552.0
0.3 SI1 552.0
0.3 SI1 552.0
0.3 SI1 552.0
0.42 SI2 552.0
0.28 VVS2 553.0
0.32 VVS1 553.0
0.31 SI1 553.0
0.31 SI1 553.0
0.24 VVS1 553.0
0.24 VVS1 553.0
0.3 SI1 554.0
0.3 SI1 554.0
0.3 SI1 554.0
0.3 SI1 554.0
0.26 VVS2 554.0
0.26 VVS2 554.0
0.26 VVS2 554.0
0.26 VVS2 554.0
0.26 VVS1 554.0
0.26 VVS1 554.0
0.26 VVS1 554.0
0.26 VVS2 554.0
0.38 SI2 554.0
0.26 VVS1 554.0
0.24 VVS1 554.0
0.24 VVS1 554.0
0.24 VVS1 554.0
0.24 VVS2 554.0
0.32 SI1 554.0
0.7 SI1 2757.0
0.86 SI2 2757.0
0.7 VS2 2757.0
0.71 VS2 2759.0
0.78 SI2 2759.0
0.7 VS2 2759.0
0.7 VS1 2759.0
0.96 SI2 2759.0
0.73 SI1 2760.0
0.8 SI1 2760.0
0.75 SI1 2760.0
0.75 SI1 2760.0
0.74 SI1 2760.0
0.75 VS2 2760.0
0.8 VS1 2760.0
0.75 SI1 2760.0
0.8 SI1 2760.0
0.74 VVS2 2761.0
0.81 SI2 2761.0
0.59 VVS2 2761.0
0.8 SI2 2761.0
0.74 SI2 2761.0
0.9 VS2 2761.0
0.74 SI1 2762.0
0.73 VS2 2762.0
0.73 VS2 2762.0
0.8 SI2 2762.0
0.71 VS2 2762.0
0.7 VS2 2762.0
0.8 SI2 2762.0
0.71 SI2 2762.0
0.74 SI1 2762.0
0.7 VS2 2762.0
0.7 VS2 2762.0
0.7 VS2 2762.0
0.7 VS2 2762.0
0.91 SI1 2763.0
0.61 VVS2 2763.0
0.91 SI2 2763.0
0.91 SI2 2763.0
0.77 VS2 2763.0
0.71 SI1 2764.0
0.71 SI1 2764.0
0.7 VS2 2765.0
0.77 VS1 2765.0
0.63 VVS1 2765.0
0.71 VS1 2765.0
0.71 VS1 2765.0
0.76 SI1 2765.0
0.64 VVS1 2766.0
0.71 VS2 2766.0
0.71 VS2 2766.0
0.7 VS2 2767.0
0.7 VS1 2767.0
0.71 SI2 2767.0
0.7 VVS2 2767.0
0.71 VS1 2768.0
0.73 SI1 2768.0
0.7 SI1 2768.0
0.7 SI1 2768.0
0.71 SI2 2768.0
0.74 SI1 2769.0
0.71 VS2 2770.0
0.73 VS2 2770.0
0.76 SI1 2770.0
0.76 SI2 2770.0
0.71 SI1 2770.0
0.73 VS2 2770.0
0.73 VS1 2770.0
0.73 SI2 2770.0
0.73 VS2 2770.0
0.72 VVS2 2771.0
0.73 SI1 2771.0
0.71 VS2 2771.0
0.79 SI2 2771.0
0.73 VVS1 2772.0
0.8 SI2 2772.0
0.58 VVS1 2772.0
0.58 VVS1 2772.0
0.71 VS2 2772.0
0.75 SI2 2773.0
0.7 VS2 2773.0
1.17 I1 2774.0
0.6 VS1 2774.0
0.7 SI1 2774.0
0.83 VS2 2774.0
0.74 VS2 2775.0
0.72 VS2 2776.0
0.71 VS2 2776.0
0.71 VS2 2776.0
0.54 VVS2 2776.0
0.54 VVS2 2776.0
0.72 SI1 2776.0
0.72 SI1 2776.0
0.72 VS2 2776.0
0.71 SI1 2776.0
0.7 VS1 2777.0
0.71 VS2 2777.0
0.71 VS2 2777.0
0.71 VS2 2777.0
0.71 VS2 2777.0
0.7 VS2 2777.0
0.7 VS2 2777.0
0.7 SI1 2777.0
0.7 SI1 2777.0
0.7 SI1 2777.0
0.7 SI1 2777.0
0.7 SI1 2777.0
0.7 SI1 2777.0
0.72 SI1 2777.0
0.7 SI1 2777.0
0.7 SI1 2777.0
0.7 SI1 2777.0
0.7 VS2 2777.0
0.98 SI2 2777.0
0.78 SI1 2777.0
0.7 SI1 2777.0
0.52 VVS1 2778.0
0.73 VS2 2779.0
0.74 SI1 2779.0
0.7 VS2 2780.0
0.77 VS2 2780.0
0.71 VS2 2780.0
0.74 VS1 2780.0
0.7 VS1 2780.0
1.01 I1 2781.0
0.77 SI1 2781.0
0.78 SI1 2781.0
0.72 VS1 2782.0
0.53 VVS2 2782.0
0.76 VS2 2782.0
0.7 VS1 2782.0
0.7 VS1 2782.0
0.75 SI2 2782.0
0.72 SI1 2782.0
0.72 SI1 2782.0
0.7 SI1 2782.0
0.84 SI1 2782.0
0.75 SI1 2782.0
0.52 IF 2783.0
0.72 VS2 2784.0
0.79 VS1 2784.0
0.72 VS2 2787.0
0.51 VVS1 2787.0
0.64 VS1 2787.0
0.7 VVS1 2788.0
0.83 VS1 2788.0
0.76 VVS2 2788.0
0.71 VS2 2788.0
0.77 VS1 2788.0
0.71 SI1 2788.0
1.01 I1 2788.0
1.01 SI2 2788.0
0.77 SI1 2789.0
0.76 SI1 2789.0
0.76 SI1 2789.0
0.76 SI1 2789.0
1.05 SI2 2789.0
0.81 SI2 2789.0
0.7 SI1 2789.0
0.55 IF 2789.0
0.81 SI2 2789.0
0.63 VVS2 2789.0
0.63 VVS1 2789.0
0.77 VS1 2789.0
1.05 SI2 2789.0
0.64 IF 2790.0
0.76 VVS1 2790.0
0.83 SI2 2790.0
0.71 VS1 2790.0
0.71 VS1 2790.0
0.87 SI1 2791.0
0.73 SI1 2791.0
0.71 SI1 2792.0
0.71 SI1 2792.0
0.71 SI1 2792.0
0.7 VS1 2792.0
0.7 VS1 2792.0
0.76 VVS2 2792.0
0.7 VS1 2792.0
0.79 SI1 2793.0
0.7 VS2 2793.0
0.7 VS2 2793.0
0.76 VS2 2793.0
0.73 VS2 2793.0
0.79 SI1 2794.0
0.71 VS2 2795.0
0.81 VVS2 2795.0
0.81 SI2 2795.0
0.72 VS1 2795.0
0.72 SI2 2795.0
0.72 IF 2795.0
0.81 VS2 2795.0
0.72 VS2 2795.0
1.0 SI2 2795.0
0.73 SI1 2796.0
0.81 SI2 2797.0
0.81 SI1 2797.0
0.71 SI1 2797.0
0.71 SI1 2797.0
0.71 SI1 2797.0
0.57 VVS2 2797.0
0.51 VVS1 2797.0
0.72 VS2 2797.0
0.74 VS1 2797.0
0.74 VS1 2797.0
0.7 VVS1 2797.0
0.8 SI2 2797.0
1.01 SI2 2797.0
0.8 VS2 2797.0
0.77 VS1 2798.0
0.83 SI2 2799.0
0.82 SI2 2799.0
0.78 SI1 2799.0
0.6 IF 2800.0
0.9 SI2 2800.0
0.7 VS1 2800.0
0.9 SI2 2800.0
0.83 SI1 2800.0
0.83 SI1 2800.0
0.83 SI1 2800.0
0.74 VS1 2800.0
0.79 VS1 2800.0
0.61 IF 2800.0
0.76 VS1 2800.0
0.96 I1 2801.0
0.73 VS2 2801.0
0.73 VS2 2801.0
0.75 SI1 2801.0
0.71 VS2 2801.0
0.71 VS2 2801.0
0.71 VS2 2801.0
0.71 VS2 2801.0
1.04 I1 2801.0
1.0 SI2 2801.0
0.87 SI2 2802.0
0.53 IF 2802.0
0.72 VS2 2802.0
0.72 VS1 2802.0
0.7 VS2 2803.0
0.74 SI1 2803.0
0.71 VS2 2803.0
0.73 SI1 2803.0
0.7 VS1 2803.0
0.71 VS2 2803.0
0.71 VS2 2803.0
0.71 VS1 2803.0
0.77 VS2 2803.0
0.71 VS2 2803.0
0.78 VS2 2803.0
0.71 VS1 2803.0
0.91 SI2 2803.0
0.71 VS2 2804.0
0.71 VS2 2804.0
0.8 SI2 2804.0
0.7 SI1 2804.0
0.72 VS1 2804.0
0.72 VS1 2804.0
0.82 VS2 2804.0
0.7 SI1 2804.0
0.72 SI1 2804.0
0.72 SI1 2804.0
0.9 SI1 2804.0
0.74 VS2 2805.0
0.74 VS2 2805.0
0.73 SI2 2805.0
0.57 VVS1 2805.0
0.73 VS2 2805.0
0.72 VS2 2805.0
0.74 VS2 2805.0
0.82 VS2 2805.0
0.81 SI1 2806.0
0.75 VVS1 2806.0
0.7 SI1 2806.0
0.71 VS1 2807.0
0.71 VS1 2807.0
0.93 SI2 2807.0
0.8 VS2 2808.0
0.7 VS1 2808.0
1.0 I1 2808.0
0.75 VS2 2808.0
0.58 VVS2 2808.0
0.73 SI1 2808.0
0.81 SI1 2809.0
0.81 SI2 2809.0
0.71 SI1 2809.0
1.2 I1 2809.0
0.7 VS1 2810.0
0.7 VS1 2810.0
0.74 SI2 2810.0
0.7 VS1 2810.0
0.8 SI1 2810.0
0.75 SI1 2811.0
0.83 SI1 2811.0
1.0 VS2 2811.0
0.99 SI2 2811.0
0.7 VS1 2812.0
0.7 VS2 2812.0
0.7 SI1 2812.0
0.7 VS2 2812.0
0.32 SI1 554.0
0.32 SI1 554.0
0.32 SI1 554.0
0.32 SI1 554.0
0.32 SI1 554.0
0.32 SI1 554.0
0.3 SI1 554.0
0.3 SI1 554.0
0.3 SI1 554.0
0.3 SI1 554.0
0.3 SI1 554.0
0.3 SI1 554.0
0.3 SI1 554.0
0.32 SI1 554.0
0.33 SI2 554.0
0.29 VS1 555.0
0.29 VS1 555.0
0.31 SI1 555.0
0.34 VS2 555.0
0.34 VS2 555.0
0.34 VS1 555.0
0.34 VS1 555.0
0.3 VS1 555.0
0.29 VS1 555.0
0.35 SI1 555.0
0.43 I1 555.0
0.32 VS2 556.0
0.36 VS2 556.0
0.3 VS2 556.0
0.26 VS1 556.0
0.7 VS2 2812.0
0.7 VS2 2812.0
0.71 SI1 2812.0
0.99 SI1 2812.0
0.73 VS2 2812.0
0.51 VVS1 2812.0
0.91 SI2 2813.0
0.84 SI1 2813.0
0.91 VS2 2813.0
0.76 SI1 2814.0
0.76 SI1 2814.0
0.75 SI1 2814.0
0.55 VVS1 2815.0
0.76 SI2 2815.0
0.74 VS1 2815.0
0.7 SI1 2815.0
0.7 SI1 2815.0
0.7 SI1 2815.0
0.7 SI1 2815.0
0.9 VS2 2815.0
0.95 SI2 2815.0
0.89 SI2 2815.0
0.72 VS2 2815.0
0.96 SI2 2815.0
1.02 I1 2815.0
0.78 VVS2 2816.0
0.61 VVS2 2816.0
0.71 VS1 2816.0
0.78 SI1 2816.0
0.87 SI2 2816.0
0.83 SI1 2816.0
0.71 SI1 2817.0
0.71 VVS2 2817.0
0.71 VS2 2817.0
0.71 VS2 2817.0
0.63 VVS2 2817.0
0.71 SI1 2817.0
0.71 SI1 2817.0
0.71 SI1 2817.0
0.71 SI1 2817.0
0.9 VS2 2817.0
0.71 SI1 2817.0
0.7 VS2 2818.0
0.7 VS2 2818.0
0.7 VS2 2818.0
1.0 I1 2818.0
0.86 SI2 2818.0
0.8 SI1 2818.0
0.7 VS1 2818.0
0.7 VS1 2818.0
0.7 VS2 2818.0
0.7 VS1 2818.0
1.0 SI2 2818.0
0.72 VS1 2819.0
0.72 VS1 2819.0
0.7 VS1 2819.0
0.86 SI2 2819.0
0.71 VS1 2820.0
0.75 SI1 2821.0
0.73 VS2 2821.0
0.53 VVS1 2821.0
0.73 SI1 2821.0
0.73 SI1 2821.0
0.73 SI1 2821.0
0.73 SI1 2821.0
0.73 SI1 2821.0
0.73 SI1 2821.0
0.7 VS1 2822.0
0.72 VS2 2822.0
0.72 VS2 2822.0
0.72 VS2 2822.0
0.7 VS2 2822.0
0.6 VVS2 2822.0
0.74 VVS1 2822.0
0.73 SI1 2822.0
0.71 SI1 2822.0
0.71 SI1 2822.0
0.7 SI1 2822.0
0.7 SI1 2822.0
0.9 VS2 2822.0
0.71 SI1 2822.0
0.7 VS2 2822.0
0.7 VS2 2822.0
0.7 SI1 2822.0
0.7 SI1 2822.0
0.7 SI1 2822.0
0.7 SI1 2822.0
0.7 VS2 2822.0
0.7 VS2 2822.0
0.7 SI1 2822.0
0.79 SI2 2823.0
0.9 SI1 2823.0
0.71 VS2 2823.0
0.61 VVS2 2823.0
0.9 SI2 2823.0
0.71 SI1 2823.0
0.71 VS2 2824.0
0.77 VVS2 2824.0
0.74 VS1 2824.0
0.82 SI2 2824.0
0.82 SI2 2824.0
0.71 VS1 2825.0
0.83 SI1 2825.0
0.73 VS1 2825.0
0.83 SI1 2825.0
1.17 I1 2825.0
0.91 SI2 2825.0
0.73 VS1 2826.0
0.7 VS1 2826.0
0.9 SI2 2826.0
0.7 VS1 2826.0
0.7 VS2 2826.0
0.7 VS1 2826.0
0.9 SI2 2826.0
0.78 SI1 2826.0
0.96 I1 2826.0
0.7 SI1 2827.0
0.72 VS2 2827.0
0.79 VVS2 2827.0
0.7 VVS1 2827.0
0.7 VVS1 2827.0
0.7 SI2 2828.0
1.01 SI2 2828.0
0.72 VS1 2829.0
0.8 SI2 2829.0
0.59 VVS1 2829.0
0.72 VS1 2829.0
0.75 SI2 2829.0
0.8 SI2 2829.0
0.71 VS2 2830.0
0.77 SI1 2830.0
0.97 I1 2830.0
0.53 VVS1 2830.0
0.53 VVS1 2830.0
0.8 VS2 2830.0
0.9 SI1 2830.0
0.76 SI2 2831.0
0.72 SI1 2831.0
0.75 SI1 2831.0
0.72 SI1 2831.0
0.79 SI1 2831.0
0.72 VS2 2832.0
0.91 SI2 2832.0
0.71 VVS2 2832.0
0.81 SI1 2832.0
0.82 SI1 2832.0
0.71 VS1 2832.0
0.9 SI1 2832.0
0.8 VS2 2833.0
0.56 IF 2833.0
0.7 VS2 2833.0
0.7 VS2 2833.0
0.61 VVS2 2833.0
0.85 SI2 2833.0
0.7 SI1 2833.0
0.8 VS2 2834.0
0.8 VS2 2834.0
0.51 VVS1 2834.0
0.53 VVS1 2834.0
0.78 VS2 2834.0
0.9 SI1 2834.0
0.9 SI2 2834.0
0.77 SI2 2834.0
0.73 VS1 2835.0
0.63 VVS2 2835.0
0.7 VS2 2835.0
0.72 VS2 2835.0
0.72 SI1 2835.0
0.75 VS2 2835.0
0.82 SI1 2836.0
0.71 VS2 2836.0
0.7 VS1 2837.0
0.7 VS1 2837.0
0.71 SI1 2838.0
0.76 SI1 2838.0
0.82 SI1 2838.0
0.72 VS1 2838.0
0.7 VS2 2838.0
0.7 VS2 2838.0
0.7 VS1 2838.0
0.7 VS2 2838.0
0.7 SI1 2838.0
0.7 VS2 2838.0
0.7 VS2 2838.0
0.7 VS1 2838.0
0.74 SI1 2839.0
0.71 VS1 2839.0
0.7 VS1 2839.0
0.71 VS1 2839.0
0.71 VS1 2839.0
0.71 VS1 2839.0
0.71 VS1 2839.0
0.7 VS1 2839.0
0.73 VS2 2839.0
0.7 VS2 2839.0
0.7 VS1 2839.0
0.71 VVS2 2839.0
0.71 VS1 2839.0
0.79 VS2 2839.0
0.71 VS1 2839.0
0.77 VS1 2840.0
0.75 SI2 2840.0
0.7 SI1 2840.0
0.71 VS2 2840.0
0.92 SI2 2840.0
0.83 SI2 2840.0
0.7 VVS1 2840.0
0.73 VS2 2841.0
0.71 VS1 2841.0
0.73 SI1 2841.0
0.82 SI2 2841.0
0.82 SI2 2841.0
0.82 SI2 2841.0
0.52 VVS1 2841.0
1.0 I1 2841.0
0.95 SI1 2841.0
0.73 SI1 2841.0
0.73 VS2 2841.0
0.73 VS1 2841.0
0.8 VS1 2842.0
0.7 VS2 2842.0
0.7 VS2 2843.0
0.7 VS2 2843.0
0.71 VS2 2843.0
0.81 SI2 2843.0
0.71 SI1 2843.0
0.73 VVS2 2843.0
0.73 VS1 2843.0
0.72 VS2 2843.0
0.81 SI2 2843.0
0.71 VVS2 2843.0
0.73 SI1 2844.0
0.7 VS1 2844.0
1.01 I1 2844.0
1.01 I1 2844.0
0.79 VS2 2844.0
0.7 VS2 2845.0
0.7 VS2 2845.0
0.8 VS2 2845.0
1.27 SI2 2845.0
0.79 SI1 2846.0
0.72 VS1 2846.0
0.73 VVS2 2846.0
1.01 SI2 2846.0
1.01 I1 2846.0
0.73 SI1 2846.0
0.7 SI1 2846.0
0.7 VS2 2846.0
0.77 SI1 2846.0
0.77 VS2 2846.0
0.77 VS1 2846.0
0.84 SI1 2847.0
0.72 SI1 2847.0
0.76 SI1 2847.0
0.7 VVS2 2848.0
0.54 VVS2 2848.0
0.75 SI2 2848.0
0.79 SI1 2849.0
0.74 VS1 2849.0
0.7 VS2 2850.0
0.7 VS2 2850.0
0.75 SI1 2850.0
1.2 I1 2850.0
0.8 SI1 2851.0
0.66 VS1 2851.0
0.87 SI2 2851.0
0.86 SI1 2851.0
0.74 SI1 2851.0
0.58 IF 2852.0
0.78 VS1 2852.0
0.74 SI1 2852.0
0.73 SI1 2852.0
0.91 SI1 2852.0
0.71 VS2 2853.0
0.71 VS1 2853.0
0.79 SI2 2853.0
0.79 SI2 2853.0
0.71 SI1 2853.0
0.82 VS1 2853.0
0.78 VS1 2854.0
0.7 VS1 2854.0
1.12 I1 2854.0
0.73 VS2 2854.0
0.91 VS2 2854.0
0.91 VS2 2854.0
0.91 VS2 2854.0
0.91 SI1 2854.0
0.7 VS1 2854.0
0.68 VVS2 2854.0
0.73 VS2 2855.0
1.03 SI1 2855.0
0.74 VS2 2855.0
0.98 SI2 2855.0
1.02 SI1 2856.0
1.0 SI2 2856.0
1.02 SI2 2856.0
0.6 VVS2 2856.0
0.8 SI2 2856.0
0.97 I1 2856.0
1.0 SI1 2856.0
0.26 VS1 556.0
0.26 VS1 556.0
0.36 SI1 556.0
0.34 VS2 556.0
0.34 SI1 556.0
0.34 SI1 556.0
0.34 SI1 556.0
0.34 VS2 556.0
0.34 SI1 556.0
0.32 VS2 556.0
0.31 VVS1 557.0
0.31 VVS1 557.0
0.31 VVS1 557.0
0.31 VVS1 557.0
0.31 VVS1 557.0
0.31 VVS1 557.0
0.31 VS2 557.0
0.31 VS1 557.0
0.31 VS1 557.0
0.33 SI2 557.0
0.33 SI2 557.0
0.33 SI2 557.0
0.33 SI2 557.0
0.33 SI2 557.0
0.33 VS1 557.0
0.33 VS1 557.0
0.33 VS1 557.0
0.33 SI1 557.0
0.33 SI1 557.0
0.33 SI1 557.0
1.0 SI2 2856.0
0.77 SI1 2856.0
0.77 SI1 2856.0
0.7 VVS2 2857.0
0.9 SI2 2857.0
0.72 SI1 2857.0
0.9 VS2 2857.0
0.72 SI1 2857.0
0.7 VVS2 2858.0
0.81 SI1 2858.0
0.81 SI1 2858.0
0.71 VS2 2858.0
0.7 VS2 2858.0
0.71 VS2 2858.0
0.71 VS2 2858.0
0.92 SI1 2858.0
0.76 SI1 2858.0
0.73 SI1 2858.0
0.71 VS2 2858.0
0.7 VVS2 2858.0
0.9 SI2 2858.0
0.71 VS2 2858.0
0.7 VS2 2859.0
0.7 VS2 2859.0
0.77 VS1 2859.0
0.71 VS1 2859.0
0.7 VS2 2859.0
0.75 VS1 2859.0
0.83 SI2 2859.0
0.71 VS2 2860.0
0.9 SI2 2860.0
0.6 VVS2 2860.0
0.71 VS1 2860.0
0.53 VVS1 2860.0
0.71 SI1 2861.0
0.62 VVS2 2861.0
0.62 VVS2 2861.0
0.9 SI1 2861.0
0.62 IF 2861.0
0.82 SI2 2861.0
0.66 VS1 2861.0
0.7 SI1 2862.0
0.8 SI1 2862.0
0.8 SI2 2862.0
0.79 SI1 2862.0
0.71 VVS1 2862.0
0.7 VS2 2862.0
0.7 VS2 2862.0
0.79 VS1 2862.0
0.7 VS2 2862.0
1.22 I1 2862.0
1.01 SI2 2862.0
0.73 VS2 2862.0
0.91 VS2 2863.0
0.71 SI1 2863.0
0.83 SI1 2863.0
0.84 SI2 2863.0
0.71 SI1 2863.0
0.71 SI1 2863.0
0.71 SI1 2863.0
0.71 SI1 2863.0
0.71 SI1 2863.0
0.71 SI1 2863.0
0.91 SI1 2863.0
0.9 VS2 2863.0
0.71 VVS2 2863.0
0.71 SI1 2863.0
0.72 VS2 2863.0
0.72 SI1 2863.0
0.71 VS2 2863.0
0.81 SI2 2864.0
0.83 VS2 2865.0
0.73 SI1 2865.0
0.56 VVS1 2866.0
0.56 VVS1 2866.0
0.71 VS1 2866.0
0.7 VVS1 2866.0
0.96 SI1 2866.0
0.71 VVS1 2867.0
0.7 VS2 2867.0
0.71 VVS1 2867.0
0.8 VS2 2867.0
0.95 SI2 2867.0
0.82 SI2 2867.0
0.52 VVS1 2867.0
0.82 SI2 2867.0
0.82 SI2 2867.0
0.82 SI2 2867.0
0.8 SI1 2867.0
0.96 SI2 2867.0
0.72 VS1 2868.0
0.62 IF 2868.0
0.79 SI2 2868.0
0.75 SI1 2868.0
1.08 I1 2869.0
0.72 SI1 2869.0
0.62 IF 2869.0
0.73 VVS2 2869.0
0.72 VVS2 2869.0
0.52 VVS2 2870.0
0.83 SI2 2870.0
0.64 VVS2 2870.0
0.8 SI1 2870.0
0.74 SI1 2870.0
0.72 SI1 2870.0
0.82 VS2 2870.0
0.73 VS1 2870.0
1.04 I1 2870.0
0.73 SI1 2871.0
0.73 SI1 2871.0
0.9 SI1 2871.0
0.75 SI1 2871.0
0.79 SI1 2871.0
0.7 SI1 2872.0
0.75 SI1 2872.0
1.02 I1 2872.0
0.7 SI2 2872.0
0.7 SI1 2872.0
0.7 SI1 2872.0
0.7 SI1 2872.0
0.7 SI1 2872.0
0.7 SI1 2872.0
0.72 VS2 2872.0
0.74 SI1 2872.0
0.84 SI1 2872.0
0.76 VS2 2873.0
0.77 SI1 2873.0
0.76 SI2 2873.0
1.0 SI2 2873.0
1.0 SI1 2873.0
0.9 SI1 2873.0
0.9 SI1 2873.0
0.9 SI1 2873.0
0.9 SI1 2873.0
0.9 SI1 2873.0
0.9 SI1 2873.0
0.78 VS2 2874.0
0.71 VS2 2874.0
0.7 VS1 2874.0
0.7 VS1 2874.0
0.7 VVS2 2874.0
0.7 VS1 2874.0
0.7 VS1 2874.0
0.7 VS1 2874.0
0.7 VVS2 2874.0
1.0 SI2 2875.0
0.77 SI1 2875.0
1.0 VS1 2875.0
1.0 SI1 2875.0
1.0 SI2 2875.0
0.73 VS1 2876.0
0.79 VS2 2876.0
0.72 VS1 2877.0
0.71 VS1 2877.0
0.74 VS2 2877.0
0.7 VVS1 2877.0
0.7 VS1 2877.0
0.79 SI1 2878.0
0.79 SI1 2878.0
0.79 SI2 2878.0
0.71 VS2 2878.0
0.79 SI1 2878.0
0.73 SI1 2879.0
0.63 IF 2879.0
0.7 VS1 2879.0
0.71 VS1 2879.0
0.84 SI2 2879.0
0.84 SI2 2879.0
1.02 SI2 2879.0
0.72 VS1 2879.0
0.72 VS1 2879.0
0.92 SI2 2880.0
0.74 SI1 2880.0
0.7 VVS1 2881.0
0.71 VS2 2881.0
1.05 I1 2881.0
0.7 IF 2882.0
0.54 VVS1 2882.0
0.73 VS2 2882.0
0.88 SI1 2882.0
0.73 VS2 2882.0
0.72 SI1 2883.0
0.9 SI2 2883.0
0.9 SI2 2883.0
1.03 SI2 2884.0
0.84 SI1 2885.0
1.01 SI1 2885.0
0.77 SI2 2885.0
0.8 SI1 2885.0
0.9 SI2 2885.0
0.73 SI1 2886.0
0.72 SI1 2886.0
0.71 SI1 2887.0
0.7 VS1 2887.0
0.79 VS1 2888.0
0.72 VVS2 2889.0
0.7 VS2 2889.0
0.7 VS1 2889.0
0.9 SI2 2889.0
0.71 VS1 2889.0
0.5 VVS2 2889.0
0.5 VVS2 2889.0
0.74 SI1 2889.0
0.77 VS2 2889.0
0.77 SI1 2889.0
0.8 SI1 2890.0
0.8 SI1 2890.0
0.8 SI1 2890.0
0.8 SI1 2890.0
0.66 VVS1 2890.0
0.71 VS2 2891.0
0.71 VS2 2891.0
0.71 VS2 2891.0
0.71 VS2 2891.0
0.72 SI1 2891.0
0.71 VS2 2891.0
0.86 SI2 2892.0
1.19 I1 2892.0
0.71 VS1 2893.0
0.82 SI2 2893.0
0.71 VVS2 2893.0
0.75 VS2 2893.0
0.7 VVS1 2893.0
0.8 SI2 2893.0
0.82 SI2 2893.0
0.82 SI1 2893.0
0.82 SI1 2893.0
0.81 SI2 2894.0
0.81 SI2 2894.0
0.76 SI1 2894.0
0.71 VS2 2895.0
0.7 VS1 2895.0
0.7 VVS2 2895.0
0.74 VS1 2896.0
0.77 VS2 2896.0
0.77 VS2 2896.0
0.53 VVS1 2896.0
0.79 SI1 2896.0
0.73 SI2 2896.0
0.77 SI2 2896.0
0.77 SI1 2896.0
1.01 I1 2896.0
1.01 I1 2896.0
0.6 VVS2 2897.0
0.76 SI1 2897.0
0.54 VVS2 2897.0
0.72 SI1 2897.0
0.72 VS1 2897.0
0.74 VS2 2897.0
1.12 SI2 2898.0
//sorting
display(spark.sql("SELECT carat, clarity, price FROM diamonds ORDER BY price DESC"))
carat clarity price
2.29 VS2 18823.0
2.0 SI1 18818.0
1.51 IF 18806.0
2.07 SI2 18804.0
2.0 SI1 18803.0
2.29 SI1 18797.0
2.0 VS1 18795.0
2.04 SI1 18795.0
1.71 VS2 18791.0
2.15 SI2 18791.0
2.8 SI2 18788.0
2.05 SI1 18787.0
2.05 SI2 18784.0
2.03 SI1 18781.0
1.6 VS1 18780.0
2.06 VS2 18779.0
1.51 VVS1 18777.0
1.71 VVS2 18768.0
2.55 VS1 18766.0
2.08 SI1 18760.0
2.0 SI1 18759.0
2.03 SI1 18757.0
2.61 SI2 18756.0
2.36 SI2 18745.0
2.01 SI1 18741.0
2.01 SI1 18741.0
2.01 SI1 18741.0
2.01 SI1 18736.0
1.94 SI1 18735.0
2.02 SI1 18731.0
1.72 VVS2 18730.0
1.51 VS1 18729.0
1.7 VVS2 18718.0
2.18 SI1 18717.0
3.01 SI2 18710.0
3.01 SI2 18710.0
2.0 SI1 18709.0
2.07 VS2 18707.0
2.22 VS1 18706.0
2.01 SI2 18705.0
3.51 VS2 18701.0
1.28 IF 18700.0
2.02 VS2 18700.0
2.19 SI2 18693.0
2.43 VS2 18692.0
2.48 SI2 18692.0
1.5 VS2 18691.0
2.67 SI2 18686.0
1.42 VVS1 18682.0
2.03 VS2 18680.0
2.02 SI2 18678.0
2.16 SI2 18678.0
2.01 SI2 18674.0
2.04 SI1 18663.0
2.05 VS2 18659.0
2.12 SI1 18656.0
2.29 VS2 18653.0
2.1 SI1 18648.0
2.01 VS2 18640.0
2.09 SI2 18640.0
2.03 SI1 18630.0
2.01 SI1 18625.0
2.42 VS2 18615.0
1.49 VVS2 18614.0
2.07 SI2 18611.0
2.01 VS2 18607.0
2.0 SI1 18604.0
1.71 VVS2 18599.0
1.7 VS1 18598.0
2.29 IF 18594.0
3.01 SI2 18593.0
2.03 SI2 18578.0
2.11 SI2 18575.0
2.01 SI1 18574.0
2.01 SI1 18572.0
1.6 VS1 18571.0
2.02 VS2 18565.0
2.01 VS2 18561.0
2.01 VS2 18561.0
2.09 SI1 18559.0
3.04 SI2 18559.0
2.38 VS2 18559.0
1.72 VS2 18557.0
1.5 IF 18552.0
1.04 IF 18542.0
2.4 SI1 18541.0
2.4 SI2 18541.0
2.03 SI2 18535.0
2.32 SI2 18532.0
2.22 VS2 18531.0
4.5 I1 18531.0
2.14 SI1 18528.0
2.14 SI2 18526.0
1.83 VS2 18525.0
2.0 SI1 18524.0
2.38 VS1 18522.0
2.0 VS2 18515.0
2.09 SI2 18509.0
2.32 SI2 18508.0
2.37 SI1 18508.0
2.01 VS2 18507.0
2.03 SI1 18507.0
2.01 VS1 18500.0
2.66 SI2 18495.0
2.0 SI1 18493.0
2.07 SI1 18489.0
2.02 SI2 18487.0
2.57 SI1 18485.0
2.21 SI2 18483.0
2.16 VS2 18481.0
2.1 SI1 18480.0
2.03 SI2 18477.0
2.19 SI1 18475.0
2.01 VS2 18474.0
2.09 SI2 18472.0
2.15 SI1 18470.0
2.04 SI1 18468.0
2.1 SI2 18462.0
2.16 VS1 18462.0
2.03 SI2 18458.0
2.5 SI2 18447.0
2.08 VS2 18447.0
1.7 VVS1 18445.0
2.09 VS2 18443.0
2.13 SI1 18442.0
2.0 SI1 18440.0
2.0 SI1 18440.0
2.06 SI1 18439.0
1.33 IF 18435.0
2.22 SI1 18432.0
1.72 VVS2 18431.0
2.44 VS2 18430.0
1.74 VS2 18430.0
1.7 VS1 18430.0
1.79 VS2 18429.0
2.26 SI1 18426.0
2.29 IF 18426.0
2.0 SI2 18426.0
2.03 SI1 18423.0
1.6 VS2 18421.0
1.79 VS1 18419.0
1.54 VS1 18416.0
2.11 SI2 18407.0
2.08 SI2 18405.0
2.01 SI1 18398.0
2.02 VS2 18398.0
1.7 VS1 18398.0
2.01 SI2 18395.0
2.01 SI2 18394.0
2.09 SI1 18392.0
1.73 VVS2 18377.0
2.0 SI1 18376.0
2.4 SI2 18374.0
2.01 SI1 18374.0
2.32 SI1 18371.0
2.0 SI1 18371.0
2.0 SI1 18371.0
2.06 SI2 18371.0
2.6 SI2 18369.0
2.2 VS2 18364.0
2.22 VS2 18363.0
2.07 VS2 18359.0
1.83 VS2 18358.0
2.07 SI2 18344.0
2.27 SI1 18343.0
1.7 VS2 18342.0
2.16 VS1 18342.0
2.01 VS1 18340.0
2.5 VS2 18325.0
2.49 SI2 18325.0
2.02 VS1 18324.0
2.02 SI2 18320.0
2.05 VS2 18318.0
1.61 VS2 18318.0
2.1 SI2 18312.0
2.03 SI2 18310.0
2.51 SI2 18308.0
1.93 SI1 18306.0
2.3 SI2 18304.0
2.24 SI1 18299.0
2.02 SI2 18296.0
2.01 SI1 18295.0
2.01 SI1 18295.0
1.54 VVS1 18294.0
2.06 SI2 18293.0
2.14 SI2 18291.0
2.03 SI2 18286.0
2.08 VS2 18281.0
1.62 VS2 18281.0
1.07 IF 18279.0
1.7 VVS1 18279.0
2.21 SI2 18276.0
2.13 SI1 18275.0
2.02 SI2 18274.0
2.01 SI2 18259.0
2.03 SI1 18257.0
2.51 SI2 18255.0
2.53 SI1 18254.0
2.52 SI1 18252.0
2.01 SI1 18252.0
1.7 VS2 18251.0
3.01 SI2 18242.0
3.01 SI2 18242.0
3.01 SI2 18242.0
3.01 SI2 18242.0
3.01 SI2 18242.0
2.3 VS2 18239.0
2.02 SI1 18236.0
2.02 SI1 18236.0
2.02 SI1 18236.0
2.02 SI2 18236.0
2.19 SI2 18232.0
2.04 SI2 18231.0
1.09 IF 18231.0
2.09 VS1 18215.0
1.73 VVS2 18211.0
2.02 VS2 18207.0
2.02 VS2 18207.0
2.01 VS1 18206.0
2.07 SI2 18198.0
2.2 SI1 18193.0
2.05 SI1 18193.0
2.07 VS2 18193.0
2.3 VS2 18190.0
2.01 SI1 18188.0
1.55 VVS2 18188.0
2.01 SI1 18186.0
2.0 SI1 18186.0
2.01 SI2 18183.0
2.05 SI2 18181.0
2.52 SI2 18179.0
1.63 VS1 18179.0
1.76 VS1 18178.0
1.7 VS2 18176.0
2.0 SI2 18172.0
2.01 VS2 18172.0
2.2 SI2 18168.0
2.03 VS2 18166.0
2.12 SI2 18164.0
1.5 VVS2 18159.0
2.04 SI1 18153.0
2.05 VS2 18152.0
2.23 VS2 18149.0
2.01 SI1 18149.0
2.03 VS2 18139.0
2.03 VS2 18139.0
2.08 SI2 18128.0
2.08 SI2 18128.0
1.78 VS2 18128.0
2.21 SI2 18128.0
2.04 VS1 18127.0
2.14 SI2 18125.0
2.08 SI2 18124.0
2.1 SI2 18124.0
2.03 SI2 18120.0
2.12 SI2 18120.0
2.33 SI1 18119.0
2.12 SI1 18118.0
2.02 SI2 18117.0
2.04 SI1 18115.0
2.04 SI1 18115.0
2.03 SI2 18115.0
1.07 IF 18114.0
2.45 VS2 18113.0
1.14 IF 18112.0
2.3 SI2 18108.0
1.7 VS2 18107.0
1.7 VS2 18107.0
2.04 VS1 18104.0
1.51 IF 18102.0
2.51 SI2 18090.0
2.32 SI1 18080.0
2.02 VS2 18077.0
2.01 SI1 18077.0
2.01 SI1 18077.0
2.01 SI1 18077.0
2.11 SI2 18071.0
2.0 SI1 18069.0
2.29 SI1 18068.0
2.19 SI2 18067.0
2.04 SI2 18066.0
2.21 SI1 18062.0
2.0 SI1 18062.0
2.2 SI1 18059.0
1.58 VS1 18057.0
1.7 VS2 18055.0
2.28 SI2 18055.0
2.02 VS2 18050.0
2.01 VS1 18041.0
2.51 SI2 18037.0
2.11 SI2 18034.0
2.25 SI1 18034.0
2.25 SI2 18034.0
2.16 SI1 18029.0
2.51 SI2 18029.0
2.26 SI2 18028.0
2.01 SI1 18027.0
2.01 SI1 18027.0
2.32 SI1 18026.0
2.03 SI1 18026.0
2.32 SI1 18026.0
2.04 SI1 18026.0
2.0 VS2 18023.0
2.51 VS2 18020.0
5.01 I1 18018.0
2.05 SI2 18017.0
1.76 VS1 18014.0
2.25 SI2 18007.0
2.06 SI2 18005.0
2.18 SI2 18003.0
2.09 SI2 18002.0
2.16 SI2 18001.0
2.35 SI1 17999.0
2.09 VS2 17999.0
2.54 SI2 17996.0
1.93 VS1 17995.0
2.24 SI2 17989.0
2.05 SI2 17988.0
2.01 VS2 17987.0
2.08 IF 17986.0
2.01 VS1 17983.0
2.03 VS2 17975.0
2.05 SI2 17957.0
2.4 SI2 17955.0
2.39 VS2 17955.0
2.0 SI2 17953.0
2.04 SI1 17952.0
2.04 SI1 17952.0
1.54 VS1 17949.0
2.02 SI2 17938.0
1.51 VS1 17936.0
2.02 VS1 17936.0
2.16 SI1 17934.0
1.29 VVS1 17932.0
2.0 SI1 17930.0
2.57 SI2 17924.0
2.41 SI2 17923.0
2.39 VS1 17920.0
2.01 VS2 17917.0
2.08 VS2 17916.0
1.07 IF 17909.0
2.2 VS2 17905.0
1.74 VS1 17904.0
1.74 VS1 17904.0
2.0 SI1 17902.0
2.0 VS2 17898.0
2.2 SI1 17895.0
1.58 VS1 17894.0
2.07 SI1 17893.0
2.48 SI1 17893.0
2.02 VS2 17893.0
1.7 VS2 17892.0
2.28 SI2 17892.0
2.01 SI1 17892.0
2.32 VS1 17891.0
2.16 VVS1 17891.0
2.0 VS1 17889.0
1.76 VVS2 17888.0
2.02 VS2 17887.0
2.02 SI1 17882.0
2.01 VS1 17877.0
2.11 SI2 17871.0
2.0 SI2 17871.0
2.0 SI1 17869.0
2.03 SI1 17864.0
2.43 SI2 17856.0
2.01 SI2 17849.0
2.01 SI1 17849.0
2.01 SI1 17849.0
2.01 SI1 17849.0
2.18 SI2 17841.0
2.09 SI2 17840.0
2.01 VS1 17838.0
2.1 VS2 17837.0
2.0 SI2 17835.0
2.36 VS1 17829.0
2.01 VS2 17826.0
1.63 VS2 17825.0
2.02 SI2 17825.0
2.16 SI1 17820.0
2.11 SI1 17816.0
2.05 SI1 17811.0
2.09 SI2 17805.0
2.17 SI2 17805.0
2.01 SI1 17804.0
2.03 SI1 17803.0
1.69 VS2 17803.0
2.72 SI2 17801.0
2.01 SI2 17798.0
2.21 SI1 17784.0
2.08 SI1 17778.0
2.05 SI1 17776.0
1.55 VS1 17773.0
2.16 SI1 17772.0
1.71 VS2 17766.0
1.72 VS1 17765.0
1.87 VS1 17761.0
2.0 SI2 17760.0
2.0 SI1 17760.0
2.0 SI1 17760.0
2.0 VS2 17760.0
2.0 VS2 17760.0
2.0 SI2 17760.0
2.01 SI2 17759.0
2.01 SI1 17759.0
2.56 SI1 17753.0
2.03 SI2 17752.0
2.01 VS1 17751.0
2.17 SI1 17747.0
2.01 SI2 17746.0
2.14 SI2 17742.0
2.0 SI1 17740.0
2.12 SI2 17730.0
1.65 IF 17729.0
2.0 SI1 17724.0
1.52 VS1 17723.0
2.0 VS2 17716.0
2.31 SI1 17715.0
2.21 SI1 17714.0
1.99 VS2 17713.0
2.11 VS2 17712.0
2.0 SI2 17710.0
2.12 SI2 17694.0
2.02 SI1 17692.0
2.52 SI2 17689.0
1.51 VVS2 17689.0
2.01 SI1 17688.0
2.01 SI2 17688.0
1.71 VS1 17685.0
2.32 SI2 17676.0
2.01 SI1 17676.0
2.0 SI2 17674.0
2.14 SI2 17673.0
2.28 SI1 17673.0
2.02 SI1 17672.0
1.5 VVS2 17667.0
2.29 SI2 17666.0
1.34 IF 17663.0
1.7 VS2 17662.0
1.52 VS1 17659.0
2.02 SI1 17658.0
2.06 SI2 17650.0
1.51 VS1 17649.0
2.39 VS1 17642.0
2.05 VS1 17640.0
2.2 SI2 17634.0
2.08 SI1 17617.0
1.74 VS2 17614.0
2.07 SI2 17614.0
2.01 SI2 17609.0
2.52 SI2 17608.0
2.0 VS1 17607.0
2.48 SI2 17607.0
2.1 VVS1 17606.0
1.72 VS1 17605.0
1.64 VVS2 17604.0
2.0 SI1 17600.0
1.38 IF 17598.0
1.7 VS2 17597.0
1.7 VS2 17597.0
1.71 VS1 17595.0
2.01 VS2 17592.0
2.53 SI2 17591.0
1.03 IF 17590.0
2.14 SI2 17582.0
2.02 VS2 17581.0
2.02 SI1 17579.0
2.0 VS2 17574.0
2.01 VS2 17570.0
2.36 SI2 17569.0
1.65 IF 17569.0
2.01 SI2 17555.0
2.19 SI2 17554.0
1.89 VS1 17553.0
1.59 VS1 17552.0
1.57 VS2 17548.0
1.45 VVS2 17545.0
2.29 VS2 17539.0
1.97 VS2 17535.0
2.36 VS2 17534.0
2.0 SI2 17534.0
2.02 VS1 17533.0
2.02 SI1 17530.0
2.21 SI2 17525.0
2.01 VS1 17523.0
2.52 SI2 17522.0
2.05 SI1 17521.0
2.32 SI2 17516.0
1.51 VS1 17515.0
2.01 SI2 17514.0
2.14 SI2 17513.0
1.91 SI1 17509.0
2.33 VS2 17504.0
1.14 IF 17499.0
2.01 VS2 17497.0
1.31 VVS1 17496.0
2.01 SI1 17492.0
1.7 VS1 17492.0
2.02 SI1 17489.0
1.7 VS2 17485.0
2.01 SI1 17476.0
2.18 SI2 17475.0
2.01 SI1 17474.0
2.18 SI2 17473.0
2.44 SI1 17472.0
2.08 SI1 17469.0
2.08 SI1 17469.0
2.2 VS2 17460.0
2.01 SI1 17458.0
1.56 VS1 17455.0
2.51 SI1 17452.0
2.31 VS1 17451.0
1.5 VVS2 17449.0
2.48 SI2 17448.0
2.0 SI1 17447.0
1.76 VS2 17442.0
1.55 VS2 17441.0
2.0 VS1 17436.0
2.01 SI1 17434.0
2.01 VS2 17433.0
2.19 SI2 17433.0
1.65 VS1 17425.0
2.01 VS2 17422.0
2.14 VS2 17416.0
1.61 VS1 17414.0
2.05 VS1 17408.0
2.64 SI2 17407.0
2.5 SI2 17405.0
2.0 SI1 17405.0
2.01 SI2 17403.0
2.01 SI1 17403.0
2.01 SI1 17403.0
2.01 SI1 17403.0
1.69 VS1 17400.0
1.59 VS1 17393.0
2.03 SI2 17393.0
2.02 SI1 17392.0
2.01 SI1 17383.0
2.17 SI1 17381.0
2.04 SI2 17379.0
1.97 VS2 17377.0
1.95 VS2 17374.0
1.59 VS1 17366.0
2.01 VS1 17365.0
2.39 VS1 17365.0
1.6 VVS1 17360.0
1.7 VS2 17360.0
1.79 VS2 17358.0
2.02 SI1 17357.0
1.21 IF 17353.0
1.93 VS2 17353.0
1.67 VS2 17351.0
2.01 SI1 17347.0
1.75 VS2 17343.0
2.54 SI2 17339.0
1.69 VS1 17338.0
2.01 SI1 17334.0
1.7 VS1 17330.0
4.13 I1 17329.0
1.58 IF 17329.0
2.04 IF 17327.0
2.04 SI1 17323.0
1.7 VS2 17323.0
1.51 VVS2 17317.0
2.04 VS2 17315.0
2.16 SI1 17313.0
2.01 SI2 17313.0
2.26 SI2 17312.0
2.03 VS2 17297.0
2.35 SI2 17294.0
2.35 SI2 17294.0
2.05 SI1 17294.0
2.05 SI2 17294.0
1.5 VVS1 17279.0
2.0 VS1 17278.0
1.8 SI1 17273.0
1.86 VVS2 17267.0
2.02 SI2 17265.0
2.02 SI1 17265.0
2.2 SI1 17265.0
2.16 SI2 17263.0
2.02 SI2 17263.0
2.42 VS1 17262.0
2.08 VS2 17258.0
1.61 VS1 17256.0
2.12 VS2 17254.0
2.1 SI1 17250.0
2.0 SI1 17247.0
2.02 SI1 17245.0
2.01 SI2 17244.0
1.54 VS2 17240.0
2.05 SI2 17237.0
2.01 SI1 17235.0
2.01 SI2 17235.0
2.25 SI1 17233.0
2.52 SI1 17231.0
1.7 VVS2 17228.0
2.01 VS1 17227.0
2.17 SI1 17224.0
1.53 VS2 17223.0
2.15 SI2 17221.0
2.01 SI2 17220.0
2.15 SI2 17219.0
1.7 VS1 17219.0
2.31 SI2 17218.0
1.41 VVS2 17216.0
2.05 VVS2 17214.0
2.14 SI1 17213.0
2.61 SI2 17209.0
2.6 SI2 17209.0
1.71 VS1 17206.0
1.71 VS1 17204.0
1.6 VVS1 17204.0
1.51 VVS2 17203.0
1.5 VVS2 17203.0
2.01 VS2 17197.0
1.71 VS1 17197.0
1.67 VS2 17194.0
2.3 SI1 17193.0
2.14 VS2 17193.0
1.21 VVS1 17192.0
2.01 VS2 17191.0
1.75 VS1 17191.0
2.56 SI1 17186.0
2.74 SI2 17184.0
2.03 SI2 17182.0
2.01 SI2 17179.0
2.01 SI2 17179.0
1.5 VVS2 17176.0
1.57 VS1 17175.0
1.75 VS2 17172.0
2.0 SI1 17172.0
2.42 VS2 17168.0
2.42 VS2 17168.0
2.02 SI2 17166.0
2.74 SI2 17164.0
2.51 SI2 17162.0
2.22 SI1 17160.0
2.09 SI2 17156.0
1.5 VS2 17153.0
2.02 VS1 17153.0
2.22 SI1 17151.0
2.27 VS1 17149.0
2.71 SI2 17146.0
1.5 VS1 17143.0
2.25 SI2 17143.0
2.02 SI1 17141.0
2.05 SI2 17138.0
2.0 SI2 17136.0
2.14 SI2 17127.0
2.01 SI1 17126.0
1.72 VS2 17125.0
2.22 SI2 17123.0
2.01 SI2 17118.0
2.57 SI2 17116.0
2.01 SI2 17115.0
2.09 SI1 17114.0
1.7 VS2 17114.0
1.51 VS1 17111.0
1.56 VS2 17108.0
2.53 SI2 17103.0
1.02 IF 17100.0
2.02 SI2 17099.0
1.93 VS1 17096.0
2.07 SI2 17095.0
2.01 SI1 17095.0
2.0 SI2 17094.0
2.0 VS2 17084.0
2.05 SI2 17081.0
2.01 SI2 17079.0
2.01 SI2 17078.0
1.61 VS1 17076.0
2.13 SI2 17073.0
1.7 VVS2 17073.0
2.01 VS1 17068.0
2.01 VS2 17068.0
2.01 VS1 17068.0
2.12 VS2 17068.0
2.01 VS1 17068.0
2.05 SI2 17066.0
2.15 SI2 17065.0
2.15 SI2 17063.0
2.31 SI2 17062.0
2.01 VS2 17057.0
1.53 VVS2 17057.0
2.32 VS2 17053.0
1.71 VS2 17052.0
2.27 VS1 17051.0
2.09 VS2 17051.0
1.6 VS1 17050.0
1.71 VS1 17049.0
2.04 SI2 17049.0
2.07 VS2 17045.0
2.13 SI2 17045.0
1.07 IF 17042.0
1.71 VS1 17041.0
2.4 SI2 17039.0
2.14 VS2 17038.0
1.75 VS1 17036.0
1.54 VS2 17029.0
2.04 VS2 17028.0
1.67 VVS2 17028.0
2.5 SI1 17028.0
2.6 SI2 17027.0
2.01 SI2 17024.0
2.01 SI2 17024.0
2.07 SI1 17019.0
1.75 VS2 17017.0
2.19 SI1 17016.0
2.01 SI2 17014.0
2.06 SI2 17012.0
2.26 VS2 17010.0
1.71 VS1 17009.0
2.05 SI1 17006.0
2.01 SI2 17005.0
2.01 SI1 17003.0
2.09 SI2 17001.0
2.31 SI2 17000.0
2.2 SI2 16996.0
2.27 VS1 16994.0
2.12 SI2 16992.0
1.5 VS1 16988.0
2.41 SI2 16987.0
1.68 VS1 16985.0
2.02 SI1 16985.0
1.73 VS2 16975.0
3.0 SI2 16970.0
3.0 SI2 16970.0
2.28 SI2 16969.0
1.73 VS2 16960.0
2.06 SI1 16960.0
1.73 VS2 16960.0
2.07 SI1 16957.0
2.01 SI1 16956.0
2.01 SI1 16956.0
1.7 VS1 16955.0
2.5 VS2 16955.0
2.28 SI2 16954.0
1.5 VVS2 16948.0
2.03 VS2 16945.0
2.02 SI2 16944.0
2.04 SI1 16942.0
1.93 VS1 16941.0
2.37 SI2 16937.0
2.53 SI2 16934.0
2.13 SI2 16931.0
2.02 SI2 16929.0
2.01 SI2 16922.0
1.54 VS2 16921.0
1.52 VS1 16916.0
2.49 SI1 16915.0
2.01 VS1 16914.0
2.63 SI2 16914.0
2.14 VS2 16914.0
2.01 VS1 16914.0
2.26 SI1 16904.0
2.01 SI1 16901.0
2.03 SI1 16900.0
2.03 SI1 16900.0
2.03 SI1 16900.0
2.05 VS2 16896.0
1.54 VS2 16889.0
2.01 SI2 16881.0
2.01 SI2 16881.0
1.76 VS2 16879.0
2.18 SI2 16878.0
2.04 VS2 16874.0
2.04 SI2 16872.0
2.0 SI1 16872.0
2.02 VS2 16861.0
2.06 SI2 16857.0
2.08 VS2 16854.0
2.18 SI2 16842.0
2.51 SI1 16842.0
2.42 SI2 16826.0
2.09 SI2 16824.0
1.25 IF 16823.0
2.48 VS2 16820.0
2.05 SI2 16819.0
1.71 VS2 16817.0
1.71 VS1 16813.0
2.01 VS1 16811.0
1.4 VVS1 16808.0
1.73 VS2 16807.0
2.23 SI2 16805.0
2.21 SI1 16804.0
1.75 VS2 16803.0
2.31 SI1 16801.0
2.04 SI2 16800.0
2.4 SI2 16797.0
1.5 VS1 16793.0
2.05 VS1 16793.0
2.03 VS2 16792.0
2.39 VS2 16791.0
1.62 VS1 16790.0
2.37 VS2 16789.0
2.02 VS2 16789.0
1.69 VS2 16789.0
2.03 SI1 16787.0
1.52 VVS2 16786.0
1.75 VS2 16783.0
2.1 SI2 16783.0
1.75 VS2 16783.0
1.5 VS1 16783.0
1.71 VS2 16779.0
1.52 VS1 16779.0
1.52 VS1 16779.0
2.01 VS2 16778.0
2.01 VS2 16778.0
2.01 VS2 16778.0
2.01 VS2 16778.0
2.01 VS2 16778.0
2.01 VS2 16778.0
2.13 SI2 16778.0
2.01 SI2 16776.0
2.01 SI2 16776.0
2.07 VVS2 16769.0
2.04 VS2 16768.0
1.51 VVS2 16754.0
1.54 VS1 16750.0
2.03 SI1 16747.0
2.01 SI1 16742.0
2.01 SI1 16737.0
1.54 VS2 16736.0
2.01 SI1 16733.0
2.01 SI1 16733.0
2.01 SI2 16733.0
2.01 SI2 16733.0
2.01 SI1 16733.0
2.01 SI1 16733.0
2.01 SI1 16731.0
2.01 SI2 16728.0
2.11 VS1 16723.0
2.14 SI2 16723.0
2.04 SI1 16718.0
2.04 SI1 16718.0
2.51 SI2 16717.0
1.51 VVS2 16716.0
1.51 VVS2 16716.0
2.17 SI1 16716.0
2.48 SI1 16715.0
2.53 SI1 16709.0
2.02 VS2 16709.0
2.24 SI2 16709.0
2.36 SI2 16707.0
2.09 SI1 16704.0
2.02 SI1 16704.0
2.09 SI2 16703.0
2.01 VS2 16700.0
2.0 VS2 16694.0
2.0 VS2 16694.0
2.03 SI1 16693.0
2.18 SI2 16690.0
2.22 VS1 16689.0
1.51 VS1 16688.0
2.39 SI1 16687.0
2.4 SI1 16687.0
1.8 VS1 16683.0
2.01 VS2 16677.0
1.52 VS1 16670.0
1.51 VS1 16669.0
2.02 SI1 16665.0
2.21 SI2 16657.0
2.24 SI2 16656.0
2.0 SI2 16650.0
2.0 SI1 16650.0
2.0 SI1 16650.0
2.38 SI1 16643.0
1.67 VS1 16643.0
2.03 SI2 16642.0
2.05 VS1 16641.0
1.52 VS1 16636.0
1.75 VS2 16632.0
2.03 SI1 16629.0
2.1 SI2 16629.0
2.1 SI2 16629.0
1.52 VS1 16628.0
1.52 VS1 16628.0
2.01 VS2 16626.0
2.01 VS2 16626.0
2.06 SI1 16626.0
2.01 VS2 16626.0
2.01 SI2 16624.0
1.71 VS1 16618.0
2.07 VVS2 16617.0
2.04 VS2 16616.0
1.51 VVS2 16613.0
2.05 SI1 16611.0
2.06 SI2 16603.0
1.53 VVS1 16601.0
1.59 VVS1 16599.0
2.02 VS2 16593.0
2.28 VS1 16592.0
2.45 SI2 16589.0
2.06 SI2 16587.0
1.69 VS2 16583.0
2.01 SI2 16582.0
2.01 SI1 16582.0
2.0 SI1 16580.0
1.71 SI1 16575.0
2.25 SI2 16575.0
1.57 VS2 16570.0
2.02 SI2 16565.0
2.28 VS2 16564.0
2.01 SI1 16562.0
2.02 SI1 16560.0
2.21 SI2 16558.0
1.5 VS1 16558.0
2.24 SI2 16558.0
1.51 VS1 16551.0
1.6 IF 16547.0
2.22 SI2 16547.0
1.41 VVS1 16545.0
2.0 VS2 16544.0
2.0 VS2 16544.0
3.01 I1 16538.0
2.44 VS2 16533.0
2.47 SI2 16532.0
2.2 SI2 16530.0
1.7 VS1 16521.0
1.51 VS1 16520.0
1.52 VS1 16519.0
1.51 VS1 16518.0
1.8 SI1 16513.0
2.11 VS2 16512.0
2.06 SI2 16512.0
1.62 VS2 16507.0
2.05 SI2 16506.0
2.1 SI2 16506.0
2.03 SI2 16505.0
2.01 SI1 16499.0
2.01 SI2 16499.0
1.52 VVS1 16492.0
1.52 VS1 16485.0
2.0 SI2 16484.0
2.03 VS2 16483.0
2.1 SI2 16479.0
2.1 SI2 16479.0
2.17 SI2 16472.0
1.0 IF 16469.0
2.46 SI2 16466.0
2.12 SI2 16466.0
2.59 VS1 16465.0
2.13 SI2 16462.0
2.0 SI2 16462.0
2.0 VS2 16459.0
1.53 VVS1 16451.0
2.28 VS2 16450.0
2.05 SI2 16446.0
2.03 SI2 16442.0
2.0 VS2 16439.0
2.06 SI2 16437.0
2.05 SI2 16431.0
2.51 SI2 16427.0
2.18 VS2 16427.0
2.18 VS2 16427.0
2.04 SI2 16426.0
2.0 SI2 16425.0
2.03 VS2 16422.0
2.04 SI1 16420.0
2.03 SI2 16412.0
2.01 SI2 16410.0
1.5 VS1 16409.0
2.0 SI1 16407.0
1.5 VS1 16407.0
1.09 IF 16406.0
2.11 SI2 16404.0
1.51 VS1 16402.0
2.16 SI2 16400.0
2.22 SI2 16398.0
2.19 SI1 16397.0
2.02 VS1 16397.0
2.11 VS2 16395.0
2.03 SI1 16392.0
2.07 SI1 16392.0
2.14 VS2 16390.0
2.04 SI2 16389.0
2.02 SI1 16386.0
1.71 VS2 16384.0
2.01 VS2 16383.0
2.0 SI2 16380.0
2.07 VS2 16378.0
1.51 VS1 16370.0
2.28 VS2 16369.0
2.02 SI1 16368.0
1.5 VVS2 16364.0
1.8 SI1 16364.0
2.11 VS2 16363.0
1.62 VS2 16358.0
2.1 SI2 16357.0
2.05 SI2 16357.0
2.54 SI2 16353.0
2.54 SI2 16353.0
1.52 VVS1 16343.0
2.35 SI2 16340.0
1.8 SI1 16340.0
1.6 VVS1 16339.0
2.07 SI2 16337.0
2.11 SI2 16336.0
2.3 SI2 16329.0
diamondsDF.printSchema // since price is double in the DF that was turned into table we can rely on the descenting sort on doubles
root
 |-- carat: double (nullable = true)
 |-- cut: string (nullable = true)
 |-- color: string (nullable = true)
 |-- clarity: string (nullable = true)
 |-- depth: double (nullable = true)
 |-- table: double (nullable = true)
 |-- price: double (nullable = true)
 |-- x: double (nullable = true)
 |-- y: double (nullable = true)
 |-- z: double (nullable = true)
// sort by multiple fields
display(spark.sql("SELECT carat, clarity, price FROM diamonds ORDER BY carat ASC, price DESC"))
carat clarity price
0.2 VS2 367.0
0.2 VS2 367.0
0.2 VS2 367.0
0.2 VS2 367.0
0.2 VS2 367.0
0.2 VS2 367.0
0.2 VS2 367.0
0.2 VS2 367.0
0.2 VS2 367.0
0.2 VS2 367.0
0.2 VS2 367.0
0.2 SI2 345.0
0.21 SI2 394.0
0.21 VS2 386.0
0.21 VS2 386.0
0.21 VS2 386.0
0.21 VS2 386.0
0.21 VS2 386.0
0.21 VS2 386.0
0.21 VS2 386.0
0.21 SI1 326.0
0.22 SI1 470.0
0.22 VS2 404.0
0.22 VS2 404.0
0.22 SI1 342.0
0.22 VS2 337.0
0.23 VVS2 688.0
0.23 VVS1 682.0
0.23 VVS1 680.0
0.23 VVS1 680.0
0.23 VVS1 680.0
0.23 VVS2 680.0
0.23 VVS2 680.0
0.23 VVS2 680.0
0.23 VVS2 680.0
0.23 VVS2 650.0
0.23 VVS2 640.0
0.23 VVS1 640.0
0.23 VS1 611.0
0.23 VVS2 600.0
0.23 VS1 586.0
0.23 VS1 586.0
0.23 VVS2 583.0
0.23 VVS2 583.0
0.23 VVS1 583.0
0.23 VVS1 583.0
0.23 VVS1 583.0
0.23 VVS1 583.0
0.23 VVS1 583.0
0.23 VVS2 583.0
0.23 VS2 577.0
0.23 VVS2 571.0
0.23 VVS2 550.0
0.23 VVS2 549.0
0.23 VS2 548.0
0.23 VS1 548.0
0.23 VS1 548.0
0.23 VS2 548.0
0.23 VS2 543.0
0.23 VVS2 538.0
0.23 VVS2 537.0
0.23 IF 536.0
0.23 VVS1 536.0
0.23 IF 536.0
0.23 VVS1 536.0
0.23 IF 536.0
0.23 VVS1 536.0
0.23 VVS1 531.0
0.23 VVS1 530.0
0.23 VVS2 530.0
0.23 VVS2 530.0
0.23 VVS2 530.0
0.23 VVS2 530.0
0.23 VVS2 530.0
0.23 VVS2 530.0
0.23 VVS2 530.0
0.23 VVS2 530.0
0.23 VVS2 530.0
0.23 VVS2 530.0
0.23 VVS2 530.0
0.23 VVS2 530.0
0.23 VVS2 530.0
0.23 VVS2 530.0
0.23 VVS2 530.0
0.23 VVS2 530.0
0.23 VVS2 530.0
0.23 VVS2 530.0
0.23 VVS2 530.0
0.23 VVS2 530.0
0.23 VVS2 530.0
0.23 VVS2 530.0
0.23 VVS2 530.0
0.23 VVS2 530.0
0.23 VVS2 530.0
0.23 VVS1 530.0
0.23 VVS1 530.0
0.23 VVS1 530.0
0.23 VVS1 530.0
0.23 VVS1 530.0
0.23 VVS1 530.0
0.23 VVS1 530.0
0.23 VVS1 530.0
0.23 VVS1 530.0
0.23 VVS1 530.0
0.23 VVS1 530.0
0.23 VVS1 530.0
0.23 VVS1 530.0
0.23 IF 530.0
0.23 VVS2 530.0
0.23 VVS2 530.0
0.23 VVS2 530.0
0.23 VVS2 530.0
0.23 VVS2 530.0
0.23 VVS2 530.0
0.23 VVS2 530.0
0.23 VVS2 530.0
0.23 VVS2 530.0
0.23 VVS1 530.0
0.23 VVS2 530.0
0.23 VVS1 530.0
0.23 VVS1 530.0
0.23 VVS1 530.0
0.23 VVS2 530.0
0.23 VVS2 530.0
0.23 VVS2 530.0
0.23 VVS1 525.0
0.23 VVS1 518.0
0.23 VS2 513.0
0.23 VS1 513.0
0.23 VS2 512.0
0.23 VVS2 511.0
0.23 VVS2 505.0
0.23 VVS2 505.0
0.23 VVS2 505.0
0.23 VVS2 505.0
0.23 VVS2 505.0
0.23 VVS2 505.0
0.23 VVS2 505.0
0.23 VVS2 505.0
0.23 VVS2 505.0
0.23 VVS2 505.0
0.23 VVS2 505.0
0.23 VVS2 505.0
0.23 VVS2 505.0
0.23 VVS2 505.0
0.23 VVS2 505.0
0.23 VVS2 505.0
0.23 VVS2 505.0
0.23 VVS2 505.0
0.23 VVS2 505.0
0.23 VVS2 505.0
0.23 VVS2 505.0
0.23 VVS2 505.0
0.23 VVS2 505.0
0.23 VVS2 505.0
0.23 VVS2 505.0
0.23 VVS2 505.0
0.23 VVS2 505.0
0.23 VVS2 505.0
0.23 VVS2 505.0
0.23 VVS2 505.0
0.23 VVS1 505.0
0.23 VVS1 505.0
0.23 VVS2 500.0
0.23 VVS1 499.0
0.23 VVS2 499.0
0.23 VVS2 498.0
0.23 VVS2 498.0
0.23 VS2 498.0
0.23 VVS1 498.0
0.23 VS2 498.0
0.23 VS2 498.0
0.23 VVS2 498.0
0.23 VS1 493.0
0.23 VS2 493.0
0.23 VVS2 492.0
0.23 VVS1 492.0
0.23 VVS1 492.0
0.23 IF 492.0
0.23 VVS2 485.0
0.23 VVS2 485.0
0.23 VVS2 485.0
0.23 VVS2 485.0
0.23 VVS2 485.0
0.23 VVS2 485.0
0.23 VVS2 485.0
0.23 VVS1 485.0
0.23 VVS1 485.0
0.23 VVS1 485.0
0.23 VVS1 485.0
0.23 VVS2 485.0
0.23 VVS2 485.0
0.23 VVS1 485.0
0.23 IF 485.0
0.23 VVS1 484.0
0.23 VVS1 484.0
0.23 VVS1 484.0
0.23 VS1 483.0
0.23 VVS1 478.0
0.23 VVS1 478.0
0.23 VVS1 478.0
0.23 VVS1 478.0
0.23 VVS1 478.0
0.23 VVS1 478.0
0.23 VVS2 478.0
0.23 VVS1 478.0
0.23 VVS1 472.0
0.23 VVS1 472.0
0.23 VVS1 472.0
0.23 VVS1 472.0
0.23 VVS1 472.0
0.23 VVS2 472.0
0.23 VVS1 472.0
0.23 VVS1 472.0
0.23 VVS2 472.0
0.23 VVS1 472.0
0.23 VVS2 472.0
0.23 VVS2 472.0
0.23 VS1 468.0
0.23 VVS2 468.0
0.23 VVS2 465.0
0.23 VVS2 465.0
0.23 VVS2 465.0
0.23 VVS2 465.0
0.23 VVS1 465.0
0.23 VVS1 465.0
0.23 VVS2 465.0
0.23 VVS2 465.0
0.23 VVS2 465.0
0.23 VVS2 465.0
0.23 VVS2 465.0
0.23 VVS2 462.0
0.23 VVS1 462.0
0.23 VVS1 458.0
0.23 VVS1 458.0
0.23 VVS1 458.0
0.23 VVS2 458.0
0.23 VVS2 458.0
0.23 VVS2 452.0
0.23 SI2 449.0
0.23 VS2 447.0
0.23 VVS2 445.0
0.23 VS1 442.0
0.23 VS2 442.0
0.23 VVS1 439.0
0.23 VVS2 438.0
0.23 VS1 434.0
0.23 VVS1 434.0
0.23 VVS2 431.0
0.23 VVS2 431.0
0.23 VVS2 431.0
0.23 VVS2 431.0
0.23 VVS2 431.0
0.23 VVS2 431.0
0.23 VVS2 428.0
0.23 VVS2 425.0
0.23 VVS2 425.0
0.23 VVS1 425.0
0.23 VS1 423.0
0.23 VS2 423.0
0.23 VVS1 415.0
0.23 VVS1 414.0
0.23 VVS1 414.0
0.23 VVS1 414.0
0.23 VS2 411.0
0.23 VS1 404.0
0.23 VS1 402.0
0.23 VS1 402.0
0.23 VS2 402.0
0.23 VS1 402.0
0.23 VS2 402.0
0.23 VS2 402.0
0.23 VS1 402.0
0.23 VS1 402.0
0.23 VS1 402.0
0.23 VS2 402.0
0.23 VS2 402.0
0.23 VS2 402.0
0.23 VS2 402.0
0.23 VS1 402.0
0.23 VS2 402.0
0.23 VS2 402.0
0.23 VS1 402.0
0.23 VS2 402.0
0.23 VS2 402.0
0.23 VS1 400.0
0.23 VS2 400.0
0.23 VVS1 395.0
0.23 VS1 391.0
0.23 VS1 391.0
0.23 VVS2 389.0
0.23 VS1 384.0
0.23 VS1 378.0
0.23 VS1 378.0
0.23 VVS2 378.0
0.23 VS1 376.0
0.23 SI1 375.0
0.23 VS1 373.0
0.23 VS2 373.0
0.23 VS1 373.0
0.23 VS1 373.0
0.23 VVS2 369.0
0.23 VS2 369.0
0.23 IF 369.0
0.23 SI1 364.0
0.23 SI1 364.0
0.23 VS2 362.0
0.23 VS2 357.0
0.23 VS1 357.0
0.23 VS2 357.0
0.23 VS1 357.0
0.23 VS2 357.0
0.23 VVS2 354.0
0.23 VS1 353.0
0.23 VS2 352.0
0.23 VS1 340.0
0.23 VS1 338.0
0.23 VS1 327.0
0.23 SI2 326.0
0.24 VVS1 963.0
0.24 VVS1 752.0
0.24 VVS1 710.0
0.24 VVS1 710.0
0.24 VS1 687.0
0.24 VVS2 678.0
0.24 VVS1 678.0
0.24 VVS1 678.0
0.24 IF 678.0
0.24 VVS2 678.0
0.24 VVS2 678.0
0.24 VVS1 678.0
0.24 VVS1 678.0
0.24 VVS1 678.0
0.24 VVS2 678.0
0.24 IF 678.0
0.24 IF 678.0
0.24 VVS2 678.0
0.24 IF 678.0
0.24 VVS2 678.0
0.24 VVS2 668.0
0.24 VVS2 668.0
0.24 VVS1 668.0
0.24 VVS1 668.0
0.24 VVS2 608.0
0.24 VVS2 608.0
0.24 VVS1 608.0
0.24 VVS1 608.0
0.24 VVS1 608.0
0.24 VVS1 608.0
0.24 VVS2 608.0
0.24 VVS2 608.0
0.24 VVS2 608.0
0.24 VS1 572.0
0.24 VS1 572.0
0.24 SI1 571.0
0.24 VVS1 559.0
0.24 IF 559.0
0.24 VVS1 559.0
0.24 VVS1 559.0
0.24 VVS1 559.0
0.24 IF 559.0
0.24 IF 559.0
0.24 IF 559.0
0.24 IF 559.0
0.24 VVS1 559.0
0.24 VVS1 554.0
0.24 VVS2 554.0
0.24 VVS1 554.0
0.24 VVS1 554.0
0.24 VVS1 553.0
0.24 VVS1 553.0
0.24 VVS2 552.0
0.24 VVS2 552.0
0.24 VVS2 552.0
0.24 VVS2 552.0
0.24 VVS2 552.0
0.24 VVS2 552.0
0.24 VVS1 552.0
0.24 VVS1 552.0
0.24 VVS1 552.0
0.24 VVS2 552.0
0.24 VVS2 552.0
0.24 VVS2 552.0
0.24 VVS2 552.0
0.24 VVS2 552.0
0.24 VVS2 552.0
0.24 VVS2 552.0
0.24 VVS2 552.0
0.24 VVS2 552.0
0.24 VVS2 552.0
0.24 VVS1 552.0
0.24 VVS1 552.0
0.24 VVS1 552.0
0.24 VVS1 552.0
0.24 VVS1 552.0
0.24 VVS1 552.0
0.24 VVS1 552.0
0.24 IF 552.0
0.24 IF 552.0
0.24 VVS2 552.0
0.24 VVS2 552.0
0.24 VVS2 552.0
0.24 VVS1 552.0
0.24 VVS2 552.0
0.24 VVS2 552.0
0.24 VVS2 552.0
0.24 VVS2 547.0
0.24 VVS2 540.0
0.24 VVS2 540.0
0.24 VVS2 538.0
0.24 VVS2 538.0
0.24 VS2 536.0
0.24 VS2 536.0
0.24 VS1 536.0
0.24 VVS2 533.0
0.24 VVS1 526.0
0.24 VVS2 526.0
0.24 VVS2 526.0
0.24 VVS1 526.0
0.24 VVS2 526.0
0.24 VVS2 526.0
0.24 VVS2 526.0
0.24 VVS2 523.0
0.24 VVS2 521.0
0.24 VVS2 521.0
0.24 VVS1 521.0
0.24 VVS1 521.0
0.24 IF 504.0
0.24 VVS2 504.0
0.24 VVS1 504.0
0.24 IF 504.0
0.24 IF 504.0
0.24 VVS1 504.0
0.24 VVS1 504.0
0.24 VVS2 499.0
0.24 VVS1 499.0
0.24 VVS2 498.0
0.24 VVS1 492.0
0.24 VVS2 492.0
0.24 VVS1 492.0
0.24 VVS2 492.0
0.24 VVS2 492.0
0.24 VVS1 492.0
0.24 VVS1 492.0
0.24 IF 492.0
0.24 VVS2 492.0
0.24 VS1 490.0
0.24 SI1 486.0
0.24 VVS2 485.0
0.24 VVS2 485.0
0.24 VVS2 485.0
0.24 VVS2 485.0
0.24 VVS1 485.0
0.24 VVS2 485.0
0.24 VVS1 485.0
0.24 VVS1 485.0
0.24 VVS2 485.0
0.24 VVS2 485.0
0.24 VVS2 485.0
0.24 VVS2 485.0
0.24 VVS2 485.0
0.24 VVS2 485.0
0.24 VVS2 485.0
0.24 VVS1 485.0
0.24 VVS1 485.0
0.24 VVS1 485.0
0.24 VVS1 485.0
0.24 VVS1 485.0
0.24 VVS1 485.0
0.24 VVS2 485.0
0.24 VVS1 485.0
0.24 VS2 485.0
0.24 VVS1 485.0
0.24 VVS2 485.0
0.24 VVS1 485.0
0.24 VVS2 478.0
0.24 VVS2 478.0
0.24 VVS1 478.0
0.24 VVS2 478.0
0.24 VVS2 478.0
0.24 VVS2 478.0
0.24 VVS1 478.0
0.24 VVS1 478.0
0.24 VVS1 478.0
0.24 VVS1 478.0
0.24 VVS1 478.0
0.24 VVS2 478.0
0.24 VVS2 478.0
0.24 VVS2 478.0
0.24 SI1 475.0
0.24 VVS2 471.0
0.24 VVS2 471.0
0.24 VS2 461.0
0.24 VS1 461.0
0.24 VS2 461.0
0.24 VS1 461.0
0.24 VVS2 449.0
0.24 VVS1 449.0
0.24 VVS2 449.0
0.24 IF 449.0
0.24 VVS1 449.0
0.24 VVS2 449.0
0.24 VVS2 449.0
0.24 VVS2 449.0
0.24 VVS1 449.0
0.24 VVS2 449.0
0.24 VVS2 449.0
0.24 VS1 442.0
0.24 VVS2 442.0
0.24 VS1 436.0
0.24 VVS2 432.0
0.24 VVS1 432.0
0.24 VVS1 432.0
0.24 VVS1 432.0
0.24 VVS2 432.0
0.24 VS1 432.0
0.24 VVS1 432.0
0.24 VS1 430.0
0.24 VS2 419.0
0.24 VS2 419.0
0.24 VS2 419.0
0.24 VS2 419.0
0.24 VS2 419.0
0.24 VS1 419.0
0.24 VS2 419.0
0.24 VS2 419.0
0.24 VS1 419.0
0.24 VS1 419.0
0.24 VS1 419.0
0.24 VS2 419.0
0.24 VS2 419.0
0.24 VS2 419.0
0.24 VS1 419.0
0.24 VS1 419.0
0.24 VS1 419.0
0.24 VS2 419.0
0.24 VS2 419.0
0.24 VS2 419.0
0.24 VS2 419.0
0.24 VS1 419.0
0.24 VS1 419.0
0.24 VS1 419.0
0.24 VS2 419.0
0.24 VS2 417.0
0.24 VS2 417.0
0.24 VS1 417.0
0.24 VS1 412.0
0.24 VS2 408.0
0.24 SI1 404.0
0.24 VS1 399.0
0.24 VS1 397.0
0.24 VS1 393.0
0.24 VS1 393.0
0.24 VS2 391.0
0.24 VS1 391.0
0.24 VS1 391.0
0.24 VS1 383.0
0.24 VS1 378.0
0.24 VS2 378.0
0.24 VS1 373.0
0.24 VS1 373.0
0.24 VS2 373.0
0.24 VS1 373.0
0.24 SI1 370.0
0.24 VS1 367.0
0.24 VS2 367.0
0.24 SI1 364.0
0.24 VS2 362.0
0.24 VS2 362.0
0.24 VS1 357.0
0.24 VS1 355.0
0.24 VVS1 336.0
0.24 VVS2 336.0
0.25 SI2 1186.0
0.25 SI2 1186.0
0.25 SI2 1013.0
0.25 VVS1 817.0
0.25 VVS1 783.0
0.25 VVS2 740.0
0.25 VVS2 740.0
0.25 VVS2 740.0
0.25 VVS1 740.0
0.25 VVS2 740.0
0.25 VVS2 740.0
0.25 VVS2 740.0
0.25 VVS2 740.0
0.25 VVS1 740.0
0.25 VVS1 740.0
0.25 VVS2 740.0
0.25 VVS1 740.0
0.25 VVS1 740.0
0.25 IF 740.0
0.25 IF 739.0
0.25 VVS2 705.0
0.25 VVS1 705.0
0.25 VVS2 705.0
0.25 VVS1 705.0
0.25 VVS2 696.0
0.25 VVS2 633.0
0.25 VVS2 633.0
0.25 VVS1 633.0
0.25 VVS1 633.0
0.25 VVS1 633.0
0.25 IF 633.0
0.25 VVS1 633.0
0.25 VVS2 633.0
0.25 VVS1 633.0
0.25 VVS2 633.0
0.25 VVS1 633.0
0.25 VVS2 633.0
0.25 VVS2 633.0
0.25 IF 633.0
0.25 VVS2 633.0
0.25 IF 624.0
0.25 VS1 595.0
0.25 VS1 595.0
0.25 VS1 595.0
0.25 VVS2 583.0
0.25 VVS1 582.0
0.25 IF 582.0
0.25 VVS1 582.0
0.25 VVS1 582.0
0.25 VVS1 577.0
0.25 VVS1 577.0
0.25 VVS1 577.0
0.25 VVS1 577.0
0.25 VVS1 576.0
0.25 VVS1 576.0
0.25 VVS2 576.0
0.25 VVS2 576.0
0.25 VVS2 576.0
0.25 VVS2 576.0
0.25 VVS1 576.0
0.25 VVS2 576.0
0.25 IF 576.0
0.25 VVS2 576.0
0.25 VVS2 576.0
0.25 VVS1 576.0
0.25 VVS2 576.0
0.25 VVS1 576.0
0.25 VVS2 575.0
0.25 VVS2 575.0
0.25 VVS2 575.0
0.25 VVS2 575.0
0.25 VVS2 575.0
0.25 VVS2 575.0
0.25 VVS2 575.0
0.25 VVS1 575.0
0.25 VVS2 575.0
0.25 VVS2 575.0
0.25 VVS2 575.0
0.25 VVS2 575.0
0.25 VVS2 575.0
0.25 VVS1 575.0
0.25 VVS1 575.0
0.25 VVS1 575.0
0.25 VVS2 575.0
0.25 VVS2 575.0
0.25 VVS2 575.0
0.25 VVS2 575.0
0.25 VVS2 575.0
0.25 VVS1 575.0
0.25 VVS1 575.0
0.25 VVS2 575.0
0.25 VS1 563.0
0.25 VVS1 560.0
0.25 IF 560.0
0.25 VVS2 560.0
0.25 VS1 558.0
0.25 VS1 558.0
0.25 VS1 558.0
0.25 VS1 558.0
0.25 VVS2 548.0
0.25 VVS2 548.0
0.25 VVS2 548.0
0.25 VVS2 548.0
0.25 VVS2 548.0
0.25 VVS2 548.0
0.25 IF 548.0
0.25 VVS2 548.0
0.25 VVS2 548.0
0.25 VVS1 548.0
0.25 VVS1 548.0
0.25 VVS2 548.0
0.25 VVS2 548.0
0.25 VVS2 548.0
0.25 VVS2 548.0
0.25 VVS1 548.0
0.25 VVS2 548.0
0.25 VS2 535.0
0.25 VS2 535.0
0.25 VS2 535.0
0.25 VS2 535.0
0.25 VS2 535.0
0.25 VS2 535.0
0.25 VVS2 533.0
0.25 VVS1 533.0
0.25 VVS2 526.0
0.25 VVS1 526.0
0.25 IF 526.0
0.25 VVS1 525.0
0.25 VVS2 525.0
0.25 VVS1 525.0
0.25 VVS2 525.0
0.25 VVS1 525.0
0.25 VVS1 525.0
0.25 VVS2 525.0
0.25 VVS2 525.0
0.25 IF 525.0
0.25 VVS1 525.0
0.25 VVS1 525.0
0.25 VVS1 525.0
0.25 VVS2 525.0
0.25 VS2 525.0
0.25 VVS1 525.0
0.25 IF 512.0
0.25 VVS2 512.0
0.25 VVS2 512.0
0.25 VVS2 505.0
0.25 VVS2 500.0
0.25 VVS1 498.0
0.25 VVS1 490.0
0.25 VS2 480.0
0.25 VS1 480.0
0.25 VVS2 476.0
0.25 VVS2 476.0
0.25 VS2 472.0
0.25 VVS2 467.0
0.25 VVS1 467.0
0.25 VVS2 462.0
0.25 VS1 460.0
0.25 VS2 460.0
0.25 VS2 459.0
0.25 VS2 459.0
0.25 VVS1 457.0
0.25 VVS2 455.0
0.25 VS1 454.0
0.25 VS1 454.0
0.25 SI1 452.0
0.25 VVS1 451.0
0.25 VS2 450.0
0.25 VVS1 450.0
0.25 VVS1 450.0
0.25 VVS1 450.0
0.25 VS1 445.0
0.25 VS1 445.0
0.25 VS1 438.0
0.25 VS1 438.0
0.25 VS1 438.0
0.25 VS1 438.0
0.25 VS1 436.0
0.25 VS2 436.0
0.25 VS1 436.0
0.25 VS1 436.0
0.25 VS1 436.0
0.25 VS1 436.0
0.25 VS1 436.0
0.25 VS2 436.0
0.25 VS1 435.0
0.25 VS1 435.0
0.25 VS1 435.0
0.25 VS1 435.0
0.25 VS1 431.0
0.25 SI1 430.0
0.25 SI1 430.0
0.25 VS1 426.0
0.25 VVS1 421.0
0.25 VS1 410.0
0.25 VS2 409.0
0.25 VS2 407.0
0.25 VS2 407.0
0.25 SI1 407.0
0.25 VS2 404.0
0.25 VVS1 401.0
0.25 VS2 399.0
0.25 VS1 399.0
0.25 SI1 395.0
0.25 VS1 388.0
0.25 VS1 388.0
0.25 VS1 388.0
0.25 VS2 367.0
0.25 SI1 363.0
0.25 VS1 361.0
0.25 VS1 361.0
0.25 SI1 357.0
0.26 VVS1 814.0
0.26 VVS1 814.0
0.26 VVS1 814.0
0.26 VVS2 777.0
0.26 VVS1 769.0
0.26 VVS1 769.0
0.26 VVS1 769.0
0.26 VVS2 769.0
0.26 VVS1 769.0
0.26 VVS1 769.0
0.26 VVS1 769.0
0.26 VVS2 769.0
0.26 VVS2 769.0
0.26 VVS2 769.0
0.26 VVS1 769.0
0.26 VVS2 769.0
0.26 VVS1 769.0
0.26 VVS2 769.0
0.26 VVS2 769.0
0.26 VVS2 769.0
0.26 VVS2 769.0
0.26 IF 768.0
0.26 VVS2 733.0
0.26 VVS1 733.0
0.26 IF 733.0
0.26 VVS2 724.0
0.26 VVS1 724.0
0.26 VVS2 724.0
0.26 VVS2 724.0
0.26 VVS2 724.0
0.26 VVS2 724.0
0.26 VVS1 679.0
0.26 VVS1 679.0
0.26 SI1 658.0
0.26 VVS2 657.0
0.26 VVS2 657.0
0.26 VVS1 657.0
0.26 IF 657.0
0.26 VVS1 657.0
0.26 VVS1 657.0
0.26 VVS1 657.0
0.26 VVS1 657.0
0.26 IF 648.0
0.26 IF 648.0
0.26 VVS1 635.0
0.26 VVS1 635.0
0.26 VVS1 635.0
0.26 VS1 618.0
0.26 VS1 618.0
0.26 VVS2 614.0
0.26 IF 605.0
0.26 IF 605.0
0.26 VVS1 605.0
0.26 VVS1 605.0
0.26 VS2 601.0
0.26 IF 600.0
0.26 VVS1 600.0
0.26 VVS1 600.0
0.26 VVS2 600.0
0.26 VVS1 600.0
0.26 VVS2 600.0
0.26 VVS1 599.0
0.26 VVS1 599.0
0.26 VVS1 599.0
0.26 VVS1 599.0
0.26 VVS1 599.0
0.26 VVS1 599.0
0.26 VVS2 599.0
0.26 VVS2 599.0
0.26 VVS2 599.0
0.26 VVS1 599.0
0.26 VVS1 599.0
0.26 VVS2 599.0
0.26 VVS2 599.0
0.26 VVS2 599.0
0.26 VVS2 599.0
0.26 VVS2 599.0
0.26 VVS2 599.0
0.26 VVS2 599.0
0.26 VVS2 597.0
0.26 VVS2 597.0
0.26 VVS2 597.0
0.26 VVS2 597.0
0.26 VVS2 597.0
0.26 VVS1 597.0
0.26 VVS1 597.0
0.26 VVS2 597.0
0.26 VVS1 597.0
0.26 VVS1 597.0
0.26 VVS2 597.0
0.26 IF 597.0
0.26 VVS2 597.0
0.26 VVS2 597.0
0.26 VVS1 597.0
0.26 VVS2 597.0
0.26 VVS2 597.0
0.26 VVS1 591.0
0.26 SI1 590.0
0.26 VVS2 584.0
0.26 VS1 580.0
0.26 VS1 580.0
0.26 VS1 580.0
0.26 VS1 580.0
0.26 VS2 580.0
0.26 VS2 580.0
0.26 VS1 580.0
0.26 VS1 580.0
0.26 VS1 580.0
0.26 VS1 578.0
0.26 VVS2 569.0
0.26 VVS2 565.0
0.26 VVS2 565.0
0.26 VVS2 564.0
0.26 VVS2 564.0
0.26 VVS2 564.0
0.26 VVS2 564.0
0.26 VVS1 564.0
0.26 VVS2 564.0
0.26 VVS1 562.0
0.26 VS1 556.0
0.26 VS1 556.0
0.26 VS1 556.0
0.26 VVS2 554.0
0.26 VVS2 554.0
0.26 VVS1 554.0
0.26 VVS2 554.0
0.26 VVS1 554.0
0.26 VVS1 554.0
0.26 VVS2 554.0
0.26 VVS1 554.0
0.26 VVS2 554.0
0.26 VVS2 547.0
0.26 IF 547.0
0.26 VVS1 547.0
0.26 VVS1 547.0
0.26 VS1 546.0
0.26 VVS2 545.0
0.26 VVS2 545.0
0.26 VVS1 545.0
0.26 IF 545.0
0.26 VVS1 545.0
0.26 IF 545.0
0.26 VVS2 545.0
0.26 VVS1 545.0
0.26 VVS1 545.0
0.26 VVS1 545.0
0.26 VVS1 539.0
0.26 VVS1 532.0
0.26 VVS1 532.0
0.26 VVS1 524.0
0.26 VVS2 517.0
0.26 VVS2 514.0
0.26 VS1 508.0
0.26 VVS2 508.0
0.26 VVS2 506.0
0.26 VS2 499.0
0.26 VS1 499.0
0.26 VS1 499.0
0.26 VS1 499.0
0.26 VS1 491.0
0.26 VS1 491.0
0.26 VVS2 486.0
0.26 VVS2 486.0
0.26 VVS2 486.0
0.26 VVS2 486.0
0.26 VVS2 486.0
0.26 VVS2 486.0
0.26 VS1 482.0
0.26 VS1 482.0
0.26 VS1 482.0
0.26 VS1 482.0
0.26 VS1 482.0
0.26 VVS2 478.0
0.26 VS1 478.0
0.26 SI1 474.0
0.26 VVS2 468.0
0.26 VVS2 468.0
0.26 VVS1 468.0
0.26 VVS1 468.0
0.26 VVS1 468.0
0.26 IF 468.0
0.26 VS1 462.0
0.26 VS2 456.0
0.26 VS1 456.0
0.26 VS1 456.0
0.26 VS2 456.0
0.26 VS1 453.0
0.26 VS1 453.0
0.26 VS1 453.0
0.26 VS2 453.0
0.26 VS2 453.0
0.26 VS1 453.0
0.26 VS1 453.0
0.26 VS2 453.0
0.26 VS2 453.0
0.26 VS2 453.0
0.26 VS2 453.0
0.26 VS1 453.0
0.26 VS2 453.0
0.26 VS1 452.0
0.26 VS1 452.0
0.26 VS1 452.0
0.26 VS2 452.0
0.26 VS1 452.0
0.26 VS1 452.0
0.26 VS1 452.0
0.26 VS2 452.0
0.26 VS1 452.0
0.26 VS1 448.0
0.26 VS1 448.0
0.26 VS1 448.0
0.26 VS1 448.0
0.26 SI1 447.0
0.26 SI1 445.0
0.26 VVS2 440.0
// use this to type cast strings into Int when the table is loaded with string-valued columns
//display(spark.sql("select cast(carat as Int) as carat, clarity, cast(price as Int) as price from diamond order by carat asc, price desc"))
// sort by multiple fields and limit to first 5
// I prefer lowercase for SQL - and you can use either in this course - but in the field do what your Boss or your colleagues prefer :)
display(spark.sql("select carat, clarity, price from diamonds order by carat desc, price desc limit 5"))
carat clarity price
5.01 I1 18018.0
4.5 I1 18531.0
4.13 I1 17329.0
4.01 I1 15223.0
4.01 I1 15223.0
//aggregate functions
display(spark.sql("select avg(price) as avgprice from diamonds"))
avgprice
3932.799721913237
//average operator is doing an auto-type conversion from int to double
display(spark.sql("select avg(cast(price as Integer)) as avgprice from diamonds"))
avgprice
3932.799721913237
//aggregate function and grouping
display(spark.sql("select color, avg(price) as avgprice from diamonds group by color"))
color avgprice
F 3724.886396981765
E 3076.7524752475247
D 3169.9540959409596
J 5323.81801994302
G 3999.135671271697
I 5091.874953891553
H 4486.669195568401

Why do we need to know these interactive SQL queries?

Such queries can help us explore the data and thereby inform the modeling process!!!

Of course, if you don't know SQL then don't worry, we will be doing these things in scala using DataFrames.

Finally, those who are planning to take the Spark Developer Exams online, then you can't escape from SQL questions there...

ScaDaMaLe Course site and book

Power Plant ML Pipeline Application - DataFrame Part

This is the Spark SQL parts of an end-to-end example of using a number of different machine learning algorithms to solve a supervised regression problem.

This is a break-down of Power Plant ML Pipeline Application from databricks.

This will be a recurring example in the sequel

Table of Contents
  • Step 1: Business Understanding
  • Step 2: Load Your Data
  • Step 3: Explore Your Data
  • Step 4: Visualize Your Data
  • Step 5: Data Preparation
  • Step 6: Data Modeling
  • Step 7: Tuning and Evaluation
  • Step 8: Deployment

We are trying to predict power output given a set of readings from various sensors in a gas-fired power generation plant. Power generation is a complex process, and understanding and predicting power output is an important element in managing a plant and its connection to the power grid.

  • Given this business problem, we need to translate it to a Machine Learning task (actually a Statistical Machine Learning task).
  • The ML task here is regression since the label (or target) we will be trying to predict takes a continuous numeric value
    • Note: if the labels took values from a finite discrete set, such as, Spam/Not-Spam or Good/Bad/Ugly, then the ML task would be classification.

Today, we will only cover Steps 1, 2, 3 and 4 above. You need introductions to linear algebra, stochastic gradient descent and decision trees before we can accomplish the applied ML task with some intuitive understanding. If you can't wait for ML then check out Spark MLLib Programming Guide for comming attractions!

The example data is provided by UCI at UCI Machine Learning Repository Combined Cycle Power Plant Data Set

You can read the background on the UCI page, but in summary:

  • we have collected a number of readings from sensors at a Gas Fired Power Plant (also called a Peaker Plant) and
  • want to use those sensor readings to predict how much power the plant will generate in a couple weeks from now.
  • Again, today we will just focus on Steps 1-4 above that pertain to DataFrames.

More information about Peaker or Peaking Power Plants can be found on Wikipedia https://en.wikipedia.org/wiki/Peakingpowerplant.

sc.version.replace(".", "").toInt
res24: Int = 321
// a good habit to ensure the code is being run on the appropriate version of Spark - we are using Spark 3.+ actually...
require(sc.version.replace(".", "").toInt >= 140, "Spark 1.4.0+ is required to run this notebook. Please attach it to a Spark 1.4.0+ cluster.")

Step 1: Business Understanding

The first step in any machine learning task is to understand the business need.

As described in the overview we are trying to predict power output given a set of readings from various sensors in a gas-fired power generation plant.

The problem is a regression problem since the label (or target) we are trying to predict is numeric

Step 2: Load Your Data

Now that we understand what we are trying to do, we need to load our data and describe it, explore it and verify it.

Data was downloaded already as these five Tab-separated-variable or tsv files.

ls /datasets/sds/power-plant/data
path name size modificationTime
dbfs:/datasets/sds/power-plant/data/Sheet1.tsv Sheet1.tsv 308693.0 1.664295999e12
dbfs:/datasets/sds/power-plant/data/Sheet2.tsv Sheet2.tsv 308693.0 1.664295998e12
dbfs:/datasets/sds/power-plant/data/Sheet3.tsv Sheet3.tsv 308693.0 1.664295999e12
dbfs:/datasets/sds/power-plant/data/Sheet4.tsv Sheet4.tsv 308693.0 1.664295998e12
dbfs:/datasets/sds/power-plant/data/Sheet5.tsv Sheet5.tsv 308693.0 1.664295998e12

Now let us load the data from the Tab-separated-variable or tsv text file into an RDD[String] using the familiar textFile method.

val powerPlantRDD = sc.textFile("/datasets/sds/power-plant/data/Sheet1.tsv") // Ctrl+Enter
powerPlantRDD: org.apache.spark.rdd.RDD[String] = /datasets/sds/power-plant/data/Sheet1.tsv MapPartitionsRDD[778] at textFile at command-2971213210277530:1
powerPlantRDD.take(5).foreach(println) // Ctrl+Enter to print first 5 lines
AT	V	AP	RH	PE
14.96	41.76	1024.07	73.17	463.26
25.18	62.96	1020.04	59.08	444.37
5.11	39.4	1012.16	92.14	488.56
20.86	57.32	1010.24	76.64	446.48
// let us make sure we are using Spark version greater than 2.2 - we need a version closer to 2.0 if we want to use SparkSession and SQLContext 
require(sc.version.replace(".", "").toInt >= 220, "Spark 2.2.0+ is required to run this notebook. Please attach it to a Spark 2.2.0+ cluster.")
// this reads the tsv file and turns it into a dataframe
val powerPlantDF = spark.read // use 'sqlContext.read' instead if you want to use older Spark version > 1.3  see 008_ notebook
    .format("csv") // use spark.csv package
    .option("header", "true") // Use first line of all files as header
    .option("inferSchema", "true") // Automatically infer data types
    .option("delimiter", "\t") // Specify the delimiter as Tab or '\t'
    .load("/datasets/sds/power-plant/data/Sheet1.tsv")
powerPlantDF: org.apache.spark.sql.DataFrame = [AT: double, V: double ... 3 more fields]
powerPlantDF.printSchema // print the schema of the DataFrame that was inferred
root
 |-- AT: double (nullable = true)
 |-- V: double (nullable = true)
 |-- AP: double (nullable = true)
 |-- RH: double (nullable = true)
 |-- PE: double (nullable = true)
powerPlantDF.count
res31: Long = 9568

2.1. Alternatively, load data via the upload GUI feature in databricks

USE THIS FOR OTHER SMALLish DataSets you want to import to your CE

Since the dataset is relatively small, we can also use the upload feature in Databricks to upload the data as a table.

First download the Data Folder from UCI Machine Learning Repository Combined Cycle Power Plant Data Set

The file is a multi-tab Excel document so you will need to save each tab as a Text file export.

I prefer exporting as a Tab-Separated-Values (TSV) since it is more consistent than CSV.

Call each file Folds5x2_pp<Sheet 1..5>.tsv and save to your machine.

Refer to https://docs.databricks.com/user-guide/importing-data.html for latest methods to import data.

Now that your data is loaded let's explore it.

Step 3: Explore Your Data

Now that we understand what we are trying to do, we need to load our data and describe it, explore it and verify it.

Viewing the table as text

By uisng .show method we can see some of the contents of the table in plain text.

This works in pure Apache Spark, say in Spark-Shell without any notebook layer on top of Spark like databricks, zeppelin or jupyter.

It is a good idea to use this method when possible.

powerPlantDF.show(10) // try putting 1000 here instead of 10
+-----+-----+-------+-----+------+
|   AT|    V|     AP|   RH|    PE|
+-----+-----+-------+-----+------+
|14.96|41.76|1024.07|73.17|463.26|
|25.18|62.96|1020.04|59.08|444.37|
| 5.11| 39.4|1012.16|92.14|488.56|
|20.86|57.32|1010.24|76.64|446.48|
|10.82| 37.5|1009.23|96.62| 473.9|
|26.27|59.44|1012.23|58.77|443.67|
|15.89|43.96|1014.02|75.24|467.35|
| 9.48|44.71|1019.12|66.43|478.42|
|14.64| 45.0|1021.78|41.25|475.98|
|11.74|43.56|1015.14|70.72| 477.5|
+-----+-----+-------+-----+------+
only showing top 10 rows

Viewing as DataFrame

This is almost necessary for a data scientist to gain visual insights into all pair-wise relationships between the several (3 to 6 or so) variables in question.

display(powerPlantDF) 
AT V AP RH PE
14.96 41.76 1024.07 73.17 463.26
25.18 62.96 1020.04 59.08 444.37
5.11 39.4 1012.16 92.14 488.56
20.86 57.32 1010.24 76.64 446.48
10.82 37.5 1009.23 96.62 473.9
26.27 59.44 1012.23 58.77 443.67
15.89 43.96 1014.02 75.24 467.35
9.48 44.71 1019.12 66.43 478.42
14.64 45.0 1021.78 41.25 475.98
11.74 43.56 1015.14 70.72 477.5
17.99 43.72 1008.64 75.04 453.02
20.14 46.93 1014.66 64.22 453.99
24.34 73.5 1011.31 84.15 440.29
25.71 58.59 1012.77 61.83 451.28
26.19 69.34 1009.48 87.59 433.99
21.42 43.79 1015.76 43.08 462.19
18.21 45.0 1022.86 48.84 467.54
11.04 41.74 1022.6 77.51 477.2
14.45 52.75 1023.97 63.59 459.85
13.97 38.47 1015.15 55.28 464.3
17.76 42.42 1009.09 66.26 468.27
5.41 40.07 1019.16 64.77 495.24
7.76 42.28 1008.52 83.31 483.8
27.23 63.9 1014.3 47.19 443.61
27.36 48.6 1003.18 54.93 436.06
27.47 70.72 1009.97 74.62 443.25
14.6 39.31 1011.11 72.52 464.16
7.91 39.96 1023.57 88.44 475.52
5.81 35.79 1012.14 92.28 484.41
30.53 65.18 1012.69 41.85 437.89
23.87 63.94 1019.02 44.28 445.11
26.09 58.41 1013.64 64.58 438.86
29.27 66.85 1011.11 63.25 440.98
27.38 74.16 1010.08 78.61 436.65
24.81 63.94 1018.76 44.51 444.26
12.75 44.03 1007.29 89.46 465.86
24.66 63.73 1011.4 74.52 444.37
16.38 47.45 1010.08 88.86 450.69
13.91 39.35 1014.69 75.51 469.02
23.18 51.3 1012.04 78.64 448.86
22.47 47.45 1007.62 76.65 447.14
13.39 44.85 1017.24 80.44 469.18
9.28 41.54 1018.33 79.89 482.8
11.82 42.86 1014.12 88.28 476.7
10.27 40.64 1020.63 84.6 474.99
22.92 63.94 1019.28 42.69 444.22
16.0 37.87 1020.24 78.41 461.33
21.22 43.43 1010.96 61.07 448.06
13.46 44.71 1014.51 50.0 474.6
9.39 40.11 1029.14 77.29 473.05
31.07 73.5 1010.58 43.66 432.06
12.82 38.62 1018.71 83.8 467.41
32.57 78.92 1011.6 66.47 430.12
8.11 42.18 1014.82 93.09 473.62
13.92 39.39 1012.94 80.52 471.81
23.04 59.43 1010.23 68.99 442.99
27.31 64.44 1014.65 57.27 442.77
5.91 39.33 1010.18 95.53 491.49
25.26 61.08 1013.68 71.72 447.46
27.97 58.84 1002.25 57.88 446.11
26.08 52.3 1007.03 63.34 442.44
29.01 65.71 1013.61 48.07 446.22
12.18 40.1 1016.67 91.87 471.49
13.76 45.87 1008.89 87.27 463.5
25.5 58.79 1016.02 64.4 440.01
28.26 65.34 1014.56 43.4 441.03
21.39 62.96 1019.49 72.24 452.68
7.26 40.69 1020.43 90.22 474.91
10.54 34.03 1018.71 74.0 478.77
27.71 74.34 998.14 71.85 434.2
23.11 68.3 1017.83 86.62 437.91
7.51 41.01 1024.61 97.41 477.61
26.46 74.67 1016.65 84.44 431.65
29.34 74.34 998.58 81.55 430.57
10.32 42.28 1008.82 75.66 481.09
22.74 61.02 1009.56 79.41 445.56
13.48 39.85 1012.71 58.91 475.74
25.52 69.75 1010.36 90.06 435.12
21.58 67.25 1017.39 79.0 446.15
27.66 76.86 1001.31 69.47 436.64
26.96 69.45 1013.89 51.47 436.69
12.29 42.18 1016.53 83.13 468.75
15.86 43.02 1012.18 40.33 466.6
13.87 45.08 1024.42 81.69 465.48
24.09 73.68 1014.93 94.55 441.34
20.45 69.45 1012.53 91.81 441.83
15.07 39.3 1019.0 63.62 464.7
32.72 69.75 1009.6 49.35 437.99
18.23 58.96 1015.55 69.61 459.12
35.56 68.94 1006.56 38.75 429.69
18.36 51.43 1010.57 90.17 459.8
26.35 64.05 1009.81 81.24 433.63
25.92 60.95 1014.62 48.46 442.84
8.01 41.66 1014.49 76.72 485.13
19.63 52.72 1025.09 51.16 459.12
20.02 67.32 1012.05 76.34 445.31
10.08 40.72 1022.7 67.3 480.8
27.23 66.48 1005.23 52.38 432.55
23.37 63.77 1013.42 76.44 443.86
18.74 59.21 1018.3 91.55 449.77
14.81 43.69 1017.19 71.9 470.71
23.1 51.3 1011.93 80.05 452.17
10.72 41.38 1021.6 63.77 478.29
29.46 71.94 1006.96 62.26 428.54
8.1 40.64 1020.66 89.04 478.27
27.29 62.66 1007.63 58.02 439.58
17.1 49.69 1005.53 81.82 457.32
11.49 44.2 1018.79 91.14 475.51
23.69 65.59 1010.85 88.92 439.66
13.51 40.89 1011.03 84.83 471.99
9.64 39.35 1015.1 91.76 479.81
25.65 78.92 1010.83 86.56 434.78
21.59 61.87 1011.18 57.21 446.58
27.98 58.33 1013.92 54.25 437.76
18.8 39.72 1001.24 63.8 459.36
18.28 44.71 1016.99 33.71 462.28
13.55 43.48 1016.08 67.25 464.33
22.99 46.21 1010.71 60.11 444.36
23.94 59.39 1014.32 74.55 438.64
13.74 34.03 1018.69 67.34 470.49
21.3 41.1 1001.86 42.75 455.13
27.54 66.93 1017.06 55.2 450.22
24.81 63.73 1009.34 83.61 440.43
4.97 42.85 1014.02 88.78 482.98
15.22 50.88 1014.19 100.12 460.44
23.88 54.2 1012.81 64.52 444.97
33.01 68.67 1005.2 51.41 433.94
25.98 73.18 1012.28 85.78 439.73
28.18 73.88 1005.89 75.41 434.48
21.67 60.84 1017.93 81.63 442.33
17.67 45.09 1014.26 51.92 457.67
21.37 57.76 1018.8 70.12 454.66
28.69 67.25 1017.71 53.83 432.21
16.61 43.77 1012.25 77.23 457.66
27.91 63.76 1010.27 65.67 435.21
20.97 47.43 1007.64 71.18 448.22
10.8 41.66 1013.79 81.96 475.51
20.61 62.91 1013.24 79.54 446.53
25.45 57.32 1011.7 47.09 441.3
30.16 69.34 1007.67 57.69 433.54
4.99 39.04 1020.45 78.89 472.52
10.51 44.78 1012.59 85.29 474.77
33.79 69.05 1001.62 40.13 435.1
21.34 59.8 1016.92 77.06 450.74
23.4 65.06 1014.32 67.38 442.7
32.21 68.14 1003.34 62.44 426.56
14.26 42.32 1016.0 77.43 463.71
27.71 66.93 1016.85 58.77 447.06
21.95 57.76 1018.02 67.72 452.27
25.76 63.94 1018.49 42.14 445.78
23.68 68.3 1017.93 84.16 438.65
8.28 40.77 1011.55 89.79 480.15
23.44 62.52 1016.46 67.21 447.19
25.32 48.41 1008.47 72.14 443.04
3.94 39.9 1008.06 97.49 488.81
17.3 57.76 1016.26 87.74 455.75
18.2 49.39 1018.83 96.3 455.86
21.43 46.97 1013.94 61.25 457.68
11.16 40.05 1014.95 88.38 479.11
30.38 74.16 1007.44 74.77 432.84
23.36 62.52 1016.18 68.18 448.37
21.69 47.45 1007.56 77.2 447.06
23.62 49.21 1014.1 49.54 443.53
21.87 61.45 1011.13 92.22 445.21
29.25 66.51 1015.53 33.65 441.7
20.03 66.86 1013.05 64.59 450.93
18.14 49.78 1002.95 100.09 451.44
24.23 56.89 1012.32 68.04 441.29
18.11 44.85 1014.48 48.94 458.85
6.57 43.65 1018.24 74.47 481.46
12.56 43.41 1016.93 81.02 467.19
13.4 41.58 1020.5 71.17 461.54
27.1 52.84 1006.28 53.85 439.08
14.28 42.74 1028.79 70.67 467.22
16.29 44.34 1019.49 59.36 468.8
31.24 71.98 1004.66 57.17 426.93
10.57 37.73 1024.36 70.29 474.65
13.8 44.21 1022.93 83.37 468.97
25.3 71.58 1010.18 87.36 433.97
18.06 50.16 1009.52 100.09 450.53
25.42 59.04 1011.98 68.78 444.51
15.07 40.69 1015.29 70.98 469.03
11.75 71.14 1019.36 75.68 466.56
20.23 52.05 1012.15 47.49 457.57
27.31 59.54 1006.24 71.99 440.13
28.57 69.84 1003.57 66.55 433.24
17.9 43.72 1008.64 74.73 452.55
23.83 71.37 1002.04 64.78 443.29
27.92 74.99 1005.47 75.13 431.76
17.34 44.78 1007.81 56.38 454.97
17.94 63.07 1012.42 94.35 456.7
6.4 39.9 1007.75 86.55 486.03
11.78 39.96 1011.37 82.95 472.79
20.28 57.25 1010.12 88.42 452.03
21.04 54.2 1012.26 85.61 443.41
25.11 67.32 1014.49 58.39 441.93
30.28 70.98 1007.51 74.28 432.64
8.14 36.24 1013.15 87.85 480.25
16.86 39.63 1004.47 83.5 466.68
6.25 40.07 1020.19 65.24 494.39
22.35 54.42 1012.46 75.01 454.72
17.98 56.85 1012.28 84.52 448.71
21.19 42.48 1013.43 80.52 469.76
20.94 44.89 1009.64 75.14 450.71
24.23 58.79 1009.8 75.75 444.01
19.18 58.2 1017.46 76.72 453.2
20.88 57.85 1012.39 85.47 450.87
23.67 63.86 1019.67 57.95 441.73
14.12 39.52 1018.41 78.32 465.09
25.23 64.63 1020.59 52.2 447.28
6.54 39.33 1011.54 93.69 491.16
20.08 62.52 1017.99 75.74 450.98
24.67 63.56 1013.75 67.56 446.3
27.82 79.74 1008.37 69.46 436.48
15.55 42.03 1017.41 74.58 460.84
24.26 69.51 1013.43 53.23 442.56
13.45 41.49 1020.19 88.72 467.3
11.06 40.64 1021.47 96.16 479.13
24.91 52.3 1008.72 68.26 441.15
22.39 59.04 1011.78 86.39 445.52
11.95 40.69 1015.62 85.34 475.4
14.85 40.69 1014.91 72.64 469.3
10.11 41.62 1017.17 97.82 463.57
23.67 68.67 1006.71 77.22 445.32
16.14 44.21 1020.36 80.59 461.03
15.11 43.13 1014.99 46.91 466.74
24.14 59.87 1018.47 57.76 444.04
30.08 67.25 1017.6 53.09 434.01
14.77 44.9 1020.5 84.31 465.23
27.6 69.34 1009.63 71.58 440.6
13.89 44.84 1023.66 92.97 466.74
26.85 75.6 1017.43 74.55 433.48
12.41 40.96 1023.36 78.96 473.59
13.08 41.74 1020.75 64.44 474.81
18.93 44.06 1017.58 68.23 454.75
20.5 49.69 1009.6 70.81 452.94
30.72 69.13 1009.94 61.66 435.83
7.55 39.22 1014.53 77.76 482.19
13.49 44.47 1030.46 69.49 466.66
15.62 40.12 1013.03 96.26 462.59
24.8 64.63 1020.69 55.74 447.82
10.03 41.62 1014.55 95.61 462.73
22.43 63.21 1012.06 84.75 447.98
14.95 39.31 1009.15 75.3 462.72
24.78 58.46 1016.82 67.5 442.42
23.2 48.41 1008.64 80.92 444.69
14.01 39.0 1016.73 79.23 466.7
19.4 64.63 1020.38 81.1 453.84
30.15 67.32 1013.83 32.8 436.92
6.91 36.08 1021.82 84.31 486.37
29.04 60.07 1015.42 46.15 440.43
26.02 63.07 1010.94 53.96 446.82
5.89 39.48 1005.11 59.83 484.91
26.52 71.64 1008.27 75.3 437.76
28.53 68.08 1013.27 42.53 438.91
16.59 39.54 1007.97 70.58 464.19
22.95 67.79 1009.89 91.69 442.19
23.96 47.43 1008.38 63.55 446.86
17.48 44.2 1018.89 61.51 457.15
6.69 43.65 1020.14 69.55 482.57
10.25 41.26 1007.44 98.08 476.03
28.87 72.58 1008.69 79.34 428.89
12.04 40.23 1018.07 81.28 472.7
22.58 52.3 1009.04 78.99 445.6
15.12 52.05 1014.63 80.38 464.78
25.48 58.95 1017.02 51.16 440.42
27.87 70.79 1003.96 72.17 428.41
23.72 70.47 1010.65 75.39 438.5
25.0 59.43 1007.84 68.91 438.28
8.42 40.64 1022.35 96.38 476.29
22.46 58.49 1011.5 70.54 448.46
29.92 57.19 1008.62 45.8 438.99
11.68 39.22 1017.9 57.95 471.8
14.04 42.44 1012.74 81.89 471.81
19.86 59.14 1016.12 69.32 449.82
25.99 68.08 1013.13 59.14 442.14
23.42 58.79 1009.74 81.54 441.46
10.6 40.22 1011.37 85.81 477.62
20.97 61.87 1011.45 65.41 446.76
14.14 39.82 1012.46 81.15 472.52
8.56 40.71 1021.27 95.87 471.58
24.86 72.39 1001.15 90.24 440.85
29.0 77.54 1011.33 75.13 431.37
27.59 71.97 1008.64 88.22 437.33
10.45 40.71 1015.68 81.48 469.22
8.51 40.78 1023.51 89.84 471.11
29.82 66.51 1010.98 43.57 439.17
22.56 62.26 1012.11 63.16 445.33
11.38 39.22 1018.62 57.14 473.71
20.25 57.76 1016.28 77.76 452.66
22.42 59.43 1007.12 90.56 440.99
14.85 38.91 1014.48 60.98 467.42
25.62 58.82 1010.02 70.31 444.14
19.85 56.53 1020.57 74.05 457.17
13.67 54.3 1015.92 75.42 467.87
24.39 70.72 1009.78 82.25 442.04
16.07 44.58 1019.52 67.95 471.36
11.6 39.1 1009.81 100.09 460.7
31.38 70.83 1010.35 47.28 431.33
29.91 76.86 998.59 72.41 432.6
19.67 59.39 1014.07 77.67 447.61
27.18 64.79 1016.27 63.7 443.87
21.39 52.3 1009.2 79.77 446.87
10.45 41.01 1020.57 93.84 465.74
19.46 56.89 1014.02 84.95 447.86
23.55 62.96 1020.16 70.16 447.65
23.35 63.47 1011.78 84.24 437.87
9.26 41.66 1016.87 73.32 483.51
10.3 41.46 1018.21 86.17 479.65
20.94 58.16 1016.88 65.43 455.16
23.13 71.25 1002.49 94.59 431.91
12.77 41.5 1014.13 86.8 470.68
28.29 69.13 1009.29 58.18 429.28
19.13 59.21 1018.32 89.66 450.81
24.44 73.5 1011.49 87.39 437.73
20.32 44.6 1015.16 36.35 460.21
20.54 69.05 1001.6 79.62 442.86
12.16 45.0 1021.51 50.52 482.99
28.09 65.27 1013.27 51.96 440.0
9.25 41.82 1033.25 74.73 478.48
21.75 49.82 1015.01 78.33 455.28
23.7 66.56 1002.07 85.19 436.94
16.22 37.87 1022.36 83.13 461.06
24.75 69.45 1013.97 53.49 438.28
10.48 39.58 1011.81 88.86 472.61
29.53 70.79 1003.7 60.89 426.85
12.59 39.72 1017.76 61.14 470.18
23.5 54.42 1012.31 68.29 455.38
29.01 66.56 1006.44 57.62 428.32
9.75 42.49 1010.57 83.63 480.35
19.55 56.53 1020.2 78.1 455.56
21.05 58.33 1013.14 66.34 447.66
24.72 68.67 1006.74 79.02 443.06
21.19 58.86 1014.19 68.96 452.43
10.77 41.54 1019.94 71.13 477.81
28.68 73.77 1004.72 87.01 431.66
29.87 73.91 1004.53 74.3 431.8
22.99 68.67 1006.65 77.62 446.67
24.66 60.29 1018.0 59.56 445.26
32.63 69.89 1013.85 41.66 425.72
31.38 72.29 1008.73 73.27 430.58
23.87 60.27 1018.94 77.16 439.86
25.6 59.15 1013.31 67.02 441.11
27.62 71.14 1011.6 52.8 434.72
30.1 67.45 1014.23 39.04 434.01
12.19 41.17 1019.43 65.47 475.64
13.11 41.58 1020.43 74.32 460.44
28.29 68.67 1005.46 69.22 436.4
13.45 40.73 1018.7 93.88 461.03
10.98 41.54 1019.94 69.83 479.08
26.48 69.14 1009.31 84.11 435.76
13.07 45.51 1015.22 78.65 460.14
25.56 75.6 1017.37 69.31 442.2
22.68 50.78 1008.83 70.3 447.69
28.86 73.67 1006.65 68.23 431.15
22.7 63.56 1014.32 71.76 445.0
27.89 73.21 1001.32 85.88 431.59
13.78 44.47 1027.94 71.09 467.22
28.14 51.43 1012.16 52.67 445.33
11.8 45.09 1013.21 89.68 470.57
10.71 39.61 1018.72 73.66 473.77
24.54 60.29 1017.42 58.94 447.67
11.54 40.05 1014.78 87.05 474.29
29.47 71.32 1008.07 67.0 437.14
29.24 69.05 1003.12 43.18 432.56
14.51 41.79 1009.72 80.62 459.14
22.91 60.07 1016.03 59.72 446.19
27.02 71.77 1006.38 72.1 428.1
13.49 44.47 1030.18 69.15 468.46
30.24 66.75 1017.95 55.66 435.02
23.19 48.6 1002.38 61.19 445.52
17.73 40.55 1003.36 74.62 462.69
18.62 61.27 1019.26 73.35 455.75
12.85 40.0 1015.89 68.85 463.74
32.33 69.68 1011.95 39.89 439.79
25.09 58.95 1016.99 53.16 443.26
29.45 69.13 1009.3 52.97 432.04
16.91 43.96 1013.32 79.87 465.86
14.09 45.87 1009.05 84.09 465.6
10.73 25.36 1009.35 100.15 469.43
23.2 49.3 1003.4 79.77 440.75
8.21 38.91 1015.82 88.99 481.32
9.3 40.56 1022.64 76.14 479.87
16.97 39.16 1005.7 69.13 458.59
23.69 71.97 1009.62 93.03 438.62
25.13 59.44 1012.38 77.92 445.59
9.86 43.56 1015.13 74.89 481.87
11.33 41.5 1013.58 88.7 475.01
26.95 48.41 1008.53 62.94 436.54
15.0 40.66 1016.28 89.62 456.63
20.76 62.52 1015.63 81.04 451.69
14.29 39.59 1010.93 94.53 463.04
19.74 67.71 1007.68 64.02 446.1
26.68 59.92 1009.94 70.57 438.67
14.24 41.4 1019.7 70.32 466.88
21.98 48.41 1008.42 84.86 444.6
22.75 59.39 1015.4 81.41 440.26
8.34 40.96 1023.28 89.45 483.92
11.8 41.2 1017.18 82.71 475.19
8.81 44.68 1023.06 93.93 479.24
30.05 73.68 1014.95 70.6 434.92
16.01 65.46 1014.0 87.68 454.16
21.75 58.79 1012.42 87.58 447.58
13.94 41.26 1021.67 74.4 467.9
29.25 69.13 1010.27 67.35 426.29
22.33 45.87 1007.8 63.61 447.02
16.43 41.79 1005.47 76.89 455.85
11.5 40.22 1010.31 78.08 476.46
23.53 68.94 1007.53 69.17 437.48
21.86 49.21 1014.61 53.31 452.77
6.17 39.33 1012.57 93.32 491.54
30.19 64.79 1017.22 42.47 438.41
11.67 41.93 1019.81 82.58 476.1
15.34 36.99 1007.87 94.59 464.58
11.5 40.78 1023.91 86.31 467.74
25.53 57.17 1010.0 72.57 442.12
21.27 57.5 1014.53 80.76 453.34
28.37 69.13 1010.44 71.93 425.29
28.39 51.43 1011.74 47.54 449.63
13.78 45.78 1025.27 95.72 462.88
14.6 42.32 1015.71 77.03 464.67
5.1 35.57 1027.17 80.49 489.96
7.0 38.08 1020.27 77.67 482.38
26.3 77.95 1009.45 78.72 437.95
30.56 71.98 1004.74 58.77 429.2
21.09 46.63 1013.03 74.8 453.34
28.21 70.02 1010.58 51.34 442.47
15.84 49.69 1015.14 90.41 462.6
10.03 40.96 1024.57 91.1 478.79
20.37 52.05 1012.34 62.57 456.11
21.19 50.16 1005.81 84.27 450.33
33.73 69.88 1007.21 42.93 434.83
29.87 73.68 1015.1 40.96 433.43
19.62 62.96 1020.76 76.53 456.02
9.93 40.67 1018.08 69.74 485.23
9.43 37.14 1013.03 74.99 473.57
14.24 39.58 1011.17 70.45 469.94
12.97 49.83 1008.69 91.49 452.07
7.6 41.04 1021.82 88.97 475.32
8.39 36.24 1013.39 89.13 480.69
25.41 48.06 1013.12 46.52 444.01
18.43 56.03 1020.41 60.55 465.17
10.31 39.82 1012.87 88.71 480.61
11.29 41.5 1013.39 89.15 476.04
22.61 49.3 1003.51 83.02 441.76
29.34 71.98 1005.19 75.19 428.24
18.87 67.71 1004.0 87.35 444.77
13.21 45.87 1008.58 85.66 463.1
11.3 44.6 1018.19 91.66 470.5
29.23 72.99 1007.04 63.47 431.0
27.76 69.4 1004.27 72.25 430.68
29.26 67.17 1006.6 70.58 436.42
25.72 49.82 1016.19 60.1 452.33
23.43 63.94 1010.64 89.29 440.16
25.6 63.76 1010.18 67.43 435.75
22.3 44.57 1008.48 67.58 449.74
27.91 72.24 1010.74 70.8 430.73
30.35 77.17 1009.55 63.62 432.75
21.78 47.43 1007.88 66.68 446.79
7.19 41.39 1018.12 90.76 486.35
20.88 59.8 1015.66 75.34 453.18
24.19 50.23 1015.73 59.77 458.31
9.98 41.54 1019.7 80.79 480.26
23.47 51.3 1011.89 74.1 448.65
26.35 49.5 1012.67 41.34 458.41
29.89 64.69 1006.37 58.78 435.39
19.29 50.16 1010.49 97.78 450.21
17.48 43.14 1018.68 74.85 459.59
25.21 75.6 1017.19 69.84 445.84
23.3 48.78 1018.17 75.36 441.08
15.42 37.85 1009.89 85.8 467.33
21.44 63.09 1016.56 90.11 444.19
29.45 68.27 1007.96 61.63 432.96
29.69 47.93 1002.85 44.76 438.09
15.52 36.99 1006.86 89.7 467.9
11.47 43.67 1012.68 72.51 475.72
9.77 34.69 1027.72 74.98 477.51
22.6 69.84 1006.37 79.59 435.13
8.24 39.61 1017.99 78.42 477.9
17.01 44.2 1019.18 61.23 457.26
19.64 44.6 1015.88 47.56 467.53
10.61 41.58 1021.08 93.06 465.15
12.04 40.1 1014.42 89.65 474.28
29.19 65.71 1013.85 50.5 444.49
21.75 45.09 1014.15 44.84 452.84
23.66 77.54 1008.5 85.32 435.38
27.05 75.33 1003.88 82.94 433.57
29.63 69.71 1009.04 67.26 435.27
18.2 39.63 1005.35 79.05 468.49
32.22 70.8 1009.9 62.03 433.07
26.88 73.56 1004.85 94.36 430.63
29.05 65.74 1013.29 60.02 440.74
8.9 39.96 1026.31 95.46 474.49
18.93 48.6 1005.72 84.92 449.74
27.49 63.76 1010.09 62.8 436.73
23.1 70.79 1006.53 90.81 434.58
11.22 43.13 1017.24 80.9 473.93
31.97 79.74 1007.03 55.84 435.99
13.32 43.22 1009.45 75.3 466.83
31.68 68.24 1005.29 37.34 427.22
23.69 63.77 1013.39 79.5 444.07
13.83 41.49 1020.11 87.29 469.57
18.32 66.51 1015.18 81.5 459.89
11.05 40.71 1024.91 76.42 479.59
22.03 64.69 1007.21 75.75 440.92
10.23 41.46 1020.45 84.95 480.87
23.92 66.54 1009.93 62.37 441.9
29.38 69.68 1011.35 49.25 430.2
17.35 42.86 1014.62 74.16 465.16
9.81 44.45 1021.19 90.55 471.32
4.97 40.64 1020.91 94.28 485.43
5.15 40.07 1012.27 63.31 495.35
21.54 58.49 1010.85 78.9 449.12
7.94 42.02 1006.22 90.97 480.53
18.77 50.66 1014.89 87.34 457.07
21.69 69.94 1010.7 80.8 443.67
10.07 44.68 1023.44 90.95 477.52
13.83 39.64 1012.52 69.97 472.95
10.45 39.69 1003.92 89.45 472.54
11.56 40.71 1015.85 76.08 469.17
23.64 70.04 1011.09 83.35 435.21
10.48 40.22 1004.81 92.16 477.78
13.09 39.85 1012.86 58.42 475.89
10.67 40.23 1017.75 85.06 483.9
12.57 39.16 1016.53 88.91 476.2
14.45 43.34 1015.47 83.33 462.16
14.22 37.85 1011.24 88.49 471.05
6.97 41.26 1010.6 96.88 484.71
20.61 63.86 1015.43 73.86 446.34
14.67 42.28 1007.21 65.17 469.02
29.06 72.86 1004.23 69.41 432.12
14.38 40.1 1015.51 81.23 467.28
32.51 69.98 1013.29 54.07 429.66
11.79 45.09 1013.16 89.17 469.49
8.65 40.56 1023.23 78.85 485.87
9.75 40.81 1026.0 84.44 481.95
9.11 40.02 1031.1 83.02 479.03
23.39 69.13 1010.99 90.66 434.5
14.3 54.3 1015.16 75.29 464.9
17.49 63.94 1020.02 82.6 452.71
31.1 69.51 1010.84 45.4 429.74
19.77 56.65 1020.67 66.33 457.09
28.61 72.29 1011.61 45.33 446.77
13.52 41.48 1014.46 67.12 460.76
13.52 40.83 1008.31 84.14 471.95
17.57 46.21 1014.09 80.81 453.29
28.18 60.07 1016.34 49.13 441.61
14.29 46.18 1017.01 87.29 464.73
18.12 43.69 1016.91 52.95 464.68
31.27 73.91 1003.72 68.92 430.59
26.24 77.95 1014.19 85.21 438.01
7.44 41.04 1021.84 88.56 479.08
29.78 74.78 1009.28 55.09 436.39
23.37 65.46 1016.25 48.64 447.07
10.62 39.58 1011.9 87.85 479.91
5.84 43.02 1013.88 87.42 489.05
14.51 53.82 1016.46 62.75 463.17
11.31 42.02 1001.18 94.86 471.26
11.25 40.67 1011.64 63.54 480.49
9.18 39.42 1025.41 69.46 473.78
19.82 58.16 1016.76 74.66 455.5
24.77 58.41 1013.78 80.57 446.27
9.66 41.06 1021.21 84.7 482.2
21.96 59.8 1016.72 72.6 452.48
18.59 43.14 1011.92 52.63 464.48
24.75 69.89 1015.29 82.01 438.1
24.37 63.47 1012.77 75.22 445.6
29.6 67.79 1010.37 51.05 442.43
25.32 61.25 1011.56 80.1 436.67
16.15 41.85 1016.54 81.58 466.56
15.74 71.14 1019.65 65.94 457.29
5.97 36.25 1029.65 86.74 487.03
15.84 52.72 1026.45 62.57 464.93
14.84 44.63 1019.28 57.37 466.0
12.25 48.79 1017.44 88.91 469.52
27.38 70.04 1011.18 72.26 428.88
8.76 41.48 1018.49 74.98 474.3
15.54 39.31 1009.69 71.19 461.06
18.71 39.39 1014.09 62.82 465.57
13.06 41.78 1012.3 55.31 467.67
12.72 40.71 1016.02 71.57 466.99
19.83 39.39 1013.73 59.16 463.72
27.23 49.16 1004.03 40.8 443.78
24.27 68.28 1005.43 67.63 445.23
11.8 40.66 1017.13 97.2 464.43
6.76 36.25 1028.31 91.16 484.36
25.99 63.07 1012.5 64.81 442.16
16.3 39.63 1004.64 85.61 464.11
16.5 49.39 1018.35 93.42 462.48
10.59 42.49 1009.59 77.36 477.49
26.05 65.59 1012.78 67.03 437.04
19.5 40.79 1003.8 89.45 457.09
22.21 45.01 1012.22 54.84 450.6
17.86 45.0 1023.25 53.48 465.78
29.96 70.04 1010.15 54.47 427.1
19.08 44.63 1020.14 43.36 459.81
23.59 47.43 1006.64 48.92 447.36
3.38 39.64 1011.0 81.22 488.92
26.39 66.49 1012.96 60.35 433.36
8.99 39.04 1021.99 75.98 483.35
10.91 41.04 1026.57 74.24 469.53
13.08 39.82 1012.27 85.21 476.96
23.95 58.46 1017.5 68.46 440.75
15.64 43.71 1024.51 78.31 462.55
18.78 54.2 1012.05 89.25 448.04
20.65 50.59 1016.22 68.57 455.24
4.96 40.07 1011.8 67.38 494.75
23.51 57.32 1012.55 53.6 444.58
5.99 35.79 1011.56 91.69 484.82
23.65 66.05 1019.6 78.21 442.9
5.17 39.33 1009.68 94.19 485.46
26.38 49.5 1012.82 37.19 457.81
6.02 43.65 1013.85 83.53 481.92
23.2 61.02 1009.63 79.45 443.23
8.57 39.69 1000.91 99.9 474.29
30.72 71.58 1009.98 50.39 430.46
21.52 50.66 1013.56 74.33 455.71
22.93 62.26 1011.25 83.66 438.34
5.71 41.31 1003.24 89.48 485.83
18.62 44.06 1017.76 64.59 452.82
27.88 68.94 1007.68 75.68 435.04
22.32 59.8 1016.82 64.18 451.21
14.55 42.74 1028.41 70.09 465.81
17.83 44.92 1025.04 70.58 458.42
9.68 39.96 1026.09 99.28 470.22
19.41 49.39 1020.84 81.89 449.24
13.22 44.92 1023.84 87.99 471.43
12.24 44.92 1023.74 88.21 473.26
19.21 58.49 1011.7 91.29 452.82
29.74 70.32 1008.1 52.72 432.69
23.28 60.84 1017.91 67.5 444.13
8.02 41.92 1029.8 92.05 467.21
22.47 48.6 1002.33 63.23 445.98
27.51 73.77 1002.42 90.88 436.91
17.51 44.9 1009.05 74.91 455.01
23.22 66.56 1002.47 85.39 437.11
11.73 40.64 1020.68 96.98 477.06
21.19 67.71 1006.65 56.28 441.71
5.48 40.07 1019.63 65.62 495.76
24.26 66.44 1011.33 55.32 445.63
12.32 41.62 1012.88 88.88 464.72
31.26 68.94 1005.94 39.49 438.03
32.09 72.86 1003.47 54.59 434.78
24.98 60.32 1015.63 57.19 444.67
27.48 61.41 1012.2 45.06 452.24
21.04 45.09 1014.19 40.62 450.92
27.75 70.4 1006.65 90.21 436.53
22.79 71.77 1005.75 90.91 435.53
24.22 68.51 1013.23 74.96 440.01
27.06 64.45 1008.72 54.21 443.1
29.25 71.94 1007.18 63.62 427.49
26.86 68.08 1012.99 50.04 436.25
29.64 67.79 1009.99 51.23 440.74
19.92 63.31 1015.02 82.71 443.54
18.5 51.43 1010.82 92.04 459.42
23.71 60.23 1009.76 90.67 439.66
14.39 44.84 1023.55 91.14 464.15
19.3 56.65 1020.55 70.43 459.1
24.65 52.36 1014.76 66.63 455.68
13.5 45.51 1015.33 86.95 469.08
9.82 41.26 1007.71 96.69 478.02
18.4 44.06 1017.36 70.88 456.8
28.12 44.89 1009.18 47.14 441.13
17.15 43.69 1017.05 63.36 463.88
30.69 73.67 1006.14 60.58 430.45
28.82 65.71 1014.24 54.3 449.18
21.3 48.92 1010.92 65.09 447.89
30.58 70.04 1010.4 48.16 431.59
21.17 52.3 1009.36 81.51 447.5
9.87 41.82 1033.04 68.57 475.58
22.18 59.8 1016.77 73.16 453.24
24.39 63.21 1012.59 80.88 446.4
10.73 44.92 1025.1 85.4 476.81
9.38 40.46 1019.29 75.77 474.1
20.27 57.76 1016.66 75.76 450.71
24.82 66.48 1006.4 70.21 433.62
16.55 41.66 1011.45 55.53 465.14
20.73 59.87 1019.08 80.48 445.18
9.51 39.22 1015.3 72.41 474.12
8.63 43.79 1016.08 83.25 483.91
6.48 40.27 1010.55 82.12 486.68
14.95 43.52 1022.43 94.75 464.98
5.76 45.87 1010.83 95.79 481.4
10.94 39.04 1021.81 86.02 479.2
15.87 41.16 1005.85 78.29 463.86
12.42 38.25 1012.76 82.23 472.3
29.12 58.84 1001.31 52.86 446.51
29.12 51.43 1005.93 60.66 437.71
19.08 41.1 1001.96 62.77 458.94
31.06 67.17 1007.62 65.54 437.91
5.72 39.33 1009.96 95.4 490.76
26.52 65.06 1013.4 51.78 439.66
13.84 44.9 1007.58 63.62 463.27
13.03 39.52 1016.68 83.09 473.99
25.94 66.49 1012.83 61.81 433.38
16.64 53.82 1015.13 68.24 459.01
14.13 40.75 1016.05 72.41 471.44
13.65 39.28 1012.97 79.64 471.91
14.5 44.47 1028.2 66.95 465.15
19.8 51.19 1008.25 91.98 446.66
25.2 63.76 1009.78 64.96 438.15
20.66 51.19 1008.81 88.93 447.14
12.07 43.71 1025.53 85.62 472.32
25.64 70.72 1010.16 84.0 441.68
23.33 72.99 1009.33 89.41 440.04
29.41 64.05 1009.82 67.4 444.82
16.6 53.16 1014.5 76.75 457.26
27.53 72.58 1009.13 89.06 428.83
20.62 43.43 1009.93 64.02 449.07
26.02 71.94 1009.38 64.12 435.21
12.75 44.2 1017.59 81.22 471.03
12.87 48.04 1012.47 100.13 465.56
25.77 62.96 1019.86 58.07 442.83
14.84 41.48 1017.26 63.42 460.3
7.41 40.71 1023.07 83.32 474.25
8.87 41.82 1033.3 74.28 477.97
9.69 40.46 1019.1 71.91 472.16
16.17 46.97 1014.22 85.8 456.08
26.24 49.82 1014.9 55.58 452.41
13.78 43.22 1011.31 69.7 463.71
26.3 67.07 1006.26 63.79 433.72
17.37 57.76 1016.0 86.59 456.4
23.6 48.98 1015.41 48.28 448.43
8.3 36.08 1020.63 80.42 481.6
18.86 42.18 1001.16 98.58 457.07
22.12 49.39 1019.8 72.83 451.0
28.41 75.6 1018.48 56.07 440.28
29.42 71.32 1002.26 67.13 437.47
18.61 67.71 1004.07 84.49 443.57
27.57 69.84 1004.91 68.37 426.6
12.83 41.5 1013.12 86.07 470.87
9.64 39.85 1012.9 83.82 478.37
19.13 58.66 1013.32 74.86 453.92
15.92 40.56 1020.79 53.52 470.22
24.64 72.24 1011.37 80.61 434.54
27.62 63.9 1013.11 43.56 442.89
8.9 36.24 1013.29 89.35 479.03
9.55 43.99 1020.5 97.28 476.06
10.57 36.71 1022.62 80.49 473.88
19.8 57.25 1010.84 88.9 451.75
25.63 56.85 1012.68 49.7 439.2
24.7 58.46 1015.58 68.64 439.7
15.26 46.18 1013.68 98.58 463.6
20.06 52.84 1004.21 82.12 447.47
19.84 56.89 1013.23 78.32 447.92
11.49 44.63 1020.44 86.04 471.08
23.74 72.43 1007.99 91.36 437.55
22.62 51.3 1012.36 81.02 448.27
29.53 72.39 998.47 76.05 431.69
21.32 48.14 1016.57 71.81 449.09
20.3 58.46 1015.93 82.13 448.79
16.97 44.92 1025.21 74.27 460.21
12.07 41.17 1013.54 71.32 479.28
7.46 41.82 1032.67 74.59 483.11
19.2 54.2 1011.46 84.44 450.75
28.64 66.54 1010.43 43.39 437.97
13.56 41.48 1008.53 87.2 459.76
17.4 44.9 1020.5 77.11 457.75
14.08 40.1 1015.48 82.81 469.33
27.11 69.75 1009.74 85.67 433.28
20.92 70.02 1010.23 95.58 444.64
16.18 44.9 1021.3 74.46 463.1
15.57 44.68 1022.01 90.02 460.91
10.37 39.04 1023.95 81.93 479.35
19.6 59.21 1017.65 86.29 449.23
9.22 40.92 1021.83 85.43 474.51
27.76 72.99 1007.81 71.66 435.02
28.68 70.72 1009.43 71.33 435.45
20.95 48.14 1013.3 67.72 452.38
9.06 39.3 1019.73 84.23 480.41
9.21 39.72 1019.54 74.44 478.96
13.65 42.74 1026.58 71.48 468.87
31.79 76.2 1007.89 56.3 434.01
14.32 44.6 1013.85 68.13 466.36
26.28 75.23 1011.44 68.35 435.28
7.69 43.02 1014.51 85.23 486.46
14.44 40.1 1015.51 79.78 468.19
9.19 41.01 1022.14 98.98 468.37
13.35 41.39 1019.17 72.87 474.19
23.04 74.22 1009.52 90.93 440.32
4.83 38.44 1015.35 72.94 485.32
17.29 42.86 1014.38 72.3 464.27
8.73 36.18 1013.66 77.74 479.25
26.21 70.32 1007.0 78.29 430.4
23.72 58.62 1016.65 69.1 447.49
29.27 64.69 1006.85 55.79 438.23
10.4 40.43 1025.46 75.09 492.09
12.19 40.75 1015.13 88.98 475.36
20.4 54.9 1016.68 64.26 452.56
34.3 74.67 1015.98 25.89 427.84
27.56 68.08 1010.8 59.18 433.95
30.9 70.8 1008.48 67.48 435.27
14.85 58.59 1014.04 89.85 454.62
16.42 40.56 1020.36 50.62 472.17
16.45 63.31 1015.96 83.97 452.42
10.14 42.02 1003.19 96.51 472.17
9.53 41.44 1018.01 80.09 481.83
17.01 49.15 1021.83 84.02 458.78
23.94 62.08 1022.47 61.97 447.5
15.95 49.25 1019.04 88.51 463.4
11.15 41.26 1022.67 81.83 473.57
25.56 70.32 1009.07 90.63 433.72
27.16 66.44 1011.2 73.37 431.85
26.71 77.95 1012.13 77.5 433.47
29.56 74.22 1007.45 57.46 432.84
31.19 70.94 1007.29 51.91 436.6
6.86 41.17 1020.12 79.14 490.23
12.36 41.74 1020.58 69.24 477.16
32.82 68.31 1010.44 41.85 441.06
25.3 70.98 1007.22 95.1 440.86
8.71 41.82 1033.08 74.53 477.94
13.34 40.8 1026.56 64.85 474.47
14.2 43.02 1012.18 57.07 470.67
23.74 65.34 1013.7 62.9 447.31
16.9 44.88 1018.14 72.21 466.8
28.54 71.94 1007.4 65.99 430.91
30.15 69.88 1007.2 73.67 434.75
14.33 42.86 1010.82 88.59 469.52
25.57 59.43 1008.88 61.19 438.9
30.55 70.04 1010.51 49.37 429.56
28.04 74.33 1013.53 48.65 432.92
26.39 49.16 1005.68 56.18 442.87
15.3 41.76 1022.57 71.56 466.59
6.03 41.14 1028.04 87.46 479.61
13.49 44.63 1019.12 70.02 471.08
27.67 59.14 1016.51 61.2 433.37
24.19 65.48 1018.8 60.54 443.92
24.44 59.14 1016.74 71.82 443.5
29.86 64.79 1017.37 44.8 439.89
30.2 69.59 1008.9 67.32 434.66
7.99 41.38 1021.95 78.77 487.57
9.93 41.62 1013.76 96.02 464.64
11.03 42.32 1017.26 90.56 470.92
22.34 63.73 1014.37 83.19 444.39
25.33 48.6 1002.54 68.45 442.48
18.87 52.08 1005.25 99.19 449.61
25.97 69.34 1009.43 88.11 435.02
16.58 43.99 1021.81 79.29 458.67
14.35 46.18 1016.63 87.76 461.74
25.06 62.39 1008.09 82.56 438.31
13.85 48.92 1011.68 79.24 462.38
16.09 44.2 1019.39 67.24 460.56
26.34 59.21 1013.37 58.98 439.22
23.01 58.79 1009.71 84.22 444.64
26.39 71.25 999.8 89.12 430.34
31.32 71.29 1008.37 50.07 430.46
16.64 45.87 1009.02 98.86 456.79
13.42 41.23 994.17 95.79 468.82
20.06 44.9 1008.79 70.06 448.51
14.8 44.71 1014.67 41.71 470.77
12.59 41.14 1025.79 86.55 465.74
26.7 66.56 1005.31 71.97 430.21
19.78 50.32 1008.62 96.4 449.23
15.17 49.15 1021.91 91.73 461.89
21.71 61.45 1010.97 91.62 445.72
19.09 39.39 1013.36 59.14 466.13
19.76 51.19 1008.38 92.56 448.71
14.68 41.23 998.43 83.71 469.25
21.3 66.86 1013.04 55.43 450.56
16.73 39.64 1008.94 74.91 464.46
12.26 41.5 1014.87 89.41 471.13
14.77 48.06 1010.92 69.81 461.52
18.26 59.15 1012.04 86.01 451.09
27.1 79.74 1005.43 86.05 431.51
14.72 40.83 1009.65 80.98 469.8
26.3 51.43 1012.05 63.62 442.28
16.48 48.92 1011.84 64.16 458.67
17.99 43.79 1016.13 75.63 462.4
20.34 59.8 1015.18 80.21 453.54
25.53 62.96 1019.81 59.7 444.38
31.59 58.9 1003.39 47.6 440.52
30.8 69.14 1007.68 63.78 433.62
10.75 45.0 1023.68 89.37 481.96
19.3 44.9 1008.89 70.55 452.75
4.71 39.42 1026.4 84.42 481.28
23.1 66.05 1020.28 80.62 439.03
32.63 73.88 1005.64 52.56 435.75
26.63 74.16 1009.72 83.26 436.03
24.35 58.49 1011.03 70.64 445.6
15.11 56.03 1020.27 89.95 462.65
29.1 50.05 1005.87 51.53 438.66
21.24 50.32 1008.54 84.83 447.32
6.16 39.48 1004.85 59.68 484.55
7.36 41.01 1024.9 97.88 476.8
10.44 39.04 1023.99 85.03 480.34
26.76 48.41 1010.53 47.38 440.63
16.79 44.6 1014.27 48.08 459.48
10.76 40.43 1025.98 79.65 490.78
6.07 38.91 1019.25 83.39 483.56
27.33 73.18 1012.26 82.18 429.38
27.15 59.21 1013.49 51.71 440.27
22.35 51.43 1011.34 77.33 445.34
21.82 65.27 1013.86 72.81 447.43
21.11 69.94 1004.37 84.26 439.91
19.95 50.59 1016.11 73.23 459.27
7.45 39.61 1017.88 79.73 478.89
15.36 41.66 1012.41 62.32 466.7
15.65 43.5 1021.39 78.58 463.5
25.31 74.33 1015.04 79.88 436.21
25.88 63.47 1011.95 65.87 443.94
24.6 63.94 1012.87 80.28 439.63
22.58 41.54 1013.21 71.33 460.95
19.69 59.14 1015.99 70.33 448.69
25.85 75.08 1006.24 57.73 444.63
10.06 37.83 1005.49 99.46 473.51
18.59 39.54 1008.56 68.61 462.56
18.27 50.16 1011.07 95.91 451.76
8.85 40.43 1025.68 80.42 491.81
30.04 68.08 1011.04 51.01 429.52
26.06 49.02 1007.59 74.08 437.9
14.8 38.73 1003.18 80.73 467.54
23.93 64.45 1015.35 54.71 449.97
23.72 66.48 1003.61 73.75 436.62
11.44 40.55 1023.37 88.43 477.68
20.28 63.86 1016.04 74.66 447.26
27.9 63.13 1011.8 70.04 439.76
24.74 59.39 1015.23 74.64 437.49
14.8 58.2 1018.29 85.11 455.14
8.22 41.03 1021.76 82.97 485.5
27.56 66.93 1016.81 55.59 444.1
32.07 70.94 1006.91 49.9 432.33
9.53 44.03 1008.87 89.99 471.23
13.61 42.34 1017.93 91.61 463.89
22.2 51.19 1009.2 82.95 445.54
21.36 59.54 1007.99 92.62 446.09
23.25 63.86 1017.82 59.64 445.12
23.5 59.21 1018.29 63.0 443.31
8.46 39.66 1015.14 85.38 484.16
8.19 40.69 1019.86 85.23 477.76
30.67 71.29 1008.36 52.08 430.28
32.48 62.04 1010.39 38.05 446.48
8.99 36.66 1028.11 71.98 481.03
13.77 47.83 1007.41 90.66 466.07
19.05 67.32 1013.2 83.14 447.47
21.19 55.5 1019.83 65.22 455.93
10.12 40.0 1021.15 91.67 479.62
24.93 47.01 1014.28 66.04 455.06
8.47 40.46 1019.87 78.19 475.06
24.52 56.85 1012.59 54.47 438.89
28.55 69.84 1003.38 67.26 432.7
20.58 50.9 1011.89 72.56 452.6
18.31 46.21 1010.46 82.15 451.75
27.18 71.06 1008.16 86.32 430.66
4.43 38.91 1019.04 88.17 491.9
26.02 74.78 1010.04 72.78 439.82
15.75 39.0 1015.91 69.58 460.73
22.99 60.95 1015.14 69.86 449.7
25.52 59.15 1013.88 65.37 439.42
27.04 65.06 1013.33 52.37 439.84
6.42 35.57 1025.58 79.63 485.86
17.04 40.12 1011.81 83.14 458.1
10.79 39.82 1012.89 88.25 479.92
20.41 56.03 1019.94 55.85 458.29
7.36 40.07 1017.29 52.55 489.45
28.08 73.42 1012.17 62.74 434.0
24.74 69.13 1010.69 90.08 431.24
28.32 47.93 1003.26 54.5 439.5
16.71 40.56 1019.48 49.88 467.46
30.7 71.58 1010.0 48.96 429.27
18.42 58.95 1016.95 86.77 452.1
10.62 42.02 999.83 96.66 472.41
22.18 69.05 1002.75 70.84 442.14
22.38 49.3 1003.56 83.83 441.0
13.94 41.58 1020.76 68.22 463.07
21.24 60.84 1017.99 82.22 445.71
6.76 39.81 1017.11 87.9 483.16
26.73 68.84 1010.75 66.83 440.45
7.24 38.06 1020.6 85.36 481.83
10.84 40.62 1015.53 60.9 467.6
19.32 52.84 1004.29 83.51 450.88
29.0 69.13 1001.22 52.96 425.5
23.38 54.42 1013.95 73.02 451.87
31.17 69.51 1010.51 43.11 428.94
26.17 48.6 1002.59 61.41 439.86
30.9 73.42 1011.21 65.32 433.44
24.92 73.68 1015.12 93.68 438.23
32.77 71.32 1007.68 42.39 436.95
14.37 40.56 1021.67 68.18 470.19
8.36 40.22 1011.6 89.18 484.66
31.45 68.27 1007.56 64.79 430.81
31.6 73.17 1010.05 43.48 433.37
17.9 48.98 1014.17 80.4 453.02
20.35 50.9 1012.6 72.43 453.5
16.21 41.23 995.88 80.0 463.09
19.36 44.6 1016.25 45.65 464.56
21.04 65.46 1017.22 63.02 452.12
14.05 40.69 1015.66 74.39 470.9
23.48 64.15 1021.08 57.77 450.89
21.91 63.76 1009.85 76.8 445.04
24.42 63.07 1011.49 67.39 444.72
14.26 40.92 1022.07 73.96 460.38
21.38 58.33 1013.05 72.75 446.8
15.71 44.06 1018.34 71.69 465.05
5.78 40.62 1016.55 84.98 484.13
6.77 39.81 1017.01 87.68 488.27
23.84 49.21 1013.85 50.36 447.09
21.17 58.16 1017.16 68.11 452.02
19.94 58.96 1014.16 66.27 455.55
8.73 41.92 1029.41 89.72 480.99
16.39 41.67 1012.96 61.07 467.68
powerPlantDF.count() // count the number of rows in DF
res34: Long = 9568

We need to create a temporary view of the DataFrame as a table before being able to access it via SQL.

powerPlantDF.createOrReplaceTempView("power_plant_table") // Shift+Enter
spark.catalog.listTables.where($"name" startsWith "power").show()
+-----------------+--------+-----------+---------+-----------+
|             name|database|description|tableType|isTemporary|
+-----------------+--------+-----------+---------+-----------+
|power_plant_table|    null|       null|TEMPORARY|       true|
+-----------------+--------+-----------+---------+-----------+

Note that table names are in lower-case only!

You Try!

//sqlContext // uncomment and put . after sqlContext and hit Tab to see what methods are available
//sqlContext.dropTempTable("power_plant_table") // uncomment and Ctrl+Enter if you want to remove the table!

The following SQL statement simply selects all the columns (due to *) from powerPlantTable.

-- Ctrl+Enter to query the rows via SQL
SELECT * FROM power_plant_table
AT V AP RH PE
14.96 41.76 1024.07 73.17 463.26
25.18 62.96 1020.04 59.08 444.37
5.11 39.4 1012.16 92.14 488.56
20.86 57.32 1010.24 76.64 446.48
10.82 37.5 1009.23 96.62 473.9
26.27 59.44 1012.23 58.77 443.67
15.89 43.96 1014.02 75.24 467.35
9.48 44.71 1019.12 66.43 478.42
14.64 45.0 1021.78 41.25 475.98
11.74 43.56 1015.14 70.72 477.5
17.99 43.72 1008.64 75.04 453.02
20.14 46.93 1014.66 64.22 453.99
24.34 73.5 1011.31 84.15 440.29
25.71 58.59 1012.77 61.83 451.28
26.19 69.34 1009.48 87.59 433.99
21.42 43.79 1015.76 43.08 462.19
18.21 45.0 1022.86 48.84 467.54
11.04 41.74 1022.6 77.51 477.2
14.45 52.75 1023.97 63.59 459.85
13.97 38.47 1015.15 55.28 464.3
17.76 42.42 1009.09 66.26 468.27
5.41 40.07 1019.16 64.77 495.24
7.76 42.28 1008.52 83.31 483.8
27.23 63.9 1014.3 47.19 443.61
27.36 48.6 1003.18 54.93 436.06
27.47 70.72 1009.97 74.62 443.25
14.6 39.31 1011.11 72.52 464.16
7.91 39.96 1023.57 88.44 475.52
5.81 35.79 1012.14 92.28 484.41
30.53 65.18 1012.69 41.85 437.89
23.87 63.94 1019.02 44.28 445.11
26.09 58.41 1013.64 64.58 438.86
29.27 66.85 1011.11 63.25 440.98
27.38 74.16 1010.08 78.61 436.65
24.81 63.94 1018.76 44.51 444.26
12.75 44.03 1007.29 89.46 465.86
24.66 63.73 1011.4 74.52 444.37
16.38 47.45 1010.08 88.86 450.69
13.91 39.35 1014.69 75.51 469.02
23.18 51.3 1012.04 78.64 448.86
22.47 47.45 1007.62 76.65 447.14
13.39 44.85 1017.24 80.44 469.18
9.28 41.54 1018.33 79.89 482.8
11.82 42.86 1014.12 88.28 476.7
10.27 40.64 1020.63 84.6 474.99
22.92 63.94 1019.28 42.69 444.22
16.0 37.87 1020.24 78.41 461.33
21.22 43.43 1010.96 61.07 448.06
13.46 44.71 1014.51 50.0 474.6
9.39 40.11 1029.14 77.29 473.05
31.07 73.5 1010.58 43.66 432.06
12.82 38.62 1018.71 83.8 467.41
32.57 78.92 1011.6 66.47 430.12
8.11 42.18 1014.82 93.09 473.62
13.92 39.39 1012.94 80.52 471.81
23.04 59.43 1010.23 68.99 442.99
27.31 64.44 1014.65 57.27 442.77
5.91 39.33 1010.18 95.53 491.49
25.26 61.08 1013.68 71.72 447.46
27.97 58.84 1002.25 57.88 446.11
26.08 52.3 1007.03 63.34 442.44
29.01 65.71 1013.61 48.07 446.22
12.18 40.1 1016.67 91.87 471.49
13.76 45.87 1008.89 87.27 463.5
25.5 58.79 1016.02 64.4 440.01
28.26 65.34 1014.56 43.4 441.03
21.39 62.96 1019.49 72.24 452.68
7.26 40.69 1020.43 90.22 474.91
10.54 34.03 1018.71 74.0 478.77
27.71 74.34 998.14 71.85 434.2
23.11 68.3 1017.83 86.62 437.91
7.51 41.01 1024.61 97.41 477.61
26.46 74.67 1016.65 84.44 431.65
29.34 74.34 998.58 81.55 430.57
10.32 42.28 1008.82 75.66 481.09
22.74 61.02 1009.56 79.41 445.56
13.48 39.85 1012.71 58.91 475.74
25.52 69.75 1010.36 90.06 435.12
21.58 67.25 1017.39 79.0 446.15
27.66 76.86 1001.31 69.47 436.64
26.96 69.45 1013.89 51.47 436.69
12.29 42.18 1016.53 83.13 468.75
15.86 43.02 1012.18 40.33 466.6
13.87 45.08 1024.42 81.69 465.48
24.09 73.68 1014.93 94.55 441.34
20.45 69.45 1012.53 91.81 441.83
15.07 39.3 1019.0 63.62 464.7
32.72 69.75 1009.6 49.35 437.99
18.23 58.96 1015.55 69.61 459.12
35.56 68.94 1006.56 38.75 429.69
18.36 51.43 1010.57 90.17 459.8
26.35 64.05 1009.81 81.24 433.63
25.92 60.95 1014.62 48.46 442.84
8.01 41.66 1014.49 76.72 485.13
19.63 52.72 1025.09 51.16 459.12
20.02 67.32 1012.05 76.34 445.31
10.08 40.72 1022.7 67.3 480.8
27.23 66.48 1005.23 52.38 432.55
23.37 63.77 1013.42 76.44 443.86
18.74 59.21 1018.3 91.55 449.77
14.81 43.69 1017.19 71.9 470.71
23.1 51.3 1011.93 80.05 452.17
10.72 41.38 1021.6 63.77 478.29
29.46 71.94 1006.96 62.26 428.54
8.1 40.64 1020.66 89.04 478.27
27.29 62.66 1007.63 58.02 439.58
17.1 49.69 1005.53 81.82 457.32
11.49 44.2 1018.79 91.14 475.51
23.69 65.59 1010.85 88.92 439.66
13.51 40.89 1011.03 84.83 471.99
9.64 39.35 1015.1 91.76 479.81
25.65 78.92 1010.83 86.56 434.78
21.59 61.87 1011.18 57.21 446.58
27.98 58.33 1013.92 54.25 437.76
18.8 39.72 1001.24 63.8 459.36
18.28 44.71 1016.99 33.71 462.28
13.55 43.48 1016.08 67.25 464.33
22.99 46.21 1010.71 60.11 444.36
23.94 59.39 1014.32 74.55 438.64
13.74 34.03 1018.69 67.34 470.49
21.3 41.1 1001.86 42.75 455.13
27.54 66.93 1017.06 55.2 450.22
24.81 63.73 1009.34 83.61 440.43
4.97 42.85 1014.02 88.78 482.98
15.22 50.88 1014.19 100.12 460.44
23.88 54.2 1012.81 64.52 444.97
33.01 68.67 1005.2 51.41 433.94
25.98 73.18 1012.28 85.78 439.73
28.18 73.88 1005.89 75.41 434.48
21.67 60.84 1017.93 81.63 442.33
17.67 45.09 1014.26 51.92 457.67
21.37 57.76 1018.8 70.12 454.66
28.69 67.25 1017.71 53.83 432.21
16.61 43.77 1012.25 77.23 457.66
27.91 63.76 1010.27 65.67 435.21
20.97 47.43 1007.64 71.18 448.22
10.8 41.66 1013.79 81.96 475.51
20.61 62.91 1013.24 79.54 446.53
25.45 57.32 1011.7 47.09 441.3
30.16 69.34 1007.67 57.69 433.54
4.99 39.04 1020.45 78.89 472.52
10.51 44.78 1012.59 85.29 474.77
33.79 69.05 1001.62 40.13 435.1
21.34 59.8 1016.92 77.06 450.74
23.4 65.06 1014.32 67.38 442.7
32.21 68.14 1003.34 62.44 426.56
14.26 42.32 1016.0 77.43 463.71
27.71 66.93 1016.85 58.77 447.06
21.95 57.76 1018.02 67.72 452.27
25.76 63.94 1018.49 42.14 445.78
23.68 68.3 1017.93 84.16 438.65
8.28 40.77 1011.55 89.79 480.15
23.44 62.52 1016.46 67.21 447.19
25.32 48.41 1008.47 72.14 443.04
3.94 39.9 1008.06 97.49 488.81
17.3 57.76 1016.26 87.74 455.75
18.2 49.39 1018.83 96.3 455.86
21.43 46.97 1013.94 61.25 457.68
11.16 40.05 1014.95 88.38 479.11
30.38 74.16 1007.44 74.77 432.84
23.36 62.52 1016.18 68.18 448.37
21.69 47.45 1007.56 77.2 447.06
23.62 49.21 1014.1 49.54 443.53
21.87 61.45 1011.13 92.22 445.21
29.25 66.51 1015.53 33.65 441.7
20.03 66.86 1013.05 64.59 450.93
18.14 49.78 1002.95 100.09 451.44
24.23 56.89 1012.32 68.04 441.29
18.11 44.85 1014.48 48.94 458.85
6.57 43.65 1018.24 74.47 481.46
12.56 43.41 1016.93 81.02 467.19
13.4 41.58 1020.5 71.17 461.54
27.1 52.84 1006.28 53.85 439.08
14.28 42.74 1028.79 70.67 467.22
16.29 44.34 1019.49 59.36 468.8
31.24 71.98 1004.66 57.17 426.93
10.57 37.73 1024.36 70.29 474.65
13.8 44.21 1022.93 83.37 468.97
25.3 71.58 1010.18 87.36 433.97
18.06 50.16 1009.52 100.09 450.53
25.42 59.04 1011.98 68.78 444.51
15.07 40.69 1015.29 70.98 469.03
11.75 71.14 1019.36 75.68 466.56
20.23 52.05 1012.15 47.49 457.57
27.31 59.54 1006.24 71.99 440.13
28.57 69.84 1003.57 66.55 433.24
17.9 43.72 1008.64 74.73 452.55
23.83 71.37 1002.04 64.78 443.29
27.92 74.99 1005.47 75.13 431.76
17.34 44.78 1007.81 56.38 454.97
17.94 63.07 1012.42 94.35 456.7
6.4 39.9 1007.75 86.55 486.03
11.78 39.96 1011.37 82.95 472.79
20.28 57.25 1010.12 88.42 452.03
21.04 54.2 1012.26 85.61 443.41
25.11 67.32 1014.49 58.39 441.93
30.28 70.98 1007.51 74.28 432.64
8.14 36.24 1013.15 87.85 480.25
16.86 39.63 1004.47 83.5 466.68
6.25 40.07 1020.19 65.24 494.39
22.35 54.42 1012.46 75.01 454.72
17.98 56.85 1012.28 84.52 448.71
21.19 42.48 1013.43 80.52 469.76
20.94 44.89 1009.64 75.14 450.71
24.23 58.79 1009.8 75.75 444.01
19.18 58.2 1017.46 76.72 453.2
20.88 57.85 1012.39 85.47 450.87
23.67 63.86 1019.67 57.95 441.73
14.12 39.52 1018.41 78.32 465.09
25.23 64.63 1020.59 52.2 447.28
6.54 39.33 1011.54 93.69 491.16
20.08 62.52 1017.99 75.74 450.98
24.67 63.56 1013.75 67.56 446.3
27.82 79.74 1008.37 69.46 436.48
15.55 42.03 1017.41 74.58 460.84
24.26 69.51 1013.43 53.23 442.56
13.45 41.49 1020.19 88.72 467.3
11.06 40.64 1021.47 96.16 479.13
24.91 52.3 1008.72 68.26 441.15
22.39 59.04 1011.78 86.39 445.52
11.95 40.69 1015.62 85.34 475.4
14.85 40.69 1014.91 72.64 469.3
10.11 41.62 1017.17 97.82 463.57
23.67 68.67 1006.71 77.22 445.32
16.14 44.21 1020.36 80.59 461.03
15.11 43.13 1014.99 46.91 466.74
24.14 59.87 1018.47 57.76 444.04
30.08 67.25 1017.6 53.09 434.01
14.77 44.9 1020.5 84.31 465.23
27.6 69.34 1009.63 71.58 440.6
13.89 44.84 1023.66 92.97 466.74
26.85 75.6 1017.43 74.55 433.48
12.41 40.96 1023.36 78.96 473.59
13.08 41.74 1020.75 64.44 474.81
18.93 44.06 1017.58 68.23 454.75
20.5 49.69 1009.6 70.81 452.94
30.72 69.13 1009.94 61.66 435.83
7.55 39.22 1014.53 77.76 482.19
13.49 44.47 1030.46 69.49 466.66
15.62 40.12 1013.03 96.26 462.59
24.8 64.63 1020.69 55.74 447.82
10.03 41.62 1014.55 95.61 462.73
22.43 63.21 1012.06 84.75 447.98
14.95 39.31 1009.15 75.3 462.72
24.78 58.46 1016.82 67.5 442.42
23.2 48.41 1008.64 80.92 444.69
14.01 39.0 1016.73 79.23 466.7
19.4 64.63 1020.38 81.1 453.84
30.15 67.32 1013.83 32.8 436.92
6.91 36.08 1021.82 84.31 486.37
29.04 60.07 1015.42 46.15 440.43
26.02 63.07 1010.94 53.96 446.82
5.89 39.48 1005.11 59.83 484.91
26.52 71.64 1008.27 75.3 437.76
28.53 68.08 1013.27 42.53 438.91
16.59 39.54 1007.97 70.58 464.19
22.95 67.79 1009.89 91.69 442.19
23.96 47.43 1008.38 63.55 446.86
17.48 44.2 1018.89 61.51 457.15
6.69 43.65 1020.14 69.55 482.57
10.25 41.26 1007.44 98.08 476.03
28.87 72.58 1008.69 79.34 428.89
12.04 40.23 1018.07 81.28 472.7
22.58 52.3 1009.04 78.99 445.6
15.12 52.05 1014.63 80.38 464.78
25.48 58.95 1017.02 51.16 440.42
27.87 70.79 1003.96 72.17 428.41
23.72 70.47 1010.65 75.39 438.5
25.0 59.43 1007.84 68.91 438.28
8.42 40.64 1022.35 96.38 476.29
22.46 58.49 1011.5 70.54 448.46
29.92 57.19 1008.62 45.8 438.99
11.68 39.22 1017.9 57.95 471.8
14.04 42.44 1012.74 81.89 471.81
19.86 59.14 1016.12 69.32 449.82
25.99 68.08 1013.13 59.14 442.14
23.42 58.79 1009.74 81.54 441.46
10.6 40.22 1011.37 85.81 477.62
20.97 61.87 1011.45 65.41 446.76
14.14 39.82 1012.46 81.15 472.52
8.56 40.71 1021.27 95.87 471.58
24.86 72.39 1001.15 90.24 440.85
29.0 77.54 1011.33 75.13 431.37
27.59 71.97 1008.64 88.22 437.33
10.45 40.71 1015.68 81.48 469.22
8.51 40.78 1023.51 89.84 471.11
29.82 66.51 1010.98 43.57 439.17
22.56 62.26 1012.11 63.16 445.33
11.38 39.22 1018.62 57.14 473.71
20.25 57.76 1016.28 77.76 452.66
22.42 59.43 1007.12 90.56 440.99
14.85 38.91 1014.48 60.98 467.42
25.62 58.82 1010.02 70.31 444.14
19.85 56.53 1020.57 74.05 457.17
13.67 54.3 1015.92 75.42 467.87
24.39 70.72 1009.78 82.25 442.04
16.07 44.58 1019.52 67.95 471.36
11.6 39.1 1009.81 100.09 460.7
31.38 70.83 1010.35 47.28 431.33
29.91 76.86 998.59 72.41 432.6
19.67 59.39 1014.07 77.67 447.61
27.18 64.79 1016.27 63.7 443.87
21.39 52.3 1009.2 79.77 446.87
10.45 41.01 1020.57 93.84 465.74
19.46 56.89 1014.02 84.95 447.86
23.55 62.96 1020.16 70.16 447.65
23.35 63.47 1011.78 84.24 437.87
9.26 41.66 1016.87 73.32 483.51
10.3 41.46 1018.21 86.17 479.65
20.94 58.16 1016.88 65.43 455.16
23.13 71.25 1002.49 94.59 431.91
12.77 41.5 1014.13 86.8 470.68
28.29 69.13 1009.29 58.18 429.28
19.13 59.21 1018.32 89.66 450.81
24.44 73.5 1011.49 87.39 437.73
20.32 44.6 1015.16 36.35 460.21
20.54 69.05 1001.6 79.62 442.86
12.16 45.0 1021.51 50.52 482.99
28.09 65.27 1013.27 51.96 440.0
9.25 41.82 1033.25 74.73 478.48
21.75 49.82 1015.01 78.33 455.28
23.7 66.56 1002.07 85.19 436.94
16.22 37.87 1022.36 83.13 461.06
24.75 69.45 1013.97 53.49 438.28
10.48 39.58 1011.81 88.86 472.61
29.53 70.79 1003.7 60.89 426.85
12.59 39.72 1017.76 61.14 470.18
23.5 54.42 1012.31 68.29 455.38
29.01 66.56 1006.44 57.62 428.32
9.75 42.49 1010.57 83.63 480.35
19.55 56.53 1020.2 78.1 455.56
21.05 58.33 1013.14 66.34 447.66
24.72 68.67 1006.74 79.02 443.06
21.19 58.86 1014.19 68.96 452.43
10.77 41.54 1019.94 71.13 477.81
28.68 73.77 1004.72 87.01 431.66
29.87 73.91 1004.53 74.3 431.8
22.99 68.67 1006.65 77.62 446.67
24.66 60.29 1018.0 59.56 445.26
32.63 69.89 1013.85 41.66 425.72
31.38 72.29 1008.73 73.27 430.58
23.87 60.27 1018.94 77.16 439.86
25.6 59.15 1013.31 67.02 441.11
27.62 71.14 1011.6 52.8 434.72
30.1 67.45 1014.23 39.04 434.01
12.19 41.17 1019.43 65.47 475.64
13.11 41.58 1020.43 74.32 460.44
28.29 68.67 1005.46 69.22 436.4
13.45 40.73 1018.7 93.88 461.03
10.98 41.54 1019.94 69.83 479.08
26.48 69.14 1009.31 84.11 435.76
13.07 45.51 1015.22 78.65 460.14
25.56 75.6 1017.37 69.31 442.2
22.68 50.78 1008.83 70.3 447.69
28.86 73.67 1006.65 68.23 431.15
22.7 63.56 1014.32 71.76 445.0
27.89 73.21 1001.32 85.88 431.59
13.78 44.47 1027.94 71.09 467.22
28.14 51.43 1012.16 52.67 445.33
11.8 45.09 1013.21 89.68 470.57
10.71 39.61 1018.72 73.66 473.77
24.54 60.29 1017.42 58.94 447.67
11.54 40.05 1014.78 87.05 474.29
29.47 71.32 1008.07 67.0 437.14
29.24 69.05 1003.12 43.18 432.56
14.51 41.79 1009.72 80.62 459.14
22.91 60.07 1016.03 59.72 446.19
27.02 71.77 1006.38 72.1 428.1
13.49 44.47 1030.18 69.15 468.46
30.24 66.75 1017.95 55.66 435.02
23.19 48.6 1002.38 61.19 445.52
17.73 40.55 1003.36 74.62 462.69
18.62 61.27 1019.26 73.35 455.75
12.85 40.0 1015.89 68.85 463.74
32.33 69.68 1011.95 39.89 439.79
25.09 58.95 1016.99 53.16 443.26
29.45 69.13 1009.3 52.97 432.04
16.91 43.96 1013.32 79.87 465.86
14.09 45.87 1009.05 84.09 465.6
10.73 25.36 1009.35 100.15 469.43
23.2 49.3 1003.4 79.77 440.75
8.21 38.91 1015.82 88.99 481.32
9.3 40.56 1022.64 76.14 479.87
16.97 39.16 1005.7 69.13 458.59
23.69 71.97 1009.62 93.03 438.62
25.13 59.44 1012.38 77.92 445.59
9.86 43.56 1015.13 74.89 481.87
11.33 41.5 1013.58 88.7 475.01
26.95 48.41 1008.53 62.94 436.54
15.0 40.66 1016.28 89.62 456.63
20.76 62.52 1015.63 81.04 451.69
14.29 39.59 1010.93 94.53 463.04
19.74 67.71 1007.68 64.02 446.1
26.68 59.92 1009.94 70.57 438.67
14.24 41.4 1019.7 70.32 466.88
21.98 48.41 1008.42 84.86 444.6
22.75 59.39 1015.4 81.41 440.26
8.34 40.96 1023.28 89.45 483.92
11.8 41.2 1017.18 82.71 475.19
8.81 44.68 1023.06 93.93 479.24
30.05 73.68 1014.95 70.6 434.92
16.01 65.46 1014.0 87.68 454.16
21.75 58.79 1012.42 87.58 447.58
13.94 41.26 1021.67 74.4 467.9
29.25 69.13 1010.27 67.35 426.29
22.33 45.87 1007.8 63.61 447.02
16.43 41.79 1005.47 76.89 455.85
11.5 40.22 1010.31 78.08 476.46
23.53 68.94 1007.53 69.17 437.48
21.86 49.21 1014.61 53.31 452.77
6.17 39.33 1012.57 93.32 491.54
30.19 64.79 1017.22 42.47 438.41
11.67 41.93 1019.81 82.58 476.1
15.34 36.99 1007.87 94.59 464.58
11.5 40.78 1023.91 86.31 467.74
25.53 57.17 1010.0 72.57 442.12
21.27 57.5 1014.53 80.76 453.34
28.37 69.13 1010.44 71.93 425.29
28.39 51.43 1011.74 47.54 449.63
13.78 45.78 1025.27 95.72 462.88
14.6 42.32 1015.71 77.03 464.67
5.1 35.57 1027.17 80.49 489.96
7.0 38.08 1020.27 77.67 482.38
26.3 77.95 1009.45 78.72 437.95
30.56 71.98 1004.74 58.77 429.2
21.09 46.63 1013.03 74.8 453.34
28.21 70.02 1010.58 51.34 442.47
15.84 49.69 1015.14 90.41 462.6
10.03 40.96 1024.57 91.1 478.79
20.37 52.05 1012.34 62.57 456.11
21.19 50.16 1005.81 84.27 450.33
33.73 69.88 1007.21 42.93 434.83
29.87 73.68 1015.1 40.96 433.43
19.62 62.96 1020.76 76.53 456.02
9.93 40.67 1018.08 69.74 485.23
9.43 37.14 1013.03 74.99 473.57
14.24 39.58 1011.17 70.45 469.94
12.97 49.83 1008.69 91.49 452.07
7.6 41.04 1021.82 88.97 475.32
8.39 36.24 1013.39 89.13 480.69
25.41 48.06 1013.12 46.52 444.01
18.43 56.03 1020.41 60.55 465.17
10.31 39.82 1012.87 88.71 480.61
11.29 41.5 1013.39 89.15 476.04
22.61 49.3 1003.51 83.02 441.76
29.34 71.98 1005.19 75.19 428.24
18.87 67.71 1004.0 87.35 444.77
13.21 45.87 1008.58 85.66 463.1
11.3 44.6 1018.19 91.66 470.5
29.23 72.99 1007.04 63.47 431.0
27.76 69.4 1004.27 72.25 430.68
29.26 67.17 1006.6 70.58 436.42
25.72 49.82 1016.19 60.1 452.33
23.43 63.94 1010.64 89.29 440.16
25.6 63.76 1010.18 67.43 435.75
22.3 44.57 1008.48 67.58 449.74
27.91 72.24 1010.74 70.8 430.73
30.35 77.17 1009.55 63.62 432.75
21.78 47.43 1007.88 66.68 446.79
7.19 41.39 1018.12 90.76 486.35
20.88 59.8 1015.66 75.34 453.18
24.19 50.23 1015.73 59.77 458.31
9.98 41.54 1019.7 80.79 480.26
23.47 51.3 1011.89 74.1 448.65
26.35 49.5 1012.67 41.34 458.41
29.89 64.69 1006.37 58.78 435.39
19.29 50.16 1010.49 97.78 450.21
17.48 43.14 1018.68 74.85 459.59
25.21 75.6 1017.19 69.84 445.84
23.3 48.78 1018.17 75.36 441.08
15.42 37.85 1009.89 85.8 467.33
21.44 63.09 1016.56 90.11 444.19
29.45 68.27 1007.96 61.63 432.96
29.69 47.93 1002.85 44.76 438.09
15.52 36.99 1006.86 89.7 467.9
11.47 43.67 1012.68 72.51 475.72
9.77 34.69 1027.72 74.98 477.51
22.6 69.84 1006.37 79.59 435.13
8.24 39.61 1017.99 78.42 477.9
17.01 44.2 1019.18 61.23 457.26
19.64 44.6 1015.88 47.56 467.53
10.61 41.58 1021.08 93.06 465.15
12.04 40.1 1014.42 89.65 474.28
29.19 65.71 1013.85 50.5 444.49
21.75 45.09 1014.15 44.84 452.84
23.66 77.54 1008.5 85.32 435.38
27.05 75.33 1003.88 82.94 433.57
29.63 69.71 1009.04 67.26 435.27
18.2 39.63 1005.35 79.05 468.49
32.22 70.8 1009.9 62.03 433.07
26.88 73.56 1004.85 94.36 430.63
29.05 65.74 1013.29 60.02 440.74
8.9 39.96 1026.31 95.46 474.49
18.93 48.6 1005.72 84.92 449.74
27.49 63.76 1010.09 62.8 436.73
23.1 70.79 1006.53 90.81 434.58
11.22 43.13 1017.24 80.9 473.93
31.97 79.74 1007.03 55.84 435.99
13.32 43.22 1009.45 75.3 466.83
31.68 68.24 1005.29 37.34 427.22
23.69 63.77 1013.39 79.5 444.07
13.83 41.49 1020.11 87.29 469.57
18.32 66.51 1015.18 81.5 459.89
11.05 40.71 1024.91 76.42 479.59
22.03 64.69 1007.21 75.75 440.92
10.23 41.46 1020.45 84.95 480.87
23.92 66.54 1009.93 62.37 441.9
29.38 69.68 1011.35 49.25 430.2
17.35 42.86 1014.62 74.16 465.16
9.81 44.45 1021.19 90.55 471.32
4.97 40.64 1020.91 94.28 485.43
5.15 40.07 1012.27 63.31 495.35
21.54 58.49 1010.85 78.9 449.12
7.94 42.02 1006.22 90.97 480.53
18.77 50.66 1014.89 87.34 457.07
21.69 69.94 1010.7 80.8 443.67
10.07 44.68 1023.44 90.95 477.52
13.83 39.64 1012.52 69.97 472.95
10.45 39.69 1003.92 89.45 472.54
11.56 40.71 1015.85 76.08 469.17
23.64 70.04 1011.09 83.35 435.21
10.48 40.22 1004.81 92.16 477.78
13.09 39.85 1012.86 58.42 475.89
10.67 40.23 1017.75 85.06 483.9
12.57 39.16 1016.53 88.91 476.2
14.45 43.34 1015.47 83.33 462.16
14.22 37.85 1011.24 88.49 471.05
6.97 41.26 1010.6 96.88 484.71
20.61 63.86 1015.43 73.86 446.34
14.67 42.28 1007.21 65.17 469.02
29.06 72.86 1004.23 69.41 432.12
14.38 40.1 1015.51 81.23 467.28
32.51 69.98 1013.29 54.07 429.66
11.79 45.09 1013.16 89.17 469.49
8.65 40.56 1023.23 78.85 485.87
9.75 40.81 1026.0 84.44 481.95
9.11 40.02 1031.1 83.02 479.03
23.39 69.13 1010.99 90.66 434.5
14.3 54.3 1015.16 75.29 464.9
17.49 63.94 1020.02 82.6 452.71
31.1 69.51 1010.84 45.4 429.74
19.77 56.65 1020.67 66.33 457.09
28.61 72.29 1011.61 45.33 446.77
13.52 41.48 1014.46 67.12 460.76
13.52 40.83 1008.31 84.14 471.95
17.57 46.21 1014.09 80.81 453.29
28.18 60.07 1016.34 49.13 441.61
14.29 46.18 1017.01 87.29 464.73
18.12 43.69 1016.91 52.95 464.68
31.27 73.91 1003.72 68.92 430.59
26.24 77.95 1014.19 85.21 438.01
7.44 41.04 1021.84 88.56 479.08
29.78 74.78 1009.28 55.09 436.39
23.37 65.46 1016.25 48.64 447.07
10.62 39.58 1011.9 87.85 479.91
5.84 43.02 1013.88 87.42 489.05
14.51 53.82 1016.46 62.75 463.17
11.31 42.02 1001.18 94.86 471.26
11.25 40.67 1011.64 63.54 480.49
9.18 39.42 1025.41 69.46 473.78
19.82 58.16 1016.76 74.66 455.5
24.77 58.41 1013.78 80.57 446.27
9.66 41.06 1021.21 84.7 482.2
21.96 59.8 1016.72 72.6 452.48
18.59 43.14 1011.92 52.63 464.48
24.75 69.89 1015.29 82.01 438.1
24.37 63.47 1012.77 75.22 445.6
29.6 67.79 1010.37 51.05 442.43
25.32 61.25 1011.56 80.1 436.67
16.15 41.85 1016.54 81.58 466.56
15.74 71.14 1019.65 65.94 457.29
5.97 36.25 1029.65 86.74 487.03
15.84 52.72 1026.45 62.57 464.93
14.84 44.63 1019.28 57.37 466.0
12.25 48.79 1017.44 88.91 469.52
27.38 70.04 1011.18 72.26 428.88
8.76 41.48 1018.49 74.98 474.3
15.54 39.31 1009.69 71.19 461.06
18.71 39.39 1014.09 62.82 465.57
13.06 41.78 1012.3 55.31 467.67
12.72 40.71 1016.02 71.57 466.99
19.83 39.39 1013.73 59.16 463.72
27.23 49.16 1004.03 40.8 443.78
24.27 68.28 1005.43 67.63 445.23
11.8 40.66 1017.13 97.2 464.43
6.76 36.25 1028.31 91.16 484.36
25.99 63.07 1012.5 64.81 442.16
16.3 39.63 1004.64 85.61 464.11
16.5 49.39 1018.35 93.42 462.48
10.59 42.49 1009.59 77.36 477.49
26.05 65.59 1012.78 67.03 437.04
19.5 40.79 1003.8 89.45 457.09
22.21 45.01 1012.22 54.84 450.6
17.86 45.0 1023.25 53.48 465.78
29.96 70.04 1010.15 54.47 427.1
19.08 44.63 1020.14 43.36 459.81
23.59 47.43 1006.64 48.92 447.36
3.38 39.64 1011.0 81.22 488.92
26.39 66.49 1012.96 60.35 433.36
8.99 39.04 1021.99 75.98 483.35
10.91 41.04 1026.57 74.24 469.53
13.08 39.82 1012.27 85.21 476.96
23.95 58.46 1017.5 68.46 440.75
15.64 43.71 1024.51 78.31 462.55
18.78 54.2 1012.05 89.25 448.04
20.65 50.59 1016.22 68.57 455.24
4.96 40.07 1011.8 67.38 494.75
23.51 57.32 1012.55 53.6 444.58
5.99 35.79 1011.56 91.69 484.82
23.65 66.05 1019.6 78.21 442.9
5.17 39.33 1009.68 94.19 485.46
26.38 49.5 1012.82 37.19 457.81
6.02 43.65 1013.85 83.53 481.92
23.2 61.02 1009.63 79.45 443.23
8.57 39.69 1000.91 99.9 474.29
30.72 71.58 1009.98 50.39 430.46
21.52 50.66 1013.56 74.33 455.71
22.93 62.26 1011.25 83.66 438.34
5.71 41.31 1003.24 89.48 485.83
18.62 44.06 1017.76 64.59 452.82
27.88 68.94 1007.68 75.68 435.04
22.32 59.8 1016.82 64.18 451.21
14.55 42.74 1028.41 70.09 465.81
17.83 44.92 1025.04 70.58 458.42
9.68 39.96 1026.09 99.28 470.22
19.41 49.39 1020.84 81.89 449.24
13.22 44.92 1023.84 87.99 471.43
12.24 44.92 1023.74 88.21 473.26
19.21 58.49 1011.7 91.29 452.82
29.74 70.32 1008.1 52.72 432.69
23.28 60.84 1017.91 67.5 444.13
8.02 41.92 1029.8 92.05 467.21
22.47 48.6 1002.33 63.23 445.98
27.51 73.77 1002.42 90.88 436.91
17.51 44.9 1009.05 74.91 455.01
23.22 66.56 1002.47 85.39 437.11
11.73 40.64 1020.68 96.98 477.06
21.19 67.71 1006.65 56.28 441.71
5.48 40.07 1019.63 65.62 495.76
24.26 66.44 1011.33 55.32 445.63
12.32 41.62 1012.88 88.88 464.72
31.26 68.94 1005.94 39.49 438.03
32.09 72.86 1003.47 54.59 434.78
24.98 60.32 1015.63 57.19 444.67
27.48 61.41 1012.2 45.06 452.24
21.04 45.09 1014.19 40.62 450.92
27.75 70.4 1006.65 90.21 436.53
22.79 71.77 1005.75 90.91 435.53
24.22 68.51 1013.23 74.96 440.01
27.06 64.45 1008.72 54.21 443.1
29.25 71.94 1007.18 63.62 427.49
26.86 68.08 1012.99 50.04 436.25
29.64 67.79 1009.99 51.23 440.74
19.92 63.31 1015.02 82.71 443.54
18.5 51.43 1010.82 92.04 459.42
23.71 60.23 1009.76 90.67 439.66
14.39 44.84 1023.55 91.14 464.15
19.3 56.65 1020.55 70.43 459.1
24.65 52.36 1014.76 66.63 455.68
13.5 45.51 1015.33 86.95 469.08
9.82 41.26 1007.71 96.69 478.02
18.4 44.06 1017.36 70.88 456.8
28.12 44.89 1009.18 47.14 441.13
17.15 43.69 1017.05 63.36 463.88
30.69 73.67 1006.14 60.58 430.45
28.82 65.71 1014.24 54.3 449.18
21.3 48.92 1010.92 65.09 447.89
30.58 70.04 1010.4 48.16 431.59
21.17 52.3 1009.36 81.51 447.5
9.87 41.82 1033.04 68.57 475.58
22.18 59.8 1016.77 73.16 453.24
24.39 63.21 1012.59 80.88 446.4
10.73 44.92 1025.1 85.4 476.81
9.38 40.46 1019.29 75.77 474.1
20.27 57.76 1016.66 75.76 450.71
24.82 66.48 1006.4 70.21 433.62
16.55 41.66 1011.45 55.53 465.14
20.73 59.87 1019.08 80.48 445.18
9.51 39.22 1015.3 72.41 474.12
8.63 43.79 1016.08 83.25 483.91
6.48 40.27 1010.55 82.12 486.68
14.95 43.52 1022.43 94.75 464.98
5.76 45.87 1010.83 95.79 481.4
10.94 39.04 1021.81 86.02 479.2
15.87 41.16 1005.85 78.29 463.86
12.42 38.25 1012.76 82.23 472.3
29.12 58.84 1001.31 52.86 446.51
29.12 51.43 1005.93 60.66 437.71
19.08 41.1 1001.96 62.77 458.94
31.06 67.17 1007.62 65.54 437.91
5.72 39.33 1009.96 95.4 490.76
26.52 65.06 1013.4 51.78 439.66
13.84 44.9 1007.58 63.62 463.27
13.03 39.52 1016.68 83.09 473.99
25.94 66.49 1012.83 61.81 433.38
16.64 53.82 1015.13 68.24 459.01
14.13 40.75 1016.05 72.41 471.44
13.65 39.28 1012.97 79.64 471.91
14.5 44.47 1028.2 66.95 465.15
19.8 51.19 1008.25 91.98 446.66
25.2 63.76 1009.78 64.96 438.15
20.66 51.19 1008.81 88.93 447.14
12.07 43.71 1025.53 85.62 472.32
25.64 70.72 1010.16 84.0 441.68
23.33 72.99 1009.33 89.41 440.04
29.41 64.05 1009.82 67.4 444.82
16.6 53.16 1014.5 76.75 457.26
27.53 72.58 1009.13 89.06 428.83
20.62 43.43 1009.93 64.02 449.07
26.02 71.94 1009.38 64.12 435.21
12.75 44.2 1017.59 81.22 471.03
12.87 48.04 1012.47 100.13 465.56
25.77 62.96 1019.86 58.07 442.83
14.84 41.48 1017.26 63.42 460.3
7.41 40.71 1023.07 83.32 474.25
8.87 41.82 1033.3 74.28 477.97
9.69 40.46 1019.1 71.91 472.16
16.17 46.97 1014.22 85.8 456.08
26.24 49.82 1014.9 55.58 452.41
13.78 43.22 1011.31 69.7 463.71
26.3 67.07 1006.26 63.79 433.72
17.37 57.76 1016.0 86.59 456.4
23.6 48.98 1015.41 48.28 448.43
8.3 36.08 1020.63 80.42 481.6
18.86 42.18 1001.16 98.58 457.07
22.12 49.39 1019.8 72.83 451.0
28.41 75.6 1018.48 56.07 440.28
29.42 71.32 1002.26 67.13 437.47
18.61 67.71 1004.07 84.49 443.57
27.57 69.84 1004.91 68.37 426.6
12.83 41.5 1013.12 86.07 470.87
9.64 39.85 1012.9 83.82 478.37
19.13 58.66 1013.32 74.86 453.92
15.92 40.56 1020.79 53.52 470.22
24.64 72.24 1011.37 80.61 434.54
27.62 63.9 1013.11 43.56 442.89
8.9 36.24 1013.29 89.35 479.03
9.55 43.99 1020.5 97.28 476.06
10.57 36.71 1022.62 80.49 473.88
19.8 57.25 1010.84 88.9 451.75
25.63 56.85 1012.68 49.7 439.2
24.7 58.46 1015.58 68.64 439.7
15.26 46.18 1013.68 98.58 463.6
20.06 52.84 1004.21 82.12 447.47
19.84 56.89 1013.23 78.32 447.92
11.49 44.63 1020.44 86.04 471.08
23.74 72.43 1007.99 91.36 437.55
22.62 51.3 1012.36 81.02 448.27
29.53 72.39 998.47 76.05 431.69
21.32 48.14 1016.57 71.81 449.09
20.3 58.46 1015.93 82.13 448.79
16.97 44.92 1025.21 74.27 460.21
12.07 41.17 1013.54 71.32 479.28
7.46 41.82 1032.67 74.59 483.11
19.2 54.2 1011.46 84.44 450.75
28.64 66.54 1010.43 43.39 437.97
13.56 41.48 1008.53 87.2 459.76
17.4 44.9 1020.5 77.11 457.75
14.08 40.1 1015.48 82.81 469.33
27.11 69.75 1009.74 85.67 433.28
20.92 70.02 1010.23 95.58 444.64
16.18 44.9 1021.3 74.46 463.1
15.57 44.68 1022.01 90.02 460.91
10.37 39.04 1023.95 81.93 479.35
19.6 59.21 1017.65 86.29 449.23
9.22 40.92 1021.83 85.43 474.51
27.76 72.99 1007.81 71.66 435.02
28.68 70.72 1009.43 71.33 435.45
20.95 48.14 1013.3 67.72 452.38
9.06 39.3 1019.73 84.23 480.41
9.21 39.72 1019.54 74.44 478.96
13.65 42.74 1026.58 71.48 468.87
31.79 76.2 1007.89 56.3 434.01
14.32 44.6 1013.85 68.13 466.36
26.28 75.23 1011.44 68.35 435.28
7.69 43.02 1014.51 85.23 486.46
14.44 40.1 1015.51 79.78 468.19
9.19 41.01 1022.14 98.98 468.37
13.35 41.39 1019.17 72.87 474.19
23.04 74.22 1009.52 90.93 440.32
4.83 38.44 1015.35 72.94 485.32
17.29 42.86 1014.38 72.3 464.27
8.73 36.18 1013.66 77.74 479.25
26.21 70.32 1007.0 78.29 430.4
23.72 58.62 1016.65 69.1 447.49
29.27 64.69 1006.85 55.79 438.23
10.4 40.43 1025.46 75.09 492.09
12.19 40.75 1015.13 88.98 475.36
20.4 54.9 1016.68 64.26 452.56
34.3 74.67 1015.98 25.89 427.84
27.56 68.08 1010.8 59.18 433.95
30.9 70.8 1008.48 67.48 435.27
14.85 58.59 1014.04 89.85 454.62
16.42 40.56 1020.36 50.62 472.17
16.45 63.31 1015.96 83.97 452.42
10.14 42.02 1003.19 96.51 472.17
9.53 41.44 1018.01 80.09 481.83
17.01 49.15 1021.83 84.02 458.78
23.94 62.08 1022.47 61.97 447.5
15.95 49.25 1019.04 88.51 463.4
11.15 41.26 1022.67 81.83 473.57
25.56 70.32 1009.07 90.63 433.72
27.16 66.44 1011.2 73.37 431.85
26.71 77.95 1012.13 77.5 433.47
29.56 74.22 1007.45 57.46 432.84
31.19 70.94 1007.29 51.91 436.6
6.86 41.17 1020.12 79.14 490.23
12.36 41.74 1020.58 69.24 477.16
32.82 68.31 1010.44 41.85 441.06
25.3 70.98 1007.22 95.1 440.86
8.71 41.82 1033.08 74.53 477.94
13.34 40.8 1026.56 64.85 474.47
14.2 43.02 1012.18 57.07 470.67
23.74 65.34 1013.7 62.9 447.31
16.9 44.88 1018.14 72.21 466.8
28.54 71.94 1007.4 65.99 430.91
30.15 69.88 1007.2 73.67 434.75
14.33 42.86 1010.82 88.59 469.52
25.57 59.43 1008.88 61.19 438.9
30.55 70.04 1010.51 49.37 429.56
28.04 74.33 1013.53 48.65 432.92
26.39 49.16 1005.68 56.18 442.87
15.3 41.76 1022.57 71.56 466.59
6.03 41.14 1028.04 87.46 479.61
13.49 44.63 1019.12 70.02 471.08
27.67 59.14 1016.51 61.2 433.37
24.19 65.48 1018.8 60.54 443.92
24.44 59.14 1016.74 71.82 443.5
29.86 64.79 1017.37 44.8 439.89
30.2 69.59 1008.9 67.32 434.66
7.99 41.38 1021.95 78.77 487.57
9.93 41.62 1013.76 96.02 464.64
11.03 42.32 1017.26 90.56 470.92
22.34 63.73 1014.37 83.19 444.39
25.33 48.6 1002.54 68.45 442.48
18.87 52.08 1005.25 99.19 449.61
25.97 69.34 1009.43 88.11 435.02
16.58 43.99 1021.81 79.29 458.67
14.35 46.18 1016.63 87.76 461.74
25.06 62.39 1008.09 82.56 438.31
13.85 48.92 1011.68 79.24 462.38
16.09 44.2 1019.39 67.24 460.56
26.34 59.21 1013.37 58.98 439.22
23.01 58.79 1009.71 84.22 444.64
26.39 71.25 999.8 89.12 430.34
31.32 71.29 1008.37 50.07 430.46
16.64 45.87 1009.02 98.86 456.79
13.42 41.23 994.17 95.79 468.82
20.06 44.9 1008.79 70.06 448.51
14.8 44.71 1014.67 41.71 470.77
12.59 41.14 1025.79 86.55 465.74
26.7 66.56 1005.31 71.97 430.21
19.78 50.32 1008.62 96.4 449.23
15.17 49.15 1021.91 91.73 461.89
21.71 61.45 1010.97 91.62 445.72
19.09 39.39 1013.36 59.14 466.13
19.76 51.19 1008.38 92.56 448.71
14.68 41.23 998.43 83.71 469.25
21.3 66.86 1013.04 55.43 450.56
16.73 39.64 1008.94 74.91 464.46
12.26 41.5 1014.87 89.41 471.13
14.77 48.06 1010.92 69.81 461.52
18.26 59.15 1012.04 86.01 451.09
27.1 79.74 1005.43 86.05 431.51
14.72 40.83 1009.65 80.98 469.8
26.3 51.43 1012.05 63.62 442.28
16.48 48.92 1011.84 64.16 458.67
17.99 43.79 1016.13 75.63 462.4
20.34 59.8 1015.18 80.21 453.54
25.53 62.96 1019.81 59.7 444.38
31.59 58.9 1003.39 47.6 440.52
30.8 69.14 1007.68 63.78 433.62
10.75 45.0 1023.68 89.37 481.96
19.3 44.9 1008.89 70.55 452.75
4.71 39.42 1026.4 84.42 481.28
23.1 66.05 1020.28 80.62 439.03
32.63 73.88 1005.64 52.56 435.75
26.63 74.16 1009.72 83.26 436.03
24.35 58.49 1011.03 70.64 445.6
15.11 56.03 1020.27 89.95 462.65
29.1 50.05 1005.87 51.53 438.66
21.24 50.32 1008.54 84.83 447.32
6.16 39.48 1004.85 59.68 484.55
7.36 41.01 1024.9 97.88 476.8
10.44 39.04 1023.99 85.03 480.34
26.76 48.41 1010.53 47.38 440.63
16.79 44.6 1014.27 48.08 459.48
10.76 40.43 1025.98 79.65 490.78
6.07 38.91 1019.25 83.39 483.56
27.33 73.18 1012.26 82.18 429.38
27.15 59.21 1013.49 51.71 440.27
22.35 51.43 1011.34 77.33 445.34
21.82 65.27 1013.86 72.81 447.43
21.11 69.94 1004.37 84.26 439.91
19.95 50.59 1016.11 73.23 459.27
7.45 39.61 1017.88 79.73 478.89
15.36 41.66 1012.41 62.32 466.7
15.65 43.5 1021.39 78.58 463.5
25.31 74.33 1015.04 79.88 436.21
25.88 63.47 1011.95 65.87 443.94
24.6 63.94 1012.87 80.28 439.63
22.58 41.54 1013.21 71.33 460.95
19.69 59.14 1015.99 70.33 448.69
25.85 75.08 1006.24 57.73 444.63
10.06 37.83 1005.49 99.46 473.51
18.59 39.54 1008.56 68.61 462.56
18.27 50.16 1011.07 95.91 451.76
8.85 40.43 1025.68 80.42 491.81
30.04 68.08 1011.04 51.01 429.52
26.06 49.02 1007.59 74.08 437.9
14.8 38.73 1003.18 80.73 467.54
23.93 64.45 1015.35 54.71 449.97
23.72 66.48 1003.61 73.75 436.62
11.44 40.55 1023.37 88.43 477.68
20.28 63.86 1016.04 74.66 447.26
27.9 63.13 1011.8 70.04 439.76
24.74 59.39 1015.23 74.64 437.49
14.8 58.2 1018.29 85.11 455.14
8.22 41.03 1021.76 82.97 485.5
27.56 66.93 1016.81 55.59 444.1
32.07 70.94 1006.91 49.9 432.33
9.53 44.03 1008.87 89.99 471.23
13.61 42.34 1017.93 91.61 463.89
22.2 51.19 1009.2 82.95 445.54
21.36 59.54 1007.99 92.62 446.09
23.25 63.86 1017.82 59.64 445.12
23.5 59.21 1018.29 63.0 443.31
8.46 39.66 1015.14 85.38 484.16
8.19 40.69 1019.86 85.23 477.76
30.67 71.29 1008.36 52.08 430.28
32.48 62.04 1010.39 38.05 446.48
8.99 36.66 1028.11 71.98 481.03
13.77 47.83 1007.41 90.66 466.07
19.05 67.32 1013.2 83.14 447.47
21.19 55.5 1019.83 65.22 455.93
10.12 40.0 1021.15 91.67 479.62
24.93 47.01 1014.28 66.04 455.06
8.47 40.46 1019.87 78.19 475.06
24.52 56.85 1012.59 54.47 438.89
28.55 69.84 1003.38 67.26 432.7
20.58 50.9 1011.89 72.56 452.6
18.31 46.21 1010.46 82.15 451.75
27.18 71.06 1008.16 86.32 430.66
4.43 38.91 1019.04 88.17 491.9
26.02 74.78 1010.04 72.78 439.82
15.75 39.0 1015.91 69.58 460.73
22.99 60.95 1015.14 69.86 449.7
25.52 59.15 1013.88 65.37 439.42
27.04 65.06 1013.33 52.37 439.84
6.42 35.57 1025.58 79.63 485.86
17.04 40.12 1011.81 83.14 458.1
10.79 39.82 1012.89 88.25 479.92
20.41 56.03 1019.94 55.85 458.29
7.36 40.07 1017.29 52.55 489.45
28.08 73.42 1012.17 62.74 434.0
24.74 69.13 1010.69 90.08 431.24
28.32 47.93 1003.26 54.5 439.5
16.71 40.56 1019.48 49.88 467.46
30.7 71.58 1010.0 48.96 429.27
18.42 58.95 1016.95 86.77 452.1
10.62 42.02 999.83 96.66 472.41
22.18 69.05 1002.75 70.84 442.14
22.38 49.3 1003.56 83.83 441.0
13.94 41.58 1020.76 68.22 463.07
21.24 60.84 1017.99 82.22 445.71
6.76 39.81 1017.11 87.9 483.16
26.73 68.84 1010.75 66.83 440.45
7.24 38.06 1020.6 85.36 481.83
10.84 40.62 1015.53 60.9 467.6
19.32 52.84 1004.29 83.51 450.88
29.0 69.13 1001.22 52.96 425.5
23.38 54.42 1013.95 73.02 451.87
31.17 69.51 1010.51 43.11 428.94
26.17 48.6 1002.59 61.41 439.86
30.9 73.42 1011.21 65.32 433.44
24.92 73.68 1015.12 93.68 438.23
32.77 71.32 1007.68 42.39 436.95
14.37 40.56 1021.67 68.18 470.19
8.36 40.22 1011.6 89.18 484.66
31.45 68.27 1007.56 64.79 430.81
31.6 73.17 1010.05 43.48 433.37
17.9 48.98 1014.17 80.4 453.02
20.35 50.9 1012.6 72.43 453.5
16.21 41.23 995.88 80.0 463.09
19.36 44.6 1016.25 45.65 464.56
21.04 65.46 1017.22 63.02 452.12
14.05 40.69 1015.66 74.39 470.9
23.48 64.15 1021.08 57.77 450.89
21.91 63.76 1009.85 76.8 445.04
24.42 63.07 1011.49 67.39 444.72
14.26 40.92 1022.07 73.96 460.38
21.38 58.33 1013.05 72.75 446.8
15.71 44.06 1018.34 71.69 465.05
5.78 40.62 1016.55 84.98 484.13
6.77 39.81 1017.01 87.68 488.27
23.84 49.21 1013.85 50.36 447.09
21.17 58.16 1017.16 68.11 452.02
19.94 58.96 1014.16 66.27 455.55
8.73 41.92 1029.41 89.72 480.99
16.39 41.67 1012.96 61.07 467.68

Note that the output of the above command is the same as display(powerPlantDF) we did earlier.

We can use the SQL desc command to describe the schema. This is the SQL equivalent of powerPlantDF.printSchema we saw earlier.

desc power_plant_table
col_name data_type comment
AT double null
V double null
AP double null
RH double null
PE double null

Schema Definition

Our schema definition from UCI appears below:

  • AT = Atmospheric Temperature in C
  • V = Exhaust Vaccum Speed
  • AP = Atmospheric Pressure
  • RH = Relative Humidity
  • PE = Power Output

PE is our label or target. This is the value we are trying to predict given the measurements.

Reference UCI Machine Learning Repository Combined Cycle Power Plant Data Set

Let's do some basic statistical analysis of all the columns.

We can use the describe function with no parameters to get some basic stats for each column like count, mean, max, min and standard deviation. More information can be found in the Spark API docs

display(powerPlantDF.describe())
summary AT V AP RH PE
count 9568 9568 9568 9568 9568
mean 19.65123118729102 54.30580372073601 1013.2590781772603 73.30897784280926 454.3650094063554
stddev 7.4524732296110825 12.707892998326784 5.938783705811581 14.600268756728964 17.066994999803402
min 1.81 25.36 992.89 25.56 420.26
max 37.11 81.56 1033.3 100.16 495.76

Step 4: Visualize Your Data

To understand our data, we will look for correlations between features and the label. This can be important when choosing a model. E.g., if features and a label are linearly correlated, a linear model like Linear Regression can do well; if the relationship is very non-linear, more complex models such as Decision Trees or neural networks can be better. We use the Databricks built in visualization to view each of our predictors in relation to the label column as a scatter plot to see the correlation between the predictors and the label.

select AT as Temperature, PE as Power from power_plant_table
Temperature Power
14.96 463.26
25.18 444.37
5.11 488.56
20.86 446.48
10.82 473.9
26.27 443.67
15.89 467.35
9.48 478.42
14.64 475.98
11.74 477.5
17.99 453.02
20.14 453.99
24.34 440.29
25.71 451.28
26.19 433.99
21.42 462.19
18.21 467.54
11.04 477.2
14.45 459.85
13.97 464.3
17.76 468.27
5.41 495.24
7.76 483.8
27.23 443.61
27.36 436.06
27.47 443.25
14.6 464.16
7.91 475.52
5.81 484.41
30.53 437.89
23.87 445.11
26.09 438.86
29.27 440.98
27.38 436.65
24.81 444.26
12.75 465.86
24.66 444.37
16.38 450.69
13.91 469.02
23.18 448.86
22.47 447.14
13.39 469.18
9.28 482.8
11.82 476.7
10.27 474.99
22.92 444.22
16.0 461.33
21.22 448.06
13.46 474.6
9.39 473.05
31.07 432.06
12.82 467.41
32.57 430.12
8.11 473.62
13.92 471.81
23.04 442.99
27.31 442.77
5.91 491.49
25.26 447.46
27.97 446.11
26.08 442.44
29.01 446.22
12.18 471.49
13.76 463.5
25.5 440.01
28.26 441.03
21.39 452.68
7.26 474.91
10.54 478.77
27.71 434.2
23.11 437.91
7.51 477.61
26.46 431.65
29.34 430.57
10.32 481.09
22.74 445.56
13.48 475.74
25.52 435.12
21.58 446.15
27.66 436.64
26.96 436.69
12.29 468.75
15.86 466.6
13.87 465.48
24.09 441.34
20.45 441.83
15.07 464.7
32.72 437.99
18.23 459.12
35.56 429.69
18.36 459.8
26.35 433.63
25.92 442.84
8.01 485.13
19.63 459.12
20.02 445.31
10.08 480.8
27.23 432.55
23.37 443.86
18.74 449.77
14.81 470.71
23.1 452.17
10.72 478.29
29.46 428.54
8.1 478.27
27.29 439.58
17.1 457.32
11.49 475.51
23.69 439.66
13.51 471.99
9.64 479.81
25.65 434.78
21.59 446.58
27.98 437.76
18.8 459.36
18.28 462.28
13.55 464.33
22.99 444.36
23.94 438.64
13.74 470.49
21.3 455.13
27.54 450.22
24.81 440.43
4.97 482.98
15.22 460.44
23.88 444.97
33.01 433.94
25.98 439.73
28.18 434.48
21.67 442.33
17.67 457.67
21.37 454.66
28.69 432.21
16.61 457.66
27.91 435.21
20.97 448.22
10.8 475.51
20.61 446.53
25.45 441.3
30.16 433.54
4.99 472.52
10.51 474.77
33.79 435.1
21.34 450.74
23.4 442.7
32.21 426.56
14.26 463.71
27.71 447.06
21.95 452.27
25.76 445.78
23.68 438.65
8.28 480.15
23.44 447.19
25.32 443.04
3.94 488.81
17.3 455.75
18.2 455.86
21.43 457.68
11.16 479.11
30.38 432.84
23.36 448.37
21.69 447.06
23.62 443.53
21.87 445.21
29.25 441.7
20.03 450.93
18.14 451.44
24.23 441.29
18.11 458.85
6.57 481.46
12.56 467.19
13.4 461.54
27.1 439.08
14.28 467.22
16.29 468.8
31.24 426.93
10.57 474.65
13.8 468.97
25.3 433.97
18.06 450.53
25.42 444.51
15.07 469.03
11.75 466.56
20.23 457.57
27.31 440.13
28.57 433.24
17.9 452.55
23.83 443.29
27.92 431.76
17.34 454.97
17.94 456.7
6.4 486.03
11.78 472.79
20.28 452.03
21.04 443.41
25.11 441.93
30.28 432.64
8.14 480.25
16.86 466.68
6.25 494.39
22.35 454.72
17.98 448.71
21.19 469.76
20.94 450.71
24.23 444.01
19.18 453.2
20.88 450.87
23.67 441.73
14.12 465.09
25.23 447.28
6.54 491.16
20.08 450.98
24.67 446.3
27.82 436.48
15.55 460.84
24.26 442.56
13.45 467.3
11.06 479.13
24.91 441.15
22.39 445.52
11.95 475.4
14.85 469.3
10.11 463.57
23.67 445.32
16.14 461.03
15.11 466.74
24.14 444.04
30.08 434.01
14.77 465.23
27.6 440.6
13.89 466.74
26.85 433.48
12.41 473.59
13.08 474.81
18.93 454.75
20.5 452.94
30.72 435.83
7.55 482.19
13.49 466.66
15.62 462.59
24.8 447.82
10.03 462.73
22.43 447.98
14.95 462.72
24.78 442.42
23.2 444.69
14.01 466.7
19.4 453.84
30.15 436.92
6.91 486.37
29.04 440.43
26.02 446.82
5.89 484.91
26.52 437.76
28.53 438.91
16.59 464.19
22.95 442.19
23.96 446.86
17.48 457.15
6.69 482.57
10.25 476.03
28.87 428.89
12.04 472.7
22.58 445.6
15.12 464.78
25.48 440.42
27.87 428.41
23.72 438.5
25.0 438.28
8.42 476.29
22.46 448.46
29.92 438.99
11.68 471.8
14.04 471.81
19.86 449.82
25.99 442.14
23.42 441.46
10.6 477.62
20.97 446.76
14.14 472.52
8.56 471.58
24.86 440.85
29.0 431.37
27.59 437.33
10.45 469.22
8.51 471.11
29.82 439.17
22.56 445.33
11.38 473.71
20.25 452.66
22.42 440.99
14.85 467.42
25.62 444.14
19.85 457.17
13.67 467.87
24.39 442.04
16.07 471.36
11.6 460.7
31.38 431.33
29.91 432.6
19.67 447.61
27.18 443.87
21.39 446.87
10.45 465.74
19.46 447.86
23.55 447.65
23.35 437.87
9.26 483.51
10.3 479.65
20.94 455.16
23.13 431.91
12.77 470.68
28.29 429.28
19.13 450.81
24.44 437.73
20.32 460.21
20.54 442.86
12.16 482.99
28.09 440.0
9.25 478.48
21.75 455.28
23.7 436.94
16.22 461.06
24.75 438.28
10.48 472.61
29.53 426.85
12.59 470.18
23.5 455.38
29.01 428.32
9.75 480.35
19.55 455.56
21.05 447.66
24.72 443.06
21.19 452.43
10.77 477.81
28.68 431.66
29.87 431.8
22.99 446.67
24.66 445.26
32.63 425.72
31.38 430.58
23.87 439.86
25.6 441.11
27.62 434.72
30.1 434.01
12.19 475.64
13.11 460.44
28.29 436.4
13.45 461.03
10.98 479.08
26.48 435.76
13.07 460.14
25.56 442.2
22.68 447.69
28.86 431.15
22.7 445.0
27.89 431.59
13.78 467.22
28.14 445.33
11.8 470.57
10.71 473.77
24.54 447.67
11.54 474.29
29.47 437.14
29.24 432.56
14.51 459.14
22.91 446.19
27.02 428.1
13.49 468.46
30.24 435.02
23.19 445.52
17.73 462.69
18.62 455.75
12.85 463.74
32.33 439.79
25.09 443.26
29.45 432.04
16.91 465.86
14.09 465.6
10.73 469.43
23.2 440.75
8.21 481.32
9.3 479.87
16.97 458.59
23.69 438.62
25.13 445.59
9.86 481.87
11.33 475.01
26.95 436.54
15.0 456.63
20.76 451.69
14.29 463.04
19.74 446.1
26.68 438.67
14.24 466.88
21.98 444.6
22.75 440.26
8.34 483.92
11.8 475.19
8.81 479.24
30.05 434.92
16.01 454.16
21.75 447.58
13.94 467.9
29.25 426.29
22.33 447.02
16.43 455.85
11.5 476.46
23.53 437.48
21.86 452.77
6.17 491.54
30.19 438.41
11.67 476.1
15.34 464.58
11.5 467.74
25.53 442.12
21.27 453.34
28.37 425.29
28.39 449.63
13.78 462.88
14.6 464.67
5.1 489.96
7.0 482.38
26.3 437.95
30.56 429.2
21.09 453.34
28.21 442.47
15.84 462.6
10.03 478.79
20.37 456.11
21.19 450.33
33.73 434.83
29.87 433.43
19.62 456.02
9.93 485.23
9.43 473.57
14.24 469.94
12.97 452.07
7.6 475.32
8.39 480.69
25.41 444.01
18.43 465.17
10.31 480.61
11.29 476.04
22.61 441.76
29.34 428.24
18.87 444.77
13.21 463.1
11.3 470.5
29.23 431.0
27.76 430.68
29.26 436.42
25.72 452.33
23.43 440.16
25.6 435.75
22.3 449.74
27.91 430.73
30.35 432.75
21.78 446.79
7.19 486.35
20.88 453.18
24.19 458.31
9.98 480.26
23.47 448.65
26.35 458.41
29.89 435.39
19.29 450.21
17.48 459.59
25.21 445.84
23.3 441.08
15.42 467.33
21.44 444.19
29.45 432.96
29.69 438.09
15.52 467.9
11.47 475.72
9.77 477.51
22.6 435.13
8.24 477.9
17.01 457.26
19.64 467.53
10.61 465.15
12.04 474.28
29.19 444.49
21.75 452.84
23.66 435.38
27.05 433.57
29.63 435.27
18.2 468.49
32.22 433.07
26.88 430.63
29.05 440.74
8.9 474.49
18.93 449.74
27.49 436.73
23.1 434.58
11.22 473.93
31.97 435.99
13.32 466.83
31.68 427.22
23.69 444.07
13.83 469.57
18.32 459.89
11.05 479.59
22.03 440.92
10.23 480.87
23.92 441.9
29.38 430.2
17.35 465.16
9.81 471.32
4.97 485.43
5.15 495.35
21.54 449.12
7.94 480.53
18.77 457.07
21.69 443.67
10.07 477.52
13.83 472.95
10.45 472.54
11.56 469.17
23.64 435.21
10.48 477.78
13.09 475.89
10.67 483.9
12.57 476.2
14.45 462.16
14.22 471.05
6.97 484.71
20.61 446.34
14.67 469.02
29.06 432.12
14.38 467.28
32.51 429.66
11.79 469.49
8.65 485.87
9.75 481.95
9.11 479.03
23.39 434.5
14.3 464.9
17.49 452.71
31.1 429.74
19.77 457.09
28.61 446.77
13.52 460.76
13.52 471.95
17.57 453.29
28.18 441.61
14.29 464.73
18.12 464.68
31.27 430.59
26.24 438.01
7.44 479.08
29.78 436.39
23.37 447.07
10.62 479.91
5.84 489.05
14.51 463.17
11.31 471.26
11.25 480.49
9.18 473.78
19.82 455.5
24.77 446.27
9.66 482.2
21.96 452.48
18.59 464.48
24.75 438.1
24.37 445.6
29.6 442.43
25.32 436.67
16.15 466.56
15.74 457.29
5.97 487.03
15.84 464.93
14.84 466.0
12.25 469.52
27.38 428.88
8.76 474.3
15.54 461.06
18.71 465.57
13.06 467.67
12.72 466.99
19.83 463.72
27.23 443.78
24.27 445.23
11.8 464.43
6.76 484.36
25.99 442.16
16.3 464.11
16.5 462.48
10.59 477.49
26.05 437.04
19.5 457.09
22.21 450.6
17.86 465.78
29.96 427.1
19.08 459.81
23.59 447.36
3.38 488.92
26.39 433.36
8.99 483.35
10.91 469.53
13.08 476.96
23.95 440.75
15.64 462.55
18.78 448.04
20.65 455.24
4.96 494.75
23.51 444.58
5.99 484.82
23.65 442.9
5.17 485.46
26.38 457.81
6.02 481.92
23.2 443.23
8.57 474.29
30.72 430.46
21.52 455.71
22.93 438.34
5.71 485.83
18.62 452.82
27.88 435.04
22.32 451.21
14.55 465.81
17.83 458.42
9.68 470.22
19.41 449.24
13.22 471.43
12.24 473.26
19.21 452.82
29.74 432.69
23.28 444.13
8.02 467.21
22.47 445.98
27.51 436.91
17.51 455.01
23.22 437.11
11.73 477.06
21.19 441.71
5.48 495.76
24.26 445.63
12.32 464.72
31.26 438.03
32.09 434.78
24.98 444.67
27.48 452.24
21.04 450.92
27.75 436.53
22.79 435.53
24.22 440.01
27.06 443.1
29.25 427.49
26.86 436.25
29.64 440.74
19.92 443.54
18.5 459.42
23.71 439.66
14.39 464.15
19.3 459.1
24.65 455.68
13.5 469.08
9.82 478.02
18.4 456.8
28.12 441.13
17.15 463.88
30.69 430.45
28.82 449.18
21.3 447.89
30.58 431.59
21.17 447.5
9.87 475.58
22.18 453.24
24.39 446.4
10.73 476.81
9.38 474.1
20.27 450.71
24.82 433.62
16.55 465.14
20.73 445.18
9.51 474.12
8.63 483.91
6.48 486.68
14.95 464.98
5.76 481.4
10.94 479.2
15.87 463.86
12.42 472.3
29.12 446.51
29.12 437.71
19.08 458.94
31.06 437.91
5.72 490.76
26.52 439.66
13.84 463.27
13.03 473.99
25.94 433.38
16.64 459.01
14.13 471.44
13.65 471.91
14.5 465.15
19.8 446.66
25.2 438.15
20.66 447.14
12.07 472.32
25.64 441.68
23.33 440.04
29.41 444.82
16.6 457.26
27.53 428.83
20.62 449.07
26.02 435.21
12.75 471.03
12.87 465.56
25.77 442.83
14.84 460.3
7.41 474.25
8.87 477.97
9.69 472.16
16.17 456.08
26.24 452.41
13.78 463.71
26.3 433.72
17.37 456.4
23.6 448.43
8.3 481.6
18.86 457.07
22.12 451.0
28.41 440.28
29.42 437.47
18.61 443.57
27.57 426.6
12.83 470.87
9.64 478.37
19.13 453.92
15.92 470.22
24.64 434.54
27.62 442.89
8.9 479.03
9.55 476.06
10.57 473.88
19.8 451.75
25.63 439.2
24.7 439.7
15.26 463.6
20.06 447.47
19.84 447.92
11.49 471.08
23.74 437.55
22.62 448.27
29.53 431.69
21.32 449.09
20.3 448.79
16.97 460.21
12.07 479.28
7.46 483.11
19.2 450.75
28.64 437.97
13.56 459.76
17.4 457.75
14.08 469.33
27.11 433.28
20.92 444.64
16.18 463.1
15.57 460.91
10.37 479.35
19.6 449.23
9.22 474.51
27.76 435.02
28.68 435.45
20.95 452.38
9.06 480.41
9.21 478.96
13.65 468.87
31.79 434.01
14.32 466.36
26.28 435.28
7.69 486.46
14.44 468.19
9.19 468.37
13.35 474.19
23.04 440.32
4.83 485.32
17.29 464.27
8.73 479.25
26.21 430.4
23.72 447.49
29.27 438.23
10.4 492.09
12.19 475.36
20.4 452.56
34.3 427.84
27.56 433.95
30.9 435.27
14.85 454.62
16.42 472.17
16.45 452.42
10.14 472.17
9.53 481.83
17.01 458.78
23.94 447.5
15.95 463.4
11.15 473.57
25.56 433.72
27.16 431.85
26.71 433.47
29.56 432.84
31.19 436.6
6.86 490.23
12.36 477.16
32.82 441.06
25.3 440.86
8.71 477.94
13.34 474.47
14.2 470.67
23.74 447.31
16.9 466.8
28.54 430.91
30.15 434.75
14.33 469.52
25.57 438.9
30.55 429.56
28.04 432.92
26.39 442.87
15.3 466.59
6.03 479.61
13.49 471.08
27.67 433.37
24.19 443.92
24.44 443.5
29.86 439.89
30.2 434.66
7.99 487.57
9.93 464.64
11.03 470.92
22.34 444.39
25.33 442.48
18.87 449.61
25.97 435.02
16.58 458.67
14.35 461.74
25.06 438.31
13.85 462.38
16.09 460.56
26.34 439.22
23.01 444.64
26.39 430.34
31.32 430.46
16.64 456.79
13.42 468.82
20.06 448.51
14.8 470.77
12.59 465.74
26.7 430.21
19.78 449.23
15.17 461.89
21.71 445.72
19.09 466.13
19.76 448.71
14.68 469.25
21.3 450.56
16.73 464.46
12.26 471.13
14.77 461.52
18.26 451.09
27.1 431.51
14.72 469.8
26.3 442.28
16.48 458.67
17.99 462.4
20.34 453.54
25.53 444.38
31.59 440.52
30.8 433.62
10.75 481.96
19.3 452.75
4.71 481.28
23.1 439.03
32.63 435.75
26.63 436.03
24.35 445.6
15.11 462.65
29.1 438.66
21.24 447.32
6.16 484.55
7.36 476.8
10.44 480.34
26.76 440.63
16.79 459.48
10.76 490.78
6.07 483.56
27.33 429.38
27.15 440.27
22.35 445.34
21.82 447.43
21.11 439.91
19.95 459.27
7.45 478.89
15.36 466.7
15.65 463.5
25.31 436.21
25.88 443.94
24.6 439.63
22.58 460.95
19.69 448.69
25.85 444.63
10.06 473.51
18.59 462.56
18.27 451.76
8.85 491.81
30.04 429.52
26.06 437.9
14.8 467.54
23.93 449.97
23.72 436.62
11.44 477.68
20.28 447.26
27.9 439.76
24.74 437.49
14.8 455.14
8.22 485.5
27.56 444.1
32.07 432.33
9.53 471.23
13.61 463.89
22.2 445.54
21.36 446.09
23.25 445.12
23.5 443.31
8.46 484.16
8.19 477.76
30.67 430.28
32.48 446.48
8.99 481.03
13.77 466.07
19.05 447.47
21.19 455.93
10.12 479.62
24.93 455.06
8.47 475.06
24.52 438.89
28.55 432.7
20.58 452.6
18.31 451.75
27.18 430.66
4.43 491.9
26.02 439.82
15.75 460.73
22.99 449.7
25.52 439.42
27.04 439.84
6.42 485.86
17.04 458.1
10.79 479.92
20.41 458.29
7.36 489.45
28.08 434.0
24.74 431.24
28.32 439.5
16.71 467.46
30.7 429.27
18.42 452.1
10.62 472.41
22.18 442.14
22.38 441.0
13.94 463.07
21.24 445.71
6.76 483.16
26.73 440.45
7.24 481.83
10.84 467.6
19.32 450.88
29.0 425.5
23.38 451.87
31.17 428.94
26.17 439.86
30.9 433.44
24.92 438.23
32.77 436.95
14.37 470.19
8.36 484.66
31.45 430.81
31.6 433.37
17.9 453.02
20.35 453.5
16.21 463.09
19.36 464.56
21.04 452.12
14.05 470.9
23.48 450.89
21.91 445.04
24.42 444.72
14.26 460.38
21.38 446.8
15.71 465.05
5.78 484.13
6.77 488.27
23.84 447.09
21.17 452.02
19.94 455.55
8.73 480.99
16.39 467.68

From the above plot, it looks like there is strong linear correlation between temperature and Power Output!

select V as ExhaustVaccum, PE as Power from power_plant_table;
ExhaustVaccum Power
41.76 463.26
62.96 444.37
39.4 488.56
57.32 446.48
37.5 473.9
59.44 443.67
43.96 467.35
44.71 478.42
45.0 475.98
43.56 477.5
43.72 453.02
46.93 453.99
73.5 440.29
58.59 451.28
69.34 433.99
43.79 462.19
45.0 467.54
41.74 477.2
52.75 459.85
38.47 464.3
42.42 468.27
40.07 495.24
42.28 483.8
63.9 443.61
48.6 436.06
70.72 443.25
39.31 464.16
39.96 475.52
35.79 484.41
65.18 437.89
63.94 445.11
58.41 438.86
66.85 440.98
74.16 436.65
63.94 444.26
44.03 465.86
63.73 444.37
47.45 450.69
39.35 469.02
51.3 448.86
47.45 447.14
44.85 469.18
41.54 482.8
42.86 476.7
40.64 474.99
63.94 444.22
37.87 461.33
43.43 448.06
44.71 474.6
40.11 473.05
73.5 432.06
38.62 467.41
78.92 430.12
42.18 473.62
39.39 471.81
59.43 442.99
64.44 442.77
39.33 491.49
61.08 447.46
58.84 446.11
52.3 442.44
65.71 446.22
40.1 471.49
45.87 463.5
58.79 440.01
65.34 441.03
62.96 452.68
40.69 474.91
34.03 478.77
74.34 434.2
68.3 437.91
41.01 477.61
74.67 431.65
74.34 430.57
42.28 481.09
61.02 445.56
39.85 475.74
69.75 435.12
67.25 446.15
76.86 436.64
69.45 436.69
42.18 468.75
43.02 466.6
45.08 465.48
73.68 441.34
69.45 441.83
39.3 464.7
69.75 437.99
58.96 459.12
68.94 429.69
51.43 459.8
64.05 433.63
60.95 442.84
41.66 485.13
52.72 459.12
67.32 445.31
40.72 480.8
66.48 432.55
63.77 443.86
59.21 449.77
43.69 470.71
51.3 452.17
41.38 478.29
71.94 428.54
40.64 478.27
62.66 439.58
49.69 457.32
44.2 475.51
65.59 439.66
40.89 471.99
39.35 479.81
78.92 434.78
61.87 446.58
58.33 437.76
39.72 459.36
44.71 462.28
43.48 464.33
46.21 444.36
59.39 438.64
34.03 470.49
41.1 455.13
66.93 450.22
63.73 440.43
42.85 482.98
50.88 460.44
54.2 444.97
68.67 433.94
73.18 439.73
73.88 434.48
60.84 442.33
45.09 457.67
57.76 454.66
67.25 432.21
43.77 457.66
63.76 435.21
47.43 448.22
41.66 475.51
62.91 446.53
57.32 441.3
69.34 433.54
39.04 472.52
44.78 474.77
69.05 435.1
59.8 450.74
65.06 442.7
68.14 426.56
42.32 463.71
66.93 447.06
57.76 452.27
63.94 445.78
68.3 438.65
40.77 480.15
62.52 447.19
48.41 443.04
39.9 488.81
57.76 455.75
49.39 455.86
46.97 457.68
40.05 479.11
74.16 432.84
62.52 448.37
47.45 447.06
49.21 443.53
61.45 445.21
66.51 441.7
66.86 450.93
49.78 451.44
56.89 441.29
44.85 458.85
43.65 481.46
43.41 467.19
41.58 461.54
52.84 439.08
42.74 467.22
44.34 468.8
71.98 426.93
37.73 474.65
44.21 468.97
71.58 433.97
50.16 450.53
59.04 444.51
40.69 469.03
71.14 466.56
52.05 457.57
59.54 440.13
69.84 433.24
43.72 452.55
71.37 443.29
74.99 431.76
44.78 454.97
63.07 456.7
39.9 486.03
39.96 472.79
57.25 452.03
54.2 443.41
67.32 441.93
70.98 432.64
36.24 480.25
39.63 466.68
40.07 494.39
54.42 454.72
56.85 448.71
42.48 469.76
44.89 450.71
58.79 444.01
58.2 453.2
57.85 450.87
63.86 441.73
39.52 465.09
64.63 447.28
39.33 491.16
62.52 450.98
63.56 446.3
79.74 436.48
42.03 460.84
69.51 442.56
41.49 467.3
40.64 479.13
52.3 441.15
59.04 445.52
40.69 475.4
40.69 469.3
41.62 463.57
68.67 445.32
44.21 461.03
43.13 466.74
59.87 444.04
67.25 434.01
44.9 465.23
69.34 440.6
44.84 466.74
75.6 433.48
40.96 473.59
41.74 474.81
44.06 454.75
49.69 452.94
69.13 435.83
39.22 482.19
44.47 466.66
40.12 462.59
64.63 447.82
41.62 462.73
63.21 447.98
39.31 462.72
58.46 442.42
48.41 444.69
39.0 466.7
64.63 453.84
67.32 436.92
36.08 486.37
60.07 440.43
63.07 446.82
39.48 484.91
71.64 437.76
68.08 438.91
39.54 464.19
67.79 442.19
47.43 446.86
44.2 457.15
43.65 482.57
41.26 476.03
72.58 428.89
40.23 472.7
52.3 445.6
52.05 464.78
58.95 440.42
70.79 428.41
70.47 438.5
59.43 438.28
40.64 476.29
58.49 448.46
57.19 438.99
39.22 471.8
42.44 471.81
59.14 449.82
68.08 442.14
58.79 441.46
40.22 477.62
61.87 446.76
39.82 472.52
40.71 471.58
72.39 440.85
77.54 431.37
71.97 437.33
40.71 469.22
40.78 471.11
66.51 439.17
62.26 445.33
39.22 473.71
57.76 452.66
59.43 440.99
38.91 467.42
58.82 444.14
56.53 457.17
54.3 467.87
70.72 442.04
44.58 471.36
39.1 460.7
70.83 431.33
76.86 432.6
59.39 447.61
64.79 443.87
52.3 446.87
41.01 465.74
56.89 447.86
62.96 447.65
63.47 437.87
41.66 483.51
41.46 479.65
58.16 455.16
71.25 431.91
41.5 470.68
69.13 429.28
59.21 450.81
73.5 437.73
44.6 460.21
69.05 442.86
45.0 482.99
65.27 440.0
41.82 478.48
49.82 455.28
66.56 436.94
37.87 461.06
69.45 438.28
39.58 472.61
70.79 426.85
39.72 470.18
54.42 455.38
66.56 428.32
42.49 480.35
56.53 455.56
58.33 447.66
68.67 443.06
58.86 452.43
41.54 477.81
73.77 431.66
73.91 431.8
68.67 446.67
60.29 445.26
69.89 425.72
72.29 430.58
60.27 439.86
59.15 441.11
71.14 434.72
67.45 434.01
41.17 475.64
41.58 460.44
68.67 436.4
40.73 461.03
41.54 479.08
69.14 435.76
45.51 460.14
75.6 442.2
50.78 447.69
73.67 431.15
63.56 445.0
73.21 431.59
44.47 467.22
51.43 445.33
45.09 470.57
39.61 473.77
60.29 447.67
40.05 474.29
71.32 437.14
69.05 432.56
41.79 459.14
60.07 446.19
71.77 428.1
44.47 468.46
66.75 435.02
48.6 445.52
40.55 462.69
61.27 455.75
40.0 463.74
69.68 439.79
58.95 443.26
69.13 432.04
43.96 465.86
45.87 465.6
25.36 469.43
49.3 440.75
38.91 481.32
40.56 479.87
39.16 458.59
71.97 438.62
59.44 445.59
43.56 481.87
41.5 475.01
48.41 436.54
40.66 456.63
62.52 451.69
39.59 463.04
67.71 446.1
59.92 438.67
41.4 466.88
48.41 444.6
59.39 440.26
40.96 483.92
41.2 475.19
44.68 479.24
73.68 434.92
65.46 454.16
58.79 447.58
41.26 467.9
69.13 426.29
45.87 447.02
41.79 455.85
40.22 476.46
68.94 437.48
49.21 452.77
39.33 491.54
64.79 438.41
41.93 476.1
36.99 464.58
40.78 467.74
57.17 442.12
57.5 453.34
69.13 425.29
51.43 449.63
45.78 462.88
42.32 464.67
35.57 489.96
38.08 482.38
77.95 437.95
71.98 429.2
46.63 453.34
70.02 442.47
49.69 462.6
40.96 478.79
52.05 456.11
50.16 450.33
69.88 434.83
73.68 433.43
62.96 456.02
40.67 485.23
37.14 473.57
39.58 469.94
49.83 452.07
41.04 475.32
36.24 480.69
48.06 444.01
56.03 465.17
39.82 480.61
41.5 476.04
49.3 441.76
71.98 428.24
67.71 444.77
45.87 463.1
44.6 470.5
72.99 431.0
69.4 430.68
67.17 436.42
49.82 452.33
63.94 440.16
63.76 435.75
44.57 449.74
72.24 430.73
77.17 432.75
47.43 446.79
41.39 486.35
59.8 453.18
50.23 458.31
41.54 480.26
51.3 448.65
49.5 458.41
64.69 435.39
50.16 450.21
43.14 459.59
75.6 445.84
48.78 441.08
37.85 467.33
63.09 444.19
68.27 432.96
47.93 438.09
36.99 467.9
43.67 475.72
34.69 477.51
69.84 435.13
39.61 477.9
44.2 457.26
44.6 467.53
41.58 465.15
40.1 474.28
65.71 444.49
45.09 452.84
77.54 435.38
75.33 433.57
69.71 435.27
39.63 468.49
70.8 433.07
73.56 430.63
65.74 440.74
39.96 474.49
48.6 449.74
63.76 436.73
70.79 434.58
43.13 473.93
79.74 435.99
43.22 466.83
68.24 427.22
63.77 444.07
41.49 469.57
66.51 459.89
40.71 479.59
64.69 440.92
41.46 480.87
66.54 441.9
69.68 430.2
42.86 465.16
44.45 471.32
40.64 485.43
40.07 495.35
58.49 449.12
42.02 480.53
50.66 457.07
69.94 443.67
44.68 477.52
39.64 472.95
39.69 472.54
40.71 469.17
70.04 435.21
40.22 477.78
39.85 475.89
40.23 483.9
39.16 476.2
43.34 462.16
37.85 471.05
41.26 484.71
63.86 446.34
42.28 469.02
72.86 432.12
40.1 467.28
69.98 429.66
45.09 469.49
40.56 485.87
40.81 481.95
40.02 479.03
69.13 434.5
54.3 464.9
63.94 452.71
69.51 429.74
56.65 457.09
72.29 446.77
41.48 460.76
40.83 471.95
46.21 453.29
60.07 441.61
46.18 464.73
43.69 464.68
73.91 430.59
77.95 438.01
41.04 479.08
74.78 436.39
65.46 447.07
39.58 479.91
43.02 489.05
53.82 463.17
42.02 471.26
40.67 480.49
39.42 473.78
58.16 455.5
58.41 446.27
41.06 482.2
59.8 452.48
43.14 464.48
69.89 438.1
63.47 445.6
67.79 442.43
61.25 436.67
41.85 466.56
71.14 457.29
36.25 487.03
52.72 464.93
44.63 466.0
48.79 469.52
70.04 428.88
41.48 474.3
39.31 461.06
39.39 465.57
41.78 467.67
40.71 466.99
39.39 463.72
49.16 443.78
68.28 445.23
40.66 464.43
36.25 484.36
63.07 442.16
39.63 464.11
49.39 462.48
42.49 477.49
65.59 437.04
40.79 457.09
45.01 450.6
45.0 465.78
70.04 427.1
44.63 459.81
47.43 447.36
39.64 488.92
66.49 433.36
39.04 483.35
41.04 469.53
39.82 476.96
58.46 440.75
43.71 462.55
54.2 448.04
50.59 455.24
40.07 494.75
57.32 444.58
35.79 484.82
66.05 442.9
39.33 485.46
49.5 457.81
43.65 481.92
61.02 443.23
39.69 474.29
71.58 430.46
50.66 455.71
62.26 438.34
41.31 485.83
44.06 452.82
68.94 435.04
59.8 451.21
42.74 465.81
44.92 458.42
39.96 470.22
49.39 449.24
44.92 471.43
44.92 473.26
58.49 452.82
70.32 432.69
60.84 444.13
41.92 467.21
48.6 445.98
73.77 436.91
44.9 455.01
66.56 437.11
40.64 477.06
67.71 441.71
40.07 495.76
66.44 445.63
41.62 464.72
68.94 438.03
72.86 434.78
60.32 444.67
61.41 452.24
45.09 450.92
70.4 436.53
71.77 435.53
68.51 440.01
64.45 443.1
71.94 427.49
68.08 436.25
67.79 440.74
63.31 443.54
51.43 459.42
60.23 439.66
44.84 464.15
56.65 459.1
52.36 455.68
45.51 469.08
41.26 478.02
44.06 456.8
44.89 441.13
43.69 463.88
73.67 430.45
65.71 449.18
48.92 447.89
70.04 431.59
52.3 447.5
41.82 475.58
59.8 453.24
63.21 446.4
44.92 476.81
40.46 474.1
57.76 450.71
66.48 433.62
41.66 465.14
59.87 445.18
39.22 474.12
43.79 483.91
40.27 486.68
43.52 464.98
45.87 481.4
39.04 479.2
41.16 463.86
38.25 472.3
58.84 446.51
51.43 437.71
41.1 458.94
67.17 437.91
39.33 490.76
65.06 439.66
44.9 463.27
39.52 473.99
66.49 433.38
53.82 459.01
40.75 471.44
39.28 471.91
44.47 465.15
51.19 446.66
63.76 438.15
51.19 447.14
43.71 472.32
70.72 441.68
72.99 440.04
64.05 444.82
53.16 457.26
72.58 428.83
43.43 449.07
71.94 435.21
44.2 471.03
48.04 465.56
62.96 442.83
41.48 460.3
40.71 474.25
41.82 477.97
40.46 472.16
46.97 456.08
49.82 452.41
43.22 463.71
67.07 433.72
57.76 456.4
48.98 448.43
36.08 481.6
42.18 457.07
49.39 451.0
75.6 440.28
71.32 437.47
67.71 443.57
69.84 426.6
41.5 470.87
39.85 478.37
58.66 453.92
40.56 470.22
72.24 434.54
63.9 442.89
36.24 479.03
43.99 476.06
36.71 473.88
57.25 451.75
56.85 439.2
58.46 439.7
46.18 463.6
52.84 447.47
56.89 447.92
44.63 471.08
72.43 437.55
51.3 448.27
72.39 431.69
48.14 449.09
58.46 448.79
44.92 460.21
41.17 479.28
41.82 483.11
54.2 450.75
66.54 437.97
41.48 459.76
44.9 457.75
40.1 469.33
69.75 433.28
70.02 444.64
44.9 463.1
44.68 460.91
39.04 479.35
59.21 449.23
40.92 474.51
72.99 435.02
70.72 435.45
48.14 452.38
39.3 480.41
39.72 478.96
42.74 468.87
76.2 434.01
44.6 466.36
75.23 435.28
43.02 486.46
40.1 468.19
41.01 468.37
41.39 474.19
74.22 440.32
38.44 485.32
42.86 464.27
36.18 479.25
70.32 430.4
58.62 447.49
64.69 438.23
40.43 492.09
40.75 475.36
54.9 452.56
74.67 427.84
68.08 433.95
70.8 435.27
58.59 454.62
40.56 472.17
63.31 452.42
42.02 472.17
41.44 481.83
49.15 458.78
62.08 447.5
49.25 463.4
41.26 473.57
70.32 433.72
66.44 431.85
77.95 433.47
74.22 432.84
70.94 436.6
41.17 490.23
41.74 477.16
68.31 441.06
70.98 440.86
41.82 477.94
40.8 474.47
43.02 470.67
65.34 447.31
44.88 466.8
71.94 430.91
69.88 434.75
42.86 469.52
59.43 438.9
70.04 429.56
74.33 432.92
49.16 442.87
41.76 466.59
41.14 479.61
44.63 471.08
59.14 433.37
65.48 443.92
59.14 443.5
64.79 439.89
69.59 434.66
41.38 487.57
41.62 464.64
42.32 470.92
63.73 444.39
48.6 442.48
52.08 449.61
69.34 435.02
43.99 458.67
46.18 461.74
62.39 438.31
48.92 462.38
44.2 460.56
59.21 439.22
58.79 444.64
71.25 430.34
71.29 430.46
45.87 456.79
41.23 468.82
44.9 448.51
44.71 470.77
41.14 465.74
66.56 430.21
50.32 449.23
49.15 461.89
61.45 445.72
39.39 466.13
51.19 448.71
41.23 469.25
66.86 450.56
39.64 464.46
41.5 471.13
48.06 461.52
59.15 451.09
79.74 431.51
40.83 469.8
51.43 442.28
48.92 458.67
43.79 462.4
59.8 453.54
62.96 444.38
58.9 440.52
69.14 433.62
45.0 481.96
44.9 452.75
39.42 481.28
66.05 439.03
73.88 435.75
74.16 436.03
58.49 445.6
56.03 462.65
50.05 438.66
50.32 447.32
39.48 484.55
41.01 476.8
39.04 480.34
48.41 440.63
44.6 459.48
40.43 490.78
38.91 483.56
73.18 429.38
59.21 440.27
51.43 445.34
65.27 447.43
69.94 439.91
50.59 459.27
39.61 478.89
41.66 466.7
43.5 463.5
74.33 436.21
63.47 443.94
63.94 439.63
41.54 460.95
59.14 448.69
75.08 444.63
37.83 473.51
39.54 462.56
50.16 451.76
40.43 491.81
68.08 429.52
49.02 437.9
38.73 467.54
64.45 449.97
66.48 436.62
40.55 477.68
63.86 447.26
63.13 439.76
59.39 437.49
58.2 455.14
41.03 485.5
66.93 444.1
70.94 432.33
44.03 471.23
42.34 463.89
51.19 445.54
59.54 446.09
63.86 445.12
59.21 443.31
39.66 484.16
40.69 477.76
71.29 430.28
62.04 446.48
36.66 481.03
47.83 466.07
67.32 447.47
55.5 455.93
40.0 479.62
47.01 455.06
40.46 475.06
56.85 438.89
69.84 432.7
50.9 452.6
46.21 451.75
71.06 430.66
38.91 491.9
74.78 439.82
39.0 460.73
60.95 449.7
59.15 439.42
65.06 439.84
35.57 485.86
40.12 458.1
39.82 479.92
56.03 458.29
40.07 489.45
73.42 434.0
69.13 431.24
47.93 439.5
40.56 467.46
71.58 429.27
58.95 452.1
42.02 472.41
69.05 442.14
49.3 441.0
41.58 463.07
60.84 445.71
39.81 483.16
68.84 440.45
38.06 481.83
40.62 467.6
52.84 450.88
69.13 425.5
54.42 451.87
69.51 428.94
48.6 439.86
73.42 433.44
73.68 438.23
71.32 436.95
40.56 470.19
40.22 484.66
68.27 430.81
73.17 433.37
48.98 453.02
50.9 453.5
41.23 463.09
44.6 464.56
65.46 452.12
40.69 470.9
64.15 450.89
63.76 445.04
63.07 444.72
40.92 460.38
58.33 446.8
44.06 465.05
40.62 484.13
39.81 488.27
49.21 447.09
58.16 452.02
58.96 455.55
41.92 480.99
41.67 467.68

The linear correlation is not as strong between Exhaust Vacuum Speed and Power Output but there is some semblance of a pattern.

select AP as Pressure, PE as Power from power_plant_table;
Pressure Power
1024.07 463.26
1020.04 444.37
1012.16 488.56
1010.24 446.48
1009.23 473.9
1012.23 443.67
1014.02 467.35
1019.12 478.42
1021.78 475.98
1015.14 477.5
1008.64 453.02
1014.66 453.99
1011.31 440.29
1012.77 451.28
1009.48 433.99
1015.76 462.19
1022.86 467.54
1022.6 477.2
1023.97 459.85
1015.15 464.3
1009.09 468.27
1019.16 495.24
1008.52 483.8
1014.3 443.61
1003.18 436.06
1009.97 443.25
1011.11 464.16
1023.57 475.52
1012.14 484.41
1012.69 437.89
1019.02 445.11
1013.64 438.86
1011.11 440.98
1010.08 436.65
1018.76 444.26
1007.29 465.86
1011.4 444.37
1010.08 450.69
1014.69 469.02
1012.04 448.86
1007.62 447.14
1017.24 469.18
1018.33 482.8
1014.12 476.7
1020.63 474.99
1019.28 444.22
1020.24 461.33
1010.96 448.06
1014.51 474.6
1029.14 473.05
1010.58 432.06
1018.71 467.41
1011.6 430.12
1014.82 473.62
1012.94 471.81
1010.23 442.99
1014.65 442.77
1010.18 491.49
1013.68 447.46
1002.25 446.11
1007.03 442.44
1013.61 446.22
1016.67 471.49
1008.89 463.5
1016.02 440.01
1014.56 441.03
1019.49 452.68
1020.43 474.91
1018.71 478.77
998.14 434.2
1017.83 437.91
1024.61 477.61
1016.65 431.65
998.58 430.57
1008.82 481.09
1009.56 445.56
1012.71 475.74
1010.36 435.12
1017.39 446.15
1001.31 436.64
1013.89 436.69
1016.53 468.75
1012.18 466.6
1024.42 465.48
1014.93 441.34
1012.53 441.83
1019.0 464.7
1009.6 437.99
1015.55 459.12
1006.56 429.69
1010.57 459.8
1009.81 433.63
1014.62 442.84
1014.49 485.13
1025.09 459.12
1012.05 445.31
1022.7 480.8
1005.23 432.55
1013.42 443.86
1018.3 449.77
1017.19 470.71
1011.93 452.17
1021.6 478.29
1006.96 428.54
1020.66 478.27
1007.63 439.58
1005.53 457.32
1018.79 475.51
1010.85 439.66
1011.03 471.99
1015.1 479.81
1010.83 434.78
1011.18 446.58
1013.92 437.76
1001.24 459.36
1016.99 462.28
1016.08 464.33
1010.71 444.36
1014.32 438.64
1018.69 470.49
1001.86 455.13
1017.06 450.22
1009.34 440.43
1014.02 482.98
1014.19 460.44
1012.81 444.97
1005.2 433.94
1012.28 439.73
1005.89 434.48
1017.93 442.33
1014.26 457.67
1018.8 454.66
1017.71 432.21
1012.25 457.66
1010.27 435.21
1007.64 448.22
1013.79 475.51
1013.24 446.53
1011.7 441.3
1007.67 433.54
1020.45 472.52
1012.59 474.77
1001.62 435.1
1016.92 450.74
1014.32 442.7
1003.34 426.56
1016.0 463.71
1016.85 447.06
1018.02 452.27
1018.49 445.78
1017.93 438.65
1011.55 480.15
1016.46 447.19
1008.47 443.04
1008.06 488.81
1016.26 455.75
1018.83 455.86
1013.94 457.68
1014.95 479.11
1007.44 432.84
1016.18 448.37
1007.56 447.06
1014.1 443.53
1011.13 445.21
1015.53 441.7
1013.05 450.93
1002.95 451.44
1012.32 441.29
1014.48 458.85
1018.24 481.46
1016.93 467.19
1020.5 461.54
1006.28 439.08
1028.79 467.22
1019.49 468.8
1004.66 426.93
1024.36 474.65
1022.93 468.97
1010.18 433.97
1009.52 450.53
1011.98 444.51
1015.29 469.03
1019.36 466.56
1012.15 457.57
1006.24 440.13
1003.57 433.24
1008.64 452.55
1002.04 443.29
1005.47 431.76
1007.81 454.97
1012.42 456.7
1007.75 486.03
1011.37 472.79
1010.12 452.03
1012.26 443.41
1014.49 441.93
1007.51 432.64
1013.15 480.25
1004.47 466.68
1020.19 494.39
1012.46 454.72
1012.28 448.71
1013.43 469.76
1009.64 450.71
1009.8 444.01
1017.46 453.2
1012.39 450.87
1019.67 441.73
1018.41 465.09
1020.59 447.28
1011.54 491.16
1017.99 450.98
1013.75 446.3
1008.37 436.48
1017.41 460.84
1013.43 442.56
1020.19 467.3
1021.47 479.13
1008.72 441.15
1011.78 445.52
1015.62 475.4
1014.91 469.3
1017.17 463.57
1006.71 445.32
1020.36 461.03
1014.99 466.74
1018.47 444.04
1017.6 434.01
1020.5 465.23
1009.63 440.6
1023.66 466.74
1017.43 433.48
1023.36 473.59
1020.75 474.81
1017.58 454.75
1009.6 452.94
1009.94 435.83
1014.53 482.19
1030.46 466.66
1013.03 462.59
1020.69 447.82
1014.55 462.73
1012.06 447.98
1009.15 462.72
1016.82 442.42
1008.64 444.69
1016.73 466.7
1020.38 453.84
1013.83 436.92
1021.82 486.37
1015.42 440.43
1010.94 446.82
1005.11 484.91
1008.27 437.76
1013.27 438.91
1007.97 464.19
1009.89 442.19
1008.38 446.86
1018.89 457.15
1020.14 482.57
1007.44 476.03
1008.69 428.89
1018.07 472.7
1009.04 445.6
1014.63 464.78
1017.02 440.42
1003.96 428.41
1010.65 438.5
1007.84 438.28
1022.35 476.29
1011.5 448.46
1008.62 438.99
1017.9 471.8
1012.74 471.81
1016.12 449.82
1013.13 442.14
1009.74 441.46
1011.37 477.62
1011.45 446.76
1012.46 472.52
1021.27 471.58
1001.15 440.85
1011.33 431.37
1008.64 437.33
1015.68 469.22
1023.51 471.11
1010.98 439.17
1012.11 445.33
1018.62 473.71
1016.28 452.66
1007.12 440.99
1014.48 467.42
1010.02 444.14
1020.57 457.17
1015.92 467.87
1009.78 442.04
1019.52 471.36
1009.81 460.7
1010.35 431.33
998.59 432.6
1014.07 447.61
1016.27 443.87
1009.2 446.87
1020.57 465.74
1014.02 447.86
1020.16 447.65
1011.78 437.87
1016.87 483.51
1018.21 479.65
1016.88 455.16
1002.49 431.91
1014.13 470.68
1009.29 429.28
1018.32 450.81
1011.49 437.73
1015.16 460.21
1001.6 442.86
1021.51 482.99
1013.27 440.0
1033.25 478.48
1015.01 455.28
1002.07 436.94
1022.36 461.06
1013.97 438.28
1011.81 472.61
1003.7 426.85
1017.76 470.18
1012.31 455.38
1006.44 428.32
1010.57 480.35
1020.2 455.56
1013.14 447.66
1006.74 443.06
1014.19 452.43
1019.94 477.81
1004.72 431.66
1004.53 431.8
1006.65 446.67
1018.0 445.26
1013.85 425.72
1008.73 430.58
1018.94 439.86
1013.31 441.11
1011.6 434.72
1014.23 434.01
1019.43 475.64
1020.43 460.44
1005.46 436.4
1018.7 461.03
1019.94 479.08
1009.31 435.76
1015.22 460.14
1017.37 442.2
1008.83 447.69
1006.65 431.15
1014.32 445.0
1001.32 431.59
1027.94 467.22
1012.16 445.33
1013.21 470.57
1018.72 473.77
1017.42 447.67
1014.78 474.29
1008.07 437.14
1003.12 432.56
1009.72 459.14
1016.03 446.19
1006.38 428.1
1030.18 468.46
1017.95 435.02
1002.38 445.52
1003.36 462.69
1019.26 455.75
1015.89 463.74
1011.95 439.79
1016.99 443.26
1009.3 432.04
1013.32 465.86
1009.05 465.6
1009.35 469.43
1003.4 440.75
1015.82 481.32
1022.64 479.87
1005.7 458.59
1009.62 438.62
1012.38 445.59
1015.13 481.87
1013.58 475.01
1008.53 436.54
1016.28 456.63
1015.63 451.69
1010.93 463.04
1007.68 446.1
1009.94 438.67
1019.7 466.88
1008.42 444.6
1015.4 440.26
1023.28 483.92
1017.18 475.19
1023.06 479.24
1014.95 434.92
1014.0 454.16
1012.42 447.58
1021.67 467.9
1010.27 426.29
1007.8 447.02
1005.47 455.85
1010.31 476.46
1007.53 437.48
1014.61 452.77
1012.57 491.54
1017.22 438.41
1019.81 476.1
1007.87 464.58
1023.91 467.74
1010.0 442.12
1014.53 453.34
1010.44 425.29
1011.74 449.63
1025.27 462.88
1015.71 464.67
1027.17 489.96
1020.27 482.38
1009.45 437.95
1004.74 429.2
1013.03 453.34
1010.58 442.47
1015.14 462.6
1024.57 478.79
1012.34 456.11
1005.81 450.33
1007.21 434.83
1015.1 433.43
1020.76 456.02
1018.08 485.23
1013.03 473.57
1011.17 469.94
1008.69 452.07
1021.82 475.32
1013.39 480.69
1013.12 444.01
1020.41 465.17
1012.87 480.61
1013.39 476.04
1003.51 441.76
1005.19 428.24
1004.0 444.77
1008.58 463.1
1018.19 470.5
1007.04 431.0
1004.27 430.68
1006.6 436.42
1016.19 452.33
1010.64 440.16
1010.18 435.75
1008.48 449.74
1010.74 430.73
1009.55 432.75
1007.88 446.79
1018.12 486.35
1015.66 453.18
1015.73 458.31
1019.7 480.26
1011.89 448.65
1012.67 458.41
1006.37 435.39
1010.49 450.21
1018.68 459.59
1017.19 445.84
1018.17 441.08
1009.89 467.33
1016.56 444.19
1007.96 432.96
1002.85 438.09
1006.86 467.9
1012.68 475.72
1027.72 477.51
1006.37 435.13
1017.99 477.9
1019.18 457.26
1015.88 467.53
1021.08 465.15
1014.42 474.28
1013.85 444.49
1014.15 452.84
1008.5 435.38
1003.88 433.57
1009.04 435.27
1005.35 468.49
1009.9 433.07
1004.85 430.63
1013.29 440.74
1026.31 474.49
1005.72 449.74
1010.09 436.73
1006.53 434.58
1017.24 473.93
1007.03 435.99
1009.45 466.83
1005.29 427.22
1013.39 444.07
1020.11 469.57
1015.18 459.89
1024.91 479.59
1007.21 440.92
1020.45 480.87
1009.93 441.9
1011.35 430.2
1014.62 465.16
1021.19 471.32
1020.91 485.43
1012.27 495.35
1010.85 449.12
1006.22 480.53
1014.89 457.07
1010.7 443.67
1023.44 477.52
1012.52 472.95
1003.92 472.54
1015.85 469.17
1011.09 435.21
1004.81 477.78
1012.86 475.89
1017.75 483.9
1016.53 476.2
1015.47 462.16
1011.24 471.05
1010.6 484.71
1015.43 446.34
1007.21 469.02
1004.23 432.12
1015.51 467.28
1013.29 429.66
1013.16 469.49
1023.23 485.87
1026.0 481.95
1031.1 479.03
1010.99 434.5
1015.16 464.9
1020.02 452.71
1010.84 429.74
1020.67 457.09
1011.61 446.77
1014.46 460.76
1008.31 471.95
1014.09 453.29
1016.34 441.61
1017.01 464.73
1016.91 464.68
1003.72 430.59
1014.19 438.01
1021.84 479.08
1009.28 436.39
1016.25 447.07
1011.9 479.91
1013.88 489.05
1016.46 463.17
1001.18 471.26
1011.64 480.49
1025.41 473.78
1016.76 455.5
1013.78 446.27
1021.21 482.2
1016.72 452.48
1011.92 464.48
1015.29 438.1
1012.77 445.6
1010.37 442.43
1011.56 436.67
1016.54 466.56
1019.65 457.29
1029.65 487.03
1026.45 464.93
1019.28 466.0
1017.44 469.52
1011.18 428.88
1018.49 474.3
1009.69 461.06
1014.09 465.57
1012.3 467.67
1016.02 466.99
1013.73 463.72
1004.03 443.78
1005.43 445.23
1017.13 464.43
1028.31 484.36
1012.5 442.16
1004.64 464.11
1018.35 462.48
1009.59 477.49
1012.78 437.04
1003.8 457.09
1012.22 450.6
1023.25 465.78
1010.15 427.1
1020.14 459.81
1006.64 447.36
1011.0 488.92
1012.96 433.36
1021.99 483.35
1026.57 469.53
1012.27 476.96
1017.5 440.75
1024.51 462.55
1012.05 448.04
1016.22 455.24
1011.8 494.75
1012.55 444.58
1011.56 484.82
1019.6 442.9
1009.68 485.46
1012.82 457.81
1013.85 481.92
1009.63 443.23
1000.91 474.29
1009.98 430.46
1013.56 455.71
1011.25 438.34
1003.24 485.83
1017.76 452.82
1007.68 435.04
1016.82 451.21
1028.41 465.81
1025.04 458.42
1026.09 470.22
1020.84 449.24
1023.84 471.43
1023.74 473.26
1011.7 452.82
1008.1 432.69
1017.91 444.13
1029.8 467.21
1002.33 445.98
1002.42 436.91
1009.05 455.01
1002.47 437.11
1020.68 477.06
1006.65 441.71
1019.63 495.76
1011.33 445.63
1012.88 464.72
1005.94 438.03
1003.47 434.78
1015.63 444.67
1012.2 452.24
1014.19 450.92
1006.65 436.53
1005.75 435.53
1013.23 440.01
1008.72 443.1
1007.18 427.49
1012.99 436.25
1009.99 440.74
1015.02 443.54
1010.82 459.42
1009.76 439.66
1023.55 464.15
1020.55 459.1
1014.76 455.68
1015.33 469.08
1007.71 478.02
1017.36 456.8
1009.18 441.13
1017.05 463.88
1006.14 430.45
1014.24 449.18
1010.92 447.89
1010.4 431.59
1009.36 447.5
1033.04 475.58
1016.77 453.24
1012.59 446.4
1025.1 476.81
1019.29 474.1
1016.66 450.71
1006.4 433.62
1011.45 465.14
1019.08 445.18
1015.3 474.12
1016.08 483.91
1010.55 486.68
1022.43 464.98
1010.83 481.4
1021.81 479.2
1005.85 463.86
1012.76 472.3
1001.31 446.51
1005.93 437.71
1001.96 458.94
1007.62 437.91
1009.96 490.76
1013.4 439.66
1007.58 463.27
1016.68 473.99
1012.83 433.38
1015.13 459.01
1016.05 471.44
1012.97 471.91
1028.2 465.15
1008.25 446.66
1009.78 438.15
1008.81 447.14
1025.53 472.32
1010.16 441.68
1009.33 440.04
1009.82 444.82
1014.5 457.26
1009.13 428.83
1009.93 449.07
1009.38 435.21
1017.59 471.03
1012.47 465.56
1019.86 442.83
1017.26 460.3
1023.07 474.25
1033.3 477.97
1019.1 472.16
1014.22 456.08
1014.9 452.41
1011.31 463.71
1006.26 433.72
1016.0 456.4
1015.41 448.43
1020.63 481.6
1001.16 457.07
1019.8 451.0
1018.48 440.28
1002.26 437.47
1004.07 443.57
1004.91 426.6
1013.12 470.87
1012.9 478.37
1013.32 453.92
1020.79 470.22
1011.37 434.54
1013.11 442.89
1013.29 479.03
1020.5 476.06
1022.62 473.88
1010.84 451.75
1012.68 439.2
1015.58 439.7
1013.68 463.6
1004.21 447.47
1013.23 447.92
1020.44 471.08
1007.99 437.55
1012.36 448.27
998.47 431.69
1016.57 449.09
1015.93 448.79
1025.21 460.21
1013.54 479.28
1032.67 483.11
1011.46 450.75
1010.43 437.97
1008.53 459.76
1020.5 457.75
1015.48 469.33
1009.74 433.28
1010.23 444.64
1021.3 463.1
1022.01 460.91
1023.95 479.35
1017.65 449.23
1021.83 474.51
1007.81 435.02
1009.43 435.45
1013.3 452.38
1019.73 480.41
1019.54 478.96
1026.58 468.87
1007.89 434.01
1013.85 466.36
1011.44 435.28
1014.51 486.46
1015.51 468.19
1022.14 468.37
1019.17 474.19
1009.52 440.32
1015.35 485.32
1014.38 464.27
1013.66 479.25
1007.0 430.4
1016.65 447.49
1006.85 438.23
1025.46 492.09
1015.13 475.36
1016.68 452.56
1015.98 427.84
1010.8 433.95
1008.48 435.27
1014.04 454.62
1020.36 472.17
1015.96 452.42
1003.19 472.17
1018.01 481.83
1021.83 458.78
1022.47 447.5
1019.04 463.4
1022.67 473.57
1009.07 433.72
1011.2 431.85
1012.13 433.47
1007.45 432.84
1007.29 436.6
1020.12 490.23
1020.58 477.16
1010.44 441.06
1007.22 440.86
1033.08 477.94
1026.56 474.47
1012.18 470.67
1013.7 447.31
1018.14 466.8
1007.4 430.91
1007.2 434.75
1010.82 469.52
1008.88 438.9
1010.51 429.56
1013.53 432.92
1005.68 442.87
1022.57 466.59
1028.04 479.61
1019.12 471.08
1016.51 433.37
1018.8 443.92
1016.74 443.5
1017.37 439.89
1008.9 434.66
1021.95 487.57
1013.76 464.64
1017.26 470.92
1014.37 444.39
1002.54 442.48
1005.25 449.61
1009.43 435.02
1021.81 458.67
1016.63 461.74
1008.09 438.31
1011.68 462.38
1019.39 460.56
1013.37 439.22
1009.71 444.64
999.8 430.34
1008.37 430.46
1009.02 456.79
994.17 468.82
1008.79 448.51
1014.67 470.77
1025.79 465.74
1005.31 430.21
1008.62 449.23
1021.91 461.89
1010.97 445.72
1013.36 466.13
1008.38 448.71
998.43 469.25
1013.04 450.56
1008.94 464.46
1014.87 471.13
1010.92 461.52
1012.04 451.09
1005.43 431.51
1009.65 469.8
1012.05 442.28
1011.84 458.67
1016.13 462.4
1015.18 453.54
1019.81 444.38
1003.39 440.52
1007.68 433.62
1023.68 481.96
1008.89 452.75
1026.4 481.28
1020.28 439.03
1005.64 435.75
1009.72 436.03
1011.03 445.6
1020.27 462.65
1005.87 438.66
1008.54 447.32
1004.85 484.55
1024.9 476.8
1023.99 480.34
1010.53 440.63
1014.27 459.48
1025.98 490.78
1019.25 483.56
1012.26 429.38
1013.49 440.27
1011.34 445.34
1013.86 447.43
1004.37 439.91
1016.11 459.27
1017.88 478.89
1012.41 466.7
1021.39 463.5
1015.04 436.21
1011.95 443.94
1012.87 439.63
1013.21 460.95
1015.99 448.69
1006.24 444.63
1005.49 473.51
1008.56 462.56
1011.07 451.76
1025.68 491.81
1011.04 429.52
1007.59 437.9
1003.18 467.54
1015.35 449.97
1003.61 436.62
1023.37 477.68
1016.04 447.26
1011.8 439.76
1015.23 437.49
1018.29 455.14
1021.76 485.5
1016.81 444.1
1006.91 432.33
1008.87 471.23
1017.93 463.89
1009.2 445.54
1007.99 446.09
1017.82 445.12
1018.29 443.31
1015.14 484.16
1019.86 477.76
1008.36 430.28
1010.39 446.48
1028.11 481.03
1007.41 466.07
1013.2 447.47
1019.83 455.93
1021.15 479.62
1014.28 455.06
1019.87 475.06
1012.59 438.89
1003.38 432.7
1011.89 452.6
1010.46 451.75
1008.16 430.66
1019.04 491.9
1010.04 439.82
1015.91 460.73
1015.14 449.7
1013.88 439.42
1013.33 439.84
1025.58 485.86
1011.81 458.1
1012.89 479.92
1019.94 458.29
1017.29 489.45
1012.17 434.0
1010.69 431.24
1003.26 439.5
1019.48 467.46
1010.0 429.27
1016.95 452.1
999.83 472.41
1002.75 442.14
1003.56 441.0
1020.76 463.07
1017.99 445.71
1017.11 483.16
1010.75 440.45
1020.6 481.83
1015.53 467.6
1004.29 450.88
1001.22 425.5
1013.95 451.87
1010.51 428.94
1002.59 439.86
1011.21 433.44
1015.12 438.23
1007.68 436.95
1021.67 470.19
1011.6 484.66
1007.56 430.81
1010.05 433.37
1014.17 453.02
1012.6 453.5
995.88 463.09
1016.25 464.56
1017.22 452.12
1015.66 470.9
1021.08 450.89
1009.85 445.04
1011.49 444.72
1022.07 460.38
1013.05 446.8
1018.34 465.05
1016.55 484.13
1017.01 488.27
1013.85 447.09
1017.16 452.02
1014.16 455.55
1029.41 480.99
1012.96 467.68
select RH as Humidity, PE as Power from power_plant_table;
Humidity Power
73.17 463.26
59.08 444.37
92.14 488.56
76.64 446.48
96.62 473.9
58.77 443.67
75.24 467.35
66.43 478.42
41.25 475.98
70.72 477.5
75.04 453.02
64.22 453.99
84.15 440.29
61.83 451.28
87.59 433.99
43.08 462.19
48.84 467.54
77.51 477.2
63.59 459.85
55.28 464.3
66.26 468.27
64.77 495.24
83.31 483.8
47.19 443.61
54.93 436.06
74.62 443.25
72.52 464.16
88.44 475.52
92.28 484.41
41.85 437.89
44.28 445.11
64.58 438.86
63.25 440.98
78.61 436.65
44.51 444.26
89.46 465.86
74.52 444.37
88.86 450.69
75.51 469.02
78.64 448.86
76.65 447.14
80.44 469.18
79.89 482.8
88.28 476.7
84.6 474.99
42.69 444.22
78.41 461.33
61.07 448.06
50.0 474.6
77.29 473.05
43.66 432.06
83.8 467.41
66.47 430.12
93.09 473.62
80.52 471.81
68.99 442.99
57.27 442.77
95.53 491.49
71.72 447.46
57.88 446.11
63.34 442.44
48.07 446.22
91.87 471.49
87.27 463.5
64.4 440.01
43.4 441.03
72.24 452.68
90.22 474.91
74.0 478.77
71.85 434.2
86.62 437.91
97.41 477.61
84.44 431.65
81.55 430.57
75.66 481.09
79.41 445.56
58.91 475.74
90.06 435.12
79.0 446.15
69.47 436.64
51.47 436.69
83.13 468.75
40.33 466.6
81.69 465.48
94.55 441.34
91.81 441.83
63.62 464.7
49.35 437.99
69.61 459.12
38.75 429.69
90.17 459.8
81.24 433.63
48.46 442.84
76.72 485.13
51.16 459.12
76.34 445.31
67.3 480.8
52.38 432.55
76.44 443.86
91.55 449.77
71.9 470.71
80.05 452.17
63.77 478.29
62.26 428.54
89.04 478.27
58.02 439.58
81.82 457.32
91.14 475.51
88.92 439.66
84.83 471.99
91.76 479.81
86.56 434.78
57.21 446.58
54.25 437.76
63.8 459.36
33.71 462.28
67.25 464.33
60.11 444.36
74.55 438.64
67.34 470.49
42.75 455.13
55.2 450.22
83.61 440.43
88.78 482.98
100.12 460.44
64.52 444.97
51.41 433.94
85.78 439.73
75.41 434.48
81.63 442.33
51.92 457.67
70.12 454.66
53.83 432.21
77.23 457.66
65.67 435.21
71.18 448.22
81.96 475.51
79.54 446.53
47.09 441.3
57.69 433.54
78.89 472.52
85.29 474.77
40.13 435.1
77.06 450.74
67.38 442.7
62.44 426.56
77.43 463.71
58.77 447.06
67.72 452.27
42.14 445.78
84.16 438.65
89.79 480.15
67.21 447.19
72.14 443.04
97.49 488.81
87.74 455.75
96.3 455.86
61.25 457.68
88.38 479.11
74.77 432.84
68.18 448.37
77.2 447.06
49.54 443.53
92.22 445.21
33.65 441.7
64.59 450.93
100.09 451.44
68.04 441.29
48.94 458.85
74.47 481.46
81.02 467.19
71.17 461.54
53.85 439.08
70.67 467.22
59.36 468.8
57.17 426.93
70.29 474.65
83.37 468.97
87.36 433.97
100.09 450.53
68.78 444.51
70.98 469.03
75.68 466.56
47.49 457.57
71.99 440.13
66.55 433.24
74.73 452.55
64.78 443.29
75.13 431.76
56.38 454.97
94.35 456.7
86.55 486.03
82.95 472.79
88.42 452.03
85.61 443.41
58.39 441.93
74.28 432.64
87.85 480.25
83.5 466.68
65.24 494.39
75.01 454.72
84.52 448.71
80.52 469.76
75.14 450.71
75.75 444.01
76.72 453.2
85.47 450.87
57.95 441.73
78.32 465.09
52.2 447.28
93.69 491.16
75.74 450.98
67.56 446.3
69.46 436.48
74.58 460.84
53.23 442.56
88.72 467.3
96.16 479.13
68.26 441.15
86.39 445.52
85.34 475.4
72.64 469.3
97.82 463.57
77.22 445.32
80.59 461.03
46.91 466.74
57.76 444.04
53.09 434.01
84.31 465.23
71.58 440.6
92.97 466.74
74.55 433.48
78.96 473.59
64.44 474.81
68.23 454.75
70.81 452.94
61.66 435.83
77.76 482.19
69.49 466.66
96.26 462.59
55.74 447.82
95.61 462.73
84.75 447.98
75.3 462.72
67.5 442.42
80.92 444.69
79.23 466.7
81.1 453.84
32.8 436.92
84.31 486.37
46.15 440.43
53.96 446.82
59.83 484.91
75.3 437.76
42.53 438.91
70.58 464.19
91.69 442.19
63.55 446.86
61.51 457.15
69.55 482.57
98.08 476.03
79.34 428.89
81.28 472.7
78.99 445.6
80.38 464.78
51.16 440.42
72.17 428.41
75.39 438.5
68.91 438.28
96.38 476.29
70.54 448.46
45.8 438.99
57.95 471.8
81.89 471.81
69.32 449.82
59.14 442.14
81.54 441.46
85.81 477.62
65.41 446.76
81.15 472.52
95.87 471.58
90.24 440.85
75.13 431.37
88.22 437.33
81.48 469.22
89.84 471.11
43.57 439.17
63.16 445.33
57.14 473.71
77.76 452.66
90.56 440.99
60.98 467.42
70.31 444.14
74.05 457.17
75.42 467.87
82.25 442.04
67.95 471.36
100.09 460.7
47.28 431.33
72.41 432.6
77.67 447.61
63.7 443.87
79.77 446.87
93.84 465.74
84.95 447.86
70.16 447.65
84.24 437.87
73.32 483.51
86.17 479.65
65.43 455.16
94.59 431.91
86.8 470.68
58.18 429.28
89.66 450.81
87.39 437.73
36.35 460.21
79.62 442.86
50.52 482.99
51.96 440.0
74.73 478.48
78.33 455.28
85.19 436.94
83.13 461.06
53.49 438.28
88.86 472.61
60.89 426.85
61.14 470.18
68.29 455.38
57.62 428.32
83.63 480.35
78.1 455.56
66.34 447.66
79.02 443.06
68.96 452.43
71.13 477.81
87.01 431.66
74.3 431.8
77.62 446.67
59.56 445.26
41.66 425.72
73.27 430.58
77.16 439.86
67.02 441.11
52.8 434.72
39.04 434.01
65.47 475.64
74.32 460.44
69.22 436.4
93.88 461.03
69.83 479.08
84.11 435.76
78.65 460.14
69.31 442.2
70.3 447.69
68.23 431.15
71.76 445.0
85.88 431.59
71.09 467.22
52.67 445.33
89.68 470.57
73.66 473.77
58.94 447.67
87.05 474.29
67.0 437.14
43.18 432.56
80.62 459.14
59.72 446.19
72.1 428.1
69.15 468.46
55.66 435.02
61.19 445.52
74.62 462.69
73.35 455.75
68.85 463.74
39.89 439.79
53.16 443.26
52.97 432.04
79.87 465.86
84.09 465.6
100.15 469.43
79.77 440.75
88.99 481.32
76.14 479.87
69.13 458.59
93.03 438.62
77.92 445.59
74.89 481.87
88.7 475.01
62.94 436.54
89.62 456.63
81.04 451.69
94.53 463.04
64.02 446.1
70.57 438.67
70.32 466.88
84.86 444.6
81.41 440.26
89.45 483.92
82.71 475.19
93.93 479.24
70.6 434.92
87.68 454.16
87.58 447.58
74.4 467.9
67.35 426.29
63.61 447.02
76.89 455.85
78.08 476.46
69.17 437.48
53.31 452.77
93.32 491.54
42.47 438.41
82.58 476.1
94.59 464.58
86.31 467.74
72.57 442.12
80.76 453.34
71.93 425.29
47.54 449.63
95.72 462.88
77.03 464.67
80.49 489.96
77.67 482.38
78.72 437.95
58.77 429.2
74.8 453.34
51.34 442.47
90.41 462.6
91.1 478.79
62.57 456.11
84.27 450.33
42.93 434.83
40.96 433.43
76.53 456.02
69.74 485.23
74.99 473.57
70.45 469.94
91.49 452.07
88.97 475.32
89.13 480.69
46.52 444.01
60.55 465.17
88.71 480.61
89.15 476.04
83.02 441.76
75.19 428.24
87.35 444.77
85.66 463.1
91.66 470.5
63.47 431.0
72.25 430.68
70.58 436.42
60.1 452.33
89.29 440.16
67.43 435.75
67.58 449.74
70.8 430.73
63.62 432.75
66.68 446.79
90.76 486.35
75.34 453.18
59.77 458.31
80.79 480.26
74.1 448.65
41.34 458.41
58.78 435.39
97.78 450.21
74.85 459.59
69.84 445.84
75.36 441.08
85.8 467.33
90.11 444.19
61.63 432.96
44.76 438.09
89.7 467.9
72.51 475.72
74.98 477.51
79.59 435.13
78.42 477.9
61.23 457.26
47.56 467.53
93.06 465.15
89.65 474.28
50.5 444.49
44.84 452.84
85.32 435.38
82.94 433.57
67.26 435.27
79.05 468.49
62.03 433.07
94.36 430.63
60.02 440.74
95.46 474.49
84.92 449.74
62.8 436.73
90.81 434.58
80.9 473.93
55.84 435.99
75.3 466.83
37.34 427.22
79.5 444.07
87.29 469.57
81.5 459.89
76.42 479.59
75.75 440.92
84.95 480.87
62.37 441.9
49.25 430.2
74.16 465.16
90.55 471.32
94.28 485.43
63.31 495.35
78.9 449.12
90.97 480.53
87.34 457.07
80.8 443.67
90.95 477.52
69.97 472.95
89.45 472.54
76.08 469.17
83.35 435.21
92.16 477.78
58.42 475.89
85.06 483.9
88.91 476.2
83.33 462.16
88.49 471.05
96.88 484.71
73.86 446.34
65.17 469.02
69.41 432.12
81.23 467.28
54.07 429.66
89.17 469.49
78.85 485.87
84.44 481.95
83.02 479.03
90.66 434.5
75.29 464.9
82.6 452.71
45.4 429.74
66.33 457.09
45.33 446.77
67.12 460.76
84.14 471.95
80.81 453.29
49.13 441.61
87.29 464.73
52.95 464.68
68.92 430.59
85.21 438.01
88.56 479.08
55.09 436.39
48.64 447.07
87.85 479.91
87.42 489.05
62.75 463.17
94.86 471.26
63.54 480.49
69.46 473.78
74.66 455.5
80.57 446.27
84.7 482.2
72.6 452.48
52.63 464.48
82.01 438.1
75.22 445.6
51.05 442.43
80.1 436.67
81.58 466.56
65.94 457.29
86.74 487.03
62.57 464.93
57.37 466.0
88.91 469.52
72.26 428.88
74.98 474.3
71.19 461.06
62.82 465.57
55.31 467.67
71.57 466.99
59.16 463.72
40.8 443.78
67.63 445.23
97.2 464.43
91.16 484.36
64.81 442.16
85.61 464.11
93.42 462.48
77.36 477.49
67.03 437.04
89.45 457.09
54.84 450.6
53.48 465.78
54.47 427.1
43.36 459.81
48.92 447.36
81.22 488.92
60.35 433.36
75.98 483.35
74.24 469.53
85.21 476.96
68.46 440.75
78.31 462.55
89.25 448.04
68.57 455.24
67.38 494.75
53.6 444.58
91.69 484.82
78.21 442.9
94.19 485.46
37.19 457.81
83.53 481.92
79.45 443.23
99.9 474.29
50.39 430.46
74.33 455.71
83.66 438.34
89.48 485.83
64.59 452.82
75.68 435.04
64.18 451.21
70.09 465.81
70.58 458.42
99.28 470.22
81.89 449.24
87.99 471.43
88.21 473.26
91.29 452.82
52.72 432.69
67.5 444.13
92.05 467.21
63.23 445.98
90.88 436.91
74.91 455.01
85.39 437.11
96.98 477.06
56.28 441.71
65.62 495.76
55.32 445.63
88.88 464.72
39.49 438.03
54.59 434.78
57.19 444.67
45.06 452.24
40.62 450.92
90.21 436.53
90.91 435.53
74.96 440.01
54.21 443.1
63.62 427.49
50.04 436.25
51.23 440.74
82.71 443.54
92.04 459.42
90.67 439.66
91.14 464.15
70.43 459.1
66.63 455.68
86.95 469.08
96.69 478.02
70.88 456.8
47.14 441.13
63.36 463.88
60.58 430.45
54.3 449.18
65.09 447.89
48.16 431.59
81.51 447.5
68.57 475.58
73.16 453.24
80.88 446.4
85.4 476.81
75.77 474.1
75.76 450.71
70.21 433.62
55.53 465.14
80.48 445.18
72.41 474.12
83.25 483.91
82.12 486.68
94.75 464.98
95.79 481.4
86.02 479.2
78.29 463.86
82.23 472.3
52.86 446.51
60.66 437.71
62.77 458.94
65.54 437.91
95.4 490.76
51.78 439.66
63.62 463.27
83.09 473.99
61.81 433.38
68.24 459.01
72.41 471.44
79.64 471.91
66.95 465.15
91.98 446.66
64.96 438.15
88.93 447.14
85.62 472.32
84.0 441.68
89.41 440.04
67.4 444.82
76.75 457.26
89.06 428.83
64.02 449.07
64.12 435.21
81.22 471.03
100.13 465.56
58.07 442.83
63.42 460.3
83.32 474.25
74.28 477.97
71.91 472.16
85.8 456.08
55.58 452.41
69.7 463.71
63.79 433.72
86.59 456.4
48.28 448.43
80.42 481.6
98.58 457.07
72.83 451.0
56.07 440.28
67.13 437.47
84.49 443.57
68.37 426.6
86.07 470.87
83.82 478.37
74.86 453.92
53.52 470.22
80.61 434.54
43.56 442.89
89.35 479.03
97.28 476.06
80.49 473.88
88.9 451.75
49.7 439.2
68.64 439.7
98.58 463.6
82.12 447.47
78.32 447.92
86.04 471.08
91.36 437.55
81.02 448.27
76.05 431.69
71.81 449.09
82.13 448.79
74.27 460.21
71.32 479.28
74.59 483.11
84.44 450.75
43.39 437.97
87.2 459.76
77.11 457.75
82.81 469.33
85.67 433.28
95.58 444.64
74.46 463.1
90.02 460.91
81.93 479.35
86.29 449.23
85.43 474.51
71.66 435.02
71.33 435.45
67.72 452.38
84.23 480.41
74.44 478.96
71.48 468.87
56.3 434.01
68.13 466.36
68.35 435.28
85.23 486.46
79.78 468.19
98.98 468.37
72.87 474.19
90.93 440.32
72.94 485.32
72.3 464.27
77.74 479.25
78.29 430.4
69.1 447.49
55.79 438.23
75.09 492.09
88.98 475.36
64.26 452.56
25.89 427.84
59.18 433.95
67.48 435.27
89.85 454.62
50.62 472.17
83.97 452.42
96.51 472.17
80.09 481.83
84.02 458.78
61.97 447.5
88.51 463.4
81.83 473.57
90.63 433.72
73.37 431.85
77.5 433.47
57.46 432.84
51.91 436.6
79.14 490.23
69.24 477.16
41.85 441.06
95.1 440.86
74.53 477.94
64.85 474.47
57.07 470.67
62.9 447.31
72.21 466.8
65.99 430.91
73.67 434.75
88.59 469.52
61.19 438.9
49.37 429.56
48.65 432.92
56.18 442.87
71.56 466.59
87.46 479.61
70.02 471.08
61.2 433.37
60.54 443.92
71.82 443.5
44.8 439.89
67.32 434.66
78.77 487.57
96.02 464.64
90.56 470.92
83.19 444.39
68.45 442.48
99.19 449.61
88.11 435.02
79.29 458.67
87.76 461.74
82.56 438.31
79.24 462.38
67.24 460.56
58.98 439.22
84.22 444.64
89.12 430.34
50.07 430.46
98.86 456.79
95.79 468.82
70.06 448.51
41.71 470.77
86.55 465.74
71.97 430.21
96.4 449.23
91.73 461.89
91.62 445.72
59.14 466.13
92.56 448.71
83.71 469.25
55.43 450.56
74.91 464.46
89.41 471.13
69.81 461.52
86.01 451.09
86.05 431.51
80.98 469.8
63.62 442.28
64.16 458.67
75.63 462.4
80.21 453.54
59.7 444.38
47.6 440.52
63.78 433.62
89.37 481.96
70.55 452.75
84.42 481.28
80.62 439.03
52.56 435.75
83.26 436.03
70.64 445.6
89.95 462.65
51.53 438.66
84.83 447.32
59.68 484.55
97.88 476.8
85.03 480.34
47.38 440.63
48.08 459.48
79.65 490.78
83.39 483.56
82.18 429.38
51.71 440.27
77.33 445.34
72.81 447.43
84.26 439.91
73.23 459.27
79.73 478.89
62.32 466.7
78.58 463.5
79.88 436.21
65.87 443.94
80.28 439.63
71.33 460.95
70.33 448.69
57.73 444.63
99.46 473.51
68.61 462.56
95.91 451.76
80.42 491.81
51.01 429.52
74.08 437.9
80.73 467.54
54.71 449.97
73.75 436.62
88.43 477.68
74.66 447.26
70.04 439.76
74.64 437.49
85.11 455.14
82.97 485.5
55.59 444.1
49.9 432.33
89.99 471.23
91.61 463.89
82.95 445.54
92.62 446.09
59.64 445.12
63.0 443.31
85.38 484.16
85.23 477.76
52.08 430.28
38.05 446.48
71.98 481.03
90.66 466.07
83.14 447.47
65.22 455.93
91.67 479.62
66.04 455.06
78.19 475.06
54.47 438.89
67.26 432.7
72.56 452.6
82.15 451.75
86.32 430.66
88.17 491.9
72.78 439.82
69.58 460.73
69.86 449.7
65.37 439.42
52.37 439.84
79.63 485.86
83.14 458.1
88.25 479.92
55.85 458.29
52.55 489.45
62.74 434.0
90.08 431.24
54.5 439.5
49.88 467.46
48.96 429.27
86.77 452.1
96.66 472.41
70.84 442.14
83.83 441.0
68.22 463.07
82.22 445.71
87.9 483.16
66.83 440.45
85.36 481.83
60.9 467.6
83.51 450.88
52.96 425.5
73.02 451.87
43.11 428.94
61.41 439.86
65.32 433.44
93.68 438.23
42.39 436.95
68.18 470.19
89.18 484.66
64.79 430.81
43.48 433.37
80.4 453.02
72.43 453.5
80.0 463.09
45.65 464.56
63.02 452.12
74.39 470.9
57.77 450.89
76.8 445.04
67.39 444.72
73.96 460.38
72.75 446.8
71.69 465.05
84.98 484.13
87.68 488.27
50.36 447.09
68.11 452.02
66.27 455.55
89.72 480.99
61.07 467.68

...and atmospheric pressure and relative humidity seem to have little to no linear correlation.

These pairwise plots can also be done directly using display on selected columns of the DataFrame powerPlantDF.

In general we will shy from SQL as much as possible to focus on ML pipelines written with DataFrames and DataSets with occassional in-and-out of RDDs.

The illustations in %sql above are to mainly reassure those with a RDBMS background and SQL that their SQL expressibility can be directly used in Apache Spark and in databricks notebooks.

display(powerPlantDF.select($"RH", $"PE"))
RH PE
73.17 463.26
59.08 444.37
92.14 488.56
76.64 446.48
96.62 473.9
58.77 443.67
75.24 467.35
66.43 478.42
41.25 475.98
70.72 477.5
75.04 453.02
64.22 453.99
84.15 440.29
61.83 451.28
87.59 433.99
43.08 462.19
48.84 467.54
77.51 477.2
63.59 459.85
55.28 464.3
66.26 468.27
64.77 495.24
83.31 483.8
47.19 443.61
54.93 436.06
74.62 443.25
72.52 464.16
88.44 475.52
92.28 484.41
41.85 437.89
44.28 445.11
64.58 438.86
63.25 440.98
78.61 436.65
44.51 444.26
89.46 465.86
74.52 444.37
88.86 450.69
75.51 469.02
78.64 448.86
76.65 447.14
80.44 469.18
79.89 482.8
88.28 476.7
84.6 474.99
42.69 444.22
78.41 461.33
61.07 448.06
50.0 474.6
77.29 473.05
43.66 432.06
83.8 467.41
66.47 430.12
93.09 473.62
80.52 471.81
68.99 442.99
57.27 442.77
95.53 491.49
71.72 447.46
57.88 446.11
63.34 442.44
48.07 446.22
91.87 471.49
87.27 463.5
64.4 440.01
43.4 441.03
72.24 452.68
90.22 474.91
74.0 478.77
71.85 434.2
86.62 437.91
97.41 477.61
84.44 431.65
81.55 430.57
75.66 481.09
79.41 445.56
58.91 475.74
90.06 435.12
79.0 446.15
69.47 436.64
51.47 436.69
83.13 468.75
40.33 466.6
81.69 465.48
94.55 441.34
91.81 441.83
63.62 464.7
49.35 437.99
69.61 459.12
38.75 429.69
90.17 459.8
81.24 433.63
48.46 442.84
76.72 485.13
51.16 459.12
76.34 445.31
67.3 480.8
52.38 432.55
76.44 443.86
91.55 449.77
71.9 470.71
80.05 452.17
63.77 478.29
62.26 428.54
89.04 478.27
58.02 439.58
81.82 457.32
91.14 475.51
88.92 439.66
84.83 471.99
91.76 479.81
86.56 434.78
57.21 446.58
54.25 437.76
63.8 459.36
33.71 462.28
67.25 464.33
60.11 444.36
74.55 438.64
67.34 470.49
42.75 455.13
55.2 450.22
83.61 440.43
88.78 482.98
100.12 460.44
64.52 444.97
51.41 433.94
85.78 439.73
75.41 434.48
81.63 442.33
51.92 457.67
70.12 454.66
53.83 432.21
77.23 457.66
65.67 435.21
71.18 448.22
81.96 475.51
79.54 446.53
47.09 441.3
57.69 433.54
78.89 472.52
85.29 474.77
40.13 435.1
77.06 450.74
67.38 442.7
62.44 426.56
77.43 463.71
58.77 447.06
67.72 452.27
42.14 445.78
84.16 438.65
89.79 480.15
67.21 447.19
72.14 443.04
97.49 488.81
87.74 455.75
96.3 455.86
61.25 457.68
88.38 479.11
74.77 432.84
68.18 448.37
77.2 447.06
49.54 443.53
92.22 445.21
33.65 441.7
64.59 450.93
100.09 451.44
68.04 441.29
48.94 458.85
74.47 481.46
81.02 467.19
71.17 461.54
53.85 439.08
70.67 467.22
59.36 468.8
57.17 426.93
70.29 474.65
83.37 468.97
87.36 433.97
100.09 450.53
68.78 444.51
70.98 469.03
75.68 466.56
47.49 457.57
71.99 440.13
66.55 433.24
74.73 452.55
64.78 443.29
75.13 431.76
56.38 454.97
94.35 456.7
86.55 486.03
82.95 472.79
88.42 452.03
85.61 443.41
58.39 441.93
74.28 432.64
87.85 480.25
83.5 466.68
65.24 494.39
75.01 454.72
84.52 448.71
80.52 469.76
75.14 450.71
75.75 444.01
76.72 453.2
85.47 450.87
57.95 441.73
78.32 465.09
52.2 447.28
93.69 491.16
75.74 450.98
67.56 446.3
69.46 436.48
74.58 460.84
53.23 442.56
88.72 467.3
96.16 479.13
68.26 441.15
86.39 445.52
85.34 475.4
72.64 469.3
97.82 463.57
77.22 445.32
80.59 461.03
46.91 466.74
57.76 444.04
53.09 434.01
84.31 465.23
71.58 440.6
92.97 466.74
74.55 433.48
78.96 473.59
64.44 474.81
68.23 454.75
70.81 452.94
61.66 435.83
77.76 482.19
69.49 466.66
96.26 462.59
55.74 447.82
95.61 462.73
84.75 447.98
75.3 462.72
67.5 442.42
80.92 444.69
79.23 466.7
81.1 453.84
32.8 436.92
84.31 486.37
46.15 440.43
53.96 446.82
59.83 484.91
75.3 437.76
42.53 438.91
70.58 464.19
91.69 442.19
63.55 446.86
61.51 457.15
69.55 482.57
98.08 476.03
79.34 428.89
81.28 472.7
78.99 445.6
80.38 464.78
51.16 440.42
72.17 428.41
75.39 438.5
68.91 438.28
96.38 476.29
70.54 448.46
45.8 438.99
57.95 471.8
81.89 471.81
69.32 449.82
59.14 442.14
81.54 441.46
85.81 477.62
65.41 446.76
81.15 472.52
95.87 471.58
90.24 440.85
75.13 431.37
88.22 437.33
81.48 469.22
89.84 471.11
43.57 439.17
63.16 445.33
57.14 473.71
77.76 452.66
90.56 440.99
60.98 467.42
70.31 444.14
74.05 457.17
75.42 467.87
82.25 442.04
67.95 471.36
100.09 460.7
47.28 431.33
72.41 432.6
77.67 447.61
63.7 443.87
79.77 446.87
93.84 465.74
84.95 447.86
70.16 447.65
84.24 437.87
73.32 483.51
86.17 479.65
65.43 455.16
94.59 431.91
86.8 470.68
58.18 429.28
89.66 450.81
87.39 437.73
36.35 460.21
79.62 442.86
50.52 482.99
51.96 440.0
74.73 478.48
78.33 455.28
85.19 436.94
83.13 461.06
53.49 438.28
88.86 472.61
60.89 426.85
61.14 470.18
68.29 455.38
57.62 428.32
83.63 480.35
78.1 455.56
66.34 447.66
79.02 443.06
68.96 452.43
71.13 477.81
87.01 431.66
74.3 431.8
77.62 446.67
59.56 445.26
41.66 425.72
73.27 430.58
77.16 439.86
67.02 441.11
52.8 434.72
39.04 434.01
65.47 475.64
74.32 460.44
69.22 436.4
93.88 461.03
69.83 479.08
84.11 435.76
78.65 460.14
69.31 442.2
70.3 447.69
68.23 431.15
71.76 445.0
85.88 431.59
71.09 467.22
52.67 445.33
89.68 470.57
73.66 473.77
58.94 447.67
87.05 474.29
67.0 437.14
43.18 432.56
80.62 459.14
59.72 446.19
72.1 428.1
69.15 468.46
55.66 435.02
61.19 445.52
74.62 462.69
73.35 455.75
68.85 463.74
39.89 439.79
53.16 443.26
52.97 432.04
79.87 465.86
84.09 465.6
100.15 469.43
79.77 440.75
88.99 481.32
76.14 479.87
69.13 458.59
93.03 438.62
77.92 445.59
74.89 481.87
88.7 475.01
62.94 436.54
89.62 456.63
81.04 451.69
94.53 463.04
64.02 446.1
70.57 438.67
70.32 466.88
84.86 444.6
81.41 440.26
89.45 483.92
82.71 475.19
93.93 479.24
70.6 434.92
87.68 454.16
87.58 447.58
74.4 467.9
67.35 426.29
63.61 447.02
76.89 455.85
78.08 476.46
69.17 437.48
53.31 452.77
93.32 491.54
42.47 438.41
82.58 476.1
94.59 464.58
86.31 467.74
72.57 442.12
80.76 453.34
71.93 425.29
47.54 449.63
95.72 462.88
77.03 464.67
80.49 489.96
77.67 482.38
78.72 437.95
58.77 429.2
74.8 453.34
51.34 442.47
90.41 462.6
91.1 478.79
62.57 456.11
84.27 450.33
42.93 434.83
40.96 433.43
76.53 456.02
69.74 485.23
74.99 473.57
70.45 469.94
91.49 452.07
88.97 475.32
89.13 480.69
46.52 444.01
60.55 465.17
88.71 480.61
89.15 476.04
83.02 441.76
75.19 428.24
87.35 444.77
85.66 463.1
91.66 470.5
63.47 431.0
72.25 430.68
70.58 436.42
60.1 452.33
89.29 440.16
67.43 435.75
67.58 449.74
70.8 430.73
63.62 432.75
66.68 446.79
90.76 486.35
75.34 453.18
59.77 458.31
80.79 480.26
74.1 448.65
41.34 458.41
58.78 435.39
97.78 450.21
74.85 459.59
69.84 445.84
75.36 441.08
85.8 467.33
90.11 444.19
61.63 432.96
44.76 438.09
89.7 467.9
72.51 475.72
74.98 477.51
79.59 435.13
78.42 477.9
61.23 457.26
47.56 467.53
93.06 465.15
89.65 474.28
50.5 444.49
44.84 452.84
85.32 435.38
82.94 433.57
67.26 435.27
79.05 468.49
62.03 433.07
94.36 430.63
60.02 440.74
95.46 474.49
84.92 449.74
62.8 436.73
90.81 434.58
80.9 473.93
55.84 435.99
75.3 466.83
37.34 427.22
79.5 444.07
87.29 469.57
81.5 459.89
76.42 479.59
75.75 440.92
84.95 480.87
62.37 441.9
49.25 430.2
74.16 465.16
90.55 471.32
94.28 485.43
63.31 495.35
78.9 449.12
90.97 480.53
87.34 457.07
80.8 443.67
90.95 477.52
69.97 472.95
89.45 472.54
76.08 469.17
83.35 435.21
92.16 477.78
58.42 475.89
85.06 483.9
88.91 476.2
83.33 462.16
88.49 471.05
96.88 484.71
73.86 446.34
65.17 469.02
69.41 432.12
81.23 467.28
54.07 429.66
89.17 469.49
78.85 485.87
84.44 481.95
83.02 479.03
90.66 434.5
75.29 464.9
82.6 452.71
45.4 429.74
66.33 457.09
45.33 446.77
67.12 460.76
84.14 471.95
80.81 453.29
49.13 441.61
87.29 464.73
52.95 464.68
68.92 430.59
85.21 438.01
88.56 479.08
55.09 436.39
48.64 447.07
87.85 479.91
87.42 489.05
62.75 463.17
94.86 471.26
63.54 480.49
69.46 473.78
74.66 455.5
80.57 446.27
84.7 482.2
72.6 452.48
52.63 464.48
82.01 438.1
75.22 445.6
51.05 442.43
80.1 436.67
81.58 466.56
65.94 457.29
86.74 487.03
62.57 464.93
57.37 466.0
88.91 469.52
72.26 428.88
74.98 474.3
71.19 461.06
62.82 465.57
55.31 467.67
71.57 466.99
59.16 463.72
40.8 443.78
67.63 445.23
97.2 464.43
91.16 484.36
64.81 442.16
85.61 464.11
93.42 462.48
77.36 477.49
67.03 437.04
89.45 457.09
54.84 450.6
53.48 465.78
54.47 427.1
43.36 459.81
48.92 447.36
81.22 488.92
60.35 433.36
75.98 483.35
74.24 469.53
85.21 476.96
68.46 440.75
78.31 462.55
89.25 448.04
68.57 455.24
67.38 494.75
53.6 444.58
91.69 484.82
78.21 442.9
94.19 485.46
37.19 457.81
83.53 481.92
79.45 443.23
99.9 474.29
50.39 430.46
74.33 455.71
83.66 438.34
89.48 485.83
64.59 452.82
75.68 435.04
64.18 451.21
70.09 465.81
70.58 458.42
99.28 470.22
81.89 449.24
87.99 471.43
88.21 473.26
91.29 452.82
52.72 432.69
67.5 444.13
92.05 467.21
63.23 445.98
90.88 436.91
74.91 455.01
85.39 437.11
96.98 477.06
56.28 441.71
65.62 495.76
55.32 445.63
88.88 464.72
39.49 438.03
54.59 434.78
57.19 444.67
45.06 452.24
40.62 450.92
90.21 436.53
90.91 435.53
74.96 440.01
54.21 443.1
63.62 427.49
50.04 436.25
51.23 440.74
82.71 443.54
92.04 459.42
90.67 439.66
91.14 464.15
70.43 459.1
66.63 455.68
86.95 469.08
96.69 478.02
70.88 456.8
47.14 441.13
63.36 463.88
60.58 430.45
54.3 449.18
65.09 447.89
48.16 431.59
81.51 447.5
68.57 475.58
73.16 453.24
80.88 446.4
85.4 476.81
75.77 474.1
75.76 450.71
70.21 433.62
55.53 465.14
80.48 445.18
72.41 474.12
83.25 483.91
82.12 486.68
94.75 464.98
95.79 481.4
86.02 479.2
78.29 463.86
82.23 472.3
52.86 446.51
60.66 437.71
62.77 458.94
65.54 437.91
95.4 490.76
51.78 439.66
63.62 463.27
83.09 473.99
61.81 433.38
68.24 459.01
72.41 471.44
79.64 471.91
66.95 465.15
91.98 446.66
64.96 438.15
88.93 447.14
85.62 472.32
84.0 441.68
89.41 440.04
67.4 444.82
76.75 457.26
89.06 428.83
64.02 449.07
64.12 435.21
81.22 471.03
100.13 465.56
58.07 442.83
63.42 460.3
83.32 474.25
74.28 477.97
71.91 472.16
85.8 456.08
55.58 452.41
69.7 463.71
63.79 433.72
86.59 456.4
48.28 448.43
80.42 481.6
98.58 457.07
72.83 451.0
56.07 440.28
67.13 437.47
84.49 443.57
68.37 426.6
86.07 470.87
83.82 478.37
74.86 453.92
53.52 470.22
80.61 434.54
43.56 442.89
89.35 479.03
97.28 476.06
80.49 473.88
88.9 451.75
49.7 439.2
68.64 439.7
98.58 463.6
82.12 447.47
78.32 447.92
86.04 471.08
91.36 437.55
81.02 448.27
76.05 431.69
71.81 449.09
82.13 448.79
74.27 460.21
71.32 479.28
74.59 483.11
84.44 450.75
43.39 437.97
87.2 459.76
77.11 457.75
82.81 469.33
85.67 433.28
95.58 444.64
74.46 463.1
90.02 460.91
81.93 479.35
86.29 449.23
85.43 474.51
71.66 435.02
71.33 435.45
67.72 452.38
84.23 480.41
74.44 478.96
71.48 468.87
56.3 434.01
68.13 466.36
68.35 435.28
85.23 486.46
79.78 468.19
98.98 468.37
72.87 474.19
90.93 440.32
72.94 485.32
72.3 464.27
77.74 479.25
78.29 430.4
69.1 447.49
55.79 438.23
75.09 492.09
88.98 475.36
64.26 452.56
25.89 427.84
59.18 433.95
67.48 435.27
89.85 454.62
50.62 472.17
83.97 452.42
96.51 472.17
80.09 481.83
84.02 458.78
61.97 447.5
88.51 463.4
81.83 473.57
90.63 433.72
73.37 431.85
77.5 433.47
57.46 432.84
51.91 436.6
79.14 490.23
69.24 477.16
41.85 441.06
95.1 440.86
74.53 477.94
64.85 474.47
57.07 470.67
62.9 447.31
72.21 466.8
65.99 430.91
73.67 434.75
88.59 469.52
61.19 438.9
49.37 429.56
48.65 432.92
56.18 442.87
71.56 466.59
87.46 479.61
70.02 471.08
61.2 433.37
60.54 443.92
71.82 443.5
44.8 439.89
67.32 434.66
78.77 487.57
96.02 464.64
90.56 470.92
83.19 444.39
68.45 442.48
99.19 449.61
88.11 435.02
79.29 458.67
87.76 461.74
82.56 438.31
79.24 462.38
67.24 460.56
58.98 439.22
84.22 444.64
89.12 430.34
50.07 430.46
98.86 456.79
95.79 468.82
70.06 448.51
41.71 470.77
86.55 465.74
71.97 430.21
96.4 449.23
91.73 461.89
91.62 445.72
59.14 466.13
92.56 448.71
83.71 469.25
55.43 450.56
74.91 464.46
89.41 471.13
69.81 461.52
86.01 451.09
86.05 431.51
80.98 469.8
63.62 442.28
64.16 458.67
75.63 462.4
80.21 453.54
59.7 444.38
47.6 440.52
63.78 433.62
89.37 481.96
70.55 452.75
84.42 481.28
80.62 439.03
52.56 435.75
83.26 436.03
70.64 445.6
89.95 462.65
51.53 438.66
84.83 447.32
59.68 484.55
97.88 476.8
85.03 480.34
47.38 440.63
48.08 459.48
79.65 490.78
83.39 483.56
82.18 429.38
51.71 440.27
77.33 445.34
72.81 447.43
84.26 439.91
73.23 459.27
79.73 478.89
62.32 466.7
78.58 463.5
79.88 436.21
65.87 443.94
80.28 439.63
71.33 460.95
70.33 448.69
57.73 444.63
99.46 473.51
68.61 462.56
95.91 451.76
80.42 491.81
51.01 429.52
74.08 437.9
80.73 467.54
54.71 449.97
73.75 436.62
88.43 477.68
74.66 447.26
70.04 439.76
74.64 437.49
85.11 455.14
82.97 485.5
55.59 444.1
49.9 432.33
89.99 471.23
91.61 463.89
82.95 445.54
92.62 446.09
59.64 445.12
63.0 443.31
85.38 484.16
85.23 477.76
52.08 430.28
38.05 446.48
71.98 481.03
90.66 466.07
83.14 447.47
65.22 455.93
91.67 479.62
66.04 455.06
78.19 475.06
54.47 438.89
67.26 432.7
72.56 452.6
82.15 451.75
86.32 430.66
88.17 491.9
72.78 439.82
69.58 460.73
69.86 449.7
65.37 439.42
52.37 439.84
79.63 485.86
83.14 458.1
88.25 479.92
55.85 458.29
52.55 489.45
62.74 434.0
90.08 431.24
54.5 439.5
49.88 467.46
48.96 429.27
86.77 452.1
96.66 472.41
70.84 442.14
83.83 441.0
68.22 463.07
82.22 445.71
87.9 483.16
66.83 440.45
85.36 481.83
60.9 467.6
83.51 450.88
52.96 425.5
73.02 451.87
43.11 428.94
61.41 439.86
65.32 433.44
93.68 438.23
42.39 436.95
68.18 470.19
89.18 484.66
64.79 430.81
43.48 433.37
80.4 453.02
72.43 453.5
80.0 463.09
45.65 464.56
63.02 452.12
74.39 470.9
57.77 450.89
76.8 445.04
67.39 444.72
73.96 460.38
72.75 446.8
71.69 465.05
84.98 484.13
87.68 488.27
50.36 447.09
68.11 452.02
66.27 455.55
89.72 480.99
61.07 467.68

Furthermore, you can interactively start playing with display on the full DataFrame!

display(powerPlantDF) // just as we did for the diamonds dataset
AT V AP RH PE
14.96 41.76 1024.07 73.17 463.26
25.18 62.96 1020.04 59.08 444.37
5.11 39.4 1012.16 92.14 488.56
20.86 57.32 1010.24 76.64 446.48
10.82 37.5 1009.23 96.62 473.9
26.27 59.44 1012.23 58.77 443.67
15.89 43.96 1014.02 75.24 467.35
9.48 44.71 1019.12 66.43 478.42
14.64 45.0 1021.78 41.25 475.98
11.74 43.56 1015.14 70.72 477.5
17.99 43.72 1008.64 75.04 453.02
20.14 46.93 1014.66 64.22 453.99
24.34 73.5 1011.31 84.15 440.29
25.71 58.59 1012.77 61.83 451.28
26.19 69.34 1009.48 87.59 433.99
21.42 43.79 1015.76 43.08 462.19
18.21 45.0 1022.86 48.84 467.54
11.04 41.74 1022.6 77.51 477.2
14.45 52.75 1023.97 63.59 459.85
13.97 38.47 1015.15 55.28 464.3
17.76 42.42 1009.09 66.26 468.27
5.41 40.07 1019.16 64.77 495.24
7.76 42.28 1008.52 83.31 483.8
27.23 63.9 1014.3 47.19 443.61
27.36 48.6 1003.18 54.93 436.06
27.47 70.72 1009.97 74.62 443.25
14.6 39.31 1011.11 72.52 464.16
7.91 39.96 1023.57 88.44 475.52
5.81 35.79 1012.14 92.28 484.41
30.53 65.18 1012.69 41.85 437.89
23.87 63.94 1019.02 44.28 445.11
26.09 58.41 1013.64 64.58 438.86
29.27 66.85 1011.11 63.25 440.98
27.38 74.16 1010.08 78.61 436.65
24.81 63.94 1018.76 44.51 444.26
12.75 44.03 1007.29 89.46 465.86
24.66 63.73 1011.4 74.52 444.37
16.38 47.45 1010.08 88.86 450.69
13.91 39.35 1014.69 75.51 469.02
23.18 51.3 1012.04 78.64 448.86
22.47 47.45 1007.62 76.65 447.14
13.39 44.85 1017.24 80.44 469.18
9.28 41.54 1018.33 79.89 482.8
11.82 42.86 1014.12 88.28 476.7
10.27 40.64 1020.63 84.6 474.99
22.92 63.94 1019.28 42.69 444.22
16.0 37.87 1020.24 78.41 461.33
21.22 43.43 1010.96 61.07 448.06
13.46 44.71 1014.51 50.0 474.6
9.39 40.11 1029.14 77.29 473.05
31.07 73.5 1010.58 43.66 432.06
12.82 38.62 1018.71 83.8 467.41
32.57 78.92 1011.6 66.47 430.12
8.11 42.18 1014.82 93.09 473.62
13.92 39.39 1012.94 80.52 471.81
23.04 59.43 1010.23 68.99 442.99
27.31 64.44 1014.65 57.27 442.77
5.91 39.33 1010.18 95.53 491.49
25.26 61.08 1013.68 71.72 447.46
27.97 58.84 1002.25 57.88 446.11
26.08 52.3 1007.03 63.34 442.44
29.01 65.71 1013.61 48.07 446.22
12.18 40.1 1016.67 91.87 471.49
13.76 45.87 1008.89 87.27 463.5
25.5 58.79 1016.02 64.4 440.01
28.26 65.34 1014.56 43.4 441.03
21.39 62.96 1019.49 72.24 452.68
7.26 40.69 1020.43 90.22 474.91
10.54 34.03 1018.71 74.0 478.77
27.71 74.34 998.14 71.85 434.2
23.11 68.3 1017.83 86.62 437.91
7.51 41.01 1024.61 97.41 477.61
26.46 74.67 1016.65 84.44 431.65
29.34 74.34 998.58 81.55 430.57
10.32 42.28 1008.82 75.66 481.09
22.74 61.02 1009.56 79.41 445.56
13.48 39.85 1012.71 58.91 475.74
25.52 69.75 1010.36 90.06 435.12
21.58 67.25 1017.39 79.0 446.15
27.66 76.86 1001.31 69.47 436.64
26.96 69.45 1013.89 51.47 436.69
12.29 42.18 1016.53 83.13 468.75
15.86 43.02 1012.18 40.33 466.6
13.87 45.08 1024.42 81.69 465.48
24.09 73.68 1014.93 94.55 441.34
20.45 69.45 1012.53 91.81 441.83
15.07 39.3 1019.0 63.62 464.7
32.72 69.75 1009.6 49.35 437.99
18.23 58.96 1015.55 69.61 459.12
35.56 68.94 1006.56 38.75 429.69
18.36 51.43 1010.57 90.17 459.8
26.35 64.05 1009.81 81.24 433.63
25.92 60.95 1014.62 48.46 442.84
8.01 41.66 1014.49 76.72 485.13
19.63 52.72 1025.09 51.16 459.12
20.02 67.32 1012.05 76.34 445.31
10.08 40.72 1022.7 67.3 480.8
27.23 66.48 1005.23 52.38 432.55
23.37 63.77 1013.42 76.44 443.86
18.74 59.21 1018.3 91.55 449.77
14.81 43.69 1017.19 71.9 470.71
23.1 51.3 1011.93 80.05 452.17
10.72 41.38 1021.6 63.77 478.29
29.46 71.94 1006.96 62.26 428.54
8.1 40.64 1020.66 89.04 478.27
27.29 62.66 1007.63 58.02 439.58
17.1 49.69 1005.53 81.82 457.32
11.49 44.2 1018.79 91.14 475.51
23.69 65.59 1010.85 88.92 439.66
13.51 40.89 1011.03 84.83 471.99
9.64 39.35 1015.1 91.76 479.81
25.65 78.92 1010.83 86.56 434.78
21.59 61.87 1011.18 57.21 446.58
27.98 58.33 1013.92 54.25 437.76
18.8 39.72 1001.24 63.8 459.36
18.28 44.71 1016.99 33.71 462.28
13.55 43.48 1016.08 67.25 464.33
22.99 46.21 1010.71 60.11 444.36
23.94 59.39 1014.32 74.55 438.64
13.74 34.03 1018.69 67.34 470.49
21.3 41.1 1001.86 42.75 455.13
27.54 66.93 1017.06 55.2 450.22
24.81 63.73 1009.34 83.61 440.43
4.97 42.85 1014.02 88.78 482.98
15.22 50.88 1014.19 100.12 460.44
23.88 54.2 1012.81 64.52 444.97
33.01 68.67 1005.2 51.41 433.94
25.98 73.18 1012.28 85.78 439.73
28.18 73.88 1005.89 75.41 434.48
21.67 60.84 1017.93 81.63 442.33
17.67 45.09 1014.26 51.92 457.67
21.37 57.76 1018.8 70.12 454.66
28.69 67.25 1017.71 53.83 432.21
16.61 43.77 1012.25 77.23 457.66
27.91 63.76 1010.27 65.67 435.21
20.97 47.43 1007.64 71.18 448.22
10.8 41.66 1013.79 81.96 475.51
20.61 62.91 1013.24 79.54 446.53
25.45 57.32 1011.7 47.09 441.3
30.16 69.34 1007.67 57.69 433.54
4.99 39.04 1020.45 78.89 472.52
10.51 44.78 1012.59 85.29 474.77
33.79 69.05 1001.62 40.13 435.1
21.34 59.8 1016.92 77.06 450.74
23.4 65.06 1014.32 67.38 442.7
32.21 68.14 1003.34 62.44 426.56
14.26 42.32 1016.0 77.43 463.71
27.71 66.93 1016.85 58.77 447.06
21.95 57.76 1018.02 67.72 452.27
25.76 63.94 1018.49 42.14 445.78
23.68 68.3 1017.93 84.16 438.65
8.28 40.77 1011.55 89.79 480.15
23.44 62.52 1016.46 67.21 447.19
25.32 48.41 1008.47 72.14 443.04
3.94 39.9 1008.06 97.49 488.81
17.3 57.76 1016.26 87.74 455.75
18.2 49.39 1018.83 96.3 455.86
21.43 46.97 1013.94 61.25 457.68
11.16 40.05 1014.95 88.38 479.11
30.38 74.16 1007.44 74.77 432.84
23.36 62.52 1016.18 68.18 448.37
21.69 47.45 1007.56 77.2 447.06
23.62 49.21 1014.1 49.54 443.53
21.87 61.45 1011.13 92.22 445.21
29.25 66.51 1015.53 33.65 441.7
20.03 66.86 1013.05 64.59 450.93
18.14 49.78 1002.95 100.09 451.44
24.23 56.89 1012.32 68.04 441.29
18.11 44.85 1014.48 48.94 458.85
6.57 43.65 1018.24 74.47 481.46
12.56 43.41 1016.93 81.02 467.19
13.4 41.58 1020.5 71.17 461.54
27.1 52.84 1006.28 53.85 439.08
14.28 42.74 1028.79 70.67 467.22
16.29 44.34 1019.49 59.36 468.8
31.24 71.98 1004.66 57.17 426.93
10.57 37.73 1024.36 70.29 474.65
13.8 44.21 1022.93 83.37 468.97
25.3 71.58 1010.18 87.36 433.97
18.06 50.16 1009.52 100.09 450.53
25.42 59.04 1011.98 68.78 444.51
15.07 40.69 1015.29 70.98 469.03
11.75 71.14 1019.36 75.68 466.56
20.23 52.05 1012.15 47.49 457.57
27.31 59.54 1006.24 71.99 440.13
28.57 69.84 1003.57 66.55 433.24
17.9 43.72 1008.64 74.73 452.55
23.83 71.37 1002.04 64.78 443.29
27.92 74.99 1005.47 75.13 431.76
17.34 44.78 1007.81 56.38 454.97
17.94 63.07 1012.42 94.35 456.7
6.4 39.9 1007.75 86.55 486.03
11.78 39.96 1011.37 82.95 472.79
20.28 57.25 1010.12 88.42 452.03
21.04 54.2 1012.26 85.61 443.41
25.11 67.32 1014.49 58.39 441.93
30.28 70.98 1007.51 74.28 432.64
8.14 36.24 1013.15 87.85 480.25
16.86 39.63 1004.47 83.5 466.68
6.25 40.07 1020.19 65.24 494.39
22.35 54.42 1012.46 75.01 454.72
17.98 56.85 1012.28 84.52 448.71
21.19 42.48 1013.43 80.52 469.76
20.94 44.89 1009.64 75.14 450.71
24.23 58.79 1009.8 75.75 444.01
19.18 58.2 1017.46 76.72 453.2
20.88 57.85 1012.39 85.47 450.87
23.67 63.86 1019.67 57.95 441.73
14.12 39.52 1018.41 78.32 465.09
25.23 64.63 1020.59 52.2 447.28
6.54 39.33 1011.54 93.69 491.16
20.08 62.52 1017.99 75.74 450.98
24.67 63.56 1013.75 67.56 446.3
27.82 79.74 1008.37 69.46 436.48
15.55 42.03 1017.41 74.58 460.84
24.26 69.51 1013.43 53.23 442.56
13.45 41.49 1020.19 88.72 467.3
11.06 40.64 1021.47 96.16 479.13
24.91 52.3 1008.72 68.26 441.15
22.39 59.04 1011.78 86.39 445.52
11.95 40.69 1015.62 85.34 475.4
14.85 40.69 1014.91 72.64 469.3
10.11 41.62 1017.17 97.82 463.57
23.67 68.67 1006.71 77.22 445.32
16.14 44.21 1020.36 80.59 461.03
15.11 43.13 1014.99 46.91 466.74
24.14 59.87 1018.47 57.76 444.04
30.08 67.25 1017.6 53.09 434.01
14.77 44.9 1020.5 84.31 465.23
27.6 69.34 1009.63 71.58 440.6
13.89 44.84 1023.66 92.97 466.74
26.85 75.6 1017.43 74.55 433.48
12.41 40.96 1023.36 78.96 473.59
13.08 41.74 1020.75 64.44 474.81
18.93 44.06 1017.58 68.23 454.75
20.5 49.69 1009.6 70.81 452.94
30.72 69.13 1009.94 61.66 435.83
7.55 39.22 1014.53 77.76 482.19
13.49 44.47 1030.46 69.49 466.66
15.62 40.12 1013.03 96.26 462.59
24.8 64.63 1020.69 55.74 447.82
10.03 41.62 1014.55 95.61 462.73
22.43 63.21 1012.06 84.75 447.98
14.95 39.31 1009.15 75.3 462.72
24.78 58.46 1016.82 67.5 442.42
23.2 48.41 1008.64 80.92 444.69
14.01 39.0 1016.73 79.23 466.7
19.4 64.63 1020.38 81.1 453.84
30.15 67.32 1013.83 32.8 436.92
6.91 36.08 1021.82 84.31 486.37
29.04 60.07 1015.42 46.15 440.43
26.02 63.07 1010.94 53.96 446.82
5.89 39.48 1005.11 59.83 484.91
26.52 71.64 1008.27 75.3 437.76
28.53 68.08 1013.27 42.53 438.91
16.59 39.54 1007.97 70.58 464.19
22.95 67.79 1009.89 91.69 442.19
23.96 47.43 1008.38 63.55 446.86
17.48 44.2 1018.89 61.51 457.15
6.69 43.65 1020.14 69.55 482.57
10.25 41.26 1007.44 98.08 476.03
28.87 72.58 1008.69 79.34 428.89
12.04 40.23 1018.07 81.28 472.7
22.58 52.3 1009.04 78.99 445.6
15.12 52.05 1014.63 80.38 464.78
25.48 58.95 1017.02 51.16 440.42
27.87 70.79 1003.96 72.17 428.41
23.72 70.47 1010.65 75.39 438.5
25.0 59.43 1007.84 68.91 438.28
8.42 40.64 1022.35 96.38 476.29
22.46 58.49 1011.5 70.54 448.46
29.92 57.19 1008.62 45.8 438.99
11.68 39.22 1017.9 57.95 471.8
14.04 42.44 1012.74 81.89 471.81
19.86 59.14 1016.12 69.32 449.82
25.99 68.08 1013.13 59.14 442.14
23.42 58.79 1009.74 81.54 441.46
10.6 40.22 1011.37 85.81 477.62
20.97 61.87 1011.45 65.41 446.76
14.14 39.82 1012.46 81.15 472.52
8.56 40.71 1021.27 95.87 471.58
24.86 72.39 1001.15 90.24 440.85
29.0 77.54 1011.33 75.13 431.37
27.59 71.97 1008.64 88.22 437.33
10.45 40.71 1015.68 81.48 469.22
8.51 40.78 1023.51 89.84 471.11
29.82 66.51 1010.98 43.57 439.17
22.56 62.26 1012.11 63.16 445.33
11.38 39.22 1018.62 57.14 473.71
20.25 57.76 1016.28 77.76 452.66
22.42 59.43 1007.12 90.56 440.99
14.85 38.91 1014.48 60.98 467.42
25.62 58.82 1010.02 70.31 444.14
19.85 56.53 1020.57 74.05 457.17
13.67 54.3 1015.92 75.42 467.87
24.39 70.72 1009.78 82.25 442.04
16.07 44.58 1019.52 67.95 471.36
11.6 39.1 1009.81 100.09 460.7
31.38 70.83 1010.35 47.28 431.33
29.91 76.86 998.59 72.41 432.6
19.67 59.39 1014.07 77.67 447.61
27.18 64.79 1016.27 63.7 443.87
21.39 52.3 1009.2 79.77 446.87
10.45 41.01 1020.57 93.84 465.74
19.46 56.89 1014.02 84.95 447.86
23.55 62.96 1020.16 70.16 447.65
23.35 63.47 1011.78 84.24 437.87
9.26 41.66 1016.87 73.32 483.51
10.3 41.46 1018.21 86.17 479.65
20.94 58.16 1016.88 65.43 455.16
23.13 71.25 1002.49 94.59 431.91
12.77 41.5 1014.13 86.8 470.68
28.29 69.13 1009.29 58.18 429.28
19.13 59.21 1018.32 89.66 450.81
24.44 73.5 1011.49 87.39 437.73
20.32 44.6 1015.16 36.35 460.21
20.54 69.05 1001.6 79.62 442.86
12.16 45.0 1021.51 50.52 482.99
28.09 65.27 1013.27 51.96 440.0
9.25 41.82 1033.25 74.73 478.48
21.75 49.82 1015.01 78.33 455.28
23.7 66.56 1002.07 85.19 436.94
16.22 37.87 1022.36 83.13 461.06
24.75 69.45 1013.97 53.49 438.28
10.48 39.58 1011.81 88.86 472.61
29.53 70.79 1003.7 60.89 426.85
12.59 39.72 1017.76 61.14 470.18
23.5 54.42 1012.31 68.29 455.38
29.01 66.56 1006.44 57.62 428.32
9.75 42.49 1010.57 83.63 480.35
19.55 56.53 1020.2 78.1 455.56
21.05 58.33 1013.14 66.34 447.66
24.72 68.67 1006.74 79.02 443.06
21.19 58.86 1014.19 68.96 452.43
10.77 41.54 1019.94 71.13 477.81
28.68 73.77 1004.72 87.01 431.66
29.87 73.91 1004.53 74.3 431.8
22.99 68.67 1006.65 77.62 446.67
24.66 60.29 1018.0 59.56 445.26
32.63 69.89 1013.85 41.66 425.72
31.38 72.29 1008.73 73.27 430.58
23.87 60.27 1018.94 77.16 439.86
25.6 59.15 1013.31 67.02 441.11
27.62 71.14 1011.6 52.8 434.72
30.1 67.45 1014.23 39.04 434.01
12.19 41.17 1019.43 65.47 475.64
13.11 41.58 1020.43 74.32 460.44
28.29 68.67 1005.46 69.22 436.4
13.45 40.73 1018.7 93.88 461.03
10.98 41.54 1019.94 69.83 479.08
26.48 69.14 1009.31 84.11 435.76
13.07 45.51 1015.22 78.65 460.14
25.56 75.6 1017.37 69.31 442.2
22.68 50.78 1008.83 70.3 447.69
28.86 73.67 1006.65 68.23 431.15
22.7 63.56 1014.32 71.76 445.0
27.89 73.21 1001.32 85.88 431.59
13.78 44.47 1027.94 71.09 467.22
28.14 51.43 1012.16 52.67 445.33
11.8 45.09 1013.21 89.68 470.57
10.71 39.61 1018.72 73.66 473.77
24.54 60.29 1017.42 58.94 447.67
11.54 40.05 1014.78 87.05 474.29
29.47 71.32 1008.07 67.0 437.14
29.24 69.05 1003.12 43.18 432.56
14.51 41.79 1009.72 80.62 459.14
22.91 60.07 1016.03 59.72 446.19
27.02 71.77 1006.38 72.1 428.1
13.49 44.47 1030.18 69.15 468.46
30.24 66.75 1017.95 55.66 435.02
23.19 48.6 1002.38 61.19 445.52
17.73 40.55 1003.36 74.62 462.69
18.62 61.27 1019.26 73.35 455.75
12.85 40.0 1015.89 68.85 463.74
32.33 69.68 1011.95 39.89 439.79
25.09 58.95 1016.99 53.16 443.26
29.45 69.13 1009.3 52.97 432.04
16.91 43.96 1013.32 79.87 465.86
14.09 45.87 1009.05 84.09 465.6
10.73 25.36 1009.35 100.15 469.43
23.2 49.3 1003.4 79.77 440.75
8.21 38.91 1015.82 88.99 481.32
9.3 40.56 1022.64 76.14 479.87
16.97 39.16 1005.7 69.13 458.59
23.69 71.97 1009.62 93.03 438.62
25.13 59.44 1012.38 77.92 445.59
9.86 43.56 1015.13 74.89 481.87
11.33 41.5 1013.58 88.7 475.01
26.95 48.41 1008.53 62.94 436.54
15.0 40.66 1016.28 89.62 456.63
20.76 62.52 1015.63 81.04 451.69
14.29 39.59 1010.93 94.53 463.04
19.74 67.71 1007.68 64.02 446.1
26.68 59.92 1009.94 70.57 438.67
14.24 41.4 1019.7 70.32 466.88
21.98 48.41 1008.42 84.86 444.6
22.75 59.39 1015.4 81.41 440.26
8.34 40.96 1023.28 89.45 483.92
11.8 41.2 1017.18 82.71 475.19
8.81 44.68 1023.06 93.93 479.24
30.05 73.68 1014.95 70.6 434.92
16.01 65.46 1014.0 87.68 454.16
21.75 58.79 1012.42 87.58 447.58
13.94 41.26 1021.67 74.4 467.9
29.25 69.13 1010.27 67.35 426.29
22.33 45.87 1007.8 63.61 447.02
16.43 41.79 1005.47 76.89 455.85
11.5 40.22 1010.31 78.08 476.46
23.53 68.94 1007.53 69.17 437.48
21.86 49.21 1014.61 53.31 452.77
6.17 39.33 1012.57 93.32 491.54
30.19 64.79 1017.22 42.47 438.41
11.67 41.93 1019.81 82.58 476.1
15.34 36.99 1007.87 94.59 464.58
11.5 40.78 1023.91 86.31 467.74
25.53 57.17 1010.0 72.57 442.12
21.27 57.5 1014.53 80.76 453.34
28.37 69.13 1010.44 71.93 425.29
28.39 51.43 1011.74 47.54 449.63
13.78 45.78 1025.27 95.72 462.88
14.6 42.32 1015.71 77.03 464.67
5.1 35.57 1027.17 80.49 489.96
7.0 38.08 1020.27 77.67 482.38
26.3 77.95 1009.45 78.72 437.95
30.56 71.98 1004.74 58.77 429.2
21.09 46.63 1013.03 74.8 453.34
28.21 70.02 1010.58 51.34 442.47
15.84 49.69 1015.14 90.41 462.6
10.03 40.96 1024.57 91.1 478.79
20.37 52.05 1012.34 62.57 456.11
21.19 50.16 1005.81 84.27 450.33
33.73 69.88 1007.21 42.93 434.83
29.87 73.68 1015.1 40.96 433.43
19.62 62.96 1020.76 76.53 456.02
9.93 40.67 1018.08 69.74 485.23
9.43 37.14 1013.03 74.99 473.57
14.24 39.58 1011.17 70.45 469.94
12.97 49.83 1008.69 91.49 452.07
7.6 41.04 1021.82 88.97 475.32
8.39 36.24 1013.39 89.13 480.69
25.41 48.06 1013.12 46.52 444.01
18.43 56.03 1020.41 60.55 465.17
10.31 39.82 1012.87 88.71 480.61
11.29 41.5 1013.39 89.15 476.04
22.61 49.3 1003.51 83.02 441.76
29.34 71.98 1005.19 75.19 428.24
18.87 67.71 1004.0 87.35 444.77
13.21 45.87 1008.58 85.66 463.1
11.3 44.6 1018.19 91.66 470.5
29.23 72.99 1007.04 63.47 431.0
27.76 69.4 1004.27 72.25 430.68
29.26 67.17 1006.6 70.58 436.42
25.72 49.82 1016.19 60.1 452.33
23.43 63.94 1010.64 89.29 440.16
25.6 63.76 1010.18 67.43 435.75
22.3 44.57 1008.48 67.58 449.74
27.91 72.24 1010.74 70.8 430.73
30.35 77.17 1009.55 63.62 432.75
21.78 47.43 1007.88 66.68 446.79
7.19 41.39 1018.12 90.76 486.35
20.88 59.8 1015.66 75.34 453.18
24.19 50.23 1015.73 59.77 458.31
9.98 41.54 1019.7 80.79 480.26
23.47 51.3 1011.89 74.1 448.65
26.35 49.5 1012.67 41.34 458.41
29.89 64.69 1006.37 58.78 435.39
19.29 50.16 1010.49 97.78 450.21
17.48 43.14 1018.68 74.85 459.59
25.21 75.6 1017.19 69.84 445.84
23.3 48.78 1018.17 75.36 441.08
15.42 37.85 1009.89 85.8 467.33
21.44 63.09 1016.56 90.11 444.19
29.45 68.27 1007.96 61.63 432.96
29.69 47.93 1002.85 44.76 438.09
15.52 36.99 1006.86 89.7 467.9
11.47 43.67 1012.68 72.51 475.72
9.77 34.69 1027.72 74.98 477.51
22.6 69.84 1006.37 79.59 435.13
8.24 39.61 1017.99 78.42 477.9
17.01 44.2 1019.18 61.23 457.26
19.64 44.6 1015.88 47.56 467.53
10.61 41.58 1021.08 93.06 465.15
12.04 40.1 1014.42 89.65 474.28
29.19 65.71 1013.85 50.5 444.49
21.75 45.09 1014.15 44.84 452.84
23.66 77.54 1008.5 85.32 435.38
27.05 75.33 1003.88 82.94 433.57
29.63 69.71 1009.04 67.26 435.27
18.2 39.63 1005.35 79.05 468.49
32.22 70.8 1009.9 62.03 433.07
26.88 73.56 1004.85 94.36 430.63
29.05 65.74 1013.29 60.02 440.74
8.9 39.96 1026.31 95.46 474.49
18.93 48.6 1005.72 84.92 449.74
27.49 63.76 1010.09 62.8 436.73
23.1 70.79 1006.53 90.81 434.58
11.22 43.13 1017.24 80.9 473.93
31.97 79.74 1007.03 55.84 435.99
13.32 43.22 1009.45 75.3 466.83
31.68 68.24 1005.29 37.34 427.22
23.69 63.77 1013.39 79.5 444.07
13.83 41.49 1020.11 87.29 469.57
18.32 66.51 1015.18 81.5 459.89
11.05 40.71 1024.91 76.42 479.59
22.03 64.69 1007.21 75.75 440.92
10.23 41.46 1020.45 84.95 480.87
23.92 66.54 1009.93 62.37 441.9
29.38 69.68 1011.35 49.25 430.2
17.35 42.86 1014.62 74.16 465.16
9.81 44.45 1021.19 90.55 471.32
4.97 40.64 1020.91 94.28 485.43
5.15 40.07 1012.27 63.31 495.35
21.54 58.49 1010.85 78.9 449.12
7.94 42.02 1006.22 90.97 480.53
18.77 50.66 1014.89 87.34 457.07
21.69 69.94 1010.7 80.8 443.67
10.07 44.68 1023.44 90.95 477.52
13.83 39.64 1012.52 69.97 472.95
10.45 39.69 1003.92 89.45 472.54
11.56 40.71 1015.85 76.08 469.17
23.64 70.04 1011.09 83.35 435.21
10.48 40.22 1004.81 92.16 477.78
13.09 39.85 1012.86 58.42 475.89
10.67 40.23 1017.75 85.06 483.9
12.57 39.16 1016.53 88.91 476.2
14.45 43.34 1015.47 83.33 462.16
14.22 37.85 1011.24 88.49 471.05
6.97 41.26 1010.6 96.88 484.71
20.61 63.86 1015.43 73.86 446.34
14.67 42.28 1007.21 65.17 469.02
29.06 72.86 1004.23 69.41 432.12
14.38 40.1 1015.51 81.23 467.28
32.51 69.98 1013.29 54.07 429.66
11.79 45.09 1013.16 89.17 469.49
8.65 40.56 1023.23 78.85 485.87
9.75 40.81 1026.0 84.44 481.95
9.11 40.02 1031.1 83.02 479.03
23.39 69.13 1010.99 90.66 434.5
14.3 54.3 1015.16 75.29 464.9
17.49 63.94 1020.02 82.6 452.71
31.1 69.51 1010.84 45.4 429.74
19.77 56.65 1020.67 66.33 457.09
28.61 72.29 1011.61 45.33 446.77
13.52 41.48 1014.46 67.12 460.76
13.52 40.83 1008.31 84.14 471.95
17.57 46.21 1014.09 80.81 453.29
28.18 60.07 1016.34 49.13 441.61
14.29 46.18 1017.01 87.29 464.73
18.12 43.69 1016.91 52.95 464.68
31.27 73.91 1003.72 68.92 430.59
26.24 77.95 1014.19 85.21 438.01
7.44 41.04 1021.84 88.56 479.08
29.78 74.78 1009.28 55.09 436.39
23.37 65.46 1016.25 48.64 447.07
10.62 39.58 1011.9 87.85 479.91
5.84 43.02 1013.88 87.42 489.05
14.51 53.82 1016.46 62.75 463.17
11.31 42.02 1001.18 94.86 471.26
11.25 40.67 1011.64 63.54 480.49
9.18 39.42 1025.41 69.46 473.78
19.82 58.16 1016.76 74.66 455.5
24.77 58.41 1013.78 80.57 446.27
9.66 41.06 1021.21 84.7 482.2
21.96 59.8 1016.72 72.6 452.48
18.59 43.14 1011.92 52.63 464.48
24.75 69.89 1015.29 82.01 438.1
24.37 63.47 1012.77 75.22 445.6
29.6 67.79 1010.37 51.05 442.43
25.32 61.25 1011.56 80.1 436.67
16.15 41.85 1016.54 81.58 466.56
15.74 71.14 1019.65 65.94 457.29
5.97 36.25 1029.65 86.74 487.03
15.84 52.72 1026.45 62.57 464.93
14.84 44.63 1019.28 57.37 466.0
12.25 48.79 1017.44 88.91 469.52
27.38 70.04 1011.18 72.26 428.88
8.76 41.48 1018.49 74.98 474.3
15.54 39.31 1009.69 71.19 461.06
18.71 39.39 1014.09 62.82 465.57
13.06 41.78 1012.3 55.31 467.67
12.72 40.71 1016.02 71.57 466.99
19.83 39.39 1013.73 59.16 463.72
27.23 49.16 1004.03 40.8 443.78
24.27 68.28 1005.43 67.63 445.23
11.8 40.66 1017.13 97.2 464.43
6.76 36.25 1028.31 91.16 484.36
25.99 63.07 1012.5 64.81 442.16
16.3 39.63 1004.64 85.61 464.11
16.5 49.39 1018.35 93.42 462.48
10.59 42.49 1009.59 77.36 477.49
26.05 65.59 1012.78 67.03 437.04
19.5 40.79 1003.8 89.45 457.09
22.21 45.01 1012.22 54.84 450.6
17.86 45.0 1023.25 53.48 465.78
29.96 70.04 1010.15 54.47 427.1
19.08 44.63 1020.14 43.36 459.81
23.59 47.43 1006.64 48.92 447.36
3.38 39.64 1011.0 81.22 488.92
26.39 66.49 1012.96 60.35 433.36
8.99 39.04 1021.99 75.98 483.35
10.91 41.04 1026.57 74.24 469.53
13.08 39.82 1012.27 85.21 476.96
23.95 58.46 1017.5 68.46 440.75
15.64 43.71 1024.51 78.31 462.55
18.78 54.2 1012.05 89.25 448.04
20.65 50.59 1016.22 68.57 455.24
4.96 40.07 1011.8 67.38 494.75
23.51 57.32 1012.55 53.6 444.58
5.99 35.79 1011.56 91.69 484.82
23.65 66.05 1019.6 78.21 442.9
5.17 39.33 1009.68 94.19 485.46
26.38 49.5 1012.82 37.19 457.81
6.02 43.65 1013.85 83.53 481.92
23.2 61.02 1009.63 79.45 443.23
8.57 39.69 1000.91 99.9 474.29
30.72 71.58 1009.98 50.39 430.46
21.52 50.66 1013.56 74.33 455.71
22.93 62.26 1011.25 83.66 438.34
5.71 41.31 1003.24 89.48 485.83
18.62 44.06 1017.76 64.59 452.82
27.88 68.94 1007.68 75.68 435.04
22.32 59.8 1016.82 64.18 451.21
14.55 42.74 1028.41 70.09 465.81
17.83 44.92 1025.04 70.58 458.42
9.68 39.96 1026.09 99.28 470.22
19.41 49.39 1020.84 81.89 449.24
13.22 44.92 1023.84 87.99 471.43
12.24 44.92 1023.74 88.21 473.26
19.21 58.49 1011.7 91.29 452.82
29.74 70.32 1008.1 52.72 432.69
23.28 60.84 1017.91 67.5 444.13
8.02 41.92 1029.8 92.05 467.21
22.47 48.6 1002.33 63.23 445.98
27.51 73.77 1002.42 90.88 436.91
17.51 44.9 1009.05 74.91 455.01
23.22 66.56 1002.47 85.39 437.11
11.73 40.64 1020.68 96.98 477.06
21.19 67.71 1006.65 56.28 441.71
5.48 40.07 1019.63 65.62 495.76
24.26 66.44 1011.33 55.32 445.63
12.32 41.62 1012.88 88.88 464.72
31.26 68.94 1005.94 39.49 438.03
32.09 72.86 1003.47 54.59 434.78
24.98 60.32 1015.63 57.19 444.67
27.48 61.41 1012.2 45.06 452.24
21.04 45.09 1014.19 40.62 450.92
27.75 70.4 1006.65 90.21 436.53
22.79 71.77 1005.75 90.91 435.53
24.22 68.51 1013.23 74.96 440.01
27.06 64.45 1008.72 54.21 443.1
29.25 71.94 1007.18 63.62 427.49
26.86 68.08 1012.99 50.04 436.25
29.64 67.79 1009.99 51.23 440.74
19.92 63.31 1015.02 82.71 443.54
18.5 51.43 1010.82 92.04 459.42
23.71 60.23 1009.76 90.67 439.66
14.39 44.84 1023.55 91.14 464.15
19.3 56.65 1020.55 70.43 459.1
24.65 52.36 1014.76 66.63 455.68
13.5 45.51 1015.33 86.95 469.08
9.82 41.26 1007.71 96.69 478.02
18.4 44.06 1017.36 70.88 456.8
28.12 44.89 1009.18 47.14 441.13
17.15 43.69 1017.05 63.36 463.88
30.69 73.67 1006.14 60.58 430.45
28.82 65.71 1014.24 54.3 449.18
21.3 48.92 1010.92 65.09 447.89
30.58 70.04 1010.4 48.16 431.59
21.17 52.3 1009.36 81.51 447.5
9.87 41.82 1033.04 68.57 475.58
22.18 59.8 1016.77 73.16 453.24
24.39 63.21 1012.59 80.88 446.4
10.73 44.92 1025.1 85.4 476.81
9.38 40.46 1019.29 75.77 474.1
20.27 57.76 1016.66 75.76 450.71
24.82 66.48 1006.4 70.21 433.62
16.55 41.66 1011.45 55.53 465.14
20.73 59.87 1019.08 80.48 445.18
9.51 39.22 1015.3 72.41 474.12
8.63 43.79 1016.08 83.25 483.91
6.48 40.27 1010.55 82.12 486.68
14.95 43.52 1022.43 94.75 464.98
5.76 45.87 1010.83 95.79 481.4
10.94 39.04 1021.81 86.02 479.2
15.87 41.16 1005.85 78.29 463.86
12.42 38.25 1012.76 82.23 472.3
29.12 58.84 1001.31 52.86 446.51
29.12 51.43 1005.93 60.66 437.71
19.08 41.1 1001.96 62.77 458.94
31.06 67.17 1007.62 65.54 437.91
5.72 39.33 1009.96 95.4 490.76
26.52 65.06 1013.4 51.78 439.66
13.84 44.9 1007.58 63.62 463.27
13.03 39.52 1016.68 83.09 473.99
25.94 66.49 1012.83 61.81 433.38
16.64 53.82 1015.13 68.24 459.01
14.13 40.75 1016.05 72.41 471.44
13.65 39.28 1012.97 79.64 471.91
14.5 44.47 1028.2 66.95 465.15
19.8 51.19 1008.25 91.98 446.66
25.2 63.76 1009.78 64.96 438.15
20.66 51.19 1008.81 88.93 447.14
12.07 43.71 1025.53 85.62 472.32
25.64 70.72 1010.16 84.0 441.68
23.33 72.99 1009.33 89.41 440.04
29.41 64.05 1009.82 67.4 444.82
16.6 53.16 1014.5 76.75 457.26
27.53 72.58 1009.13 89.06 428.83
20.62 43.43 1009.93 64.02 449.07
26.02 71.94 1009.38 64.12 435.21
12.75 44.2 1017.59 81.22 471.03
12.87 48.04 1012.47 100.13 465.56
25.77 62.96 1019.86 58.07 442.83
14.84 41.48 1017.26 63.42 460.3
7.41 40.71 1023.07 83.32 474.25
8.87 41.82 1033.3 74.28 477.97
9.69 40.46 1019.1 71.91 472.16
16.17 46.97 1014.22 85.8 456.08
26.24 49.82 1014.9 55.58 452.41
13.78 43.22 1011.31 69.7 463.71
26.3 67.07 1006.26 63.79 433.72
17.37 57.76 1016.0 86.59 456.4
23.6 48.98 1015.41 48.28 448.43
8.3 36.08 1020.63 80.42 481.6
18.86 42.18 1001.16 98.58 457.07
22.12 49.39 1019.8 72.83 451.0
28.41 75.6 1018.48 56.07 440.28
29.42 71.32 1002.26 67.13 437.47
18.61 67.71 1004.07 84.49 443.57
27.57 69.84 1004.91 68.37 426.6
12.83 41.5 1013.12 86.07 470.87
9.64 39.85 1012.9 83.82 478.37
19.13 58.66 1013.32 74.86 453.92
15.92 40.56 1020.79 53.52 470.22
24.64 72.24 1011.37 80.61 434.54
27.62 63.9 1013.11 43.56 442.89
8.9 36.24 1013.29 89.35 479.03
9.55 43.99 1020.5 97.28 476.06
10.57 36.71 1022.62 80.49 473.88
19.8 57.25 1010.84 88.9 451.75
25.63 56.85 1012.68 49.7 439.2
24.7 58.46 1015.58 68.64 439.7
15.26 46.18 1013.68 98.58 463.6
20.06 52.84 1004.21 82.12 447.47
19.84 56.89 1013.23 78.32 447.92
11.49 44.63 1020.44 86.04 471.08
23.74 72.43 1007.99 91.36 437.55
22.62 51.3 1012.36 81.02 448.27
29.53 72.39 998.47 76.05 431.69
21.32 48.14 1016.57 71.81 449.09
20.3 58.46 1015.93 82.13 448.79
16.97 44.92 1025.21 74.27 460.21
12.07 41.17 1013.54 71.32 479.28
7.46 41.82 1032.67 74.59 483.11
19.2 54.2 1011.46 84.44 450.75
28.64 66.54 1010.43 43.39 437.97
13.56 41.48 1008.53 87.2 459.76
17.4 44.9 1020.5 77.11 457.75
14.08 40.1 1015.48 82.81 469.33
27.11 69.75 1009.74 85.67 433.28
20.92 70.02 1010.23 95.58 444.64
16.18 44.9 1021.3 74.46 463.1
15.57 44.68 1022.01 90.02 460.91
10.37 39.04 1023.95 81.93 479.35
19.6 59.21 1017.65 86.29 449.23
9.22 40.92 1021.83 85.43 474.51
27.76 72.99 1007.81 71.66 435.02
28.68 70.72 1009.43 71.33 435.45
20.95 48.14 1013.3 67.72 452.38
9.06 39.3 1019.73 84.23 480.41
9.21 39.72 1019.54 74.44 478.96
13.65 42.74 1026.58 71.48 468.87
31.79 76.2 1007.89 56.3 434.01
14.32 44.6 1013.85 68.13 466.36
26.28 75.23 1011.44 68.35 435.28
7.69 43.02 1014.51 85.23 486.46
14.44 40.1 1015.51 79.78 468.19
9.19 41.01 1022.14 98.98 468.37
13.35 41.39 1019.17 72.87 474.19
23.04 74.22 1009.52 90.93 440.32
4.83 38.44 1015.35 72.94 485.32
17.29 42.86 1014.38 72.3 464.27
8.73 36.18 1013.66 77.74 479.25
26.21 70.32 1007.0 78.29 430.4
23.72 58.62 1016.65 69.1 447.49
29.27 64.69 1006.85 55.79 438.23
10.4 40.43 1025.46 75.09 492.09
12.19 40.75 1015.13 88.98 475.36
20.4 54.9 1016.68 64.26 452.56
34.3 74.67 1015.98 25.89 427.84
27.56 68.08 1010.8 59.18 433.95
30.9 70.8 1008.48 67.48 435.27
14.85 58.59 1014.04 89.85 454.62
16.42 40.56 1020.36 50.62 472.17
16.45 63.31 1015.96 83.97 452.42
10.14 42.02 1003.19 96.51 472.17
9.53 41.44 1018.01 80.09 481.83
17.01 49.15 1021.83 84.02 458.78
23.94 62.08 1022.47 61.97 447.5
15.95 49.25 1019.04 88.51 463.4
11.15 41.26 1022.67 81.83 473.57
25.56 70.32 1009.07 90.63 433.72
27.16 66.44 1011.2 73.37 431.85
26.71 77.95 1012.13 77.5 433.47
29.56 74.22 1007.45 57.46 432.84
31.19 70.94 1007.29 51.91 436.6
6.86 41.17 1020.12 79.14 490.23
12.36 41.74 1020.58 69.24 477.16
32.82 68.31 1010.44 41.85 441.06
25.3 70.98 1007.22 95.1 440.86
8.71 41.82 1033.08 74.53 477.94
13.34 40.8 1026.56 64.85 474.47
14.2 43.02 1012.18 57.07 470.67
23.74 65.34 1013.7 62.9 447.31
16.9 44.88 1018.14 72.21 466.8
28.54 71.94 1007.4 65.99 430.91
30.15 69.88 1007.2 73.67 434.75
14.33 42.86 1010.82 88.59 469.52
25.57 59.43 1008.88 61.19 438.9
30.55 70.04 1010.51 49.37 429.56
28.04 74.33 1013.53 48.65 432.92
26.39 49.16 1005.68 56.18 442.87
15.3 41.76 1022.57 71.56 466.59
6.03 41.14 1028.04 87.46 479.61
13.49 44.63 1019.12 70.02 471.08
27.67 59.14 1016.51 61.2 433.37
24.19 65.48 1018.8 60.54 443.92
24.44 59.14 1016.74 71.82 443.5
29.86 64.79 1017.37 44.8 439.89
30.2 69.59 1008.9 67.32 434.66
7.99 41.38 1021.95 78.77 487.57
9.93 41.62 1013.76 96.02 464.64
11.03 42.32 1017.26 90.56 470.92
22.34 63.73 1014.37 83.19 444.39
25.33 48.6 1002.54 68.45 442.48
18.87 52.08 1005.25 99.19 449.61
25.97 69.34 1009.43 88.11 435.02
16.58 43.99 1021.81 79.29 458.67
14.35 46.18 1016.63 87.76 461.74
25.06 62.39 1008.09 82.56 438.31
13.85 48.92 1011.68 79.24 462.38
16.09 44.2 1019.39 67.24 460.56
26.34 59.21 1013.37 58.98 439.22
23.01 58.79 1009.71 84.22 444.64
26.39 71.25 999.8 89.12 430.34
31.32 71.29 1008.37 50.07 430.46
16.64 45.87 1009.02 98.86 456.79
13.42 41.23 994.17 95.79 468.82
20.06 44.9 1008.79 70.06 448.51
14.8 44.71 1014.67 41.71 470.77
12.59 41.14 1025.79 86.55 465.74
26.7 66.56 1005.31 71.97 430.21
19.78 50.32 1008.62 96.4 449.23
15.17 49.15 1021.91 91.73 461.89
21.71 61.45 1010.97 91.62 445.72
19.09 39.39 1013.36 59.14 466.13
19.76 51.19 1008.38 92.56 448.71
14.68 41.23 998.43 83.71 469.25
21.3 66.86 1013.04 55.43 450.56
16.73 39.64 1008.94 74.91 464.46
12.26 41.5 1014.87 89.41 471.13
14.77 48.06 1010.92 69.81 461.52
18.26 59.15 1012.04 86.01 451.09
27.1 79.74 1005.43 86.05 431.51
14.72 40.83 1009.65 80.98 469.8
26.3 51.43 1012.05 63.62 442.28
16.48 48.92 1011.84 64.16 458.67
17.99 43.79 1016.13 75.63 462.4
20.34 59.8 1015.18 80.21 453.54
25.53 62.96 1019.81 59.7 444.38
31.59 58.9 1003.39 47.6 440.52
30.8 69.14 1007.68 63.78 433.62
10.75 45.0 1023.68 89.37 481.96
19.3 44.9 1008.89 70.55 452.75
4.71 39.42 1026.4 84.42 481.28
23.1 66.05 1020.28 80.62 439.03
32.63 73.88 1005.64 52.56 435.75
26.63 74.16 1009.72 83.26 436.03
24.35 58.49 1011.03 70.64 445.6
15.11 56.03 1020.27 89.95 462.65
29.1 50.05 1005.87 51.53 438.66
21.24 50.32 1008.54 84.83 447.32
6.16 39.48 1004.85 59.68 484.55
7.36 41.01 1024.9 97.88 476.8
10.44 39.04 1023.99 85.03 480.34
26.76 48.41 1010.53 47.38 440.63
16.79 44.6 1014.27 48.08 459.48
10.76 40.43 1025.98 79.65 490.78
6.07 38.91 1019.25 83.39 483.56
27.33 73.18 1012.26 82.18 429.38
27.15 59.21 1013.49 51.71 440.27
22.35 51.43 1011.34 77.33 445.34
21.82 65.27 1013.86 72.81 447.43
21.11 69.94 1004.37 84.26 439.91
19.95 50.59 1016.11 73.23 459.27
7.45 39.61 1017.88 79.73 478.89
15.36 41.66 1012.41 62.32 466.7
15.65 43.5 1021.39 78.58 463.5
25.31 74.33 1015.04 79.88 436.21
25.88 63.47 1011.95 65.87 443.94
24.6 63.94 1012.87 80.28 439.63
22.58 41.54 1013.21 71.33 460.95
19.69 59.14 1015.99 70.33 448.69
25.85 75.08 1006.24 57.73 444.63
10.06 37.83 1005.49 99.46 473.51
18.59 39.54 1008.56 68.61 462.56
18.27 50.16 1011.07 95.91 451.76
8.85 40.43 1025.68 80.42 491.81
30.04 68.08 1011.04 51.01 429.52
26.06 49.02 1007.59 74.08 437.9
14.8 38.73 1003.18 80.73 467.54
23.93 64.45 1015.35 54.71 449.97
23.72 66.48 1003.61 73.75 436.62
11.44 40.55 1023.37 88.43 477.68
20.28 63.86 1016.04 74.66 447.26
27.9 63.13 1011.8 70.04 439.76
24.74 59.39 1015.23 74.64 437.49
14.8 58.2 1018.29 85.11 455.14
8.22 41.03 1021.76 82.97 485.5
27.56 66.93 1016.81 55.59 444.1
32.07 70.94 1006.91 49.9 432.33
9.53 44.03 1008.87 89.99 471.23
13.61 42.34 1017.93 91.61 463.89
22.2 51.19 1009.2 82.95 445.54
21.36 59.54 1007.99 92.62 446.09
23.25 63.86 1017.82 59.64 445.12
23.5 59.21 1018.29 63.0 443.31
8.46 39.66 1015.14 85.38 484.16
8.19 40.69 1019.86 85.23 477.76
30.67 71.29 1008.36 52.08 430.28
32.48 62.04 1010.39 38.05 446.48
8.99 36.66 1028.11 71.98 481.03
13.77 47.83 1007.41 90.66 466.07
19.05 67.32 1013.2 83.14 447.47
21.19 55.5 1019.83 65.22 455.93
10.12 40.0 1021.15 91.67 479.62
24.93 47.01 1014.28 66.04 455.06
8.47 40.46 1019.87 78.19 475.06
24.52 56.85 1012.59 54.47 438.89
28.55 69.84 1003.38 67.26 432.7
20.58 50.9 1011.89 72.56 452.6
18.31 46.21 1010.46 82.15 451.75
27.18 71.06 1008.16 86.32 430.66
4.43 38.91 1019.04 88.17 491.9
26.02 74.78 1010.04 72.78 439.82
15.75 39.0 1015.91 69.58 460.73
22.99 60.95 1015.14 69.86 449.7
25.52 59.15 1013.88 65.37 439.42
27.04 65.06 1013.33 52.37 439.84
6.42 35.57 1025.58 79.63 485.86
17.04 40.12 1011.81 83.14 458.1
10.79 39.82 1012.89 88.25 479.92
20.41 56.03 1019.94 55.85 458.29
7.36 40.07 1017.29 52.55 489.45
28.08 73.42 1012.17 62.74 434.0
24.74 69.13 1010.69 90.08 431.24
28.32 47.93 1003.26 54.5 439.5
16.71 40.56 1019.48 49.88 467.46
30.7 71.58 1010.0 48.96 429.27
18.42 58.95 1016.95 86.77 452.1
10.62 42.02 999.83 96.66 472.41
22.18 69.05 1002.75 70.84 442.14
22.38 49.3 1003.56 83.83 441.0
13.94 41.58 1020.76 68.22 463.07
21.24 60.84 1017.99 82.22 445.71
6.76 39.81 1017.11 87.9 483.16
26.73 68.84 1010.75 66.83 440.45
7.24 38.06 1020.6 85.36 481.83
10.84 40.62 1015.53 60.9 467.6
19.32 52.84 1004.29 83.51 450.88
29.0 69.13 1001.22 52.96 425.5
23.38 54.42 1013.95 73.02 451.87
31.17 69.51 1010.51 43.11 428.94
26.17 48.6 1002.59 61.41 439.86
30.9 73.42 1011.21 65.32 433.44
24.92 73.68 1015.12 93.68 438.23
32.77 71.32 1007.68 42.39 436.95
14.37 40.56 1021.67 68.18 470.19
8.36 40.22 1011.6 89.18 484.66
31.45 68.27 1007.56 64.79 430.81
31.6 73.17 1010.05 43.48 433.37
17.9 48.98 1014.17 80.4 453.02
20.35 50.9 1012.6 72.43 453.5
16.21 41.23 995.88 80.0 463.09
19.36 44.6 1016.25 45.65 464.56
21.04 65.46 1017.22 63.02 452.12
14.05 40.69 1015.66 74.39 470.9
23.48 64.15 1021.08 57.77 450.89
21.91 63.76 1009.85 76.8 445.04
24.42 63.07 1011.49 67.39 444.72
14.26 40.92 1022.07 73.96 460.38
21.38 58.33 1013.05 72.75 446.8
15.71 44.06 1018.34 71.69 465.05
5.78 40.62 1016.55 84.98 484.13
6.77 39.81 1017.01 87.68 488.27
23.84 49.21 1013.85 50.36 447.09
21.17 58.16 1017.16 68.11 452.02
19.94 58.96 1014.16 66.27 455.55
8.73 41.92 1029.41 89.72 480.99
16.39 41.67 1012.96 61.07 467.68

We will do the following steps in the sequel.

  • Step 5: Data Preparation
  • Step 6: Data Modeling
  • Step 7: Tuning and Evaluation
  • Step 8: Deployment

Datasource References:

  • Pinar Tüfekci, Prediction of full load electrical power output of a base load operated combined cycle power plant using machine learning methods, International Journal of Electrical Power & Energy Systems, Volume 60, September 2014, Pages 126-140, ISSN 0142-0615, Web Link
  • Heysem Kaya, Pinar Tüfekci , Sadik Fikret Gürgen: Local and Global Learning Methods for Predicting Power of a Combined Gas & Steam Turbine, Proceedings of the International Conference on Emerging Trends in Computer and Electronics Engineering ICETCEE 2012, pp. 13-18 (Mar. 2012, Dubai) Web Link

ScaDaMaLe Course site and book

Wiki Clickstream Analysis

Dataset: 3.2 billion requests collected during the month of February 2015 grouped by (src, dest).

Source: https://datahub.io/dataset/wikipedia-clickstream/

NY clickstream image

This notebook was originally a data analysis workflow developed with Databricks Community Edition, a free version of Databricks designed for learning Apache Spark.

Here we elucidate in Scala the original Python notebook used in the talk by Michael Armbrust at Spark Summit East February 2016 shared from https://twitter.com/michaelarmbrust/status/699969850475737088 (watch later)

Michael Armbrust Spark Summit East

Data set

Wikipedia Logo

The data we are exploring in this lab is the February 2015 English Wikipedia Clickstream data, and it is available here.

According to Wikimedia:

"The data contains counts of (referer, resource) pairs extracted from the request logs of English Wikipedia. When a client requests a resource by following a link or performing a search, the URI of the webpage that linked to the resource is included with the request in an HTTP header called the "referer". This data captures 22 million (referer, resource) pairs from a total of 3.2 billion requests collected during the month of February 2015."

The data is approximately 1.2GB and it is hosted in the following Databricks file:

  • /databricks-datasets/wikipedia-datasets/data-001/clickstream/raw-uncompressed/2015_2_clickstream.tsv

if you are on databricks.

We will work with a smaller sample of this dataset here first:

  • /datasets/sds/wikipedia-datasets/2015_2_clickstream_subsampled.tsv.

Let us first understand this Wikimedia data set a bit more

Let's read the datahub-hosted link https://datahub.io/dataset/wikipedia-clickstream.

Also click the blog by Ellery Wulczyn, Data Scientist at The Wikimedia Foundation, to better understand how the data was generated.

Run the next two cells for some housekeeping.

val data = sc.textFile("/datasets/sds/wikipedia-datasets/2015_2_clickstream_subsampled.tsv")
data: org.apache.spark.rdd.RDD[String] = /datasets/sds/wikipedia-datasets/2015_2_clickstream_subsampled.tsv MapPartitionsRDD[831] at textFile at command-2971213210277624:1
data.take(5).foreach(println) 
prev_id	curr_id	n	prev_title	curr_title	type
37284.0	197438.0	41	Brain_tumor	Fontanelle	link
2904478.0	29932496.0	14	Ottoman_Reform_Edict_of_1856	Hatt-i_humayun	other
	412127.0	39	other-wikipedia	Tony_Zale	other
2368683.0	209811.0	15	Trajan's_Forum	Looting	other
data.take(2)
res1: Array[String] = Array(prev_id	curr_id	n	prev_title	curr_title	type, 37284.0	197438.0	41	Brain_tumor	Fontanelle	link)
  • The first line looks like a header
  • The second line (separated from the first by ",") contains data organized according to the header, i.e., prev_id = 3632887, curr_id = 121", and so on.

Actually, here is the meaning of each column:

  • prev_id: if the referer does not correspond to an article in the main namespace of English Wikipedia, this value will be empty. Otherwise, it contains the unique MediaWiki page ID of the article corresponding to the referer i.e. the previous article the client was on

  • curr_id: the MediaWiki unique page ID of the article the client requested

  • prev_title: the result of mapping the referer URL to the fixed set of values described below

  • curr_title: the title of the article the client requested

  • n: the number of occurrences of the (referer, resource) pair

  • type

    • "link" if the referer and request are both articles and the referer links to the request
    • "redlink" if the referer is an article and links to the request, but the request is not in the production enwiki.page table
    • "other" if the referer and request are both articles but the referer does not link to the request. This can happen when clients search or spoof their refer

Referers were mapped to a fixed set of values corresponding to internal traffic or external traffic from one of the top 5 global traffic sources to English Wikipedia, based on this scheme:

  • an article in the main namespace of English Wikipedia -> the article title
  • any Wikipedia page that is not in the main namespace of English Wikipedia -> other-wikipedia
  • an empty referer -> other-empty
  • a page from any other Wikimedia project -> other-internal
  • Google -> other-google
  • Yahoo -> other-yahoo
  • Bing -> other-bing
  • Facebook -> other-facebook
  • Twitter -> other-twitter
  • anything else -> other-other

In the second line of the file above, we can see there were 121 clicks from Google to the Wikipedia page on "!!" (double exclamation marks). People search for everything!

  • prev_id = (nothing)
  • curr_id = 3632887 --> (Wikipedia page ID)
  • n = 121 (People clicked from Google to this page 121 times in this month.)
  • prev_title = other-google (This data record is for referals from Google.)
  • curr_title = !! (This Wikipedia page is about a double exclamation mark.)
  • type = other

Create a DataFrame from this CSV

  • it's as easy as the following code.
// Load the raw dataset stored as a CSV file
val clickstream = sqlContext
    .read
    .format("csv")
    .options(Map("header" -> "true", "delimiter" -> "\t", "mode" -> "PERMISSIVE", "inferSchema" -> "true"))
    .load("/datasets/sds/wikipedia-datasets/2015_2_clickstream_subsampled.tsv")
clickstream: org.apache.spark.sql.DataFrame = [prev_id: double, curr_id: double ... 4 more fields]
clickstream.count
res4: Long = 224809
clickstream.printSchema
root
 |-- prev_id: double (nullable = true)
 |-- curr_id: double (nullable = true)
 |-- n: integer (nullable = true)
 |-- prev_title: string (nullable = true)
 |-- curr_title: string (nullable = true)
 |-- type: string (nullable = true)
display(clickstream)
prev_id curr_id n prev_title curr_title type
37284.0 197438.0 41.0 Brain_tumor Fontanelle link
2904478.0 2.9932496e7 14.0 Ottoman_Reform_Edict_of_1856 Hatt-i_humayun other
null 412127.0 39.0 other-wikipedia Tony_Zale other
2368683.0 209811.0 15.0 Trajan's_Forum Looting other
7691324.0 17616.0 14.0 Zonal_flow Latitude link
null 1.9730805e7 170.0 other-empty Ryan_Jones_(ice_hockey) other
null 129375.0 32.0 other-empty Reading,_Ohio other
1.5580374e7 2554846.0 14.0 Main_Page Toots other
348959.0 3.5701262e7 53.0 Japanese_dialects Hokkaido_dialects link
null 614279.0 16.0 other-yahoo Nick_13 other
148898.0 3.6496677e7 13.0 Asian_American Watsonville_riots other
7997964.0 2.0992676e7 11.0 Chikara_Campeonatos_de_Parejas The_Colony_(professional_wrestling) link
null 5922274.0 22.0 other-empty John_William_Loudon other
null 1997924.0 30.0 other-empty Mister_Cartoon other
null 2774369.0 10.0 other-empty Knottingley_railway_station other
null 2.7283921e7 13.0 other-google Caret,_Virginia other
4940115.0 2370506.0 68.0 Drew_Mitchell RC_Toulonnais link
6514702.0 6982829.0 2755.0 Lonelygirl15 Jessica_Lee_Rose link
4.3010378e7 451169.0 74.0 Pro_Evolution_Soccer_2015 Swansea_City_A.F.C. other
null 2653829.0 12.0 other-google List_of_places_in_New_York:_V other
2979541.0 1.5580374e7 16.0 The_Kooks Main_Page other
2378072.0 877225.0 42.0 The_Crow:_Wicked_Prayer The_Crow:_Salvation link
49706.0 77491.0 14.0 Paul_Newman Gregory_Peck link
3084058.0 1352229.0 10.0 Sport_coat Harrington_jacket link
null 7079047.0 30.0 other-empty Mario_Gjurovski other
752246.0 1.1507227e7 13.0 Miller_Park_(Milwaukee) United_Football_League_(2009–12) other
412214.0 278018.0 18.0 Bill_Russell NBA_All-Star_Game link
158696.0 26272.0 11.0 Roland_TR-808 Ryuichi_Sakamoto link
322055.0 322060.0 254.0 USS_Constellation_(1797) USS_Constellation_(1854) link
525928.0 3.3402653e7 13.0 Special_agent Gus_Fring link
2.4930946e7 27169.0 113.0 Eric_Mangini San_Francisco_49ers link
null 3.5027653e7 48.0 other-wikipedia Mara_Maru other
2.1073732e7 3.3188989e7 12.0 Mexican–American_War Santa_Ana other
1.4912557e7 4.1491686e7 18.0 Caucasus_Emirate December_2013_Volgograd_bombings link
3219844.0 3002478.0 29.0 Azure_Dragon Horn_(Chinese_constellation) link
null 116508.0 43.0 other-empty Brentwood,_Maryland other
26750.0 220636.0 18.0 Sri_Lanka Universal_suffrage link
34404.0 341594.0 78.0 Economy_of_Zimbabwe Land_reform_in_Zimbabwe link
null 3.074887e7 25.0 other-google Sam_Weiss other
3.8870894e7 1.5580374e7 17.0 Lupe_Fuentes Main_Page other
null 5020425.0 1375.0 other-google FEG_PA-63 other
null 99891.0 341.0 other-empty Gulf_Shores,_Alabama other
1260484.0 2.2763421e7 23.0 Ubud Ubud_District link
1.6113578e7 2.065561e7 12.0 Ronald_Allen_Smith Montana_State_Prison link
null 1.199021e7 24.0 other-yahoo List_of_awards_and_nominations_received_by_Shah_Rukh_Khan other
3841.0 2266430.0 187.0 Bud_Spencer Ace_High_(1968_film) link
846862.0 1148.0 15.0 W._O._Bentley Adelaide link
null 34168.0 18.0 other-twitter Xenogears other
2.3712589e7 557091.0 76.0 Alternative_hip_hop Underground_hip_hop link
null 1.379992e7 31.0 other-google Gheorghe_Păun other
24140.0 28436.0 10.0 Paul_the_Apostle Saint other
null 8343572.0 16.0 other-wikipedia Hulwan other
null 2.392808e7 110.0 other-empty Craig_Dawson other
null 3.6155348e7 15.0 other-bing List_of_world_championships_medalists_in_powerlifting_(men) other
4673783.0 4676317.0 51.0 Double_fisherman's_knot Double_overhand_knot link
null 619770.0 10.0 other-wikipedia Royal_Academy_summer_exhibition other
5577654.0 601316.0 42.0 List_of_Fables_characters Ichabod_Crane link
11006.0 2.8363011e7 39.0 February_19 Sérgio_Júnior other
null 1.4978921e7 16.0 other-wikipedia Herbert_Walther other
19374.0 57546.0 154.0 Model_organism Caenorhabditis_elegans link
null 4049702.0 15.0 other-bing A-91 other
2.4572232e7 2.2008992e7 12.0 Joey_Sturgis Someday_Came_Suddenly link
1122303.0 6950178.0 13.0 Ben_Olsen Ben_Olson link
null 4403167.0 15.0 other-other Fereydoon_Moshiri other
null 423041.0 23.0 other-empty First_Circle other
1.3078837e7 2.7332848e7 12.0 2007_US_Open_–_Boys'_Singles Matteo_Trevisan link
null 6493680.0 11.0 other-google Mentai_Rock other
1.1197284e7 1.744141e7 12.0 Kurt_Pellegrino Júnior_Assunção other
887544.0 2420403.0 552.0 Alisha_Klass Seymore_Butts link
1.0609116e7 4.3173611e7 17.0 Arizona_Wildcats_men's_basketball Rondae_Hollis-Jefferson link
null 616622.0 724.0 other-wikipedia Andriy_Shevchenko other
2296379.0 1771587.0 14.0 Palmar_erythema Pregnancy link
6678.0 99426.0 19.0 Cat Naphthalene link
55502.0 55501.0 40.0 860s_BC 850s_BC link
2.181477e7 74940.0 106.0 Langston_Hughes Marian_Anderson link
763905.0 3.9681124e7 62.0 Tien Tien_(surname) link
4.4251018e7 3.7049649e7 14.0 HyperDex Spanner_(database) link
5697437.0 192755.0 65.0 The_Cockpit_(OVA) Yokosuka_MXY7_Ohka link
3084191.0 null 17.0 Dany_Verissimo John_B._Root redlink
9624289.0 1.9985931e7 329.0 DirecTV DirecTV_satellite_fleet link
31734.0 72227.0 18.0 Urea Plywood link
1078676.0 4486620.0 13.0 Burundian_Civil_War United_Nations_Operation_in_Burundi link
4408.0 1147922.0 168.0 Buddy_Holly Music_of_Lubbock,_Texas other
null 444112.0 33.0 other-yahoo Gnoll other
3369981.0 1516915.0 16.0 Flu_(disambiguation) Swine_influenza other
468301.0 982480.0 229.0 Samantha_Morton John_Carter_(film) link
null 6469961.0 35.0 other-wikipedia Sertorian_War other
null 3.8758012e7 109.0 other-other List_of_federal_subjects_of_Russia_by_GDP_per_capita other
null 3.6390439e7 25.0 other-google Major_Mining_Sites_of_Wallonia other
11887.0 895357.0 43.0 Greek_language English_words_of_Greek_origin link
21383.0 7901223.0 14.0 Nigeria Nigerian_general_election,_2007 link
731774.0 2539671.0 324.0 Law_of_Moses Ten_Commandments link
2.3896488e7 1.1263766e7 11.0 Vampire_Academy_(novel) Diary_of_a_Wimpy_Kid link
null 1508712.0 20.0 other-empty Mikoyan_MiG-110 other
2.1536106e7 2.7520075e7 17.0 Jennifer_Blake_(wrestler) Mari_Apache link
null 2.4052308e7 58.0 other-empty Francis_Crowley other
null 1.6792585e7 75.0 other-google Phulkian_sardars other
2812945.0 1.6230289e7 25.0 Double_Dutch_Bus Raven-Symoné_(album) link
3.9522631e7 253375.0 70.0 News_Corp HarperCollins link
null 2.1329684e7 17.0 other-other David_Lyon_(sociologist) other
338344.0 3.9032732e7 84.0 List_of_tallest_buildings_in_the_world Discovery_Primea link
3.9839062e7 1.9087186e7 12.0 List_of_unicorns Noah's_Ark_(2007_film) link
58666.0 3550910.0 12.0 United_States_Environmental_Protection_Agency Marine_Mammal_Protection_Act_of_1972 link
null 3.5675894e7 11.0 other-wikipedia AdMarketplace other
627321.0 203426.0 16.0 Burma_Campaign Nyasaland link
234382.0 378561.0 180.0 Elephant_(2003_film) Eric_Harris_and_Dylan_Klebold link
null 434919.0 81.0 other-google Alan_Meale other
null 2.6552915e7 21.0 other-empty Gertrude_Abercrombie other
1.9172225e7 206790.0 84.0 Prokaryote Spirochaete link
null 8759210.0 46.0 other-google TM_and_Cult_Mania other
34742.0 75831.0 27.0 5th_century Flavius_Aetius other
737160.0 4445580.0 10.0 Soilwork Sonic_Syndicate link
null 2.0594394e7 12.0 other-wikipedia Claus_Costa other
490391.0 682513.0 10.0 Adnan_Gulshair_el_Shukrijumah Abderraouf_Jdey link
null 5772621.0 21.0 other-wikipedia The_Watsons other
1.7015795e7 1.336124e7 23.0 Teyana_Taylor Blue_Magic_(song) link
8026795.0 14653.0 12.0 Iran–Turkey_relations Iran link
null 3.7569668e7 15.0 other-google I.O.U._(Jimmy_Dean_song) other
null 2.3416901e7 21.0 other-google Strathcarron_Sports_Cars other
303241.0 719034.0 14.0 Strong_Guy Mephisto_(comics) link
null 673275.0 203.0 other-other Scale_insect other
null 5925339.0 151.0 other-google Flintham other
11092.0 653246.0 39.0 Finger_Lakes Canadice_Lake link
45715.0 2011918.0 22.0 Arecaceae Minoo_Island link
8427319.0 245765.0 16.0 List_of_alternative_country_musicians The_Jayhawks other
93036.0 129736.0 13.0 Portage_County,_Ohio Hiram,_Ohio link
null 3.1998475e7 10.0 other-empty Royal_Botanical_Expedition_to_New_Granada other
3717.0 4911607.0 33.0 Brain Anterior_grey_column other
null 4.3030724e7 15.0 other-wikipedia 99th_Infantry_Battalion_(United_States) other
3615880.0 2.9646991e7 11.0 Ranger_(Dungeons_&_Dragons) The_Complete_Fighter's_Handbook link
null 3.6599164e7 26.0 other-wikipedia Rudding_Park_House other
null 2.0000187e7 405.0 other-wikipedia Inflection other
3.593072e7 574988.0 41.0 Stairway_to_Hell Ugly_Kid_Joe link
78127.0 3358304.0 10.0 James_Doohan Star_Trek_(film_franchise) other
4.0530767e7 19281.0 42.0 Visa_requirements_for_Palestinian_citizens Montserrat link
4100885.0 29301.0 27.0 Meaning_(linguistics) Semiotics link
null 1.9535017e7 48.0 other-google Aumism other
null 3.9268446e7 10.0 other-other Raúl_Duarte_(basketball) other
2.2942232e7 9891690.0 24.0 Lists_of_academic_journals List_of_pharmaceutical_sciences_journals link
null 9108276.0 797.0 other-other Kesh_(Sikhism) other
4306874.0 1615009.0 14.0 Khartoum_International_Airport Flynas link
3.3948854e7 2.7572278e7 10.0 Robert_A._J._Gagnon Jack_Rogers_(clergy) link
2.5318118e7 17867.0 19.0 Government_of_the_United_Kingdom London link
4.4729787e7 5070615.0 12.0 Marvel_Contest_of_Champions Marvel_Super_Heroes:_War_of_the_Gems link
65192.0 141976.0 17.0 Three_Gorges_Dam Alstom link
194664.0 924738.0 86.0 The_Flintstones_(film) Richard_Moll link
null 2370304.0 1180.0 other-google Sriperumbudur other
null 2.6678779e7 19.0 other-google Geeklog other
null 109049.0 55.0 other-other Margate,_Florida other
null 365310.0 961.0 other-google Vostok_(spacecraft) other
null 191537.0 271.0 other-other Internment other
null 1250542.0 27.0 other-other Jake_Peavy other
1069442.0 180425.0 39.0 Charles_Bickford Woodlawn_Memorial_Cemetery,_Santa_Monica link
null 3.8641017e7 3802.0 other-google GeForce_800M_series other
80387.0 77944.0 29.0 Hamadryad Hesperides link
18819.0 2399697.0 35.0 Microeconomics Heterodox_economics link
9566994.0 9574674.0 12.0 The_Renegade_(short_story) The_Silent_Men link
333703.0 161436.0 15.0 Angelo_Dundee Ernest_Borgnine link
2.6301553e7 10150.0 34.0 António_de_Oliveira_Salazar Engelbert_Dollfuss other
4.2255443e7 null 21.0 List_of_United_States_bomber_aircraft List_of_United_States_attack_aircraft redlink
null 1.2290729e7 40.0 other-wikipedia Monte_Carlo_(biscuit) other
7619598.0 2916479.0 33.0 Fusiliers_Commandos_de_l'Air German_Air_Force_Regiment link
7457961.0 1651671.0 28.0 List_of_Castlevania:_Aria_of_Sorrow_and_Dawn_of_Sorrow_characters Soma_Cruz link
null 1.8755551e7 167.0 other-google Hooli other
null 2.557502e7 12.0 other-wikipedia Leucadendron_album other
null 9041425.0 57.0 other-empty Country_Teasers other
null 1.9226982e7 17.0 other-google The_Wits other
5625186.0 2.3378704e7 22.0 Chadian–Libyan_conflict Chadian_Civil_War_(1965–79) link
null 2783508.0 17.0 other-other Encephalartos other
192042.0 155627.0 22.0 Over-the-counter_drug Ibuprofen link
null 3.8764007e7 31.0 other-empty Secret_Story_4_(Portugal) other
22205.0 1147409.0 305.0 Oasis Ein_Gedi link
1.8899195e7 4527556.0 12.0 Rhynchosaurus Rhynchosaur link
null 2.61932e7 42.0 other-wikipedia Charl_Van_Den_Berg other
1.9567899e7 2.5757144e7 224.0 Nintendo_DSi Foto_Showdown link
1.7981721e7 518667.0 13.0 Afamelanotide Erythropoietic_protoporphyria link
637310.0 1.0684273e7 41.0 Japanese_submarine_I-25 Leninets-class_submarine other
null 3958341.0 27.0 other-empty Atsushi_Itō_(actor) other
30085.0 1185285.0 20.0 Thomas_Mann Kilchberg,_Zürich link
2652103.0 2.65684e7 294.0 Tiling_window_manager Awesome_(window_manager) link
null 9653709.0 36.0 other-google Clint_Warwick other
1.802006e7 4857534.0 17.0 Brimonidine/timolol Brimonidine link
156778.0 52671.0 122.0 Hypochondriasis Psychosomatic_medicine link
1.3335045e7 335195.0 18.0 Dungeons_&_Dragons_in_popular_culture Stephen_Colbert link
3338747.0 4134000.0 18.0 Travis_Willingham Ouran_High_School_Host_Club link
2.4370563e7 1.9866746e7 66.0 Harry_Potter_and_the_Forbidden_Journey Transformers:_The_Ride link
3.6438468e7 2.483689e7 12.0 Alternative_versions_of_Joker Homosexuality_in_the_Batman_franchise link
8119611.0 33158.0 21.0 100_Photographs_that_Changed_the_World War link
2.4102801e7 8986839.0 72.0 List_of_Stradivarius_instruments Hammer_Stradivarius link
null 4043540.0 37.0 other-empty The_Silverado_Squatters other
1.2801225e7 198606.0 28.0 Blair_Tindall Malcolm_McDowell link
737.0 1.1203313e7 47.0 Afghanistan Afghanistan–Pakistan_skirmishes link
null 26039.0 189.0 other-empty Photek other
null 7008604.0 44.0 other-empty Edward_S._Walker,_Jr. other
null 5031359.0 18.0 other-google Families_Acting_for_Innocent_Relatives other
null 5545390.0 21.0 other-other EWR_VJ_101 other
4503997.0 1.4695141e7 127.0 What_About_Brian List_of_What_About_Brian_episodes link
null 4.137716e7 14.0 other-empty Frederick_Fox_(milliner) other
3096312.0 99459.0 15.0 Dana_Wynter Airport_(1970_film) link
null 1.2995239e7 18.0 other-wikipedia Nervous_Night_(album) other
null 1.47815e7 49.0 other-google Thermal_dose_unit other
1.8186074e7 1.608506e7 33.0 National_Highway_7A_(India)(old_numbering) National_Highway_7_(India)(old_numbering) link
null 3880952.0 19.0 other-empty Marion_Dudley other
1171412.0 6847270.0 22.0 Ivatan_language Batanic_languages link
8837050.0 30027.0 14.0 Copernican_heliocentrism Tycho_Brahe link
4.3380319e7 4.2654791e7 49.0 Scandal_(season_4) Grey's_Anatomy_(season_11) other
1928831.0 1928617.0 15.0 List_of_moths Laothoe_populi link
222417.0 1793072.0 21.0 A._R._Rahman Dil_Se.. link
1537974.0 3.3757237e7 18.0 Angelo_Scola Francesco_Moraglia link
2.0755574e7 8249183.0 37.0 Venezuelan_of_European_descent María_Rivas other
524874.0 2204.0 16.0 Voiced_pharyngeal_fricative Arabic_alphabet link
60730.0 212182.0 33.0 Lucy_Maud_Montgomery Order_of_the_British_Empire link
null 2.2965354e7 13.0 other-google Shelek other
2.0646706e7 1977645.0 10.0 2009_Campeonato_Brasileiro_Série_A Diego_Tardelli link
2.1513743e7 2.4708146e7 17.0 Jason_Aldean_discography The_Truth_(Jason_Aldean_song) link
null 9277.0 1925.0 other-wikipedia Ellipse other
null 2.0958639e7 40.0 other-empty Trailer_Park_of_Terror other
33265.0 6723726.0 74.0 Winston_Churchill Operation_Overlord link
3.0533402e7 2.1723891e7 25.0 Tryblidiida Micropilina link
2721803.0 1.5146151e7 16.0 Battle_of_Baugé Baugé link
153784.0 17098.0 14.0 Naginata Kendo other
null 1.4347506e7 15.0 other-bing The_Hat other
9768801.0 3.3196141e7 12.0 Manchester_United_F.C._Reserves_and_Academy Peter_O'Sullivan_(Welsh_footballer) link
330447.0 1718041.0 14.0 Binti_Jua Western_lowland_gorilla link
18522.0 17730.0 10.0 Latino_(demonym) Latin link
1.1610928e7 1179269.0 25.0 Peacock-class_corvette Naval_Service_(Ireland) other
625758.0 144367.0 11.0 Bill_Willingham Justice_Society_of_America link
3.3930403e7 5064492.0 108.0 South_Park:_The_Stick_of_Truth South_Park_(video_game) link
148682.0 3891002.0 10.0 Pertinax Marcomannic_Wars link
null 2.5584664e7 43.0 other-other Discharge_coefficient other
null 2374342.0 27.0 other-yahoo Axillary_vein other
null 3.5009269e7 47.0 other-bing Successful_aging other
62069.0 2925739.0 50.0 Panthéon San_Pietro_in_Montorio link
1602491.0 3568081.0 24.0 Ghazipur Abdul_Hamid_(soldier) link
null 4849167.0 29.0 other-other Brethren_of_Purity other
null 1.8592807e7 53.0 other-wikipedia Doc_Shaw other
2.3576946e7 4440840.0 13.0 The_Collector_(2009_film) The_Collection other
null 2.7612808e7 24.0 other-bing "Benjamin_Miles_""C-Note""_Franklin" other
null 560667.0 25.0 other-wikipedia Index_of_Babylon_5_articles other
1.5432504e7 315269.0 22.0 Death_Scream Murder_of_Kitty_Genovese other
null 1904377.0 40.0 other-yahoo Canon_A-1 other
8556123.0 1.3036212e7 1866.0 Rules_of_Engagement_(TV_series) David_Spade link
null 2843028.0 17.0 other-wikipedia Mid-Canada_Communications other
null 4724437.0 345.0 other-google University_of_the_Balearic_Islands other
null 2.9457801e7 247.0 other-wikipedia Age_of_Heroes_(film) other
3.9120425e7 3.9868659e7 90.0 Santhosh_Narayanan Lucia_(2013_film) link
1904112.0 1004953.0 42.0 Lamberto_Bava Mario_Bava link
null 2.1372915e7 33.0 other-yahoo Christian_views_on_divorce other
91320.0 91267.0 15.0 Franklin_County,_Virginia Pittsylvania_County,_Virginia link
1334948.0 238273.0 221.0 Lupus_anticoagulant Antiphospholipid_syndrome link
null 3.2782542e7 24.0 other-wikipedia Sara_Pichelli other
null 1.7954948e7 12.0 other-google Club_Sportif_Makthar other
null 2.1880672e7 14.0 other-bing Bitte_Orca other
10128.0 75899.0 32.0 Elizabeth_I_of_England Huguenot link
7437933.0 1.1869952e7 29.0 Space_and_Upper_Atmosphere_Research_Commission Chronology_of_Pakistan's_rocket_tests link
2.4426866e7 1.2757148e7 27.0 Huamei Li_hing_mui link
2942161.0 2933074.0 28.0 Canon_T_series Canon_T50 link
null 1.4084798e7 18.0 other-empty 2008_Russian_Premier_League other
3.8317679e7 3.8780447e7 11.0 Marxist_humanism Structural_Marxism link
null 2.2571562e7 13.0 other-bing Jennifer_Servo other
2.0618103e7 275978.0 25.0 Camphora Camphor link
1.5580374e7 2.4128239e7 36.0 Main_Page Selena_Gomez_&_the_Scene other
4.5274337e7 32538.0 133.0 Gotlandsdricka Viking_Age link
null 1.1252905e7 205.0 other-yahoo Biblical_manuscript other
239038.0 1170.0 18.0 Construction Architect link
3.8465988e7 2540476.0 12.0 List_of_Republic_of_Ireland_international_footballers Kevin_Kilbane link
3.9413121e7 49696.0 68.0 Giorgio_Moroder_discography Metropolis_(1927_film) link
null 36743.0 281.0 other-other Atalanta other
28237.0 177534.0 192.0 Space_Shuttle_Columbia Ilan_Ramon link
3.148073e7 715008.0 36.0 2011–12_Football_League_Championship Football_League_Championship link
null 1014389.0 37.0 other-bing No_Man's_Land_Fort other
502307.0 323983.0 46.0 Cole_Turner Billy_Zane link
167745.0 5301493.0 10.0 Vaquero Mesoamerica link
1925385.0 1925397.0 12.0 Walter_F._Murphy The_Vicar_of_Christ other
null 2.1766442e7 113.0 other-empty David_Álvarez other
3244595.0 1584677.0 22.0 Harpe_brothers War_of_the_Regulation link
1.1221038e7 1510621.0 13.0 2007–08_Rangers_F.C._season Alan_Hutton link
1.5580374e7 1.7408264e7 10.0 Main_Page Vorapaxar other
null 2.5008567e7 18.0 other-empty Roderic_Noble other
3424149.0 5912292.0 27.0 1975–76_NBA_season 1976_NBA_Playoffs link
4.2736926e7 418286.0 158.0 The_Blacklist_(season_1) Justin_Kirk link
null 1.3598815e7 17.0 other-google Niquero other
2.4509049e7 1.4952458e7 10.0 Maritime_flag_signalling Flag_semaphore other
3660711.0 1335545.0 62.0 Duncan_Keith 2002_NHL_Entry_Draft link
30450.0 49172.0 15.0 Topological_space Interval_(mathematics) link
1424791.0 60368.0 60.0 Anne_Wiazemsky Jean-Luc_Godard link
null 4.0712418e7 14.0 other-google NEWS_(Austrian_magazine) other
null 1214237.0 111.0 other-wikipedia Fold_Your_Hands_Child,_You_Walk_Like_a_Peasant other
null 1.5832772e7 223.0 other-other Osu! other
966943.0 5092756.0 22.0 List_of_Family_Guy_episodes List_of_Top_Gear_episodes other
1.3638115e7 4.0387861e7 23.0 Carciofi_alla_giudia Carciofi_alla_romana link
null 6787685.0 33.0 other-google Bovalino other
null 1386389.0 81.0 other-other Southern_African_Customs_Union other
576635.0 986684.0 12.0 Port_scanner Rate_limiting link
56462.0 8639835.0 24.0 Carpal_tunnel_syndrome Radiculopathy link
1.602284e7 1.8895122e7 29.0 2008_Major_League_Baseball_Draft Brett_Lawrie link
null 1834626.0 17.0 other-bing Knowsley_Safari_Park other
861686.0 343056.0 16.0 Tyrone_Guthrie Stratford_Shakespeare_Festival other
1.5580374e7 288197.0 124.0 Main_Page Kapil_Dev other
3.4075076e7 3.6541863e7 59.0 Gopichand_Malineni Balupu link
171141.0 3966054.0 19.0 Guava Mexico link
null 9441043.0 77.0 other-empty Cooliris other
null 5500697.0 3623.0 other-google Kiss_Me_(Sixpence_None_the_Richer_song) other
null 2.287596e7 11.0 other-bing Abblasen other
66297.0 53682.0 12.0 Chinese_art Calligraphy link
null 4.5282472e7 75.0 other-other Aspire_Music_Group other
3.0588065e7 86817.0 14.0 List_of_authoritarian_regimes_supported_by_the_United_States Omar_Torrijos link
null 1.7341725e7 15.0 other-empty Nasim_Wali_Khan other
46426.0 1.0834159e7 21.0 Basil_II Battle_of_Kreta link
59653.0 226141.0 123.0 Foreign_and_Commonwealth_Office Secretary_of_State_for_Commonwealth_Affairs link
9378717.0 164227.0 50.0 The_Spy_Who_Came_in_from_the_Cold_(film) Michael_Hordern link
5444617.0 5485318.0 12.0 Military_of_Montenegro UTVA_75 other
796141.0 8532006.0 17.0 1966–67_United_States_network_television_schedule Coronet_Blue link
381658.0 3.7085064e7 21.0 FK_Partizan History_of_FK_Partizan link
null 2.8350129e7 11.0 other-empty Arizona_Mountains_forests other
3.1927202e7 2.9381422e7 11.0 List_of_songs_recorded_by_My_Chemical_Romance Sing_(My_Chemical_Romance_song) link
null 7289088.0 30.0 other-google 1938_World_Men's_Handball_Championship other
8926582.0 174104.0 12.0 Imperial_Japanese_Army_General_Staff_Office Yamagata_Aritomo link
1.9258996e7 1.4116605e7 14.0 Den_Saakaldte Niklas_Kvarforth link
44682.0 6988539.0 48.0 CMYK_color_model Screen_angle link
2.3608452e7 1.5062239e7 152.0 Galatasaray_S.K._(football) Sercan_Yıldırım link
2617605.0 4.3680163e7 27.0 Saipa_F.C. Hamed_Shiri link
1.7645814e7 1.7519198e7 20.0 I_Hate_You_with_a_Passion Andre_Nickatina link
2.1327889e7 1110833.0 12.0 Kröd_Mändoon_and_the_Flaming_Sword_of_Fire Roger_Allam link
2.3487767e7 2.6323418e7 27.0 The_Tempest True_Reportory link
20869.0 2.3041952e7 12.0 Monoamine_oxidase_inhibitor Mebanazine link
null 3.1056907e7 12.0 other-yahoo Adebisi_Shank other
16716.0 1.5580374e7 68.0 Kansas Main_Page other
2.5731835e7 2.6693897e7 16.0 Colourist_painting Fauvism other
763785.0 3540456.0 111.0 Wellcome_Trust List_of_wealthiest_charitable_foundations link
null 177953.0 357.0 other-wikipedia America's_Army other
null 2138855.0 22.0 other-empty Garchitorena,_Camarines_Sur other
1161802.0 1.4216651e7 10.0 Nokia_3110 Nokia_1011 link
null 1.410232e7 318.0 other-google Take_Me_in_Your_Arms_(Rock_Me_a_Little_While) other
162036.0 110232.0 30.0 List_of_United_States_military_bases Moody_Air_Force_Base link
3984468.0 3166244.0 11.0 Oscar_Peterson_discography Ella_and_Oscar link
4255996.0 1.4312625e7 12.0 Western_Sahara_conflict Sahrawi_refugee_camps link
null 2.2661704e7 11.0 other-google Robin_Backhaus other
null 1988.0 65.0 other-yahoo Abel_Tasman other
null 6051097.0 482.0 other-google Trifluridine other
null 2309869.0 11.0 other-bing Frank_Henenlotter other
1248129.0 1.1718319e7 16.0 Port-Gentil Stephane_Lasme link
null 2872565.0 22.0 other-other Starfire_(board_wargame) other
1.0659362e7 1.7160872e7 16.0 Yoon_Dong-sik Gegard_Mousasi link
null 1792018.0 55.0 other-empty Nintendo_Gamebooks other
null 1.1223317e7 344.0 other-other Prince_Devitt other
null 3.5910828e7 10.0 other-wikipedia Sheffield_United_F.C._Player_of_the_Year other
null 2.8027307e7 109.0 other-yahoo Bones_(season_6) other
1.1911941e7 215619.0 10.0 All_Out_of_Love VH1 link
299717.0 3.5724251e7 192.0 Courteney_Cox Go_On_(TV_series) link
2.1930714e7 9265058.0 19.0 Outline_of_Florida List_of_ghost_towns_in_Florida link
581009.0 1439662.0 59.0 Ford_GT Super_GT other
null 985187.0 92.0 other-wikipedia Driving_simulator other
null 2995929.0 13.0 other-empty Edwards_Gardens other
34282.0 34393.0 116.0 Yule Yule_log link
355852.0 874356.0 13.0 Dachau_concentration_camp Miklós_Horthy,_Jr. link
601127.0 4194741.0 12.0 List_of_Democratic_National_Conventions Denver_Auditorium_Arena link
null 2.6415234e7 60.0 other-empty Customs_valuation other
null 2285190.0 20.0 other-empty Gastein_Convention other
499451.0 598952.0 15.0 STS-42 Ronald_J._Grabe link
null 3.279898e7 11.0 other-empty Jalan_Besar_MRT_Station other
2.5864167e7 920737.0 51.0 List_of_crowdsourcing_projects Clickworkers link
3.4199866e7 7253509.0 318.0 AKB0048 AKB48 link
3.3526094e7 3.1748786e7 36.0 QUnit Jasmine_(JavaScript_framework) link
null 5290240.0 10.0 other-empty Angus_M._Cannon other
1.52438e7 492820.0 13.0 Eric_Valentine Lostprophets link
20566.0 3564279.0 26.0 Mandy_Patinkin Tony_Award_for_Best_Featured_Actor_in_a_Musical other
5139911.0 1.4619184e7 12.0 Entering_Heaven_alive Ramalinga_Swamigal link
886579.0 1.1921455e7 14.0 Eva_Longoria Longoria link
null 8743616.0 14.0 other-other Chung_Ling_High_School other
4593958.0 2069950.0 23.0 The_Maltese_Falcon_(1941_film) The_Celluloid_Closet link
211913.0 89235.0 107.0 Christian_metal Christian_rock link
null 2.3834973e7 14.0 other-google Jitender_Kumar other
93135.0 58116.0 13.0 Butler_County,_Ohio Montgomery_County,_Ohio link
null 2.9085741e7 15.0 other-wikipedia Parris_Cues other
null 57564.0 33.0 other-empty Anselme_Payen other
1.9172199e7 1.9167679e7 10.0 Monera Virus other
379518.0 7301806.0 45.0 Panoramic_photography Panography link
null 4.5383298e7 92.0 other-wikipedia Vachagan_Khalatyan other
3618502.0 1.9337279e7 69.0 Echelon_Place Great_Recession link
null 3.1716175e7 59.0 other-google Nikki,_Wild_Dog_of_the_North other
null 3.990522e7 16.0 other-google Harold_Harris_(disambiguation) other
3.5038133e7 8721272.0 13.0 Pathogen PHI-base link
null 2016556.0 72.0 other-google Timmins/Victor_M._Power_Airport other
3.9746293e7 1758267.0 12.0 Tulpa_(film) Giallo link
480658.0 2.3742879e7 10.0 List_of_web_service_specifications XQuery link
1.7176729e7 2.010093e7 14.0 Mondo_Meyer_Upakhyan Samata_Das link
1259342.0 389664.0 11.0 The_Street The_Streets link
null 3.2124266e7 26.0 other-google SnoRNA_prediction_software other
null 136247.0 113.0 other-yahoo Boerne,_Texas other
2.4662654e7 1249019.0 13.0 Killer_Klowns_from_Outer_Space_(album) Killer_Klowns_from_Outer_Space link
null 1565005.0 82.0 other-yahoo Retinal_haemorrhage other
null 9315616.0 21.0 other-empty Gene_Youngblood other
1548943.0 802895.0 22.0 Limited_partnership Private_limited_company link
null 3.1110904e7 147.0 other-bing Effects_of_stress_on_memory other
2.7553159e7 58250.0 13.0 Health_care_in_the_United_States United_States_Department_of_Health_and_Human_Services link
791155.0 60626.0 61.0 Marty_Stuart Lester_Flatt link
4.4740812e7 3.8382626e7 48.0 Jin_Kyung Gu_Family_Book link
1.5610217e7 2756348.0 132.0 Useless_Loop,_Western_Australia Monkey_Mia link
null 1055437.0 46.0 other-wikipedia Deniable_encryption other
null 1.7962717e7 10.0 other-empty Ayres_LM200_Loadmaster other
2502077.0 1.3015878e7 14.0 Sales_taxes_in_the_United_States Washington_(state) other
82933.0 349335.0 38.0 Chloroform Anesthesiologist link
1278087.0 1277999.0 21.0 Nissan_Titan North_American_Car_of_the_Year link
null 718020.0 76.0 other-yahoo Equality other
3445929.0 3445909.0 54.0 Obscura_(album) The_Erosion_of_Sanity link
null 2036467.0 21.0 other-empty Overath other
1.7647526e7 664019.0 10.0 Alternative_versions_of_the_Punisher Owl_(Marvel_Comics) link
1619743.0 3.1636392e7 16.0 York—Simcoe York—Simcoe_(provincial_electoral_district) link
2.5121085e7 7499.0 10.0 APAV40 RDX other
null 1.2778103e7 13.0 other-google Shaden_Abu-Hijleh other
3.2866171e7 1221420.0 14.0 John_F._Kennedy_assassination_conspiracy_theories William_Greer link
1.0567624e7 173305.0 25.0 Tert-Butyl_chloride Isobutane link
null 5625238.0 23.0 other-wikipedia Prehistoric_man other
6819181.0 5755695.0 14.0 Valmara_59 PROM-1 link
2453648.0 2012734.0 11.0 List_of_comic_book_supervillain_debuts Floronic_Man link
97006.0 96981.0 14.0 Costilla_County,_Colorado Las_Animas_County,_Colorado link
null 1230540.0 79.0 other-other Daniel_Yergin other
null 1.6615403e7 147.0 other-wikipedia List_of_women_warriors_in_folklore other
74326.0 3993162.0 30.0 Nadia_Comăneci Art_of_Mentoring link
2232219.0 2920148.0 28.0 Tarquinius Sextus_Tarquinius link
6014615.0 4.3471582e7 21.0 Fox_Interactive Anastasia:_Adventures_with_Pooka_and_Bartok link
null 5191431.0 15.0 other-google Mansfield_Woodhouse_railway_station other
null 2.6523084e7 16.0 other-google Dominic_Arizona_Bonuccelli other
null 616958.0 26.0 other-empty Electrostatic_deflection other
485118.0 2249026.0 220.0 List_of_countries_by_GDP_(PPP) List_of_countries_by_income_equality link
1.3141832e7 47660.0 14.0 Versailles_restaurant Espresso link
null 1777994.0 14.0 other-twitter 1,1-Difluoroethane other
1091514.0 3.8702216e7 70.0 Bila_Tserkva 1941_Bila_Tserkva_massacre link
11362.0 3.1300186e7 24.0 February_16 André_Berthomieu link
7994183.0 4460532.0 35.0 Zicam ICAM-1 link
3.1436814e7 1.6455081e7 99.0 Brynne_Edelsten Geoffrey_Edelsten link
null 2139618.0 592.0 other-google Lee_Chapman other
2464121.0 2.1552009e7 13.0 Asra_Nomani Aisha link
59949.0 2845319.0 11.0 Anglo-Catholicism Traditional_Anglican_Communion link
2019904.0 3.9574425e7 17.0 CSC_Media_Group True_Drama link
null 4168628.0 58.0 other-empty 1997_NCAA_Division_I-A_football_season other
null 3.5535478e7 497.0 other-wikipedia Port_Royale_3:_Pirates_&_Merchants other
4477.0 9288.0 20.0 The_Beach_Boys Elvis_Presley other
null 3899315.0 20.0 other-google Anglican_Diocese_of_Jos other
null 2210627.0 26.0 other-wikipedia Zayed_Port other
null 1467229.0 24.0 other-facebook Fetal_viability other
7922200.0 1.3386129e7 12.0 The_Outfit_(1973_film) Felice_Orlandi link
1.5905472e7 1.3753303e7 178.0 Miss_Universe_1972 Miss_Universe_1973 link
434695.0 null 10.0 İzmir_Province Mustafa_Toprak redlink
277289.0 1055890.0 43.0 Wind_power Sustainable_energy link
1.9332171e7 5215871.0 12.0 Richard_Wright_(musician) Delay_(audio_effect) link
37417.0 840184.0 10.0 Mercury_(mythology) Hendrik_Goltzius link
356617.0 510764.0 35.0 Operation_Tannenberg Einsatzgruppen link
3.8174481e7 196279.0 43.0 List_of_sports_cars Lamborghini_Diablo link
1371727.0 433285.0 51.0 Bullet_(disambiguation) Bullitt link
null 1914019.0 121.0 other-google GTB other
5057404.0 null 12.0 Eureka_Forbes Aushim_Gupta_&_Company_Ltd. redlink
null 3.5394122e7 14.0 other-empty Joseph_Maynard other
1065035.0 2987765.0 10.0 Benson_&_Hedges Assault_occasioning_actual_bodily_harm link
null 1.9353281e7 13.0 other-google 1160_AM other
null 4155017.0 10.0 other-bing Gökhan_Özoğuz other
3.6909154e7 3.0563823e7 20.0 Dani_Carvajal Jesé link
null 4.2675612e7 6560.0 other-google Elfrid_Payton_(basketball) other
189048.0 6919555.0 24.0 Klaus_Schulze Angst_(soundtrack) link
null 4.1501305e7 19.0 other-twitter Telegram_(software) other
4848945.0 143163.0 76.0 Enclave_and_exclave Gadsden_Purchase link
1.163353e7 2657310.0 17.0 Injection_molding_machine Sprue_(manufacturing) link
null 2645023.0 34.0 other-google Monash,_Australian_Capital_Territory other
null 543169.0 111.0 other-empty William_Jackson_Palmer other
1.4174205e7 1920113.0 29.0 1981_UEFA_Cup_Final Arnold_Mühren link
null 1924219.0 229.0 other-empty Emperor:_Battle_for_Dune other
2.3670849e7 3623280.0 34.0 No_One's_Gonna_Love_You Band_of_Horses link
null 1.5964132e7 15.0 other-empty Jun_He_Law_Offices other
null 5545139.0 14.0 other-google Chief_Examiner other
2998033.0 1.5472645e7 16.0 West_Hampstead_Thameslink_railway_station West_Hampstead_railway_station link
null 1649162.0 90.0 other-other Starbreeze_Studios other
3.9617946e7 4.2742632e7 18.0 2014_French_Open 2014_French_Open_–_Girls'_Singles link
null 2.3915342e7 1813.0 other-google Cracked_tooth_syndrome other
null 2.4459316e7 11.0 other-empty OARnet other
174750.0 2.0974012e7 135.0 High/Low_(Nada_Surf_album) Popular_(Nada_Surf_song) link
6595367.0 7482590.0 12.0 Hewitt–Savage_zero–one_law Edwin_Hewitt link
1.3036961e7 199630.0 24.0 Something_for_Nothing Pop_punk link
null 2.2407888e7 101.0 other-google PSR_B1509-58 other
397145.0 3336186.0 23.0 List_of_districts_of_Maharashtra Nagpur_division link
345356.0 54251.0 106.0 Skate_(fish) Myliobatiformes link
3.7386608e7 4.069023e7 11.0 2015_in_film Sonic_Boom_(TV_series) other
8864153.0 1303857.0 20.0 Energy_policy_of_the_European_Union Electricity_liberalization link
null 1681589.0 58.0 other-empty Girl_on_the_Bridge other
27318.0 2701625.0 194.0 Singapore List_of_countries_by_life_expectancy link
1.7994862e7 1.3126459e7 13.0 Elastix Unified_communications link
418179.0 1880887.0 23.0 Whyte_notation 2-12-2 link
1.9769679e7 1057476.0 60.0 List_of_Nobel_Memorial_Prize_laureates_in_Economics Finn_E._Kydland link
null 1.2175763e7 14.0 other-wikipedia Palm_rat other
null 53884.0 30.0 other-other Penalty_area other
null 1819715.0 18.0 other-other Marine_geology other
5866621.0 3.6655551e7 12.0 Nayantara Vijay_Sethupathi link
5683212.0 1531683.0 15.0 McLeod_syndrome Myopathy link
50409.0 2563761.0 13.0 Cistercians Monastery_of_the_Holy_Spirit link
null 2.0720709e7 24.0 other-other 2-Pentyne other
null 1602370.0 29.0 other-google Kaleb_Toth other
740070.0 1.9891664e7 13.0 Body_farm Stephen_Fry_in_America link
null 8308.0 118.0 other-yahoo Delft other
3.4396117e7 2.2992135e7 129.0 Sofia_the_First Rapunzel_(Disney) link
2.6552124e7 735009.0 14.0 Sammy_Adams Pharrell_Williams other
6254282.0 164365.0 10.0 John_Chambers_(make-up_artist) The_China_Syndrome link
1468740.0 1.3964958e7 17.0 Gran_Turismo_4 Gran_Turismo_official_steering_wheel link
null 3233890.0 76.0 other-bing The_Bronx_(band) other
17851.0 6501490.0 81.0 Lambda Half-Life_(series) link
5574932.0 292086.0 13.0 Princess_Niloufer Abdülmecid_II link
401530.0 1.9283982e7 14.0 National_Democratic_Party New_Democratic_Party other
492211.0 3024814.0 113.0 Power_Rangers_in_Space Jason_Narvy link
32053.0 4.346807e7 115.0 Utrecht_University List_of_people_associated_with_Utrecht_University other
57731.0 62276.0 10.0 Leopold_II_of_Belgium Monarchy_of_Belgium link
298518.0 18189.0 17.0 Staraya_Ladoga Lake_Ladoga link
2929855.0 9962.0 56.0 Aenor_de_Châtellerault Eleanor_of_Aquitaine link
1196902.0 3.2826589e7 15.0 Saif_al-Islam_Gaddafi Rixos_Al_Nasr link
3119136.0 488105.0 144.0 British_Army_order_of_precedence Foot_Guards link
null 3.024141e7 10.0 other-empty Glover_v._United_States other
32308.0 21888.0 70.0 United_States_customary_units National_Institute_of_Standards_and_Technology link
1.4015242e7 345792.0 32.0 One_Night_Stand_(2008) The_Undertaker link
null 4.0759212e7 25.0 other-google Janet_Trujillo other
172369.0 51932.0 81.0 Challenger_2 Kinetic_energy_penetrator link
118444.0 3.6190531e7 12.0 Almere Almere_(lake) link
2.0395872e7 4066670.0 273.0 Oprah_Winfrey Kpelle_people link
5611605.0 1.6757325e7 57.0 Korean_People's_Army_Ground_Force Vz._52_machine_gun link
4.2411677e7 4.2567991e7 14.0 Cinedigm Chris_McGurk link
null 3.1257583e7 15.0 other-empty Bueng_Kan other
2.2258861e7 4.2534471e7 10.0 Total_penumbral_lunar_eclipse Tetrad_(astronomy) link
808402.0 1.6527475e7 115.0 Mexico_national_football_team Luis_Montes link
354296.0 994976.0 12.0 Vestment Monstrance link
362116.0 51784.0 85.0 Robinson_projection Map_projection link
4.2766864e7 2431566.0 37.0 2014–15_Northern_Premier_League Stamford_A.F.C. link
4251160.0 9313589.0 17.0 Taylor_Negron Wizards_of_Waverly_Place link
182214.0 2.0656228e7 17.0 Tassel Maize link
null 1595922.0 91.0 other-other Photosensitivity other
38872.0 4469999.0 15.0 Bessarabia Hotin_County link
1.3502823e7 1.252086e7 12.0 Brassiere Demi_Lovato link
4.4539913e7 1255017.0 26.0 The_Flintstones_&_WWE:_Stone_Age_SmackDown! Warner_Home_Video link
null 1.0433852e7 11.0 other-empty Robbie_Ellis other
1.2373195e7 33422.0 12.0 1984–85_Edmonton_Oilers_season Wayne_Gretzky link
1705212.0 133117.0 26.0 Lucknow_Pact Sarojini_Naidu link
309431.0 3.193663e7 24.0 Rhino_(wrestler) Leva_Bates other
1515653.0 1905405.0 20.0 Satellite_navigation Differential_GPS link
1.3099067e7 1.4306615e7 11.0 2007_Texas_Tech_Red_Raiders_football_team 2006_Texas_Tech_Red_Raiders_football_team link
4.4696649e7 1.3014023e7 31.0 Academy_Stadium Manchester_City_F.C._Reserves_and_Academy link
104650.0 38556.0 54.0 Oscar_II_of_Sweden List_of_Swedish_monarchs link
3.2457372e7 2.7152057e7 10.0 Sound_of_My_Voice Rostam_Batmanglij link
null 948103.0 44.0 other-other Meow_Mix other
553772.0 2100323.0 41.0 Robert_Pirès FWA_Footballer_of_the_Year link
6212853.0 4960.0 178.0 List_of_BSA_motorcycles Birmingham_Small_Arms_Company link
1.7006496e7 4.5002693e7 79.0 Batty_boy Homophobia_in_Jamaica link
4993017.0 706379.0 1033.0 The_Beach_Boys_discography Surfin'_Safari link
null 8790509.0 10.0 other-empty Folk_Lore_Museum_Mysore other
48946.0 5245439.0 29.0 Graz UPC-Arena other
92421.0 1.8396282e7 28.0 Interstate_64 Interstate_64_in_Indiana link
1.5580374e7 425059.0 81.0 Main_Page Ranch_dressing other
null 1.1172066e7 98.0 other-google Gerald_Stone other
3521882.0 81024.0 18.0 Airline_bankruptcies_in_the_United_States Pan_American_World_Airways link
null 294627.0 34.0 other-bing Mr._Garrison other
null 774449.0 75.0 other-wikipedia Richard_Lynn other
1.9975991e7 47707.0 49.0 Lex_Immers Feyenoord link
372478.0 1654769.0 15.0 Video_game_industry Artificial_intelligence_(video_games) link
null 207132.0 677.0 other-google Star_Trek:_Armada other
null 1863524.0 49.0 other-google Disney_Time other
null 780419.0 103.0 other-empty Philip_Don_Estridge other
null 1.4024425e7 17.0 other-wikipedia Nightwing_(novel) other
1020829.0 6260.0 23.0 20th-century_music Claude_Debussy link
4810477.0 1042506.0 24.0 Tawagalawa_letter Wilusa link
3821983.0 262233.0 167.0 1998_Russian_financial_crisis 1997_Asian_financial_crisis link
null 2.435744e7 68.0 other-wikipedia Académica_Petróleos_do_Lobito other
2.4097561e7 735348.0 13.0 List_of_newspaper_comic_strips_P–Z Pickles_(comic_strip) link
1669185.0 1624131.0 19.0 Phantasmagoria_(The_Damned_album) Roman_Jugg link
null 7197167.0 155.0 other-google Matei other
null 1.2315045e7 21.0 other-empty Dead_Air_(2009_film) other
null 969363.0 133.0 other-google Arlberg_technique other
891692.0 2.1803312e7 17.0 United_States_District_Court_for_the_Northern_District_of_California William_Haskell_Alsup link
957326.0 4915083.0 74.0 Italy_national_rugby_union_team Rugby_union_in_Italy link
null 8399976.0 146.0 other-wikipedia ASU-57 other
null 1558354.0 684.0 other-empty Demographics_of_Europe other
null 1.305149e7 134.0 other-google Courtney_Love_discography other
3.9654815e7 1034678.0 29.0 MFi_Program MFI other
1.9283769e7 3192804.0 14.0 Ausar Ausar_Auset_Society link
null 472075.0 65.0 other-google Viscount_Brookeborough other
262376.0 3183896.0 12.0 Roger_Federer 2006_Tennis_Masters_Cup link
1.1161932e7 1.1590877e7 12.0 Saints_Row_2 Red_Faction:_Guerrilla link
7712434.0 4579172.0 36.0 Vijay_Arora Yaadon_Ki_Baaraat link
null 4496629.0 14.0 other-empty One_Must_Fall:_Battlegrounds other
null 3.6894375e7 26.0 other-google Dragan_Lakićević other
211080.0 170459.0 110.0 Method_Man LL_Cool_J link
2646730.0 6733556.0 27.0 Pokémon:_The_First_Movie_(soundtrack) Pokémon_2.B.A._Master link
null 9472399.0 10.0 other-google Black_Duck_(group) other
2663129.0 16321.0 411.0 Good_Night,_and_Good_Luck Joseph_McCarthy link
1.1024497e7 1.0398699e7 35.0 Brown-Séquard_syndrome Central_cord_syndrome link
null 72434.0 14.0 other-empty Maximilian_Kaller other
202886.0 5869.0 10.0 Covariance_and_contravariance_of_vectors Category_theory link
6742.0 653196.0 16.0 Central_Asia Economic_Cooperation_Organization link
942704.0 46525.0 14.0 Song_of_Songs_(disambiguation) Wilfred_Owen link
null 1.0987478e7 62.0 other-google Pouch_Cove other
null 2122098.0 266.0 other-google Seabiscuit:_An_American_Legend other
null 454779.0 24.0 other-wikipedia Twitch_City other
96775.0 96815.0 24.0 Fulton_County,_Georgia Carroll_County,_Georgia link
296849.0 2520715.0 10.0 Ernest_Becker Sam_Keen link
347713.0 284262.0 28.0 Huia Callaeidae link
1.9137828e7 2.0753462e7 370.0 List_of_stadiums_under_construction Estadio_La_Peineta link
19859.0 277696.0 13.0 Moby-Dick The_Scarlet_Letter other
16844.0 523445.0 169.0 Kofi_Annan Lakhdar_Brahimi link
null 2.1230974e7 20.0 other-google Chhantyal other
2.892839e7 113519.0 36.0 Nucky_Thompson Short_(finance) link
219731.0 36168.0 30.0 Myst 3DO_Interactive_Multiplayer link
4.4845611e7 3.1597122e7 1377.0 My_Sunshine Wallace_Chung link
1107477.0 1345497.0 24.0 River_gunboat USS_Cairo link
4.2688351e7 5933689.0 439.0 2014–15_FC_Barcelona_season Jérémy_Mathieu link
4461110.0 1.0640807e7 41.0 Shilpa_Shirodkar Bhrashtachar link
77390.0 2.3769406e7 155.0 Natalie_Wood Sex_and_the_Single_Girl_(film) link
57877.0 7280414.0 53.0 Sodium_hydroxide List_of_commonly_available_chemicals other
null 683450.0 983.0 other-google Dawson's_Field_hijackings other
1990194.0 1.6290655e7 10.0 Ima_Hogg Thomas_Elisha_Hogg link
14015.0 2226.0 32.0 Herstory Ad_hominem other
3.1990324e7 4.2269335e7 19.0 List_of_Switched_at_Birth_episodes The_Futon_Critic link
null 9081713.0 30.0 other-bing The_Bitter_End other
108956.0 575052.0 120.0 Washington,_D.C. Verizon_Center link
1415821.0 1998515.0 72.0 Fiat_Ducato JTD_engine link
218742.0 490089.0 159.0 Ontario_Hockey_League Peterborough_Petes link
null 1792799.0 9379.0 other-google Stakeholder_theory other
null 216639.0 33.0 other-wikipedia Gondwanatheria other
3.0875744e7 26961.0 13.0 Artur_Rasizade Shia_Islam link
2.4378497e7 null 28.0 Mark_Salling Rocky_Road_(TV_Movie) redlink
1.1900681e7 312228.0 143.0 List_of_teen_films House_Party_(film) link
2608405.0 2418357.0 13.0 Roy_Mayorga Shelter_(band) link
null 2.1312852e7 10.0 other-yahoo Igor_Sijsling other
2809559.0 414267.0 12.0 Salirophilia Lust_murder link
null 3.0876253e7 116.0 other-wikipedia List_of_Jewish_prayers_and_blessings other
3966054.0 537551.0 28.0 Mexico Azteca_(multimedia_conglomerate) other
null 2.7437943e7 42.0 other-wikipedia Donna_Simpson_(internet_celebrity) other
null 1318322.0 89.0 other-other Callable_bond other
1972785.0 1053430.0 16.0 Illinois_(album) 2005_in_music link
null 2875803.0 68.0 other-google Lennon_(musical) other
438417.0 26977.0 11.0 Orange_County_(film) Stanford_University link
4178394.0 1589455.0 17.0 Chamar Kanshi_Ram link
null 390818.0 650.0 other-empty Endive other
null 2186604.0 21.0 other-google Caldecott,_Rutland other
976050.0 1.9084502e7 23.0 Fell's_Point,_Baltimore Michael_Phelps link
1027240.0 1.165959e7 130.0 Biryani Hyderabadi_cuisine link
4887.0 11523.0 236.0 British_Army Falklands_War link
498478.0 1.1395198e7 12.0 Shanghai_World_Financial_Center World_Trade_Center link
193918.0 39282.0 10.0 Kurgan Caucasus link
4754.0 1009445.0 12.0 Blue_Streak_(missile) European_Launcher_Development_Organisation link
206928.0 236519.0 11.0 List_of_birds_of_New_Zealand Australian_pelican link
7720774.0 1908172.0 15.0 WIN.INI INI_file link
null 3.101165e7 26.0 other-empty Betty_Jane_Gorin-Smith other
1.8952953e7 411723.0 173.0 Peyote Chihuahuan_Desert other
3.3469792e7 2412317.0 11.0 Here's_to_You_(song) Franz_Josef_Degenhardt link
null 2205218.0 40.0 other-wikipedia Six_Pieces_for_Piano,_Op._118_(Brahms) other
null 3872704.0 20.0 other-empty Caridad_de_la_Luz other
1.7621236e7 1.4485544e7 11.0 H._Eugene_Stanley List_of_members_of_the_National_Academy_of_Sciences_(Applied_physical_sciences) link
null 1495605.0 31.0 other-bing Hew_Strachan other
1.2136846e7 358527.0 27.0 The_Unfairground Kevin_Ayers link
4.3047831e7 4.3797734e7 113.0 Giant_in_My_Heart Sound_of_a_Woman link
null 9656353.0 73.0 other-google New_folk_media other
2909374.0 1.895149e7 11.0 Health_issues_in_American_football American_football link
null 595888.0 11.0 other-google Hail,_Vermont! other
null 2.3306201e7 11.0 other-empty EFestivals other
4446461.0 524502.0 63.0 Film_budgeting Box_office_bomb link
null 1.3437204e7 18.0 other-bing New_classical_macroeconomics other
2772399.0 1.8595033e7 134.0 Bill_T._Jones Arnie_Zane link
null 5903.0 13.0 other-yahoo Cultural_movement other
551711.0 408652.0 25.0 Ron_Dellums Barbara_Lee link
3.2628378e7 2.9017966e7 39.0 List_of_horror_films_of_1972 The_Fiend_(film) link
2945357.0 1770333.0 31.0 Amateur_rocketry High-power_rocketry link
280929.0 1278771.0 36.0 Erechtheion Palladium_(classical_antiquity) other
null 2342731.0 23.0 other-yahoo Walther_von_Seydlitz-Kurzbach other
18727.0 579219.0 17.0 List_of_food_additives,_Codex_Alimentarius Beta-Carotene other
2.548813e7 219287.0 33.0 Separate_legal_entity Legal_personality other
376581.0 2.0611504e7 12.0 Transnistria Russian_Empire link
3.3623665e7 2592839.0 18.0 Thimar Anouar_Brahem link
null 2.5650291e7 11.0 other-bing 2010_Tampa_Bay_Buccaneers_season other
2064142.0 346585.0 11.0 Chafing_dish Brazier link
null 461211.0 4961.0 other-google Mizoram other
660231.0 1257944.0 14.0 Adelaide_Oval WACA_Ground link
null 2.5679855e7 21.0 other-wikipedia Banco_Bicentenario other
1.30362e7 3.6604477e7 25.0 Automobile_drag_coefficient Jaguar_XE link
417606.0 1.4265701e7 76.0 Expedia_(website) Wotif.com link
3.5645946e7 1629175.0 22.0 Luís_Leal_(footballer) G.D._Estoril_Praia link
2.6458478e7 959456.0 100.0 German_military_brothels_in_World_War_II Roundup_(history) link
null 2.9027399e7 22.0 other-google The_Dog_Who_Saved_Christmas_Vacation other
null 1988582.0 33.0 other-empty Kovač other
null 987320.0 37.0 other-other Neurotechnology other
1022665.0 2366194.0 18.0 String_Quartets_Nos._7–9,_Op._59_–_Rasumovsky_(Beethoven) String_Quartet_No._10_(Beethoven) link
1373758.0 843532.0 49.0 Luther_Adler Stella_Adler link
null 9118436.0 13.0 other-google Jeremy_Huw_Williams other
1677928.0 268683.0 23.0 Children's_song Riddle other
null 7448450.0 18.0 other-other Louis_Zorich other
2.0803357e7 4.0026015e7 20.0 Parks_and_Recreation Drew_Barrymore_filmography other
4604481.0 4.02451e7 61.0 Storm_Model_Management Tiah_Delaney link
7481030.0 3098376.0 70.0 MPEG-4_Part_14 Comparison_of_audio_coding_formats link
null 3.920506e7 10.0 other-empty 72nd_Division_(United_Kingdom) other
1.7401306e7 2488280.0 139.0 Ripstop Ballistic_nylon link
1.0904287e7 423689.0 25.0 Termcap Curses_(programming_library) link
null 2.3268278e7 16.0 other-wikipedia Trakr other
null 458310.0 19.0 other-wikipedia Frank_Teschemacher other
47271.0 83835.0 16.0 Sponge Gonad link
null 1793967.0 43.0 other-other Pitman_arm other
3.1192297e7 2.1377045e7 298.0 Magic_City_(TV_series) Rick_Ross link
null 1362205.0 100.0 other-other Area_code_920 other
58478.0 2260425.0 15.0 Airborne_forces 502nd_Infantry_Regiment_(United_States) other
1510249.0 414822.0 35.0 Bering_Strait_crossing Tung-Yen_Lin other
1601795.0 5231575.0 11.0 V24_engine V5_engine link
44784.0 5533243.0 12.0 Bari The_Bridges_of_Madison_County_(film) link
null 4038132.0 47.0 other-empty Gary_Gaines other
1.6378571e7 1.6085877e7 273.0 Genealogies_in_the_Bible Abraham's_family_tree link
762354.0 1127973.0 11.0 Greatest_Hits_(ZZ_Top_album) Billy_Gibbons link
462091.0 3056665.0 113.0 Vairocana Sambhogakāya link
null 1210333.0 1335.0 other-google Millfield other
397810.0 2035119.0 12.0 Alexander_McQueen Elie_Saab link
8169386.0 3.7918222e7 13.0 Rick_Sebak Yinztagram link
1928513.0 3662295.0 13.0 Government_of_India_Act_1858 Dominion_of_Pakistan link
7919595.0 2615949.0 72.0 Clos_network Omega_network link
null 4.0485213e7 15.0 other-google Singapore_national_under-19_football_team other
null 2.0911775e7 14.0 other-wikipedia Fyodor_Druzhinin other
84112.0 84109.0 21.0 Berenice Berenice_II_of_Egypt link
null 1252215.0 30.0 other-google Dancer_with_Bruised_Knees other
null 17703.0 363.0 other-bing Leo_(constellation) other
1.6975268e7 3.0873764e7 26.0 Chess_theory Scandinavian_Defense link
null 1.6787376e7 56.0 other-empty Universiti_Malaysia_Terengganu other
null 347603.0 227.0 other-empty Carpenter_bee other
1.7798548e7 1.9183413e7 10.0 Rush_(2008_TV_series) Stephen_Rae_(composer) link
314628.0 1.0188712e7 15.0 Tooth_enamel Cusp_(anatomy) link
null 44903.0 169.0 other-yahoo Astor_Piazzolla other
1.9988138e7 1885136.0 23.0 Ramsay_(surname) Clan_Ramsay link
null 4.4010295e7 2438.0 other-google Survivor's_Remorse other
null 735443.0 25.0 other-bing Neumann_boundary_condition other
714047.0 544762.0 39.0 Chromolithography Offset_printing link
null 1686225.0 15.0 other-yahoo Tamara_Karsavina other
5859950.0 7616334.0 39.0 When_Worlds_Collide_(1951_film) Larry_Keating link
2.4132083e7 2955815.0 126.0 Dexter_(season_4) Julia_Campbell link
null 466851.0 16.0 other-empty Azteca_(band) other
1164252.0 527125.0 10.0 GamePro Game_Informer other
null 3.1863547e7 34.0 other-other Tarun_Khanna other
null 9038883.0 26.0 other-empty College_of_Physicians_and_Surgeons_Pakistan other
958572.0 592436.0 10.0 Glenn_Hall Ted_Lindsay link
3820404.0 3.0812082e7 10.0 Cross-dressing_in_film_and_television Bucket_&_Skinner's_Epic_Adventures link
2.7205785e7 2.8233212e7 25.0 School_attacks_in_China_(2010–12) 2010_Hebei_tractor_rampage other
3.4445585e7 3.8764549e7 40.0 American_Idol_(season_12) Curtis_Finch,_Jr. link
402942.0 436614.0 80.0 List_of_traditional_children's_games Pat-a-cake,_pat-a-cake,_baker's_man link
null 893198.0 16.0 other-other What_I_Learned_About_Ego,_Opinion,_Art_&_Commerce other
871210.0 1292261.0 38.0 Utricularia Utricularia_vulgaris link
5575722.0 11585.0 33.0 Fuck Show_Me_Love_(film) other
5465550.0 5512301.0 19.0 Morphinan Levomethorphan link
3.086259e7 9499.0 116.0 Link_layer Ethernet link
46336.0 5376.0 28.0 Passerine Cladistics other
158558.0 154820.0 18.0 King_of_the_Romanians List_of_rulers_of_Wallachia link
2.7804243e7 416577.0 10.0 List_of_birds_of_Pennsylvania Alder_flycatcher link
851800.0 915646.0 83.0 Air_America_(film) Pilatus_PC-6_Porter other
null 2.2976039e7 150.0 other-google Armenian_Wikipedia other
2019407.0 764428.0 28.0 Ali_Azmat Bhat link
4097772.0 2162718.0 11.0 Battle_of_Honey_Springs James_G._Blunt link
null 3824728.0 31.0 other-google Can_U_Get_wit_It other
null 1.0567795e7 135.0 other-google Robbie_van_Leeuwen other
null 4100412.0 111.0 other-google The_Truth_(Melbourne_newspaper) other
436522.0 229703.0 11.0 Hot_rod Roots-type_supercharger link
4.1660623e7 1.3280198e7 577.0 Tokyo_Ghoul Ling_Tosite_Sigure link
3.5034514e7 3.554456e7 12.0 2012_World_Junior_Championships_in_Athletics 2012_World_Junior_Championships_in_Athletics_–_Men's_100_metres link
null 2.4303131e7 18.0 other-wikipedia Treska other
null 1.7262978e7 19.0 other-bing Shake_It_(Metro_Station_song) other
6833695.0 1.2237982e7 10.0 Demihypercube Hypercubic_honeycomb other
1253121.0 264458.0 38.0 Battle_of_Kennesaw_Mountain Joseph_E._Johnston link
null 2.6626591e7 15.0 other-wikipedia Banqiao_Station other
null 239930.0 929.0 other-empty Temple_University other
679346.0 172063.0 153.0 Lucozade Ribena link
null 2.4619717e7 146.0 other-google Much_the_Same other
1.5609213e7 1602398.0 138.0 List_of_airlines_of_Nigeria Associated_Aviation link
1.2727445e7 null 22.0 I'm_Not_Like_Everybody_Else The_Sacred_Mushroom redlink
1478064.0 3.7165545e7 34.0 Andy_Souwer Steve_Moxon link
36396.0 39995.0 57.0 1214 1213 link
null 212416.0 14.0 other-yahoo Phitsanulok_Province other
1.8619244e7 305854.0 53.0 SMS_language Text_messaging link
180437.0 27071.0 89.0 Pavel_Chekov Star_Trek:_The_Original_Series link
null 4064.0 394.0 other-google Borsuk–Ulam_theorem other
null 1018286.0 284.0 other-empty Capri_Sun other
null 160753.0 28.0 other-twitter Manuel_L._Quezon other
3.6169584e7 4.0603571e7 23.0 2014–15_figure_skating_season Lombardia_Trophy link
1251507.0 1.0564133e7 207.0 Kirk_Acevedo Joe_Toye link
1302191.0 4393323.0 10.0 Opel_Commodore Ranger_(automobile) other
3.1523612e7 3.1348196e7 17.0 Mark_McNeill Phillip_Danault link
5906626.0 2.0534384e7 33.0 Horace_Trumbauer Elkins_Estate link
1006148.0 59003.0 10.0 Ludlow_(disambiguation) Ludlow link
3515315.0 1.0074452e7 19.0 Reebok_Freestyle Reebok_Classic other
null 5257744.0 40.0 other-wikipedia Some_Kind_of_Hero other
573177.0 2.3740297e7 31.0 Wendish_Crusade Wagria link
1.0367494e7 107204.0 12.0 Fried_pickle Atkins,_Arkansas link
743895.0 2278793.0 19.0 Timeline_of_Eastern_philosophers Parashara link
null 2.2484087e7 16.0 other-google Nnooo other
5824627.0 5042916.0 37.0 Inheritance_tax Canada link
40656.0 7247.0 29.0 13th_century_BC Cemetery_H_culture link
null 4.0389354e7 19.0 other-wikipedia ASAN_service other
null 334882.0 13.0 other-wikipedia Chamarajanagar_district other
299404.0 741705.0 18.0 Gunnery_sergeant Mark_Harmon link
1424575.0 4.0371665e7 31.0 Battle_of_Bailén Dominique_Honoré_Antoine_Vedel link
null 6598147.0 1322.0 other-google Concentration_risk other
1858211.0 21444.0 24.0 The_Jew_of_Malta Niccolò_Machiavelli link
16880.0 46853.0 13.0 Karnataka Indus_Valley_Civilization link
null 2.7255423e7 28.0 other-google Fornham_St_Genevieve other
2437139.0 1552544.0 15.0 Russian_architecture Onion_dome link
56315.0 6423327.0 14.0 Mango 2-Furanone other
8087287.0 1254779.0 164.0 Hot_Blooded Double_Vision_(Foreigner_album) link
null 3555863.0 114.0 other-other Frank_Hamer other
null 1.4270466e7 148.0 other-wikipedia Gautam_Adani other
2.0519849e7 4.0736758e7 38.0 Cage_discography Kill_the_Architect link
null 2.0776944e7 16.0 other-google Phintys other
null 2.4509692e7 38.0 other-empty Tri-City_Medical_Center other
3.6295719e7 1.817631e7 10.0 Garrett_(character) Guinness_World_Records_Gamer's_Edition link
87603.0 9295254.0 613.0 Robert_Mitchum Bentley_Mitchum link
2.1173707e7 3.6897865e7 20.0 Moreton_Bay_Rail_Link Murrumba_Downs_railway_station link
null 969732.0 15.0 other-wikipedia Brand_(disambiguation) other
1699425.0 3152733.0 19.0 Power-on_self-test Memory_refresh link
null 144948.0 137.0 other-yahoo Universal_joint other
571462.0 37585.0 13.0 National_Museum_of_the_United_States_Air_Force Museum link
890293.0 1.9147563e7 36.0 Mike_Smith_(actor) Thorburn,_Nova_Scotia link
876966.0 1.7702228e7 10.0 Vancouver_Island_University Higher_education_in_British_Columbia link
2003796.0 142421.0 22.0 Roscoe_Lee_Browne Babe_(film) link
null 528150.0 15.0 other-wikipedia Brothers_Keepers other
null 4958063.0 87.0 other-google Palma_di_Montechiaro other
null 4.2035866e7 14.0 other-wikipedia Collection_manager other
1257770.0 4799962.0 22.0 Tim_Drake Jaime_Reyes link
null 675786.0 231.0 other-google Scaffold_(disambiguation) other
null 1630583.0 39.0 other-empty Arica_School other
2753730.0 1203602.0 12.0 Copa_Airlines_Flight_201 United_Airlines_Flight_585 link
1.4529239e7 16861.0 14.0 Zoophilia Kurt_Vonnegut link
null 4.2270307e7 13.0 other-bing Bad_(David_Guetta_and_Showtek_song) other
64946.0 21724.0 18.0 Danelaw Normandy link
3285435.0 39205.0 13.0 2010_Asian_Games Asian_Games link
null 467628.0 736.0 other-google Margaret_Dumont other
592456.0 598977.0 16.0 This_Is_My_Truth_Tell_Me_Yours James_Dean_Bradfield link
null 19965.0 105.0 other-yahoo Morphogenesis other
95185.0 6939163.0 886.0 Frantz_Fanon Black_Skin,_White_Masks link
null 2.7633566e7 24.0 other-wikipedia ConsensusDOCS other
80777.0 8309183.0 24.0 Kurdistan Koçgiri_Rebellion link
null 5064426.0 24.0 other-yahoo Misha_Glenny other
null 4.0846967e7 23.0 other-google Hitkarini_Sabha other
null 2.2600019e7 17.0 other-wikipedia Country_folk other
99782.0 1154193.0 50.0 Vritra Aesir-Asura_correspondence link
246020.0 3516576.0 13.0 Freydís_Eiríksdóttir Greenland_saga other
12463.0 6124461.0 124.0 Glacier Quelccaya_Ice_Cap link
3628651.0 67234.0 10.0 New_Jersey's_10th_congressional_district Newark,_New_Jersey link
1878882.0 2.5016782e7 30.0 List_of_anthropomorphic_animal_superheroes Quick_Draw_McGraw link
null 126987.0 91.0 other-google Cleveland,_New_York other
6310617.0 2711314.0 19.0 Cristine_Rose How_I_Met_Your_Mother link
null 266033.0 308.0 other-empty Robert_I,_Duke_of_Normandy other
3125454.0 3536263.0 130.0 David_J._O'Reilly Kenneth_T._Derr link
1.9165698e7 228211.0 12.0 Teletoon_at_Night Futurama link
6452550.0 3.8121496e7 130.0 Hayley_Tamaddon List_of_Coronation_Street_characters_(2013) other
null 4.171206e7 45.0 other-empty Consider_the_Source other
1104597.0 494926.0 49.0 Kirovohrad Kirovohrad_Oblast link
1928711.0 896897.0 16.0 Etheric_plane Plane_(Dungeons_&_Dragons) other
null 2.2654444e7 419.0 other-google Pineapple_Dance_Studios other
1.3895544e7 788074.0 36.0 Vigor Physical_strength link
2583157.0 2.2509614e7 21.0 Byzantine_dress English_medieval_clothing link
null 8055634.0 15.0 other-google Unbarred_lenticular_galaxy other
1445268.0 236723.0 13.0 Master_of_Arts_(Oxbridge_and_Dublin) Master_of_Arts_(disambiguation) link
2.7656285e7 1739962.0 20.0 Geo_URI ICBM_address link
null 1.2892672e7 110.0 other-google Leandra other
1.5580374e7 3722614.0 18.0 Main_Page African_Cup_Winners'_Cup other
1.0078096e7 245335.0 55.0 This_Is_Just_To_Say Found_poetry link
2172281.0 1.0774494e7 32.0 Mumtaz_(actress) Apna_Desh link
null 2.7289759e7 19.0 other-empty 2010_Santos_FC_season other
39021.0 1.2230576e7 40.0 Daytona_500 Coke_Zero_400 link
3016712.0 4682876.0 10.0 Terminal_degree Professional_degrees_of_public_health other
939423.0 7216989.0 49.0 Mr._Lawrence The_Grim_Adventures_of_Billy_&_Mandy other
null 2697919.0 11.0 other-wikipedia Antemnae other
2463448.0 3.0668895e7 23.0 Ted_McCarty Gibson_Guitar_Corporation link
845407.0 2.8320131e7 25.0 Sezen_Aksu Ağlamak_Güzeldir link
null 1.4454507e7 79.0 other-google Eleider_Álvarez other
null 4923077.0 209.0 other-google Demon_Seed_(novel) other
4666669.0 2796527.0 10.0 Area_code_904 T-Pain link
null 1.9111554e7 11.0 other-empty Hermann_Hauser,_Sr. other
31827.0 145144.0 78.0 Demographics_of_Ukraine Ukrainians link
1980240.0 3.9277098e7 138.0 List_of_American_comedy_films About_Last_Night_(2014_film) link
730462.0 558569.0 16.0 Flower-class_corvette HMCS_Oakville_(K178) link
609002.0 1.4814799e7 13.0 Biloela Callide_Dam link
27695.0 30403.0 15.0 Structured_programming Turing_machine link
12449.0 2013048.0 45.0 Mobile_Suit_Gundam_Wing Mobile_weapons link
1.094599e7 1.7608953e7 14.0 Alternative_versions_of_Wolverine Marvel_Zombies_2 link
null 1.6917052e7 26.0 other-empty Kacy_Rodgers other
null 1.3805947e7 30.0 other-empty Unlimited_Touch other
2.1444421e7 1.915323e7 62.0 Roberta_Flack_discography Born_to_Love link
null 6493684.0 10.0 other-google Oświęcim_County other
11033.0 1.1991546e7 13.0 Frederick_Douglass Civilization_Revolution link
1.8302482e7 38170.0 12.0 List_of_bisexual_people_(A–F) Bi-curious link
1009423.0 4528243.0 44.0 Talysh_people Talysh_Khanate link
3.6355277e7 3.0214103e7 23.0 Vikings_(TV_series) Falling_Skies other
null 2.996522e7 14.0 other-empty Peter_White_(Michigan) other
8035013.0 3.377343e7 11.0 Lee_Jung Saturday_Freedom link
3.3050531e7 1.1612491e7 153.0 List_of_Deadly_Women_episodes Murder_of_Shanda_Sharer link
8670674.0 7364118.0 22.0 U218_Videos U218_Singles link
1374327.0 826555.0 11.0 ETA_SA Breitling_SA link
2312056.0 202652.0 12.0 Pride_&_Prejudice_(2005_film) Romeo_+_Juliet link
6027027.0 2697824.0 23.0 House_of_Carters Andy_Samberg link
null 3.3484283e7 22.0 other-google Dovedale_by_Moonlight other
null 3360692.0 11.0 other-google Harvey_Hodder other
null 1.4940878e7 10.0 other-empty 1982_Baltimore_Colts_season other
453246.0 142058.0 18.0 Breakout_(video_game) Homebrew_Computer_Club link
65910.0 3.8481732e7 219.0 Printed_circuit_board Chemical_milling other
null 9646491.0 47.0 other-google Fouad_Abou_Nader other
253868.0 524481.0 45.0 Eye_of_the_Beholder_(video_game) Gold_Box link
null 2.0207353e7 14.0 other-wikipedia Type-90 other
2.9156836e7 2.8439144e7 96.0 Park_Ha-sun Dong_Yi_(TV_series) link
null 54530.0 30.0 other-wikipedia Bookmark_(disambiguation) other
197181.0 1.7898921e7 18.0 Kunming Yuantong_Temple link
5043734.0 14800.0 10.0 Wikipedia Icon other
973639.0 1929375.0 16.0 Lacombe Lacombe,_Alberta link
380569.0 53607.0 46.0 John_F._Kennedy_Center_for_the_Performing_Arts Edward_Durell_Stone link
149689.0 190226.0 10.0 Midnight's_Children 1981_in_literature link
null 1.73024e7 24.0 other-other Baltimore_City_Circuit_Courthouses other
null 2.1313911e7 10.0 other-bing Wind_power_in_Wyoming other
null 1.3005006e7 12.0 other-empty Douglas_Guest other
714928.0 1.5704166e7 30.0 Greenland_Dog Inuit link
null 9471611.0 28.0 other-empty Memphis_Light,_Gas_and_Water other
1585091.0 348208.0 16.0 List_of_Turkish_artists Avni_Arbaş link
null 4120275.0 56.0 other-google Marjie_Lundstrom other
39482.0 2079614.0 27.0 Mai_Zetterling Tutte_Lemkow link
46526.0 57744.0 10.0 419_scams Ivory_Coast link
52967.0 411914.0 16.0 Gynaecology Oophorectomy link
6059111.0 2.8039598e7 17.0 Ethan_Spaulding The_Legend_of_Korra link
null 501536.0 20.0 other-yahoo Ministry_of_Intelligence other
480634.0 1989200.0 27.0 Absorbance Densitometry link
3924114.0 2455426.0 26.0 Bottom_Dollar_Food PriceRite link
null 2.1715001e7 15.0 other-google You_Are_My_Joy other
1.8899968e7 44700.0 11.0 List_of_Chinese_discoveries Leprosy link
984322.0 2176065.0 45.0 Krome_Studios_Melbourne Nightshade_(1992_video_game) other
null 867983.0 24.0 other-wikipedia Microsoft_Narrator other
3303790.0 2.3384265e7 21.0 Military_history_of_Mexico Mexican_Armed_Forces other
null 5435750.0 14.0 other-wikipedia Punk-O-Rama_5 other
24555.0 129618.0 11.0 Photosynthetic_pigment Cyanobacteria link
null 1.2391537e7 11.0 other-other Flores_(canton) other
null 8703722.0 16.0 other-wikipedia Get_down other
117337.0 181005.0 35.0 Westlake_Village,_California Robert_Young_(actor) link
1266404.0 201829.0 20.0 Hypalon DuPont link
null 150521.0 63.0 other-yahoo Henry_Armstrong other
null 46933.0 20.0 other-yahoo Spelljammer other
9947607.0 1610870.0 11.0 Nick_Raskulinecz In_Your_Honor link
7890238.0 5400269.0 20.0 Test_Drive_4 TVR_Cerbera_Speed_12 link
3051596.0 611873.0 17.0 Chaminade_College_Preparatory_School_(California) West_Hills,_Los_Angeles other
null 2054489.0 92.0 other-empty Jean_van_de_Velde_(golfer) other
1.8220755e7 3.9815494e7 123.0 Holly_Hunter Bonnie_&_Clyde_(2013_miniseries) link
null 1.2201032e7 83.0 other-wikipedia M_jak_miłość other
1511052.0 1182345.0 40.0 Jim_Kelly_(martial_artist) Undercover_Brother link
127894.0 3.2564669e7 30.0 Winston-Salem,_North_Carolina Novant_Health link
72566.0 806290.0 29.0 Carmina_Burana Cockaigne link
1018512.0 1.2219012e7 12.0 Culture_of_Burma Burmese_dance link
4.4789934e7 1.9156186e7 10.0 Deaths_in_2015 Adrian_Peterson other
null 1615103.0 51.0 other-other Lucy_Ford:_The_Atmosphere_EP's other
null 3.4567346e7 17.0 other-google Simon_Paulli other
1113778.0 235321.0 40.0 Heaven_Tonight Rick_Nielsen link
null 2.1158505e7 97.0 other-other SDL_Trados other
null 1.7838929e7 13.0 other-google Christian_Maclagan other
294791.0 4.248568e7 18.0 Steven_Moffat Time_Heist link
null 3.5096982e7 14.0 other-empty Even_If_It_Breaks_Your_Heart other
null 3352391.0 33.0 other-wikipedia Pecheneg_language other
109495.0 2835130.0 25.0 Key_West,_Florida Key_Haven,_Florida link
22093.0 1658814.0 233.0 National_Basketball_Association Barclays_Center link
24096.0 1.9049004e7 16.0 Plough Stump-jump_plough link
null 5576399.0 10.0 other-wikipedia Santahamina other
null 1.3646286e7 29.0 other-google WYFI other
null 1803482.0 21.0 other-empty Aldene_Connection other
308142.0 4.1443125e7 26.0 General_Santos SM_City_General_Santos link
1.9486157e7 2.0975298e7 116.0 Mirotic Mirotic_(song) other
192381.0 1.7161967e7 27.0 Joe_Pantoliano The_Handler_(TV_series) link
967278.0 2604085.0 49.0 KiKa Bernd_das_Brot link
null 8252419.0 853.0 other-google Foxhole other
55906.0 17391.0 16.0 Zagreb Kosovo link
null 130195.0 23.0 other-google Covington,_Oklahoma other
null 2984353.0 21.0 other-empty 251_(number) other
23324.0 20474.0 22.0 Platinum Mohs_scale_of_mineral_hardness link
1298.0 2.3555068e7 10.0 Ames,_Iowa Neva_Morris link
2117651.0 1.808438e7 18.0 AFI's_100_Years...100_Movie_Quotes George_M._Cohan link
null 2.8370582e7 14.0 other-google Palača other
138022.0 267590.0 34.0 North_Bend,_Washington Mount_Si link
null 4.0810754e7 325.0 other-google List_of_travel_books other
2387806.0 12301.0 26.0 Harry_Potter A_Song_of_Ice_and_Fire other
48630.0 7159144.0 11.0 2014 Michael_Sata link
1.1976532e7 3.7562767e7 84.0 TOP500 Graph500 link
599365.0 1.904146e7 78.0 Liiga Kanada-malja link
null 4.1274079e7 44.0 other-wikipedia Jeremy_Jamm other
4.0379651e7 8744746.0 144.0 IBM Big_Blue_(disambiguation) link
52036.0 5573.0 50.0 Istria Croatia link
null 8260496.0 106.0 other-wikipedia Wrestle_Kingdom other
402652.0 581760.0 27.0 Compulsive_hoarding Plyushkin link
4.2839033e7 2.1418097e7 13.0 List_of_hot_dog_restaurants Montreal_Pool_Room link
null 2.394422e7 28.0 other-wikipedia Iván_Pillud other
206004.0 988219.0 14.0 Military_history_of_Egypt_during_World_War_II East_African_Campaign_(World_War_II) link
null 1.0235545e7 14.0 other-other D'Ieteren other
null 475805.0 548.0 other-google Pansy_Division other
49966.0 287855.0 199.0 Carlos_Menem Cecilia_Bolocco link
null 1.4750893e7 44.0 other-google Ain't_No_Shame_in_My_Game other
2.0550801e7 2.6771113e7 10.0 La_Scala_(album) Tokyo_'96 link
null 7304198.0 22.0 other-empty Ernst_Fischer other
1.5851039e7 10618.0 11.0 Davy_Crockett_and_the_River_Pirates Fiddle other
null 1.1927959e7 17.0 other-wikipedia Metal_Gear_Solid_2:_Sons_of_Liberty_Soundtrack_2:_The_Other_Side other
null 38325.0 15.0 other-bing Descent other
null 6017828.0 11.0 other-empty Center_for_Libertarian_Studies other
3.0873608e7 245390.0 21.0 Metal_Gear_(video_game) Stealth_game link
2.0846219e7 2.084622e7 12.0 Benzathine Benzathine_phenoxymethylpenicillin link
null 3.666971e7 21.0 other-google Sydney_state_by-election,_2012 other
498348.0 1587778.0 12.0 Guinea_Pig_(film_series) Japanese_horror other

Display is a utility provided by Databricks. If you are programming directly in Spark, use the show(numRows: Int) function of DataFrame

clickstream.show(5)
+---------+-----------+---+--------------------+--------------+-----+
|  prev_id|    curr_id|  n|          prev_title|    curr_title| type|
+---------+-----------+---+--------------------+--------------+-----+
|  37284.0|   197438.0| 41|         Brain_tumor|    Fontanelle| link|
|2904478.0|2.9932496E7| 14|Ottoman_Reform_Ed...|Hatt-i_humayun|other|
|     null|   412127.0| 39|     other-wikipedia|     Tony_Zale|other|
|2368683.0|   209811.0| 15|      Trajan's_Forum|       Looting|other|
|7691324.0|    17616.0| 14|          Zonal_flow|      Latitude| link|
+---------+-----------+---+--------------------+--------------+-----+
only showing top 5 rows

Reading from disk vs memory

The 1.2 GB Clickstream file is currently on S3, which means each time you scan through it, your Spark cluster has to read the 1.2 GB of data remotely over the network.

Call the count() action to check how many rows are in the DataFrame and to see how long it takes to read the DataFrame from S3.

clickstream.cache().count()
res8: Long = 224809
  • It took about several minutes to read the 1.2 GB file into your Spark cluster. The file has 22.5 million rows/lines.
  • Although we have called cache, remember that it is evaluated (cached) only when an action(count) is called

Now call count again to see how much faster it is to read from memory

clickstream.count()
res9: Long = 224809
  • Orders of magnitude faster!
  • If you are going to be using the same data source multiple times, it is better to cache it in memory

What are the top 10 articles requested?

To do this we also need to order by the sum of column n, in descending order.

//Type in your answer here...
display(clickstream
  .select(clickstream("curr_title"), clickstream("n"))
  .groupBy("curr_title")
  .sum()
  .orderBy($"sum(n)".desc)
  .limit(10))
curr_title sum(n)
Anna_Kendrick 137305.0
Avengers:_Age_of_Ultron 128626.0
Supernatural_(U.S._TV_series) 116686.0
Clint_Eastwood 106473.0
List_of_Person_of_Interest_episodes 100807.0
List_of_James_Bond_films 96604.0
E._L._James 95611.0
Uber_(company) 93768.0
Marco_Polo 75583.0
Main_Page 71087.0

Who sent the most traffic to Wikipedia in Feb 2015?

In other words, who were the top referers to Wikipedia?

display(clickstream
  .select(clickstream("prev_title"), clickstream("n"))
  .groupBy("prev_title")
  .sum()
  .orderBy($"sum(n)".desc)
  .limit(10))
prev_title sum(n)
other-google 1.5189547e7
other-empty 2345980.0
other-wikipedia 1169428.0
other-other 750332.0
other-bing 623218.0
other-yahoo 437115.0
Main_Page 348099.0
other-twitter 234104.0
Beck 68092.0
Deaths_in_2015 38579.0

As expected, the top referer by a large margin is Google. Next comes refererless traffic (usually clients using HTTPS). The third largest sender of traffic to English Wikipedia are Wikipedia pages that are not in the main namespace (ns = 0) of English Wikipedia. Learn about the Wikipedia namespaces here: https://en.wikipedia.org/wiki/Wikipedia:Project_namespace

Also, note that Twitter sends 10x more requests to Wikipedia than Facebook.

//Type in your answer here...
display(clickstream
  .select(clickstream("curr_title"), clickstream("prev_title"), clickstream("n"))
  .filter("prev_title = 'other-twitter'")
  .groupBy("curr_title")
  .sum()
  .orderBy($"sum(n)".desc)
  .limit(5))
curr_title sum(n)
12_Angry_Men_(1957_film) 54643.0
Beit_Aghion 20819.0
Mongolian_horse 9829.0
Vermiform_appendix 9678.0
American_Ninja 8266.0
val allClicks = clickstream.selectExpr("sum(n)").first.getLong(0)
val referals = clickstream.
                filter(clickstream("prev_id").isNotNull).
                selectExpr("sum(n)").first.getLong(0)
(referals * 100.0) / allClicks
allClicks: Long = 31696586
referals: Long = 10915724
res13: Double = 34.4381694609003
clickstream.createOrReplaceTempView("clicks")
SELECT *
FROM clicks
WHERE 
  curr_title = 'Donald_Trump' AND
  prev_id IS NOT NULL AND prev_title != 'Main_Page'
ORDER BY n DESC
LIMIT 20
prev_id curr_id n prev_title curr_title type
290327.0 4848272.0 596.0 German_American Donald_Trump link
1.0477604e7 4848272.0 68.0 ACN_Inc. Donald_Trump link
891829.0 4848272.0 12.0 Celebrity_(film) Donald_Trump link

YouTry: Top referrers to other 2016 US presidential candidate pages

'Donald_Trump', 'Bernie_Sanders', 'Hillary_Rodham_Clinton', 'Ted_Cruz'

-- YouTry 
---
-- fill in the right sql query here
val clicksDFsql = sql("""
    SELECT 
      prev_title AS src,
      curr_title AS dest,
      n AS count FROM clicks
    WHERE 
      curr_title IN ('Donald_Trump', 'Bernie_Sanders', 'Hillary_Rodham_Clinton', 'Ted_Cruz') AND
      prev_id IS NOT NULL AND prev_title != 'Main_Page'
    ORDER BY n DESC
    LIMIT 20""")
clicksDFsql: org.apache.spark.sql.DataFrame = [src: string, dest: string ... 1 more field]
clicksDFsql.show(false)
+-----------------------------------------------------+----------------------+-----+
|src                                                  |dest                  |count|
+-----------------------------------------------------+----------------------+-----+
|German_American                                      |Donald_Trump          |596  |
|List_of_current_United_States_Senators               |Ted_Cruz              |207  |
|Hilary_(name)                                        |Hillary_Rodham_Clinton|109  |
|ACN_Inc.                                             |Donald_Trump          |68   |
|United_States_congressional_delegations_from_Texas   |Ted_Cruz              |27   |
|Hillary_Rodham_Clinton                               |Bernie_Sanders        |20   |
|Angus_King                                           |Bernie_Sanders        |13   |
|David_Sirota                                         |Bernie_Sanders        |13   |
|Celebrity_(film)                                     |Donald_Trump          |12   |
|List_of_Southern_Baptist_Convention_affiliated_people|Ted_Cruz              |10   |
+-----------------------------------------------------+----------------------+-----+

Convert raw data to parquet

Recall:

Apache Parquet is a columnar storage format available to any project in the Hadoop ecosystem, regardless of the choice of data processing framework, data model or programming language. It is a more efficient way to store data frames.

// Convert the DatFrame to a more efficent format to speed up our analysis
// writing intermediate files to temporary dir .../tmp/... 
clickstream
  .write
  .mode(SaveMode.Overwrite)
  .parquet("/datasets/sds/tmp/wiki-clickstream") 

Load parquet file efficiently and quickly into a DataFrame

Now we can simply load from this parquet file next time instead of creating the RDD from the text file (much slower).

Also using parquet files to store DataFrames allows us to go between languages quickly in a a scalable manner.

val clicks = spark.read.parquet("/datasets/sds/tmp/wiki-clickstream")
clicks: org.apache.spark.sql.DataFrame = [prev_id: double, curr_id: double ... 4 more fields]
clicks.printSchema
root
 |-- prev_id: double (nullable = true)
 |-- curr_id: double (nullable = true)
 |-- n: integer (nullable = true)
 |-- prev_title: string (nullable = true)
 |-- curr_title: string (nullable = true)
 |-- type: string (nullable = true)
display(clicks)  // let's display this DataFrame
prev_id curr_id n prev_title curr_title type
37284.0 197438.0 41.0 Brain_tumor Fontanelle link
2904478.0 2.9932496e7 14.0 Ottoman_Reform_Edict_of_1856 Hatt-i_humayun other
null 412127.0 39.0 other-wikipedia Tony_Zale other
2368683.0 209811.0 15.0 Trajan's_Forum Looting other
7691324.0 17616.0 14.0 Zonal_flow Latitude link
null 1.9730805e7 170.0 other-empty Ryan_Jones_(ice_hockey) other
null 129375.0 32.0 other-empty Reading,_Ohio other
1.5580374e7 2554846.0 14.0 Main_Page Toots other
348959.0 3.5701262e7 53.0 Japanese_dialects Hokkaido_dialects link
null 614279.0 16.0 other-yahoo Nick_13 other
148898.0 3.6496677e7 13.0 Asian_American Watsonville_riots other
7997964.0 2.0992676e7 11.0 Chikara_Campeonatos_de_Parejas The_Colony_(professional_wrestling) link
null 5922274.0 22.0 other-empty John_William_Loudon other
null 1997924.0 30.0 other-empty Mister_Cartoon other
null 2774369.0 10.0 other-empty Knottingley_railway_station other
null 2.7283921e7 13.0 other-google Caret,_Virginia other
4940115.0 2370506.0 68.0 Drew_Mitchell RC_Toulonnais link
6514702.0 6982829.0 2755.0 Lonelygirl15 Jessica_Lee_Rose link
4.3010378e7 451169.0 74.0 Pro_Evolution_Soccer_2015 Swansea_City_A.F.C. other
null 2653829.0 12.0 other-google List_of_places_in_New_York:_V other
2979541.0 1.5580374e7 16.0 The_Kooks Main_Page other
2378072.0 877225.0 42.0 The_Crow:_Wicked_Prayer The_Crow:_Salvation link
49706.0 77491.0 14.0 Paul_Newman Gregory_Peck link
3084058.0 1352229.0 10.0 Sport_coat Harrington_jacket link
null 7079047.0 30.0 other-empty Mario_Gjurovski other
752246.0 1.1507227e7 13.0 Miller_Park_(Milwaukee) United_Football_League_(2009–12) other
412214.0 278018.0 18.0 Bill_Russell NBA_All-Star_Game link
158696.0 26272.0 11.0 Roland_TR-808 Ryuichi_Sakamoto link
322055.0 322060.0 254.0 USS_Constellation_(1797) USS_Constellation_(1854) link
525928.0 3.3402653e7 13.0 Special_agent Gus_Fring link
2.4930946e7 27169.0 113.0 Eric_Mangini San_Francisco_49ers link
null 3.5027653e7 48.0 other-wikipedia Mara_Maru other
2.1073732e7 3.3188989e7 12.0 Mexican–American_War Santa_Ana other
1.4912557e7 4.1491686e7 18.0 Caucasus_Emirate December_2013_Volgograd_bombings link
3219844.0 3002478.0 29.0 Azure_Dragon Horn_(Chinese_constellation) link
null 116508.0 43.0 other-empty Brentwood,_Maryland other
26750.0 220636.0 18.0 Sri_Lanka Universal_suffrage link
34404.0 341594.0 78.0 Economy_of_Zimbabwe Land_reform_in_Zimbabwe link
null 3.074887e7 25.0 other-google Sam_Weiss other
3.8870894e7 1.5580374e7 17.0 Lupe_Fuentes Main_Page other
null 5020425.0 1375.0 other-google FEG_PA-63 other
null 99891.0 341.0 other-empty Gulf_Shores,_Alabama other
1260484.0 2.2763421e7 23.0 Ubud Ubud_District link
1.6113578e7 2.065561e7 12.0 Ronald_Allen_Smith Montana_State_Prison link
null 1.199021e7 24.0 other-yahoo List_of_awards_and_nominations_received_by_Shah_Rukh_Khan other
3841.0 2266430.0 187.0 Bud_Spencer Ace_High_(1968_film) link
846862.0 1148.0 15.0 W._O._Bentley Adelaide link
null 34168.0 18.0 other-twitter Xenogears other
2.3712589e7 557091.0 76.0 Alternative_hip_hop Underground_hip_hop link
null 1.379992e7 31.0 other-google Gheorghe_Păun other
24140.0 28436.0 10.0 Paul_the_Apostle Saint other
null 8343572.0 16.0 other-wikipedia Hulwan other
null 2.392808e7 110.0 other-empty Craig_Dawson other
null 3.6155348e7 15.0 other-bing List_of_world_championships_medalists_in_powerlifting_(men) other
4673783.0 4676317.0 51.0 Double_fisherman's_knot Double_overhand_knot link
null 619770.0 10.0 other-wikipedia Royal_Academy_summer_exhibition other
5577654.0 601316.0 42.0 List_of_Fables_characters Ichabod_Crane link
11006.0 2.8363011e7 39.0 February_19 Sérgio_Júnior other
null 1.4978921e7 16.0 other-wikipedia Herbert_Walther other
19374.0 57546.0 154.0 Model_organism Caenorhabditis_elegans link
null 4049702.0 15.0 other-bing A-91 other
2.4572232e7 2.2008992e7 12.0 Joey_Sturgis Someday_Came_Suddenly link
1122303.0 6950178.0 13.0 Ben_Olsen Ben_Olson link
null 4403167.0 15.0 other-other Fereydoon_Moshiri other
null 423041.0 23.0 other-empty First_Circle other
1.3078837e7 2.7332848e7 12.0 2007_US_Open_–_Boys'_Singles Matteo_Trevisan link
null 6493680.0 11.0 other-google Mentai_Rock other
1.1197284e7 1.744141e7 12.0 Kurt_Pellegrino Júnior_Assunção other
887544.0 2420403.0 552.0 Alisha_Klass Seymore_Butts link
1.0609116e7 4.3173611e7 17.0 Arizona_Wildcats_men's_basketball Rondae_Hollis-Jefferson link
null 616622.0 724.0 other-wikipedia Andriy_Shevchenko other
2296379.0 1771587.0 14.0 Palmar_erythema Pregnancy link
6678.0 99426.0 19.0 Cat Naphthalene link
55502.0 55501.0 40.0 860s_BC 850s_BC link
2.181477e7 74940.0 106.0 Langston_Hughes Marian_Anderson link
763905.0 3.9681124e7 62.0 Tien Tien_(surname) link
4.4251018e7 3.7049649e7 14.0 HyperDex Spanner_(database) link
5697437.0 192755.0 65.0 The_Cockpit_(OVA) Yokosuka_MXY7_Ohka link
3084191.0 null 17.0 Dany_Verissimo John_B._Root redlink
9624289.0 1.9985931e7 329.0 DirecTV DirecTV_satellite_fleet link
31734.0 72227.0 18.0 Urea Plywood link
1078676.0 4486620.0 13.0 Burundian_Civil_War United_Nations_Operation_in_Burundi link
4408.0 1147922.0 168.0 Buddy_Holly Music_of_Lubbock,_Texas other
null 444112.0 33.0 other-yahoo Gnoll other
3369981.0 1516915.0 16.0 Flu_(disambiguation) Swine_influenza other
468301.0 982480.0 229.0 Samantha_Morton John_Carter_(film) link
null 6469961.0 35.0 other-wikipedia Sertorian_War other
null 3.8758012e7 109.0 other-other List_of_federal_subjects_of_Russia_by_GDP_per_capita other
null 3.6390439e7 25.0 other-google Major_Mining_Sites_of_Wallonia other
11887.0 895357.0 43.0 Greek_language English_words_of_Greek_origin link
21383.0 7901223.0 14.0 Nigeria Nigerian_general_election,_2007 link
731774.0 2539671.0 324.0 Law_of_Moses Ten_Commandments link
2.3896488e7 1.1263766e7 11.0 Vampire_Academy_(novel) Diary_of_a_Wimpy_Kid link
null 1508712.0 20.0 other-empty Mikoyan_MiG-110 other
2.1536106e7 2.7520075e7 17.0 Jennifer_Blake_(wrestler) Mari_Apache link
null 2.4052308e7 58.0 other-empty Francis_Crowley other
null 1.6792585e7 75.0 other-google Phulkian_sardars other
2812945.0 1.6230289e7 25.0 Double_Dutch_Bus Raven-Symoné_(album) link
3.9522631e7 253375.0 70.0 News_Corp HarperCollins link
null 2.1329684e7 17.0 other-other David_Lyon_(sociologist) other
338344.0 3.9032732e7 84.0 List_of_tallest_buildings_in_the_world Discovery_Primea link
3.9839062e7 1.9087186e7 12.0 List_of_unicorns Noah's_Ark_(2007_film) link
58666.0 3550910.0 12.0 United_States_Environmental_Protection_Agency Marine_Mammal_Protection_Act_of_1972 link
null 3.5675894e7 11.0 other-wikipedia AdMarketplace other
627321.0 203426.0 16.0 Burma_Campaign Nyasaland link
234382.0 378561.0 180.0 Elephant_(2003_film) Eric_Harris_and_Dylan_Klebold link
null 434919.0 81.0 other-google Alan_Meale other
null 2.6552915e7 21.0 other-empty Gertrude_Abercrombie other
1.9172225e7 206790.0 84.0 Prokaryote Spirochaete link
null 8759210.0 46.0 other-google TM_and_Cult_Mania other
34742.0 75831.0 27.0 5th_century Flavius_Aetius other
737160.0 4445580.0 10.0 Soilwork Sonic_Syndicate link
null 2.0594394e7 12.0 other-wikipedia Claus_Costa other
490391.0 682513.0 10.0 Adnan_Gulshair_el_Shukrijumah Abderraouf_Jdey link
null 5772621.0 21.0 other-wikipedia The_Watsons other
1.7015795e7 1.336124e7 23.0 Teyana_Taylor Blue_Magic_(song) link
8026795.0 14653.0 12.0 Iran–Turkey_relations Iran link
null 3.7569668e7 15.0 other-google I.O.U._(Jimmy_Dean_song) other
null 2.3416901e7 21.0 other-google Strathcarron_Sports_Cars other
303241.0 719034.0 14.0 Strong_Guy Mephisto_(comics) link
null 673275.0 203.0 other-other Scale_insect other
null 5925339.0 151.0 other-google Flintham other
11092.0 653246.0 39.0 Finger_Lakes Canadice_Lake link
45715.0 2011918.0 22.0 Arecaceae Minoo_Island link
8427319.0 245765.0 16.0 List_of_alternative_country_musicians The_Jayhawks other
93036.0 129736.0 13.0 Portage_County,_Ohio Hiram,_Ohio link
null 3.1998475e7 10.0 other-empty Royal_Botanical_Expedition_to_New_Granada other
3717.0 4911607.0 33.0 Brain Anterior_grey_column other
null 4.3030724e7 15.0 other-wikipedia 99th_Infantry_Battalion_(United_States) other
3615880.0 2.9646991e7 11.0 Ranger_(Dungeons_&_Dragons) The_Complete_Fighter's_Handbook link
null 3.6599164e7 26.0 other-wikipedia Rudding_Park_House other
null 2.0000187e7 405.0 other-wikipedia Inflection other
3.593072e7 574988.0 41.0 Stairway_to_Hell Ugly_Kid_Joe link
78127.0 3358304.0 10.0 James_Doohan Star_Trek_(film_franchise) other
4.0530767e7 19281.0 42.0 Visa_requirements_for_Palestinian_citizens Montserrat link
4100885.0 29301.0 27.0 Meaning_(linguistics) Semiotics link
null 1.9535017e7 48.0 other-google Aumism other
null 3.9268446e7 10.0 other-other Raúl_Duarte_(basketball) other
2.2942232e7 9891690.0 24.0 Lists_of_academic_journals List_of_pharmaceutical_sciences_journals link
null 9108276.0 797.0 other-other Kesh_(Sikhism) other
4306874.0 1615009.0 14.0 Khartoum_International_Airport Flynas link
3.3948854e7 2.7572278e7 10.0 Robert_A._J._Gagnon Jack_Rogers_(clergy) link
2.5318118e7 17867.0 19.0 Government_of_the_United_Kingdom London link
4.4729787e7 5070615.0 12.0 Marvel_Contest_of_Champions Marvel_Super_Heroes:_War_of_the_Gems link
65192.0 141976.0 17.0 Three_Gorges_Dam Alstom link
194664.0 924738.0 86.0 The_Flintstones_(film) Richard_Moll link
null 2370304.0 1180.0 other-google Sriperumbudur other
null 2.6678779e7 19.0 other-google Geeklog other
null 109049.0 55.0 other-other Margate,_Florida other
null 365310.0 961.0 other-google Vostok_(spacecraft) other
null 191537.0 271.0 other-other Internment other
null 1250542.0 27.0 other-other Jake_Peavy other
1069442.0 180425.0 39.0 Charles_Bickford Woodlawn_Memorial_Cemetery,_Santa_Monica link
null 3.8641017e7 3802.0 other-google GeForce_800M_series other
80387.0 77944.0 29.0 Hamadryad Hesperides link
18819.0 2399697.0 35.0 Microeconomics Heterodox_economics link
9566994.0 9574674.0 12.0 The_Renegade_(short_story) The_Silent_Men link
333703.0 161436.0 15.0 Angelo_Dundee Ernest_Borgnine link
2.6301553e7 10150.0 34.0 António_de_Oliveira_Salazar Engelbert_Dollfuss other
4.2255443e7 null 21.0 List_of_United_States_bomber_aircraft List_of_United_States_attack_aircraft redlink
null 1.2290729e7 40.0 other-wikipedia Monte_Carlo_(biscuit) other
7619598.0 2916479.0 33.0 Fusiliers_Commandos_de_l'Air German_Air_Force_Regiment link
7457961.0 1651671.0 28.0 List_of_Castlevania:_Aria_of_Sorrow_and_Dawn_of_Sorrow_characters Soma_Cruz link
null 1.8755551e7 167.0 other-google Hooli other
null 2.557502e7 12.0 other-wikipedia Leucadendron_album other
null 9041425.0 57.0 other-empty Country_Teasers other
null 1.9226982e7 17.0 other-google The_Wits other
5625186.0 2.3378704e7 22.0 Chadian–Libyan_conflict Chadian_Civil_War_(1965–79) link
null 2783508.0 17.0 other-other Encephalartos other
192042.0 155627.0 22.0 Over-the-counter_drug Ibuprofen link
null 3.8764007e7 31.0 other-empty Secret_Story_4_(Portugal) other
22205.0 1147409.0 305.0 Oasis Ein_Gedi link
1.8899195e7 4527556.0 12.0 Rhynchosaurus Rhynchosaur link
null 2.61932e7 42.0 other-wikipedia Charl_Van_Den_Berg other
1.9567899e7 2.5757144e7 224.0 Nintendo_DSi Foto_Showdown link
1.7981721e7 518667.0 13.0 Afamelanotide Erythropoietic_protoporphyria link
637310.0 1.0684273e7 41.0 Japanese_submarine_I-25 Leninets-class_submarine other
null 3958341.0 27.0 other-empty Atsushi_Itō_(actor) other
30085.0 1185285.0 20.0 Thomas_Mann Kilchberg,_Zürich link
2652103.0 2.65684e7 294.0 Tiling_window_manager Awesome_(window_manager) link
null 9653709.0 36.0 other-google Clint_Warwick other
1.802006e7 4857534.0 17.0 Brimonidine/timolol Brimonidine link
156778.0 52671.0 122.0 Hypochondriasis Psychosomatic_medicine link
1.3335045e7 335195.0 18.0 Dungeons_&_Dragons_in_popular_culture Stephen_Colbert link
3338747.0 4134000.0 18.0 Travis_Willingham Ouran_High_School_Host_Club link
2.4370563e7 1.9866746e7 66.0 Harry_Potter_and_the_Forbidden_Journey Transformers:_The_Ride link
3.6438468e7 2.483689e7 12.0 Alternative_versions_of_Joker Homosexuality_in_the_Batman_franchise link
8119611.0 33158.0 21.0 100_Photographs_that_Changed_the_World War link
2.4102801e7 8986839.0 72.0 List_of_Stradivarius_instruments Hammer_Stradivarius link
null 4043540.0 37.0 other-empty The_Silverado_Squatters other
1.2801225e7 198606.0 28.0 Blair_Tindall Malcolm_McDowell link
737.0 1.1203313e7 47.0 Afghanistan Afghanistan–Pakistan_skirmishes link
null 26039.0 189.0 other-empty Photek other
null 7008604.0 44.0 other-empty Edward_S._Walker,_Jr. other
null 5031359.0 18.0 other-google Families_Acting_for_Innocent_Relatives other
null 5545390.0 21.0 other-other EWR_VJ_101 other
4503997.0 1.4695141e7 127.0 What_About_Brian List_of_What_About_Brian_episodes link
null 4.137716e7 14.0 other-empty Frederick_Fox_(milliner) other
3096312.0 99459.0 15.0 Dana_Wynter Airport_(1970_film) link
null 1.2995239e7 18.0 other-wikipedia Nervous_Night_(album) other
null 1.47815e7 49.0 other-google Thermal_dose_unit other
1.8186074e7 1.608506e7 33.0 National_Highway_7A_(India)(old_numbering) National_Highway_7_(India)(old_numbering) link
null 3880952.0 19.0 other-empty Marion_Dudley other
1171412.0 6847270.0 22.0 Ivatan_language Batanic_languages link
8837050.0 30027.0 14.0 Copernican_heliocentrism Tycho_Brahe link
4.3380319e7 4.2654791e7 49.0 Scandal_(season_4) Grey's_Anatomy_(season_11) other
1928831.0 1928617.0 15.0 List_of_moths Laothoe_populi link
222417.0 1793072.0 21.0 A._R._Rahman Dil_Se.. link
1537974.0 3.3757237e7 18.0 Angelo_Scola Francesco_Moraglia link
2.0755574e7 8249183.0 37.0 Venezuelan_of_European_descent María_Rivas other
524874.0 2204.0 16.0 Voiced_pharyngeal_fricative Arabic_alphabet link
60730.0 212182.0 33.0 Lucy_Maud_Montgomery Order_of_the_British_Empire link
null 2.2965354e7 13.0 other-google Shelek other
2.0646706e7 1977645.0 10.0 2009_Campeonato_Brasileiro_Série_A Diego_Tardelli link
2.1513743e7 2.4708146e7 17.0 Jason_Aldean_discography The_Truth_(Jason_Aldean_song) link
null 9277.0 1925.0 other-wikipedia Ellipse other
null 2.0958639e7 40.0 other-empty Trailer_Park_of_Terror other
33265.0 6723726.0 74.0 Winston_Churchill Operation_Overlord link
3.0533402e7 2.1723891e7 25.0 Tryblidiida Micropilina link
2721803.0 1.5146151e7 16.0 Battle_of_Baugé Baugé link
153784.0 17098.0 14.0 Naginata Kendo other
null 1.4347506e7 15.0 other-bing The_Hat other
9768801.0 3.3196141e7 12.0 Manchester_United_F.C._Reserves_and_Academy Peter_O'Sullivan_(Welsh_footballer) link
330447.0 1718041.0 14.0 Binti_Jua Western_lowland_gorilla link
18522.0 17730.0 10.0 Latino_(demonym) Latin link
1.1610928e7 1179269.0 25.0 Peacock-class_corvette Naval_Service_(Ireland) other
625758.0 144367.0 11.0 Bill_Willingham Justice_Society_of_America link
3.3930403e7 5064492.0 108.0 South_Park:_The_Stick_of_Truth South_Park_(video_game) link
148682.0 3891002.0 10.0 Pertinax Marcomannic_Wars link
null 2.5584664e7 43.0 other-other Discharge_coefficient other
null 2374342.0 27.0 other-yahoo Axillary_vein other
null 3.5009269e7 47.0 other-bing Successful_aging other
62069.0 2925739.0 50.0 Panthéon San_Pietro_in_Montorio link
1602491.0 3568081.0 24.0 Ghazipur Abdul_Hamid_(soldier) link
null 4849167.0 29.0 other-other Brethren_of_Purity other
null 1.8592807e7 53.0 other-wikipedia Doc_Shaw other
2.3576946e7 4440840.0 13.0 The_Collector_(2009_film) The_Collection other
null 2.7612808e7 24.0 other-bing "Benjamin_Miles_""C-Note""_Franklin" other
null 560667.0 25.0 other-wikipedia Index_of_Babylon_5_articles other
1.5432504e7 315269.0 22.0 Death_Scream Murder_of_Kitty_Genovese other
null 1904377.0 40.0 other-yahoo Canon_A-1 other
8556123.0 1.3036212e7 1866.0 Rules_of_Engagement_(TV_series) David_Spade link
null 2843028.0 17.0 other-wikipedia Mid-Canada_Communications other
null 4724437.0 345.0 other-google University_of_the_Balearic_Islands other
null 2.9457801e7 247.0 other-wikipedia Age_of_Heroes_(film) other
3.9120425e7 3.9868659e7 90.0 Santhosh_Narayanan Lucia_(2013_film) link
1904112.0 1004953.0 42.0 Lamberto_Bava Mario_Bava link
null 2.1372915e7 33.0 other-yahoo Christian_views_on_divorce other
91320.0 91267.0 15.0 Franklin_County,_Virginia Pittsylvania_County,_Virginia link
1334948.0 238273.0 221.0 Lupus_anticoagulant Antiphospholipid_syndrome link
null 3.2782542e7 24.0 other-wikipedia Sara_Pichelli other
null 1.7954948e7 12.0 other-google Club_Sportif_Makthar other
null 2.1880672e7 14.0 other-bing Bitte_Orca other
10128.0 75899.0 32.0 Elizabeth_I_of_England Huguenot link
7437933.0 1.1869952e7 29.0 Space_and_Upper_Atmosphere_Research_Commission Chronology_of_Pakistan's_rocket_tests link
2.4426866e7 1.2757148e7 27.0 Huamei Li_hing_mui link
2942161.0 2933074.0 28.0 Canon_T_series Canon_T50 link
null 1.4084798e7 18.0 other-empty 2008_Russian_Premier_League other
3.8317679e7 3.8780447e7 11.0 Marxist_humanism Structural_Marxism link
null 2.2571562e7 13.0 other-bing Jennifer_Servo other
2.0618103e7 275978.0 25.0 Camphora Camphor link
1.5580374e7 2.4128239e7 36.0 Main_Page Selena_Gomez_&_the_Scene other
4.5274337e7 32538.0 133.0 Gotlandsdricka Viking_Age link
null 1.1252905e7 205.0 other-yahoo Biblical_manuscript other
239038.0 1170.0 18.0 Construction Architect link
3.8465988e7 2540476.0 12.0 List_of_Republic_of_Ireland_international_footballers Kevin_Kilbane link
3.9413121e7 49696.0 68.0 Giorgio_Moroder_discography Metropolis_(1927_film) link
null 36743.0 281.0 other-other Atalanta other
28237.0 177534.0 192.0 Space_Shuttle_Columbia Ilan_Ramon link
3.148073e7 715008.0 36.0 2011–12_Football_League_Championship Football_League_Championship link
null 1014389.0 37.0 other-bing No_Man's_Land_Fort other
502307.0 323983.0 46.0 Cole_Turner Billy_Zane link
167745.0 5301493.0 10.0 Vaquero Mesoamerica link
1925385.0 1925397.0 12.0 Walter_F._Murphy The_Vicar_of_Christ other
null 2.1766442e7 113.0 other-empty David_Álvarez other
3244595.0 1584677.0 22.0 Harpe_brothers War_of_the_Regulation link
1.1221038e7 1510621.0 13.0 2007–08_Rangers_F.C._season Alan_Hutton link
1.5580374e7 1.7408264e7 10.0 Main_Page Vorapaxar other
null 2.5008567e7 18.0 other-empty Roderic_Noble other
3424149.0 5912292.0 27.0 1975–76_NBA_season 1976_NBA_Playoffs link
4.2736926e7 418286.0 158.0 The_Blacklist_(season_1) Justin_Kirk link
null 1.3598815e7 17.0 other-google Niquero other
2.4509049e7 1.4952458e7 10.0 Maritime_flag_signalling Flag_semaphore other
3660711.0 1335545.0 62.0 Duncan_Keith 2002_NHL_Entry_Draft link
30450.0 49172.0 15.0 Topological_space Interval_(mathematics) link
1424791.0 60368.0 60.0 Anne_Wiazemsky Jean-Luc_Godard link
null 4.0712418e7 14.0 other-google NEWS_(Austrian_magazine) other
null 1214237.0 111.0 other-wikipedia Fold_Your_Hands_Child,_You_Walk_Like_a_Peasant other
null 1.5832772e7 223.0 other-other Osu! other
966943.0 5092756.0 22.0 List_of_Family_Guy_episodes List_of_Top_Gear_episodes other
1.3638115e7 4.0387861e7 23.0 Carciofi_alla_giudia Carciofi_alla_romana link
null 6787685.0 33.0 other-google Bovalino other
null 1386389.0 81.0 other-other Southern_African_Customs_Union other
576635.0 986684.0 12.0 Port_scanner Rate_limiting link
56462.0 8639835.0 24.0 Carpal_tunnel_syndrome Radiculopathy link
1.602284e7 1.8895122e7 29.0 2008_Major_League_Baseball_Draft Brett_Lawrie link
null 1834626.0 17.0 other-bing Knowsley_Safari_Park other
861686.0 343056.0 16.0 Tyrone_Guthrie Stratford_Shakespeare_Festival other
1.5580374e7 288197.0 124.0 Main_Page Kapil_Dev other
3.4075076e7 3.6541863e7 59.0 Gopichand_Malineni Balupu link
171141.0 3966054.0 19.0 Guava Mexico link
null 9441043.0 77.0 other-empty Cooliris other
null 5500697.0 3623.0 other-google Kiss_Me_(Sixpence_None_the_Richer_song) other
null 2.287596e7 11.0 other-bing Abblasen other
66297.0 53682.0 12.0 Chinese_art Calligraphy link
null 4.5282472e7 75.0 other-other Aspire_Music_Group other
3.0588065e7 86817.0 14.0 List_of_authoritarian_regimes_supported_by_the_United_States Omar_Torrijos link
null 1.7341725e7 15.0 other-empty Nasim_Wali_Khan other
46426.0 1.0834159e7 21.0 Basil_II Battle_of_Kreta link
59653.0 226141.0 123.0 Foreign_and_Commonwealth_Office Secretary_of_State_for_Commonwealth_Affairs link
9378717.0 164227.0 50.0 The_Spy_Who_Came_in_from_the_Cold_(film) Michael_Hordern link
5444617.0 5485318.0 12.0 Military_of_Montenegro UTVA_75 other
796141.0 8532006.0 17.0 1966–67_United_States_network_television_schedule Coronet_Blue link
381658.0 3.7085064e7 21.0 FK_Partizan History_of_FK_Partizan link
null 2.8350129e7 11.0 other-empty Arizona_Mountains_forests other
3.1927202e7 2.9381422e7 11.0 List_of_songs_recorded_by_My_Chemical_Romance Sing_(My_Chemical_Romance_song) link
null 7289088.0 30.0 other-google 1938_World_Men's_Handball_Championship other
8926582.0 174104.0 12.0 Imperial_Japanese_Army_General_Staff_Office Yamagata_Aritomo link
1.9258996e7 1.4116605e7 14.0 Den_Saakaldte Niklas_Kvarforth link
44682.0 6988539.0 48.0 CMYK_color_model Screen_angle link
2.3608452e7 1.5062239e7 152.0 Galatasaray_S.K._(football) Sercan_Yıldırım link
2617605.0 4.3680163e7 27.0 Saipa_F.C. Hamed_Shiri link
1.7645814e7 1.7519198e7 20.0 I_Hate_You_with_a_Passion Andre_Nickatina link
2.1327889e7 1110833.0 12.0 Kröd_Mändoon_and_the_Flaming_Sword_of_Fire Roger_Allam link
2.3487767e7 2.6323418e7 27.0 The_Tempest True_Reportory link
20869.0 2.3041952e7 12.0 Monoamine_oxidase_inhibitor Mebanazine link
null 3.1056907e7 12.0 other-yahoo Adebisi_Shank other
16716.0 1.5580374e7 68.0 Kansas Main_Page other
2.5731835e7 2.6693897e7 16.0 Colourist_painting Fauvism other
763785.0 3540456.0 111.0 Wellcome_Trust List_of_wealthiest_charitable_foundations link
null 177953.0 357.0 other-wikipedia America's_Army other
null 2138855.0 22.0 other-empty Garchitorena,_Camarines_Sur other
1161802.0 1.4216651e7 10.0 Nokia_3110 Nokia_1011 link
null 1.410232e7 318.0 other-google Take_Me_in_Your_Arms_(Rock_Me_a_Little_While) other
162036.0 110232.0 30.0 List_of_United_States_military_bases Moody_Air_Force_Base link
3984468.0 3166244.0 11.0 Oscar_Peterson_discography Ella_and_Oscar link
4255996.0 1.4312625e7 12.0 Western_Sahara_conflict Sahrawi_refugee_camps link
null 2.2661704e7 11.0 other-google Robin_Backhaus other
null 1988.0 65.0 other-yahoo Abel_Tasman other
null 6051097.0 482.0 other-google Trifluridine other
null 2309869.0 11.0 other-bing Frank_Henenlotter other
1248129.0 1.1718319e7 16.0 Port-Gentil Stephane_Lasme link
null 2872565.0 22.0 other-other Starfire_(board_wargame) other
1.0659362e7 1.7160872e7 16.0 Yoon_Dong-sik Gegard_Mousasi link
null 1792018.0 55.0 other-empty Nintendo_Gamebooks other
null 1.1223317e7 344.0 other-other Prince_Devitt other
null 3.5910828e7 10.0 other-wikipedia Sheffield_United_F.C._Player_of_the_Year other
null 2.8027307e7 109.0 other-yahoo Bones_(season_6) other
1.1911941e7 215619.0 10.0 All_Out_of_Love VH1 link
299717.0 3.5724251e7 192.0 Courteney_Cox Go_On_(TV_series) link
2.1930714e7 9265058.0 19.0 Outline_of_Florida List_of_ghost_towns_in_Florida link
581009.0 1439662.0 59.0 Ford_GT Super_GT other
null 985187.0 92.0 other-wikipedia Driving_simulator other
null 2995929.0 13.0 other-empty Edwards_Gardens other
34282.0 34393.0 116.0 Yule Yule_log link
355852.0 874356.0 13.0 Dachau_concentration_camp Miklós_Horthy,_Jr. link
601127.0 4194741.0 12.0 List_of_Democratic_National_Conventions Denver_Auditorium_Arena link
null 2.6415234e7 60.0 other-empty Customs_valuation other
null 2285190.0 20.0 other-empty Gastein_Convention other
499451.0 598952.0 15.0 STS-42 Ronald_J._Grabe link
null 3.279898e7 11.0 other-empty Jalan_Besar_MRT_Station other
2.5864167e7 920737.0 51.0 List_of_crowdsourcing_projects Clickworkers link
3.4199866e7 7253509.0 318.0 AKB0048 AKB48 link
3.3526094e7 3.1748786e7 36.0 QUnit Jasmine_(JavaScript_framework) link
null 5290240.0 10.0 other-empty Angus_M._Cannon other
1.52438e7 492820.0 13.0 Eric_Valentine Lostprophets link
20566.0 3564279.0 26.0 Mandy_Patinkin Tony_Award_for_Best_Featured_Actor_in_a_Musical other
5139911.0 1.4619184e7 12.0 Entering_Heaven_alive Ramalinga_Swamigal link
886579.0 1.1921455e7 14.0 Eva_Longoria Longoria link
null 8743616.0 14.0 other-other Chung_Ling_High_School other
4593958.0 2069950.0 23.0 The_Maltese_Falcon_(1941_film) The_Celluloid_Closet link
211913.0 89235.0 107.0 Christian_metal Christian_rock link
null 2.3834973e7 14.0 other-google Jitender_Kumar other
93135.0 58116.0 13.0 Butler_County,_Ohio Montgomery_County,_Ohio link
null 2.9085741e7 15.0 other-wikipedia Parris_Cues other
null 57564.0 33.0 other-empty Anselme_Payen other
1.9172199e7 1.9167679e7 10.0 Monera Virus other
379518.0 7301806.0 45.0 Panoramic_photography Panography link
null 4.5383298e7 92.0 other-wikipedia Vachagan_Khalatyan other
3618502.0 1.9337279e7 69.0 Echelon_Place Great_Recession link
null 3.1716175e7 59.0 other-google Nikki,_Wild_Dog_of_the_North other
null 3.990522e7 16.0 other-google Harold_Harris_(disambiguation) other
3.5038133e7 8721272.0 13.0 Pathogen PHI-base link
null 2016556.0 72.0 other-google Timmins/Victor_M._Power_Airport other
3.9746293e7 1758267.0 12.0 Tulpa_(film) Giallo link
480658.0 2.3742879e7 10.0 List_of_web_service_specifications XQuery link
1.7176729e7 2.010093e7 14.0 Mondo_Meyer_Upakhyan Samata_Das link
1259342.0 389664.0 11.0 The_Street The_Streets link
null 3.2124266e7 26.0 other-google SnoRNA_prediction_software other
null 136247.0 113.0 other-yahoo Boerne,_Texas other
2.4662654e7 1249019.0 13.0 Killer_Klowns_from_Outer_Space_(album) Killer_Klowns_from_Outer_Space link
null 1565005.0 82.0 other-yahoo Retinal_haemorrhage other
null 9315616.0 21.0 other-empty Gene_Youngblood other
1548943.0 802895.0 22.0 Limited_partnership Private_limited_company link
null 3.1110904e7 147.0 other-bing Effects_of_stress_on_memory other
2.7553159e7 58250.0 13.0 Health_care_in_the_United_States United_States_Department_of_Health_and_Human_Services link
791155.0 60626.0 61.0 Marty_Stuart Lester_Flatt link
4.4740812e7 3.8382626e7 48.0 Jin_Kyung Gu_Family_Book link
1.5610217e7 2756348.0 132.0 Useless_Loop,_Western_Australia Monkey_Mia link
null 1055437.0 46.0 other-wikipedia Deniable_encryption other
null 1.7962717e7 10.0 other-empty Ayres_LM200_Loadmaster other
2502077.0 1.3015878e7 14.0 Sales_taxes_in_the_United_States Washington_(state) other
82933.0 349335.0 38.0 Chloroform Anesthesiologist link
1278087.0 1277999.0 21.0 Nissan_Titan North_American_Car_of_the_Year link
null 718020.0 76.0 other-yahoo Equality other
3445929.0 3445909.0 54.0 Obscura_(album) The_Erosion_of_Sanity link
null 2036467.0 21.0 other-empty Overath other
1.7647526e7 664019.0 10.0 Alternative_versions_of_the_Punisher Owl_(Marvel_Comics) link
1619743.0 3.1636392e7 16.0 York—Simcoe York—Simcoe_(provincial_electoral_district) link
2.5121085e7 7499.0 10.0 APAV40 RDX other
null 1.2778103e7 13.0 other-google Shaden_Abu-Hijleh other
3.2866171e7 1221420.0 14.0 John_F._Kennedy_assassination_conspiracy_theories William_Greer link
1.0567624e7 173305.0 25.0 Tert-Butyl_chloride Isobutane link
null 5625238.0 23.0 other-wikipedia Prehistoric_man other
6819181.0 5755695.0 14.0 Valmara_59 PROM-1 link
2453648.0 2012734.0 11.0 List_of_comic_book_supervillain_debuts Floronic_Man link
97006.0 96981.0 14.0 Costilla_County,_Colorado Las_Animas_County,_Colorado link
null 1230540.0 79.0 other-other Daniel_Yergin other
null 1.6615403e7 147.0 other-wikipedia List_of_women_warriors_in_folklore other
74326.0 3993162.0 30.0 Nadia_Comăneci Art_of_Mentoring link
2232219.0 2920148.0 28.0 Tarquinius Sextus_Tarquinius link
6014615.0 4.3471582e7 21.0 Fox_Interactive Anastasia:_Adventures_with_Pooka_and_Bartok link
null 5191431.0 15.0 other-google Mansfield_Woodhouse_railway_station other
null 2.6523084e7 16.0 other-google Dominic_Arizona_Bonuccelli other
null 616958.0 26.0 other-empty Electrostatic_deflection other
485118.0 2249026.0 220.0 List_of_countries_by_GDP_(PPP) List_of_countries_by_income_equality link
1.3141832e7 47660.0 14.0 Versailles_restaurant Espresso link
null 1777994.0 14.0 other-twitter 1,1-Difluoroethane other
1091514.0 3.8702216e7 70.0 Bila_Tserkva 1941_Bila_Tserkva_massacre link
11362.0 3.1300186e7 24.0 February_16 André_Berthomieu link
7994183.0 4460532.0 35.0 Zicam ICAM-1 link
3.1436814e7 1.6455081e7 99.0 Brynne_Edelsten Geoffrey_Edelsten link
null 2139618.0 592.0 other-google Lee_Chapman other
2464121.0 2.1552009e7 13.0 Asra_Nomani Aisha link
59949.0 2845319.0 11.0 Anglo-Catholicism Traditional_Anglican_Communion link
2019904.0 3.9574425e7 17.0 CSC_Media_Group True_Drama link
null 4168628.0 58.0 other-empty 1997_NCAA_Division_I-A_football_season other
null 3.5535478e7 497.0 other-wikipedia Port_Royale_3:_Pirates_&_Merchants other
4477.0 9288.0 20.0 The_Beach_Boys Elvis_Presley other
null 3899315.0 20.0 other-google Anglican_Diocese_of_Jos other
null 2210627.0 26.0 other-wikipedia Zayed_Port other
null 1467229.0 24.0 other-facebook Fetal_viability other
7922200.0 1.3386129e7 12.0 The_Outfit_(1973_film) Felice_Orlandi link
1.5905472e7 1.3753303e7 178.0 Miss_Universe_1972 Miss_Universe_1973 link
434695.0 null 10.0 İzmir_Province Mustafa_Toprak redlink
277289.0 1055890.0 43.0 Wind_power Sustainable_energy link
1.9332171e7 5215871.0 12.0 Richard_Wright_(musician) Delay_(audio_effect) link
37417.0 840184.0 10.0 Mercury_(mythology) Hendrik_Goltzius link
356617.0 510764.0 35.0 Operation_Tannenberg Einsatzgruppen link
3.8174481e7 196279.0 43.0 List_of_sports_cars Lamborghini_Diablo link
1371727.0 433285.0 51.0 Bullet_(disambiguation) Bullitt link
null 1914019.0 121.0 other-google GTB other
5057404.0 null 12.0 Eureka_Forbes Aushim_Gupta_&_Company_Ltd. redlink
null 3.5394122e7 14.0 other-empty Joseph_Maynard other
1065035.0 2987765.0 10.0 Benson_&_Hedges Assault_occasioning_actual_bodily_harm link
null 1.9353281e7 13.0 other-google 1160_AM other
null 4155017.0 10.0 other-bing Gökhan_Özoğuz other
3.6909154e7 3.0563823e7 20.0 Dani_Carvajal Jesé link
null 4.2675612e7 6560.0 other-google Elfrid_Payton_(basketball) other
189048.0 6919555.0 24.0 Klaus_Schulze Angst_(soundtrack) link
null 4.1501305e7 19.0 other-twitter Telegram_(software) other
4848945.0 143163.0 76.0 Enclave_and_exclave Gadsden_Purchase link
1.163353e7 2657310.0 17.0 Injection_molding_machine Sprue_(manufacturing) link
null 2645023.0 34.0 other-google Monash,_Australian_Capital_Territory other
null 543169.0 111.0 other-empty William_Jackson_Palmer other
1.4174205e7 1920113.0 29.0 1981_UEFA_Cup_Final Arnold_Mühren link
null 1924219.0 229.0 other-empty Emperor:_Battle_for_Dune other
2.3670849e7 3623280.0 34.0 No_One's_Gonna_Love_You Band_of_Horses link
null 1.5964132e7 15.0 other-empty Jun_He_Law_Offices other
null 5545139.0 14.0 other-google Chief_Examiner other
2998033.0 1.5472645e7 16.0 West_Hampstead_Thameslink_railway_station West_Hampstead_railway_station link
null 1649162.0 90.0 other-other Starbreeze_Studios other
3.9617946e7 4.2742632e7 18.0 2014_French_Open 2014_French_Open_–_Girls'_Singles link
null 2.3915342e7 1813.0 other-google Cracked_tooth_syndrome other
null 2.4459316e7 11.0 other-empty OARnet other
174750.0 2.0974012e7 135.0 High/Low_(Nada_Surf_album) Popular_(Nada_Surf_song) link
6595367.0 7482590.0 12.0 Hewitt–Savage_zero–one_law Edwin_Hewitt link
1.3036961e7 199630.0 24.0 Something_for_Nothing Pop_punk link
null 2.2407888e7 101.0 other-google PSR_B1509-58 other
397145.0 3336186.0 23.0 List_of_districts_of_Maharashtra Nagpur_division link
345356.0 54251.0 106.0 Skate_(fish) Myliobatiformes link
3.7386608e7 4.069023e7 11.0 2015_in_film Sonic_Boom_(TV_series) other
8864153.0 1303857.0 20.0 Energy_policy_of_the_European_Union Electricity_liberalization link
null 1681589.0 58.0 other-empty Girl_on_the_Bridge other
27318.0 2701625.0 194.0 Singapore List_of_countries_by_life_expectancy link
1.7994862e7 1.3126459e7 13.0 Elastix Unified_communications link
418179.0 1880887.0 23.0 Whyte_notation 2-12-2 link
1.9769679e7 1057476.0 60.0 List_of_Nobel_Memorial_Prize_laureates_in_Economics Finn_E._Kydland link
null 1.2175763e7 14.0 other-wikipedia Palm_rat other
null 53884.0 30.0 other-other Penalty_area other
null 1819715.0 18.0 other-other Marine_geology other
5866621.0 3.6655551e7 12.0 Nayantara Vijay_Sethupathi link
5683212.0 1531683.0 15.0 McLeod_syndrome Myopathy link
50409.0 2563761.0 13.0 Cistercians Monastery_of_the_Holy_Spirit link
null 2.0720709e7 24.0 other-other 2-Pentyne other
null 1602370.0 29.0 other-google Kaleb_Toth other
740070.0 1.9891664e7 13.0 Body_farm Stephen_Fry_in_America link
null 8308.0 118.0 other-yahoo Delft other
3.4396117e7 2.2992135e7 129.0 Sofia_the_First Rapunzel_(Disney) link
2.6552124e7 735009.0 14.0 Sammy_Adams Pharrell_Williams other
6254282.0 164365.0 10.0 John_Chambers_(make-up_artist) The_China_Syndrome link
1468740.0 1.3964958e7 17.0 Gran_Turismo_4 Gran_Turismo_official_steering_wheel link
null 3233890.0 76.0 other-bing The_Bronx_(band) other
17851.0 6501490.0 81.0 Lambda Half-Life_(series) link
5574932.0 292086.0 13.0 Princess_Niloufer Abdülmecid_II link
401530.0 1.9283982e7 14.0 National_Democratic_Party New_Democratic_Party other
492211.0 3024814.0 113.0 Power_Rangers_in_Space Jason_Narvy link
32053.0 4.346807e7 115.0 Utrecht_University List_of_people_associated_with_Utrecht_University other
57731.0 62276.0 10.0 Leopold_II_of_Belgium Monarchy_of_Belgium link
298518.0 18189.0 17.0 Staraya_Ladoga Lake_Ladoga link
2929855.0 9962.0 56.0 Aenor_de_Châtellerault Eleanor_of_Aquitaine link
1196902.0 3.2826589e7 15.0 Saif_al-Islam_Gaddafi Rixos_Al_Nasr link
3119136.0 488105.0 144.0 British_Army_order_of_precedence Foot_Guards link
null 3.024141e7 10.0 other-empty Glover_v._United_States other
32308.0 21888.0 70.0 United_States_customary_units National_Institute_of_Standards_and_Technology link
1.4015242e7 345792.0 32.0 One_Night_Stand_(2008) The_Undertaker link
null 4.0759212e7 25.0 other-google Janet_Trujillo other
172369.0 51932.0 81.0 Challenger_2 Kinetic_energy_penetrator link
118444.0 3.6190531e7 12.0 Almere Almere_(lake) link
2.0395872e7 4066670.0 273.0 Oprah_Winfrey Kpelle_people link
5611605.0 1.6757325e7 57.0 Korean_People's_Army_Ground_Force Vz._52_machine_gun link
4.2411677e7 4.2567991e7 14.0 Cinedigm Chris_McGurk link
null 3.1257583e7 15.0 other-empty Bueng_Kan other
2.2258861e7 4.2534471e7 10.0 Total_penumbral_lunar_eclipse Tetrad_(astronomy) link
808402.0 1.6527475e7 115.0 Mexico_national_football_team Luis_Montes link
354296.0 994976.0 12.0 Vestment Monstrance link
362116.0 51784.0 85.0 Robinson_projection Map_projection link
4.2766864e7 2431566.0 37.0 2014–15_Northern_Premier_League Stamford_A.F.C. link
4251160.0 9313589.0 17.0 Taylor_Negron Wizards_of_Waverly_Place link
182214.0 2.0656228e7 17.0 Tassel Maize link
null 1595922.0 91.0 other-other Photosensitivity other
38872.0 4469999.0 15.0 Bessarabia Hotin_County link
1.3502823e7 1.252086e7 12.0 Brassiere Demi_Lovato link
4.4539913e7 1255017.0 26.0 The_Flintstones_&_WWE:_Stone_Age_SmackDown! Warner_Home_Video link
null 1.0433852e7 11.0 other-empty Robbie_Ellis other
1.2373195e7 33422.0 12.0 1984–85_Edmonton_Oilers_season Wayne_Gretzky link
1705212.0 133117.0 26.0 Lucknow_Pact Sarojini_Naidu link
309431.0 3.193663e7 24.0 Rhino_(wrestler) Leva_Bates other
1515653.0 1905405.0 20.0 Satellite_navigation Differential_GPS link
1.3099067e7 1.4306615e7 11.0 2007_Texas_Tech_Red_Raiders_football_team 2006_Texas_Tech_Red_Raiders_football_team link
4.4696649e7 1.3014023e7 31.0 Academy_Stadium Manchester_City_F.C._Reserves_and_Academy link
104650.0 38556.0 54.0 Oscar_II_of_Sweden List_of_Swedish_monarchs link
3.2457372e7 2.7152057e7 10.0 Sound_of_My_Voice Rostam_Batmanglij link
null 948103.0 44.0 other-other Meow_Mix other
553772.0 2100323.0 41.0 Robert_Pirès FWA_Footballer_of_the_Year link
6212853.0 4960.0 178.0 List_of_BSA_motorcycles Birmingham_Small_Arms_Company link
1.7006496e7 4.5002693e7 79.0 Batty_boy Homophobia_in_Jamaica link
4993017.0 706379.0 1033.0 The_Beach_Boys_discography Surfin'_Safari link
null 8790509.0 10.0 other-empty Folk_Lore_Museum_Mysore other
48946.0 5245439.0 29.0 Graz UPC-Arena other
92421.0 1.8396282e7 28.0 Interstate_64 Interstate_64_in_Indiana link
1.5580374e7 425059.0 81.0 Main_Page Ranch_dressing other
null 1.1172066e7 98.0 other-google Gerald_Stone other
3521882.0 81024.0 18.0 Airline_bankruptcies_in_the_United_States Pan_American_World_Airways link
null 294627.0 34.0 other-bing Mr._Garrison other
null 774449.0 75.0 other-wikipedia Richard_Lynn other
1.9975991e7 47707.0 49.0 Lex_Immers Feyenoord link
372478.0 1654769.0 15.0 Video_game_industry Artificial_intelligence_(video_games) link
null 207132.0 677.0 other-google Star_Trek:_Armada other
null 1863524.0 49.0 other-google Disney_Time other
null 780419.0 103.0 other-empty Philip_Don_Estridge other
null 1.4024425e7 17.0 other-wikipedia Nightwing_(novel) other
1020829.0 6260.0 23.0 20th-century_music Claude_Debussy link
4810477.0 1042506.0 24.0 Tawagalawa_letter Wilusa link
3821983.0 262233.0 167.0 1998_Russian_financial_crisis 1997_Asian_financial_crisis link
null 2.435744e7 68.0 other-wikipedia Académica_Petróleos_do_Lobito other
2.4097561e7 735348.0 13.0 List_of_newspaper_comic_strips_P–Z Pickles_(comic_strip) link
1669185.0 1624131.0 19.0 Phantasmagoria_(The_Damned_album) Roman_Jugg link
null 7197167.0 155.0 other-google Matei other
null 1.2315045e7 21.0 other-empty Dead_Air_(2009_film) other
null 969363.0 133.0 other-google Arlberg_technique other
891692.0 2.1803312e7 17.0 United_States_District_Court_for_the_Northern_District_of_California William_Haskell_Alsup link
957326.0 4915083.0 74.0 Italy_national_rugby_union_team Rugby_union_in_Italy link
null 8399976.0 146.0 other-wikipedia ASU-57 other
null 1558354.0 684.0 other-empty Demographics_of_Europe other
null 1.305149e7 134.0 other-google Courtney_Love_discography other
3.9654815e7 1034678.0 29.0 MFi_Program MFI other
1.9283769e7 3192804.0 14.0 Ausar Ausar_Auset_Society link
null 472075.0 65.0 other-google Viscount_Brookeborough other
262376.0 3183896.0 12.0 Roger_Federer 2006_Tennis_Masters_Cup link
1.1161932e7 1.1590877e7 12.0 Saints_Row_2 Red_Faction:_Guerrilla link
7712434.0 4579172.0 36.0 Vijay_Arora Yaadon_Ki_Baaraat link
null 4496629.0 14.0 other-empty One_Must_Fall:_Battlegrounds other
null 3.6894375e7 26.0 other-google Dragan_Lakićević other
211080.0 170459.0 110.0 Method_Man LL_Cool_J link
2646730.0 6733556.0 27.0 Pokémon:_The_First_Movie_(soundtrack) Pokémon_2.B.A._Master link
null 9472399.0 10.0 other-google Black_Duck_(group) other
2663129.0 16321.0 411.0 Good_Night,_and_Good_Luck Joseph_McCarthy link
1.1024497e7 1.0398699e7 35.0 Brown-Séquard_syndrome Central_cord_syndrome link
null 72434.0 14.0 other-empty Maximilian_Kaller other
202886.0 5869.0 10.0 Covariance_and_contravariance_of_vectors Category_theory link
6742.0 653196.0 16.0 Central_Asia Economic_Cooperation_Organization link
942704.0 46525.0 14.0 Song_of_Songs_(disambiguation) Wilfred_Owen link
null 1.0987478e7 62.0 other-google Pouch_Cove other
null 2122098.0 266.0 other-google Seabiscuit:_An_American_Legend other
null 454779.0 24.0 other-wikipedia Twitch_City other
96775.0 96815.0 24.0 Fulton_County,_Georgia Carroll_County,_Georgia link
296849.0 2520715.0 10.0 Ernest_Becker Sam_Keen link
347713.0 284262.0 28.0 Huia Callaeidae link
1.9137828e7 2.0753462e7 370.0 List_of_stadiums_under_construction Estadio_La_Peineta link
19859.0 277696.0 13.0 Moby-Dick The_Scarlet_Letter other
16844.0 523445.0 169.0 Kofi_Annan Lakhdar_Brahimi link
null 2.1230974e7 20.0 other-google Chhantyal other
2.892839e7 113519.0 36.0 Nucky_Thompson Short_(finance) link
219731.0 36168.0 30.0 Myst 3DO_Interactive_Multiplayer link
4.4845611e7 3.1597122e7 1377.0 My_Sunshine Wallace_Chung link
1107477.0 1345497.0 24.0 River_gunboat USS_Cairo link
4.2688351e7 5933689.0 439.0 2014–15_FC_Barcelona_season Jérémy_Mathieu link
4461110.0 1.0640807e7 41.0 Shilpa_Shirodkar Bhrashtachar link
77390.0 2.3769406e7 155.0 Natalie_Wood Sex_and_the_Single_Girl_(film) link
57877.0 7280414.0 53.0 Sodium_hydroxide List_of_commonly_available_chemicals other
null 683450.0 983.0 other-google Dawson's_Field_hijackings other
1990194.0 1.6290655e7 10.0 Ima_Hogg Thomas_Elisha_Hogg link
14015.0 2226.0 32.0 Herstory Ad_hominem other
3.1990324e7 4.2269335e7 19.0 List_of_Switched_at_Birth_episodes The_Futon_Critic link
null 9081713.0 30.0 other-bing The_Bitter_End other
108956.0 575052.0 120.0 Washington,_D.C. Verizon_Center link
1415821.0 1998515.0 72.0 Fiat_Ducato JTD_engine link
218742.0 490089.0 159.0 Ontario_Hockey_League Peterborough_Petes link
null 1792799.0 9379.0 other-google Stakeholder_theory other
null 216639.0 33.0 other-wikipedia Gondwanatheria other
3.0875744e7 26961.0 13.0 Artur_Rasizade Shia_Islam link
2.4378497e7 null 28.0 Mark_Salling Rocky_Road_(TV_Movie) redlink
1.1900681e7 312228.0 143.0 List_of_teen_films House_Party_(film) link
2608405.0 2418357.0 13.0 Roy_Mayorga Shelter_(band) link
null 2.1312852e7 10.0 other-yahoo Igor_Sijsling other
2809559.0 414267.0 12.0 Salirophilia Lust_murder link
null 3.0876253e7 116.0 other-wikipedia List_of_Jewish_prayers_and_blessings other
3966054.0 537551.0 28.0 Mexico Azteca_(multimedia_conglomerate) other
null 2.7437943e7 42.0 other-wikipedia Donna_Simpson_(internet_celebrity) other
null 1318322.0 89.0 other-other Callable_bond other
1972785.0 1053430.0 16.0 Illinois_(album) 2005_in_music link
null 2875803.0 68.0 other-google Lennon_(musical) other
438417.0 26977.0 11.0 Orange_County_(film) Stanford_University link
4178394.0 1589455.0 17.0 Chamar Kanshi_Ram link
null 390818.0 650.0 other-empty Endive other
null 2186604.0 21.0 other-google Caldecott,_Rutland other
976050.0 1.9084502e7 23.0 Fell's_Point,_Baltimore Michael_Phelps link
1027240.0 1.165959e7 130.0 Biryani Hyderabadi_cuisine link
4887.0 11523.0 236.0 British_Army Falklands_War link
498478.0 1.1395198e7 12.0 Shanghai_World_Financial_Center World_Trade_Center link
193918.0 39282.0 10.0 Kurgan Caucasus link
4754.0 1009445.0 12.0 Blue_Streak_(missile) European_Launcher_Development_Organisation link
206928.0 236519.0 11.0 List_of_birds_of_New_Zealand Australian_pelican link
7720774.0 1908172.0 15.0 WIN.INI INI_file link
null 3.101165e7 26.0 other-empty Betty_Jane_Gorin-Smith other
1.8952953e7 411723.0 173.0 Peyote Chihuahuan_Desert other
3.3469792e7 2412317.0 11.0 Here's_to_You_(song) Franz_Josef_Degenhardt link
null 2205218.0 40.0 other-wikipedia Six_Pieces_for_Piano,_Op._118_(Brahms) other
null 3872704.0 20.0 other-empty Caridad_de_la_Luz other
1.7621236e7 1.4485544e7 11.0 H._Eugene_Stanley List_of_members_of_the_National_Academy_of_Sciences_(Applied_physical_sciences) link
null 1495605.0 31.0 other-bing Hew_Strachan other
1.2136846e7 358527.0 27.0 The_Unfairground Kevin_Ayers link
4.3047831e7 4.3797734e7 113.0 Giant_in_My_Heart Sound_of_a_Woman link
null 9656353.0 73.0 other-google New_folk_media other
2909374.0 1.895149e7 11.0 Health_issues_in_American_football American_football link
null 595888.0 11.0 other-google Hail,_Vermont! other
null 2.3306201e7 11.0 other-empty EFestivals other
4446461.0 524502.0 63.0 Film_budgeting Box_office_bomb link
null 1.3437204e7 18.0 other-bing New_classical_macroeconomics other
2772399.0 1.8595033e7 134.0 Bill_T._Jones Arnie_Zane link
null 5903.0 13.0 other-yahoo Cultural_movement other
551711.0 408652.0 25.0 Ron_Dellums Barbara_Lee link
3.2628378e7 2.9017966e7 39.0 List_of_horror_films_of_1972 The_Fiend_(film) link
2945357.0 1770333.0 31.0 Amateur_rocketry High-power_rocketry link
280929.0 1278771.0 36.0 Erechtheion Palladium_(classical_antiquity) other
null 2342731.0 23.0 other-yahoo Walther_von_Seydlitz-Kurzbach other
18727.0 579219.0 17.0 List_of_food_additives,_Codex_Alimentarius Beta-Carotene other
2.548813e7 219287.0 33.0 Separate_legal_entity Legal_personality other
376581.0 2.0611504e7 12.0 Transnistria Russian_Empire link
3.3623665e7 2592839.0 18.0 Thimar Anouar_Brahem link
null 2.5650291e7 11.0 other-bing 2010_Tampa_Bay_Buccaneers_season other
2064142.0 346585.0 11.0 Chafing_dish Brazier link
null 461211.0 4961.0 other-google Mizoram other
660231.0 1257944.0 14.0 Adelaide_Oval WACA_Ground link
null 2.5679855e7 21.0 other-wikipedia Banco_Bicentenario other
1.30362e7 3.6604477e7 25.0 Automobile_drag_coefficient Jaguar_XE link
417606.0 1.4265701e7 76.0 Expedia_(website) Wotif.com link
3.5645946e7 1629175.0 22.0 Luís_Leal_(footballer) G.D._Estoril_Praia link
2.6458478e7 959456.0 100.0 German_military_brothels_in_World_War_II Roundup_(history) link
null 2.9027399e7 22.0 other-google The_Dog_Who_Saved_Christmas_Vacation other
null 1988582.0 33.0 other-empty Kovač other
null 987320.0 37.0 other-other Neurotechnology other
1022665.0 2366194.0 18.0 String_Quartets_Nos._7–9,_Op._59_–_Rasumovsky_(Beethoven) String_Quartet_No._10_(Beethoven) link
1373758.0 843532.0 49.0 Luther_Adler Stella_Adler link
null 9118436.0 13.0 other-google Jeremy_Huw_Williams other
1677928.0 268683.0 23.0 Children's_song Riddle other
null 7448450.0 18.0 other-other Louis_Zorich other
2.0803357e7 4.0026015e7 20.0 Parks_and_Recreation Drew_Barrymore_filmography other
4604481.0 4.02451e7 61.0 Storm_Model_Management Tiah_Delaney link
7481030.0 3098376.0 70.0 MPEG-4_Part_14 Comparison_of_audio_coding_formats link
null 3.920506e7 10.0 other-empty 72nd_Division_(United_Kingdom) other
1.7401306e7 2488280.0 139.0 Ripstop Ballistic_nylon link
1.0904287e7 423689.0 25.0 Termcap Curses_(programming_library) link
null 2.3268278e7 16.0 other-wikipedia Trakr other
null 458310.0 19.0 other-wikipedia Frank_Teschemacher other
47271.0 83835.0 16.0 Sponge Gonad link
null 1793967.0 43.0 other-other Pitman_arm other
3.1192297e7 2.1377045e7 298.0 Magic_City_(TV_series) Rick_Ross link
null 1362205.0 100.0 other-other Area_code_920 other
58478.0 2260425.0 15.0 Airborne_forces 502nd_Infantry_Regiment_(United_States) other
1510249.0 414822.0 35.0 Bering_Strait_crossing Tung-Yen_Lin other
1601795.0 5231575.0 11.0 V24_engine V5_engine link
44784.0 5533243.0 12.0 Bari The_Bridges_of_Madison_County_(film) link
null 4038132.0 47.0 other-empty Gary_Gaines other
1.6378571e7 1.6085877e7 273.0 Genealogies_in_the_Bible Abraham's_family_tree link
762354.0 1127973.0 11.0 Greatest_Hits_(ZZ_Top_album) Billy_Gibbons link
462091.0 3056665.0 113.0 Vairocana Sambhogakāya link
null 1210333.0 1335.0 other-google Millfield other
397810.0 2035119.0 12.0 Alexander_McQueen Elie_Saab link
8169386.0 3.7918222e7 13.0 Rick_Sebak Yinztagram link
1928513.0 3662295.0 13.0 Government_of_India_Act_1858 Dominion_of_Pakistan link
7919595.0 2615949.0 72.0 Clos_network Omega_network link
null 4.0485213e7 15.0 other-google Singapore_national_under-19_football_team other
null 2.0911775e7 14.0 other-wikipedia Fyodor_Druzhinin other
84112.0 84109.0 21.0 Berenice Berenice_II_of_Egypt link
null 1252215.0 30.0 other-google Dancer_with_Bruised_Knees other
null 17703.0 363.0 other-bing Leo_(constellation) other
1.6975268e7 3.0873764e7 26.0 Chess_theory Scandinavian_Defense link
null 1.6787376e7 56.0 other-empty Universiti_Malaysia_Terengganu other
null 347603.0 227.0 other-empty Carpenter_bee other
1.7798548e7 1.9183413e7 10.0 Rush_(2008_TV_series) Stephen_Rae_(composer) link
314628.0 1.0188712e7 15.0 Tooth_enamel Cusp_(anatomy) link
null 44903.0 169.0 other-yahoo Astor_Piazzolla other
1.9988138e7 1885136.0 23.0 Ramsay_(surname) Clan_Ramsay link
null 4.4010295e7 2438.0 other-google Survivor's_Remorse other
null 735443.0 25.0 other-bing Neumann_boundary_condition other
714047.0 544762.0 39.0 Chromolithography Offset_printing link
null 1686225.0 15.0 other-yahoo Tamara_Karsavina other
5859950.0 7616334.0 39.0 When_Worlds_Collide_(1951_film) Larry_Keating link
2.4132083e7 2955815.0 126.0 Dexter_(season_4) Julia_Campbell link
null 466851.0 16.0 other-empty Azteca_(band) other
1164252.0 527125.0 10.0 GamePro Game_Informer other
null 3.1863547e7 34.0 other-other Tarun_Khanna other
null 9038883.0 26.0 other-empty College_of_Physicians_and_Surgeons_Pakistan other
958572.0 592436.0 10.0 Glenn_Hall Ted_Lindsay link
3820404.0 3.0812082e7 10.0 Cross-dressing_in_film_and_television Bucket_&_Skinner's_Epic_Adventures link
2.7205785e7 2.8233212e7 25.0 School_attacks_in_China_(2010–12) 2010_Hebei_tractor_rampage other
3.4445585e7 3.8764549e7 40.0 American_Idol_(season_12) Curtis_Finch,_Jr. link
402942.0 436614.0 80.0 List_of_traditional_children's_games Pat-a-cake,_pat-a-cake,_baker's_man link
null 893198.0 16.0 other-other What_I_Learned_About_Ego,_Opinion,_Art_&_Commerce other
871210.0 1292261.0 38.0 Utricularia Utricularia_vulgaris link
5575722.0 11585.0 33.0 Fuck Show_Me_Love_(film) other
5465550.0 5512301.0 19.0 Morphinan Levomethorphan link
3.086259e7 9499.0 116.0 Link_layer Ethernet link
46336.0 5376.0 28.0 Passerine Cladistics other
158558.0 154820.0 18.0 King_of_the_Romanians List_of_rulers_of_Wallachia link
2.7804243e7 416577.0 10.0 List_of_birds_of_Pennsylvania Alder_flycatcher link
851800.0 915646.0 83.0 Air_America_(film) Pilatus_PC-6_Porter other
null 2.2976039e7 150.0 other-google Armenian_Wikipedia other
2019407.0 764428.0 28.0 Ali_Azmat Bhat link
4097772.0 2162718.0 11.0 Battle_of_Honey_Springs James_G._Blunt link
null 3824728.0 31.0 other-google Can_U_Get_wit_It other
null 1.0567795e7 135.0 other-google Robbie_van_Leeuwen other
null 4100412.0 111.0 other-google The_Truth_(Melbourne_newspaper) other
436522.0 229703.0 11.0 Hot_rod Roots-type_supercharger link
4.1660623e7 1.3280198e7 577.0 Tokyo_Ghoul Ling_Tosite_Sigure link
3.5034514e7 3.554456e7 12.0 2012_World_Junior_Championships_in_Athletics 2012_World_Junior_Championships_in_Athletics_–_Men's_100_metres link
null 2.4303131e7 18.0 other-wikipedia Treska other
null 1.7262978e7 19.0 other-bing Shake_It_(Metro_Station_song) other
6833695.0 1.2237982e7 10.0 Demihypercube Hypercubic_honeycomb other
1253121.0 264458.0 38.0 Battle_of_Kennesaw_Mountain Joseph_E._Johnston link
null 2.6626591e7 15.0 other-wikipedia Banqiao_Station other
null 239930.0 929.0 other-empty Temple_University other
679346.0 172063.0 153.0 Lucozade Ribena link
null 2.4619717e7 146.0 other-google Much_the_Same other
1.5609213e7 1602398.0 138.0 List_of_airlines_of_Nigeria Associated_Aviation link
1.2727445e7 null 22.0 I'm_Not_Like_Everybody_Else The_Sacred_Mushroom redlink
1478064.0 3.7165545e7 34.0 Andy_Souwer Steve_Moxon link
36396.0 39995.0 57.0 1214 1213 link
null 212416.0 14.0 other-yahoo Phitsanulok_Province other
1.8619244e7 305854.0 53.0 SMS_language Text_messaging link
180437.0 27071.0 89.0 Pavel_Chekov Star_Trek:_The_Original_Series link
null 4064.0 394.0 other-google Borsuk–Ulam_theorem other
null 1018286.0 284.0 other-empty Capri_Sun other
null 160753.0 28.0 other-twitter Manuel_L._Quezon other
3.6169584e7 4.0603571e7 23.0 2014–15_figure_skating_season Lombardia_Trophy link
1251507.0 1.0564133e7 207.0 Kirk_Acevedo Joe_Toye link
1302191.0 4393323.0 10.0 Opel_Commodore Ranger_(automobile) other
3.1523612e7 3.1348196e7 17.0 Mark_McNeill Phillip_Danault link
5906626.0 2.0534384e7 33.0 Horace_Trumbauer Elkins_Estate link
1006148.0 59003.0 10.0 Ludlow_(disambiguation) Ludlow link
3515315.0 1.0074452e7 19.0 Reebok_Freestyle Reebok_Classic other
null 5257744.0 40.0 other-wikipedia Some_Kind_of_Hero other
573177.0 2.3740297e7 31.0 Wendish_Crusade Wagria link
1.0367494e7 107204.0 12.0 Fried_pickle Atkins,_Arkansas link
743895.0 2278793.0 19.0 Timeline_of_Eastern_philosophers Parashara link
null 2.2484087e7 16.0 other-google Nnooo other
5824627.0 5042916.0 37.0 Inheritance_tax Canada link
40656.0 7247.0 29.0 13th_century_BC Cemetery_H_culture link
null 4.0389354e7 19.0 other-wikipedia ASAN_service other
null 334882.0 13.0 other-wikipedia Chamarajanagar_district other
299404.0 741705.0 18.0 Gunnery_sergeant Mark_Harmon link
1424575.0 4.0371665e7 31.0 Battle_of_Bailén Dominique_Honoré_Antoine_Vedel link
null 6598147.0 1322.0 other-google Concentration_risk other
1858211.0 21444.0 24.0 The_Jew_of_Malta Niccolò_Machiavelli link
16880.0 46853.0 13.0 Karnataka Indus_Valley_Civilization link
null 2.7255423e7 28.0 other-google Fornham_St_Genevieve other
2437139.0 1552544.0 15.0 Russian_architecture Onion_dome link
56315.0 6423327.0 14.0 Mango 2-Furanone other
8087287.0 1254779.0 164.0 Hot_Blooded Double_Vision_(Foreigner_album) link
null 3555863.0 114.0 other-other Frank_Hamer other
null 1.4270466e7 148.0 other-wikipedia Gautam_Adani other
2.0519849e7 4.0736758e7 38.0 Cage_discography Kill_the_Architect link
null 2.0776944e7 16.0 other-google Phintys other
null 2.4509692e7 38.0 other-empty Tri-City_Medical_Center other
3.6295719e7 1.817631e7 10.0 Garrett_(character) Guinness_World_Records_Gamer's_Edition link
87603.0 9295254.0 613.0 Robert_Mitchum Bentley_Mitchum link
2.1173707e7 3.6897865e7 20.0 Moreton_Bay_Rail_Link Murrumba_Downs_railway_station link
null 969732.0 15.0 other-wikipedia Brand_(disambiguation) other
1699425.0 3152733.0 19.0 Power-on_self-test Memory_refresh link
null 144948.0 137.0 other-yahoo Universal_joint other
571462.0 37585.0 13.0 National_Museum_of_the_United_States_Air_Force Museum link
890293.0 1.9147563e7 36.0 Mike_Smith_(actor) Thorburn,_Nova_Scotia link
876966.0 1.7702228e7 10.0 Vancouver_Island_University Higher_education_in_British_Columbia link
2003796.0 142421.0 22.0 Roscoe_Lee_Browne Babe_(film) link
null 528150.0 15.0 other-wikipedia Brothers_Keepers other
null 4958063.0 87.0 other-google Palma_di_Montechiaro other
null 4.2035866e7 14.0 other-wikipedia Collection_manager other
1257770.0 4799962.0 22.0 Tim_Drake Jaime_Reyes link
null 675786.0 231.0 other-google Scaffold_(disambiguation) other
null 1630583.0 39.0 other-empty Arica_School other
2753730.0 1203602.0 12.0 Copa_Airlines_Flight_201 United_Airlines_Flight_585 link
1.4529239e7 16861.0 14.0 Zoophilia Kurt_Vonnegut link
null 4.2270307e7 13.0 other-bing Bad_(David_Guetta_and_Showtek_song) other
64946.0 21724.0 18.0 Danelaw Normandy link
3285435.0 39205.0 13.0 2010_Asian_Games Asian_Games link
null 467628.0 736.0 other-google Margaret_Dumont other
592456.0 598977.0 16.0 This_Is_My_Truth_Tell_Me_Yours James_Dean_Bradfield link
null 19965.0 105.0 other-yahoo Morphogenesis other
95185.0 6939163.0 886.0 Frantz_Fanon Black_Skin,_White_Masks link
null 2.7633566e7 24.0 other-wikipedia ConsensusDOCS other
80777.0 8309183.0 24.0 Kurdistan Koçgiri_Rebellion link
null 5064426.0 24.0 other-yahoo Misha_Glenny other
null 4.0846967e7 23.0 other-google Hitkarini_Sabha other
null 2.2600019e7 17.0 other-wikipedia Country_folk other
99782.0 1154193.0 50.0 Vritra Aesir-Asura_correspondence link
246020.0 3516576.0 13.0 Freydís_Eiríksdóttir Greenland_saga other
12463.0 6124461.0 124.0 Glacier Quelccaya_Ice_Cap link
3628651.0 67234.0 10.0 New_Jersey's_10th_congressional_district Newark,_New_Jersey link
1878882.0 2.5016782e7 30.0 List_of_anthropomorphic_animal_superheroes Quick_Draw_McGraw link
null 126987.0 91.0 other-google Cleveland,_New_York other
6310617.0 2711314.0 19.0 Cristine_Rose How_I_Met_Your_Mother link
null 266033.0 308.0 other-empty Robert_I,_Duke_of_Normandy other
3125454.0 3536263.0 130.0 David_J._O'Reilly Kenneth_T._Derr link
1.9165698e7 228211.0 12.0 Teletoon_at_Night Futurama link
6452550.0 3.8121496e7 130.0 Hayley_Tamaddon List_of_Coronation_Street_characters_(2013) other
null 4.171206e7 45.0 other-empty Consider_the_Source other
1104597.0 494926.0 49.0 Kirovohrad Kirovohrad_Oblast link
1928711.0 896897.0 16.0 Etheric_plane Plane_(Dungeons_&_Dragons) other
null 2.2654444e7 419.0 other-google Pineapple_Dance_Studios other
1.3895544e7 788074.0 36.0 Vigor Physical_strength link
2583157.0 2.2509614e7 21.0 Byzantine_dress English_medieval_clothing link
null 8055634.0 15.0 other-google Unbarred_lenticular_galaxy other
1445268.0 236723.0 13.0 Master_of_Arts_(Oxbridge_and_Dublin) Master_of_Arts_(disambiguation) link
2.7656285e7 1739962.0 20.0 Geo_URI ICBM_address link
null 1.2892672e7 110.0 other-google Leandra other
1.5580374e7 3722614.0 18.0 Main_Page African_Cup_Winners'_Cup other
1.0078096e7 245335.0 55.0 This_Is_Just_To_Say Found_poetry link
2172281.0 1.0774494e7 32.0 Mumtaz_(actress) Apna_Desh link
null 2.7289759e7 19.0 other-empty 2010_Santos_FC_season other
39021.0 1.2230576e7 40.0 Daytona_500 Coke_Zero_400 link
3016712.0 4682876.0 10.0 Terminal_degree Professional_degrees_of_public_health other
939423.0 7216989.0 49.0 Mr._Lawrence The_Grim_Adventures_of_Billy_&_Mandy other
null 2697919.0 11.0 other-wikipedia Antemnae other
2463448.0 3.0668895e7 23.0 Ted_McCarty Gibson_Guitar_Corporation link
845407.0 2.8320131e7 25.0 Sezen_Aksu Ağlamak_Güzeldir link
null 1.4454507e7 79.0 other-google Eleider_Álvarez other
null 4923077.0 209.0 other-google Demon_Seed_(novel) other
4666669.0 2796527.0 10.0 Area_code_904 T-Pain link
null 1.9111554e7 11.0 other-empty Hermann_Hauser,_Sr. other
31827.0 145144.0 78.0 Demographics_of_Ukraine Ukrainians link
1980240.0 3.9277098e7 138.0 List_of_American_comedy_films About_Last_Night_(2014_film) link
730462.0 558569.0 16.0 Flower-class_corvette HMCS_Oakville_(K178) link
609002.0 1.4814799e7 13.0 Biloela Callide_Dam link
27695.0 30403.0 15.0 Structured_programming Turing_machine link
12449.0 2013048.0 45.0 Mobile_Suit_Gundam_Wing Mobile_weapons link
1.094599e7 1.7608953e7 14.0 Alternative_versions_of_Wolverine Marvel_Zombies_2 link
null 1.6917052e7 26.0 other-empty Kacy_Rodgers other
null 1.3805947e7 30.0 other-empty Unlimited_Touch other
2.1444421e7 1.915323e7 62.0 Roberta_Flack_discography Born_to_Love link
null 6493684.0 10.0 other-google Oświęcim_County other
11033.0 1.1991546e7 13.0 Frederick_Douglass Civilization_Revolution link
1.8302482e7 38170.0 12.0 List_of_bisexual_people_(A–F) Bi-curious link
1009423.0 4528243.0 44.0 Talysh_people Talysh_Khanate link
3.6355277e7 3.0214103e7 23.0 Vikings_(TV_series) Falling_Skies other
null 2.996522e7 14.0 other-empty Peter_White_(Michigan) other
8035013.0 3.377343e7 11.0 Lee_Jung Saturday_Freedom link
3.3050531e7 1.1612491e7 153.0 List_of_Deadly_Women_episodes Murder_of_Shanda_Sharer link
8670674.0 7364118.0 22.0 U218_Videos U218_Singles link
1374327.0 826555.0 11.0 ETA_SA Breitling_SA link
2312056.0 202652.0 12.0 Pride_&_Prejudice_(2005_film) Romeo_+_Juliet link
6027027.0 2697824.0 23.0 House_of_Carters Andy_Samberg link
null 3.3484283e7 22.0 other-google Dovedale_by_Moonlight other
null 3360692.0 11.0 other-google Harvey_Hodder other
null 1.4940878e7 10.0 other-empty 1982_Baltimore_Colts_season other
453246.0 142058.0 18.0 Breakout_(video_game) Homebrew_Computer_Club link
65910.0 3.8481732e7 219.0 Printed_circuit_board Chemical_milling other
null 9646491.0 47.0 other-google Fouad_Abou_Nader other
253868.0 524481.0 45.0 Eye_of_the_Beholder_(video_game) Gold_Box link
null 2.0207353e7 14.0 other-wikipedia Type-90 other
2.9156836e7 2.8439144e7 96.0 Park_Ha-sun Dong_Yi_(TV_series) link
null 54530.0 30.0 other-wikipedia Bookmark_(disambiguation) other
197181.0 1.7898921e7 18.0 Kunming Yuantong_Temple link
5043734.0 14800.0 10.0 Wikipedia Icon other
973639.0 1929375.0 16.0 Lacombe Lacombe,_Alberta link
380569.0 53607.0 46.0 John_F._Kennedy_Center_for_the_Performing_Arts Edward_Durell_Stone link
149689.0 190226.0 10.0 Midnight's_Children 1981_in_literature link
null 1.73024e7 24.0 other-other Baltimore_City_Circuit_Courthouses other
null 2.1313911e7 10.0 other-bing Wind_power_in_Wyoming other
null 1.3005006e7 12.0 other-empty Douglas_Guest other
714928.0 1.5704166e7 30.0 Greenland_Dog Inuit link
null 9471611.0 28.0 other-empty Memphis_Light,_Gas_and_Water other
1585091.0 348208.0 16.0 List_of_Turkish_artists Avni_Arbaş link
null 4120275.0 56.0 other-google Marjie_Lundstrom other
39482.0 2079614.0 27.0 Mai_Zetterling Tutte_Lemkow link
46526.0 57744.0 10.0 419_scams Ivory_Coast link
52967.0 411914.0 16.0 Gynaecology Oophorectomy link
6059111.0 2.8039598e7 17.0 Ethan_Spaulding The_Legend_of_Korra link
null 501536.0 20.0 other-yahoo Ministry_of_Intelligence other
480634.0 1989200.0 27.0 Absorbance Densitometry link
3924114.0 2455426.0 26.0 Bottom_Dollar_Food PriceRite link
null 2.1715001e7 15.0 other-google You_Are_My_Joy other
1.8899968e7 44700.0 11.0 List_of_Chinese_discoveries Leprosy link
984322.0 2176065.0 45.0 Krome_Studios_Melbourne Nightshade_(1992_video_game) other
null 867983.0 24.0 other-wikipedia Microsoft_Narrator other
3303790.0 2.3384265e7 21.0 Military_history_of_Mexico Mexican_Armed_Forces other
null 5435750.0 14.0 other-wikipedia Punk-O-Rama_5 other
24555.0 129618.0 11.0 Photosynthetic_pigment Cyanobacteria link
null 1.2391537e7 11.0 other-other Flores_(canton) other
null 8703722.0 16.0 other-wikipedia Get_down other
117337.0 181005.0 35.0 Westlake_Village,_California Robert_Young_(actor) link
1266404.0 201829.0 20.0 Hypalon DuPont link
null 150521.0 63.0 other-yahoo Henry_Armstrong other
null 46933.0 20.0 other-yahoo Spelljammer other
9947607.0 1610870.0 11.0 Nick_Raskulinecz In_Your_Honor link
7890238.0 5400269.0 20.0 Test_Drive_4 TVR_Cerbera_Speed_12 link
3051596.0 611873.0 17.0 Chaminade_College_Preparatory_School_(California) West_Hills,_Los_Angeles other
null 2054489.0 92.0 other-empty Jean_van_de_Velde_(golfer) other
1.8220755e7 3.9815494e7 123.0 Holly_Hunter Bonnie_&_Clyde_(2013_miniseries) link
null 1.2201032e7 83.0 other-wikipedia M_jak_miłość other
1511052.0 1182345.0 40.0 Jim_Kelly_(martial_artist) Undercover_Brother link
127894.0 3.2564669e7 30.0 Winston-Salem,_North_Carolina Novant_Health link
72566.0 806290.0 29.0 Carmina_Burana Cockaigne link
1018512.0 1.2219012e7 12.0 Culture_of_Burma Burmese_dance link
4.4789934e7 1.9156186e7 10.0 Deaths_in_2015 Adrian_Peterson other
null 1615103.0 51.0 other-other Lucy_Ford:_The_Atmosphere_EP's other
null 3.4567346e7 17.0 other-google Simon_Paulli other
1113778.0 235321.0 40.0 Heaven_Tonight Rick_Nielsen link
null 2.1158505e7 97.0 other-other SDL_Trados other
null 1.7838929e7 13.0 other-google Christian_Maclagan other
294791.0 4.248568e7 18.0 Steven_Moffat Time_Heist link
null 3.5096982e7 14.0 other-empty Even_If_It_Breaks_Your_Heart other
null 3352391.0 33.0 other-wikipedia Pecheneg_language other
109495.0 2835130.0 25.0 Key_West,_Florida Key_Haven,_Florida link
22093.0 1658814.0 233.0 National_Basketball_Association Barclays_Center link
24096.0 1.9049004e7 16.0 Plough Stump-jump_plough link
null 5576399.0 10.0 other-wikipedia Santahamina other
null 1.3646286e7 29.0 other-google WYFI other
null 1803482.0 21.0 other-empty Aldene_Connection other
308142.0 4.1443125e7 26.0 General_Santos SM_City_General_Santos link
1.9486157e7 2.0975298e7 116.0 Mirotic Mirotic_(song) other
192381.0 1.7161967e7 27.0 Joe_Pantoliano The_Handler_(TV_series) link
967278.0 2604085.0 49.0 KiKa Bernd_das_Brot link
null 8252419.0 853.0 other-google Foxhole other
55906.0 17391.0 16.0 Zagreb Kosovo link
null 130195.0 23.0 other-google Covington,_Oklahoma other
null 2984353.0 21.0 other-empty 251_(number) other
23324.0 20474.0 22.0 Platinum Mohs_scale_of_mineral_hardness link
1298.0 2.3555068e7 10.0 Ames,_Iowa Neva_Morris link
2117651.0 1.808438e7 18.0 AFI's_100_Years...100_Movie_Quotes George_M._Cohan link
null 2.8370582e7 14.0 other-google Palača other
138022.0 267590.0 34.0 North_Bend,_Washington Mount_Si link
null 4.0810754e7 325.0 other-google List_of_travel_books other
2387806.0 12301.0 26.0 Harry_Potter A_Song_of_Ice_and_Fire other
48630.0 7159144.0 11.0 2014 Michael_Sata link
1.1976532e7 3.7562767e7 84.0 TOP500 Graph500 link
599365.0 1.904146e7 78.0 Liiga Kanada-malja link
null 4.1274079e7 44.0 other-wikipedia Jeremy_Jamm other
4.0379651e7 8744746.0 144.0 IBM Big_Blue_(disambiguation) link
52036.0 5573.0 50.0 Istria Croatia link
null 8260496.0 106.0 other-wikipedia Wrestle_Kingdom other
402652.0 581760.0 27.0 Compulsive_hoarding Plyushkin link
4.2839033e7 2.1418097e7 13.0 List_of_hot_dog_restaurants Montreal_Pool_Room link
null 2.394422e7 28.0 other-wikipedia Iván_Pillud other
206004.0 988219.0 14.0 Military_history_of_Egypt_during_World_War_II East_African_Campaign_(World_War_II) link
null 1.0235545e7 14.0 other-other D'Ieteren other
null 475805.0 548.0 other-google Pansy_Division other
49966.0 287855.0 199.0 Carlos_Menem Cecilia_Bolocco link
null 1.4750893e7 44.0 other-google Ain't_No_Shame_in_My_Game other
2.0550801e7 2.6771113e7 10.0 La_Scala_(album) Tokyo_'96 link
null 7304198.0 22.0 other-empty Ernst_Fischer other
1.5851039e7 10618.0 11.0 Davy_Crockett_and_the_River_Pirates Fiddle other
null 1.1927959e7 17.0 other-wikipedia Metal_Gear_Solid_2:_Sons_of_Liberty_Soundtrack_2:_The_Other_Side other
null 38325.0 15.0 other-bing Descent other
null 6017828.0 11.0 other-empty Center_for_Libertarian_Studies other
3.0873608e7 245390.0 21.0 Metal_Gear_(video_game) Stealth_game link
2.0846219e7 2.084622e7 12.0 Benzathine Benzathine_phenoxymethylpenicillin link
null 3.666971e7 21.0 other-google Sydney_state_by-election,_2012 other
498348.0 1587778.0 12.0 Guinea_Pig_(film_series) Japanese_horror other

DataFrame in Pyspark

Python API for Spark

clicksPy = sqlContext.read.parquet("/datasets/sds/tmp/wiki-clickstream")
# in Python you need to put the object int its own line like this to get the type information
clicksPy 
clicksPy.show()

DataFrame in SparkR

We can now load a DataFrame in R (using SparkR) from the parquet file just as we did for python. Read the docs in databricks guide first:

And see the R example in the Programming Guide:

library(SparkR)

# just a quick test
df <- createDataFrame(faithful)
head(df)
# Read in the Parquet file created above. Parquet files are self-describing so the schema is preserved.
# The result of loading a parquet file is also a DataFrame.
clicksR <- read.df("/datasets/sds/tmp/wiki-clickstream", source = "parquet")
clicksR # in R you need to put the object int its own line like this to get the type information
head(clicksR)
display(clicksR)

YouTry! in databricks

If you are on databricks then clone this notebook and run all cells in the cloned notebook by using the full dataset and not the small sample we used here.

Loading and exploring the data Cmd 9 should be the following to load the full data:

val data = sc.textFile("databricks-datasets/wikipedia-datasets/data-001/clickstream/raw-uncompressed/2015_2_clickstream.tsv")

We are avoiding this YouTry! in zeppelin as we do not want to load very large datasets into a spark server running on possibly limited resources.

Editors

Here is a list of the editors who have helped improve this book