You are viewing a plain text version of this content. The canonical link for it is here.
Posted to issues@hbase.apache.org by "Benjamin Kim (JIRA)" <ji...@apache.org> on 2012/05/30 03:11:23 UTC

[jira] [Created] (HBASE-6132) ColumnCountGetFilter & PageFilter not working with FilterList

Benjamin Kim created HBASE-6132:
-----------------------------------

             Summary: ColumnCountGetFilter & PageFilter not working with FilterList
                 Key: HBASE-6132
                 URL: https://issues.apache.org/jira/browse/HBASE-6132
             Project: HBase
          Issue Type: Bug
          Components: filters
    Affects Versions: 0.94.0, 0.92.1, 0.92.0
         Environment: Cent OS 5.5 distributed hbase cluster. Hadoop 1.0.0, zookeeper 3.4.3
            Reporter: Benjamin Kim


Thanks to Anoop and Ramkrishna, here's what we found with FilterList

If I use FilterList to include ColumnCountGetFilter among other filters, the returning Result has no keyvalues.

This problem seems to occur when specified column count is less then actual number of existing columns.

Also same problem arises with PageFilter

Following is the code of the problem:

Configuration conf = HBaseConfiguration.create();
HTable table = new HTable(conf, "test");
Get get = new Get(Bytes.toBytes("test00001"));
FilterList filterList = new FilterList();
filterList.addFilter(new ColumnCountGetFilter(100));           
get.setFilter(filterList);
Result r = table.get(get);
System.out.println(r.size()); // prints zero

--
This message is automatically generated by JIRA.
If you think it was sent incorrectly, please contact your JIRA administrators: https://issues.apache.org/jira/secure/ContactAdministrators!default.jspa
For more information on JIRA, see: http://www.atlassian.com/software/jira

        

[jira] [Commented] (HBASE-6132) ColumnCountGetFilter & PageFilter not working with FilterList

