You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@madlib.apache.org by nj...@apache.org on 2017/01/26 00:56:39 UTC

incubator-madlib git commit: Association Rules: Add rule counts and limit itemset size feature

Repository: incubator-madlib
Updated Branches:
  refs/heads/master e384c1fc7 -> 8e5da2ff0


Association Rules: Add rule counts and limit itemset size feature

JIRA: MADLIB-1044, MADLIB-1031

- The output column currently lists support, along with other columns.
The output table now contains count too, which is essentially
support*number_of_transactions for any rule. This could have been
obtained by post-processing the original output table content too,
but listing it explicitly might be useful.
- Add a new optional parameter named max_itemset_size that determines
the maximum size of frequent itemsets that are used for generating
association rules. Must be 2 or more, and the default behavior is to
generate itemsets of all sizes. This parameter can be used to reduce
run time for data sets where itemset size is large.
- This commit also includes improved online and user documentation
with examples of usage.

Closes #87


Project: http://git-wip-us.apache.org/repos/asf/incubator-madlib/repo
Commit: http://git-wip-us.apache.org/repos/asf/incubator-madlib/commit/8e5da2ff
Tree: http://git-wip-us.apache.org/repos/asf/incubator-madlib/tree/8e5da2ff
Diff: http://git-wip-us.apache.org/repos/asf/incubator-madlib/diff/8e5da2ff

Branch: refs/heads/master
Commit: 8e5da2ff053ca49b3c08bdb5e540a3370f6e8d09
Parents: e384c1f
Author: Nandish Jayaram <nj...@users.noreply.github.com>
Authored: Wed Jan 25 16:44:58 2017 -0800
Committer: Nandish Jayaram <nj...@users.noreply.github.com>
Committed: Wed Jan 25 16:44:58 2017 -0800

----------------------------------------------------------------------
 .../modules/assoc_rules/assoc_rules.py_in       | 132 ++++++++++++-
 .../modules/assoc_rules/assoc_rules.sql_in      | 186 +++++++++++++++----
 .../modules/assoc_rules/test/assoc_rules.sql_in |   9 +
 3 files changed, 282 insertions(+), 45 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/incubator-madlib/blob/8e5da2ff/src/ports/postgres/modules/assoc_rules/assoc_rules.py_in
----------------------------------------------------------------------
diff --git a/src/ports/postgres/modules/assoc_rules/assoc_rules.py_in b/src/ports/postgres/modules/assoc_rules/assoc_rules.py_in
index d5eee48..abc8b50 100644
--- a/src/ports/postgres/modules/assoc_rules/assoc_rules.py_in
+++ b/src/ports/postgres/modules/assoc_rules/assoc_rules.py_in
@@ -7,7 +7,6 @@
 
 @namespace assoc_rules
 """
-
 import time
 import plpy
 from utilities.validate_args import columns_exist_in_table
@@ -55,13 +54,19 @@ def __float_le(val1, val2):
 @param input_table     name of the table where the data is stored
 @param output_schema   name of the schema where the final results will be stored
 @param verbose         determining if output contains comments
+@param max_itemset_size determines the maximum size of frequent itemsets allowed
+                        to generate association rules from
 """
 def assoc_rules(madlib_schema, support, confidence, tid_col,
-                item_col, input_table, output_schema, verbose):
+                item_col, input_table, output_schema, verbose, max_itemset_size):
 
     begin_func_exec = time.time();
     begin_step_exec = time.time();
     cal_itemsets_time = 0;
