[SPARK-28133] Adding inverse hyperbolic functions in SQL#24994
[SPARK-28133] Adding inverse hyperbolic functions in SQL#24994Tonix517 wants to merge 15 commits intoapache:masterfrom
Conversation
sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/mathExpressions.scala
Outdated
Show resolved
Hide resolved
| extends UnaryMathExpression((x: Double) => math.log(x + math.sqrt(x * x + 1.0)), "ASINH") { | ||
| override def doGenCode(ctx: CodegenContext, ev: ExprCode): ExprCode = { | ||
| defineCodeGen(ctx, ev, c => | ||
| s"${ev.value} = java.lang.Math.log($c + java.lang.Math.sqrt($c * $c + 1.0));") |
There was a problem hiding this comment.
what if the input is negative?
There was a problem hiding this comment.
It will return NaN, the same behavior as for existing sinh implementation.
There was a problem hiding this comment.
mmh..what do other DBs do in this case? I checked the implementation of FastMath and it handles negatives...
There was a problem hiding this comment.
thanks for reminding me of FastMath lib. Maybe we can simply use that lib here, without this hand-crafted calculation?
There was a problem hiding this comment.
Also I just tried the code with several negative values, i got correct return values.
There was a problem hiding this comment.
Also I just tried the code with several negative values, i got correct return values.
Sorry what did you try?
Anyway, if we don't have a dependency on it, I am not sure it is a good idea to add a dependency for only a couple of functions...
There was a problem hiding this comment.
took a closer look at asinh\atanh implementation in FastMath. the negative handling inside is to apply calculation optimization on abs(double a) and then apply proper sign at the end of the function to return, which should produce the same value as the original formula i'm using here (https://en.wikipedia.org/wiki/Inverse_hyperbolic_functions#Definitions_in_terms_of_logarithms). But FastMath runs faster than the original functions though.
sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/mathExpressions.scala
Outdated
Show resolved
Hide resolved
| 0.0 | ||
| """) | ||
| case class Atanh(child: Expression) | ||
| extends UnaryMathExpression((x: Double) => 0.5 * math.log((1.0 + x) / (1.0 - x)), "ATANH") { |
There was a problem hiding this comment.
again, what if the input is negative?
srowen
left a comment
There was a problem hiding this comment.
These are very niche functions. Does any other DB support it?
|
Copied the link from the JIRA ticket: Just tried Hive and these functions were not supported indeed.
|
| * @since 1.4.0 | ||
| * @since 3.0.0 | ||
| */ | ||
| def asinh(columnName: String): Column = asinh(Column(columnName)) |
There was a problem hiding this comment.
Shall we remove those APIs in Scala and R sides?
There was a problem hiding this comment.
Hi @HyukjinKwon, do you mean we remove then from R for the same reason we removed them from Python, and just keep them in Scala?
There was a problem hiding this comment.
Sorry I had to clarify it. Yes, let's add it only into SQL side and remove them in Scala, Python and R per https://github.com/apache/spark/blob/master/sql/core/src/main/scala/org/apache/spark/sql/functions.scala#L42-L56
These are only added into other language APIs when they are commonly used. I think we don't have to add them in the first place in general.
There was a problem hiding this comment.
thanks Hyukjin. just removed them from language APIs. now only available in SQL
|
Let me leave this triggered. cc @wangyum and @gatorsmile it looks like the same feature party of PostgreSQL |
|
ok to test |
|
Test build #107119 has finished for PR 24994 at commit
|
… early
## What changes were proposed in this pull request?
Closes the generator when Python UDFs stop early.
### Manually verification on pandas iterator UDF and mapPartitions
```python
from pyspark.sql import SparkSession
from pyspark.sql.functions import pandas_udf, PandasUDFType
from pyspark.sql.functions import col, udf
from pyspark.taskcontext import TaskContext
import time
import os
spark.conf.set('spark.sql.execution.arrow.maxRecordsPerBatch', '1')
spark.conf.set('spark.sql.pandas.udf.buffer.size', '4')
pandas_udf("int", PandasUDFType.SCALAR_ITER)
def fi1(it):
try:
for batch in it:
yield batch + 100
time.sleep(1.0)
except BaseException as be:
print("Debug: exception raised: " + str(type(be)))
raise be
finally:
open("/tmp/000001.tmp", "a").close()
df1 = spark.range(10).select(col('id').alias('a')).repartition(1)
# will see log Debug: exception raised: <class 'GeneratorExit'>
# and file "/tmp/000001.tmp" generated.
df1.select(col('a'), fi1('a')).limit(2).collect()
def mapper(it):
try:
for batch in it:
yield batch
except BaseException as be:
print("Debug: exception raised: " + str(type(be)))
raise be
finally:
open("/tmp/000002.tmp", "a").close()
df2 = spark.range(10000000).repartition(1)
# will see log Debug: exception raised: <class 'GeneratorExit'>
# and file "/tmp/000002.tmp" generated.
df2.rdd.mapPartitions(mapper).take(2)
```
## How was this patch tested?
Unit test added.
Please review https://spark.apache.org/contributing.html before opening a pull request.
Closes apache#24986 from WeichenXu123/pandas_iter_udf_limit.
Authored-by: WeichenXu <weichen.xu@databricks.com>
Signed-off-by: HyukjinKwon <gurwls223@apache.org>
## What changes were proposed in this pull request?
The `OVERLAY` function is a `ANSI` `SQL`.
For example:
```
SELECT OVERLAY('abcdef' PLACING '45' FROM 4);
SELECT OVERLAY('yabadoo' PLACING 'daba' FROM 5);
SELECT OVERLAY('yabadoo' PLACING 'daba' FROM 5 FOR 0);
SELECT OVERLAY('babosa' PLACING 'ubb' FROM 2 FOR 4);
```
The results of the above four `SQL` are:
```
abc45f
yabadaba
yabadabadoo
bubba
```
Note: If the input string is null, then the result is null too.
There are some mainstream database support the syntax.
**PostgreSQL:**
https://www.postgresql.org/docs/11/functions-string.html
**Vertica:** https://www.vertica.com/docs/9.2.x/HTML/Content/Authoring/SQLReferenceManual/Functions/String/OVERLAY.htm?zoom_highlight=overlay
**Oracle:**
https://docs.oracle.com/en/database/oracle/oracle-database/19/arpls/UTL_RAW.html#GUID-342E37E7-FE43-4CE1-A0E9-7DAABD000369
**DB2:**
https://www.ibm.com/support/knowledgecenter/SSGMCP_5.3.0/com.ibm.cics.rexx.doc/rexx/overlay.html
There are some show of the PR on my production environment.
```
spark-sql> SELECT OVERLAY('abcdef' PLACING '45' FROM 4);
abc45f
Time taken: 6.385 seconds, Fetched 1 row(s)
spark-sql> SELECT OVERLAY('yabadoo' PLACING 'daba' FROM 5);
yabadaba
Time taken: 0.191 seconds, Fetched 1 row(s)
spark-sql> SELECT OVERLAY('yabadoo' PLACING 'daba' FROM 5 FOR 0);
yabadabadoo
Time taken: 0.186 seconds, Fetched 1 row(s)
spark-sql> SELECT OVERLAY('babosa' PLACING 'ubb' FROM 2 FOR 4);
bubba
Time taken: 0.151 seconds, Fetched 1 row(s)
spark-sql> SELECT OVERLAY(null PLACING '45' FROM 4);
NULL
Time taken: 0.22 seconds, Fetched 1 row(s)
spark-sql> SELECT OVERLAY(null PLACING 'daba' FROM 5);
NULL
Time taken: 0.157 seconds, Fetched 1 row(s)
spark-sql> SELECT OVERLAY(null PLACING 'daba' FROM 5 FOR 0);
NULL
Time taken: 0.254 seconds, Fetched 1 row(s)
spark-sql> SELECT OVERLAY(null PLACING 'ubb' FROM 2 FOR 4);
NULL
Time taken: 0.159 seconds, Fetched 1 row(s)
```
## How was this patch tested?
Exists UT and new UT.
Closes apache#24918 from beliefer/ansi-sql-overlay.
Lead-authored-by: gengjiaan <gengjiaan@360.cn>
Co-authored-by: Jiaan Geng <beliefer@163.com>
Signed-off-by: Takuya UESHIN <ueshin@databricks.com>
## What changes were proposed in this pull request? Add error handling to `ExecutorPodsPollingSnapshotSource` Closes apache#24952 from onursatici/os/polling-source. Authored-by: Onur Satici <onursatici@gmail.com> Signed-off-by: Sean Owen <sean.owen@databricks.com>
## What changes were proposed in this pull request? This is the first part of [SPARK-27396](https://issues.apache.org/jira/browse/SPARK-27396). This is the minimum set of changes necessary to support a pluggable back end for columnar processing. Follow on JIRAs would cover removing some of the duplication between functionality in this patch and functionality currently covered by things like ColumnarBatchScan. ## How was this patch tested? I added in a new unit test to cover new code not really covered in other places. I also did manual testing by implementing two plugins/extensions that take advantage of the new APIs to allow for columnar processing for some simple queries. One version runs on the [CPU](https://gist.github.com/revans2/c3cad77075c4fa5d9d271308ee2f1b1d). The other version run on a GPU, but because it has unreleased dependencies I will not include a link to it yet. The CPU version I would expect to add in as an example with other documentation in a follow on JIRA This is contributed on behalf of NVIDIA Corporation. Closes apache#24795 from revans2/columnar-basic. Authored-by: Robert (Bobby) Evans <bobby@apache.org> Signed-off-by: Thomas Graves <tgraves@apache.org>
…Pandas UDF ## What changes were proposed in this pull request? Add docstring/doctest for `SCALAR_ITER` Pandas UDF. I explicitly mentioned that per-partition execution is an implementation detail, not guaranteed. I will submit another PR to add the same to user guide, just to keep this PR minimal. I didn't add "doctest: +SKIP" in the first commit so it is easy to test locally. cc: HyukjinKwon gatorsmile icexelloss BryanCutler WeichenXu123   ## How was this patch tested? doctest Closes apache#25005 from mengxr/SPARK-28056.2. Authored-by: Xiangrui Meng <meng@databricks.com> Signed-off-by: gatorsmile <gatorsmile@gmail.com>
…ning in binary files ## What changes were proposed in this pull request? SPARK-27534 missed to address my own comments at WeichenXu123#8 It's better to push this in since the codes are already cleaned up. ## How was this patch tested? Unittests fixed Closes apache#25003 from HyukjinKwon/SPARK-27534. Authored-by: HyukjinKwon <gurwls223@apache.org> Signed-off-by: HyukjinKwon <gurwls223@apache.org>
## What changes were proposed in this pull request?
Currently, ORC's `inferSchema` is implemented as randomly choosing one ORC file and reading its schema.
This PR follows the behavior of Parquet, it implements merge schemas logic by reading all ORC files in parallel through a spark job.
Users can enable merge schema by `spark.read.orc("xxx").option("mergeSchema", "true")` or by setting `spark.sql.orc.mergeSchema` to `true`, the prior one has higher priority.
## How was this patch tested?
tested by UT OrcUtilsSuite.scala
Closes apache#24043 from WangGuangxin/SPARK-11412.
Lead-authored-by: wangguangxin.cn <wangguangxin.cn@gmail.com>
Co-authored-by: wangguangxin.cn <wangguangxin.cn@bytedance.com>
Signed-off-by: gatorsmile <gatorsmile@gmail.com>
…OperationSuite ## What changes were proposed in this pull request? To make the apache#24972 change smaller. This pr improves `SparkMetadataOperationSuite` to avoid creating new sessions when getSchemas/getTables/getColumns. ## How was this patch tested? N/A Closes apache#24985 from wangyum/SPARK-28184. Authored-by: Yuming Wang <yumwang@ebay.com> Signed-off-by: gatorsmile <gatorsmile@gmail.com>
…Is for SessionCatalog ## What changes were proposed in this pull request? This pr add two API for [SessionCatalog](https://github.com/apache/spark/blob/df4cb471c9712a2fe496664028d9303caebd8777/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/catalog/SessionCatalog.scala): ```scala def listTables(db: String, pattern: String, includeLocalTempViews: Boolean): Seq[TableIdentifier] def listLocalTempViews(pattern: String): Seq[TableIdentifier] ``` Because in some cases `listTables` does not need local temporary view and sometimes only need list local temporary view. ## How was this patch tested? unit tests Closes apache#24995 from wangyum/SPARK-28196. Authored-by: Yuming Wang <yumwang@ebay.com> Signed-off-by: gatorsmile <gatorsmile@gmail.com>
…checked exception missed ## What changes were proposed in this pull request? This is very like apache#23590 . `ByteBuffer.allocate` may throw `OutOfMemoryError` when the response is large but no enough memory is available. However, when this happens, `TransportClient.sendRpcSync` will just hang forever if the timeout set to unlimited. This PR catches `Throwable` and uses the error to complete `SettableFuture`. ## How was this patch tested? I tested in my IDE by setting the value of size to -1 to verify the result. Without this patch, it won't be finished until timeout (May hang forever if timeout set to MAX_INT), or the expected `IllegalArgumentException` will be caught. ```java Override public void onSuccess(ByteBuffer response) { try { int size = response.remaining(); ByteBuffer copy = ByteBuffer.allocate(size); // set size to -1 in runtime when debug copy.put(response); // flip "copy" to make it readable copy.flip(); result.set(copy); } catch (Throwable t) { result.setException(t); } } ``` Closes apache#24964 from LantaoJin/SPARK-28160. Lead-authored-by: LantaoJin <jinlantao@gmail.com> Co-authored-by: lajin <lajin@ebay.com> Signed-off-by: Sean Owen <sean.owen@databricks.com>
## What changes were proposed in this pull request? The documentation in `linalg.py` is not consistent. This PR uniforms the documentation. ## How was this patch tested? NA Closes apache#25011 from mgaido91/SPARK-28170. Authored-by: Marco Gaido <marcogaido91@gmail.com> Signed-off-by: HyukjinKwon <gurwls223@apache.org>
|
Looks something when wrong during rebase. Can you resolve it or open a new PR? |
|
Test build #107149 has finished for PR 24994 at commit
|
What changes were proposed in this pull request?
Inverse hyperbolic functions (asinh, acosh, atanh) were missing from SQL. Now we add support to those functions.
How was this patch tested?
Added new tests.
./build/sbt "testOnly org.apache.spark.sql.catalyst.expressions.MathExpressionsSuite
./build/sbt "testOnly org.apache.spark.sql.MathFunctionsSuite
[In spark-shell]
spark.sql("select asinh(xx)").show
spark.sql("select acosh(xx)").show
spark.sql("select atanh(xx)").show
Please review https://spark.apache.org/contributing.html before opening a pull request.
@wangyum @mgaido91