Posted by "Anoop Sam John (JIRA)" <ji...@apache.org>.
    [ https://issues.apache.org/jira/browse/HBASE-6132?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=13465414#comment-13465414 ] 

Anoop Sam John commented on HBASE-6132:
---------------------------------------

[~saint.ack@gmail.com]  Can you please see the attached test case once. Using HRegion directly. Owners of Filter- can some one pls tell the reason for code in FilterList#filterRow() using filterAllRemaining()
                
> ColumnCountGetFilter & PageFilter not working with FilterList
> -------------------------------------------------------------
>
>                 Key: HBASE-6132
>                 URL: https://issues.apache.org/jira/browse/HBASE-6132
>             Project: HBase
>          Issue Type: Bug
>          Components: Filters
>    Affects Versions: 0.92.0, 0.92.1, 0.94.0
>         Environment: Cent OS 5.5 distributed hbase cluster. Hadoop 1.0.0, zookeeper 3.4.3
>            Reporter: Benjamin Kim
>            Assignee: Anoop Sam John
>         Attachments: TestColumnCountGetFilter.java
>
>
> Thanks to Anoop and Ramkrishna, here's what we found with FilterList
> If I use FilterList to include ColumnCountGetFilter among other filters, the returning Result has no keyvalues.
> This problem seems to occur when specified column count is less then actual number of existing columns.
> Also same problem arises with PageFilter
> Following is the code of the problem:
> {code}
> Configuration conf = HBaseConfiguration.create();
> HTable table = new HTable(conf, "test");
> Get get = new Get(Bytes.toBytes("test00001"));
> FilterList filterList = new FilterList();
> filterList.addFilter(new ColumnCountGetFilter(100));           
> get.setFilter(filterList);
> Result r = table.get(get);
> System.out.println(r.size()); // prints zero
> {code}

--
This message is automatically generated by JIRA.
If you think it was sent incorrectly, please contact your JIRA administrators
For more information on JIRA, see: http://www.atlassian.com/software/jira

[jira] [Commented] (HBASE-6132) ColumnCountGetFilter & PageFilter not working with FilterList

Posted by "ramkrishna.s.vasudevan (JIRA)" <ji...@apache.org>.
    [ https://issues.apache.org/jira/browse/HBASE-6132?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=13289142#comment-13289142 ] 

ramkrishna.s.vasudevan commented on HBASE-6132:
-----------------------------------------------

May be we can prepare a patch and then go for review from others.  That would make things more clearer.
@NN
Anoop is already working on it.
                
> ColumnCountGetFilter & PageFilter not working with FilterList
> -------------------------------------------------------------
>
>                 Key: HBASE-6132
>                 URL: https://issues.apache.org/jira/browse/HBASE-6132
>             Project: HBase
>          Issue Type: Bug
>          Components: filters
>    Affects Versions: 0.92.0, 0.92.1, 0.94.0
>         Environment: Cent OS 5.5 distributed hbase cluster. Hadoop 1.0.0, zookeeper 3.4.3
>            Reporter: Benjamin Kim
>            Assignee: Anoop Sam John
>
> Thanks to Anoop and Ramkrishna, here's what we found with FilterList
> If I use FilterList to include ColumnCountGetFilter among other filters, the returning Result has no keyvalues.
> This problem seems to occur when specified column count is less then actual number of existing columns.
> Also same problem arises with PageFilter
> Following is the code of the problem:
> {code}
> Configuration conf = HBaseConfiguration.create();
> HTable table = new HTable(conf, "test");
> Get get = new Get(Bytes.toBytes("test00001"));
> FilterList filterList = new FilterList();
> filterList.addFilter(new ColumnCountGetFilter(100));           
> get.setFilter(filterList);
> Result r = table.get(get);
> System.out.println(r.size()); // prints zero
> {code}

--
This message is automatically generated by JIRA.
If you think it was sent incorrectly, please contact your JIRA administrators: https://issues.apache.org/jira/secure/ContactAdministrators!default.jspa
For more information on JIRA, see: http://www.atlassian.com/software/jira

        

[jira] [Updated] (HBASE-6132) ColumnCountGetFilter & PageFilter not working with FilterList

Posted by "Anoop Sam John (JIRA)" <ji...@apache.org>.
     [ https://issues.apache.org/jira/browse/HBASE-6132?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel ]

Anoop Sam John updated HBASE-6132:
----------------------------------

    Attachment: TestColumnCountGetFilter.java

Test case which clearly shows the diff when using the Filter alone and filter enclosed within FilterList.
Test case written on 0.94 code base
                
> ColumnCountGetFilter & PageFilter not working with FilterList
> -------------------------------------------------------------
>
>                 Key: HBASE-6132
>                 URL: https://issues.apache.org/jira/browse/HBASE-6132
>             Project: HBase
>          Issue Type: Bug
>          Components: filters
>    Affects Versions: 0.92.0, 0.92.1, 0.94.0
>         Environment: Cent OS 5.5 distributed hbase cluster. Hadoop 1.0.0, zookeeper 3.4.3
>            Reporter: Benjamin Kim
>            Assignee: Anoop Sam John
>         Attachments: TestColumnCountGetFilter.java
>
>
> Thanks to Anoop and Ramkrishna, here's what we found with FilterList
> If I use FilterList to include ColumnCountGetFilter among other filters, the returning Result has no keyvalues.
> This problem seems to occur when specified column count is less then actual number of existing columns.
> Also same problem arises with PageFilter
> Following is the code of the problem:
> {code}
> Configuration conf = HBaseConfiguration.create();
> HTable table = new HTable(conf, "test");
> Get get = new Get(Bytes.toBytes("test00001"));
> FilterList filterList = new FilterList();
> filterList.addFilter(new ColumnCountGetFilter(100));           
> get.setFilter(filterList);
> Result r = table.get(get);
> System.out.println(r.size()); // prints zero
> {code}

--
This message is automatically generated by JIRA.
If you think it was sent incorrectly, please contact your JIRA administrators: https://issues.apache.org/jira/secure/ContactAdministrators!default.jspa
For more information on JIRA, see: http://www.atlassian.com/software/jira

        

[jira] [Commented] (HBASE-6132) ColumnCountGetFilter & PageFilter not working with FilterList

Posted by "Anoop Sam John (JIRA)" <ji...@apache.org>.
    [ https://issues.apache.org/jira/browse/HBASE-6132?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=13288396#comment-13288396 ] 

Anoop Sam John commented on HBASE-6132:
---------------------------------------

I think that filterRow() in FilterList should not check with the filterAllRemaining() on individual filters. filterRow() is being called to check whether the current row under check needs to be included or not. filterAllRemaining() speaks abt the rows coming after the current row [In this context]
Note : filterAllRemaining() will be called during the read of individual KVs belonging to one row where it tells abt skipping the coming KVs.

Whatever KVs were included for one row should get returned by the scanner.  So usage of filterAllRemaining() as part of the filterRow() check should be avoided I feel.

Going through other method impls in FilterList. Getting doubts with some others also.
                
> ColumnCountGetFilter & PageFilter not working with FilterList
> -------------------------------------------------------------
>
>                 Key: HBASE-6132
>                 URL: https://issues.apache.org/jira/browse/HBASE-6132
>             Project: HBase
>          Issue Type: Bug
>          Components: filters
>    Affects Versions: 0.92.0, 0.92.1, 0.94.0
>         Environment: Cent OS 5.5 distributed hbase cluster. Hadoop 1.0.0, zookeeper 3.4.3
>            Reporter: Benjamin Kim
>
> Thanks to Anoop and Ramkrishna, here's what we found with FilterList
> If I use FilterList to include ColumnCountGetFilter among other filters, the returning Result has no keyvalues.
> This problem seems to occur when specified column count is less then actual number of existing columns.
> Also same problem arises with PageFilter
> Following is the code of the problem:
> {code}
> Configuration conf = HBaseConfiguration.create();
> HTable table = new HTable(conf, "test");
> Get get = new Get(Bytes.toBytes("test00001"));
> FilterList filterList = new FilterList();
> filterList.addFilter(new ColumnCountGetFilter(100));           
> get.setFilter(filterList);
> Result r = table.get(get);
> System.out.println(r.size()); // prints zero
> {code}

--
This message is automatically generated by JIRA.
If you think it was sent incorrectly, please contact your JIRA administrators: https://issues.apache.org/jira/secure/ContactAdministrators!default.jspa
For more information on JIRA, see: http://www.atlassian.com/software/jira

        

[jira] [Commented] (HBASE-6132) ColumnCountGetFilter & PageFilter not working with FilterList

Posted by "stack (JIRA)" <ji...@apache.org>.
    [ https://issues.apache.org/jira/browse/HBASE-6132?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=13289149#comment-13289149 ] 

stack commented on HBASE-6132:
------------------------------

Can you fellas put up the filters in a little test harness (maybe stand up a region as TestColumnPrefixFilter does)?  That'd give you more confidence about your theory that filterAllRemaining is not needed here.  This stuff is tricky.  Thank you for digging in on this.
                
> ColumnCountGetFilter & PageFilter not working with FilterList
> -------------------------------------------------------------
>
>                 Key: HBASE-6132
>                 URL: https://issues.apache.org/jira/browse/HBASE-6132
>             Project: HBase
>          Issue Type: Bug
>          Components: filters
>    Affects Versions: 0.92.0, 0.92.1, 0.94.0
>         Environment: Cent OS 5.5 distributed hbase cluster. Hadoop 1.0.0, zookeeper 3.4.3
>            Reporter: Benjamin Kim
>            Assignee: Anoop Sam John
>
> Thanks to Anoop and Ramkrishna, here's what we found with FilterList
> If I use FilterList to include ColumnCountGetFilter among other filters, the returning Result has no keyvalues.
> This problem seems to occur when specified column count is less then actual number of existing columns.
> Also same problem arises with PageFilter
> Following is the code of the problem:
> {code}
> Configuration conf = HBaseConfiguration.create();
> HTable table = new HTable(conf, "test");
> Get get = new Get(Bytes.toBytes("test00001"));
> FilterList filterList = new FilterList();
> filterList.addFilter(new ColumnCountGetFilter(100));           
> get.setFilter(filterList);
> Result r = table.get(get);
> System.out.println(r.size()); // prints zero
> {code}

--
This message is automatically generated by JIRA.
If you think it was sent incorrectly, please contact your JIRA administrators: https://issues.apache.org/jira/secure/ContactAdministrators!default.jspa
For more information on JIRA, see: http://www.atlassian.com/software/jira

        

[jira] [Commented] (HBASE-6132) ColumnCountGetFilter & PageFilter not working with FilterList

Posted by "nneverwei (JIRA)" <ji...@apache.org>.
    [ https://issues.apache.org/jira/browse/HBASE-6132?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=13289053#comment-13289053 ] 

nneverwei commented on HBASE-6132:
----------------------------------

Can someone explain why 'filter.filterAllRemaining()' goes there? I also cannot find any possible usecase that need filter.filterAllRemaining() there...
                
> ColumnCountGetFilter & PageFilter not working with FilterList
> -------------------------------------------------------------
>
>                 Key: HBASE-6132
>                 URL: https://issues.apache.org/jira/browse/HBASE-6132
>             Project: HBase
>          Issue Type: Bug
>          Components: filters
>    Affects Versions: 0.92.0, 0.92.1, 0.94.0
>         Environment: Cent OS 5.5 distributed hbase cluster. Hadoop 1.0.0, zookeeper 3.4.3
>            Reporter: Benjamin Kim
>            Assignee: Anoop Sam John
>
> Thanks to Anoop and Ramkrishna, here's what we found with FilterList
> If I use FilterList to include ColumnCountGetFilter among other filters, the returning Result has no keyvalues.
> This problem seems to occur when specified column count is less then actual number of existing columns.
> Also same problem arises with PageFilter
> Following is the code of the problem:
> {code}
> Configuration conf = HBaseConfiguration.create();
> HTable table = new HTable(conf, "test");
> Get get = new Get(Bytes.toBytes("test00001"));
> FilterList filterList = new FilterList();
> filterList.addFilter(new ColumnCountGetFilter(100));           
> get.setFilter(filterList);
> Result r = table.get(get);
> System.out.println(r.size()); // prints zero
> {code}

--
This message is automatically generated by JIRA.
If you think it was sent incorrectly, please contact your JIRA administrators: https://issues.apache.org/jira/secure/ContactAdministrators!default.jspa
For more information on JIRA, see: http://www.atlassian.com/software/jira

        

[jira] [Assigned] (HBASE-6132) ColumnCountGetFilter & PageFilter not working with FilterList

Posted by "Anoop Sam John (JIRA)" <ji...@apache.org>.
     [ https://issues.apache.org/jira/browse/HBASE-6132?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel ]

Anoop Sam John reassigned HBASE-6132:
-------------------------------------

    Assignee: Anoop Sam John
    
> ColumnCountGetFilter & PageFilter not working with FilterList
> -------------------------------------------------------------
>
>                 Key: HBASE-6132
>                 URL: https://issues.apache.org/jira/browse/HBASE-6132
>             Project: HBase
>          Issue Type: Bug
>          Components: filters
>    Affects Versions: 0.92.0, 0.92.1, 0.94.0
>         Environment: Cent OS 5.5 distributed hbase cluster. Hadoop 1.0.0, zookeeper 3.4.3
>            Reporter: Benjamin Kim
>            Assignee: Anoop Sam John
>
> Thanks to Anoop and Ramkrishna, here's what we found with FilterList
> If I use FilterList to include ColumnCountGetFilter among other filters, the returning Result has no keyvalues.
> This problem seems to occur when specified column count is less then actual number of existing columns.
> Also same problem arises with PageFilter
> Following is the code of the problem:
> {code}
> Configuration conf = HBaseConfiguration.create();
> HTable table = new HTable(conf, "test");
> Get get = new Get(Bytes.toBytes("test00001"));
> FilterList filterList = new FilterList();
> filterList.addFilter(new ColumnCountGetFilter(100));           
> get.setFilter(filterList);
> Result r = table.get(get);
> System.out.println(r.size()); // prints zero
> {code}

--
This message is automatically generated by JIRA.
If you think it was sent incorrectly, please contact your JIRA administrators: https://issues.apache.org/jira/secure/ContactAdministrators!default.jspa
For more information on JIRA, see: http://www.atlassian.com/software/jira

        

[jira] [Commented] (HBASE-6132) ColumnCountGetFilter & PageFilter not working with FilterList

Posted by "ramkrishna.s.vasudevan (JIRA)" <ji...@apache.org>.
    [ https://issues.apache.org/jira/browse/HBASE-6132?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=13465648#comment-13465648 ] 

ramkrishna.s.vasudevan commented on HBASE-6132:
-----------------------------------------------

bq.I think that filterRow() in FilterList should not check with the filterAllRemaining() on individual filters
Yes i too feel the same after going thro its usage.  The attached testcase reveals the problem very well.
                
> ColumnCountGetFilter & PageFilter not working with FilterList
> -------------------------------------------------------------
>
>                 Key: HBASE-6132
>                 URL: https://issues.apache.org/jira/browse/HBASE-6132
>             Project: HBase
>          Issue Type: Bug
>          Components: Filters
>    Affects Versions: 0.92.0, 0.92.1, 0.94.0
>         Environment: Cent OS 5.5 distributed hbase cluster. Hadoop 1.0.0, zookeeper 3.4.3
>            Reporter: Benjamin Kim
>            Assignee: Anoop Sam John
>         Attachments: TestColumnCountGetFilter.java
>
>
> Thanks to Anoop and Ramkrishna, here's what we found with FilterList
> If I use FilterList to include ColumnCountGetFilter among other filters, the returning Result has no keyvalues.
> This problem seems to occur when specified column count is less then actual number of existing columns.
> Also same problem arises with PageFilter
> Following is the code of the problem:
> {code}
> Configuration conf = HBaseConfiguration.create();
> HTable table = new HTable(conf, "test");
> Get get = new Get(Bytes.toBytes("test00001"));
> FilterList filterList = new FilterList();
> filterList.addFilter(new ColumnCountGetFilter(100));           
> get.setFilter(filterList);
> Result r = table.get(get);
> System.out.println(r.size()); // prints zero
> {code}

--
This message is automatically generated by JIRA.
If you think it was sent incorrectly, please contact your JIRA administrators
For more information on JIRA, see: http://www.atlassian.com/software/jira

[jira] [Updated] (HBASE-6132) ColumnCountGetFilter & PageFilter not working with FilterList

Posted by "Benjamin Kim (JIRA)" <ji...@apache.org>.
     [ https://issues.apache.org/jira/browse/HBASE-6132?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel ]

Benjamin Kim updated HBASE-6132:
--------------------------------

    Description: 
Thanks to Anoop and Ramkrishna, here's what we found with FilterList

If I use FilterList to include ColumnCountGetFilter among other filters, the returning Result has no keyvalues.

This problem seems to occur when specified column count is less then actual number of existing columns.

Also same problem arises with PageFilter

Following is the code of the problem:

{code}
Configuration conf = HBaseConfiguration.create();
HTable table = new HTable(conf, "test");
Get get = new Get(Bytes.toBytes("test00001"));
FilterList filterList = new FilterList();
filterList.addFilter(new ColumnCountGetFilter(100));           
get.setFilter(filterList);
Result r = table.get(get);
System.out.println(r.size()); // prints zero
{code}

  was:
Thanks to Anoop and Ramkrishna, here's what we found with FilterList

If I use FilterList to include ColumnCountGetFilter among other filters, the returning Result has no keyvalues.

This problem seems to occur when specified column count is less then actual number of existing columns.

Also same problem arises with PageFilter

Following is the code of the problem:

Configuration conf = HBaseConfiguration.create();
HTable table = new HTable(conf, "test");
Get get = new Get(Bytes.toBytes("test00001"));
FilterList filterList = new FilterList();
filterList.addFilter(new ColumnCountGetFilter(100));           
get.setFilter(filterList);
Result r = table.get(get);
System.out.println(r.size()); // prints zero

    
> ColumnCountGetFilter & PageFilter not working with FilterList
> -------------------------------------------------------------
>
>                 Key: HBASE-6132
>                 URL: https://issues.apache.org/jira/browse/HBASE-6132
>             Project: HBase
>          Issue Type: Bug
>          Components: filters
>    Affects Versions: 0.92.0, 0.92.1, 0.94.0
>         Environment: Cent OS 5.5 distributed hbase cluster. Hadoop 1.0.0, zookeeper 3.4.3
>            Reporter: Benjamin Kim
>
> Thanks to Anoop and Ramkrishna, here's what we found with FilterList
> If I use FilterList to include ColumnCountGetFilter among other filters, the returning Result has no keyvalues.
> This problem seems to occur when specified column count is less then actual number of existing columns.
> Also same problem arises with PageFilter
> Following is the code of the problem:
> {code}
> Configuration conf = HBaseConfiguration.create();
> HTable table = new HTable(conf, "test");
> Get get = new Get(Bytes.toBytes("test00001"));
> FilterList filterList = new FilterList();
> filterList.addFilter(new ColumnCountGetFilter(100));           
> get.setFilter(filterList);
> Result r = table.get(get);
> System.out.println(r.size()); // prints zero
> {code}

--
This message is automatically generated by JIRA.
If you think it was sent incorrectly, please contact your JIRA administrators: https://issues.apache.org/jira/secure/ContactAdministrators!default.jspa
For more information on JIRA, see: http://www.atlassian.com/software/jira

        

[jira] [Commented] (HBASE-6132) ColumnCountGetFilter & PageFilter not working with FilterList

Posted by "Anoop Sam John (JIRA)" <ji...@apache.org>.
    [ https://issues.apache.org/jira/browse/HBASE-6132?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=13285363#comment-13285363 ] 

Anoop Sam John commented on HBASE-6132:
---------------------------------------

The problem is with the below code
In FilterList.java
{code}
public boolean filterRow() {
    for (Filter filter : filters) {
      if (operator == Operator.MUST_PASS_ALL) {
        if (filter.filterAllRemaining() || filter.filterRow()) {
          return true;
        }
      } else if (operator == Operator.MUST_PASS_ONE) {
        if (!filter.filterAllRemaining()
            && !filter.filterRow()) {
          return false;
        }
      }
    }
    return  operator == Operator.MUST_PASS_ONE;
  }
{code}
Here filterRow checks for filterAllRemaining() also.
In the issue case, the desirable number of KVs are included and remaining filtered out using the ColumnCountGetFilter. This filter implements the filterAllRemaining() also which returns true once the desirable number of KVs are included.  Later when the filterRow is called, as per the above code it will call the filterAllRemaining() on ColumnCountGetFilter and this row also will get avoided.

Same issue with PaginationFilter where the desired number of rows suppose 10.  9 rows will get included and once 10th row is reached because of the  filterAllRemaining() returns true from then onwards, the 10th row wont get included.

Why filterRow() impl of FilterList check with the filterAllRemaining() on the contained filters?
                
> ColumnCountGetFilter & PageFilter not working with FilterList
> -------------------------------------------------------------
>
>                 Key: HBASE-6132
>                 URL: https://issues.apache.org/jira/browse/HBASE-6132
>             Project: HBase
>          Issue Type: Bug
>          Components: filters
>    Affects Versions: 0.92.0, 0.92.1, 0.94.0
>         Environment: Cent OS 5.5 distributed hbase cluster. Hadoop 1.0.0, zookeeper 3.4.3
>            Reporter: Benjamin Kim
>
> Thanks to Anoop and Ramkrishna, here's what we found with FilterList
> If I use FilterList to include ColumnCountGetFilter among other filters, the returning Result has no keyvalues.
> This problem seems to occur when specified column count is less then actual number of existing columns.
> Also same problem arises with PageFilter
> Following is the code of the problem:
> Configuration conf = HBaseConfiguration.create();
> HTable table = new HTable(conf, "test");
> Get get = new Get(Bytes.toBytes("test00001"));
> FilterList filterList = new FilterList();
> filterList.addFilter(new ColumnCountGetFilter(100));           
> get.setFilter(filterList);
> Result r = table.get(get);
> System.out.println(r.size()); // prints zero

--
This message is automatically generated by JIRA.
If you think it was sent incorrectly, please contact your JIRA administrators: https://issues.apache.org/jira/secure/ContactAdministrators!default.jspa
For more information on JIRA, see: http://www.atlassian.com/software/jira