+    if max_itemset_size is None:
+        max_itemset_size = float('inf')
+    elif max_itemset_size <= 1:
+        plpy.error("ERROR: max_itemset_size has to be greater than 1.")
 
     #check parameters
     __assert(
@@ -113,6 +118,7 @@ def assoc_rules(madlib_schema, support, confidence, tid_col,
             ruleId      INT,
             pre         TEXT[],
             post        TEXT[],
+            count       INT,
             support     FLOAT8,
             confidence  FLOAT8,
             lift        FLOAT8,
@@ -304,7 +310,7 @@ def assoc_rules(madlib_schema, support, confidence, tid_col,
 
     iter = 0;
 
-    while num_item_loop > 0 :
+    while num_item_loop > 0 and iter < max_itemset_size:
         begin_step_exec = time.time();
         iter = iter + 1;
 
@@ -459,14 +465,14 @@ def assoc_rules(madlib_schema, support, confidence, tid_col,
 
         plpy.execute("""
              INSERT INTO {0}.assoc_rules
-             SELECT t1.ruleId, t2.pre, t3.post, support,
+             SELECT t1.ruleId, t2.pre, t3.post, t1.support*{1}::INT AS count, support,
                     confidence, lift, conviction
              FROM
                 assoc_rules_aux_tmp t1,
                 pre_tmp_table t2,
                 post_tmp_table t3
              WHERE t1.ruleId = t2.ruleId AND t1.ruleId = t3.ruleId
-             """.format(output_schema)
+             """.format(output_schema, num_tranx)
              );
 
         # if in verbose mode, we will keep all the intermediate tables
@@ -504,3 +510,119 @@ def assoc_rules(madlib_schema, support, confidence, tid_col,
             total_rules,
             time.time() - begin_func_exec
            );
+
+def assoc_rules_help_message(schema_madlib, message=None, **kwargs):
+    """
+    Given a help string, provide usage information
+
+    Args:
+        @param schema_madlib Name of the MADlib schema
+        @param message  Helper message to print
+
+    Returns:
+        None
+    """
+    if message is not None and \
+            message.lower() in ("usage", "help", "?"):
+        return """
+-----------------------------------------------------------------------
+                                USAGE
+-----------------------------------------------------------------------
+SELECT {schema_madlib}.assoc_rules(
+    support,            -- FLOAT8, minimum level of support needed for each itemset to be included in result
+    confidence,         -- FLOAT8, minimum level of confidence needed for each rule to be included in result
+    tid_col,            -- TEXT, name of the column storing the transaction ids
+    item_col,           -- TEXT, name of the column storing the products
+    input_table,        -- TEXT, name of the table containing the input data
+    output_schema,      -- TEXT, name of the schema where the final results will be stored.
+                                The schema must be created before calling the function.  Alternatively, use
+                                <tt>NULL</tt> to output to the current schema.
+    verbose,            -- BOOLEAN, (optional, default: False) determines if details are printed for each
+                                iteration as the algorithm progresses
+    max_itemset_size    -- INTEGER, (optional, default: itemsets of all sizes) determines the maximum size of frequent
+                                itemsets allowed that are used for generating association rules. Value less
+                                than 2 throws an error.
+);
+-------------------------------------------------------------------------
+                                OUTPUT TABLES
+-------------------------------------------------------------------------
+The output table "assoc_rules" in the "output_schema" contains a unique rule of the form "If X, then Y
+(i.e., X => Y)" in each row. X and Y are non-empty itemsets, called the antecedent and consequent, or
+the left-hand-side (LHS) and right-hand-side (LHS), of the rule respectively.
+
+in each row, with the following columns:
+    ruleid,     -- INTEGER, row number
+    pre,        -- TEXT, specifies the antecedent, or the LHS of the rule
+    post,       -- DOUBLE, specifies the consequent, or the RHS of the rule
+    support,    -- DOUBLE, support of the frequent itemset X,Y
+    count,      -- INTEGER, number of transactions in the input table that contain X,Y
+    confidence, -- DOUBLE, the ratio of number of transactions that contain X,Y to the number of transactions
+                        that contain X
+    lift,       -- DOUBLE, the ratio of observed support of X,Y to the expected support of X,Y, assuming X and
+                        Y are independent.
+    conviction  -- DOUBLE, the ratio of expected support of X occurring without Y assuming X and Y are
+                        independent, to the observed support of X occuring without Y
+        """.format(schema_madlib=schema_madlib)
+    else:
+        if message.lower() in ("example", "examples"):
+            return """
+------------------------------------------------------------------------
+                                EXAMPLES
+------------------------------------------------------------------------
+DROP TABLE IF EXISTS test_data;
+CREATE TABLE test_data (
+    trans_id INT,
+    product TEXT
+);
+INSERT INTO test_data VALUES (1, 'beer');
+INSERT INTO test_data VALUES (1, 'diapers');
+INSERT INTO test_data VALUES (1, 'chips');
+INSERT INTO test_data VALUES (2, 'beer');
+INSERT INTO test_data VALUES (2, 'diapers');
+INSERT INTO test_data VALUES (3, 'beer');
+INSERT INTO test_data VALUES (3, 'diapers');
+INSERT INTO test_data VALUES (4, 'beer');
+INSERT INTO test_data VALUES (4, 'chips');
+INSERT INTO test_data VALUES (5, 'beer');
+INSERT INTO test_data VALUES (6, 'beer');
+INSERT INTO test_data VALUES (6, 'diapers');
+INSERT INTO test_data VALUES (6, 'chips');
+INSERT INTO test_data VALUES (7, 'beer');
+INSERT INTO test_data VALUES (7, 'diapers');
+
+Find all association rules with a support and threshold value of
+at least 0.25 and 0.5 respectively.
+
+SELECT * FROM {schema_madlib}.assoc_rules( .25,
+                          .5,
+                          'trans_id',
+                          'product',
+                          'test_data',
+                          NULL,
+                          TRUE
+                        );
+
+View output results:
+SELECT * FROM assoc_rules;
+
+Find association rules generated from itemsets of size at most 2,
+and a support and threshold value of at least 0.25 and 0.5 respectively.
+
+SELECT * FROM {schema_madlib}.assoc_rules( .25,
+                          .5,
+                          'trans_id',
+                          'product',
+                          'test_data',
+                          NULL,
+                          TRUE,
+                          2
+                        );
+
+View output results:
+SELECT * FROM assoc_rules;
+            """.format(schema_madlib=schema_madlib)
+        else:
+            return """
+For an overview on usage, run: SELECT {schema_madlib}.assoc_rules('usage');
+For an example of using assoc_rules, run: SELECT {schema_madlib}.assoc_rules('example');
+            """.format(schema_madlib=schema_madlib)

http://git-wip-us.apache.org/repos/asf/incubator-madlib/blob/8e5da2ff/src/ports/postgres/modules/assoc_rules/assoc_rules.sql_in
----------------------------------------------------------------------
diff --git a/src/ports/postgres/modules/assoc_rules/assoc_rules.sql_in b/src/ports/postgres/modules/assoc_rules/assoc_rules.sql_in
index 99b8f00..fc87b16 100644
--- a/src/ports/postgres/modules/assoc_rules/assoc_rules.sql_in
+++ b/src/ports/postgres/modules/assoc_rules/assoc_rules.sql_in
@@ -43,7 +43,7 @@ support and confidence values, this function generates all single and
 multidimensional association rules that meet the minimum thresholds.
 
 Association rule mining is a widely used technique for discovering relationships
-between variables in a large data set (e.g items in a store that are commonly
+between variables in a large data set (e.g., items in a store that are commonly
 purchased together). The classic market basket analysis example using
 association rules is the "beer and diapers" rule. According to data mining urban
 legend, a study of customer purchase behavior in a supermarket found that men
@@ -62,7 +62,7 @@ data is stored in two columns with one item and transaction id per row.
 Transactions with multiple items will span multiple rows with one row per item.
 
 <pre>
-     tran_id | product
+    trans_id | product
     ---------+---------
            1 | 1
            1 | 2
@@ -81,18 +81,28 @@ Transactions with multiple items will span multiple rows with one row per item.
 @par Rules
 
 Association rules take the form "If X, then Y", where X and Y are non-empty
-itemsets. X and Y are called the antecedent and consequent, or the left-hand-side 
+itemsets. X and Y are called the antecedent and consequent, or the left-hand-side
 and right-hand-side, of the rule respectively. Using our previous example,
 the association rule may state "If {diapers}, then {beer}" with .2 support and
 .85 confidence.
 
-Given any association rule "If X, then Y", the association rules function will
-also calculate the following metrics:
+The following metrics are defined for any given itemset "X".
+- Count: The number of transactions that contain X
+
 - Support: The ratio of transactions that contain X to all transactions, T
 \f[
 S (X) = \frac{Total X}{Total transactions}
 \f]
 
+Given any association rule "If X, then Y", the association rules function will
+also calculate the following metrics:
+- Count: The number of transactions that contain X,Y
+
+- Support: The ratio of transactions that contain X,Y to all transactions, T
+\f[
+S (X \Rightarrow Y) = \frac{Total(X \cup Y)}{Total transactions}
+\f]
+
 - Confidence: The ratio of transactions that contain \f$ X,Y \f$ to
 transactions that contain \f$ X \f$. One could view this metric as the
 conditional probability of \f$ Y \f$ , given \f$ X \f$ . \f$ P(Y|X) \f$
@@ -124,7 +134,7 @@ Conv (X \Rightarrow Y) = \frac{1 - S(Y)}{1 - C(X \Rightarrow Y)}
 @par Apriori Algorithm
 
 Although there are many algorithms that generate association rules, the classic
-algorithm used is called Apriori [1] which we have implemented in this module. It is a
+algorithm is called Apriori [1] which we have implemented in this module. It is a
 breadth-first search, as opposed to depth-first searches like Eclat. Frequent
 itemsets of order \f$ n \f$ are generated from sets of order \f$ n - 1 \f$.
 Using the downward closure property, all sets must have frequent subsets. There
@@ -143,7 +153,7 @@ itemsets of order \f$ n - 1 \f$.
 This is done by doing the union of two itemsets that have identical items except one.
 -# Eliminate itemsets that have (n-1) order subsets with insufficient support.
 -# Eliminate itemsets with insufficient support.
--# Repeat until itemsets cannot be generated.
+-# Repeat until itemsets cannot be generated, or maximum itemset size is exceeded.
 
 \e Association \e rule \e generation
 
@@ -162,7 +172,8 @@ assoc_rules( support,
              item_col,
              input_table,
              output_schema,
-             verbose
+             verbose,
+             max_itemset_size
            );</pre>
 This generates all association rules that satisfy the specified minimum
 <em>support</em> and <em>confidence</em>.
@@ -170,19 +181,19 @@ This generates all association rules that satisfy the specified minimum
 \b Arguments
 <dl class="arglist">
   <dt>support</dt>
-  <dd>The minimum level of support needed for each itemset to be included in result.</dd>
+  <dd>Minimum level of support needed for each itemset to be included in result.</dd>
 
   <dt>confidence</dt>
-  <dd>The minimum level of confidence needed for each rule to be included in result.</dd>
+  <dd>Minimum level of confidence needed for each rule to be included in result.</dd>
 
   <dt>tid_col</dt>
-  <dd>The name of the column storing the transaction ids.</dd>
+  <dd>Name of the column storing the transaction ids.</dd>
 
   <dt>item_col</dt>
-  <dd>The name of the column storing the products.</dd>
+  <dd>Name of the column storing the products.</dd>
 
   <dt>input_table</dt>
-  <dd>The name of the table containing the input data.
+  <dd>Name of the table containing the input data.
 
   The input data is expected to be of the following form:
 <pre>{TABLE|VIEW} <em>input_table</em> (
@@ -199,7 +210,7 @@ This generates all association rules that satisfy the specified minimum
   The schema must be created before calling the function.  Alternatively, use
   <tt>NULL</tt> to output to the current schema.
 
-  The results containing the rules, support, confidence, lift, and
+  The results containing the rules, support, count, confidence, lift, and
   conviction are stored in the table \c assoc_rules in the schema
   specified by \c output_schema.
 
@@ -218,6 +229,10 @@ This generates all association rules that satisfy the specified minimum
         <td>text</td>
       </tr>
       <tr>
+        <th>count</th>
+        <td>integer</td>
+      </tr>
+      <tr>
         <th>support</th>
         <td>double</td>
       </tr>
@@ -243,15 +258,20 @@ This generates all association rules that satisfy the specified minimum
   </dd>
 
   <dt>verbose</dt>
-  <dd>BOOLEAN, default FALSE. Determines if details are printed for each iteration
+  <dd>BOOLEAN, default: FALSE. Determines if details are printed for each iteration
   as the algorithm progresses.</dd>
+
+  <dt>max_itemset_size</dt>
+  <dd>INTEGER, default: generate itemsets of all sizes. Determines the maximum size of frequent
+  itemsets that are used for generating association rules. Must be 2 or more.
+  This parameter can be used to reduce run time for data sets where itemset size is large. </dd>
 </dl>
 
 
 @anchor examples
 @examp
 
-Let us take a look at some sample transactional data and generate association rules.
+Let's look at some sample transactional data and generate association rules.
 
 -# Create an input dataset:
 <pre class="example">
@@ -283,20 +303,20 @@ In this example we set verbose to
 TRUE so that we have some insight into progress of the function. We
 can now generate association rules as follows:
 <pre class="example">
-SELECT * FROM madlib.assoc_rules( .25,
-                                  .5,
-                                  'trans_id',
-                                  'product',
-                                  'test_data',
-                                  NULL,
-                                  TRUE
+SELECT * FROM madlib.assoc_rules( .25,            -- Support
+                                  .5,             -- Confidence
+                                  'trans_id',     -- Transaction id col
+                                  'product',      -- Product col
+                                  'test_data',    -- Input data
+                                  NULL,           -- Output schema
+                                  TRUE            -- Verbose output
                                 );
 </pre>
 Result (iteration details not shown):
 <pre class="result">
  output_schema | output_table | total_rules |   total_time    
 ---------------+--------------+-------------+-----------------
- public        | assoc_rules  |           7 | 00:00:00.028534
+ public        | assoc_rules  |           7 | 00:00:00.569254
 (1 row)
 </pre>
 The association rules are stored in the assoc_rules table:
@@ -306,18 +326,53 @@ ORDER BY support DESC, confidence DESC;
 </pre>
 Result:
 <pre class="result">
- ruleid |       pre       |      post      |      support      |    confidence     |       lift        |    conviction     
---------+-----------------+----------------+-------------------+-------------------+-------------------+-------------------
-      4 | {diapers}       | {beer}         | 0.714285714285714 |                 1 |                 1 |                 0
-      3 | {beer}          | {diapers}      | 0.714285714285714 | 0.714285714285714 |                 1 |                 1
-      1 | {chips}         | {beer}         | 0.428571428571429 |                 1 |                 1 |                 0
-      7 | {diapers,chips} | {beer}         | 0.285714285714286 |                 1 |                 1 |                 0
-      2 | {chips}         | {diapers}      | 0.285714285714286 | 0.666666666666667 | 0.933333333333333 | 0.857142857142857
-      5 | {chips}         | {beer,diapers} | 0.285714285714286 | 0.666666666666667 | 0.933333333333333 | 0.857142857142857
-      6 | {beer,chips}    | {diapers}      | 0.285714285714286 | 0.666666666666667 | 0.933333333333333 | 0.857142857142857
+ ruleid |       pre       |      post      | count |      support      |    confidence     |       lift        |    conviction     
+--------+-----------------+----------------+-------+-------------------+-------------------+-------------------+-------------------
+      2 | {diapers}       | {beer}         |     5 | 0.714285714285714 |                 1 |                 1 |                 0
+      6 | {beer}          | {diapers}      |     5 | 0.714285714285714 | 0.714285714285714 |                 1 |                 1
+      5 | {chips}         | {beer}         |     3 | 0.428571428571429 |                 1 |                 1 |                 0
+      4 | {chips,diapers} | {beer}         |     2 | 0.285714285714286 |                 1 |                 1 |                 0
+      1 | {chips}         | {diapers,beer} |     2 | 0.285714285714286 | 0.666666666666667 | 0.933333333333333 | 0.857142857142857
+      7 | {chips}         | {diapers}      |     2 | 0.285714285714286 | 0.666666666666667 | 0.933333333333333 | 0.857142857142857
+      3 | {beer,chips}    | {diapers}      |     2 | 0.285714285714286 | 0.666666666666667 | 0.933333333333333 | 0.857142857142857
 (7 rows)
 </pre>
 
+-# Limit association rules generated from itemsets of size at most 2:
+<pre class="example">
+SELECT * FROM madlib.assoc_rules( .25,            -- Support
+                                  .5,             -- Confidence
+                                  'trans_id',     -- Transaction id col
+                                  'product',      -- Product col
+                                  'test_data',    -- Input data
+                                  NULL,           -- Output schema
+                                  TRUE,           -- Verbose output
+                                  2               -- Max itemset size
+                                );
+</pre>
+Result (iteration details not shown):
+<pre class="result">
+ output_schema | output_table | total_rules |   total_time    
+---------------+--------------+-------------+-----------------
+ public        | assoc_rules  |           4 | 00:00:00.565176
+(1 row)
+</pre>
+The association rules are again stored in the assoc_rules table:
+<pre class="example">
+SELECT * FROM assoc_rules
+ORDER BY support DESC, confidence DESC;
+</pre>
+Result:
+<pre class="result">
+ ruleid |    pre    |   post    | count |      support      |    confidence     |       lift        |    conviction     
+--------+-----------+-----------+-------+-------------------+-------------------+-------------------+-------------------
+      1 | {diapers} | {beer}    |     5 | 0.714285714285714 |                 1 |                 1 |                 0
+      2 | {beer}    | {diapers} |     5 | 0.714285714285714 | 0.714285714285714 |                 1 |                 1
+      3 | {chips}   | {beer}    |     3 | 0.428571428571429 |                 1 |                 1 |                 0
+      4 | {chips}   | {diapers} |     2 | 0.285714285714286 | 0.666666666666667 | 0.933333333333333 | 0.857142857142857
+(4 rows)
+</pre>
+
 -# Post-processing can now be done on the output table in the case that
 you want to filter the results.  For example, if you want any single item on the left hand side 
 and a particular item on the right hand side:
@@ -326,10 +381,10 @@ SELECT * FROM assoc_rules WHERE array_upper(pre,1) = 1 AND post = array['beer'];
 </pre>
 Result:
 <pre class="result">
- ruleid |    pre    |  post  |      support      | confidence | lift | conviction 
---------+-----------+--------+-------------------+------------+------+------------
-      1 | {chips}   | {beer} | 0.428571428571429 |          1 |    1 |          0
-      4 | {diapers} | {beer} | 0.714285714285714 |          1 |    1 |          0
+ ruleid |    pre    |  post  | count |      support      | confidence | lift | conviction 
+--------+-----------+--------+-------+-------------------+------------+------+------------
+      1 | {diapers} | {beer} |     5 | 0.714285714285714 |          1 |    1 |          0
+      3 | {chips}   | {beer} |     3 | 0.428571428571429 |          1 |    1 |          0
 (2 rows)
 </pre>
 
@@ -433,7 +488,8 @@ CREATE OR REPLACE FUNCTION MADLIB_SCHEMA.assoc_rules
     item_col TEXT,
     input_table TEXT,
     output_schema TEXT,
-    verbose BOOLEAN
+    verbose BOOLEAN,
+    max_itemset_size INTEGER
    )
 RETURNS MADLIB_SCHEMA.assoc_rules_results
 AS $$
@@ -451,7 +507,8 @@ AS $$
         item_col,
         input_table,
         output_schema,
-        verbose
+        verbose,
+        max_itemset_size
         );
 
 $$ LANGUAGE plpythonu
@@ -488,8 +545,57 @@ AS $$
         item_col,
         input_table,
         output_schema,
-        False
+        False,
+        'NULL'
+        );
+
+$$ LANGUAGE plpythonu
+m4_ifdef(`__HAS_FUNCTION_PROPERTIES__', `MODIFIES SQL DATA', `');
+
+
+CREATE OR REPLACE FUNCTION MADLIB_SCHEMA.assoc_rules
+    (
+    support FLOAT8,
+    confidence FLOAT8,
+    tid_col TEXT,
+    item_col TEXT,
+    input_table TEXT,
+    output_schema TEXT,
+    verbose BOOLEAN
+    )
+RETURNS MADLIB_SCHEMA.assoc_rules_results
+AS $$
+
+    PythonFunctionBodyOnly(`assoc_rules', `assoc_rules')
+
+    plpy.execute("SET client_min_messages = error;")
+
+    # schema_madlib comes from PythonFunctionBodyOnly
+    return assoc_rules.assoc_rules(
+        schema_madlib,
+        support,
+        confidence,
+        tid_col,
+        item_col,
+        input_table,
+        output_schema,
+        verbose,
+        'NULL'
         );
 
 $$ LANGUAGE plpythonu
 m4_ifdef(`__HAS_FUNCTION_PROPERTIES__', `MODIFIES SQL DATA', `');
+
+--------------------------------------------------------------------------
+
+CREATE OR REPLACE FUNCTION MADLIB_SCHEMA.assoc_rules(message TEXT)
+RETURNS text AS $$
+PythonFunction(assoc_rules, assoc_rules, assoc_rules_help_message)
+$$ language plpythonu
+m4_ifdef(`__HAS_FUNCTION_PROPERTIES__', `CONTAINS SQL', `');
+
+CREATE OR REPLACE FUNCTION MADLIB_SCHEMA.assoc_rules()
+RETURNS text AS $$
+    SELECT MADLIB_SCHEMA.assoc_rules('');
+$$ language SQL
+m4_ifdef(`__HAS_FUNCTION_PROPERTIES__', `CONTAINS SQL', `');

http://git-wip-us.apache.org/repos/asf/incubator-madlib/blob/8e5da2ff/src/ports/postgres/modules/assoc_rules/test/assoc_rules.sql_in
----------------------------------------------------------------------
diff --git a/src/ports/postgres/modules/assoc_rules/test/assoc_rules.sql_in b/src/ports/postgres/modules/assoc_rules/test/assoc_rules.sql_in
index 59aba05..5a6275e 100644
--- a/src/ports/postgres/modules/assoc_rules/test/assoc_rules.sql_in
+++ b/src/ports/postgres/modules/assoc_rules/test/assoc_rules.sql_in
@@ -28,6 +28,7 @@ declare
     result1        TEXT;
     result2        TEXT;
     result3        TEXT;
+    result_maxiter TEXT;
     res            MADLIB_SCHEMA.assoc_rules_results;
     output_schema  TEXT;
     output_table   TEXT;
@@ -137,6 +138,10 @@ begin
           abs(t1.support - t2.support) < 1E-10 AND
           abs(t1.confidence - t2.confidence) < 1E-10;
 
+    PERFORM MADLIB_SCHEMA.assoc_rules (.1, .5, 'trans_id', 'product', 'test_data2','madlib_installcheck_assoc_rules', false, 2);
+    SELECT INTO result_maxiter CASE WHEN count(*) = 4 then 'PASS' ELSE 'FAIL' END
+    FROM assoc_rules;
+
     DROP TABLE IF EXISTS test_data1;
     DROP TABLE IF EXISTS test_data2;
     DROP TABLE IF EXISTS test2_exp_result;
@@ -150,6 +155,10 @@ begin
         RAISE EXCEPTION 'Association rules mining failed. No results were returned.';
     END IF;
 
+    IF result_maxiter = 'FAIL' THEN
+        RAISE EXCEPTION 'Association rules mining error when max_iter parameter specified.';
+    END IF;
+
     RAISE INFO 'Association rules install check passed.';
     RETURN;